<?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</title>
  <updated>2026-03-05T15:05:17.532963+00:00</updated>
  <link href="https://blog.dask.org"/>
  <link href="https://blog.dask.org/blog/atom.xml" rel="self"/>
  <generator uri="https://ablog.readthedocs.io/" version="0.11.12">ABlog</generator>
  <entry>
    <id>https://blog.dask.org/2024/11/21/dask-detrending/</id>
    <title>Improving GroupBy.map with Dask and Xarray</title>
    <updated>2024-11-21T00:00:00+00:00</updated>
    <author>
      <name>Patrick Hoefler</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This post was originally published on the &lt;a class="reference external" href="https://xarray.dev/blog/dask-detrending"&gt;Xarray blog&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Running large-scale GroupBy-Map patterns with Xarray that are backed by &lt;a class="reference external" href="https://docs.dask.org/en/stable/array.html"&gt;Dask arrays&lt;/a&gt; is
an essential part of a lot of typical geospatial workloads. Detrending is a very common
operation where this pattern is needed.&lt;/p&gt;
&lt;p&gt;In this post, we will explore how and why this caused so many pitfalls for Xarray users in
the past and how we improved performance and scalability with a few changes to how Dask
subselects 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/2024/11/21/dask-detrending.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="what-is-groupby-map"&gt;

&lt;p&gt;&lt;a class="reference external" href="https://docs.xarray.dev/en/stable/generated/xarray.core.groupby.DatasetGroupBy.map.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GroupBy.map&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; lets you apply a User Defined Function (UDF)
that accepts and returns Xarray objects. The UDF will receive an Xarray object (either a Dataset or a DataArray) containing Dask arrays corresponding to one single group.
&lt;a class="reference external" href="https://docs.xarray.dev/en/stable/generated/xarray.core.groupby.DatasetGroupBy.reduce.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Groupby.reduce&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; is quite similar
in that it applies a UDF, but in this case the UDF will receive the underlying Dask arrays, &lt;em&gt;not&lt;/em&gt; Xarray 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/2024/11/21/dask-detrending.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="the-application"&gt;
&lt;h1&gt;The Application&lt;/h1&gt;
&lt;p&gt;Consider a typical workflow where you want to apply a detrending step. You may want to smooth out
the data by removing the trends over time. This is a common operation in climate science
and normally looks roughly like this:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;detrending_step&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;arr&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;DataArray&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;DataArray&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="c1"&gt;# important: the rolling operation is applied within a group&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;arr&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;arr&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rolling&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;30&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;min_periods&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;mean&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;time.dayofyear&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;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;detrending_step&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We are grouping by the day of the year and then are calculating the rolling average over
30-year windows for a particular day.&lt;/p&gt;
&lt;p&gt;Our example will run on a 1 TiB array, 64 years worth of data and the following structure:&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Python repr output of 1 TiB Dask array with shape (1801, 3600, 233376) split into 5460, 250 MiB chunks of (300, 300, 365)" src="/images/dask-detrending/input-array.png" style="width: 800px;"/&gt;
&lt;/figure&gt;
&lt;p&gt;The array isn’t overly huge and the chunks are reasonably sized.&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/2024/11/21/dask-detrending.md&lt;/span&gt;, line 52)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="the-problem"&gt;
&lt;h1&gt;The Problem&lt;/h1&gt;
&lt;p&gt;The general application seems straightforward. Group by the day of the year and apply a UDF
to every group. There are a few pitfalls in this application that can make the result of
this operation unusable. Our array is sorted by time, which means that we have to pick
entries from many different areas in the array to create a single group (corresponding to a single day of the year).
Picking the same day of every year is basically a slicing operation with a step size of 365.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Schematic showing an array sorted by time, where data is selected from many different areas in the array to create a single group (corresponding to a specific day of the year)." src="/images/dask-detrending/indexing-data-selection.png" title="Data Selection Pattern" style="width: 800px;"/&gt;
&lt;/figure&gt;
&lt;p&gt;Our example has a year worth of data in a single chunk along the time axis. The general problem
exists for any workload where you have to access random entries of data. This
particular access pattern means that we have to pick one value per chunk, which is pretty
inefficient. The right side shows the individual groups that we are operating on.&lt;/p&gt;
&lt;p&gt;One of the main issues with this pattern is that Dask will create a single output chunk per time
entry, e.g. each group will consist of as many chunks as we have year.&lt;/p&gt;
&lt;p&gt;This results in a huge increase in the number of chunks:&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Python repr output of a 1 TiB Dask array with nearly 2 million, 700 kiB chunks." src="/images/dask-detrending/output-array-old.png" style="width: 800px;"/&gt;
&lt;/figure&gt;
&lt;p&gt;This simple operation increases the number of chunks from 5000 to close to 2 million. Each
chunk only has a few hundred kilobytes of data. &lt;strong&gt;This is pretty bad!&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;Dask computations generally scale along the number of chunks you have. Increasing the chunks by such
a large factor is catastrophic. Each follow-up operation, as simple as &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;a-b&lt;/span&gt;&lt;/code&gt; will create 2 million
additional tasks.&lt;/p&gt;
&lt;p&gt;The only workaround for users was to rechunk to something more sensible afterward, but it
still keeps the incredibly expensive indexing operation in the graph.&lt;/p&gt;
&lt;p&gt;Note this is the underlying problem that is &lt;a class="reference external" href="https://xarray.dev/blog/flox"&gt;solved by flox&lt;/a&gt; for aggregations like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.mean()&lt;/span&gt;&lt;/code&gt;
using parallel-native algorithms to avoid the expense of indexing out each group.&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/2024/11/21/dask-detrending.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="improvements-to-the-data-selection-algorithm"&gt;
&lt;h1&gt;Improvements to the Data Selection algorithm&lt;/h1&gt;
&lt;p&gt;The method of how Dask selected the data was objectively pretty bad.
A rewrite of the underlying algorithm enabled us to achieve a much more robust result. The new
algorithm is a lot smarter about how to pick values from each individual chunk, but most importantly,
it will try to preserve the input chunksize as closely as possible.&lt;/p&gt;
&lt;p&gt;For our initial example, it will put every group into a single chunk. This means that we will
end up with the number of chunks along the time axis being equal to the number of groups, i.e. 365.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Python repr output of a 1 TiB Dask array with 31164, 43 MiB chunks" src="/images/dask-detrending/output-array-new.png" style="width: 800px;"/&gt;
&lt;/figure&gt;
&lt;p&gt;The algorithm reduces the number of chunks from 2 million to roughly 30 thousand, which is a huge improvement
and a scale that Dask can easily handle. The graph is now much smaller, and the follow-up operations
will run a lot faster as well.&lt;/p&gt;
&lt;p&gt;This improvement will help every operation that we listed above and make the scale a lot more
reliably than before. The algorithm is used very widely across Dask and Xarray and thus, influences
many methods.&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/2024/11/21/dask-detrending.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="what-s-next"&gt;
&lt;h1&gt;What’s next?&lt;/h1&gt;
&lt;p&gt;Xarray selects one group at a time for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;groupby(...).map(...)&lt;/span&gt;&lt;/code&gt;, i.e. this requires one operation
per group. This will hurt scalability if the dataset has a very large number of groups, because
the computation will create a very expensive graph. There is currently an effort to implement alternative
APIs that are shuffle-based to circumvent that problem. A current PR is available &lt;a class="reference external" href="https://github.com/pydata/xarray/pull/9320"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The fragmentation of the output chunks by indexing is something that will hurt every workflow that is selecting data in a random
pattern. This also includes:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.sel&lt;/span&gt;&lt;/code&gt; if you aren’t using slices explicitly&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.isel&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;.sortby&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;groupby(...).quantile()&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;and many more.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We expect all of these workloads to be substantially improved now.&lt;/p&gt;
&lt;p&gt;Additionally, &lt;a class="reference external" href="https://docs.dask.org/en/stable/changelog.html#v2024-11-1"&gt;Dask improved a lot of things&lt;/a&gt; related to either increasing chunksizes or fragmentation
of chunks over the cycle of a workload with more improvements to come. This will help a lot of
users to get better and more reliable performance.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2024/11/21/dask-detrending/"/>
    <summary>This post was originally published on the Xarray blog.</summary>
    <category term="daskarray" label="dask array"/>
    <category term="xarray" label="xarray"/>
    <published>2024-11-21T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2024/05/30/dask-is-fast/</id>
    <title>Dask DataFrame is Fast Now</title>
    <updated>2024-05-30T00:00:00+00:00</updated>
    <author>
      <name>Patrick Hoefler</name>
    </author>
    <content type="html">&lt;meta content="Dask DataFrame is faster and more reliable, especially for TBs of data. This is due to engineering improvements like adding a query optimizer, integrating with Apache Arrow, and more efficient data shuffling." name="description" /&gt;
&lt;p&gt;&lt;em&gt;This work was engineered and supported by &lt;a class="reference external" href="https://coiled.io/?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-expr"&gt;Coiled&lt;/a&gt; and &lt;a class="reference external" href="https://www.nvidia.com/"&gt;NVIDIA&lt;/a&gt;. Thanks to &lt;a class="reference external" href="https://github.com/phofl"&gt;Patrick Hoefler&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/rjzamora"&gt;Rick Zamora&lt;/a&gt;, in particular. Original version of this post appears on &lt;a class="reference external" href="https://docs.coiled.io/blog/dask-dataframe-is-fast.html?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-is-fast"&gt;docs.coiled.io&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Bar chart showing a nearly 20x improvement in Dask DataFrame performance with the addition of Arrow stings, more efficient shufffling, and a query optimizer." src="/images/dask-improvement.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Performance Improvements for Dask DataFrames&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&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/2024/05/30/dask-is-fast.md&lt;/span&gt;, line 21)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="intro"&gt;

&lt;p&gt;Dask DataFrame scales out pandas DataFrames to operate at the 100GB-100TB scale.&lt;/p&gt;
&lt;p&gt;Historically, &lt;a class="reference external" href="https://www.dask.org/"&gt;Dask&lt;/a&gt; was pretty slow compared to other tools in this space (like Spark). Due to a number of improvements focused on performance,
it’s now pretty fast (about 20x faster than before). The new implementation moved Dask from getting destroyed by
Spark on every benchmark to regularly outperforming Spark on TPC-H queries by a significant margin.&lt;/p&gt;
&lt;p&gt;Dask DataFrame workloads struggled with many things. Performance and memory usage were
commonly seen pain points, shuffling was unstable for bigger datasets, making scaling out
hard. Writing efficient code required understanding too much of the internals of Dask.&lt;/p&gt;
&lt;p&gt;The new implementation changed all of this. Things that didn’t work were completely rewritten from scratch and existing
implementations were improved upon. This puts Dask DataFrames on a solid foundation that
allows faster iteration cycles in the future.&lt;/p&gt;
&lt;p&gt;We’ll go through the three most prominent changes:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#arrow"&gt;&lt;span class="xref myst"&gt;Apache Arrow support&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#p2p"&gt;&lt;span class="xref myst"&gt;Faster joins&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#optimizer"&gt;&lt;span class="xref myst"&gt;Query optimization&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We’ll cover how these changes impact performance and make it easier to use Dask efficiently, even for users that are new to distributed computing. We’ll also discuss plans for future improvements.&lt;/p&gt;
&lt;p&gt;&lt;a name="arrow"&gt;&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/2024/05/30/dask-is-fast.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="apache-arrow-support-efficient-string-datatype"&gt;
&lt;h1&gt;1. Apache Arrow Support: Efficient String Datatype&lt;/h1&gt;
&lt;p&gt;A Dask DataFrame consists of many pandas DataFrames. Historically, pandas used NumPy for numeric data,
but Python objects for text data, which are inefficient and blow up memory usage. Operations on
object data also hold the GIL, which doesn’t matter much for pandas, but is a catastrophy for
performance with a parallel system like Dask.&lt;/p&gt;
&lt;p&gt;The pandas 2.0 release introduced support for general-purpose Arrow datatypes, so &lt;a class="reference external" href="https://docs.coiled.io/blog/pyarrow-in-pandas-and-dask.html?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-is-fast"&gt;Dask now uses PyArrow-backed
strings by default&lt;/a&gt;. These are &lt;em&gt;much&lt;/em&gt; better. PyArrow strings reduce memory usage by up to 80% and
unlock multi-threading for string operations. Workloads that previously
struggled with available memory now fit comfortably in much less space, and are
a lot faster because they no longer constantly spill excess data to disk.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Bar chart comparing memory usage (GB) for Dask DataFrame with and without PyArrow strings. Dask DataFrame uses up to 80% less memory with PyArrow strings." src="/images/arrow-strings-memory-usage.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Memory Usage of the Legacy DataFrames Compared with Arrow Strings&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;&lt;a name="p2p"&gt;&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/2024/05/30/dask-is-fast.md&lt;/span&gt;, line 69)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="faster-joins-with-a-new-shuffle-algorithm"&gt;
&lt;h1&gt;2. Faster Joins with a New Shuffle Algorithm&lt;/h1&gt;
&lt;p&gt;Shuffling is an essential component of distributed systems to enable sorting, joins, and complex group by operations. It is an all-to-all, network-intensive operation that’s often the most expensive component in a workflow. Dask has a new shuffling system, which greatly impacts overall performance, especially on complex, data-intensive workloads.&lt;/p&gt;
&lt;p&gt;A shuffle operation is intrinsically an all-to-all communication operation where every
input partition has to provide a tiny slice of data to every output partition. Dask was already
using it’s own task-based algorithm that managed to reduce the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;O(n&lt;/span&gt; &lt;span class="pre"&gt;*&lt;/span&gt; &lt;span class="pre"&gt;n)&lt;/span&gt;&lt;/code&gt; task
complexity to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;O(log(n)&lt;/span&gt; &lt;span class="pre"&gt;*&lt;/span&gt; &lt;span class="pre"&gt;n)&lt;/span&gt;&lt;/code&gt; where &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n&lt;/span&gt;&lt;/code&gt; is the number of partitions. This was a drastic
reduction in the number of tasks, but the non-linear scaling ultimately did not allow Dask to process
arbitrarily large datasets.&lt;/p&gt;
&lt;p&gt;Dask introduced a new &lt;a class="reference external" href="https://docs.coiled.io/blog/shuffling-large-data-at-constant-memory.html?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-is-fast"&gt;P2P (peer-to-peer) shuffle method&lt;/a&gt; that reduced the task complexity to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;O(n)&lt;/span&gt;&lt;/code&gt;
which scales linearly with the size of the dataset and the size of the cluster. It also
incorporates an efficient disk integration which allows easily shuffling datasets which are much
larger than memory. The new system is extremely stable and “just works” across any scale of data.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Line plot of memory usage (GB) over time (seconds) comparing Dask DataFrame with the peer-to-peer shuffling to task-based shuffling. With P2P shuffling, Dask DataFrame memory usage remains consistently low throughout the computation." src="/images/shuffle-memory-comparison.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Memory Usage of the Legacy Shuffle Compared with P2P&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;&lt;a name="optimizer"&gt;&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/2024/05/30/dask-is-fast.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="optimizer"&gt;
&lt;h1&gt;3. Optimizer&lt;/h1&gt;
&lt;p&gt;Dask itself is lazy, which means that it registers your whole query before doing any actual work.
This is a powerful concept that enables a lot of optimizations, but historically Dask wasn’t taking advantage of this
knowledge in the past. Dask also did a bad job of hiding internal complexities and
left users on their own while navigating the difficulties of distributed computing and running
large scale queries. It made writing efficient code painful for non-experts.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://docs.dask.org/en/stable/changelog.html#query-planning"&gt;The Dask release in March&lt;/a&gt; includes a complete re-implementation of the DataFrame API to support query optimization. This is a big deal.
The new engine centers around a query optimizer that rewrites your code to make it more efficient and
better tailored to Dask’s strengths. Let’s dive into some optimization strategies, how they make
Dask run faster and scale better.&lt;/p&gt;
&lt;p&gt;We will start with a couple general-purpose optimizations:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#column-projection"&gt;&lt;span class="xref myst"&gt;Column projection&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#filter-pushdown"&gt;&lt;span class="xref myst"&gt;Filter pushdown&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;And then dive into more specific techniques that are tailored to distributed systems generally
and Dask more specifically:&lt;/p&gt;
&lt;ol class="arabic simple" start="3"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#partition-resizing"&gt;&lt;span class="xref myst"&gt;Automatic partition resizing&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#trivial-merge"&gt;&lt;span class="xref myst"&gt;Trivial merge and join operations&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;a name="column-projection"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;section id="column-projection"&gt;
&lt;h2&gt;3.1 Column Projection&lt;/h2&gt;
&lt;p&gt;Most datasets have more columns than needed. Dropping them requires foresight (“What columns will I need for this query? 🤔”) so most people don’t think about this when loading data. This is bad for performance because carrying around lots of excess data slows everything down.
Column Projection drops columns as soon as they aren’t needed anymore. It’s a straightforward optimization, but highly beneficial.&lt;/p&gt;
&lt;p&gt;The legacy implementation always reads all columns from storage and only drops columns when explicitly specified by the user.
Simply operating on less data is a big win for performance and memory usage.&lt;/p&gt;
&lt;p&gt;The optimizer looks at the query and figures out which columns are needed for each operation.
It looks at the final step of the query and then works backwards step by
step to the data source, injecting drop operations to get rid of unnecessary columns.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Diagram explaining how column projection works for the Dask DataFrame optimizer." src="/images/projection.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Only require a subset of columns are needed. Replace doesn't need access to all columns, so Dask drops unnecessary columns directly in the IO step.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;&lt;a name="filter-pushdown"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="filter-pushdown"&gt;
&lt;h2&gt;3.2 Filter Pushdown&lt;/h2&gt;
&lt;p&gt;Filter pushdown is another general-purpose optimization with
the same goal as column projection: operate on less data. The legacy implementation did not reorder filter operations. The new implementation executes filter operations as early as
possible while maintaining the same results.&lt;/p&gt;
&lt;p&gt;The optimizer identifies every filter in the query and looks at the previous operation to see if we
can move the filter closer to the data source. It will repeat this until it finds an operation that
can’t be switched with a filter. This is a bit harder than
column projections, because Dask has to make sure that the operations don’t change the values of the
DataFrame. For example, switching a filter and a merge operation is fine (values don’t change), but switching a filter
and a replace operation is invalid, because the values might change and rows that would previously have been filtered out now won’t be, or vice versa.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Diagram explaining how filter pushdown (or predicate pushdown) works to reduce the amount of data being processed in the Dask DataFrame query optimizer." src="/images/filter.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Initially, the filter happens after the Dropna, but Dask can execute the filter before Dropna without changing the result. This allows Dask to push the filter into the IO step.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;Additionally, if the filter is strong enough then Dask can potentially drop complete files in the IO step.
This is a best-case scenario, where an earlier filter brings a huge performance improvement and even
requires reading less data from remote storage.&lt;/p&gt;
&lt;p&gt;&lt;a name="partition-resizing"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="automatically-resizing-partitions"&gt;
&lt;h2&gt;3.3 Automatically Resizing Partitions&lt;/h2&gt;
&lt;p&gt;In addition to implementing the common optimization techniques described above, we’ve also improved a
common pain point specific to distributed systems generally and Dask users specifically: optimal partition sizes.&lt;/p&gt;
&lt;p&gt;Dask DataFrames consist of many small pandas DataFrames called &lt;em&gt;partitions&lt;/em&gt;. Often, the number of
partitions is decided for you and Dask users are advised to manually “repartition” after reducing
or expanding their data (for example by dropping columns, filtering data, or expanding with joins) (see the &lt;a class="reference external" href="https://docs.dask.org/en/stable/dataframe-best-practices.html#repartition-to-reduce-overhead"&gt;Dask docs&lt;/a&gt;).
Without this extra step,
the (usually small) overhead from Dask can become a bottleneck if the pandas DataFrames
become too small, making Dask workflows painfully slow.&lt;/p&gt;
&lt;p&gt;Manually controlling the partition size is a difficult task that we, as Dask users, shouldn’t have
to worry about. It is also slow because it requires network transfer of some partitions.
Dask DataFrame now automatically does two things to help when the partitions get
too small:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Keeps the size of each partition constant, based on the ratio of data you want to compute vs.
the original file size. If, for example, you filter out 80% of the original dataset, Dask will
automatically combine the resulting smaller partitions into fewer, larger partitions.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Combines too-small partitions into larger partitions, based on an absolute minimum
(default is 75 MB). If, for example, your original dataset is split into many tiny files,
Dask will automatically combine them.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Diagram representing how column selection can reduce the size of a partition when reading parquet data." src="/images/automatic_repartitioning_1.png" style="width: 800px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Select two columns that take up 40 MB of memory out of the 200 MB from the whole parquet file.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;The optimizer will look at the number of columns and the size of the data within those. It
calculates a ratio that is used to combine multiple files into one partition.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Diagram showing how automatic repartitioning works for Dask DataFrame." src="/images/automatic_repartitioning_2.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;The ratio of 40/200 results in combining five files into a single partition.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;This step is currently limited to IO operations (like reading in a parquet dataset), but we plan
to extend it to other operations that allow cheaply combining partitions.&lt;/p&gt;
&lt;p&gt;&lt;a name="trivial-merge"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="trivial-merge-and-join-operations"&gt;
&lt;h2&gt;3.4 Trivial Merge and Join Operations&lt;/h2&gt;
&lt;p&gt;Merge and join operations are typically cheap on a single machine with pandas but expensive in a
distributed setting. Merging data in shared memory is cheap, while merging data across a network is quite slow,
due to the shuffle operations explained earlier.&lt;/p&gt;
&lt;p&gt;This is one of the most expensive operations in a distributed system. The legacy implementation triggered
a network transfer of both input DataFrames for every merge operation. This is sometimes necessary, but very
expensive.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Diagram representing how an expensive shuffle operation is avoided automatically if two DataFrames are already aligned before joining." src="/images/avoiding-shuffles.png" style="width: 800px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Both joins are performed on the same column. The left DataFrame is already properly partitioned after the first join, so Dask can avoid shuffling again with the new implementation.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;The optimizer will determine when shuffling is necessary versus when
a trivial join is sufficient because the data is already aligned properly. This can make individual merges
an order of magnitude faster. This also applies to other operations that normally require a shuffle
like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;groupby().apply()&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Dask merges used to be inefficient, which caused long runtimes. The optimizer fixes this for
the trivial case where these operations happen after each other, but the technique isn’t very
advanced yet. There is still a lot of potential for improvement.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Diagram showing how the query optimizer for Dask DataFrame automatically shuffles data earlier to make a groupby aggregation more efficient." src="/images/avoiding-shuffles-advanced.png" style="width: 800px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;The current implementation shuffles both branches that originate from the same table. Injecting a shuffle node further up avoids one of the expensive operations.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;The optimizer will look at the expression and inject shuffle nodes where necessary to avoid
unnecessary shuffles.&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/2024/05/30/dask-is-fast.md&lt;/span&gt;, line 250)&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="how-do-the-improvements-stack-up-compared-to-the-legacy-implementation"&gt;
&lt;h1&gt;How do the improvements stack up compared to the legacy implementation?&lt;/h1&gt;
&lt;p&gt;Dask is now 20x faster than before. This improvement applies to the entire
DataFrame API (not just isolated components), with no known
performance regressions. Dask now runs workloads that were impossible to
complete in an acceptable timeframe before. This performance boost is due to many
improvements all layered on top of each other. It’s not about doing one thing
especially well, but about doing nothing especially poorly.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Bar chart showing a nearly 20x improvement in Dask DataFrame performance with the addition of Arrow stings, more efficient shufffling, and a query optimizer." src="/images/dask-improvement.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Performance Improvements on Query 3 of the TPC-H Benchmarks from &lt;a class="reference external" href="https://github.com/coiled/benchmarks/tree/main/tests/tpch"&gt;https://github.com/coiled/benchmarks/tree/main/tests/tpch&lt;/a&gt; &lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;Performance, while the most enticing improvement, is not the only thing that got better. The
optimizer hides a lot of complexity from the user and makes the transition from pandas to Dask a
lot easier because it’s now much more difficult to write poorly performing code.
The whole system is more robust.&lt;/p&gt;
&lt;p&gt;The new architecture of the API is a lot easier to work with as well. The legacy implementation leaked
a lot of internal complexities into high-level API implementations, making changes cumbersome. Improvements
are almost trivial to add now.&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/2024/05/30/dask-is-fast.md&lt;/span&gt;, line 275)&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-s-to-come"&gt;
&lt;h1&gt;What’s to come?&lt;/h1&gt;
&lt;p&gt;Dask DataFrame changed a lot over the last 18 months. The legacy API was often difficult to work with and
struggled with scaling out. The new implementation dropped things that didn’t work and
improved existing implementations. The heavy lifting is finished now, which allows for
faster iteration cycles to improve upon the status quo. Incremental improvements are now
trivial to add.&lt;/p&gt;
&lt;p&gt;A few things that are on the immediate roadmap:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Auto repartitioning:&lt;/strong&gt; this is partially implemented, but there is more potential to choose a more
efficient partition size during optimization.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Faster Joins:&lt;/strong&gt; there’s still lots of fine-tuning to be done here.
For example, there is a PR in flight with a 30-40% improvement.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Join Reordering:&lt;/strong&gt; Dask doesn’t do this yet, but it’s on the immediate roadmap&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/2024/05/30/dask-is-fast.md&lt;/span&gt;, line 291)&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="learn-more"&gt;
&lt;h1&gt;Learn more&lt;/h1&gt;
&lt;p&gt;This article focuses on a number of improvements to Dask DataFrame and how much faster and more reliable it is as a result. If you’re choosing between Dask and other popular DataFrame tools, you might also consider:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://docs.coiled.io/blog/tpch.html?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-is-fast"&gt;DataFrames at Scale Comparison: TPC-H&lt;/a&gt; which compares Dask, Spark, Polars, and DuckDB performance on datasets ranging from 10 GB to 10 TB both locally and on the cloud&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2024/05/30/dask-is-fast/"/>
    <summary>This work was engineered and supported by Coiled and NVIDIA. Thanks to Patrick Hoefler and Rick Zamora, in particular. Original version of this post appears on docs.coiled.io</summary>
    <category term="dask" label="dask"/>
    <category term="dataframe" label="dataframe"/>
    <category term="queryoptimizer" label="query optimizer"/>
    <published>2024-05-30T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2023/08/25/dask-expr-introduction/</id>
    <title>High Level Query Optimization in Dask</title>
    <updated>2023-08-25T00:00:00+00:00</updated>
    <author>
      <name>Patrick Hoefler</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This work was engineered and supported by &lt;a class="reference external" href="https://coiled.io/?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-expr"&gt;Coiled&lt;/a&gt; and &lt;a class="reference external" href="https://www.nvidia.com/"&gt;NVIDIA&lt;/a&gt;. Thanks to &lt;a class="reference external" href="https://github.com/phofl"&gt;Patrick Hoefler&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/rjzamora"&gt;Rick Zamora&lt;/a&gt;, in particular. Original version of this post appears on &lt;a class="reference external" href="https://blog.coiled.io/blog/dask-expr-introduction.html?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-expr"&gt;blog.coiled.io&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Expression tree encoded by dask-expr" src="/images/dask_expr/dask-expr-introduction-title.png" style="width: 70%;"/&gt;
&lt;/figure&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/08/25/dask-expr-introduction.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="introduction"&gt;

&lt;p&gt;Dask DataFrame doesn’t currently optimize your code for you (like Spark or a SQL database would).
This means that users waste a lot of computation. Let’s look at a common example
which looks ok at first glance, but is actually pretty inefficient.&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;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;s3://coiled-datasets/uber-lyft-tlc/&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="c1"&gt;# unnecessarily reads all rows and columns&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="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;hvfhs_license_num&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;HV0003&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;    &lt;span class="c1"&gt;# could push the filter into the read parquet call&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;numeric_only&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;tips&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;                                &lt;span class="c1"&gt;# should read only necessary columns&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We can make this run much faster with a few simple steps:&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;s3://coiled-datasets/uber-lyft-tlc/&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;filters&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;hvfhs_license_num&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;==&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;HV0003&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)],&lt;/span&gt;
    &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;tips&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tips&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Currently, Dask DataFrame wouldn’t optimize this for you, but a new effort that is built around
logical query planning in Dask DataFrame will do this for you. This article introduces some of
those changes that are developed in &lt;a class="reference external" href="https://github.com/dask-contrib/dask-expr"&gt;dask-expr&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;You can install and try &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt; with:&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;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;expr&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We are using the &lt;a class="reference external" href="https://www.nyc.gov/site/tlc/about/tlc-trip-record-data.page"&gt;NYC Taxi&lt;/a&gt;
dataset in this post.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/08/25/dask-expr-introduction.md&lt;/span&gt;, line 60)&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="dask-expressions"&gt;
&lt;h1&gt;Dask Expressions&lt;/h1&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask-contrib/dask-expr"&gt;Dask expressions&lt;/a&gt; provides a logical query planning layer on
top of Dask DataFrames. Let’s look at our initial example and investigate how we can improve the efficiency
through a query optimization layer. As noted initially, there are a couple of things that aren’t ideal:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;We are reading all rows into memory instead of filtering while reading the parquet files.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We are reading all columns into memory instead of only the columns that are necessary.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We are applying the filter and the aggregation onto all columns instead of only &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;&amp;quot;tips&amp;quot;&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The query optimization layer from &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt; can help us with that. It will look at this expression
and determine that not all rows are needed. An intermediate layer will transpile the filter into
a valid filter-expression for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_parquet&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;s3://coiled-datasets/uber-lyft-tlc/&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;filters&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;hvfhs_license_num&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;==&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;HV0003&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)],&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;numeric_only&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;tips&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This still reads every column into memory and will compute the sum of every numeric column. The
next optimization step is to push the column selection into the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_parquet&lt;/span&gt;&lt;/code&gt; call as well.&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;s3://coiled-datasets/uber-lyft-tlc/&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;tips&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
    &lt;span class="n"&gt;filters&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;hvfhs_license_num&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;==&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;HV0003&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)],&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;numeric_only&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is a basic example that you could rewrite by hand. Use cases that are closer to real
workflows might potentially have hundreds of columns, which makes rewriting them very strenuous
if you need a non-trivial subset of them.&lt;/p&gt;
&lt;p&gt;Let’s take a look at how we can achieve this. &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt; records the expression as given by the
user in an expression tree:&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;result&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;pprint&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;Projection&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;tips&amp;#39;&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;numeric_only&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&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;ReadParquet&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://coiled-datasets/uber-lyft-tlc/&amp;#39;&lt;/span&gt;
      &lt;span class="n"&gt;EQ&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;right&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;HV0003&amp;#39;&lt;/span&gt;
        &lt;span class="n"&gt;Projection&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;hvfhs_license_num&amp;#39;&lt;/span&gt;
          &lt;span class="n"&gt;ReadParquet&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://coiled-datasets/uber-lyft-tlc/&amp;#39;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This tree represents the expression as is. We can observe that we would read the whole dataset into
memory before we apply the projections and filters. One observation of note: It seems like we
are reading the dataset twice, but Dask is able to fuse tasks that are doing the same to avoid
computing these things twice. Let’s reorder the expression to make it more efficient:&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;result&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;simplify&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;pprint&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;Sum&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;numeric_only&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;
  &lt;span class="n"&gt;ReadParquet&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://coiled-datasets/uber-lyft-tlc/&amp;#39;&lt;/span&gt;
               &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;tips&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
               &lt;span class="n"&gt;filters&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;hvfhs_license_num&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;HV0003&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;This looks quite a bit simpler. &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt; reordered the query and pushed the filter and the column
projection into the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_parquet&lt;/span&gt;&lt;/code&gt; call. We were able to remove quite a few steps from our expression
tree and make the remaining expressions more efficient as well. This represents the steps that
we did manually in the beginning. &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt; performs these steps for arbitrary many columns without
increasing the burden on the developers.&lt;/p&gt;
&lt;p&gt;These are only the two most common and easy to illustrate optimization techniques from &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt;.
Some other useful optimizations are already available:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;len(...)&lt;/span&gt;&lt;/code&gt; will only use the Index to compute the length; additionally we can ignore many operations
that won’t change the shape of a DataFrame, like a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;replace&lt;/span&gt;&lt;/code&gt; call.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;set_index&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sort_values&lt;/span&gt;&lt;/code&gt; won’t eagerly trigger computations.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Better informed selection of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;merge&lt;/span&gt;&lt;/code&gt; algorithms.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;…&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We are still adding more optimization techniques to make Dask DataFrame queries more efficient.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/08/25/dask-expr-introduction.md&lt;/span&gt;, line 145)&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="try-it-out"&gt;
&lt;h1&gt;Try it out&lt;/h1&gt;
&lt;p&gt;The project is in a state where interested users should try it out. We published a couple of
releases. The API covers a big chunk of the Dask DataFrame API, and we keep adding more.
We have already observed very impressive performance improvements for workflows that would benefit
from query optimization. Memory usage is down for these workflows as well.&lt;/p&gt;
&lt;p&gt;We are very much looking for feedback and potential avenues to improve the library. Please give it
a shot and share your experience with us.&lt;/p&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt; is not integrated into the main Dask DataFrame implementation yet. You can install it
with:&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;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;expr&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The API is very similar to what Dask DataFrame provides. It exposes mostly the same methods as
Dask DataFrame does. You can use the same methods in most cases.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_expr&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You can find a list of supported operations in the
&lt;a class="reference external" href="https://github.com/dask-contrib/dask-expr#api-coverage"&gt;Readme&lt;/a&gt;. This project is still very much
in progress. The API might change without warning. We are aiming for weekly releases to push new
features out as fast as possible.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/08/25/dask-expr-introduction.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="why-are-we-adding-this-now"&gt;
&lt;h1&gt;Why are we adding this now?&lt;/h1&gt;
&lt;p&gt;Historically, Dask focused on flexibility and smart scheduling instead of query optimization.
The distributed scheduler built into Dask uses sophisticated algorithms to ensure ideal scheduling
of individual tasks. It tries to ensure that your resources are utilized as efficient as possible.
The graph construction process enables Dask users to build very
flexible and complicated graphs that reach beyond SQL operations. The flexibility that is provided
by the &lt;a class="reference external" href="https://docs.dask.org/en/latest/futures.html"&gt;Dask futures API&lt;/a&gt; requires very intelligent
algorithms, but it enables users to build highly sophisticated graphs. The following picture shows
the graph for a credit risk model:&lt;/p&gt;
&lt;a href="/images/dask_expr/graph_credit_risk_model.png"&gt;
&lt;img src="/images/dask_expr/graph_credit_risk_model.png"
     width="70%"
     alt="Computation graph representing a credit risk model"&gt;&lt;/a&gt;
&lt;p&gt;The nature of the powerful scheduler and the physical optimizations enables us to build very
complicated programs that will then run efficiently. Unfortunately, the nature of these optimizations
does not enable us to avoid scheduling work that is not necessary. This is where the current effort
to build high level query optimization into Dask comes in.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/08/25/dask-expr-introduction.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="conclusion"&gt;
&lt;h1&gt;Conclusion&lt;/h1&gt;
&lt;p&gt;Dask comes with a very smart distributed scheduler but without much logical query planning. This
is something we are rectifying now through building a high level query optimizer into Dask
DataFrame. We expect to improve performance and reduce memory usage for an average Dask workflow.&lt;/p&gt;
&lt;p&gt;This API is read for interested users to play around with. It covers a good chunk of the DataFrame
API. The library is under active development, we expect to add many more interesting things over
the coming weeks and months.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2023/08/25/dask-expr-introduction/"/>
    <summary>This work was engineered and supported by Coiled and NVIDIA. Thanks to Patrick Hoefler and Rick Zamora, in particular. Original version of this post appears on blog.coiled.io</summary>
    <category term="dask" label="dask"/>
    <category term="performance" label="performance"/>
    <category term="queryoptimizer" label="query optimizer"/>
    <published>2023-08-25T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2023/04/18/dask-upstream-testing/</id>
    <title>Upstream testing in Dask</title>
    <updated>2023-04-18T00:00:00+00:00</updated>
    <author>
      <name>James Bourbeau</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;Original version of this post appears on &lt;a class="reference external" href="https://blog.coiled.io/blog/dask-upstream-testing.html?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-upstream-testing"&gt;blog.coiled.io&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://www.dask.org/"&gt;Dask&lt;/a&gt; has deep integrations with other libraries in the PyData ecosystem like NumPy, pandas, Zarr, PyArrow, and more.
Part of providing a good experience for Dask users is making sure that Dask continues to work well with this community
of libraries as they push out new releases. This post walks through how Dask maintainers proactively ensure Dask
continuously works with its surrounding ecosystem.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/04/18/dask-upstream-testing.md&lt;/span&gt;, line 17)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="nightly-testing"&gt;

&lt;p&gt;Dask has a &lt;a class="reference external" href="https://github.com/dask/dask/blob/834a19eaeb6a5d756ca4ea90b56ca9ac943cb051/.github/workflows/upstream.yml"&gt;dedicated CI build&lt;/a&gt;
that runs Dask’s normal test suite once a day with unreleased, nightly versions of
several libraries installed. This lets us check whether or not a recent change in a library like NumPy or pandas
breaks some aspect of Dask’s functionality.&lt;/p&gt;
&lt;p&gt;To increase visibility when such a breakage occurs, as part of the upstream CI build, an
issue is automatically opened that provides a summary of what tests failed and links to the build
logs for the corresponding failure (&lt;a class="reference external" href="https://github.com/dask/dask/issues/9736"&gt;here’s an example issue&lt;/a&gt;).&lt;/p&gt;
&lt;div align="center"&gt;
&lt;img src="/images/example-upstream-CI-issue.png" style="max-width: 100%;" width="100%" /&gt;
&lt;/div&gt;
&lt;p&gt;This makes it less likely that a failing upstream build goes unnoticed.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/04/18/dask-upstream-testing.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="how-things-can-break-and-are-fixed"&gt;
&lt;h1&gt;How things can break and are fixed&lt;/h1&gt;
&lt;p&gt;There are usually two different ways in which things break. Either:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;A library made an intentional change in behavior and a corresponding compatibility change needs to be made in
Dask (the next section has an example of this case).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;There was some unintentional consequence of a change made in a library that resulted in a breakage in Dask.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;When the latter case occurs, Dask maintainers can then engage with other library maintainers to resolve the
unintended breakage. This all happens before any libraries push out a new release, so no user code breaks.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/04/18/dask-upstream-testing.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="example-pandas-2-0"&gt;
&lt;h1&gt;Example: pandas 2.0&lt;/h1&gt;
&lt;p&gt;One specific example of this process in action is the recent pandas 2.0 release.
This is a major version release and contains significant breaking changes like removing deprecated functionality.&lt;/p&gt;
&lt;p&gt;As these breaking changes were merged into pandas, we started seeing related failures in Dask’s upstream CI build.
Dask maintainers were then able to add
&lt;a class="reference external" href="https://github.com/dask/dask/pulls?q=is%3Apr+%22pandas+2.0%22+in%3Atitle"&gt;a variety of compatibility changes&lt;/a&gt; so
that Dask works well with pandas 2.0 immediately.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/04/18/dask-upstream-testing.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="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;Special thanks to &lt;a class="reference external" href="https://github.com/keewis"&gt;Justus Magin&lt;/a&gt; for his work on the
&lt;a class="reference external" href="https://github.com/xarray-contrib/issue-from-pytest-log"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;xarray-contrib/issue-from-pytest-log&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; GitHub action.
We’ve found this to be really convenient for easily opening up GitHub issues when test failures occur.&lt;/p&gt;
&lt;p&gt;Also, thanks to &lt;a class="reference external" href="https://github.com/j-bennet"&gt;Irina Truong&lt;/a&gt; (&lt;a class="reference external" href="https://www.coiled.io/?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-upstream-testing"&gt;Coiled&lt;/a&gt;), &lt;a class="reference external" href="https://github.com/phofl"&gt;Patrick Hoefler&lt;/a&gt; (&lt;a class="reference external" href="https://www.coiled.io/?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-upstream-testing"&gt;Coiled&lt;/a&gt;),
and &lt;a class="reference external" href="https://github.com/mroeschke"&gt;Matthew Roeschke&lt;/a&gt; (&lt;a class="reference external" href="https://rapids.ai/"&gt;NVIDIA&lt;/a&gt;) for their efforts ensuring pandas and Dask
continue to work together.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2023/04/18/dask-upstream-testing/"/>
    <summary>Original version of this post appears on blog.coiled.io</summary>
    <category term="dask" label="dask"/>
    <category term="ecosystem" label="ecosystem"/>
    <category term="pydata" label="pydata"/>
    <published>2023-04-18T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2023/04/14/scheduler-environment-requirements/</id>
    <title>Do you need consistent environments between the client, scheduler and workers?</title>
    <updated>2023-04-14T00:00:00+00:00</updated>
    <author>
      <name>Florian Jetter</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;Update May 3rd 2023: &lt;a class="reference external" href="https://github.com/dask/dask-blog/pull/166"&gt;Clarify GPU recommendations&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;With the release &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;2023.4.0&lt;/span&gt;&lt;/code&gt; of dask and distributed we are making a change which may require the Dask scheduler to have consistent software and hardware capabilities as the client and workers.&lt;/p&gt;
&lt;p&gt;It has always been recommended that your client and workers have a consistent software and hardware environment so that data structures and dependencies can be pickled and passed between them. However recent changes to the Dask scheduler mean that we now also require your scheduler to have the same consistent environment as everything else.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/04/14/scheduler-environment-requirements.md&lt;/span&gt;, line 15)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="what-does-this-mean-for-me"&gt;

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

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

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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

    &lt;span class="n"&gt;func&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ReadFeather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/04/12/from-map.md&lt;/span&gt;, line 251)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="conclusion"&gt;
&lt;h1&gt;Conclusion&lt;/h1&gt;
&lt;p&gt;It is now easier than ever to create a Dask DataFrame collection from an arbitrary data source. Although the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.delayed&lt;/span&gt;&lt;/code&gt; API has already enabled similar functionality for many years, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; now makes it possible to implement a custom IO function without sacrificing any of the high-level graph optimizations leveraged by the rest of the Dask DataFrame API.&lt;/p&gt;
&lt;p&gt;Start experimenting with &lt;a class="reference external" href="https://docs.dask.org/en/stable/generated/dask.dataframe.from_map.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; today, and let us know how it goes!&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2023/04/12/from-map/"/>
    <summary>Dask DataFrame provides dedicated IO functions for several popular tabular-data formats, like CSV and Parquet. If you are working with a supported format, then the corresponding function (e.g read_csv) is likely to be the most reliable way to create a new Dask DataFrame collection. For other workflows, from_map now offers a convenient way to define a DataFrame collection as an arbitrary function mapping. While these kinds of workflows have historically required users to adopt the Dask Delayed API, from_map now makes custom collection creation both easier and more performant.</summary>
    <category term="IO" label="IO"/>
    <category term="dataframe" label="dataframe"/>
    <published>2023-04-12T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2023/03/15/shuffling-large-data-at-constant-memory/</id>
    <title>Shuffling large data at constant memory in Dask</title>
    <updated>2023-03-15T00:00:00+00:00</updated>
    <author>
      <name>Hendrik Makait</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This work was engineered and supported by &lt;a class="reference external" href="https://coiled.io?utm_source=dask-blog&amp;amp;amp;utm_medium=shuffling-large-data-at-constant-memory"&gt;Coiled&lt;/a&gt;. In particular, thanks to &lt;a class="reference external" href="https://github.com/fjetter"&gt;Florian Jetter&lt;/a&gt;, &lt;a class="reference external" href="https://github.com/gjoseph92"&gt;Gabe Joseph&lt;/a&gt;, &lt;a class="reference external" href="https://github.com/hendrikmakait"&gt;Hendrik Makait&lt;/a&gt;, and &lt;a class="reference external" href="https://matthewrocklin.com/?utm_source=dask-blog&amp;amp;amp;utm_medium=shuffling-large-data-at-constant-memory"&gt;Matt Rocklin&lt;/a&gt;. Original version of this post appears on &lt;a class="reference external" href="https://blog.coiled.io/blog/shuffling-large-data-at-constant-memory.html?utm_source=dask-blog&amp;amp;amp;utm_medium=shuffling-large-data-at-constant-memory"&gt;blog.coiled.io&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;With release &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;2023.2.1&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe&lt;/span&gt;&lt;/code&gt; introduces a new shuffling method called P2P, making sorts, merges, and joins faster and using constant memory.
Benchmarks show impressive improvements:&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="P2P shuffling uses constant memory while task-based shuffling scales linearly." src="/images/p2p_benchmark_memory_usage.png" style="width: 100%;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;P2P shuffling (blue) uses constant memory while task-based shuffling (orange) scales linearly.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;This article describes the problem, the new solution, and the impact on performance.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 24)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="what-is-shuffling"&gt;

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

&lt;span class="c1"&gt;# The dataframe.shuffle.method config is available since 2023.3.1&lt;/span&gt;
&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;dataframe.shuffle.method&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;tasks&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;}):&lt;/span&gt;
    &lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# many cluster managers send current dask config automatically&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;For more information on deactivating P2P shuffle, see the &lt;a class="reference external" href="https://github.com/dask/distributed/discussions/7509"&gt;discussion #7509&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 152)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="what-about-arrays"&gt;
&lt;h1&gt;What about arrays?&lt;/h1&gt;
&lt;p&gt;While the original motivation was to optimize large-scale dataframe joins, the P2P system is useful for all problems requiring lots of communication between tasks.
For array workloads, this often occurs when rechunking data, such as when organizing a matrix by rows when it was stored by columns.
Similar to dataframe joins, array rechunking has been inefficient in the past, which has become such a problem that the array community built specialized tools like &lt;a class="reference external" href="https://github.com/pangeo-data/rechunker"&gt;rechunker&lt;/a&gt; to avoid it entirely.&lt;/p&gt;
&lt;p&gt;There is a &lt;a class="reference external" href="https://github.com/dask/distributed/pull/7534"&gt;naive implementation&lt;/a&gt; of array rechunking using the P2P system available for experimental use.
Benchmarking this implementation shows mixed results:&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="P2P rechunking uses constant memory." src="/images/p2p_benchmark_arrays.png" style="width: 100%;"/&gt;
&lt;/figure&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;👍 &lt;strong&gt;Constant memory use:&lt;/strong&gt; As with dataframe operations, memory use is constant.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;❓ &lt;strong&gt;Variable runtime:&lt;/strong&gt; The runtime of workloads may increase with P2P.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;👎 &lt;strong&gt;Memory overhead:&lt;/strong&gt; There is a large memory overhead for many small partitions.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The constant memory use is a very promising result.
There are several ways to tackle the current limitations.
We expect this to improve as we work with collaborators from the array computing community.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 173)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="next-steps"&gt;
&lt;h1&gt;Next steps&lt;/h1&gt;
&lt;p&gt;Development on P2P shuffling is not done yet.
For the future we plan the following:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt;&lt;/strong&gt;:&lt;/p&gt;
&lt;p&gt;While the early prototype of array rechunking is promising, it’s not there yet.
We plan to do the following:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Intelligently select which algorithm to use (task-based rechunking is better sometimes)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Work with collaborators on rechunking to improve performance&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Seek out other use cases, like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt;, where this might be helpful&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Failure recovery&lt;/strong&gt;:&lt;/p&gt;
&lt;p&gt;Make P2P shuffling resilient to worker loss; currently, it has to restart entirely.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Performance tuning&lt;/strong&gt;:&lt;/p&gt;
&lt;p&gt;Performance today is good, but not yet at peak hardware speeds.
We can improve this in a few ways:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Hiding disk I/O&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Using more memory when appropriate for smaller shuffles&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Improved batching of small operations&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;More efficient serialization&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;To follow along with the development, subscribe to the &lt;a class="reference external" href="https://github.com/dask/distributed/issues/7507"&gt;tracking issue&lt;/a&gt; on GitHub.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 203)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="summary"&gt;
&lt;h1&gt;Summary&lt;/h1&gt;
&lt;p&gt;Shuffling data is a common operation in dataframe workloads.
Since &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;2023.2.1&lt;/span&gt;&lt;/code&gt;, Dask defaults to P2P shuffling for distributed clusters and shuffles data faster and at constant memory.
This improvement unlocks previously un-runnable workload sizes and efficiently uses your clusters.
Finally, P2P shuffling demonstrates extending Dask to add new paradigms while leveraging the foundations of its distributed engine.&lt;/p&gt;
&lt;p&gt;Share results in &lt;a class="reference external" href="https://github.com/dask/distributed/issues/7507"&gt;this discussion thread&lt;/a&gt; or follow development at this &lt;a class="reference external" href="https://github.com/dask/distributed/issues/7507"&gt;tracking issue&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2023/03/15/shuffling-large-data-at-constant-memory/"/>
    <summary>This work was engineered and supported by Coiled. In particular, thanks to Florian Jetter, Gabe Joseph, Hendrik Makait, and Matt Rocklin. Original version of this post appears on blog.coiled.io</summary>
    <category term="dask" label="dask"/>
    <category term="distributed" label="distributed"/>
    <category term="p2p" label="p2p"/>
    <category term="shuffling" label="shuffling"/>
    <published>2023-03-15T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2023/02/13/dask-on-flyte/</id>
    <title>Managing dask workloads with Flyte</title>
    <updated>2023-02-13T00:00:00+00:00</updated>
    <author>
      <name>Bernhard Stadlbauer</name>
    </author>
    <content type="html">&lt;p&gt;It is now possible to manage &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; workloads using &lt;a class="reference external" href="https://flyte.org/"&gt;Flyte&lt;/a&gt; 🎉!&lt;/p&gt;
&lt;p&gt;The major advantages are:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Each &lt;a class="reference external" href="https://docs.flyte.org/projects/flytekit/en/latest/generated/flytekit.task.html"&gt;Flyte &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;task&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; spins up its own ephemeral &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; cluster using a Docker image tailored to the task, ensuring consistency in the Python environment across the client, scheduler, and workers.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Flyte will use the existing &lt;a class="reference external" href="https://kubernetes.io/"&gt;Kubernetes&lt;/a&gt; infrastructure to spin up &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; clusters.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Spot/Preemtible instances are natively supported.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The whole &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; task can be cached.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Enabling &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; support in an already running Flyte setup can be done in just a few minutes.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This is what a Flyte &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;task&lt;/span&gt;&lt;/code&gt; backed by a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; cluster with four workers looks like:&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;typing&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;List&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;Client&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;flytekit&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;task&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Resources&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;flytekitplugins.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;Dask&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;WorkerGroup&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Scheduler&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;inc&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="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;


&lt;span class="nd"&gt;@task&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;task_config&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;Dask&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
        &lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;Scheduler&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
            &lt;span class="n"&gt;requests&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;Resources&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cpu&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;2&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="p"&gt;),&lt;/span&gt;
        &lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;WorkerGroup&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
            &lt;span class="n"&gt;number_of_workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="n"&gt;limits&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;Resources&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cpu&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;8&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;mem&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;32Gi&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="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;increment_numbers&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;list_length&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;List&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt;
    &lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;futures&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inc&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="n"&gt;list_length&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;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;gather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This task can run locally using a standard &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed.Client()&lt;/span&gt;&lt;/code&gt; and can scale to arbitrary cluster sizes once &lt;a class="reference external" href="https://docs.flyte.org/projects/cookbook/en/latest/getting_started/package_register.html"&gt;registered with Flyte&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/02/13/dask-on-flyte.md&lt;/span&gt;, line 52)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="what-is-flyte"&gt;

&lt;p&gt;&lt;a class="reference external" href="https://flyte.org/"&gt;Flyte&lt;/a&gt; is a Kubernetes native workflow orchestration engine. Originally developed at &lt;a class="reference external" href="https://www.lyft.com/"&gt;Lyft&lt;/a&gt;, it is now an open-source (&lt;a class="reference external" href="https://github.com/flyteorg/flyte"&gt;Github&lt;/a&gt;) and a graduate project under the Linux Foundation. It stands out among similar tools such as &lt;a class="reference external" href="https://airflow.apache.org/"&gt;Airflow&lt;/a&gt; or &lt;a class="reference external" href="https://argoproj.github.io/"&gt;Argo&lt;/a&gt; due to its key features, which include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Caching/Memoization of previously executed tasks for improved performance&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kubernetes native&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Workflow definitions in Python, not e.g.,&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;YAML&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Strong typing between tasks and workflows using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Protobuf&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dynamic generation of workflow DAGs at runtime&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ability to run workflows locally&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;A simple workflow would look something 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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;typing&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;List&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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;flytekit&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;task&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;workflow&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Resources&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;flytekitplugins.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;Dask&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;WorkerGroup&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Scheduler&lt;/span&gt;


&lt;span class="nd"&gt;@task&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;task_config&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;Dask&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
        &lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;Scheduler&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
            &lt;span class="n"&gt;requests&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;Resources&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cpu&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;2&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="p"&gt;),&lt;/span&gt;
        &lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;WorkerGroup&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
            &lt;span class="n"&gt;number_of_workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="n"&gt;limits&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;Resources&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cpu&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;8&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;mem&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;32Gi&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="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;expensive_data_preparation&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;input_files&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;List&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="nb"&gt;str&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt; &lt;span class="o"&gt;-&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="c1"&gt;# Expensive, highly parallel `dask` code&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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Some large DataFrame, Flyte will handle serialization&lt;/span&gt;


&lt;span class="nd"&gt;@task&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;train&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;input_data&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;DataFrame&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&amp;gt;&lt;/span&gt; &lt;span class="nb"&gt;str&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="c1"&gt;# Model training, can also use GPU, etc.&lt;/span&gt;
    &lt;span class="o"&gt;...&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;s3://path-to-model&amp;quot;&lt;/span&gt;


&lt;span class="nd"&gt;@workflow&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;train_model&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;input_files&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;List&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="nb"&gt;str&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt; &lt;span class="o"&gt;-&amp;gt;&lt;/span&gt; &lt;span class="nb"&gt;str&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;prepared_data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;expensive_data_preparation&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;input_files&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;input_files&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;train&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;input_data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;prepared_data&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In the above, both &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;expensive_data_preparation()&lt;/span&gt;&lt;/code&gt; as well as &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;train()&lt;/span&gt;&lt;/code&gt; would be run in their own Pod(s) in Kubernetes, while the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;train_model()&lt;/span&gt;&lt;/code&gt; workflow is a DSL which creates a Directed Acyclic Graph (DAG) of the workflow. It will determine the order of tasks based on their inputs and outputs. Input and output types (based on the type hints) will be validated at registration time to avoid surprises at runtime.&lt;/p&gt;
&lt;p&gt;After registration with Flyte, the workflow can be started from the UI:&lt;/p&gt;
&lt;img src="/images/dask-flyte-workflow.png" alt="Dask workflow in the Flyte UI" style="max-width: 100%;" width="100%" /&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/02/13/dask-on-flyte.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="why-use-the-dask-plugin-for-flyte"&gt;
&lt;h1&gt;Why use the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; plugin for Flyte?&lt;/h1&gt;
&lt;p&gt;At first glance, Flyte and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; look similar in what they are trying to achieve, both capable of creating a DAG from user functions, managing inputs and outputs, etc. However, the major conceptual difference lies in their approach. While &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; has long-lived workers to run tasks, a Flyte task is a designated Kubernetes Pod that creates a significant overhead in task-runtime.&lt;/p&gt;
&lt;p&gt;While &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; tasks incur an overhead of around one millisecond (refer to the &lt;a class="reference external" href="https://distributed.dask.org/en/stable/efficiency.html#use-larger-tasks"&gt;docs&lt;/a&gt;), spinning up a new Kubernetes pod takes several seconds. The long-lived nature of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; workers allows for optimization of the DAG, running tasks that operate on the same data on the same node, reducing the need for inter-worker data serialization (known as shuffling). With Flyte tasks being ephemeral, this optimization is not possible, and task outputs are serialized to a blob storage instead.&lt;/p&gt;
&lt;p&gt;Given the limitations discussed above, why use Flyte? Flyte is not intended to replace tools such as dask or Apache Spark, but rather provides an orchestration layer on top. While workloads can be run directly in Flyte, such as training a single GPU model, Flyte offers &lt;a class="reference external" href="https://flyte.org/integrations"&gt;numerous integrations&lt;/a&gt; with other popular data processing tools.&lt;/p&gt;
&lt;p&gt;With Flyte managing the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; cluster lifecycle, each &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; Flyte task will run on its own dedicated &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; cluster made up of Kubernetes pods. When the Flyte task is triggered from the UI, Flyte will spin up a dask cluster tailored to the task, which will then be used to execute the user code. This enables the use of different Docker images with varying dependencies for different tasks, whilst always ensuring that the dependencies of the client, scheduler, and workers are consistent.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/02/13/dask-on-flyte.md&lt;/span&gt;, line 120)&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-prerequisites-are-required-to-run-dask-tasks-in-flyte"&gt;
&lt;h1&gt;What prerequisites are required to run &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; tasks in Flyte?&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The Kubernetes cluster needs to have the &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator.html"&gt;dask operator&lt;/a&gt; installed.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Flyte version &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;1.3.0&lt;/span&gt;&lt;/code&gt; or higher is required.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; plugin needs to be enabled in the Flyte propeller config. (refer to the &lt;a class="reference external" href="https://docs.flyte.org/en/latest/deployment/plugins/k8s/index.html#specify-plugin-configuration"&gt;docs&lt;/a&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The Docker image associated with the task must have the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;flytekitplugins-dask&lt;/span&gt;&lt;/code&gt; package installed in its Python environment.&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/2023/02/13/dask-on-flyte.md&lt;/span&gt;, line 127)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="how-do-things-work-under-the-hood"&gt;
&lt;h1&gt;How do things work under the hood?&lt;/h1&gt;
&lt;p&gt;&lt;em&gt;Note: The following is for reference only and is not necessary for users who only use the plugin. However, it could be useful for easier debugging.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;On a high-level overview, the following steps occur when a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; task is initiated in Flyte:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;A &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;FlyteWorkflow&lt;/span&gt;&lt;/code&gt; &lt;a class="reference external" href="https://kubernetes.io/docs/concepts/extend-kubernetes/api-extension/custom-resources/"&gt;Custom Resource (CR)&lt;/a&gt; is created in Kubernetes.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Flyte Propeller, a &lt;a class="reference external" href="https://kubernetes.io/docs/concepts/extend-kubernetes/operator/"&gt;Kubernetes Operator&lt;/a&gt;), detects the creation of the workflow.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The operator inspects the task’s spec and identifies it as a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; task. It verifies if it has the required plugin associated with it and locates the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; plugin.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; plugin within Flyte Propeller picks up the task defintion and creates a &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html#daskjob"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskJob&lt;/span&gt;&lt;/code&gt; Custom Resource&lt;/a&gt; using the &lt;a class="reference external" href="https://github.com/bstadlbauer/dask-k8s-operator-go-client/"&gt;dask-k8s-operator-go-client&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator.html"&gt;dask operator&lt;/a&gt; picks up the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskJob&lt;/span&gt;&lt;/code&gt; resource and runs the job accordingly. It spins up a pod to run the client/job-runner, one for the scheduler, and additional worker pods as designated in the Flyte task decorator.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;While the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; task is running, Flyte Propeller continuously monitors the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskJob&lt;/span&gt;&lt;/code&gt; resource, waiting on it to report success or failure. Once the job has finished or the Flyte task has been terminated, all &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; related resources will be cleaned up.&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/2023/02/13/dask-on-flyte.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="useful-links"&gt;
&lt;h1&gt;Useful links&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://docs.flyte.org/en/latest/"&gt;Flyte documentation&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://flyte.org/community"&gt;Flyte community&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://docs.flyte.org/projects/cookbook/en/latest/auto/integrations/kubernetes/k8s_dask/index.html"&gt;flytekitplugins-dask user documentation&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://docs.flyte.org/en/latest/deployment/plugins/k8s/index.html"&gt;flytekitplugins-dask deployment documentation&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/"&gt;dask-kubernetes documentation&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://blog.dask.org/2022/11/09/dask-kubernetes-operator"&gt;Blog post on the dask kubernetes operator&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;In case there are any questions or concerns, don’t hesitate to reach out. You can contact Bernhard Stadlbauer via the &lt;a class="reference external" href="https://slack.flyte.org/"&gt;Flyte Slack&lt;/a&gt; or via &lt;a class="reference external" href="https://github.com/bstadlbauer"&gt;GitHub&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;I would like to give shoutouts to &lt;a class="reference external" href="https://jacobtomlinson.dev/"&gt;Jacob Tomlinson&lt;/a&gt; (Dask) and &lt;a class="reference external" href="https://github.com/hamersaw"&gt;Dan Rammer&lt;/a&gt; (Flyte) for all of the help I’ve received. This would not have been possible without your support!&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2023/02/13/dask-on-flyte/"/>
    <summary>It is now possible to manage dask workloads using Flyte 🎉!</summary>
    <category term="Flyte" label="Flyte"/>
    <category term="Kubernetes" label="Kubernetes"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <published>2023-02-13T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2023/02/02/easy-cpu-gpu/</id>
    <title>Easy CPU/GPU Arrays and Dataframes</title>
    <updated>2023-02-02T00:00:00+00:00</updated>
    <author>
      <name>Ben Zaitlen (NVIDIA)</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This article was originally posted on the &lt;a class="reference external" href="https://medium.com/rapids-ai/easy-cpu-gpu-arrays-and-dataframes-run-your-dask-code-where-youd-like-e349d92351d"&gt;RAPIDS blog&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;It’s now easy to switch between CPU (NumPy / Pandas) and GPU (CuPy / cuDF) in Dask.
As of Dask 2022.10.0, users can optionally select the backend engine for input IO and data creation. In the short-term, the goal of the backend-configuration system is to enable Dask users to write code that will run on both CPU and GPU systems.&lt;/p&gt;
&lt;p&gt;The preferred backend can be configured using the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;array.backend&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dataframe.backend&lt;/span&gt;&lt;/code&gt; options with the standard Dask configuration system:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;array.backend&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;cupy&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;})&lt;/span&gt;
&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;dataframe.backend&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;cudf&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;})&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/02/02/easy-cpu-gpu.md&lt;/span&gt;, line 22)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="dispatching-for-array-creation"&gt;

&lt;p&gt;To see how users can easily switch between NumPy and CuPy, let’s start by creating an array of ones:&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;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;array.backend&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;cupy&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;}):&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;darr&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;10&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;# Get cupy-backed collection&lt;/span&gt;
&lt;span class="gp"&gt;...&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;darr&lt;/span&gt;
&lt;span class="go"&gt;dask.array&amp;lt;ones_like, shape=(10,), dtype=float64, chunksize=(5,), chunktype=cupy.ndarray&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chunktype&lt;/span&gt;&lt;/code&gt; informs us that the array is constructed with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cupy.ndarray&lt;/span&gt;&lt;/code&gt;
objects instead of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;numpy.ndarray&lt;/span&gt;&lt;/code&gt; objects.&lt;/p&gt;
&lt;p&gt;We’ve also improved the user experience for random array creation. Previously, if a user wanted to create a CuPy-backed Dask array, they were required to define an explicit &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomState&lt;/span&gt;&lt;/code&gt; object in Dask using CuPy. For example, the following code worked prior to Dask 2022.10.0, but seems rather verbose:&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;cupy&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;rs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;darr&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;rs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;3&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;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="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;2&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="n"&gt;darr&lt;/span&gt;
&lt;span class="go"&gt;dask.array&amp;lt;randint, shape=(10, 20), dtype=int64, chunksize=(2, 5), chunktype=cupy.ndarray&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now, we can leverage the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;array.backend&lt;/span&gt;&lt;/code&gt; configuration to create a CuPy-backed dask array for random 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="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;array.backend&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;cupy&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;}):&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;darr&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;randint&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;3&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;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="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;2&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;# Get cupy-backed collection&lt;/span&gt;
&lt;span class="gp"&gt;...&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;darr&lt;/span&gt;
&lt;span class="go"&gt;dask.array&amp;lt;randint, shape=(10, 20), dtype=int64, chunksize=(2, 5), chunktype=cupy.ndarray&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;array.backend&lt;/span&gt;&lt;/code&gt; is significantly easier and much more ergonomic – it supports all basic array creation methods including: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ones&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;zeros&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;empty&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;full&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;arange&lt;/span&gt;&lt;/code&gt;, and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;random&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_array&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_zarr&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_tiledb&lt;/span&gt;&lt;/code&gt; have not yet been implemented
with this functionality&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/2023/02/02/easy-cpu-gpu.md&lt;/span&gt;, line 64)&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="dispatching-for-dataframe-creation"&gt;
&lt;h1&gt;Dispatching for Dataframe Creation&lt;/h1&gt;
&lt;p&gt;When creating Dask Dataframes backed by either Pandas or cuDF, the beginning is often the input I/O methods: read_csv, read_parquet, etc. We’ll first start by constructing a dataframe on the fly with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_dict&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="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;dataframe.backend&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;cudf&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;}):&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;a&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;range&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="s2"&gt;&amp;quot;b&amp;quot;&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;10&lt;/span&gt;&lt;span class="p"&gt;)}&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_dict&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="n"&gt;npartitions&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="gp"&gt;...&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;dask_cudf.DataFrame | 2 tasks | 2 npartitions&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here we can tell we have a cuDF backed dataframe and we are using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-cudf&lt;/span&gt;&lt;/code&gt;
because the repr shows us the type: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;&amp;lt;dask_cudf.DataFrame&lt;/span&gt; &lt;span class="pre"&gt;|&lt;/span&gt; &lt;span class="pre"&gt;2&lt;/span&gt; &lt;span class="pre"&gt;tasks&lt;/span&gt; &lt;span class="pre"&gt;|&lt;/span&gt; &lt;span class="pre"&gt;2&lt;/span&gt; &lt;span class="pre"&gt;npartitions&amp;gt;&lt;/span&gt;&lt;/code&gt;.
Let’s also demonstrate the read functionality by generating CSV and
Parquet 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="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;example.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;single_file&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;example.parquet&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;Now we are simply repeating the config setting but instead using the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_csv&lt;/span&gt;&lt;/code&gt;
and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_parquet&lt;/span&gt;&lt;/code&gt; methods:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;dataframe.backend&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;cudf&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;}):&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;example.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;type&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="gp"&gt;...&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;class &amp;#39;dask_cudf.core.DataFrame&amp;#39;&amp;gt;&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;dataframe.backend&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;cudf&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;}):&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;example.parquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="nb"&gt;type&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;...&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;class &amp;#39;dask_cudf.core.DataFrame&amp;#39;&amp;gt;&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/02/02/easy-cpu-gpu.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="why-is-this-useful"&gt;
&lt;h1&gt;Why is this Useful ?&lt;/h1&gt;
&lt;p&gt;As hardware changes in exciting and exotic ways with: GPUs, TPUs, IPUs,
etc., we want to provide the same interface and treat hardware as an
abstraction. For example, many PyTorch workflows start with 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="n"&gt;device&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;cuda&amp;#39;&lt;/span&gt; &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;torch&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cuda&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;is_available&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="k"&gt;else&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;cpu&amp;#39;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And what follows is typically standard hardware agnostic PyTorch. This is
incredibly powerful as the user (in most cases) should not care what hardware
underlies the source. As such, it enables the user to develop PyTorch anywhere
and everywhere. The new Dask backend selection configurations gives users a
similar freedom.&lt;/p&gt;
&lt;p&gt;Conclusion&lt;/p&gt;
&lt;p&gt;Our long-term goal of this feature is to enable Dask users to use any backend library in dask.array and dask.dataframe, as long as that library conforms to the minimal “array” or “dataframe” standard defined by the data-api consortium, respectively.&lt;/p&gt;
&lt;p&gt;The RAPIDS team consistently works with the open-source community to understand and address emerging needs. If you’re an open-source maintainer interested in bringing GPU-acceleration to your project, please reach out on Github or Twitter. The RAPIDS team would love to learn how potential new algorithms or toolkits would impact your work.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2023/02/02/easy-cpu-gpu/"/>
    <summary>This article was originally posted on the RAPIDS blog.</summary>
    <category term="GPU" label="GPU"/>
    <published>2023-02-02T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2022/11/21/november-demo-day/</id>
    <title>Dask Demo Day November 2022</title>
    <updated>2022-11-21T00:00:00+00:00</updated>
    <author>
      <name>Richard Pelgrim (Coiled)</name>
    </author>
    <content type="html">&lt;p&gt;Once a month, the Dask Community team hosts Dask Demo Day: an informal and fun online hangout where folks can showcase new or lesser-known Dask features and the rest of us can learn about all the things we didn’t know Dask could do 😁&lt;/p&gt;
&lt;p&gt;November’s Dask Demo Day had five great demos. We learned about:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#visualization-at-lightning-speed"&gt;&lt;span class="xref myst"&gt;Visualizing 2-billion lightning flashes with Dask, RAPIDS and Datashader&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#the-new-dask-cli"&gt;&lt;span class="xref myst"&gt;The new Dask CLI&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#xgboost-hpo-with-dask-and-optuna"&gt;&lt;span class="xref myst"&gt;The Dask-Optuna integration for distributed hyperparameter optimization&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#dask-for-awkward-arrays"&gt;&lt;span class="xref myst"&gt;Dask-Awkward&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#profiling-dask-on-a-cluster-with-py-spy"&gt;&lt;span class="xref myst"&gt;Profiling your Dask code with Dask-PySpy&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This blog gives you a quick overview of the five demos and demonstrates how they might be useful to you. You can &lt;a class="reference external" href="https://www.youtube.com/embed/_x7oaSEJDjA"&gt;watch the full recording below&lt;/a&gt;.&lt;/p&gt;
&lt;iframe width="560" height="315" src="https://www.youtube.com/embed/_x7oaSEJDjA" title="YouTube video player" frameborder="0" allow="accelerometer; autoplay; clipboard-write; encrypted-media; gyroscope; picture-in-picture" allowfullscreen&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/2022/11/21/november-demo-day.md&lt;/span&gt;, line 23)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="visualization-at-lightning-speed"&gt;

&lt;p&gt;Would it be possible to interactively visualize all the lightning strikes in his dataset, &lt;a class="reference external" href="https://www.albany.edu/daes/faculty/kevin-tyle"&gt;Kevin Tyle&lt;/a&gt; (University of Albany) recently asked himself. In this demo, Kevin shows you how he leveraged &lt;a class="reference external" href="https://developer.nvidia.com/cuda-zone"&gt;CUDA&lt;/a&gt;, &lt;a class="reference external" href="https://rapids.ai/"&gt;RAPIDS-AI&lt;/a&gt;, &lt;a class="reference external" href="https://www.dask.org/"&gt;Dask&lt;/a&gt; and &lt;a class="reference external" href="https://datashader.org/"&gt;Datashader&lt;/a&gt; to build a smooth interactive visualization of 8 years’ worth of lightning strikes. That’s over 2 billion rows of data.&lt;/p&gt;
&lt;p&gt;Kevin shows you how to finetune performance of such a large-scale data processing workflow by:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Leveraging GPUs&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Using a Dask cluster to maximize hardware usage&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Making smart choices about file types&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;img alt="Heatmap of lightning strikes in the US" src="/images/2022-11-demo-day/lightning.png" style="max-width: 100%;" width="100%" /&gt;
&lt;p&gt;Watch the &lt;a class="reference external" href="https://youtu.be/_x7oaSEJDjA?t=167"&gt;full demo&lt;/a&gt; or read more about &lt;a class="reference external" href="https://www.coiled.io/blog/datashader-data-visualisation-performance"&gt;high-performance visualization strategies&lt;/a&gt; with Dask and Datashader.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/21/november-demo-day.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="the-new-dask-cli"&gt;
&lt;h1&gt;The New Dask CLI&lt;/h1&gt;
&lt;p&gt;During the Dask Sprint at &lt;a class="reference external" href="https://conference.scipy.org/"&gt;SciPy&lt;/a&gt; this year, a group of Dask maintainers began work on an upgraded, high-level &lt;a class="reference external" href="https://docs.dask.org/en/stable/cli.html"&gt;Dask CLI&lt;/a&gt;. &lt;a class="reference external" href="https://ddavis.io/about/"&gt;Doug Davis&lt;/a&gt; (Anaconda) walks us through how the CLI works and all the things you can do with it. After installing dask, you can access the CLI by typing dask into your terminal. The tool is designed to be easily extensible by anyone working on Dask. Doug shows you how to add your own components to the Dask CLI.&lt;/p&gt;
&lt;img alt="Screenshot of the new Dask CLI in action" src="/images/2022-11-demo-day/dask-cli.png" style="max-width: 100%;" width="100%" /&gt;
&lt;p&gt;Watch the &lt;a class="reference external" href="https://youtu.be/_x7oaSEJDjA?t=882"&gt;full demo&lt;/a&gt; or read the &lt;a class="reference external" href="https://docs.dask.org/en/stable/cli.html"&gt;Dask CLI documentation&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/21/november-demo-day.md&lt;/span&gt;, line 45)&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="xgboost-hpo-with-dask-and-optuna"&gt;
&lt;h1&gt;XGBoost HPO with Dask and Optuna&lt;/h1&gt;
&lt;p&gt;Have you ever wanted to speed up your hyperparameter searches by running them in parallel? &lt;a class="reference external" href="https://www.jamesbourbeau.com/about/"&gt;James Bourbeau&lt;/a&gt; (Coiled) shows you how you can use the brand-new &lt;a class="reference external" href="https://jrbourbeau.github.io/dask-optuna/"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-optuna&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; integration to run hundreds of hyperparameter searches in parallel on a Dask cluster. Running your Optuna HPO searches on a Dask cluster requires only two changes to your existing optuna code. After making those changes, we’re then able to run 500 HPO iterations in parallel in 25 seconds.&lt;/p&gt;
&lt;img alt="Screenshot of Dask-Optuna running" src="/images/2022-11-demo-day/optuna-dask.png" style="max-width: 100%;" width="100%" /&gt;
&lt;p&gt;Watch the &lt;a class="reference external" href="https://youtu.be/_x7oaSEJDjA?t=1300"&gt;full demo&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/21/november-demo-day.md&lt;/span&gt;, line 53)&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="dask-for-awkward-arrays"&gt;
&lt;h1&gt;Dask for Awkward Arrays&lt;/h1&gt;
&lt;p&gt;The PyData ecosystem has historically focused on rectilinear data structures like DataFrames and regular arrays. &lt;a class="reference external" href="https://awkward-array.readthedocs.io/en/stable/"&gt;Awkward Arrays&lt;/a&gt; brings NumPy-like operations to non-rectilinear data structures and &lt;a class="reference external" href="https://github.com/ContinuumIO/dask-awkward"&gt;dask-awkward&lt;/a&gt; enables you to work with awkward arrays on a distributed cluster in parallel. &lt;a class="reference external" href="https://ddavis.io/about/"&gt;Doug Davis&lt;/a&gt; (Anaconda) walks you through a quick demo of how to use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-awkward&lt;/span&gt;&lt;/code&gt; on a local cluster. This is a helpful tool if you find yourself working with nested data structures at scale.&lt;/p&gt;
&lt;img alt="Screenshot of dask-awkward" src="/images/2022-11-demo-day/awkward.png" style="max-width: 100%;" width="100%" /&gt;
&lt;p&gt;Watch the &lt;a class="reference external" href="https://youtu.be/_x7oaSEJDjA?t=2033"&gt;full demo&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/21/november-demo-day.md&lt;/span&gt;, line 61)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="profiling-dask-on-a-cluster-with-py-spy"&gt;
&lt;h1&gt;Profiling Dask on a Cluster with py-spy&lt;/h1&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/benfred/py-spy"&gt;py-spy&lt;/a&gt; is a Python profiler that lets you dig deeper into your code than just your Python functions. &lt;a class="reference external" href="https://github.com/gjoseph92"&gt;Gabe Joseph&lt;/a&gt; (Coiled) shows you how you can use &lt;a class="reference external" href="https://github.com/gjoseph92/dask-pyspy"&gt;dask-pyspy&lt;/a&gt; to profile code on a Dask cluster. By digging down into compiled code, dask-pyspy is able to discover valuable insights about why your Dask code might be running slow and what you might be able to do to resolve this.&lt;/p&gt;
&lt;img alt="Screenshot of dask-pyspy in action" src="/images/2022-11-demo-day/pyspy.png" style="max-width: 100%;" width="100%" /&gt;
&lt;p&gt;Watch the &lt;a class="reference external" href="https://youtu.be/_x7oaSEJDjA?t=2758"&gt;full demo&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/21/november-demo-day.md&lt;/span&gt;, line 69)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="join-us-for-the-next-demo-day"&gt;
&lt;h1&gt;Join us for the next Demo Day!&lt;/h1&gt;
&lt;p&gt;Dask Demo Day is a great opportunity to learn about the latest developments and features in Dask. It’s also a fun hangout where you can ask questions and interact with some of Dask’s core maintainers in an informal, casual online setting. We’d love to see you at the next Demo Day on December 15th!&lt;/p&gt;
&lt;p&gt;Curious how you can stay connected and find out about the latest Dask news and events?&lt;/p&gt;
&lt;p&gt;You can:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;follow us on Twitter &lt;a class="reference external" href="https://twitter.com/dask_dev"&gt;&amp;#64;dask_dev&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;subscribe to the Dask newsletter by sending a blank email to newsletter+subscribe&amp;#64;dask.org&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;subscribe to the &lt;a class="reference external" href="https://docs.dask.org/en/latest/support.html"&gt;Dask community calendar&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2022/11/21/november-demo-day/"/>
    <summary>Once a month, the Dask Community team hosts Dask Demo Day: an informal and fun online hangout where folks can showcase new or lesser-known Dask features and the rest of us can learn about all the things we didn’t know Dask could do 😁</summary>
    <category term="Community" label="Community"/>
    <published>2022-11-21T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2022/11/15/queuing/</id>
    <title>Reducing memory usage in Dask workloads by 80%</title>
    <updated>2022-11-15T00:00:00+00:00</updated>
    <author>
      <name>Gabe Joseph (Coiled)</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;Original version of this post appears on https://www.coiled.io/blog/reducing-dask-memory-usage&lt;/em&gt;&lt;/p&gt;
&lt;img src="/images/2022-queuing/hero.png" style="max-width: 100%;" width="100%" /&gt;
&lt;!-- Collection page image: double-diff.png --&gt;
&lt;p&gt;There’s a saying in emergency response: “slow is smooth, smooth is fast”.&lt;/p&gt;
&lt;p&gt;That saying has always bothered me, because it doesn’t make sense at first, yet it’s entirely correct.&lt;/p&gt;
&lt;p&gt;By applying this philosophy to the scheduling algorithm in the latest release of Dask, &lt;strong&gt;we’re seeing common workloads use up to 80% less memory than before. This means some workloads that used to be outright un-runnable are now running smoothly&lt;/strong&gt;—an infinity-X speedup!&lt;/p&gt;
&lt;hr class="docutils" /&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/issues/2602"&gt;The second-most upvoted and commented issue of all time&lt;/a&gt; on the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask/distributed&lt;/span&gt;&lt;/code&gt; repo describes, “tasks early in my graph generate data faster than it can be consumed downstream, causing data to pile up, eventually overwhelming my workers”.&lt;/p&gt;
&lt;p&gt;Dask users often struggle with workloads that run out of memory like this. Studying these situations, we realized that the Dask scheduler wasn’t following this “slow is smooth, smooth is fast” adage.&lt;/p&gt;
&lt;p&gt;Here’s what the problem was, and &lt;a class="reference external" href="https://github.com/dask/distributed/pull/6614"&gt;how we’ve addressed it&lt;/a&gt;:&lt;/p&gt;
&lt;hr class="docutils" /&gt;
&lt;p&gt;Historically, Dask tried hard to get each individual task done as fast as possible: if a task &lt;em&gt;could&lt;/em&gt; run, it &lt;em&gt;would&lt;/em&gt;. So sometimes, tasks would run even if their outputs weren’t going to be used immediately—leaving them sitting around in memory.&lt;/p&gt;
&lt;p&gt;If you had thousands of initial tasks loading data—say, fetching Parquet from S3, CSVs from disk, or rows from a database—all those tasks would be scheduled and sent to workers up front.&lt;/p&gt;
&lt;p&gt;The workers would churn through them, fetching chunks of data (and accumulating it in memory) as quickly as possible. A worker would tell the scheduler when each chunk was loaded, and the scheduler would respond with what to do with it next—but until that message arrived, there were more data-loading tasks runnable right now, so why not run them?&lt;/p&gt;
&lt;img alt="root task overproduction" src="/images/2022-queuing/overproduction.png" style="max-width: 100%;" width="100%" /&gt;
&lt;p&gt;This slight gap in timing—between the worker immediately starting on a less-useful task, then only later finding out about a more-useful task it should have run instead—allowed this lower-priority data to pile up in memory. We call this “root task overproduction”.&lt;/p&gt;
&lt;p&gt;Overall, there could be at least twice as much initial data in memory at once as necessary—and, therefore, twice as many intermediate results. (See &lt;a class="reference external" href="https://github.com/dask/distributed/pull/6614#discussion_r956515223"&gt;this comment&lt;/a&gt; for a detailed explanation of why the 2x happens.)&lt;/p&gt;
&lt;p&gt;When this put workers under memory pressure, this initial problem would snowball into a bigger one. Workers had to spill data to disk (slow), then read it back from disk to use it or transfer it (slow). Workers might exceed their memory limits and crash, losing progress and requiring tasks to be recomputed on a pool of workers that were already struggling.&lt;/p&gt;
&lt;p&gt;In the end, this meant that a whole class of workloads were slow, or even un-runnable, without using hugely oversized clusters.&lt;/p&gt;
&lt;hr class="docutils" /&gt;
&lt;p&gt;There were plenty of ways to approach this problem, but we wanted to try the simplest thing first: just don’t tell the workers about more tasks than they can run at once.&lt;/p&gt;
&lt;p&gt;We’re calling this mode of scheduling &lt;a class="reference external" href="https://distributed.dask.org/en/stable/scheduling-policies.html#queuing"&gt;“queuing”&lt;/a&gt;, or “root task withholding”. The scheduler puts data-loading tasks in an internal queue, and only drips one out to a worker once it’s finished its current work &lt;em&gt;and&lt;/em&gt; there’s nothing more useful to run instead that utilizes the work it just completed.&lt;/p&gt;
&lt;!-- We let the scheduler choose to do nothing, even when there was something it could do. --&gt;
&lt;img alt="screenshot showing non-queuing dashboard on left, with all root tasks in processing, vs queueing dashboard on the right, with the hash-marked progress bar indicating tasks are queued on the scheduler, instead of workers" src="/images/2022-queuing/dashboard.png" style="max-width: 100%;" width="100%" /&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/15/queuing.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 id="slow-is-smooth"&gt;

&lt;p&gt;Queuing adds a cost in latency. Every time workers finish a task, they have to ask the scheduler what to do next and sit under-utilized, or even idle, until they get an answer. (Before, they had a backlog of things they could do before the answer came back.)&lt;/p&gt;
&lt;p&gt;For a while, we hadn’t considered this approach, because intuitively, we assumed the latency would be too much of a slow-down.&lt;/p&gt;
&lt;p&gt;However, by slowing down the pace of task assignment, and running only the best tasks, scheduling gets much smoother. And with that smoothness, we see that most benchmark workloads use much less memory across the board:&lt;/p&gt;
&lt;!-- &lt;figure&gt;
    &lt;img alt="percent memory change from baseline. only long blue bars to the left" src="/images/2022-queuing/memory-benchmarks.png" style="max-width: 100%;" width="100%" /&gt;
    &lt;figcaption&gt;Percent decrease in peak memory use in the latest release. Notice the axes: up to 80% reduction.&lt;/figcaption&gt;
&lt;/figure&gt; --&gt;
&lt;hr class="docutils" /&gt;
&lt;img alt="percent memory change from baseline. only long blue bars to the left" src="/images/2022-queuing/memory-benchmarks.png" style="max-width: 100%;" width="100%" /&gt;
&lt;p&gt;&lt;em&gt;Percent decrease in peak memory used in the latest release. Notice the axes: up to 80% reduction.&lt;/em&gt;&lt;/p&gt;
&lt;hr class="docutils" /&gt;
&lt;p&gt;This large a reduction in memory use is a big deal!&lt;/p&gt;
&lt;p&gt;For many users, this might mean that workloads that weren’t runnable before can now run smoothly.&lt;/p&gt;
&lt;!-- &lt;figure&gt;
    &lt;img alt="Julius Busecke said quote by setting worker saturation, I can reliably calculate a trend over time for the first time endquote" src="/images/2022-queuing/julius-quote-2.png" style="max-width: 100%;" width="100%" /&gt;
    &lt;figcaption&gt;Julius Busecke [reports](https://github.com/dask/distributed/discussions/7128#discussioncomment-3964014) that a common geoscience task which used to always crash now works out of the box with the new scheduling mode.&lt;/figcaption&gt;
&lt;/figure&gt; --&gt;
&lt;hr class="docutils" /&gt;
&lt;img alt="Julius Busecke said quote by setting worker saturation, I can reliably calculate a trend over time for the first time endquote" src="/images/2022-queuing/julius-quote-2.png" style="max-width: 100%;" width="100%" /&gt;
&lt;p&gt;&lt;em&gt;Julius Busecke &lt;a class="reference external" href="https://github.com/dask/distributed/discussions/7128#discussioncomment-3964014"&gt;reports&lt;/a&gt; that a common geoscience task which used to always crash now works out of the box with the new scheduling mode.&lt;/em&gt;&lt;/p&gt;
&lt;hr class="docutils" /&gt;
&lt;p&gt;Some workloads will also be cheaper to run in the cloud, since they can use instances with less memory. We see some benchmarks that could, in theory, be run for 30-50% less total cost. This is not universal: others would cost &lt;em&gt;more&lt;/em&gt; because they get slower. More on that later.&lt;/p&gt;
&lt;p&gt;Beyond this, execution is just more predictable. Memory usage is much more consistent and less likely to spike rapidly.&lt;/p&gt;
&lt;img alt="constant vs peak-y memory usage: anom_mean, dataframe_align, double_diff, vorticity" src="/images/2022-queuing/constant-memory.png" style="max-width: 100%;" width="100%" /&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/15/queuing.md&lt;/span&gt;, line 101)&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="smooth-is-fast"&gt;
&lt;h1&gt;Smooth is fast&lt;/h1&gt;
&lt;p&gt;In a few cases, it turns out that smooth scheduling can be even faster.&lt;/p&gt;
&lt;p&gt;On average, one representative oceanography workload ran 20% faster. A few other workloads showed modest speedups as well. This is mostly because they no longer load too much data into memory and then have to spill it to disk, which creates significant slowdowns.&lt;/p&gt;
&lt;p&gt;Additionally, we found that the extra latency we were worried about didn’t actually slow things down in typical cases. There was no measurable change in pure task throughput on a cluster with fast networking and multi-CPU workers, like &lt;a class="reference external" href="https://coiled.io"&gt;Coiled clusters&lt;/a&gt; or &lt;a class="reference external" href="https://docs.dask.org/en/latest/deploying-python.html"&gt;a single-machine &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LocalCluster&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;. This was a good lesson in trying the simplest thing first.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/15/queuing.md&lt;/span&gt;, line 109)&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="sometimes-slow-is-still-slow-but-not-why-you-d-expect"&gt;
&lt;h1&gt;Sometimes, slow is still slow (but not why you’d expect)&lt;/h1&gt;
&lt;p&gt;However, we did notice that a few benchmarks run slower with scheduler-side queuing. The typical slowdown is 5-10%, but in the worst case, they are ~50% slower (though they also use about half the memory).&lt;/p&gt;
&lt;img alt="Memory profiles of slow workloads, showing increased runtime but decreased memory (`anom_mean`, `basic_sum`)" src="/images/2022-queuing/slow-memory-profiles.png" style="max-width: 100%;" width="100%" /&gt;
&lt;p&gt;The problem is that implementing queuing meant giving up a scheduling feature &lt;a class="reference external" href="https://github.com/dask/distributed/pull/4967"&gt;introduced last year&lt;/a&gt; called &lt;em&gt;co-assignment&lt;/em&gt;.&lt;/p&gt;
&lt;p&gt;As described &lt;a class="reference external" href="https://distributed.dask.org/en/stable/scheduling-policies.html#initial-task-placement"&gt;in the docs&lt;/a&gt;, co-assignment tries to schedule initial tasks on the same worker if their outputs will be combined later. This avoids having to transfer data from one worker to another when the downstream task runs, because all the data is already on one worker.&lt;/p&gt;
&lt;p&gt;In a graph like this, we’d want &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;a&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;b&lt;/span&gt;&lt;/code&gt; to run on the same worker. Otherwise, one of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;e&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f&lt;/span&gt;&lt;/code&gt; would have to be transferred between workers before &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;i&lt;/span&gt;&lt;/code&gt; can run.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;  &lt;span class="n"&gt;i&lt;/span&gt;       &lt;span class="n"&gt;j&lt;/span&gt;
 &lt;span class="o"&gt;/&lt;/span&gt; \     &lt;span class="o"&gt;/&lt;/span&gt; \
&lt;span class="n"&gt;e&lt;/span&gt;   &lt;span class="n"&gt;f&lt;/span&gt;   &lt;span class="n"&gt;g&lt;/span&gt;   &lt;span class="n"&gt;h&lt;/span&gt;
&lt;span class="o"&gt;|&lt;/span&gt;   &lt;span class="o"&gt;|&lt;/span&gt;   &lt;span class="o"&gt;|&lt;/span&gt;   &lt;span class="o"&gt;|&lt;/span&gt;
&lt;span class="n"&gt;a&lt;/span&gt;   &lt;span class="n"&gt;b&lt;/span&gt;   &lt;span class="n"&gt;c&lt;/span&gt;   &lt;span class="n"&gt;d&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Avoiding these transfers speeds things up, because network is &lt;a class="reference external" href="https://medium.com/&amp;#64;hondanhon/more-latency-numbers-every-programmer-should-know-3142f0cf614d"&gt;relatively slow&lt;/a&gt;. It also reduces memory usage by avoiding having to hold replicas of the same data on multiple workers.&lt;/p&gt;
&lt;p&gt;Unfortunately, the current implementation of co-assignment isn’t compatible with queuing, and updating it is non-trivial. We plan to do this next, to get the best of both worlds.&lt;/p&gt;
&lt;p&gt;But in the short term, we had to decide if queuing was beneficial enough to be worth enabling by default right away, despite the loss of co-assignment.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/15/queuing.md&lt;/span&gt;, line 135)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="a-new-default-for-scheduling"&gt;
&lt;h1&gt;A new default for scheduling&lt;/h1&gt;
&lt;p&gt;&lt;a class="reference external" href="https://observablehq.com/&amp;#64;gjoseph92/snakebench?commits=2d37536&amp;amp;amp;commits=f6ef40b&amp;amp;amp;commits=cfe91dd&amp;amp;amp;measure=peak_memory&amp;amp;amp;groupby=branch&amp;amp;amp;show=passed"&gt;After running a number of benchmarks&lt;/a&gt;, and getting some &lt;a class="reference external" href="https://github.com/dask/distributed/discussions/7128"&gt;initial community feedback&lt;/a&gt;, we think it is.&lt;/p&gt;
&lt;p&gt;Queuing makes things possible that used to not work at all. But it doesn’t break anything that works today: everything will still work, some things just could be slower. We feel that that’s a worthwhile enough tradeoff to enable it by default—especially given how much dask users have struggled with memory issues.&lt;/p&gt;
&lt;p&gt;Additionally, to avoid impacting workloads that could become latency-bound, the new algorithm is still doing a &lt;em&gt;little&lt;/em&gt; overproduction. It’s pushing a handful of extra root tasks to the worker in advance (as opposed to all of them, like before). This comes at the price of some extra memory use, but prevents painful slowdowns in high-latency clusters.&lt;/p&gt;
&lt;p&gt;So in the latest release, queuing is enabled by default. Most memory-intensive Array and DataFrame workloads should see reductions in memory use out of the box, ranging from noticeable to 🤩.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/15/queuing.md&lt;/span&gt;, line 145)&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="let-us-know-how-it-goes"&gt;
&lt;h1&gt;Let us know how it goes&lt;/h1&gt;
&lt;p&gt;We’ve opened a &lt;a class="reference external" href="https://github.com/dask/distributed/discussions/7128"&gt;discussion on GitHub&lt;/a&gt; for feedback on this change. Please let us know how it helps (or doesn’t).&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/15/queuing.md&lt;/span&gt;, line 149)&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="keeping-old-behavior"&gt;
&lt;h1&gt;Keeping old behavior&lt;/h1&gt;
&lt;p&gt;For users who are sensitive to runtime and have low memory use, you can deactivate queuing and use the old scheduling mode (including co-assignment) via &lt;a class="reference external" href="https://docs.dask.org/en/stable/configuration.html"&gt;Dask configuration&lt;/a&gt;, by setting the new &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed.scheduler.worker-saturation&lt;/span&gt;&lt;/code&gt; config value to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;inf&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;You can read more about adjusting this setting &lt;a class="reference external" href="https://distributed.dask.org/en/latest/scheduling-policies.html#adjusting-or-disabling-queuing"&gt;in the docs&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;On Coiled, you can set it with:&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;dask&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;coiled&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;distributed.scheduler.worker-saturation&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;inf&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;}):&lt;/span&gt;
    &lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;coiled&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Cluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# coiled sends current dask config automatically&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You can see examples of setting this configuration for various deployment systems on the &lt;a class="reference external" href="https://github.com/dask/distributed/discussions/7128"&gt;discussion issue&lt;/a&gt; (when copy-pasting, be sure to change the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;1.0&lt;/span&gt;&lt;/code&gt; to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;inf&lt;/span&gt;&lt;/code&gt;!). And if you find the need to set &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;worker-saturation&lt;/span&gt;&lt;/code&gt; back to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;inf&lt;/span&gt;&lt;/code&gt;, please let us know on the discussion.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2022/11/15/queuing/"/>
    <summary>Original version of this post appears on https://www.coiled.io/blog/reducing-dask-memory-usage</summary>
    <published>2022-11-15T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2022/11/09/dask-kubernetes-operator/</id>
    <title>Dask Kubernetes Operator</title>
    <updated>2022-11-09T00:00:00+00:00</updated>
    <author>
      <name>Jacob Tomlinson (NVIDIA)</name>
    </author>
    <content type="html">&lt;p&gt;We are excited to announce that the &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator.html"&gt;Dask Kubernetes Operator&lt;/a&gt; is now generally available 🎉!&lt;/p&gt;
&lt;p&gt;Notable new features include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Dask Clusters are now &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html"&gt;native custom resources&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Clusters can be managed with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;kubectl&lt;/span&gt;&lt;/code&gt; or the &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_kubecluster.html"&gt;Python API&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Cascaded deletions allow for proper teardown&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Multiple &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html#daskworkergroup"&gt;worker groups&lt;/a&gt; enable heterogenous/tagged deployments&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html#daskjob"&gt;DaskJob&lt;/a&gt;: running dask workloads with K8s batched job infrastructure&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Clusters can be reused between different Python processes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html#daskautoscaler"&gt;Autoscaling&lt;/a&gt; is handled by a custom Kubernetes controller instead of the user code&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scheduler and worker Pods and Services are &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html#daskcluster"&gt;fully configurable&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;div class="highlight-console notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;$ &lt;/span&gt;kubectl&lt;span class="w"&gt; &lt;/span&gt;get&lt;span class="w"&gt; &lt;/span&gt;daskcluster
&lt;span class="go"&gt;NAME         AGE&lt;/span&gt;
&lt;span class="go"&gt;my-cluster   4m3s&lt;/span&gt;

&lt;span class="gp"&gt;$ &lt;/span&gt;kubectl&lt;span class="w"&gt; &lt;/span&gt;get&lt;span class="w"&gt; &lt;/span&gt;all&lt;span class="w"&gt; &lt;/span&gt;-A&lt;span class="w"&gt; &lt;/span&gt;-l&lt;span class="w"&gt; &lt;/span&gt;dask.org/cluster-name&lt;span class="o"&gt;=&lt;/span&gt;my-cluster
&lt;span class="go"&gt;NAMESPACE   NAME                                       READY   STATUS    RESTARTS   AGE&lt;/span&gt;
&lt;span class="go"&gt;default     pod/my-cluster-default-worker-22bd39e33a   1/1     Running   0          3m43s&lt;/span&gt;
&lt;span class="go"&gt;default     pod/my-cluster-default-worker-5f4f2c989a   1/1     Running   0          3m43s&lt;/span&gt;
&lt;span class="go"&gt;default     pod/my-cluster-default-worker-72418a589f   1/1     Running   0          3m43s&lt;/span&gt;
&lt;span class="go"&gt;default     pod/my-cluster-default-worker-9b00a4e1fd   1/1     Running   0          3m43s&lt;/span&gt;
&lt;span class="go"&gt;default     pod/my-cluster-default-worker-d6fc172526   1/1     Running   0          3m43s&lt;/span&gt;
&lt;span class="go"&gt;default     pod/my-cluster-scheduler                   1/1     Running   0          4m21s&lt;/span&gt;

&lt;span class="go"&gt;NAMESPACE   NAME                           TYPE        CLUSTER-IP    EXTERNAL-IP   PORT(S)             AGE&lt;/span&gt;
&lt;span class="go"&gt;default     service/my-cluster-scheduler   ClusterIP   10.96.33.67   &amp;lt;none&amp;gt;        8786/TCP,8787/TCP   4m21s&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;At the start of 2022 we began the large undertaking of rewriting the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-kubernetes&lt;/span&gt;&lt;/code&gt; package in the &lt;a class="reference external" href="https://kubernetes.io/docs/concepts/extend-kubernetes/operator/"&gt;operator pattern&lt;/a&gt;. This design pattern has become very popular in the Kubernetes community with companies like &lt;a class="reference external" href="https://www.redhat.com/en/technologies/cloud-computing/openshift/what-are-openshift-operators"&gt;Red Hat building their whole Kubernetes offering Openshift around it&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/09/dask-kubernetes-operator.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 id="what-is-an-operator"&gt;

&lt;p&gt;If you’ve spent any time in the Kubernetes community you’ll have heard the term operator being thrown around seen projects like &lt;a class="reference external" href="https://github.com/operator-framework"&gt;Golang’s Operator Framework&lt;/a&gt; being used to deploy modern applications.&lt;/p&gt;
&lt;p&gt;At it’s core an operator is made up of a data structure for describing the thing you want to deploy (in our case a Dask cluster) and a controller which does the actual deploying. In Kubernetes the templates for these data structures are called &lt;a class="reference external" href="https://kubernetes.io/docs/concepts/extend-kubernetes/api-extension/custom-resources/"&gt;Custom Resource Definitions&lt;/a&gt; (CRDs) and allow you to extend the Kubernetes API with new resource types of your own design.&lt;/p&gt;
&lt;p&gt;For &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-kubernetes&lt;/span&gt;&lt;/code&gt; we have created a few CRDs to describe things like &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html#daskcluster"&gt;Dask clusters&lt;/a&gt;, groups of &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html#daskworkergroup"&gt;Dask workers&lt;/a&gt;, &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html#daskjob"&gt;adaptive autoscalers&lt;/a&gt; and a new &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html#daskautoscaler"&gt;Dask powered batch job&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;We also built a controller using &lt;a class="reference external" href="https://kopf.readthedocs.io/en/stable/"&gt;kopf&lt;/a&gt; that handles watching for changes to any of these resources and creates/updates/deletes lower level Kubernetes resources like Pods and Services.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/09/dask-kubernetes-operator.md&lt;/span&gt;, line 52)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="why-did-we-build-this"&gt;
&lt;h1&gt;Why did we build this?&lt;/h1&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/history.html"&gt;original implementation&lt;/a&gt; of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-kubernetes&lt;/span&gt;&lt;/code&gt; was started shortly after Kubernetes went &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;1.0&lt;/span&gt;&lt;/code&gt; and before any established design patterns had emerged. Its model was based on spawning Dask workers as subprocesses, except those subprocesses are Pods running in Kubernetes. This is the same way &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-jobqueue&lt;/span&gt;&lt;/code&gt; launches workers as individual job scheduler allocations or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-ssh&lt;/span&gt;&lt;/code&gt; opens many SSH connections to various machines.&lt;/p&gt;
&lt;p&gt;Over time this has been refactored, rewritten and extended multiple times. One long-asked-for change was to also place the Dask scheduler inside the Kubernetes cluster to simplify scheduler-worker communication and network connectivity. Naturally this lead to more feature requests around configuring the scheduler service and having more control over the cluster. As we extended more and more the original premise of spawning worker subprocesses on a remote system became less helpful.&lt;/p&gt;
&lt;p&gt;The final straw in the original design was folks asking for the ability to leave a cluster running and come back to it later. Either to reuse a cluster between separate jobs, or just different stages in a multi-stage pipeline. The premise of spawning subprocesses leads to an assumption that the parent process will be around for the lifetime of the cluster which makes it a reasonable place to hold state such as the template for launching new workers when scaling up. We attempted to implement this feature but it just wasn’t possible with the current design. Moving to a model where the parent process can die and new processes can pick up means that state needs to be moved elsewhere and things were too entangled to successfully pull this out.&lt;/p&gt;
&lt;p&gt;The classic implementation that had served us well for so long was creaking and becoming increasingly difficult to modify and maintain. The time had come to pay down our technical debt by rebuilding from scratch under a new model, the operator pattern.&lt;/p&gt;
&lt;p&gt;In this new model a Dask cluster is an abstract object that exists within a Kubernetes cluster. We use custom resources to store the state for each cluster and a custom controller to map that state onto reality by creating the individual components that make up the cluster. Want to scale up your cluster? Instead of having some Python code locally that spawns a new Pod on Kubernetes we just modify the state of the Dask cluster resource to specify the desired number of workers and the controller handles adding/removing Pods to match.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/09/dask-kubernetes-operator.md&lt;/span&gt;, line 64)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="new-features"&gt;
&lt;h1&gt;New features&lt;/h1&gt;
&lt;p&gt;While our primary goal was allowing cluster reuse between Python processes and paying down technical debt switching to the operator pattern has allowed us to add a bunch of nice new features. So let’s explore those.&lt;/p&gt;
&lt;section id="python-or-yaml-api"&gt;
&lt;h2&gt;Python or YAML API&lt;/h2&gt;
&lt;p&gt;With our new implementation we create Dask clusters by creating a &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html#daskcluster"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskCluster&lt;/span&gt;&lt;/code&gt; resource&lt;/a&gt; on our Kubernetes cluster. The controller sees this appear and spawns child resources for the scheduler, workers, etc.&lt;/p&gt;
&lt;img alt="Diagram of a DaskCluster resource and its child resources" src="/images/2022-kubernetes/daskcluster.png" style="max-width: 100%;" width="100%" /&gt;
&lt;p&gt;We modify our cluster by editing the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskCluster&lt;/span&gt;&lt;/code&gt; resource and our controller reacts to those changes and updates the child resources accordingly.&lt;/p&gt;
&lt;p&gt;We delete our cluster by deleting the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskCluster&lt;/span&gt;&lt;/code&gt; resource and Kubernetes handles the rest (see the next section on cascade deletion).&lt;/p&gt;
&lt;p&gt;By storing all of our state in the resource and all of our logic in the controller this means the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;KubeCluster&lt;/span&gt;&lt;/code&gt; class is now much simpler. It’s actually so simple that it is entirely optional.&lt;/p&gt;
&lt;p&gt;The primary purpose of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;KubeCluster&lt;/span&gt;&lt;/code&gt; class now is to provide a nice clean API for creating/scaling/deleting your clusters in Python. It can take a small number of keyword arguments and generate all of the YAML to submit to Kubernetes.&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_kubernetes.operator&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;KubeCluster&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;KubeCluster&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="s2"&gt;&amp;quot;my-cluster&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n_workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;env&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;FOO&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;bar&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;})&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The above snippet creates the following resource.&lt;/p&gt;
&lt;div class="highlight-yaml notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nt"&gt;apiVersion&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;kubernetes.dask.org/v1&lt;/span&gt;
&lt;span class="nt"&gt;kind&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;DaskCluster&lt;/span&gt;
&lt;span class="nt"&gt;metadata&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;name&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;my-cluster&lt;/span&gt;
&lt;span class="nt"&gt;spec&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;scheduler&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;service&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="nt"&gt;ports&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;        &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nt"&gt;name&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;tcp-comm&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;port&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;8786&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;protocol&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;TCP&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;targetPort&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;tcp-comm&lt;/span&gt;
&lt;span class="w"&gt;        &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nt"&gt;name&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;http-dashboard&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;port&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;8787&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;protocol&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;TCP&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;targetPort&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;http-dashboard&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="nt"&gt;selector&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;        &lt;/span&gt;&lt;span class="nt"&gt;dask.org/cluster-name&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;my-cluster&lt;/span&gt;
&lt;span class="w"&gt;        &lt;/span&gt;&lt;span class="nt"&gt;dask.org/component&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;scheduler&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="nt"&gt;type&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;ClusterIP&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;spec&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="nt"&gt;containers&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;        &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nt"&gt;args&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;dask-scheduler&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;--host&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;0.0.0.0&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;env&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nt"&gt;name&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;FOO&lt;/span&gt;
&lt;span class="w"&gt;              &lt;/span&gt;&lt;span class="nt"&gt;value&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;bar&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;image&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;ghcr.io/dask/dask:latest&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;livenessProbe&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="nt"&gt;httpGet&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;              &lt;/span&gt;&lt;span class="nt"&gt;path&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;/health&lt;/span&gt;
&lt;span class="w"&gt;              &lt;/span&gt;&lt;span class="nt"&gt;port&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;http-dashboard&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="nt"&gt;initialDelaySeconds&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;15&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="nt"&gt;periodSeconds&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;20&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;name&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;scheduler&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;ports&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nt"&gt;containerPort&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;8786&lt;/span&gt;
&lt;span class="w"&gt;              &lt;/span&gt;&lt;span class="nt"&gt;name&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;tcp-comm&lt;/span&gt;
&lt;span class="w"&gt;              &lt;/span&gt;&lt;span class="nt"&gt;protocol&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;TCP&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nt"&gt;containerPort&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;8787&lt;/span&gt;
&lt;span class="w"&gt;              &lt;/span&gt;&lt;span class="nt"&gt;name&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;http-dashboard&lt;/span&gt;
&lt;span class="w"&gt;              &lt;/span&gt;&lt;span class="nt"&gt;protocol&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;TCP&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;readinessProbe&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="nt"&gt;httpGet&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;              &lt;/span&gt;&lt;span class="nt"&gt;path&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;/health&lt;/span&gt;
&lt;span class="w"&gt;              &lt;/span&gt;&lt;span class="nt"&gt;port&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;http-dashboard&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="nt"&gt;initialDelaySeconds&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;5&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="nt"&gt;periodSeconds&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;10&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;resources&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;null&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;worker&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;my-cluster&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;replicas&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;3&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;spec&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="nt"&gt;containers&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;        &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nt"&gt;args&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;dask-worker&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;--name&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;$(DASK_WORKER_NAME)&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;env&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;            &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nt"&gt;name&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;FOO&lt;/span&gt;
&lt;span class="w"&gt;              &lt;/span&gt;&lt;span class="nt"&gt;value&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;bar&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;image&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;ghcr.io/dask/dask:latest&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;name&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;worker&lt;/span&gt;
&lt;span class="w"&gt;          &lt;/span&gt;&lt;span class="nt"&gt;resources&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;null&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If I want to scale up my workers to 5 I can do this 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="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scale&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;All this does is apply a patch to the resource and modify the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;spec.worker.replicas&lt;/span&gt;&lt;/code&gt; value to be &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;5&lt;/span&gt;&lt;/code&gt; and the controller handles the rest.&lt;/p&gt;
&lt;p&gt;Ultimately our Python API is generating YAML and handing it to Kubernetes to action. Everything about our cluster is contained in that YAML. If we prefer we can write and store this YAML ourselves and manage our cluster entirely via &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;kubectl&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;If we put the above YAML example into a file called &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;my-cluster.yaml&lt;/span&gt;&lt;/code&gt; we can create it like this. No Python necessary.&lt;/p&gt;
&lt;div class="highlight-console notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;$ &lt;/span&gt;kubectl&lt;span class="w"&gt; &lt;/span&gt;apply&lt;span class="w"&gt; &lt;/span&gt;-f&lt;span class="w"&gt; &lt;/span&gt;my-cluster.yaml
&lt;span class="go"&gt;daskcluster.kubernetes.dask.org/my-cluster created&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We can also scale our cluster with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;kubectl&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;div class="highlight-console notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;$ &lt;/span&gt;kubectl&lt;span class="w"&gt; &lt;/span&gt;scale&lt;span class="w"&gt; &lt;/span&gt;--replicas&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="m"&gt;5&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;daskworkergroup&lt;span class="w"&gt; &lt;/span&gt;my-cluster-default
&lt;span class="go"&gt;daskworkergroup.kubernetes.dask.org/my-cluster-default&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is extremely powerful for advanced users who want to integrate with existing Kubernetes tooling and really modify everything about their Dask cluster.&lt;/p&gt;
&lt;p&gt;You can still construct a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;KubeCluster&lt;/span&gt;&lt;/code&gt; object in the future and point it to this existing cluster for convenience.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_kubernetes.operator&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;KubeCluster&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;KubeCluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_name&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;my-cluster&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scale&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="cascade-deletion"&gt;
&lt;h2&gt;Cascade deletion&lt;/h2&gt;
&lt;p&gt;Having a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskCluster&lt;/span&gt;&lt;/code&gt; resource also makes deletion much more pleasant.&lt;/p&gt;
&lt;p&gt;In the old implementation your local Python process would spawn a bunch of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Pod&lt;/span&gt;&lt;/code&gt; resources along with supporting ones like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Service&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;PodDisruptionBudget&lt;/span&gt;&lt;/code&gt; resources. It also had some teardown functionality that was either called directly or via a finalizer that deleted all of these resources when you are done.&lt;/p&gt;
&lt;p&gt;One downside of this was that if something went wrong either due to a bug in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-kubernetes&lt;/span&gt;&lt;/code&gt; or a more severe failure that caused the Python process to exit without calling finalizers you would be left with a ton of resources that you had to clean up manually. I expect some folks have a label based selector command stored in their snippet manager somewhere but most folks would do this cleanup manually.&lt;/p&gt;
&lt;p&gt;With the new model the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskCluster&lt;/span&gt;&lt;/code&gt; resource is set as the &lt;a class="reference external" href="https://kubernetes.io/docs/concepts/overview/working-with-objects/owners-dependents/"&gt;owner&lt;/a&gt; of all of the other resources spawned by the controller. This means we can take advantage of &lt;a class="reference external" href="https://kubernetes.io/docs/tasks/administer-cluster/use-cascading-deletion/"&gt;cascade deletion&lt;/a&gt; for our cleanup. Regardless of how you create your cluster or whether the initial Python process still exists you can just delete the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskCluster&lt;/span&gt;&lt;/code&gt; resource and Kubernetes will know to automatically delete all of its children.&lt;/p&gt;
&lt;div class="highlight-console notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;$ &lt;/span&gt;kubectl&lt;span class="w"&gt; &lt;/span&gt;get&lt;span class="w"&gt; &lt;/span&gt;daskcluster&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="c1"&gt;# Here we see our Dask cluster resource&lt;/span&gt;
&lt;span class="go"&gt;NAME         AGE&lt;/span&gt;
&lt;span class="go"&gt;my-cluster   4m3s&lt;/span&gt;

&lt;span class="gp"&gt;$ &lt;/span&gt;kubectl&lt;span class="w"&gt; &lt;/span&gt;get&lt;span class="w"&gt; &lt;/span&gt;all&lt;span class="w"&gt; &lt;/span&gt;-A&lt;span class="w"&gt; &lt;/span&gt;-l&lt;span class="w"&gt; &lt;/span&gt;dask.org/cluster-name&lt;span class="o"&gt;=&lt;/span&gt;my-cluster&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="c1"&gt;# and all of its child resources&lt;/span&gt;
&lt;span class="go"&gt;NAMESPACE   NAME                                       READY   STATUS    RESTARTS   AGE&lt;/span&gt;
&lt;span class="go"&gt;default     pod/my-cluster-default-worker-22bd39e33a   1/1     Running   0          3m43s&lt;/span&gt;
&lt;span class="go"&gt;default     pod/my-cluster-default-worker-5f4f2c989a   1/1     Running   0          3m43s&lt;/span&gt;
&lt;span class="go"&gt;default     pod/my-cluster-default-worker-72418a589f   1/1     Running   0          3m43s&lt;/span&gt;
&lt;span class="go"&gt;default     pod/my-cluster-default-worker-9b00a4e1fd   1/1     Running   0          3m43s&lt;/span&gt;
&lt;span class="go"&gt;default     pod/my-cluster-default-worker-d6fc172526   1/1     Running   0          3m43s&lt;/span&gt;
&lt;span class="go"&gt;default     pod/my-cluster-scheduler                   1/1     Running   0          4m21s&lt;/span&gt;

&lt;span class="go"&gt;NAMESPACE   NAME                           TYPE        CLUSTER-IP    EXTERNAL-IP   PORT(S)             AGE&lt;/span&gt;
&lt;span class="go"&gt;default     service/my-cluster-scheduler   ClusterIP   10.96.33.67   &amp;lt;none&amp;gt;        8786/TCP,8787/TCP   4m21s&lt;/span&gt;

&lt;span class="gp"&gt;$ &lt;/span&gt;kubectl&lt;span class="w"&gt; &lt;/span&gt;delete&lt;span class="w"&gt; &lt;/span&gt;daskcluster&lt;span class="w"&gt; &lt;/span&gt;my-cluster&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="c1"&gt;# We can delete the daskcluster resource&lt;/span&gt;
&lt;span class="go"&gt;daskcluster.kubernetes.dask.org &amp;quot;my-cluster&amp;quot; deleted&lt;/span&gt;

&lt;span class="gp"&gt;$ &lt;/span&gt;kubectl&lt;span class="w"&gt; &lt;/span&gt;get&lt;span class="w"&gt; &lt;/span&gt;all&lt;span class="w"&gt; &lt;/span&gt;-A&lt;span class="w"&gt; &lt;/span&gt;-l&lt;span class="w"&gt; &lt;/span&gt;dask.org/cluster-name&lt;span class="o"&gt;=&lt;/span&gt;my-cluster&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="c1"&gt;# all of the children are removed&lt;/span&gt;
&lt;span class="go"&gt;No resources found&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="multiple-worker-groups"&gt;
&lt;h2&gt;Multiple worker groups&lt;/h2&gt;
&lt;p&gt;We also took this opportunity to add support for &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html#daskworkergroup"&gt;multiple worker groups&lt;/a&gt; as a first class principle. Some workflows benefit from having a few workers in your cluster with some additional resources. This may be a couple of workers with much higher memory than the rest, or GPUs for accelerated compute. Using &lt;a class="reference external" href="https://distributed.dask.org/en/stable/resources.html"&gt;resource annotations&lt;/a&gt; you can steer certain tasks to those workers, so if you have a single step that creates a large amount of intermediate memory you can ensure that task ends up on a worker with enough memory.&lt;/p&gt;
&lt;p&gt;By default when you create a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskCluster&lt;/span&gt;&lt;/code&gt; resource it creates a single &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskWorkerGroup&lt;/span&gt;&lt;/code&gt; which in turn creates the worker &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Pod&lt;/span&gt;&lt;/code&gt; resources for our cluster. If you wish you can add more worker group resources yourself with different resource configurations.&lt;/p&gt;
&lt;img alt="Diagram of a DaskWorkerGroup resource and its child resources" src="/images/2022-kubernetes/daskworkergroup.png" style="max-width: 100%;" width="100%" /&gt;
&lt;p&gt;Here is an example of creating a cluster with five workers that have 16GB of memory and two additional workers with 64GB 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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_kubernetes.operator&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;KubeCluster&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;KubeCluster&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;foo&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                      &lt;span class="n"&gt;n_workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                      &lt;span class="n"&gt;resources&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;
                          &lt;span class="s2"&gt;&amp;quot;requests&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;memory&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;16Gi&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt;
                          &lt;span class="s2"&gt;&amp;quot;limits&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;memory&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;16Gi&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;
                      &lt;span class="p"&gt;})&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;add_worker_group&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="s2"&gt;&amp;quot;highmem&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                         &lt;span class="n"&gt;n_workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                         &lt;span class="n"&gt;resources&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;
                             &lt;span class="s2"&gt;&amp;quot;requests&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;memory&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;64Gi&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt;
                             &lt;span class="s2"&gt;&amp;quot;limits&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;memory&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;64Gi&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;
                         &lt;span class="p"&gt;})&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="autoscaling"&gt;
&lt;h2&gt;Autoscaling&lt;/h2&gt;
&lt;p&gt;One of the much loved features of the classic implementation of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;KubeCluster&lt;/span&gt;&lt;/code&gt; was adaptive autoscaling. When enabled the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;KubeCluster&lt;/span&gt;&lt;/code&gt; object would regularly communicate with the scheduler and ask if it wanted to change the number of workers and then add/remove pods accordingly.&lt;/p&gt;
&lt;p&gt;In the new implementation this logic has moved to the controller so the cluster can autoscale even when there is no &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;KubeCluster&lt;/span&gt;&lt;/code&gt; object in existence.&lt;/p&gt;
&lt;img alt="Diagram of a DaskAutoscaler resource and how it interacts with other resources" src="/images/2022-kubernetes/daskautoscaler.png" style="max-width: 100%;" width="100%" /&gt;
&lt;p&gt;The Python API remains the same so you can still use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;KubeCluster&lt;/span&gt;&lt;/code&gt; to put your cluster into adaptive mode.&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_kubernetes.operator&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;KubeCluster&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;KubeCluster&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="s2"&gt;&amp;quot;my-cluster&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n_workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;adapt&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;minimum&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;maximum&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;100&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This call creates a &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html#daskautoscaler"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskAutoscaler&lt;/span&gt;&lt;/code&gt; resource&lt;/a&gt; which the controller sees and periodically takes action on by asking the scheduler how many workers it wants and updating the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskWorkerGroup&lt;/span&gt;&lt;/code&gt; within the configured bounds.&lt;/p&gt;
&lt;div class="highlight-yaml notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nt"&gt;apiVersion&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;kubernetes.dask.org/v1&lt;/span&gt;
&lt;span class="nt"&gt;kind&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;DaskAutoscaler&lt;/span&gt;
&lt;span class="nt"&gt;metadata&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;name&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;my-cluster&lt;/span&gt;
&lt;span class="nt"&gt;spec&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;my-cluster&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;minimum&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;1&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;maximum&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;100&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Calling &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cluster.scale(5)&lt;/span&gt;&lt;/code&gt; will also delete this resource and set the number of workers back to 5.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="daskjob"&gt;
&lt;h2&gt;DaskJob&lt;/h2&gt;
&lt;p&gt;Having composable cluster resources also allows us to put together a &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_resources.html#daskjob"&gt;new &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskJob&lt;/span&gt;&lt;/code&gt; resource&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Kubernetes has some built-in &lt;a class="reference external" href="https://kubernetes.io/docs/concepts/workloads/controllers/job/"&gt;batch job style resources&lt;/a&gt; which ensure a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Pod&lt;/span&gt;&lt;/code&gt; is run to completion one or more times. You can control how many times is should run and how many concurrent pods there should be. This is useful for fire-and-forget jobs that you want to process a specific workload.&lt;/p&gt;
&lt;p&gt;The Dask Operator introduces a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskJob&lt;/span&gt;&lt;/code&gt; resource which creates a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskCluster&lt;/span&gt;&lt;/code&gt; alongside a single client &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Pod&lt;/span&gt;&lt;/code&gt; which it attempts to run to completion. If the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Pod&lt;/span&gt;&lt;/code&gt; exits unhappily it will be restarted until it returns a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;0&lt;/span&gt;&lt;/code&gt; exit code, at which point the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskCluster&lt;/span&gt;&lt;/code&gt; is automatically cleaned up.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Diagram of a DaskJob resource and its child resources" src="/images/2022-kubernetes/daskjob.png"
style="max-width: 100%;" width="100%" /&gt;&lt;/p&gt;
&lt;p&gt;The client &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Pod&lt;/span&gt;&lt;/code&gt; has all of the configuration for the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskCluster&lt;/span&gt;&lt;/code&gt; injected at runtime via environment variables, this means your client code doesn’t need to know anything about how the Dask cluster was constructed it just connects and makes use of it. This allows for excellent separation of concerns between your business logic and your deployment tooling.&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.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="c1"&gt;# We don&amp;#39;t need to tell the Client anything about the cluster as&lt;/span&gt;
&lt;span class="c1"&gt;# it will find everything it needs in the environment variables&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="c1"&gt;# Do some work...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This new resource type is useful for some batch workflows, but also demonstrates how you could extend the Dask Operator with your own new resource types and hook them together with a controller plugin.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="extensibility-and-plugins"&gt;
&lt;h2&gt;Extensibility and plugins&lt;/h2&gt;
&lt;p&gt;By moving to native Kubernetes resources and support for the YAML API power users can treat &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskCluster&lt;/span&gt;&lt;/code&gt; resources (or any of the new Dask resources) as building blocks in larger applications. One of Kubernetes’s superpowers is managing everything as composable resources that can be combined to create complex and flexible applications.&lt;/p&gt;
&lt;p&gt;Does your Kubernetes cluster have an opinionated configuration with additional tools like &lt;a class="reference external" href="https://istio.io"&gt;Istio&lt;/a&gt; installed? Have you struggled in the last to integrate &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-kubernetes&lt;/span&gt;&lt;/code&gt; with your existing tooling because it relied on Python to create clusters?&lt;/p&gt;
&lt;p&gt;It’s increasingly common for users to need additional resources to be created alongside their Dask cluster like &lt;a class="reference external" href="https://istio.io/latest/docs/reference/config/networking/gateway/"&gt;Istio Gateway&lt;/a&gt; resources or &lt;a class="reference external" href="https://cert-manager.io/docs/concepts/certificate/"&gt;cert-manager Certificate&lt;/a&gt; resources. Now that everything in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-kubernetes&lt;/span&gt;&lt;/code&gt; uses custom resources users can mix and match resources from many different operators to construct their application.&lt;/p&gt;
&lt;p&gt;If this isn’t enough you can also &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/operator_extending.html"&gt;extend our custom controller&lt;/a&gt;. We built the controller with &lt;a class="reference external" href="https://kopf.readthedocs.io/en/stable/"&gt;kopf&lt;/a&gt; primarily because the Dask community is strong in Python and less so in Golang (the most common way to build operators). It made sense to play into our strengths rather than using the most popular option.&lt;/p&gt;
&lt;p&gt;This also means our users should be able to more easily modify the controller logic and we’ve included a plugin system that allows you to add extra logic rules by installing a custom package into the controller container image and registering them via entry points.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Source for my_controller_plugin.plugin&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;kopf&lt;/span&gt;

&lt;span class="nd"&gt;@kopf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;on&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;create&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;service&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;labels&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;dask.org/component&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;scheduler&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;})&lt;/span&gt;
&lt;span class="k"&gt;async&lt;/span&gt; &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;handle_scheduler_service_create&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;new&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;namespace&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;logger&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
   &lt;span class="c1"&gt;# Do something here like create an Istio Gateway&lt;/span&gt;
   &lt;span class="c1"&gt;# See https://kopf.readthedocs.io/en/stable/handlers for documentation on what is possible here&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-toml notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# pyproject.toml for my_controller_plugin&lt;/span&gt;

&lt;span class="k"&gt;[option.entry_points]&lt;/span&gt;
&lt;span class="n"&gt;dask_operator_plugin&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;
&lt;span class="w"&gt;   &lt;/span&gt;&lt;span class="err"&gt;my&lt;/span&gt;&lt;span class="mi"&gt;_&lt;/span&gt;&lt;span class="n"&gt;controller_plugin&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="err"&gt;my&lt;/span&gt;&lt;span class="mi"&gt;_&lt;/span&gt;&lt;span class="n"&gt;controller_plugin&lt;/span&gt;&lt;span class="p"&gt;.&lt;/span&gt;&lt;span class="n"&gt;plugin&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-dockerfile notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c"&gt;# Dockerfile&lt;/span&gt;

&lt;span class="k"&gt;FROM&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="s"&gt;ghcr.io/dask/dask-kubernetes-operator:2022.10.0&lt;/span&gt;

&lt;span class="k"&gt;RUN&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;pip&lt;span class="w"&gt; &lt;/span&gt;install&lt;span class="w"&gt; &lt;/span&gt;my-controller-plugin
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;That’s it, when the controller starts up it will also import all &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;&amp;#64;kopf&lt;/span&gt;&lt;/code&gt; methods from modules listed in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask_operator_plugin&lt;/span&gt;&lt;/code&gt; entry point alongside the core functionality.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/11/09/dask-kubernetes-operator.md&lt;/span&gt;, line 356)&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="migrating"&gt;
&lt;h1&gt;Migrating&lt;/h1&gt;
&lt;p&gt;One caveat to switching to the operator model is that you need to install the CRDs and controller on your Kubernetes before you can start using it. While a small hurdle this is a break in the user experience compared to the classic implementation.&lt;/p&gt;
&lt;div class="highlight-bash notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;helm&lt;span class="w"&gt; &lt;/span&gt;repo&lt;span class="w"&gt; &lt;/span&gt;add&lt;span class="w"&gt; &lt;/span&gt;dask&lt;span class="w"&gt; &lt;/span&gt;https://helm.dask.org&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="o"&gt;&amp;amp;&amp;amp;&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;helm&lt;span class="w"&gt; &lt;/span&gt;repo&lt;span class="w"&gt; &lt;/span&gt;update
kubectl&lt;span class="w"&gt; &lt;/span&gt;create&lt;span class="w"&gt; &lt;/span&gt;ns&lt;span class="w"&gt; &lt;/span&gt;dask-operator
helm&lt;span class="w"&gt; &lt;/span&gt;install&lt;span class="w"&gt; &lt;/span&gt;--namespace&lt;span class="w"&gt; &lt;/span&gt;dask-operator&lt;span class="w"&gt; &lt;/span&gt;dask-operator&lt;span class="w"&gt; &lt;/span&gt;dask/dask-kubernetes-operator
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We also took this opportunity to make breaking changes to the constructor of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;KubeCluster&lt;/span&gt;&lt;/code&gt; to simplify usage for beginners or folks who are happy with the default options. By adopting the YAML API power users can tinker and tweak to their hearts content without having to modify the Python library, so it made sense to make the Python library simpler and more pleasant to use for the majority of users.&lt;/p&gt;
&lt;p&gt;We made an explicit decision not to just replace the old &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;KubeCluster&lt;/span&gt;&lt;/code&gt; with the new one in place because people’s code will just stop working if we did. Instead we are asking folks to &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/kubecluster_migrating.html"&gt;read the migration guide&lt;/a&gt; and update your imports and construction code. Users of the classic cluster manager will start seeing a deprecation warning as of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;2022.10.0&lt;/span&gt;&lt;/code&gt; and at some point the classic implementation will be removed all together. If migrating is challenging to do quickly you can always pin your &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-kubernetes&lt;/span&gt;&lt;/code&gt; version, and from then on you are clearly not getting bug fixes or enhancements. But in all honesty those have been few and far between for the classic implementation lately anyway.&lt;/p&gt;
&lt;p&gt;We are optimistic that the new cleaner implementation, faster cluster startup times and bucket of new features is enough to convince you that it’s worth the migration effort.&lt;/p&gt;
&lt;p&gt;If you want some help migrating and the migration guide doesn’t cover your use case then don’t hesitate to &lt;a class="reference external" href="https://dask.discourse.group"&gt;reach out on the forum&lt;/a&gt;. We’ve also worked hard to ensure the new implementation has feature parity with the classic one, but if anything is missing or broken then please &lt;a class="reference external" href="https://github.com/dask/dask-kubernetes/issues"&gt;open an issue on GitHub&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2022/11/09/dask-kubernetes-operator/"/>
    <summary>We are excited to announce that the Dask Kubernetes Operator is now generally available 🎉!</summary>
    <category term="clusters" label="clusters"/>
    <category term="dask-kubernetes" label="dask-kubernetes"/>
    <category term="deployment" label="deployment"/>
    <category term="kubernetes" label="kubernetes"/>
    <published>2022-11-09T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2022/08/09/understanding-meta-keyword-argument/</id>
    <title>Understanding Dask’s meta keyword argument</title>
    <updated>2022-08-09T00:00:00+00:00</updated>
    <author>
      <name>Pavithra Eswaramoorthy</name>
    </author>
    <content type="html">&lt;p&gt;If you have worked with Dask DataFrames or Dask Arrays, you have probably come across the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; keyword argument. Perhaps, while using methods like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;apply()&lt;/span&gt;&lt;/code&gt;:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;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;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;datasets&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;timeseries&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;my_custom_arithmetic&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;r&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;r&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;id&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1000&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;r&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;r&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;r&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;r&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;y&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="mi"&gt;0&lt;/span&gt;


&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;my_computation&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;ddf&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;my_custom_arithmetic&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="n"&gt;meta&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Series&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="s2"&gt;&amp;quot;float64&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;ddf&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="c1"&gt;# Output:&lt;/span&gt;
&lt;span class="c1"&gt;#&lt;/span&gt;
&lt;span class="c1"&gt;#                        id    name         x         y  my_computation&lt;/span&gt;
&lt;span class="c1"&gt;# timestamp&lt;/span&gt;
&lt;span class="c1"&gt;# 2000-01-01 00:00:00  1055  Victor -0.575374  0.868320        2.067696&lt;/span&gt;
&lt;span class="c1"&gt;# 2000-01-01 00:00:01   994   Zelda  0.963684  0.972240        0.000000&lt;/span&gt;
&lt;span class="c1"&gt;# 2000-01-01 00:00:02   982  George -0.997531 -0.876222        0.000000&lt;/span&gt;
&lt;span class="c1"&gt;# 2000-01-01 00:00:03   981  Ingrid  0.852159 -0.419733        0.000000&lt;/span&gt;
&lt;span class="c1"&gt;# 2000-01-01 00:00:04  1029   Jerry -0.839431 -0.736572       -0.768500&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You might have also seen one or more of the following warnings/errors:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="ne"&gt;UserWarning&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;You&lt;/span&gt; &lt;span class="n"&gt;did&lt;/span&gt; &lt;span class="ow"&gt;not&lt;/span&gt; &lt;span class="n"&gt;provide&lt;/span&gt; &lt;span class="n"&gt;metadata&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;so&lt;/span&gt; &lt;span class="n"&gt;Dask&lt;/span&gt; &lt;span class="ow"&gt;is&lt;/span&gt; &lt;span class="n"&gt;running&lt;/span&gt; &lt;span class="n"&gt;your&lt;/span&gt; &lt;span class="n"&gt;function&lt;/span&gt; &lt;span class="n"&gt;on&lt;/span&gt; &lt;span class="n"&gt;a&lt;/span&gt; &lt;span class="n"&gt;small&lt;/span&gt; &lt;span class="n"&gt;dataset&lt;/span&gt; &lt;span class="n"&gt;to&lt;/span&gt; &lt;span class="n"&gt;guess&lt;/span&gt; &lt;span class="n"&gt;output&lt;/span&gt; &lt;span class="n"&gt;types&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt; &lt;span class="n"&gt;It&lt;/span&gt; &lt;span class="ow"&gt;is&lt;/span&gt; &lt;span class="n"&gt;possible&lt;/span&gt; &lt;span class="n"&gt;that&lt;/span&gt; &lt;span class="n"&gt;Dask&lt;/span&gt; &lt;span class="n"&gt;will&lt;/span&gt; &lt;span class="n"&gt;guess&lt;/span&gt; &lt;span class="n"&gt;incorrectly&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;UserWarning: `meta` is not specified, inferred from partial data. Please provide `meta` if the result is unexpected.
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;ValueError: Metadata inference failed in …
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If the above messages look familiar, this blog post is for you. :)&lt;/p&gt;
&lt;p&gt;We will discuss:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;what the is &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; keyword argument,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;why does Dask need &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;, and&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;how to use it effectively.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We will look at &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; mainly in the context of Dask DataFrames, however, similar principles also apply to Dask Arrays.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/08/09/understanding-meta-keyword-argument.md&lt;/span&gt;, line 65)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="what-is-meta"&gt;

&lt;p&gt;Before answering this, let’s quickly discuss &lt;a class="reference external" href="https://docs.dask.org/en/stable/dataframe.html"&gt;Dask DataFrames&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;A Dask DataFrame is a lazy object composed of multiple &lt;a class="reference external" href="https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.html"&gt;pandas DataFrames&lt;/a&gt;, where each pandas DataFrame is called a “partition”. These are stacked along the index and Dask keeps track of these partitions using “divisions”, which is a tuple representing the start and end index of each partition.&lt;/p&gt;
&lt;img src="https://docs.dask.org/en/stable/_images/dask-dataframe.svg" alt="Dask DataFrame consists of multiple pandas DataFrames" width="50%"&gt;
&lt;p&gt;When you create a Dask DataFrame, you usually see something 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="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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_dict&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="gp"&gt;... &lt;/span&gt;        &lt;span class="s2"&gt;&amp;quot;x&amp;quot;&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;6&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;        &lt;span class="s2"&gt;&amp;quot;y&amp;quot;&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;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;16&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="gp"&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;2&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;
&lt;span class="go"&gt;Dask DataFrame Structure:&lt;/span&gt;
&lt;span class="go"&gt;                   x      y&lt;/span&gt;
&lt;span class="go"&gt;npartitions=2&lt;/span&gt;
&lt;span class="go"&gt;0              int64  int64&lt;/span&gt;
&lt;span class="go"&gt;3                ...    ...&lt;/span&gt;
&lt;span class="go"&gt;5                ...    ...&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: from_pandas, 2 tasks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here, Dask has created the structure of the DataFrame using some “metadata” information about the &lt;em&gt;column names&lt;/em&gt; and their &lt;em&gt;datatypes&lt;/em&gt;. This metadata information is called &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;. Dask uses &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; for understanding Dask operations and creating accurate task graphs (i.e., the logic of your computation).&lt;/p&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; &lt;em&gt;keyword argument&lt;/em&gt; in various Dask DataFrame functions allows you to explicitly share this metadata information with Dask. Note that the keyword argument is concerned with the metadata of the &lt;em&gt;output&lt;/em&gt; of those functions.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/08/09/understanding-meta-keyword-argument.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="why-does-dask-need-meta"&gt;
&lt;h1&gt;Why does Dask need &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;?&lt;/h1&gt;
&lt;p&gt;Dask computations are evaluated &lt;em&gt;lazily&lt;/em&gt;. This means Dask creates the logic and flow, called task graph, of the computation immediately, but evaluates them only when necessary – usually, on calling &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.compute()&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;An example task graph generated to compute the sum of the DataFrame:&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;ddf&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;s&lt;/span&gt;
&lt;span class="go"&gt;Dask Series Structure:&lt;/span&gt;
&lt;span class="go"&gt;npartitions=1&lt;/span&gt;
&lt;span class="go"&gt;x    int64&lt;/span&gt;
&lt;span class="go"&gt;y      ...&lt;/span&gt;
&lt;span class="go"&gt;dtype: int64&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: dataframe-sum-agg, 5 tasks&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;visualize&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/understanding-meta-task-graph.png" alt="Dask task graph, starts with two partitions thatare input to a dataframe-sum-chunk task each. Their results are input to a single dataframe-sum-agg which produces the final output." width="50%"&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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;x    15&lt;/span&gt;
&lt;span class="go"&gt;y    75&lt;/span&gt;
&lt;span class="go"&gt;dtype: int64&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is a single operation, but Dask workflows usually have multiple such operation chained together. Therefore, to create the task graph effectively, Dask needs to know the strucutre and datatypes of the DataFame after each operation. Especially because Dask does not know the actual values/structure of the DataFrame yet.&lt;/p&gt;
&lt;p&gt;This is where &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; is comes in.&lt;/p&gt;
&lt;p&gt;In the above example, the Dask DataFrame changed into a Series after &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sum()&lt;/span&gt;&lt;/code&gt;. Dask knows this (even before we call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;compute()&lt;/span&gt;&lt;/code&gt;) only becasue of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Internally, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; is represented as an empty pandas &lt;a class="reference external" href="https://docs.dask.org/en/stable/dataframe.html"&gt;DataFrame&lt;/a&gt; or &lt;a class="reference external" href="https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.Series.html"&gt;Series&lt;/a&gt;, which has the same structure as the Dask DataFrame. To learn more about how &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; is defined internally, check out the &lt;a class="reference external" href="https://docs.dask.org/en/stable/dataframe-design.html#metadata"&gt;DataFrame Internal Design documentation&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;To see the actual metadata information for a collection, you can look at the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;._meta&lt;/span&gt;&lt;/code&gt; attribute[1]:&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;_meta&lt;/span&gt;
&lt;span class="go"&gt;Series([], dtype: int64)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/08/09/understanding-meta-keyword-argument.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="how-to-specify-meta"&gt;
&lt;h1&gt;How to specify &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;?&lt;/h1&gt;
&lt;p&gt;You can specify &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; in a few different ways, but the recommended way for Dask DataFrame is:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;“An empty &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pd.DataFrame&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pd.Series&lt;/span&gt;&lt;/code&gt; that matches the dtypes and column names of the output.”&lt;/p&gt;
&lt;p&gt;~ &lt;a class="reference external" href="https://docs.dask.org/en/stable/generated/dask.dataframe.DataFrame.apply.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DataFrame.apply()&lt;/span&gt;&lt;/code&gt; docstring&lt;/a&gt;&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;For example:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;meta_df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;x&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;y&amp;quot;&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="nb"&gt;int&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;meta_df&lt;/span&gt;

&lt;span class="go"&gt;Empty DataFrame&lt;/span&gt;
&lt;span class="go"&gt;Columns: [x, y]&lt;/span&gt;
&lt;span class="go"&gt;Index: []&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;apply&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="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="n"&gt;meta&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;meta_df&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf2&lt;/span&gt;
&lt;span class="go"&gt;   x  y&lt;/span&gt;
&lt;span class="go"&gt;0  0  0&lt;/span&gt;
&lt;span class="go"&gt;1  1  1&lt;/span&gt;
&lt;span class="go"&gt;2  2  2&lt;/span&gt;
&lt;span class="go"&gt;3  0  3&lt;/span&gt;
&lt;span class="go"&gt;4  1  4&lt;/span&gt;
&lt;span class="go"&gt;5  2  5&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://docs.dask.org/en/stable/dataframe-design.html#metadata"&gt;other ways you can describe &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; are:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;For a DataFrame, you can specify &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; as a:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Python dictionary: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;{column_name_1:&lt;/span&gt; &lt;span class="pre"&gt;dtype_1,&lt;/span&gt; &lt;span class="pre"&gt;column_name_2:&lt;/span&gt; &lt;span class="pre"&gt;dtype_2,&lt;/span&gt; &lt;span class="pre"&gt;…}&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Iterable of tuples: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;[(column_name_1,&lt;/span&gt; &lt;span class="pre"&gt;dtype_1),&lt;/span&gt; &lt;span class="pre"&gt;(columns_name_2,&lt;/span&gt; &lt;span class="pre"&gt;dtype_2,&lt;/span&gt; &lt;span class="pre"&gt;…)]&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;strong&gt;Note&lt;/strong&gt; that while describing &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; as shown above, using a dictionary or iterable of tuples, the order in which you mention column names is important. Dask will use the same order to create the pandas DataFrame for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;. If the orders don’t match, you will see the following error:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="ne"&gt;ValueError&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;The&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;the&lt;/span&gt; &lt;span class="n"&gt;computed&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="n"&gt;do&lt;/span&gt; &lt;span class="ow"&gt;not&lt;/span&gt; &lt;span class="n"&gt;match&lt;/span&gt; &lt;span class="n"&gt;the&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;the&lt;/span&gt; &lt;span class="n"&gt;provided&lt;/span&gt; &lt;span class="n"&gt;metadata&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For a Series output, you can specify &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; using a single tuple: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(coulmn_name,&lt;/span&gt; &lt;span class="pre"&gt;dtype)&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;You should &lt;strong&gt;not&lt;/strong&gt; describe &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; using just a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dtype&lt;/span&gt;&lt;/code&gt; (like: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta=&amp;quot;int64&amp;quot;&lt;/span&gt;&lt;/code&gt;), even for scalar outputs. If you do, you will see the following warning:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;FutureWarning: Meta is not valid, `map_partitions` and `map_overlap` expects output to be a pandas object. Try passing a pandas object as meta or a dict or tuple representing the (name, dtype) of the columns. In the future the meta you passed will not work.
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;During operations like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_partitions&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; (which uses &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_partitions&lt;/span&gt;&lt;/code&gt; internally), Dask coerces the scalar output of each partition into a pandas object. So, the output of functions that take &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; will never be scalar.&lt;/p&gt;
&lt;p&gt;For example:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;repartition&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;1&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;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_partitions&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="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&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;type&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="go"&gt;pandas.core.series.Series&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here, the Dask DataFrame &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ddf&lt;/span&gt;&lt;/code&gt; has only one partition. Hence, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;len(x)&lt;/span&gt;&lt;/code&gt; on that one partition would result in a scalar output of integer dtype. However, when we compute it, we see a pandas Series. This confirms that Dask is coercing the outputs to pandas objects.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Another note&lt;/strong&gt;, Dask Arrays may not always do this conversion. You can look at the &lt;a class="reference external" href="https://docs.dask.org/en/stable/array-api.html"&gt;API reference&lt;/a&gt; for your particular Array operation for details.&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="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;my_arr&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;10&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;my_arr&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_blocks&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="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;10&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/08/09/understanding-meta-keyword-argument.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="meta-does-not-force-the-structure-or-dtypes"&gt;
&lt;h1&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; does not &lt;em&gt;force&lt;/em&gt; the structure or dtypes&lt;/h1&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; can be thought of as a suggestion to Dask. Dask uses this &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; to generate the task graph until it can infer the actual metadata from the values. It &lt;strong&gt;does not&lt;/strong&gt; force the output to have the structure or dtype of the specified &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Consider the following example, and remember that we defined &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ddf&lt;/span&gt;&lt;/code&gt; with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;y&lt;/span&gt;&lt;/code&gt; column names in the previous sections.&lt;/p&gt;
&lt;p&gt;If we provide different column names (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;a&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;b&lt;/span&gt;&lt;/code&gt;) in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; description, Dask uses these new names to create the task 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="n"&gt;meta_df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;a&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;b&amp;quot;&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="nb"&gt;int&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;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;apply&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="n"&gt;x&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="n"&gt;meta&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;meta_df&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;result&lt;/span&gt;
&lt;span class="go"&gt;Dask DataFrame Structure:&lt;/span&gt;
&lt;span class="go"&gt;                   a      b&lt;/span&gt;
&lt;span class="go"&gt;npartitions=2&lt;/span&gt;
&lt;span class="go"&gt;0              int64  int64&lt;/span&gt;
&lt;span class="go"&gt;3                ...    ...&lt;/span&gt;
&lt;span class="go"&gt;5                ...    ...&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: apply, 4 tasks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;However, if we compute &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;result&lt;/span&gt;&lt;/code&gt;, we will get the following error:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;ValueError: The columns in the computed data do not match the columns in the provided metadata&lt;/span&gt;
&lt;span class="go"&gt;  Extra:   [&amp;#39;x&amp;#39;, &amp;#39;y&amp;#39;]&lt;/span&gt;
&lt;span class="go"&gt;  Missing: [&amp;#39;a&amp;#39;, &amp;#39;b&amp;#39;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;While computing, Dask evaluates the actual metadata with columns &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;y&lt;/span&gt;&lt;/code&gt;. This does not match the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; that we provided, and hence, Dask raises a helpful error message. Notice how Dask does not change the output to have &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;a&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;b&lt;/span&gt;&lt;/code&gt; here, rather uses &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;a&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;b&lt;/span&gt;&lt;/code&gt; column names only for intermediate task graphs.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/08/09/understanding-meta-keyword-argument.md&lt;/span&gt;, line 247)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="using-meta-directly"&gt;
&lt;h1&gt;Using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;._meta&lt;/span&gt;&lt;/code&gt; directly&lt;/h1&gt;
&lt;p&gt;In some rare case, you can also set the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;._meta&lt;/span&gt;&lt;/code&gt; attribute[1] directly for a Dask DataFrame. For example, if the DataFrame was created with incorrect dtypes, like:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_dict&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="gp"&gt;... &lt;/span&gt;        &lt;span class="s2"&gt;&amp;quot;x&amp;quot;&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;6&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;        &lt;span class="s2"&gt;&amp;quot;y&amp;quot;&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;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;16&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="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;object&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="c1"&gt;# Note the “object” dtype&lt;/span&gt;
&lt;span class="gp"&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;2&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;
&lt;span class="go"&gt;Dask DataFrame Structure:&lt;/span&gt;
&lt;span class="go"&gt;                    x       y&lt;/span&gt;
&lt;span class="go"&gt;npartitions=2&lt;/span&gt;
&lt;span class="go"&gt;0              object  object&lt;/span&gt;
&lt;span class="go"&gt;3                 ...     ...&lt;/span&gt;
&lt;span class="go"&gt;5                 ...     ...&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: from_pandas, 2 tasks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The values are clearly integers but the dtype says &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;object&lt;/span&gt;&lt;/code&gt;, so we can’t perform integer operations like 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;add&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;
&lt;span class="go"&gt;ValueError: Metadata inference failed in `add`.&lt;/span&gt;

&lt;span class="go"&gt;Original error is below:&lt;/span&gt;
&lt;span class="go"&gt;------------------------&lt;/span&gt;
&lt;span class="go"&gt;TypeError(&amp;#39;can only concatenate str (not &amp;quot;int&amp;quot;) to str&amp;#39;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here, we can explicitly define &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;._meta&lt;/span&gt;&lt;/code&gt;[1]:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_meta&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;x&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;y&amp;quot;&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="s2"&gt;&amp;quot;int64&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Then, perform the 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;add&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&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;add&lt;/span&gt;
&lt;span class="go"&gt;Dask DataFrame Structure:&lt;/span&gt;
&lt;span class="go"&gt;                   x      y&lt;/span&gt;
&lt;span class="go"&gt;npartitions=2&lt;/span&gt;
&lt;span class="go"&gt;0              int64  int64&lt;/span&gt;
&lt;span class="go"&gt;3                ...    ...&lt;/span&gt;
&lt;span class="go"&gt;5                ...    ...&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: add, 4 tasks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Thanks for reading!&lt;/p&gt;
&lt;p&gt;Have you run into issues with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; before? Please let us know on &lt;a class="reference external" href="https://dask.discourse.group/"&gt;Discourse&lt;/a&gt;, and we will consider including it here, or updating the Dask documentation. :)&lt;/p&gt;
&lt;p&gt;[1] NOTE: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;._meta&lt;/span&gt;&lt;/code&gt; is not a public property, so we recommend using it only when necessary. There is &lt;a class="reference external" href="https://github.com/dask/dask/issues/8585"&gt;an ongoing discussion&lt;/a&gt; around creating public methods to get, set, and view the information in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;._meta&lt;/span&gt;&lt;/code&gt;, and this blog post will be updated to use the public methods when they’re created.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2022/08/09/understanding-meta-keyword-argument/"/>
    <summary>If you have worked with Dask DataFrames or Dask Arrays, you have probably come across the meta keyword argument. Perhaps, while using methods like apply():</summary>
    <category term="dataframe" label="dataframe"/>
    <published>2022-08-09T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2022/07/19/dask-multi-cloud/</id>
    <title>Data Proximate Computation on a Dask Cluster Distributed Between Data Centres</title>
    <updated>2022-07-19T00:00:00+00:00</updated>
    <author>
      <name>Richard Care (Met Office)</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is a joint venture between the &lt;a class="reference external" href="https://www.metoffice.gov.uk"&gt;Met Office&lt;/a&gt; and the &lt;a class="reference external" href="https://www.europeanweather.cloud/"&gt;European Weather Cloud&lt;/a&gt;, which is a partnership of &lt;a class="reference external" href="https://ecmwf.int"&gt;ECMWF&lt;/a&gt; and &lt;a class="reference external" href="https://eumetsat.int/"&gt;EUMETSAT&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/19/dask-multi-cloud.md&lt;/span&gt;, line 11)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;We have devised a technique for creating a Dask cluster where worker nodes are hosted in different data centres, connected by a mesh VPN that allows the scheduler and workers to communicate and exchange results.&lt;/p&gt;
&lt;p&gt;A novel (ab)use of Dask resources allows us to run data processing tasks on the workers in the cluster closest to the source data, so that communication between data centres is minimised. If combined with zarr to give access to huge hyper-cube datasets in object storage, we believe that the technique could realise the potential of data-proximate distributed computing in the Cloud.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/19/dask-multi-cloud.md&lt;/span&gt;, line 17)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="introduction"&gt;
&lt;h1&gt;Introduction&lt;/h1&gt;
&lt;p&gt;The UK Met Office has been carrying out a study into data-proximate computing in collaboration with the European Weather Cloud. We identified Dask as a key technology, but whilst existing Dask techniques focus on parallel computation in a single data centre, we looked to extend this to computation across data centres. This way, when the data required is hosted in multiple locations, tasks can be run where the data is rather than copying it.&lt;/p&gt;
&lt;p&gt;Dask worker nodes exchange data chunks over a network, coordinated by a scheduler. There is an assumption that all nodes are freely able to communicate, which is not generally true across data centres due to firewalls, NAT, etc, so a truly distributed approach has to solve this problem. In addition, it has to manage data transfer efficiently, because moving data chunks between data centres is much more costly than between workers in the same cloud.&lt;/p&gt;
&lt;p&gt;This notebook documents a running proof-of-concept that addresses these problems. It runs a computation in 3 locations:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;This computer, where the client and scheduler are running. This was run on AWS during development.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The ECMWF data centre. This has compute resources, and hosts data containing &lt;em&gt;predictions&lt;/em&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The EUMETSAT data centre, with compute resources and data on &lt;em&gt;observations&lt;/em&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;IPython.display&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Image&lt;/span&gt;
&lt;span class="n"&gt;Image&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;filename&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;images/datacentres.png&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="c1"&gt;# this because GitHub doesn&amp;#39;t render markup images in private repos&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/dask-multi-cloud_1_0.png" /&gt;&lt;/p&gt;
&lt;p&gt;The idea is that tasks accessing data available in a location should be run there. Meanwhile the computation can be defined, invoked, and the results rendered, elsewhere. All this with minimal hinting to the computation as to how this should be done.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/19/dask-multi-cloud.md&lt;/span&gt;, line 38)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="setup"&gt;
&lt;h1&gt;Setup&lt;/h1&gt;
&lt;p&gt;First some imports and conveniences&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;os&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;time&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;sleep&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;performance_report&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;get_task_stream&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_worker_pools&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;pool&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;propagate_pools&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pytest&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;ipytest&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;xarray&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;matplotlib.pyplot&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;plt&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;orgs&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;my_org&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;tree&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;tree&lt;/span&gt;

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

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

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

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

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

    &lt;span class="n"&gt;averages&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;predictions&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;realization&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;diff&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;averages&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;isel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;height&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;observations&lt;/span&gt;
    &lt;span class="n"&gt;diff&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="c1"&gt;#scope()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;CPU times: user 10 µs, sys: 2 µs, total: 12 µs
Wall time: 13.8 µs
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;When we uncomment &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scope()&lt;/span&gt;&lt;/code&gt; and actually run this, it takes over 14 minutes to complete! Accessing the data over NFS between data centres (we run this notebook in AWS) is just too slow.&lt;/p&gt;
&lt;p&gt;In fact just copying the data files onto the computer running this notebook takes the same sort of time. At least 2.8 GiB + 4.8 MiB of data must pass from the data centres to this machine to perform the calculation.&lt;/p&gt;
&lt;p&gt;Instead we should obviously run the Dask tasks where the data is. We can do that on a Dask cluster.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/19/dask-multi-cloud.md&lt;/span&gt;, line 226)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="running-up-a-cluster"&gt;
&lt;h1&gt;Running Up a Cluster&lt;/h1&gt;
&lt;p&gt;The cluster is run up with a single command. It takes a while though&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;subprocess&lt;/span&gt;

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

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

&lt;span class="n"&gt;show_all_workers&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/19/dask-multi-cloud.md&lt;/span&gt;, line 354)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="running-on-the-cluster"&gt;
&lt;h1&gt;Running on the Cluster&lt;/h1&gt;
&lt;p&gt;Now that there is a Dask client in scope, calculations will be run on the cluster. We can define the tasks to be run&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;predictions&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xarray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/ecmwf/000490262cdd067721a34112963bcaa2b44860ab.nc&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;chunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;observations&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xarray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/eumetsat/ad-hoc/observations.nc&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;chunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;averages&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;predictions&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;realization&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;diff&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;averages&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;isel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;height&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;observations&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;But when we try to perform the calculation it fails&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;pytest&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;raises&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="ne"&gt;FileNotFoundError&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;excinfo&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;show_all_workers&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;diff&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

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

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

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

&lt;span class="n"&gt;averages&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;predictions&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;realization&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;diff&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;averages&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;isel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;height&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;observations&lt;/span&gt;

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

&lt;span class="n"&gt;averages&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;predictions&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;realization&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;diff&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;averages&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;isel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;height&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;observations&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;propagate_pools&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="n"&gt;show_all_workers&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;diff&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Of course the cluster would have to be provisioned with compute resources in the appropriate data centres, although with some work this could be made dynamic as part of the catalog code.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/19/dask-multi-cloud.md&lt;/span&gt;, line 520)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="more-information"&gt;
&lt;h1&gt;More Information&lt;/h1&gt;
&lt;p&gt;This &lt;a class="reference external" href="https://github.com/dmcg/dask-multicloud-poc/blob/main/demo/dask-multi-cloud.ipynb"&gt;notebook&lt;/a&gt;, and the code behind it, are published in a &lt;a class="reference external" href="https://github.com/dmcg/dask-multicloud-poc"&gt;GitHub repository&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;For details of the prototype implementation, and ideas for enhancements, see
&lt;a class="reference external" href="https://github.com/dmcg/dask-multicloud-poc/blob/main/demo/dask-multi-cloud-details.ipynb"&gt;dask-multi-cloud-details.ipynb&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/19/dask-multi-cloud.md&lt;/span&gt;, line 527)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;Thank you to Armagan Karatosun (EUMETSAT) and Vasileios Baousis (ECMWF) for their help and support with the infrastructure to support this proof of concept.
Gabe Joseph (Coiled) wrote the clever pool context managers, and Jacob Tomlinson (NVIDIA) reviewed this document.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Crown Copyright 2022&lt;/em&gt;&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2022/07/19/dask-multi-cloud/"/>
    <summary>This work is a joint venture between the Met Office and the European Weather Cloud, which is a partnership of ECMWF and EUMETSAT.</summary>
    <category term="deployment" label="deployment"/>
    <category term="distributed" label="distributed"/>
    <published>2022-07-19T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2022/07/15/documentation-framework/</id>
    <title>Documentation Framework</title>
    <updated>2022-07-15T00:00:00+00:00</updated>
    <author>
      <name>Julia Signell and Jacob Tomlinson</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/15/documentation-framework.md&lt;/span&gt;, line 8)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;Yesterday at the Dask BOF at &lt;a class="reference external" href="https://www.scipy2022.scipy.org/"&gt;SciPy&lt;/a&gt; we were talking about the recent docs work and how we can fill holes in our documentation. We want to come up with a strategy to improve things.&lt;/p&gt;
&lt;p&gt;For a while, we’ve been exploring moving our documentation to the &lt;a class="reference external" href="https://diataxis.fr/"&gt;Diátaxis Framework&lt;/a&gt;, and after catching up with other maintainers at SciPy it is clear that many projects are converging on this framework and we are confident about continuing on this journey. This post lays out how we will take the existing docs and apply the framework to make content clearer and easier to find. NOTE: This blog post sketches out where we are going, but the change will happen incrementally.&lt;/p&gt;
&lt;p&gt;We want the docs to quickly answer questions like:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;I know my workflow is parallizable - can Dask help?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How do I find my logs for a particular worker?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Should I be using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.persist()&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/2022/07/15/documentation-framework.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&gt;
&lt;section id="contents"&gt;
&lt;h1&gt;Contents&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#theory"&gt;&lt;span class="xref myst"&gt;Theory&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#current-documentation"&gt;&lt;span class="xref myst"&gt;Current Documentation&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#new-structure"&gt;&lt;span class="xref myst"&gt;New Structure&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#how-you-can-help"&gt;&lt;span class="xref myst"&gt;How you can help&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/15/documentation-framework.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="theory"&gt;
&lt;h1&gt;Theory&lt;/h1&gt;
&lt;p&gt;The Diataxis Framework proposes that the documentation be split into 4 entirely separate sections.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Diataxis Framework" src="https://blog.dask.org/_images/diataxis-framework.png" /&gt;
&lt;em&gt;Credit: https://diataxis.fr/&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Each section serves a unique purpose.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Tutorials&lt;/strong&gt; provide a narrative that addresses a particular larger objective such as predicting global temperature or analyzing financial data.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;How-Tos&lt;/strong&gt; target people who already know &lt;em&gt;what&lt;/em&gt; they want to do and are trying to figure out &lt;em&gt;how&lt;/em&gt; to do it. These people might ask questions like:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;How do I apply a rolling mean to a timeseries?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How do I groupby a column?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How do I write to a geotiff?&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Reference&lt;/strong&gt; provides the exact arguments and outputs of a particular operation.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Explanation&lt;/strong&gt; gives context and includes descriptions of how operations work internally.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/15/documentation-framework.md&lt;/span&gt;, line 44)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="current-documentation"&gt;
&lt;h1&gt;Current Documentation&lt;/h1&gt;
&lt;p&gt;There are several different sites that comprise different aspects of dask documentation. Of particular interest are &lt;a class="reference external" href="https://examples.dask.org"&gt;Examples&lt;/a&gt;, &lt;a class="reference external" href="https://tutorial.dask.org"&gt;Tutorials&lt;/a&gt; and &lt;a class="reference external" href="https://docs.dask.org"&gt;Docs&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The bulk of the documentation that we currently have on &lt;a class="reference external" href="https://docs.dask.org"&gt;Docs&lt;/a&gt; falls under “Explanation” and “Reference” but they are pretty intermingled. There are also some small “How-Tos” sprinkled in, particularly in the API docs.&lt;/p&gt;
&lt;p&gt;The material on &lt;a class="reference external" href="https://tutorial.dask.org"&gt;Tutorials&lt;/a&gt; is a mixture of “Tutorial” and “Explanation”. They answer questions like: “What can I do with Dask Dataframes?” &lt;em&gt;and&lt;/em&gt; questions like “What is a Dask Dataframe?”. These are styled like lectures in that there is often no motivating example and the assumption is that the audience wants to learn both about how to do specific operations in dask and how those operations work. This type of material can be consumed as standalone content and runs on binder.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://examples.dask.org"&gt;Examples&lt;/a&gt; pretty much falls under “How-To” but there is a fair amount of setup and each example isn’t split into small enough bits. They answer questions like: “How do I use dask dataframes?” and they have some more longer workflows.&lt;/p&gt;
&lt;section id="which-pages-are-most-used"&gt;
&lt;h2&gt;Which pages are most used?&lt;/h2&gt;
&lt;p&gt;From Google Analytics we can see the most commonly viewed pages.&lt;/p&gt;
&lt;img src="/images/docs-google-analytics.png" width="70%"&gt;
&lt;p&gt;It is hard to understand whether those pages are the most visible, or if they actually contain the information that people are trying to find. But either way, it conveys the importance of navigation in directing users.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/15/documentation-framework.md&lt;/span&gt;, line 62)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="new-structure"&gt;
&lt;h1&gt;New Structure&lt;/h1&gt;
&lt;p&gt;&lt;a class="reference external" href="https://tutorial.dask.org"&gt;Tutorial&lt;/a&gt; will be left as is and treated as a long-form overview.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://examples.dask.org"&gt;Examples&lt;/a&gt; will be presented more as &lt;strong&gt;How-Tos&lt;/strong&gt; with little explanation and more code. This will be similar to gallery or cookbook style documentation that you may see in other projects. Historically one of the current roles of examples is to demonstrate what Dask looks like, that role is now subsumed by “10 Minutes to Dask”.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://docs.dask.org"&gt;Docs&lt;/a&gt; will be reorganized and the left-nav will be slimmed down dramatically to provide direction. One idea for the left-nav is:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Installation&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;10 Minutes to Dask (&lt;strong&gt;How-To&lt;/strong&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tutorials &amp;amp; Talks (&lt;strong&gt;Tutorial&lt;/strong&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Examples (&lt;strong&gt;How-To&lt;/strong&gt;)- this will be a landing page that points to individual sections of &lt;a class="reference external" href="https://examples.dask.org"&gt;Examples&lt;/a&gt; or API Reference.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Best Practices (&lt;strong&gt;Explanation&lt;/strong&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;API (&lt;strong&gt;Reference&lt;/strong&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;User Guide (&lt;strong&gt;Explanation&lt;/strong&gt;)&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;DataFrame - explains what a dataframe is - links out aggressively to reference docs.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Array&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bag&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Delayed&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Futures&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Task Graphs&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scheduling&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Diagnostic Dashboard&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Configuration (&lt;strong&gt;Reference&lt;/strong&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Deploy Dask Clusters (&lt;strong&gt;Reference&lt;/strong&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Development Guidelines (&lt;strong&gt;Reference&lt;/strong&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Changelog (&lt;strong&gt;Reference&lt;/strong&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;FAQs (&lt;strong&gt;Reference&lt;/strong&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Many docstrings (aka &lt;strong&gt;Reference&lt;/strong&gt;) already contain their own short-form &lt;strong&gt;How-To&lt;/strong&gt; docs. I think this is a good place for these and we can thoroughly link from other places to these canonical docs.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/15/documentation-framework.md&lt;/span&gt;, line 96)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="how-you-can-help"&gt;
&lt;h1&gt;How you can help&lt;/h1&gt;
&lt;p&gt;Please raise issues on the dask issue tracker when you find holes in the docs! The largest gaps we see now are in “how to” which commonly are found via Google. So if you search for how to do something in Dask, and you’re looking for copy-paste examples but can’t find any then let us know.&lt;/p&gt;
&lt;p&gt;If you see other gaps please let us know about those too. And if you know how your needs fit into the diataxis framework even better :)&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2022/07/15/documentation-framework/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <published>2022-07-15T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2022/02/17/helm-multiple-worker-groups/</id>
    <title>How to run different worker types with the Dask Helm Chart</title>
    <updated>2022-02-17T00:00:00+00:00</updated>
    <author>
      <name>Matthew Murray (NVIDIA)</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/02/17/helm-multiple-worker-groups.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="introduction"&gt;

&lt;p&gt;Today, we’ll learn how to deploy &lt;a class="reference external" href="https://dask.org/"&gt;Dask&lt;/a&gt; on a &lt;a class="reference external" href="https://kubernetes.io/"&gt;Kubernetes&lt;/a&gt; cluster with the Dask Helm Chart and then run and scale different worker types with annotations.&lt;/p&gt;
&lt;section id="what-is-the-dask-helm-chart"&gt;
&lt;h2&gt;What is the Dask Helm Chart?&lt;/h2&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://github.com/dask/helm-chart"&gt;Dask Helm Chart&lt;/a&gt; is a convenient way of deploying Dask using &lt;a class="reference external" href="https://helm.sh/"&gt;Helm&lt;/a&gt;, a package manager for Kubernetes applications. After deploying Dask with the Dask Helm Chart, we can connect to our HelmCluster and begin scaling out workers.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="what-is-dask-kubernetes"&gt;
&lt;h2&gt;What is Dask Kubernetes?&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/"&gt;Dask Kubernetes&lt;/a&gt; allows you to deploy and manage your Dask deployment on a Kubernetes cluster. The Dask Kubernetes Python package has a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HelmCluster&lt;/span&gt;&lt;/code&gt; class (among other things) that will enable you to manage your cluster from Python. In this tutorial, we will use the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HelmCluster&lt;/span&gt;&lt;/code&gt; as our cluster manager.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="prerequisites"&gt;
&lt;h2&gt;Prerequisites&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;To have Helm installed and be able to run &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;helm&lt;/span&gt;&lt;/code&gt; commands&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;To have a running Kubernetes cluster. It doesn’t matter whether you’re running Kubernetes locally using &lt;a class="reference external" href="https://minikube.sigs.k8s.io/docs/"&gt;MiniKube&lt;/a&gt; or &lt;a class="reference external" href="https://kind.sigs.k8s.io/"&gt;Kind&lt;/a&gt; or you’re using a cloud provider like AWS or GCP. But your cluster will need to have access to &lt;a class="reference external" href="https://kubernetes.io/docs/tasks/manage-gpus/scheduling-gpus/"&gt;GPU nodes&lt;/a&gt; to run GPU workers. You’ll also need to install &lt;a class="reference external" href="https://rapids.ai/"&gt;RAPIDS&lt;/a&gt; to run the GPU worker example.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;To have &lt;a class="reference external" href="https://kubernetes.io/docs/tasks/tools/"&gt;kubectl&lt;/a&gt; installed. Although this is not required.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;That’s it, let’s get started!&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/02/17/helm-multiple-worker-groups.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&gt;
&lt;section id="install-dask-kubernetes"&gt;
&lt;h1&gt;Install Dask Kubernetes&lt;/h1&gt;
&lt;p&gt;From the &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/installing.html"&gt;documentation&lt;/a&gt;,&lt;/p&gt;
&lt;div class="highlight-console notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="go"&gt;pip install dask-kubernetes --upgrade&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or&lt;/p&gt;
&lt;div class="highlight-console notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="go"&gt;conda install dask-kubernetes -c conda-forge&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/02/17/helm-multiple-worker-groups.md&lt;/span&gt;, line 43)&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="install-the-dask-helm-chart"&gt;
&lt;h1&gt;Install the Dask Helm Chart&lt;/h1&gt;
&lt;p&gt;First, deploy Dask on Kubernetes with Helm:&lt;/p&gt;
&lt;div class="highlight-console notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="go"&gt;helm repo add dask https://helm.dask.org/&lt;/span&gt;
&lt;span class="go"&gt;helm repo update&lt;/span&gt;
&lt;span class="go"&gt;helm install my-dask dask/dask&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now you should have Dask running on your Kubernetes cluster. If you have kubectl installed, you can run &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;kubectl&lt;/span&gt; &lt;span class="pre"&gt;get&lt;/span&gt; &lt;span class="pre"&gt;all&lt;/span&gt; &lt;span class="pre"&gt;-n&lt;/span&gt; &lt;span class="pre"&gt;default&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;
&lt;img src="/images/default-dask-cluster.png" alt="Default Dask Cluster Installed with Helm" width="661" height="373"&gt;
&lt;p&gt;You can see that we’ve created a few resources! The main thing to know is that we start with three dask workers.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/02/17/helm-multiple-worker-groups.md&lt;/span&gt;, line 59)&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="add-gpu-worker-group-to-our-dask-deployment"&gt;
&lt;h1&gt;Add GPU worker group to our Dask Deployment&lt;/h1&gt;
&lt;p&gt;The Helm Chart has default values that it uses out of the box to deploy our Dask cluster on Kubernetes. But now, because we want to create some GPU workers, we need to change the default values in the Dask Helm Chart. To do this, we can create a copy of the current &lt;a class="reference external" href="https://github.com/dask/helm-chart/blob/main/dask/values.yaml"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;values.yaml&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;, update it to add a GPU worker group and then update our helm deployment.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;First, you can copy the contents of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;values.yaml&lt;/span&gt;&lt;/code&gt; file in the Dask Helm Chart and create a new file called &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;my-values.yaml&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Next, we’re going to update the section in the file called &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;additional_worker_groups&lt;/span&gt;&lt;/code&gt;. The section looks like this:&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;div class="highlight-yaml notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nt"&gt;additional_worker_groups&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="p p-Indicator"&gt;[]&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="c1"&gt;# Additional groups of workers to create&lt;/span&gt;
&lt;span class="c1"&gt;# - name: high-mem-workers  # Dask worker group name.&lt;/span&gt;
&lt;span class="c1"&gt;#   resources:&lt;/span&gt;
&lt;span class="c1"&gt;#     limits:&lt;/span&gt;
&lt;span class="c1"&gt;#       memory: 32G&lt;/span&gt;
&lt;span class="c1"&gt;#     requests:&lt;/span&gt;
&lt;span class="c1"&gt;#       memory: 32G&lt;/span&gt;
&lt;span class="c1"&gt;# ...&lt;/span&gt;
&lt;span class="c1"&gt;# (Defaults will be taken from the primary worker configuration)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Now we’re going to edit the section to look like this:&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;div class="highlight-yaml notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nt"&gt;additional_worker_groups&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="c1"&gt;# Additional groups of workers to create&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nt"&gt;name&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;gpu-workers&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="c1"&gt;# Dask worker group name.&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;replicas&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;1&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;image&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="nt"&gt;repository&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;rapidsai/rapidsai-core&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="nt"&gt;tag&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;21.12-cuda11.5-runtime-ubuntu20.04-py3.8&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="nt"&gt;dask_worker&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;dask-cuda-worker&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;extraArgs&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;--resources&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="s"&gt;&amp;quot;GPU=1&amp;quot;&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;resources&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="nt"&gt;limits&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;        &lt;/span&gt;&lt;span class="nt"&gt;nvidia.com/gpu&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;1&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Now we can update our deployment with our new values in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;my-values.yaml&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;div class="highlight-console notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="go"&gt;helm upgrade -f my-values.yaml my-dask dask/dask&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Again, you can run &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;kubectl&lt;/span&gt; &lt;span class="pre"&gt;get&lt;/span&gt; &lt;span class="pre"&gt;all&lt;/span&gt; &lt;span class="pre"&gt;-n&lt;/span&gt; &lt;span class="pre"&gt;default&lt;/span&gt;&lt;/code&gt;, and you’ll see our new GPU worker pod running:&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;img src="/images/gpu-worker-dask-cluster.png" alt="Dask Cluster Installed with Helm with a GPU worker" width="653" height="428"&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Now we can open up a jupyter notebook or any editor to write some code.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/02/17/helm-multiple-worker-groups.md&lt;/span&gt;, line 108)&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="scaling-the-workers-up-down"&gt;
&lt;h1&gt;Scaling the workers Up/Down&lt;/h1&gt;
&lt;p&gt;We’ll start by importing the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HelmCluster&lt;/span&gt;&lt;/code&gt; cluster manager from Dask Kubernetes. Next, we connect our cluster manager to our dask cluster by passing the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;release_name&lt;/span&gt;&lt;/code&gt; of our Dask cluster as an argument. That’s it, the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HelmCluster&lt;/span&gt;&lt;/code&gt; automatically port-forwards the scheduler to us and can give us quick access to &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/helmcluster.html#dask_kubernetes.HelmCluster.get_logs"&gt;logs&lt;/a&gt;. Next, we’re going to scale our Dask cluster.&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_kubernetes&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;HelmCluster&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;HelmCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;release_name&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;my-dask&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/dask-cluster-four-workers.png" alt="Dask Cluster with four workers" width="1002" height="659"&gt;
&lt;p&gt;To scale our cluster, we need to provide our desired number of workers as an argument to the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HelmCluster&lt;/span&gt;&lt;/code&gt;’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scale&lt;/span&gt;&lt;/code&gt; method. By default, the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scale&lt;/span&gt;&lt;/code&gt; method scales our default worker group. You can see in the first example we scaled the default worker group from three to five workers, giving us six workers in total. In the second example, we use the handy &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;worker_group&lt;/span&gt;&lt;/code&gt; keyword argument to scale our GPU worker group from one to two workers, giving us seven workers in total.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scale&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;# scale the default worker group from 3 to 5 workers&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/dask-cluster-six-workers.png" alt="Dask Cluster with six workers" width="1002" height="802"&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scale&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;worker_group&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;gpu-workers&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# scale the GPU worker group from 1 to 2 workers&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/dask-cluster-seven-workers.png" alt="Dask Cluster with seven cluster" width="992" height="845"&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/02/17/helm-multiple-worker-groups.md&lt;/span&gt;, line 136)&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-finding-the-average-new-york-city-taxi-trip-distance-in-april-2020"&gt;
&lt;h1&gt;Example: Finding the average New York City taxi trip distance in April 2020&lt;/h1&gt;
&lt;p&gt;This example will find the average distance traveled by a yellow taxi in New York City in April 2020 using the &lt;a class="reference external" href="https://www1.nyc.gov/site/tlc/about/tlc-trip-record-data.page"&gt;NY Taxi Dataset&lt;/a&gt;. We’ll compute this distance in two different ways. The first way will employ our default dask workers, and the second way will utilize our GPU worker group. We’ll load the NY Taxi dataset as a data frame in both examples and compute the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&lt;/span&gt;&lt;/code&gt; of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;trip_distance&lt;/span&gt;&lt;/code&gt; column. The main difference is that we need to run our GPU-specific computations using our GPU worker group. We can do this by utilizing Dask annotations.&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;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&lt;/span&gt;

&lt;span class="n"&gt;link&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;https://s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2020-04.csv&amp;quot;&lt;/span&gt;
&lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;link&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;assume_missing&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;avg_trip_distance&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;trip_distance&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;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="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="sa"&gt;f&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;In January 2021, the average trip distance for yellow taxis was &lt;/span&gt;&lt;span class="si"&gt;{&lt;/span&gt;&lt;span class="n"&gt;avg_trip_distance&lt;/span&gt;&lt;span class="si"&gt;}&lt;/span&gt;&lt;span class="s2"&gt; miles.&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;annotate&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;resources&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;GPU&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_cudf&lt;/span&gt;&lt;span class="o"&gt;,&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cudf&lt;/span&gt;
    &lt;span class="n"&gt;dask_cdf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_partitions&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cudf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_pandas&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;avg_trip_distance&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask_cdf&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;trip_distance&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;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="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="sa"&gt;f&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;In January 2021, the average trip distance for yellow taxis was &lt;/span&gt;&lt;span class="si"&gt;{&lt;/span&gt;&lt;span class="n"&gt;avg_trip_distance&lt;/span&gt;&lt;span class="si"&gt;}&lt;/span&gt;&lt;span class="s2"&gt; miles.&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/02/17/helm-multiple-worker-groups.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="closing"&gt;
&lt;h1&gt;Closing&lt;/h1&gt;
&lt;p&gt;That’s it! We’ve deployed Dask with Helm, created an additional GPU worker type, and used our workers to run an example calculation using the NY Taxi dataset. We’ve learned several new things:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;The Dask Helm Chart lets you create multiple worker groups with different worker types. We saw this when we made two different groups of Dask Workers: CPU and GPU workers.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You can run specific computations on your workers of choice with annotations. Our example computed the average taxi distance using the RAPIDS libraries &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cudf&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask_cudf&lt;/span&gt;&lt;/code&gt; on our GPU worker group.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HelmCluster&lt;/span&gt;&lt;/code&gt; cluster manager in Dask Kubernetes lets you scale your worker groups quickly from python. We scaled our GPU worker group by conveniently passing the worker group name as a keyword argument in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HelmCluster&lt;/span&gt;&lt;/code&gt; scale method.&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/2022/02/17/helm-multiple-worker-groups.md&lt;/span&gt;, line 164)&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;We’re thinking a lot about the concept of worker groups in the Dask community. Until now, most Dask deployments have homogenous workers, but as Dask users push Dask further, there is a growing demand for heterogeneous clusters with special-purpose workers. So we want to add worker groups throughout Dask.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2022/02/17/helm-multiple-worker-groups/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="Helm" label="Helm"/>
    <category term="Kubernetes" label="Kubernetes"/>
    <published>2022-02-17T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/12/15/dask-fellow-reflections/</id>
    <title>Reflections on one year as the Dask life science fellow</title>
    <updated>2021-12-15T00:00:00+00:00</updated>
    <author>
      <name>Genevieve Buckley</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/12/15/dask-fellow-reflections.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;&lt;a class="reference external" href="https://github.com/GenevieveBuckley/"&gt;Genevieve Buckley&lt;/a&gt; was hired as a Dask Life Science Fellow in 2021 &lt;a class="reference external" href="https://chanzuckerberg.com/eoss/proposals/"&gt;funded by CZI&lt;/a&gt;. The goal was to improve Dask, with a &lt;a class="reference external" href="https://blog.dask.org/2021/03/04/the-life-science-community"&gt;specific focus on the life science community&lt;/a&gt;. This blogpost contains another progress update, and some personal reflections looking back over this year.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/12/15/dask-fellow-reflections.md&lt;/span&gt;, line 13)&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="contents"&gt;
&lt;h1&gt;Contents&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#progress-update"&gt;&lt;span class="xref myst"&gt;Progress update&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#personal-reflections"&gt;&lt;span class="xref myst"&gt;Personal reflections&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#highlights-from-this-year"&gt;&lt;span class="xref myst"&gt;Highlights from this year&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#what-worked-well"&gt;&lt;span class="xref myst"&gt;What worked well&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#what-didnt-work-so-well"&gt;&lt;span class="xref myst"&gt;What didn’t work so well&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#whats-next-in-dask"&gt;&lt;span class="xref myst"&gt;What’s next in Dask?&lt;/span&gt;&lt;/a&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/2021/12/15/dask-fellow-reflections.md&lt;/span&gt;, line 22)&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="progress-update"&gt;
&lt;h1&gt;Progress update&lt;/h1&gt;
&lt;p&gt;A previous progress update for February to September 2021 is &lt;a class="reference external" href="https://blog.dask.org/2021/10/20/czi-eoss-update"&gt;available here&lt;/a&gt;. Read on for a progress update for the period September to December 2021.&lt;/p&gt;
&lt;p&gt;To summarize, between September and December 2021 inclusive, there were:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;32 merged pull requests acorss 7 repositories (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-image&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-tutorial&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ITK&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;napari&lt;/span&gt;&lt;/code&gt;, and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;napari.github.io&lt;/span&gt;&lt;/code&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;8 pending pull requests&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;1 new &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-image&lt;/span&gt;&lt;/code&gt; release&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;1 Dask tutorial run, and assisted with a second tutorial.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;4 new Dask blogposts published (five, if we count this one)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Read on for a more detailed description of special projects within this time.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Dask stale issues sprint&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;In two weeks I was able to:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;close 117 stale issues, and&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;identify another 25 potential easy wins for the maintainer team to investigate further.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Lots of other people did work around the same time, following up on old pull requests and other maintanence work. The sprint was very successful overall.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Dask user survey results analysis&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;In September I analyzed the results from the 2021 Dask user survey.
This was a really fun task. Because we asked a lot more questions in 2021 (18 new questions, 43 questions in total) there was was a lot more data to dig into, compared with previous years. You can read the &lt;a class="reference external" href="https://blog.dask.org/2021/09/15/user-survey"&gt;full details about it here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The biggest benefit from this work is that now we can use this data to prioritize improvements to the documentation and examples.
The top two user requests are for more documentation and more examples from their industry. But it wasn’t until this year that we started asking what industries people worked in, so we can target new narrative documentation to the areas that need it most (geoscience, life science, and finance).&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;ITK compatibility with Dask&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;I implemented &lt;a class="reference external" href="https://github.com/InsightSoftwareConsortium/ITK/pull/2829/"&gt;pickle serialization for itk images (ITK PR #2829)&lt;/a&gt;. This should be one of the last major pieces of the puzzle needed to make ITK images compatible with Dask. It builds on earlier work by Matt McCormick and John Kirkham (you can read a blog post about their earlier work &lt;a class="reference external" href="https://blog.dask.org/2019/08/09/image-itk"&gt;here&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;Better cross-compatibility for Dask with other projects was a major goal of mine, so this is an important piece of work. I outline the next steps in the section &lt;a class="reference internal" href="#whats-next-in-dask"&gt;&lt;span class="xref myst"&gt;What’s next in Dask?&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Improve rechunking&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;I implemented &lt;a class="reference external" href="https://github.com/dask/dask/pull/8124"&gt;PR #8124&lt;/a&gt; fix a bug where reshaping a Dask array can cause an output array with chunks that are much too large to fit in memory.
Feedback from the life science user survey indicates that improving Dask’s performance around rechunking is a priority. This work helps to address that.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;High level graph work&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;A major piece of work earlier this year was introducing high level graphs for array slicing and array overlap operations. That is a big effort requiring a lot of ongoing work.
&lt;a class="reference external" href="https://github.com/dask/dask/pull/8467"&gt;PR #8467&lt;/a&gt; tackles one of the next steps for this work.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Find objects function for dask-image&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;I implemented a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;find_objects&lt;/span&gt;&lt;/code&gt; function for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-image&lt;/span&gt;&lt;/code&gt; in &lt;a class="reference external" href="https://github.com/dask/dask-image/pull/240"&gt;PR #240&lt;/a&gt;. This implementation does not need to know the maximum label number ahead of time, a subtantial improement over the previous attempt. This is a major step forward, because it removes a major blocker to introducing scikit-image like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;regionprops&lt;/span&gt;&lt;/code&gt; functionality.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Blogposts&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;Dask blogposts published between September through to December 2021 include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2021/11/02/choosing-dask-chunk-sizes"&gt;Choosing good chunk sizes in Dask&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;This blogpost addresses some very common concerns and questions about using Dask.
I’m very pleased with this article, due to several thoughtful reviewers the final work is a much stronger and more comprehensive than the &lt;a class="reference external" href="https://twitter.com/DataNerdery/status/1424953376043790341"&gt;twitter thread&lt;/a&gt; that inspired it.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It’s also high impact work. In the Dask survey the most common request is for more documentation, and this content helps to address that. Twitter analytics also show much higher engagement with this content than for other similar tweets, indicating a demand in the community for this type of explanation.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2021/12/01/mosaic-fusion"&gt;Mosaic Image Fusion&lt;/a&gt; (co-authored with Volker Hisenstein and Marvin Albert)&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;This blogpost was several months in the making (started in mid-August and published in December). It’s fantastic to have people sharing some of the very cool work they do with Dask on real world problems.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2021/10/20/czi-eoss-update"&gt;CZI EOSS Update&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;This blogpost shares with the community an interim progress update provided to CZI.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2021/09/15/user-survey"&gt;2021 Dask user survey results&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Discussed in more detail above, the analysis results from the Dask User Survey were published in September 2021.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;strong&gt;Tutorials&lt;/strong&gt;&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;I presented a Dask tutorial at the &lt;a class="reference external" href="https://resbaz.github.io/resbaz2021/sydney/"&gt;ResBaz Sydney online conference&lt;/a&gt; on the 25th of November 2021. Thanks to the ResBaz organisers and to David McFarlane, Svetlana Tkachenko, and Oksana Tkachenko for monitoring the chat for questions on the day.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Naty Clementi ran a Dask tutorial for the Women Who Code DC meetup on the 4th of November 2021. I assisted Naty, mostly by monitoring questions in the chat.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/12/15/dask-fellow-reflections.md&lt;/span&gt;, line 93)&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="personal-reflections"&gt;
&lt;h1&gt;Personal reflections&lt;/h1&gt;
&lt;p&gt;Reflecting back over the whole year, there were some things that worked well and some things that were less successful.&lt;/p&gt;
&lt;section id="highlights-from-this-year"&gt;
&lt;h2&gt;Highlights from this year&lt;/h2&gt;
&lt;p&gt;My personal highlights include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;ITK + Dask integration work (discussed in more detail above).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A find objects fucntion for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-image&lt;/span&gt;&lt;/code&gt; (discussed in more detail above).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Visualization work, because it’s very high impact. We’re solving issues raised by life science groups, but the improved tools benefit EVERYONE who uses Dask.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;This bugfix from &lt;a class="reference external" href="https://github.com/dask/dask/pull/7391"&gt;dask PR #7391&lt;/a&gt;, because this single change fixed problems in four places at once (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scikit-image&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-ml&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;xgcm/xhistogram&lt;/span&gt;&lt;/code&gt;, and the cupy dask tests).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Community building, conferences, and engagement. Lots of effort went into events over this year, and it’s certainly paid dividends.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="what-worked-well"&gt;
&lt;h2&gt;What worked well&lt;/h2&gt;
&lt;p&gt;&lt;strong&gt;Dask stale issues sprint&lt;/strong&gt;&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;This was useful for the project, as well as useful for me.
Sorting through old issues was an incredibly effective way to get familiar with who the experts are for particular topics. It would have been even better if this happened in the first few months of working on Dask, instead of the last few months.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It’s been suggested that one good way to gain familiarity is spending 6 months full time managing the issue tracker. Maybe that’s true, but the much shorter stale issue sprint was a very efficient way of getting a lot of the same benefits in a short space of time. I’d recommend it for new maintainers or triage team members.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;strong&gt;Community building events&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;We had a very successful year in terms of community building and events. This included tutorials, workshops, conferences, and community outreach. Summary of major events:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Led a Dask tutorial at &lt;a class="reference external" href="https://resbaz.github.io/resbaz2021/sydney/"&gt;ResBaz Sydney 2021&lt;/a&gt; in November.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Co-led a half-day tutorial on napari and Dask at the &lt;a class="reference external" href="https://www.lmameeting.com.au/"&gt;Light Microscopy Australia Meeting&lt;/a&gt; in August.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;SciPy 2021 presentation &lt;a class="reference external" href="https://www.youtube.com/watch?v=tY_lCGS1BMk&amp;amp;amp;t=60s"&gt;Scaling Science: leveraging Dask for life sciences&lt;/a&gt; in July.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Organized the &lt;a class="reference external" href="https://blog.dask.org/2021/05/24/life-science-summit-workshop"&gt;Dask Life Science workshop&lt;/a&gt; at the Dask Summit in May 2021. The life science workshop included 15 pre-recorded talks, and 3 interactive discussions.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Co-organised the &lt;a class="reference external" href="https://blog.dask.org/2021/06/25/dask-down-under"&gt;Dask Down Under&lt;/a&gt; workshop for the Dask Summit in May 2021. Dask Down Under contained 5 talks, 2 tutorials, 1 panel discussion, and 1 meet and greet networking event.
Dask Down Under&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Expert panelist at the &lt;a class="reference external" href="https://www.vis2021.com.au/"&gt;VIS2021 symposium&lt;/a&gt; in February.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;strong&gt;Visualization work&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;This has been very high impact work, and I’m pleased with what we’ve achieved. Improved tools for visualization were requested by users in our survey of the life science community. This was a high priority, because improvements to visuzliation tools benefit EVERYONE who uses Dask.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="what-didn-t-work-so-well"&gt;
&lt;h2&gt;What didn’t work so well&lt;/h2&gt;
&lt;p&gt;&lt;strong&gt;Technical resources&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;We never really solved the problem of finding someone I could go to with technical questions. I did have people to ask about some specific projects, but in most cases I didn’t have a good way to direct questions to the right people. This is a challenging problem, especially because most Dask maintainers and contributors have full time jobs doing other things too. In my opinion, this negatively impacted the work and what we were able to achieve.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Being added to the &amp;#64;dask/maintenance team&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;There’s no point getting notifications if you don’t have GitHub permissions to do anything about them. In future I think we should add only people with at least triage or write permissions to the github teams.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Real time interaction&lt;/strong&gt;&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;We tried out “Ask a maintainer” office hours for the life science community, but they were poorly attended, so we cancelled this.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We added some “Dask social chat” events to the calendar, but they were not very well attended outside of the first few. Most often, zero people attended. (There is another social chat for the Americas/Europe time zones, which is at a more convenient time for most people and might be more popular.)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;strong&gt;Slack&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;Slack works well to DM specific people to set up meeting times, etc, but the public channels didn’t end up being very useful for me personally.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Lack of integration with other project teams&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;You can only get so much done as a solo developer. We had hoped that I would naturally end up working with teams from several different projects, but this didn’t really end up being the case. The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;napari&lt;/span&gt;&lt;/code&gt; project is an exception to this, and that relationship was well established before starting work for Dask. Perhaps there’s something more we could have done here to facilitate more interaction.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/12/15/dask-fellow-reflections.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&gt;
&lt;section id="what-s-next-for-genevieve"&gt;
&lt;h1&gt;What’s next for Genevieve?&lt;/h1&gt;
&lt;p&gt;Genevieve will be starting a new job next year, you can find her on GitHub &lt;a class="reference external" href="https://github.com/GenevieveBuckley/"&gt;&amp;#64;GeneviveeBuckley&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/2021/12/15/dask-fellow-reflections.md&lt;/span&gt;, line 158)&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-s-next-in-dask"&gt;
&lt;h1&gt;What’s next in Dask?&lt;/h1&gt;
&lt;p&gt;Lots of stuff has happened in Dask, but there is still lots left to do.
Here is a summary of the next steps for several projects. We’d love it if new people would like to take up the torch and contribute to any of these projects.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;ITK image compatibility with Dask&lt;/strong&gt;&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The next steps for the ITK + Dask project require ITK release candidate 5.3rc3 or above to become available (likely early in 2022).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;When the release is available, the next step is to try to re-run the code from the original &lt;a class="reference external" href="https://blog.dask.org/2019/08/09/image-itk"&gt;ITK blogpost&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;If there’s still work to be done we’ll need to open issues for the remaining blockers. And if it all works well, we’d like someone to write a second ITK + Dask blogpost to publicize the new functionality.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;strong&gt;Improving performance around rechunking&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;More performance improvements related to rechunking is required (see &lt;a class="reference external" href="https://github.com/dask/dask/pull/7950"&gt;#7950&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/dask/dask/pull/7980"&gt;#7980&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;High level graph work for arrays and slicing&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;The high level graph work for slicing and overlapping arrays has a number of next steps.
Ian Rose has written &lt;a class="reference external" href="https://gist.github.com/ian-r-rose/4221ebf52f3423203640c498fb815f21"&gt;an excellent summary here&lt;/a&gt;. Briefly, the&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cull&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;get_output_keys&lt;/span&gt;&lt;/code&gt; methods must be implemented, then low level fusion and optimizations can be done.&lt;/p&gt;
&lt;p&gt;Relevant links:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Implement &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cull&lt;/span&gt;&lt;/code&gt; method for ArrayOverlapLayer &lt;a class="reference external" href="https://github.com/dask/dask/issues/7789"&gt;#7789&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Implement &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;get_output_keys&lt;/span&gt;&lt;/code&gt; method for ArrayOverlapLayer &lt;a class="reference external" href="https://github.com/dask/dask/issues/7791"&gt;#7791&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/7655"&gt;Array slicing HighLevelGraph layer #7655&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;strong&gt;Documentation&lt;/strong&gt;&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Dask needs better documentation for high level graphs. Both &lt;a class="reference external" href="https://github.com/dask/dask/issues/7709"&gt;user documentation&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/dask/dask/issues/7755"&gt;developer documentation&lt;/a&gt; is required.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;At some future point, it might be worthwhile integrating blogpost content from
&lt;a class="reference external" href="https://blog.dask.org/2021/11/02/choosing-dask-chunk-sizes"&gt;Choosing good chunk sizes in Dask&lt;/a&gt; into the main &lt;a class="reference external" href="https://docs.dask.org/en/latest/"&gt;Dask documentation&lt;/a&gt;, for better discoverability.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/12/15/dask-fellow-reflections/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="lifescience" label="life science"/>
    <published>2021-12-15T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/12/01/mosaic-fusion/</id>
    <title>Mosaic Image Fusion</title>
    <updated>2021-12-01T00:00:00+00:00</updated>
    <author>
      <name>and Genevieve Buckley</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/12/01/mosaic-fusion.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;This blogpost shows a case study where a researcher uses Dask for mosaic image fusion.
Mosaic image fusion is when you combine multiple smaller images taken at known locations and stitch them together into a single image with a very large field of view. Full code examples are available on GitHub from the &lt;a class="reference external" href="https://github.com/VolkerH/DaskFusion"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskFusion&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; repository:
&lt;a class="github reference external" href="https://github.com/VolkerH/DaskFusion"&gt;VolkerH/DaskFusion&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/2021/12/01/mosaic-fusion.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&gt;
&lt;section id="the-problem"&gt;
&lt;h1&gt;The problem&lt;/h1&gt;
&lt;section id="image-mosaicing-in-microscopy"&gt;
&lt;h2&gt;Image mosaicing in microscopy&lt;/h2&gt;
&lt;p&gt;In optical microscopy, a single field of view captured with a 20x objective typically
has a diagonal on the order of a few 100 μm (exact dimensions depend on other
parts of the optical system, including the size of the camera chip). A typical
sample slide has a size of 25mm by 75mm.
Therefore, when imaging a whole slide, one has to acquire hundreds of images, typically
with some overlap between individual tiles. With increasing magnification,
the required number of images increases accordingly.&lt;/p&gt;
&lt;p&gt;To obtain an overview one has to fuse this large number of individual
image tiles into a large mosaic image. Here, we assume that the information required for
positioning and alignment of the individual image tiles is known. In the example presented here,
this information is available as metadata recorded by the microscope, namely the microscope stage
position and the pixel scale. Alternatively, this
information could also be derived from the image data directly, e.g. through a
registration step that matches corresponding image features in the areas where tiles overlap.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/12/01/mosaic-fusion.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&gt;
&lt;section id="the-solution"&gt;
&lt;h1&gt;The solution&lt;/h1&gt;
&lt;p&gt;The array that can hold the resulting mosaic image will often have a size that is too large
to fit in RAM, therefore we will use Dask arrays and the &lt;a class="reference external" href="https://docs.dask.org/en/latest/generated/dask.array.map_blocks.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; function to enable
out-of-core processing. The &lt;a class="reference external" href="https://docs.dask.org/en/latest/generated/dask.array.map_blocks.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;
function will process smaller blocks (a.k.a chunks) of the output array individually, thus eliminating the need to
hold the whole output array in memory. If sufficient resources are available, dask will also distribute the processing of blocks across several workers,
thus we also get parallel processing for free, which can help speed up the fusion process.&lt;/p&gt;
&lt;p&gt;Typically whenever we want to join Dask arrays, we use &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-stack.html"&gt;Stack, Concatenate, and Block&lt;/a&gt;. However, these are not good tools for mosaic image fusion, because:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;The image tiles will be be overlapping,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tiles may not be positioned on an exact grid and will typically also have slight rotations as the alignment of stage and camera is not perfect. In the most general case, for example in panaromic photo mosaics,
individual image tiles could be arbitrarily rotated or skewed.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The starting point for this mosaic prototype was some code that reads in the stage metadate for all tiles and calculates an affine transformation for each tile that would place it at the correct location
in the output array.&lt;/p&gt;
&lt;p&gt;The image below shows preliminary work placing mosaic image tiles into the correct positions using the napari image viewer.
Shown here is a small example with 63 image tiles.&lt;/p&gt;
&lt;img src="/images/mosaic-fusion/NapariMosaics.png" alt="Mosaic fusion images in the napari image viewer" width="700" height="265"&gt;
&lt;p&gt;And here is an animation of placing the individual tiles.&lt;/p&gt;
&lt;img src="/images/mosaic-fusion/Lama_whole_slide.gif" alt="Animation of whole slide mosaic fusion images" width="700" height="361"&gt;
&lt;p&gt;To leverage processing with Dask we created a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;fuse&lt;/span&gt;&lt;/code&gt; function that generates a small block of the final mosaic and is invoked by &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt; for each chunk of the output array.
On each invocation of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;fuse&lt;/span&gt;&lt;/code&gt; function &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt; passes a dictionary (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;block_info&lt;/span&gt;&lt;/code&gt;). From the &lt;a class="reference external" href="https://docs.dask.org/en/latest/generated/dask.array.map_blocks.html?highlight=block_info#dask.array.map_blocks"&gt;Dask documentation&lt;/a&gt;:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Your block function gets information about where it is in the array by accepting a special &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;block_info&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;block_id&lt;/span&gt;&lt;/code&gt; keyword argument.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;The basic outline of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;fuse&lt;/span&gt;&lt;/code&gt; function of the mosaic workflow is as follows.
For each chunk of the output array:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Determine which source image tiles intersect with the chunk.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Adjust the image tiles’ affine transformations to take the offset of the chunk within the array into account.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Load all intersectiong image tiles and apply their respective adjusted affine transformation to map them into the chunk.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Blend the tiles using a simple maximum projection.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Return the blended chunk.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Using a maximum projection to blend areas with overlapping tiles can lead to artifacts such as ghost images and visible tile
seams, so you would typically want to use something more sophisticated in production.&lt;/p&gt;
&lt;section id="results"&gt;
&lt;h2&gt;Results&lt;/h2&gt;
&lt;p&gt;For datasets with many image tiles (~500-1000 tiles), we could speed up the mosaic generation from several hours to tens of minutes using this Dask based method
(compared to a previous workflow using ImageJ plugins runnning on the same workstation).
Due to Dask’s ability to handle data out-of-core and chunked array storage using zarr it is also possible to run the
fusion on hardware with limited RAM.&lt;/p&gt;
&lt;p&gt;Finally, we have the final mosaic fusion result.&lt;/p&gt;
&lt;img src="/images/mosaic-fusion/final-mosaic-fusion-result.png" alt="Final mosaic fusion result" width="700" height="486"&gt;
&lt;/section&gt;
&lt;section id="code"&gt;
&lt;h2&gt;Code&lt;/h2&gt;
&lt;p&gt;Code relatiing to this mosaic image fusion project can be found in the &lt;a class="reference external" href="https://github.com/VolkerH/DaskFusion"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DaskFusion&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; GitHub repository here:
&lt;a class="github reference external" href="https://github.com/VolkerH/DaskFusion"&gt;VolkerH/DaskFusion&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;There is a self-contained example available in &lt;a class="reference external" href="https://github.com/VolkerH/DaskFusion/blob/main/DaskFusion_Example.ipynb"&gt;this notebook&lt;/a&gt;, which downloads reduced-size example data to demonstrate the process.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/12/01/mosaic-fusion.md&lt;/span&gt;, line 97)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="what-s-next"&gt;
&lt;h1&gt;What’s next?&lt;/h1&gt;
&lt;p&gt;Currently, the DaskFusion code is a proof of concept for single-channel 2D images and simple maximum projection for blending the tiles in overlapping areas, it is not production code.
However, the same principle can be used for fusing multi-channel image volumes,
such as from Light-Sheet data if the tile chunk intersection calculation is extended to higher-dimensional arrays.
Such even larger datasets will benefit even more from leveraging dask,
as the processing can be distributed across multiple nodes of a HPC cluster using &lt;a class="reference external" href="http://jobqueue.dask.org/en/latest/"&gt;dask jobqueue&lt;/a&gt;.&lt;/p&gt;
&lt;section id="also-see"&gt;
&lt;h2&gt;Also see&lt;/h2&gt;
&lt;p&gt;Marvin’s lightning talk on multi-view image fusion:
&lt;a class="reference external" href="https://www.youtube.com/watch?v=YIblUvonMvo&amp;amp;amp;list=PLJ0vO2F_f6OBAY6hjRHM_mIQ9yh32mWr0&amp;amp;amp;index=10"&gt;15 minute video available here on YouTube&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;The GitHub repository &lt;a class="reference external" href="https://github.com/m-albert/MVRegFus"&gt;MVRegFus&lt;/a&gt; that Marvin talks about in the video is available here:
&lt;a class="github reference external" href="https://github.com/m-albert/MVRegFus"&gt;m-albert/MVRegFus&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://github.com/manzt/napari-lazy-openslide"&gt;napari-lazy-openslide&lt;/a&gt; visualization plugin by &lt;a class="reference external" href="https://github.com/manzt"&gt;Trevor Manz&lt;/a&gt;: &lt;em&gt;“An experimental plugin to lazily load multiscale whole-slide tiff images with openslide and dask.”&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;For further information on alternative approaches to image stitching:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;ASHLAR: Alignment by Simultaneous Harmonization of Layer / Adjacency Registration&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://labsyspharm.github.io/ashlar/"&gt;ASHLAR homepage&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/labsyspharm/ashlar"&gt;ASHLAR GitHub repository&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://doi.org/10.1101/2021.04.20.440625"&gt;ASHLAR biorxiv pre-print&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Microscopy Image Stitching Tool (MIST)&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://pages.nist.gov/MIST/"&gt;MIST homepage&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/usnistgov/MIST"&gt;MIST GitHub repository&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://raw.githubusercontent.com/wiki/USNISTGOV/MIST/assets/mist-algorithm-documentation.pdf"&gt;MIST algorithm documentation (PDF)&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The &lt;a class="reference external" href="https://github.com/yfukai/m2stitch"&gt;m2stitch&lt;/a&gt; python package by &lt;a class="reference external" href="https://github.com/yfukai"&gt;Yohsuke T. Fukai&lt;/a&gt;: &lt;em&gt;“Provides robust stitching of tiled microscope images on a regular grid”&lt;/em&gt; (based on the MIST algorithm)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/12/01/mosaic-fusion.md&lt;/span&gt;, line 127)&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="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;This computational work was done by Volker Hilsenstein, in conjunction with Marvin Albert.
Volker Hilsenstein is a scientific software developer at &lt;a class="reference external" href="https://www.embl.org/groups/alexandrov/"&gt;EMBL in Theodore Alexandrov’s lab&lt;/a&gt; with a focus on spatial metabolomics and bio-image analysis.&lt;/p&gt;
&lt;p&gt;The sample images were prepared and imaged by Mohammed Shahraz from the Alexandrov lab at EMBL Heidelberg.&lt;/p&gt;
&lt;p&gt;Genevieve Buckley and Volker Hilsenstein wrote this blogpost.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/12/01/mosaic-fusion/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="imageanalysis" label="image analysis"/>
    <category term="lifescience" label="life science"/>
    <published>2021-12-01T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/11/02/choosing-dask-chunk-sizes/</id>
    <title>Choosing good chunk sizes in Dask</title>
    <updated>2021-11-02T00:00:00+00:00</updated>
    <author>
      <name>Genevieve Buckley</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;Confused about choosing &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-best-practices.html#select-a-good-chunk-size"&gt;a good chunk size&lt;/a&gt; for Dask arrays?&lt;/p&gt;
&lt;p&gt;Array chunks can’t be too big (we’ll run out of memory), or too small (the overhead introduced by Dask becomes overwhelming). So how can we get it right?&lt;/p&gt;
&lt;p&gt;It’s a two step process:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;First, start by choosing a chunk size similar to data you know can be processed entirely within memory (i.e. without Dask), using these &lt;a class="reference internal" href="#rough-rules-of-thumb"&gt;&lt;span class="xref myst"&gt;rough rules of thumb&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Then, watch the Dask dashboard task stream and worker memory plots, and adjust if needed. &lt;a class="reference internal" href="#what-to-watch-for-on-the-dashboard"&gt;&lt;span class="xref myst"&gt;Here are the signs to watch out for&lt;/span&gt;&lt;/a&gt;.&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/2021/11/02/choosing-dask-chunk-sizes.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&gt;
&lt;section id="contents"&gt;
&lt;h1&gt;Contents&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#what-are-dask-array-chunks"&gt;&lt;span class="xref myst"&gt;What are Dask array chunks?&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#too-small-is-a-problemg"&gt;&lt;span class="xref myst"&gt;Too small is a problem&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#too-big-is-also-a-problem"&gt;&lt;span class="xref myst"&gt;Too big is also a problem&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#choosing-an-initial-chunk-size"&gt;&lt;span class="xref myst"&gt;Choosing an initial chunk size&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#rough-rules-of-thumb"&gt;&lt;span class="xref myst"&gt;Rough rules of thumb&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#chunks-should-be-aligned-with-array-storage-on-disk"&gt;&lt;span class="xref myst"&gt;Chunks should be aligned with array storage on disk&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#using-the-dask-dashboard"&gt;&lt;span class="xref myst"&gt;Using the Dask dashboard&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#what-to-watch-for-on-the-dashboard"&gt;&lt;span class="xref myst"&gt;What to watch for on the dashboard
&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#rechunking-arrays"&gt;&lt;span class="xref myst"&gt;Rechunking arrays&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#unmanaged-memory"&gt;&lt;span class="xref myst"&gt;Unmanaged memory&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#thanks-for-reading"&gt;&lt;span class="xref myst"&gt;Thanks for reading&lt;/span&gt;&lt;/a&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/2021/11/02/choosing-dask-chunk-sizes.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="what-are-dask-array-chunks"&gt;
&lt;h1&gt;What are Dask array chunks?&lt;/h1&gt;
&lt;p&gt;Dask arrays are big structures, made out of many small chunks.
Typically, each small chunk is an individual &lt;a class="reference external" href="https://numpy.org/"&gt;numpy array&lt;/a&gt;, and they are arranged together to make a much larger Dask array.&lt;/p&gt;
&lt;img src="https://raw.githubusercontent.com/dask/dask/ac01ddc9074365e40d888f80f5bcd955ba01e872/docs/source/images/dask-array-black-text.svg" alt="Diagram: Dask array chunks" width="400" height="300" /&gt;
&lt;p&gt;You can find more information about Dask array chunks on this page of the documentation: &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-chunks.html"&gt;https://docs.dask.org/en/latest/array-chunks.html&lt;/a&gt;&lt;/p&gt;
&lt;section id="how-do-i-know-what-chunks-my-array-has"&gt;
&lt;h2&gt;How do I know what chunks my array has?&lt;/h2&gt;
&lt;p&gt;If you have a Dask array, you can use the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chunksize&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chunks&lt;/span&gt;&lt;/code&gt; attribues to see information about the chunks. You can also visualize this with the Dask array HTML representation.&lt;/p&gt;
&lt;img src="/images/choosing-good-chunk-sizes/examine-dask-array-chunks.png" alt="Visualizating Dask array chunks with the HTML repr" width="611" height="523" /&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;arr.chunksize&lt;/span&gt;&lt;/code&gt; shows the largest chunk size. For arrays where you expect roughly uniform chunk sizes, this is a good way to summarize chunk size information.&lt;/p&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;arr.chunks&lt;/span&gt;&lt;/code&gt; shows fully explicit sizes of all chunks along all dimensions within the Dask array (see &lt;a class="reference external" href="https://docs.dask.org/en/stable/array-chunks.html#specifying-chunk-shapes"&gt;item 3 here&lt;/a&gt;). This is more verbose, and is a good choice with arrays that have irregular chunks.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 54)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="too-small-is-a-problem"&gt;
&lt;h1&gt;Too small is a problem&lt;/h1&gt;
&lt;p&gt;If array chunks are too small, it’s inefficient. Why is this?&lt;/p&gt;
&lt;p&gt;Using Dask introduces some amount of overhead for each task in your computation.
This overhead is the reason the Dask best practices advise you to &lt;a class="reference external" href="https://docs.dask.org/en/latest/best-practices.html#avoid-very-large-graphs"&gt;avoid too-large graphs&lt;/a&gt;.
This is because if the amount of actual work done by each task is very tiny, then the percentage of overhead time vs useful work time is not good.&lt;/p&gt;
&lt;p&gt;Typically, the Dask scheduler takes 1 millisecond to coordinate a single task. That means we want the computation time for each task to be comparitively larger, eg: seconds instead of milliseconds.&lt;/p&gt;
&lt;p&gt;It might be hard to understand this intuitively, so here’s an analogy. Let’s imagine we’re building a house. It’s a pretty big job, and if there were only one worker it would take much too long to build.
So we have a team of workers and a site foreman. The site foreman is equivalent to the Dask scheduler: their job is to tell the workers what tasks they need to do.&lt;/p&gt;
&lt;p&gt;Say we have a big pile of bricks to build a wall, sitting in the corner of the building site.
If the foreman (the Dask scheduler) tells workers to go and fetch a single brick at a time, then bring each one to where the wall is being built, you can see how this is going to be very slow and inefficient! The workers are spending most of their time moving between the wall and the pile of bricks. Much less time is going towards doing the actual work of mortaring bricks onto the wall.&lt;/p&gt;
&lt;p&gt;Instead, we can do this in a smarter way. The foreman (Dask scheduler) can tell the workers to go and bring one full wheelbarrow load of bricks back each time. Now workers are spending much less time moving between the wall and the pile of bricks, and the wall will be finished much quicker.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 72)&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="too-big-is-also-a-problem"&gt;
&lt;h1&gt;Too big is also a problem&lt;/h1&gt;
&lt;p&gt;If the Dask array chunks are too big, this is also bad. Why?
Chunks that are too large are bad because then you are likely to run out of working memory.
You may see out of memory errors happening, or you might see performance decrease substantially as data spills to disk.&lt;/p&gt;
&lt;p&gt;When too much data is loaded in memory on too few workers, Dask will try to spill data to disk instead of crashing.
Spilling data to disk makes things run very slowly, because all the extra read/write operations to disk. Things don’t just get a little bit slower, they get a LOT slower, so it’s smart to watch out for this.&lt;/p&gt;
&lt;p&gt;To watch out for this, look at the &lt;strong&gt;worker memory plot&lt;/strong&gt; on the Dask dashboard.
Orange bars are a warning you are close to the limit, and gray means data is being spilled to disk - not good!
For more tips, see the section on &lt;a class="reference internal" href="#using-the-Dask-dashboard"&gt;&lt;span class="xref myst"&gt;using the Dask dashboard&lt;/span&gt;&lt;/a&gt; below.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 85)&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="choosing-an-initial-chunk-size"&gt;
&lt;h1&gt;Choosing an initial chunk size&lt;/h1&gt;
&lt;section id="rough-rules-of-thumb"&gt;
&lt;h2&gt;Rough rules of thumb&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;If you already created a prototype, which may not involve Dask at all, using a small subset of the data you intend to process, you’ll have a clear idea of what size of data can be processed easily for this workflow. You can use this knowledge to choose similar sized chunks in Dask.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Some people have observed that chunk sizes below 1MB are almost always bad. Chunk size between 100MB and 1GB are generally good, going over 1 or 2GB means you have a really big dataset and/or a lot of memory available per core,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Upper bound: Avoid too large task graphs. More than 10,000 or 100,000 chunks may start to perform poorly.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Lower bound: To get the advantage of parallelization, you need the number of chunks to at least equal the number of worker cores available (or better, the number of worker cores times 2). Otherwise, some workers will stay idle.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The time taken to compute each task should be much larger than the time needed to schedule the task. The Dask scheduler takes roughly 1 millisecond to coordinate a single task, so a good task computation time would be measured in seconds (not milliseconds).&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="chunks-should-be-aligned-with-array-storage-on-disk"&gt;
&lt;h2&gt;Chunks should be aligned with array storage on disk&lt;/h2&gt;
&lt;p&gt;If you are reading data from disk, the storage structure will inform what shape your Dask array chunks should be. For best performance, choose chunks that are well aligned with the way data is stored.&lt;/p&gt;
&lt;p&gt;From the Dask best practices on how to &lt;a class="reference external" href="https://docs.dask.org/en/stable/array-best-practices.html#orient-your-chunks"&gt;orient your chunks&lt;/a&gt;:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;When reading data you should align your chunks with your storage format. Most array storage formats store data in chunks themselves. If your Dask array chunks aren’t multiples of these chunk shapes then you will have to read the same data repeatedly, which can be expensive. Note though that often storage formats choose chunk sizes that are much smaller than is ideal for Dask, closer to 1MB than 100MB. In these cases you should choose a Dask chunk size that aligns with the storage chunk size and that every Dask chunk dimension is a multiple of the storage chunk dimension.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;Some examples of data storage structures on disk include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;A HDF5 or &lt;a class="reference external" href="https://zarr.readthedocs.io/en/stable/api/core.html"&gt;Zarr array&lt;/a&gt;. The size and shape of chunks/blocks stored on disk should align well with the Dask array chunks you select.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A folder full of tiff files. You might decide that each tiff file should become a single chunk in the Dask array (or that multiple tiff files should be grouped into a single chunk).&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 108)&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="using-the-dask-dashboard"&gt;
&lt;h1&gt;Using the Dask dashboard&lt;/h1&gt;
&lt;p&gt;The second part of choosing a good chunk size is monitoring the Dask dashboard to see if you need to make any adjustments.&lt;/p&gt;
&lt;p&gt;If you’re not very familiar with the Dask dashboard, or you just sometimes forget where to find certain dashboard plots (like the worker memory plot), then you’ll probably enjoy these quick video tutorials:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://www.youtube.com/watch?v=N_GqzcuGLCY"&gt;Intro to the Dask dashboard (18 minute video)&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://www.youtube.com/watch?v=EX_voquHdk0"&gt;Dask Jupyterlab extension (6 minute video)&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://distributed.dask.org/en/latest/diagnosing-performance.html"&gt;Dask dashboard documentation&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We recommend always having the dashboard up when you’re working with Dask.
It’s a fantastic way to get a sense of what’s working well, or poorly, so you can make adjustments.&lt;/p&gt;
&lt;section id="what-to-watch-for-on-the-dashboard"&gt;
&lt;h2&gt;What to watch for on the dashboard&lt;/h2&gt;
&lt;p&gt;Bad signs to watch out for include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Lots of white space in the task stream plot is a bad sign. White space means nothing is happening. Chunks may be too small.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Lots and lots of red in the task stream plot is a bad sign. Red means worker communication. Dask workers need some communication, but if they are doing almost nothing except communication then there is not much productive work going on.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;On the worker memory plot, watch out for orange bars which are a sign you are getting close to the memory limit. Chunks may be too big.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;On the worker memory plot, watch out for grey bars which mean data is being spilled to disk. Chunks may be too big.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Here is an example of the Dask dashboard during a good computation (&lt;a class="reference external" href="https://youtu.be/N_GqzcuGLCY?t=372"&gt;time 6:12 in this video&lt;/a&gt;).
&lt;img alt="Visualizating Dask array chunks with the HTML repr" src="https://blog.dask.org/_images/good-dask-dashboard.png" /&gt;&lt;/p&gt;
&lt;p&gt;For comparison, here is an example of the Dask dashboard during a bad computation (&lt;a class="reference external" href="https://youtu.be/N_GqzcuGLCY?t=417"&gt;time 6:57 in this video&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;In this example, it’s inefficient because the chunks are much too small, so we see a lot of white space and red worker communication in the task stream plot.
&lt;img alt="Visualizating Dask array chunks with the HTML repr" src="https://blog.dask.org/_images/bad-dask-dashboard-zoomedin.png" /&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 138)&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="rechunking-arrays"&gt;
&lt;h1&gt;Rechunking arrays&lt;/h1&gt;
&lt;p&gt;If you need to change the chunking of a Dask array in the middle of a computation, you can do that with the &lt;a class="reference external" href="https://docs.dask.org/en/latest/generated/dask.array.rechunk.html"&gt;rechunk&lt;/a&gt; method.&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;rechunked_array&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;original_array&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;new_shape&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Warning:&lt;/strong&gt; Rechunking Dask arrays comes at a cost.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The Dask graph must be rearranged to accomodate the new chunk structure. This happens immediately, and will block any other interaction with python until Dask has rearranged the task graph.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;This also inserts new tasks into the Dask graph. At compute time, there are now more tasks to execute.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;For these reasons, it is best to choose a good initial chunk size and avoid rechunking.&lt;/p&gt;
&lt;p&gt;However, sometimes the data is stored on disk is not well aligned and rechunking may be necessary.
For an example of this, here is Draga Doncila Pop &lt;a class="reference external" href="https://youtu.be/10Ws59NGDaE?t=833"&gt;talking about chunk alignment&lt;/a&gt; with satellite image data.&lt;/p&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://rechunker.readthedocs.io/en/latest/"&gt;rechunker&lt;/a&gt; library can be useful in these situations:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Rechunker takes an input array (or group of arrays) stored in a persistent storage device (such as a filesystem or a cloud storage bucket) and writes out an array (or group of arrays) with the same data, but different chunking scheme, to a new location. Rechunker is designed to be used within a parallel execution framework such as Dask.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 160)&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="unmanaged-memory"&gt;
&lt;h1&gt;Unmanaged memory&lt;/h1&gt;
&lt;p&gt;Last, remember that you don’t only need to consider the size of the array chunks in memory, but also the working memory consumed by your analysis functions. Sometimes that is called “unmanaged memory” in Dask.&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;“Unmanaged memory is RAM that the Dask scheduler is not directly aware of and which can cause workers to run out of memory and cause computations to hang and crash.” – Guido Imperiale&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;Here are some tips for handling unmanaged memory:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://coiled.io/blog/tackling-unmanaged-memory-with-dask/"&gt;Tackling unmanaged memory with Dask (Coiled blogpost)&lt;/a&gt; by Guido Imperiale&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://youtu.be/nwR6iGR0mb0"&gt;Handle Unmanaged Memory in Dask (8 minute video)&lt;/a&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/2021/11/02/choosing-dask-chunk-sizes.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="thanks-for-reading"&gt;
&lt;h1&gt;Thanks for reading&lt;/h1&gt;
&lt;p&gt;We hope this was helpful figuring out how to choose good chunk sizes for Dask. This blogpost was inspired by &lt;a class="reference external" href="https://twitter.com/DataNerdery/status/1424953376043790341"&gt;this twitter thread&lt;/a&gt;. If you’d like to follow Dask on Twitter, you can do that at &lt;a class="reference external" href="https://twitter.com/dask_dev"&gt;https://twitter.com/dask_dev&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/11/02/choosing-dask-chunk-sizes/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="performance" label="performance"/>
    <published>2021-11-02T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/10/20/czi-eoss-update/</id>
    <title>CZI EOSS Update</title>
    <updated>2021-10-20T00:00:00+00:00</updated>
    <author>
      <name>Genevieve Buckley</name>
    </author>
    <content type="html">&lt;p&gt;Dask was awarded funding last year in round 2 of the &lt;a class="reference external" href="https://chanzuckerberg.com/eoss/proposals/"&gt;CZI Essential Open Source Software&lt;/a&gt; grant program.
That funding was used to hire &lt;a class="reference external" href="https://github.com/GenevieveBuckley/"&gt;Genevieve Buckley&lt;/a&gt; to work on Dask with a focus on &lt;a class="reference external" href="https://blog.dask.org/2021/03/04/the-life-science-community"&gt;life sciences&lt;/a&gt;.
Last month Dask submitted an interim progress report to CZI, covering the period from February to September 2021.
That progress update is published verbatim below, to share with the wider Dask community.&lt;/p&gt;
&lt;hr class="docutils" /&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/10/20/czi-eoss-update.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="progress-overview"&gt;

&lt;section id="brief-summary"&gt;
&lt;h2&gt;Brief summary&lt;/h2&gt;
&lt;p&gt;The scope of work performed by the Dask fellow includes code contributions, conference presentations and tutorials, community engagement, and outreach including blogposts.&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;The primary deliverable of this proposal is consistency and the success of neighboring software
projects&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;Project work to date includes:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;38 pull requests merged (plus 6 draft pull requests) across 5 different repositories.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;3 conferences (presentations and organising of specialist workshops)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;1 half day workshop (plus another one upcoming)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Student supervision for Dask’s Google Summer of Code project&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;9 blogposts (plus 2 drafts for upcoming publication)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="code-contributions"&gt;
&lt;h2&gt;Code contributions&lt;/h2&gt;
&lt;p&gt;Code contributions are not limiteed to the main Dask repository, but also neighbouring software projects which use Dask as well (like the &lt;a class="reference external" href="https://napari.org/"&gt;napari&lt;/a&gt; software project), including: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-image&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-examples&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;napari&lt;/span&gt;&lt;/code&gt;, &amp;amp; &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;napari.github.io&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;To date, across the five repositories named above the Dask fellow has contributed:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;38 pull requests&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;6 draft pull requests&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;12 closed pull requests (not merged, discarded in favour of another approach)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The Dask fellow is an official maintainer of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-image&lt;/span&gt;&lt;/code&gt; project, and additional milestones achieved for that project include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The maintainer team has been grown by one (we welcome Marvin Albert to our ranks)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;2 new dask-image releases in 2020&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="code-contribution-highlights"&gt;
&lt;h2&gt;Code contribution highlights&lt;/h2&gt;
&lt;p&gt;Highlights include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Bugfixes benefitting the broader community&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/7391"&gt;dask PR #7391&lt;/a&gt;: This PR fixed slicing the output from Dask’s bincount function. The impact of this fix was substantial, as it solved issues filed in four separate projects: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scikit-image&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-ml&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;xgcm/xhistogram&lt;/span&gt;&lt;/code&gt; and the cupy dask tests.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Expanded GPU support&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/6680"&gt;dask PR #6680&lt;/a&gt;: This PR provided support for different array types in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;*_like&lt;/span&gt;&lt;/code&gt; array creation functions. Now users can create &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cupy&lt;/span&gt;&lt;/code&gt; like Dask arrays for GPU processing, or indeed any other array type (eg: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sparse&lt;/span&gt;&lt;/code&gt;).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-image/pull/157"&gt;dask-image PR #157&lt;/a&gt;: This PR provided GPU support for binary morphological functions in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-image&lt;/span&gt;&lt;/code&gt; project.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Visualization tools benefitting all Dask users&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/7716"&gt;dask PR #7716&lt;/a&gt;: This PR automatically displays the high level graph visualization in the jupyter notebook cell output (somthing already done automatically for low level graphs).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/7763"&gt;dask PR #7763&lt;/a&gt;: This PR introduced a HTML representation for Dask &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HighLevelGraph&lt;/span&gt;&lt;/code&gt; objects. This allows users and developers a much easier way to inspect the structure and status of HighLevelGraphs.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Further developed on during the Dask Google Summer of Code project, full report available &lt;a class="reference external" href="https://blog.dask.org/2021/08/23/gsoc-2021-project"&gt;here&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;High Level Graphs&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/7595"&gt;dask PR #7595&lt;/a&gt;: This PR introduced a high level graph layer for array overlaps. High level graphs are a tool we can use to optimize Dask’s performance.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/7655"&gt;dask PR #7655&lt;/a&gt; (ongoing): This PR introduces a high level graph for Dask array slicing operations.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Memory improvements (ongoing)&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/8124"&gt;dask PR #8124&lt;/a&gt; (ongoing): This PR investigates improved automatic rechunking strategies for &lt;a class="reference external" href="https://github.com/dask/dask/issues/8110"&gt;memory problems&lt;/a&gt; caused by reshaping Dask arrays.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/7950"&gt;dask PR #7950&lt;/a&gt; (ongoing): This PR aims to improve memory and performance of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;tensordot&lt;/span&gt;&lt;/code&gt; function with auto-rechunking of Dask arrays.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/7980"&gt;dask PR #7980&lt;/a&gt; (ongoing): This PR aims to fix the unbounded memory use problem in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;tensordot&lt;/span&gt;&lt;/code&gt;, reported &lt;a class="reference external" href="https://github.com/dask/dask/issues/6916"&gt;here&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="conferences"&gt;
&lt;h2&gt;Conferences&lt;/h2&gt;
&lt;p&gt;Notable conference events in 2021 included the SciPy conference, the Dask Summit, and VIS2021.&lt;/p&gt;
&lt;section id="scipy-conference"&gt;
&lt;h3&gt;SciPy conference&lt;/h3&gt;
&lt;p&gt;The Dask fellow presented a talk titled &lt;em&gt;“Scaling Science: leveraging Dask for life sciences”&lt;/em&gt; at the 2021 SciPy conference. Full recording &lt;a class="reference external" href="https://www.youtube.com/watch?v=tY_lCGS1BMk&amp;amp;amp;t=60s"&gt;available here&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-summit"&gt;
&lt;h3&gt;Dask Summit&lt;/h3&gt;
&lt;p&gt;The Dask fellow organised two workshops at the 2021 &lt;a class="reference external" href="https://summit.dask.org/"&gt;Dask Summit&lt;/a&gt;:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Dask Down Under (co-organised with Nick Mortimer), and&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The Dask life science workshop&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;section id="dask-down-under"&gt;
&lt;h4&gt;Dask Down Under&lt;/h4&gt;
&lt;p&gt;The scope of Dask Down Under was more like a mini-conference for Australian timezones, rather than a typical workshop. Dask Down Under involved two days of events, covering:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;5 talks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;2 tutorials&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;1 panel discussion&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;1 meet and greet networking event&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;It was very well recieved by the community. A full report on the Dask Down under events is available &lt;a class="reference external" href="https://blog.dask.org/2021/06/25/dask-down-under"&gt;here&lt;/a&gt;. A YouTube playlist of the Dask Down Under events is available &lt;a class="reference external" href="https://www.youtube.com/watch?v=10Ws59NGDaE&amp;amp;amp;list=PLJ0vO2F_f6OAXBfb_SAF2EbJve9k1vkQX"&gt;here on the Dask YouTube channel&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-life-science-workshop"&gt;
&lt;h4&gt;Dask life science workshop&lt;/h4&gt;
&lt;p&gt;The Dask life science workshop involved:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;15 pre-recorded lightning talks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;3 interactive discussion times (accessible across timezones in Europe, Oceania, and the Americas)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Asynchronous text chat throughout the Dask Summit&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;A full report on the Dask life science workshop is available &lt;a class="reference external" href="https://blog.dask.org/2021/05/24/life-science-summit-workshop"&gt;here&lt;/a&gt;. A YouTube playlist of all the Dask life science lightning talks is available &lt;a class="reference external" href="https://www.youtube.com/watch?v=6PerbQhcupM&amp;amp;amp;list=PLJ0vO2F_f6OBAY6hjRHM_mIQ9yh32mWr0"&gt;here on the Dask YouTube channel&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="vis2021-symposium"&gt;
&lt;h4&gt;VIS2021 symposium&lt;/h4&gt;
&lt;p&gt;The Dask fellow was an invited panellist at the &lt;a class="reference external" href="https://www.vis2021.com.au/"&gt;VIS2021 symposium&lt;/a&gt; in February 2021. The “Problem Solver” panel discussion covered practical problems in image analysis and how tools like Dask and napari can help solve them.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="tutorials-and-workshops"&gt;
&lt;h2&gt;Tutorials and workshops&lt;/h2&gt;
&lt;p&gt;The Dask fellow co-presented a half-day workshop (five hours) at the 2021 &lt;a class="reference external" href="https://www.lmameeting.com.au/"&gt;Light Microscopy Australia Meeting&lt;/a&gt; with Juan Nunez-Iglesias. &lt;a class="reference external" href="https://napari.org/"&gt;napari&lt;/a&gt; is an open source multidimensional image viewer built using Dask for out-of-core image processing. Workshop content is available at this link: &lt;a class="github reference external" href="https://github.com/jni/lma-2021-bioimage-analysis-python/"&gt;jni/lma-2021-bioimage-analysis-python&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Upcoming workshop:&lt;/strong&gt;
The Dask fellow has been invited to deliver a workshop on &lt;a class="reference external" href="https://napari.org/"&gt;napari&lt;/a&gt; and big data using &lt;a class="reference external" href="https://dask.org/"&gt;Dask&lt;/a&gt; at an upcoming &lt;a class="reference external" href="http://eubias.org/NEUBIAS/training-schools/neubias-academy-home/"&gt;NEUBIAS Academy&lt;/a&gt;. Workshop content is available at this link: &lt;a class="github reference external" href="https://github.com/GenevieveBuckley/napari-big-data-training"&gt;GenevieveBuckley/napari-big-data-training&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="google-summer-of-code"&gt;
&lt;h2&gt;Google Summer of Code&lt;/h2&gt;
&lt;p&gt;The Dask fellow supervised a Google Summer of Code student in 2021. Martin Durant acted as a secondary supervisor. The project ran over a 3 month period, and involved implementing a number of features to improve visualization of Dask graphs and objects. A full report on the Dask GSOC project is available &lt;a class="reference external" href="https://blog.dask.org/2021/08/23/gsoc-2021-project"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="blogposts"&gt;
&lt;h2&gt;Blogposts&lt;/h2&gt;
&lt;p&gt;We set a goal of one blogpost per month, and exceeded it. To date, nine blogposts have been published by the Dask fellow, with another two currently in draft status.&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2021/03/04/the-life-science-community"&gt;Getting to know the life science community&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2021/03/29/apply-pretrained-pytorch-model"&gt;Dask with PyTorch for large scale image analysis&lt;/a&gt; (co-authored with Nick Sofreniew)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2021/05/07/skeleton-analysis"&gt;Skeleton analysis&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2021/05/24/life-science-summit-workshop"&gt;Life sciences at the 2021 Dask Summit&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2021/05/25/user-survey"&gt;The 2021 Dask User Survey is out now&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2021/06/25/dask-down-under"&gt;Dask Down Under&lt;/a&gt; (co-authored with Nick Mortimer)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2021/07/02/ragged-output"&gt;Ragged output, how to handle awkward shaped results&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2021/07/07/high-level-graphs"&gt;High Level Graphs update&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2021/08/23/gsoc-2021-project"&gt;Google Summer of Code 2021 - Dask Project&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Draft status, will be published soon:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-blog/pull/108"&gt;Mosaic Image Fusion&lt;/a&gt; (co-authored with Volker Hisenstein)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-blog/pull/109"&gt;2021 Dask user survey results&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/10/20/czi-eoss-update/"/>
    <summary>Dask was awarded funding last year in round 2 of the CZI Essential Open Source Software grant program.
That funding was used to hire Genevieve Buckley to work on Dask with a focus on life sciences.
Last month Dask submitted an interim progress report to CZI, covering the period from February to September 2021.
That progress update is published verbatim below, to share with the wider Dask community.</summary>
    <category term="lifescience" label="life science"/>
    <published>2021-10-20T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/09/15/user-survey/</id>
    <title>2021 Dask User Survey</title>
    <updated>2021-09-15T00:00:00+00:00</updated>
    <author>
      <name>Genevieve Buckley</name>
    </author>
    <content type="html">&lt;p&gt;This post presents the results of the 2021 Dask User Survey, which ran earlier this year.
Thanks to everyone who took the time to fill out the survey!
These results help us better understand the Dask community and will guide future development efforts.&lt;/p&gt;
&lt;p&gt;The raw data, as well as the start of an analysis, can be found in this binder:&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://mybinder.org/v2/gh/dask/dask-examples/main?urlpath=%2Ftree%2Fsurveys%2F2021.ipynb"&gt;&lt;img alt="Binder" src="https://mybinder.org/badge_logo.svg" /&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Let us know if you find anything in the data.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/09/15/user-survey.md&lt;/span&gt;, line 19)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="contents"&gt;

&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#highlights"&gt;&lt;span class="xref myst"&gt;Highlights&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#who-are-dask-users"&gt;&lt;span class="xref myst"&gt;Who are Dask users?&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#how-people-like-to-use-dask"&gt;&lt;span class="xref myst"&gt;How people like to use Dask&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#diagnostics"&gt;&lt;span class="xref myst"&gt;Diagnostics&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../2021/05/21/stability/#stability"&gt;&lt;span class="std std-ref"&gt;Stability&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#user-satisfaction"&gt;&lt;span class="xref myst"&gt;User satisfaction, support, and documentation&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#suggestions-for-improvement"&gt;&lt;span class="xref myst"&gt;Suggestions for improvement&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#previous-survey-results"&gt;&lt;span class="xref myst"&gt;Previous survey results&lt;/span&gt;&lt;/a&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/2021/09/15/user-survey.md&lt;/span&gt;, line 30)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="highlights"&gt;
&lt;h1&gt;Highlights &lt;a class="anchor" id="highlights"&gt;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;We had 247 responses to the survey (roughly the same as last year, which had just under 240 responses). Overall, responses were similar to previous years.&lt;/p&gt;
&lt;p&gt;We asked 43 questions in the survey (an increase of 18 questions compared to the year before). We asked a bunch of new questions about the types of datasets people work with, the stability of Dask, and what kinds of industries people work in.&lt;/p&gt;
&lt;p&gt;Our community wants:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;More documentation and examples&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;More intermediate level documentation&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;To improve the resiliency of Dask (i.e. do computations complete?)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Users also value these features:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Improved scaling&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ease of deployment&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Better scikit-learn &amp;amp; machine learning support&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;section id="the-typical-dask-user"&gt;
&lt;h2&gt;The typical Dask user&lt;/h2&gt;
&lt;p&gt;The survey shows us there is a lot of diversity in our community, and there is no one way to use Dask. That said, our hypothetical “typical” Dask user:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Works with gigabyte sized datasets&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stored on a local filesystem&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Has been using Dask between 1 and 3 years&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Uses Dask occasionally, not every day&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Uses Dask interactively at least part of the time&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Uses a compute cluster (probably)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Likes to view the Dask dashboard with a web browser&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For the most part, Dask is stable enough for their needs, but improving the Dask’s resiliancy would be helpful&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Uses the Dask dataframe, delayed, and maybe the Dask Array API, alongside numpy/pandas and other python libraries&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The most useful thing that would help this person is more documentation, and more examples using Dask in their field.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;They likely work in a scientific field (perhaps geoscience, life science, physics, or astronomy), or alternatively they might work in accounting, finance, insurance, or as a tech worker.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;You can read the survey results from previous years here: &lt;a class="reference external" href="https://blog.dask.org/2020/09/22/user_survey"&gt;2020 survey results&lt;/a&gt;, &lt;a class="reference external" href="https://blog.dask.org/2019/08/05/user-survey"&gt;2019 survey results&lt;/a&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="c1"&gt;# Let&amp;#39;s load in the survey data...&lt;/span&gt;
&lt;span class="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;matplotlib&lt;/span&gt; &lt;span class="n"&gt;inline&lt;/span&gt;

&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pprint&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;pprint&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;seaborn&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;sns&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;matplotlib.pyplot&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;plt&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;textwrap&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;df2019&lt;/span&gt; &lt;span class="o"&gt;=&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="s2"&gt;&amp;quot;data/2019-user-survey-results.csv.gz&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;parse_dates&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Timestamp&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;replace&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;How often do you use Dask?&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;I use Dask all the time, even when I sleep&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Every day&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;df2020&lt;/span&gt; &lt;span class="o"&gt;=&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="s2"&gt;&amp;quot;data/2020-user-survey-results.csv.gz&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;assign&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Timestamp&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;df&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;to_datetime&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;Timestamp&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="nb"&gt;format&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;%Y/%m/&lt;/span&gt;&lt;span class="si"&gt;%d&lt;/span&gt;&lt;span class="s2"&gt; %H:%M:%S %p %Z&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;astype&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;datetime64[ns]&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;replace&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;How often do you use Dask?&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;I use Dask all the time, even when I sleep&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Every day&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;df2021&lt;/span&gt; &lt;span class="o"&gt;=&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="s2"&gt;&amp;quot;data/2021-user-survey-results.csv.gz&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;assign&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Timestamp&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;df&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;to_datetime&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;Timestamp&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;astype&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;datetime64[ns]&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;replace&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;How often do you use Dask?&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;I use Dask all the time, even when I sleep&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Every day&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;common&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2019&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;intersection&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df2020&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;intersection&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;added&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;difference&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df2020&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;dropped&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2020&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;difference&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;concat&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;df2019&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;df2020&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;Year&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Timestamp&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;year&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_index&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;Year&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Timestamp&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;sort_index&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/09/15/user-survey.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&gt;
&lt;section id="who-are-dask-users"&gt;
&lt;h1&gt;Who are Dask users? &lt;a class="anchor" id="who-are-dask-users"&gt;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Most people said they use Dask occasionally, while a smaller group use Dask every day. There is a wide variety in how long people have used Dask for, with the most common response being between one and three years.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;How often do you use Dask?&amp;quot;&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s2"&gt;&amp;quot;;&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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;());&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_7_0.png" /&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;How long have you used Dask?&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# New question in 2021&lt;/span&gt;
&lt;span class="n"&gt;order&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;More than 3 years&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;1 - 3 years&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;3 months - 1 year&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Less than 3 months&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;I&amp;#39;ve never used Dask&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s2"&gt;&amp;quot;;&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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_8_0.png" /&gt;&lt;/p&gt;
&lt;p&gt;Just over half of respondants use Dask with other people (their team or organisation), and the other half use Dask on their own.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Do you use Dask as part of a larger group?&amp;quot;&lt;/span&gt;
&lt;span class="n"&gt;order&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;I use Dask mostly on my own&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s1"&gt;&amp;#39;My team or research group also use Dask&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s1"&gt;&amp;#39;Beyond my group, many people throughout my institution use Dask&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;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s2"&gt;&amp;quot;;&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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_10_0.png" /&gt;&lt;/p&gt;
&lt;p&gt;In the last year, there has been an increase in the number of people who say that many people throughout their institution use Dask (32 people said this in 2021, compared to 19 in 2020). Between 2019 and 2020, there was a drop in the number of people who said their immediate team also uses Dask (121 people said this in 2019, compared to 94 in 2020). It’s not clear why we saw either of these changes, so it will be interesting to see what happens in future years.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Do you use Dask as part of a larger group?&amp;#39;&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;hue&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Year&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reset_index&lt;/span&gt;&lt;span class="p"&gt;());&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_12_0.png" /&gt;&lt;/p&gt;
&lt;section id="what-industry-do-you-work-in"&gt;
&lt;h2&gt;What industry do you work in?&lt;/h2&gt;
&lt;p&gt;There was a wide variety of industries represented in the survey.&lt;/p&gt;
&lt;p&gt;Almost half of responses were in an industry related to science, academia, or a governmant laboratory. Geoscicence had the most responses, while life sciences, physics, and astronomy were also popular fields.&lt;/p&gt;
&lt;p&gt;Around 30 percent of responses were from people in businesss and tech. Of these, there was a roughly even split between people in accounting/finance/insurance vs other tech workers.&lt;/p&gt;
&lt;p&gt;Around 10 percent of responses belonged to manufacturing, engineering, and other industry (energy, aerospace, etc). The remaining responses were difficult to categorise.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;What industry do you work in?&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# New question in 2021&lt;/span&gt;
&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s2"&gt;&amp;quot;;&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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;order&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;()[&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_level_values&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;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_14_0.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="how-easy-is-it-for-you-to-upgrade-to-newer-versions-of-python-libraries"&gt;
&lt;h2&gt;How easy is it for you to upgrade to newer versions of Python libraries?&lt;/h2&gt;
&lt;p&gt;The majority of users are able to easily upgrade to newer versoins of python libraries when they want.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;How easy is it for you to upgrade to newer versions of Python libraries&amp;quot;&lt;/span&gt;
&lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_ylabel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;Scale from 1 (Difficult) to 4 (Easy)&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;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_16_0.png" /&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/09/15/user-survey.md&lt;/span&gt;, line 181)&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="how-people-like-to-use-dask"&gt;
&lt;h1&gt;How people like to use Dask &lt;a class="anchor" id="how-people-like-to-use-dask"&gt;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;People like to use Dask in conjunction with numpy and pandas, along with a range of other python libraries.
The most popular Dask APIs are &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe.html"&gt;Dask Dataframes&lt;/a&gt;, &lt;a class="reference external" href="https://docs.dask.org/en/latest/delayed.html"&gt;Dask Delayed&lt;/a&gt;, and &lt;a class="reference external" href="https://docs.dask.org/en/latest/array.html"&gt;Dask Arrays&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The vast majority of people like to use Dask interactively with Jupyter or IPython at least part of the time, and most people view the &lt;a class="reference external" href="https://docs.dask.org/en/latest/diagnostics-distributed.html"&gt;Dask Dashboard&lt;/a&gt; with a web browser.&lt;/p&gt;
&lt;section id="what-are-some-other-libraries-that-you-often-use-with-dask"&gt;
&lt;h2&gt;What are some other libraries that you often use with Dask?”&lt;/h2&gt;
&lt;p&gt;The ten most common libraries people use with Dask are: &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;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;xarray&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scikit-learn&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scipy&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;statsmodels&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;matplotlib&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;xgboost&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;numba&lt;/span&gt;&lt;/code&gt;, and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;joblib&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;What are some other libraries that you often use with Dask?&amp;quot;&lt;/span&gt;
&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;str&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;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="s2"&gt;&amp;quot;, &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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;labels&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&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="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;iloc&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;index&lt;/span&gt;
&lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;labels&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_ylabel&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_20_0.png" /&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/09/15/user-survey.md&lt;/span&gt;, line 201)&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="dask-apis"&gt;
&lt;h1&gt;Dask APIs&lt;/h1&gt;
&lt;p&gt;The three most popular Dask APIs people use are:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe.html"&gt;Dask Dataframes&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://docs.dask.org/en/latest/delayed.html"&gt;Dask Delayed&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://docs.dask.org/en/latest/array.html"&gt;Dask Arrays&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In 2021, we saw a small increase in the number of people who use &lt;a class="reference external" href="https://docs.dask.org/en/latest/delayed.html"&gt;dask delayed&lt;/a&gt;, compared with previous years. This might be a good thing, it’s possible that as people develop experience and confidence with Dask, they are more likely to start using more advanced features such as &lt;a class="reference external" href="https://docs.dask.org/en/latest/delayed.html"&gt;delayed&lt;/a&gt;. Besides this change, preferences were pretty simliar to the results from previous years.&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;apis&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;Dask APIs&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;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="s2"&gt;&amp;quot;, &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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;top&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;apis&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;loc&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;&amp;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;apis&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;apis&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;apis&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;isin&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;top&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reset_index&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Dask APIs&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;apis&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_22_0.png" /&gt;&lt;/p&gt;
&lt;section id="interactive-or-batch"&gt;
&lt;h2&gt;Interactive or Batch?&lt;/h2&gt;
&lt;p&gt;The vast majority of people like to use Dask interactively with Jupyter or IPython at least part of the time. Less than 15% of Dask users only use Dask in batch mode (submitting scripts that run in the future).&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Interactive or Batch?&amp;#39;&lt;/span&gt;
&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;str&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;Interactive:  I use Dask with Jupyter or IPython when playing with data, Batch: I submit scripts that run in the future&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Interactive and Batch&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;str&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;Interactive:  I use Dask with Jupyter or IPython when playing with data&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Interactive&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;str&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;Batch: I submit scripts that run in the future&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Batch&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;order&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Interactive and Batch&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Interactive&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Batch&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_ylabel&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_24_0.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="how-do-you-view-dask-s-dashboard"&gt;
&lt;h2&gt;How do you view Dask’s dashboard?&lt;/h2&gt;
&lt;p&gt;Most people look at the Dask dashboard using a web browser. A smaller group use the &lt;a class="reference external" href="https://github.com/dask/dask-labextension"&gt;dask jupyterlab extension&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;A few people are still not sure what the dashboard is all about. If that’s you too, you might like to watch &lt;a class="reference external" href="https://youtu.be/N_GqzcuGLCY"&gt;this 20 minute video&lt;/a&gt; that explains why the dashboard is super useful, or see the rest of the docs &lt;a class="reference external" href="https://docs.dask.org/en/latest/diagnostics-distributed.html"&gt;here&lt;/a&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;How do you view Dask&amp;#39;s dashboard?&amp;quot;&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s2"&gt;&amp;quot;, &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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;());&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_26_0.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="local-machine-or-cluster"&gt;
&lt;h2&gt;Local machine or Cluster?&lt;/h2&gt;
&lt;p&gt;Roughly two thirds of respondants use a computing cluster at least part of the time.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Local machine or Cluster?&amp;#39;&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;str&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;contains&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Cluster&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;astype&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Year&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;mean&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;Year
2019    0.654902
2020    0.666667
2021    0.630081
Name: Local machine or Cluster?, dtype: float64
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Local machine or Cluster?&amp;#39;&lt;/span&gt;
&lt;span class="n"&gt;order&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;Personal laptop&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s1"&gt;&amp;#39;Large workstation&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s1"&gt;&amp;#39;Cluster of 2-10 machines&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s1"&gt;&amp;#39;Cluster with 10-100 machines&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s1"&gt;&amp;#39;Cluster with 100+ machines&amp;#39;&lt;/span&gt;
&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s2"&gt;&amp;quot;, &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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_29_0.png" /&gt;&lt;/p&gt;
&lt;section id="if-you-use-a-cluster-how-do-you-launch-dask"&gt;
&lt;h3&gt;If you use a cluster, how do you launch Dask?&lt;/h3&gt;
&lt;p&gt;SSH is the most common way to launch Dask on a compute cluster, followed by a HPC resource manager, then Kubernetes.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;If you use a cluster, how do you launch Dask? &amp;quot;&lt;/span&gt;
&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;str&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="s2"&gt;&amp;quot;HPC resource manager (SLURM, PBS, SGE, LSF or similar)&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;HPC resource manager (SLURM PBS SGE LSF or similar)&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;regex&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;str&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="s2"&gt;&amp;quot;I don&amp;#39;t know, someone else does this for me&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;I don&amp;#39;t know someone else does this for me&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;regex&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s2"&gt;&amp;quot;, &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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;order&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;()[&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_level_values&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;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_31_0.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="if-you-use-a-cluster-do-you-have-a-need-for-multiple-worker-types-in-the-same-cluster"&gt;
&lt;h3&gt;If you use a cluster, do you have a need for multiple worker types in the same cluster?&lt;/h3&gt;
&lt;p&gt;Of the people who use compute clusters, a little less than half have a need for multiple worker types in the same cluster. Examples of this might include mixed workers with GPU vs no GPU, mixed workers with low or high memory allocations, etc.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;If you use a cluster, do you have a need for multiple worker / machine types (e.g. GPU / no GPU, low / high memory) in the same cluster?&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# New question in 2021&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s2"&gt;&amp;quot;;&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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;());&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Do you need multiple worker/machine types on a cluster?&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_33_0.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="datasets"&gt;
&lt;h2&gt;Datasets&lt;/h2&gt;
&lt;section id="how-large-are-your-datasets-typically"&gt;
&lt;h3&gt;How large are your datasets typically?&lt;/h3&gt;
&lt;p&gt;Dask users most commonly work with gigabyte sized datasets. Very few users work with petabyte sized datasets.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;How large are your datasets typically?&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# New question in 2021&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s2"&gt;&amp;quot;, &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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;());&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_35_0.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="where-are-your-datasets-typically-stored"&gt;
&lt;h3&gt;Where are your datasets typically stored?&lt;/h3&gt;
&lt;p&gt;Most people store their data on a local filesystem.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Where are your datasets typically stored?&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# New question in 2021&lt;/span&gt;
&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s2"&gt;&amp;quot;, &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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;order&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;()[&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_level_values&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;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_37_0.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="what-file-formats-do-you-typically-work-with"&gt;
&lt;h3&gt;What file formats do you typically work with?&lt;/h3&gt;
&lt;p&gt;The two most common file formats (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;csv&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;parquet&lt;/span&gt;&lt;/code&gt;) are popular among Dask Dataframe users. The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;JSON&lt;/span&gt;&lt;/code&gt; file format is also very commonly used with Dask. The fourth and fifth most common filetypes (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HDF5&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;zarr&lt;/span&gt;&lt;/code&gt;) are popular among Dask Array users. This fits with what we know about the Dask Dataframe API being the most popular, with Dask Arrays close behind.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;What file formats do you typically work with?&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# New question in 2021&lt;/span&gt;
&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s2"&gt;&amp;quot;, &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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;order&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;()[&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_level_values&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;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_39_0.png" /&gt;&lt;/p&gt;
&lt;p&gt;This survey question had a long tail: a very wide variety of specialized file formats were reported, most only being used by one or two individuals who replied to the survey.&lt;/p&gt;
&lt;p&gt;A lot of these specialized file formats store image data, specific to particular fields (astronomy, geoscience, microscopy, etc.).&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;()[&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&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="p"&gt;]&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_level_values&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;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;[&amp;#39;proprietary measurement format&amp;#39;,
 &amp;#39;netCDF3&amp;#39;,
 &amp;#39;czi&amp;#39;,
 &amp;#39;specifically NetCDF4&amp;#39;,
 &amp;#39;grib2&amp;#39;,
 &amp;#39;in-house npy-like array format&amp;#39;,
 &amp;#39;jpeg2000&amp;#39;,
 &amp;#39;netCDF4 (based on HDF5)&amp;#39;,
 &amp;#39;proprietary microscopy file types. Often I convert to Zarr with a loss of metadata.&amp;#39;,
 &amp;#39;sas7bdat&amp;#39;,
 &amp;#39;npy&amp;#39;,
 &amp;#39;npy and pickle&amp;#39;,
 &amp;#39;root with uproot&amp;#39;,
 &amp;#39;root&amp;#39;,
 &amp;#39;regular GeoTiff&amp;#39;,
 &amp;#39;.npy&amp;#39;,
 &amp;#39;Text&amp;#39;,
 &amp;#39;VCF BAM CRAM&amp;#39;,
 &amp;#39;UM&amp;#39;,
 &amp;#39;CASA measurement sets&amp;#39;,
 &amp;#39;Casa Tables (Radio Astronomy specific)&amp;#39;,
 &amp;#39;Custom binary&amp;#39;,
 &amp;#39;FITS&amp;#39;,
 &amp;#39;FITS (astronomical images)&amp;#39;,
 &amp;#39;FITS and a custom semi-relational table specification that I want to kill and replace with something better&amp;#39;,
 &amp;#39;Feather (Arrow)&amp;#39;,
 &amp;#39;GPKG&amp;#39;,
 &amp;#39;GeoTIFF&amp;#39;,
 &amp;#39;NetCDF4&amp;#39;,
 &amp;#39;Netcdf&amp;#39;,
 &amp;#39;Netcdf4&amp;#39;,
 &amp;#39;PP&amp;#39;,
 &amp;#39;SQL&amp;#39;,
 &amp;#39;SQL query to remote DB&amp;#39;,
 &amp;#39;SQL to Dataframe&amp;#39;,
 &amp;#39;Seismic data (miniSEED)&amp;#39;,
 &amp;#39;TFRecords&amp;#39;,
 &amp;#39;TIFF&amp;#39;,
 &amp;#39;Testing with all file formats. Just want it as a replacement for spark. &amp;#39;,
 &amp;#39;.raw image files&amp;#39;,
 &amp;#39;ugh&amp;#39;]
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="https://imgs.xkcd.com/comics/standards.png" alt="XKCD comic 927: Standards"&gt;
&lt;p&gt;XKCD comic “Standards” https://xkcd.com/927/&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="preferred-cloud"&gt;
&lt;h2&gt;Preferred Cloud?&lt;/h2&gt;
&lt;p&gt;The most popular cloud solution is Amazon Web Services (AWS), followed by Google Cloud Platform (GCP) and Microsoft Azure.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Preferred Cloud?&amp;quot;&lt;/span&gt;
&lt;span class="n"&gt;order&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;Amazon Web Services (AWS)&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;Google Cloud Platform (GCP)&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;Microsoft Azure&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;Digital Ocean&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s2"&gt;&amp;quot;, &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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_44_0.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="do-you-use-dask-projects-to-deploy"&gt;
&lt;h2&gt;Do you use Dask projects to deploy?&lt;/h2&gt;
&lt;p&gt;Among those who use dask projects to deploy, &lt;a class="reference external" href="https://github.com/dask/dask-jobqueue"&gt;dask-jobqueue&lt;/a&gt;
and &lt;a class="reference external" href="https://github.com/dask/helm-chart"&gt;dask helm chart&lt;/a&gt; are the two most popular options.
There was a wide variety of projects people used for deployment.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Do you use Dask projects to deploy?&amp;quot;&lt;/span&gt;
&lt;span class="n"&gt;order&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;dask-jobqueue&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;dask&amp;#39;s helm chart&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;dask-kubernetes&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;dask&amp;#39;s docker image at daskdev/dask&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;dask-gateway&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;dask-ssh&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;dask-cloudprovider&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;dask-yarn&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;qhub&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;dask-mpi&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;str&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;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="s2"&gt;&amp;quot;, &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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_46_0.png" /&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/09/15/user-survey.md&lt;/span&gt;, line 450)&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="diagnostics"&gt;
&lt;h1&gt;Diagnostics &lt;a class="anchor" id="diagnostics"&gt;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;We saw earlier that most people like to view the Dask Dashboard using their web browser.&lt;/p&gt;
&lt;p&gt;In the dashboard, people said the most useful diagnostics plots were:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;The task stream plot&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The progress plot, and&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The memory useage per worker plot&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Which Diagnostic plots are most useful?&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# New question in 2021&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s1"&gt;&amp;#39;, &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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;());&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_48_0.png" /&gt;&lt;/p&gt;
&lt;p&gt;We also asked some new questions about diagnostics in 2021.&lt;/p&gt;
&lt;p&gt;We found that most people (65 percent) do not use &lt;a class="reference external" href="https://distributed.dask.org/en/latest/diagnosing-performance.html#performance-reports"&gt;Dask performance reports&lt;/a&gt;, which is a way to save the diagnostic dashboard to static HTML plots for later review.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Do you use Dask&amp;#39;s Performance reports?&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# New question in 2021&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Yes&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;No&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_50_0.png" /&gt;&lt;/p&gt;
&lt;p&gt;Very few people use Dask’s &lt;a class="reference external" href="https://docs.dask.org/en/latest/setup/prometheus.html"&gt;Prometheus metrics&lt;/a&gt;. Jacob Tomlinson has an excellent article on &lt;a class="reference external" href="https://medium.com/rapids-ai/monitoring-dask-rapids-with-prometheus-grafana-96eaf6b8f3a0"&gt;Monitoring Dask + RAPIDS with Prometheus + Grafana&lt;/a&gt;, if you’re interested in learning more about how to use this feature.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Do you use Dask&amp;#39;s Prometheus Metrics?&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# New question in 2021&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Yes&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;No&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_52_0.png" /&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/09/15/user-survey.md&lt;/span&gt;, line 490)&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="stability"&gt;
&lt;h1&gt;Stability &lt;a class="anchor" id="stability"&gt;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;We asked a number of questions around the stability of Dask, many of them new questions in 2021.&lt;/p&gt;
&lt;p&gt;The majority of people said Dask was resiliant enough for them (eg: computations complete).
However this is an area we could improve in, as 36 percent of people are not satisfied.
This was a new question 2021, so we can’t say how people opinion of Dask’s resiliancy has changed over time.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Is Dask resilient enough for you? (e.g. computations complete).&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# new question in 2021&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Yes&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;No&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Is Dask resilient enough for you?&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_54_0.png" /&gt;&lt;/p&gt;
&lt;p&gt;Most people say Dask in general is stable enough for them (eg: between different version releases). This is similar to the survey results from previous years.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Is Dask stable enough for you?&amp;quot;&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Yes&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;No&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_56_0.png" /&gt;&lt;/p&gt;
&lt;p&gt;People also say that the API of Dask is stable enough for them too.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Is Dask&amp;#39;s API stable enough for you?&amp;quot;&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Yes&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;No&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_58_0.png" /&gt;&lt;/p&gt;
&lt;p&gt;The vast majority of people are satisfied with the current release frequency (roughly once every two weeks).&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;How is Dask&amp;#39;s release frequency?&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# New question in 2021&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;());&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_60_0.png" /&gt;&lt;/p&gt;
&lt;p&gt;Most people say they would pin their code to a long term support release, if one was available for Dask.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;If Dask had Long-term support (LTS) releases, would you pin your code to use them?&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# New question in 2021&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Yes&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;No&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Would you pin to a long term support release?&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_62_0.png" /&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/09/15/user-survey.md&lt;/span&gt;, line 546)&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="user-satisfaction-support-and-documentation"&gt;
&lt;h1&gt;User satisfaction, support, and documentation &lt;a class="anchor" id="user-satisfaction"&gt;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;We asked a bunch of new questions about user satisfaction in the 2021 survey.&lt;/p&gt;
&lt;section id="how-easy-is-dask-to-use"&gt;
&lt;h2&gt;How easy is Dask to use?&lt;/h2&gt;
&lt;p&gt;The majority of people say that Dask is moderately easy to use, the same as in previous surveys.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;On a scale of 1 - 5 (1 being hardest, 5 being easiest) how easy is Dask to use?&amp;quot;&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;());&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;1 = Difficult, 5 = Easy&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;How easy is Dask to use?&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_65_0.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="how-is-dask-s-documentation"&gt;
&lt;h2&gt;How is Dask’s documentation?&lt;/h2&gt;
&lt;p&gt;Most people think that Dask’s documentation is pretty good.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;How is Dask&amp;#39;s documentation?&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# New question in 2021&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;());&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;1 = Not good, 5 = Great&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_67_0.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="how-satisfied-are-you-with-maintainer-responsiveness-on-github"&gt;
&lt;h2&gt;How satisfied are you with maintainer responsiveness on GitHub?&lt;/h2&gt;
&lt;p&gt;Almost everybody who responded feels positively about Dask’s maintainer responsiveness on GitHub .&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;How satisfied are you with maintainer responsiveness on GitHub?&amp;quot;&lt;/span&gt;  &lt;span class="c1"&gt;# New question in 2021&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;());&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;1 = Not satisfied, 5 = Thrilled&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_69_0.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="what-dask-resources-have-you-used-for-support-in-the-last-six-months"&gt;
&lt;h2&gt;What Dask resources have you used for support in the last six months?&lt;/h2&gt;
&lt;p&gt;The documentation at &lt;a class="reference external" href="https://dask.org/"&gt;dask.org&lt;/a&gt; is the first place most users look for help.&lt;/p&gt;
&lt;p&gt;The breakdown of responses to this question in 2021 was very similar to previous years, with the exception that no-one seemed to know that the &lt;a class="reference external" href="https://www.youtube.com/c/Dask-dev/videos"&gt;Dask YouTube channel&lt;/a&gt; or Gitter chat existed in 2019.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;What Dask resources have you used for support in the last six months?&amp;#39;&lt;/span&gt;

&lt;span class="n"&gt;resource_map&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;Tutorial&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Tutorial at tutorial.dask.org&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;YouTube&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;YouTube channel&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;gitter&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Gitter chat&amp;quot;&lt;/span&gt;
&lt;span class="p"&gt;}&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;str&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;;&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="c1"&gt;# Make separator values consistent&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;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s1"&gt;&amp;#39;, &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;explode&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="n"&gt;resource_map&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;top&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;value_counts&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;index&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;d&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;isin&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;top&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;

&lt;span class="n"&gt;fig&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;subplots&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;figsize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;8&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;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;hue&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Year&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;d&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reset_index&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_71_0.png" /&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/09/15/user-survey.md&lt;/span&gt;, line 613)&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="suggestions-for-improvement"&gt;
&lt;h1&gt;Suggestions for improvement &lt;a class="anchor" id="suggestions-for-improvement"&gt;&lt;/a&gt;&lt;/h1&gt;
&lt;section id="which-would-help-you-most-right-now"&gt;
&lt;h2&gt;Which would help you most right now?&lt;/h2&gt;
&lt;p&gt;The two top priorities people said would help most right now are both related to documentation. People want more documentation, and more examples in their field. Performance improvements were also commonly mentioned as something that would help the most right now.&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;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Which would help you most right now?&amp;quot;&lt;/span&gt;
&lt;span class="n"&gt;order&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;More documentation&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;More examples in my field&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;Performance improvements&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;New features&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;Bug fixes&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_74_0.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="how-can-dask-improve"&gt;
&lt;h2&gt;How can Dask improve?&lt;/h2&gt;
&lt;p&gt;We also gave people the opportunity for a free text response to the question “How can Dask imporove?”&lt;/p&gt;
&lt;p&gt;Matt has previously written an &lt;a class="reference external" href="https://blog.dask.org/2021/06/18/early-survey"&gt;early anecdotes blogpost&lt;/a&gt;
that dives into the responses to this question in more detail.&lt;/p&gt;
&lt;p&gt;He found these recurring themes:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Intermediate Documentation&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Documentation Organization&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Functionality&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;High Level Optimization&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Runtime Stability and Advanced Troubleshooting&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Since more documentation and examples were the two most requested improvements, I’ll summarize some of the steps forward in that area here:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Regarding more intermediate documentation, Matt says:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;There is a lot of good potential material that advanced users have around performance and debugging that could be fun to publish.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matt points out that Dask has excellent &lt;em&gt;reference documentation&lt;/em&gt;, but lacks a lot of good &lt;em&gt;narrative documentation&lt;/em&gt;. To address this, Julia Signell is currently investigating how we could improve the organization of Dask’s documentation (you can subscribe to &lt;a class="reference external" href="https://github.com/dask/community/issues/170"&gt;this issue thread&lt;/a&gt; if you want to follow that discussion)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matt comments that it’s hard to have good &lt;em&gt;narrative documentation&lt;/em&gt; when there are so many different &lt;em&gt;user narratives&lt;/em&gt; (i.e. Dask is used by people from many different industries). This year, we added a new question to the survey asking for the industry people work in. We added this because &lt;em&gt;“More examples in my field”&lt;/em&gt; has been one of the top two requests for the last three years. Now we can use that information to better target narrative documentation to the areas that need it most (geoscience, life science, and finance).&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;What industry do you work in?&amp;#39;&lt;/span&gt;
&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;df2021&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Which would help you most right now?&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;More examples in my field&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;order&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;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&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="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]&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="n"&gt;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countplot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dropna&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&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="s1"&gt;&amp;#39;, &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;explode&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;What field do you want more documentation examples for?&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_76_0.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="what-common-feature-requests-do-you-care-about-most"&gt;
&lt;h2&gt;What common feature requests do you care about most?&lt;/h2&gt;
&lt;p&gt;Good support for numpy and pandas is critical for most users.
Users also value:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Improved scaling&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ease of deployment&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Resiliancy of Dask&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Better scikit-learn &amp;amp; machine learning support&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Most feature requests are similar to the survey results from previous years, although there was an increase in the number of people who say better scikit-learn/ML support is critical to them. We also added a new question about Dask’s resiliancy in 2021.&lt;/p&gt;
&lt;p&gt;In the figure below you can see how people rated the importance of each feature request, for each of the three years we’ve run this survey.&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;common&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;str&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;startswith&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;What common feature&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;rename&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="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="n"&gt;lstrip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;What common feature requests do you care about most?[&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;rstrip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="sa"&gt;r&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;]&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)))&lt;/span&gt;
&lt;span class="n"&gt;a&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;common&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;loc&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;2019&lt;/span&gt;&lt;span class="p"&gt;]&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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;)&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;stack&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reset_index&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rename&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;level_0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Question&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;level_1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Importance&amp;quot;&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="s2"&gt;&amp;quot;count&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;assign&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Year&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;2019&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;common&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;loc&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;2020&lt;/span&gt;&lt;span class="p"&gt;]&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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;)&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;stack&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reset_index&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rename&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;level_0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Question&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;level_1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Importance&amp;quot;&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="s2"&gt;&amp;quot;count&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;assign&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Year&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;2020&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;common&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;loc&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;2021&lt;/span&gt;&lt;span class="p"&gt;]&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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&lt;/span&gt;&lt;span class="p"&gt;)&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;stack&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reset_index&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rename&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;level_0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Question&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;level_1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Importance&amp;quot;&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="s2"&gt;&amp;quot;count&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;assign&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Year&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;2021&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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;concat&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="n"&gt;c&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;ignore_index&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;d&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;common&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reset_index&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rename&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;level_2&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Feature&amp;quot;&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="s2"&gt;&amp;quot;Importance&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;})&lt;/span&gt;
&lt;span class="n"&gt;order&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Not relevant for me&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;Somewhat useful&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Critical to me&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;sns&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;catplot&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="s1"&gt;&amp;#39;Importance&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;row&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Feature&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;kind&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;count&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;col&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Year&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;d&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sharex&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;order&lt;/span&gt;&lt;span class="p"&gt;);&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/2021_survey_78_0.png" /&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/09/15/user-survey.md&lt;/span&gt;, line 699)&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="previous-survey-results"&gt;
&lt;h1&gt;Previous survey results &lt;a class="anchor" id="previous-survey-results"&gt;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Thanks to everyone who took the survey!&lt;/p&gt;
&lt;p&gt;If you want to read more about the 2021 Dask survey, the blogpost on early anecdotes from the Dask 2021 survey &lt;a class="reference external" href="https://blog.dask.org/2021/06/18/early-survey"&gt;is available here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;You can read the survey results from previous years here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2020/09/22/user_survey"&gt;2020 survey results&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2019/08/05/user-survey"&gt;2019 survey results&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/09/15/user-survey/"/>
    <summary>This post presents the results of the 2021 Dask User Survey, which ran earlier this year.
Thanks to everyone who took the time to fill out the survey!
These results help us better understand the Dask community and will guide future development efforts.</summary>
    <category term="UserSurvey" label="User Survey"/>
    <published>2021-09-15T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/08/23/gsoc-2021-project/</id>
    <title>Google Summer of Code 2021 - Dask Project</title>
    <updated>2021-08-23T00:00:00+00:00</updated>
    <author>
      <name>Freyam Mehta and Genevieve Buckley</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/08/23/gsoc-2021-project.md&lt;/span&gt;, line 8)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="overview"&gt;

&lt;p&gt;Here’s an update on new features related to visualizing Dask graphs and HTML representations. You can try these new features today with version &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;2021.08.1&lt;/span&gt;&lt;/code&gt; or above. This work was done by Freyam Mehta during the Google Summer of Code 2021. Dask took part in the program under the NumFOCUS umbrella organization.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/08/23/gsoc-2021-project.md&lt;/span&gt;, line 12)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="contents"&gt;
&lt;h1&gt;Contents&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#visualizing-dask-graphs"&gt;&lt;span class="xref myst"&gt;Visualizing Dask graphs&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#graphviz-node-size-scaling"&gt;&lt;span class="xref myst"&gt;Graphviz node size scaling&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#new-tooltips"&gt;&lt;span class="xref myst"&gt;New tooltips&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#color-by-layer-type"&gt;&lt;span class="xref myst"&gt;Color by layer type&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#bugfix-in-visualize-method"&gt;&lt;span class="xref myst"&gt;Bugfix in visualize method&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#html-representations"&gt;&lt;span class="xref myst"&gt;HTML Representations&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#array-images-in-html-repr-for-high-level-graphs"&gt;&lt;span class="xref myst"&gt;Array images in HTML repr for high level graphs&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#new-html-repr-for-processinterface-class"&gt;&lt;span class="xref myst"&gt;New HTML repr for ProcessInterface class&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#new-html-repr-for-security-class"&gt;&lt;span class="xref myst"&gt;New HTML repr for Security class&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/08/23/gsoc-2021-project.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="visualizing-dask-graphs"&gt;
&lt;h1&gt;Visualizing Dask graphs&lt;/h1&gt;
&lt;p&gt;There are several new features involving Dask &lt;a class="reference external" href="https://docs.dask.org/en/latest/graphs.html"&gt;task graph&lt;/a&gt; visualization. Task graphs are a visual representation of the order and dependencies of each individual task within a dask computation. They are a very userful diagnostic tool, and have been used for a long time.&lt;/p&gt;
&lt;img src="/images/gsoc21/dask-simple.png" alt="An example task graph visualization." height=300&gt;
&lt;p&gt;Freyam worked on making these visualizations more illustrative, engaging, and informative. The &lt;a class="reference external" href="https://docs.dask.org/en/latest/graphviz.html"&gt;Graphviz&lt;/a&gt; library boasts a great set of attributes which can be modifified to create a more visually appealing output.&lt;/p&gt;
&lt;p&gt;These features primarily improve the Dask &lt;a class="reference external" href="https://docs.dask.org/en/latest/high-level-graphs.html"&gt;high level graph&lt;/a&gt; visualizations. Both low level and high level Dask graphs can be accessed with very similar methods:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Dask low level graph: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;result.visualize()&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask high level graph: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;result.dask.visualize()&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;…where &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;result&lt;/span&gt;&lt;/code&gt; is a dask object or collection.&lt;/p&gt;
&lt;section id="graphviz-node-size-scaling"&gt;
&lt;h2&gt;Graphviz node size scaling&lt;/h2&gt;
&lt;p&gt;The first change you may notice to the Dask high level graphs, is that the node sizes have been adjusted to scale with the number of tasks in each layer. Layers with more tasks would appear larger than the rest.&lt;/p&gt;
&lt;p&gt;This is a helpful feature to have, because now users can get a much more intuitive sense of where the bulk of their computation takes place.&lt;/p&gt;
&lt;p&gt;Example:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;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="n"&gt;array&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;10000&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="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;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="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;array&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;array&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;array&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;result&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;visualize&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# Dask high level graph&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/gsoc21/7869.png" alt="Example: graphviz node size scaling, pull request #7869" height=414 width=736&gt;
&lt;p&gt;Note: this change only affects the graphviz output for Dask high level graphs. Low level graphs are left unchanged, because each visual node corresponds to one task.&lt;/p&gt;
&lt;p&gt;Reference: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7869"&gt;Pull request #7869 by Freyam Mehta &lt;em&gt;“Add node size scaling to the Graphviz output for the high level graphs”&lt;/em&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="new-tooltips"&gt;
&lt;h2&gt;New tooltips&lt;/h2&gt;
&lt;p&gt;Dask high level graphs now include hover tooltips to provide a brief summary of more detailed information. To use the tooltips, generate a dask high level graph (eg: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;result.dask.visualize()&lt;/span&gt;&lt;/code&gt;) then hover your mouse above the layer you are interested in.&lt;/p&gt;
&lt;img src="/images/gsoc21/7973.png" alt="Example: tooltips provide extra information, pull request #7973" height=414 width=736&gt;
&lt;p&gt;Tooltips provide information such as the layer type and number of tasks associated with it. There is additional information provided for specific dask collections, like dask arrays and dataframes.&lt;/p&gt;
&lt;p&gt;Dask array tooltip information additionally includes:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Array shape&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Chunk size&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Chunk type (eg: are the array chunks numpy, cupy, sparse, etc.)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Data type (eg: are the array values float, integer, boolean, etc.)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Dask dataframe tooltip information additionally includes:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Number of partitions&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dataframe type&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dataframe columns&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Users have asked for a less overwhelming view into the dask task graph. We hope the high level graph view coupled with more detailed tooltip information can provide this middle ground, with enough information to be useful, but not so much as to become overwhelming (like the low level task graphs for large computations).&lt;/p&gt;
&lt;p&gt;Note: This feature is available for SVG output. Other image formats, like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.png&lt;/span&gt;&lt;/code&gt;, etc. do not support tooltips.&lt;/p&gt;
&lt;p&gt;Reference: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7973"&gt;Pull request #7973 by Freyam Mehta &lt;em&gt;“Add tooltips to graphviz”&lt;/em&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="color-by-layer-type"&gt;
&lt;h2&gt;Color by layer type&lt;/h2&gt;
&lt;p&gt;There is also a new feature enabling users to color code a high level graph according to layer type. This option can be enabled by passing the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;color=&amp;quot;layer_type&amp;quot;&lt;/span&gt;&lt;/code&gt; keyword argument, eg: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;result.dask.visualize(color=&amp;quot;layer_type&amp;quot;)&lt;/span&gt;&lt;/code&gt;. This change is intended to make it easier for users to see which layer types predominate.&lt;/p&gt;
&lt;p&gt;While there are no hard and fast rules about what makes a Dask computation efficient, there are some general guidelines:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Dataframe shuffles are particularly expensive operations. You can &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe-best-practices.html#avoid-full-data-shuffling"&gt;read more about this here&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Reading and writing data to/from storage/network services is often high-latency and therefore a bottleneck.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Blockwise layers are generally efficient for computation.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;All layers are materialized during computation.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;See the &lt;a class="reference external" href="https://docs.dask.org/en/latest/best-practices.html"&gt;Dask best pracices&lt;/a&gt; pages for more information on creating more efficient Dask computations.&lt;/p&gt;
&lt;p&gt;Example:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;datasets&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;timeseries&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;df2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;&amp;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;df3&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;name&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;std&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;df3&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;visualize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;color&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;layer_type&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Dask high level graph with colored nodes by layer type&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/gsoc21/7974.png" alt="Example: Dask graph colored by layer type, pull request #7974" height=414 width=736&gt;
&lt;p&gt;Reference: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7974"&gt;Pull request #7974 by Freyam Mehta &lt;em&gt;“Add colors to represent high level layer types”&lt;/em&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="bugfix-in-visualize-method"&gt;
&lt;h2&gt;Bugfix in visualize method&lt;/h2&gt;
&lt;p&gt;Freyam also fixed a bug which caused an error when users tried to call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.visualize()&lt;/span&gt;&lt;/code&gt; with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;filename=None&lt;/span&gt;&lt;/code&gt; (issue &lt;a class="reference external" href="https://github.com/dask/dask/issues/7685"&gt;#7685&lt;/a&gt;, fixed by pull request &lt;a class="reference external" href="https://github.com/dask/dask/pull/7740"&gt;#7740&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;The bug was fixed by adding an extra condition before it reaches the error. If the format is &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;None&lt;/span&gt;&lt;/code&gt;, Dask now uses use a default &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;png&lt;/span&gt;&lt;/code&gt; format.&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;dask&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="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="n"&gt;array&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;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;visualize&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;filename&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# success&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Reference: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7740"&gt;Pull request #7740 by Freyam Mehta &lt;em&gt;“Fixing calling .visualize() with filename=None”&lt;/em&gt;&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/2021/08/23/gsoc-2021-project.md&lt;/span&gt;, line 135)&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="html-representations"&gt;
&lt;h1&gt;HTML representations&lt;/h1&gt;
&lt;p&gt;Dask makes use of HTML representations in several places, for example in Dask collections like the Array and Dataframe classes (for background reading, see &lt;a class="reference external" href="https://matthewrocklin.com/blog/2019/07/04/html-repr"&gt;this blogpost&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;More recently, we’ve introduced HTML representations for high level graphs into Dask, and Jacob Tomlinson has implemented HTML representations in several places in the dask distributed library (for further reading, see &lt;a class="reference external" href="https://blog.dask.org/2021/07/07/high-level-graphs#visualization"&gt;this other blogpost&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;During Freyam’s Google Summer of Code project, he extended the HTML representations for Dask high level graphs to include images, and introduced two entirely new HTML representations to the dask distributed library.&lt;/p&gt;
&lt;section id="array-images-in-html-repr-for-high-level-graphs"&gt;
&lt;h2&gt;Array images in HTML repr for high level graphs&lt;/h2&gt;
&lt;p&gt;The HTML representation for dask high level graphs has been extended, and now includes SVG images of dask arrays at intermediate stages of computation.&lt;/p&gt;
&lt;p&gt;The motivation for this feature is similar to the motivation behind adding tooltips, discussed above. Users want easier ways to access information about the way a Dask computation changes as it moves through each stage of computation. We hope this improvement to the HTML representation for Dask high level graphs will provide an at a glance summary of array shape and chunk size at each stage.&lt;/p&gt;
&lt;p&gt;Example:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;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="n"&gt;array&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;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="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="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;array&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;T&lt;/span&gt;

&lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;  &lt;span class="c1"&gt;# shows the HTML representation in Jupyter&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/gsoc21/7886.png" alt="Example: Array images now included in HTML representation of Dask high level graphs, pull request #7886" height=414 width=736&gt;
&lt;p&gt;Reference: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7886"&gt;Pull request #7886 by Freyam Mehta &lt;em&gt;“Add dask.array SVG to the HTML Repr”&lt;/em&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="new-html-repr-for-processinterface-class"&gt;
&lt;h2&gt;New HTML repr for ProcessInterface class&lt;/h2&gt;
&lt;p&gt;A new HTML representation has been created for the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ProcessInterface&lt;/span&gt;&lt;/code&gt; class in &lt;a class="reference external" href="https://github.com/dask/distributed/"&gt;dask distributed&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The HTML representation displays the status, address, and external address of the process.&lt;/p&gt;
&lt;p&gt;There are three possible status options:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Process created, not yet running (blue icon)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Process is running (green icon)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Process closed (orange icon)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;img src="/images/gsoc21/5181-1.png" alt="Example: New HTML representation for distributed ProcessInterface class, pull request #5181" height=414 width=736&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ProcessInterface&lt;/span&gt;&lt;/code&gt; class is not intended to be used directly. Instead, more typically this information will be accessed via subclasses such as the SSH scheduler or workers.&lt;/p&gt;
&lt;p&gt;Example:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;SSHCluster&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;SSHCluster&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;127.0.0.1&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;127.0.0.1&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;127.0.0.1&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;  &lt;span class="c1"&gt;# HTML representation for the SSH scheduler, shown in Jupyter&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;workers&lt;/span&gt;  &lt;span class="c1"&gt;# dict of all the workers&lt;/span&gt;
&lt;span class="c1"&gt;# or&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;workers&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="c1"&gt;# HTML representation for the first SSH worker in the cluster&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/gsoc21/5181-2.png" alt="Example: New HTML representation for distributed ProcessInterface class, pull request #5181" height=414 width=736&gt;
&lt;p&gt;Reference: &lt;a class="reference external" href="https://github.com/dask/distributed/pull/5181"&gt;Pull request #5181 by Freyam Mehta &lt;em&gt;“Add HTML Repr for ProcessInterface Class and all its subclasses”&lt;/em&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="new-html-repr-for-security-class"&gt;
&lt;h2&gt;New HTML repr for Security class&lt;/h2&gt;
&lt;p&gt;Pull request &lt;a class="reference external" href="https://github.com/dask/distributed/pull/5178"&gt;#5178&lt;/a&gt; added a new HTML representation for the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Security&lt;/span&gt;&lt;/code&gt; class in the &lt;a class="reference external" href="https://github.com/dask/distributed/"&gt;dask distributed library&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Security&lt;/span&gt;&lt;/code&gt; HTML representation shows:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Whether encryption is required&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Whether the object instance was created using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Security.temporary()&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Security(**paths_to_keys)&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;For temporary security objects, keys are generated dynamically and the only copy is kept in memory.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For security objects created using keys stored on disk, the HTML representation will show the full filepath to the relevant security certificates on disk.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Example: temporary security object&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.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;Security&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;Security&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;temporary&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;s&lt;/span&gt;  &lt;span class="c1"&gt;# shows the HTML representation in Jupyter&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Example: security object using certificates saved to 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="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;Security&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;Security&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;require_encryption&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;tls_ca_file&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;ca.pem&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;tls_scheduler_cert&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;scert.pem&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;s&lt;/span&gt;  &lt;span class="c1"&gt;# shows the HTML representation in Jupyter&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/gsoc21/5178-2.png" alt="Example: New HTML representation for distributed Security class, pull request #5178" height=414 width=736&gt;
&lt;p&gt;In addition, the text representation has also been updated to reflect the same information shown in the HTML representation.&lt;/p&gt;
&lt;img src="/images/gsoc21/5178-1.png" alt="Example: New text representation for distributed Security class, pull request #5178" height=414 width=736&gt;
&lt;p&gt;Reference: &lt;a class="reference external" href="https://github.com/dask/distributed/pull/5178/"&gt;Pull request #5178 by Freyam Mehta &lt;em&gt;“Add HTML Repr for Security Class”&lt;/em&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/08/23/gsoc-2021-project/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <published>2021-08-23T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/07/07/high-level-graphs/</id>
    <title>High Level Graphs update</title>
    <updated>2021-07-07T00:00:00+00:00</updated>
    <author>
      <name>Genevieve Buckley</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 8)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;There is a lot of work happening in Dask right now on high level graphs. We’d like to share a snapshot of current work in this area. This post is for people interested in technical details of behind the scenes work improving performance in Dask. You don’t need to know anything about it in order to 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/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 12)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="contents"&gt;
&lt;h1&gt;Contents&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#brief-background"&gt;&lt;span class="xref myst"&gt;Brief background&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#blockwise-layers-progress"&gt;&lt;span class="xref myst"&gt;Blockwise layers progress&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#a-high-level-graph-for-map-overlap"&gt;&lt;span class="xref myst"&gt;A high level graph for map overlap&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#slicing-and-high-level-graphs"&gt;&lt;span class="xref myst"&gt;Slicing and high level graphs&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#visualization"&gt;&lt;span class="xref myst"&gt;Visualization&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#documentation"&gt;&lt;span class="xref myst"&gt;Documentation&lt;/span&gt;&lt;/a&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/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 21)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="brief-background"&gt;
&lt;h1&gt;Brief background&lt;/h1&gt;
&lt;section id="what-are-high-level-graphs"&gt;
&lt;h2&gt;What are high level graphs?&lt;/h2&gt;
&lt;p&gt;High level graphs are a more compact representation of instructions needed to generate the full low level task graph.
The documentation page on Dask high level graphs is here:
https://docs.dask.org/en/latest/high-level-graphs.html&lt;/p&gt;
&lt;/section&gt;
&lt;section id="why-are-they-useful"&gt;
&lt;h2&gt;Why are they useful?&lt;/h2&gt;
&lt;p&gt;High level graphs are useful for faster scheduling.
Instead of sending very large task graphs between the scheduler and the workers, we can instead send the smaller high level graph representation to the worker. Reducing the amount of data that needs to be passed around allows us to improve the overall performance.&lt;/p&gt;
&lt;p&gt;You can read more about faster scheduling in &lt;a class="reference external" href="https://blog.dask.org/2020/07/21/faster-scheduling"&gt;our previous blogpost&lt;/a&gt;.
More recently, Adam Breindel has written about this over on the Coiled blog (&lt;a class="reference external" href="https://coiled.io/blog/dask-under-the-hood-scheduler-refactor/"&gt;link&lt;/a&gt;).&lt;/p&gt;
&lt;/section&gt;
&lt;section id="do-i-need-to-change-my-code-to-use-them"&gt;
&lt;h2&gt;Do I need to change my code to use them?&lt;/h2&gt;
&lt;p&gt;No, you won’t need to change anything. This work is being done under the hood in Dask, and you should see some speed improvements without having to change anything in your code.&lt;/p&gt;
&lt;p&gt;In fact, you might already be benefitting from high level graphs:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;“Starting with Dask 2021.05.0, Dask DataFrame computations will start sending HighLevelGraph’s directly from the client to the scheduler by default. Because of this, users should observe a much smaller delay between when they call .compute() and when the corresponding tasks begin running on workers for large DataFrame computations” https://coiled.io/blog/dask-heartbeat-by-coiled-2021-06-10/&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;Read on for a snapshot of progress in other areas.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/07/07/high-level-graphs.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&gt;
&lt;section id="blockwise-layers-progress"&gt;
&lt;h1&gt;Blockwise layers progress&lt;/h1&gt;
&lt;section id="summary"&gt;
&lt;h2&gt;Summary&lt;/h2&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Blockwise&lt;/span&gt;&lt;/code&gt; high level graph layer was introduced in the 2020.12.0 Dask release. Since then, there has been a lot of effort made to use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Blockwise&lt;/span&gt;&lt;/code&gt; high level graph layer whereever possible for improved performance, most especially for IO operations. The following is a non-exhaustive list.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="work-to-date"&gt;
&lt;h2&gt;Work to date&lt;/h2&gt;
&lt;p&gt;Highlights include (in no particular order):&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Merged PR by Rick Zamora: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7415"&gt;Use Blockwise for DataFrame IO (parquet, csv, and orc) #7415&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Merged PR by Rick Zamora: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7625"&gt;Move read_hdf to Blockwise 7625&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Merged PR by Rick Zamora: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7615"&gt;Move timeseries and daily-stock to Blockwise #7615&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Merged PR by John Kirkham: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7704"&gt;Rewrite da.fromfunction w/ da.blockwise #7704&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
&lt;section id="ongoing-work"&gt;
&lt;h2&gt;Ongoing work&lt;/h2&gt;
&lt;p&gt;Lots of other work with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Blockwise&lt;/span&gt;&lt;/code&gt; is currently in progress:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Ian Rose: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7417"&gt;Blockwise array creation redux #7417&lt;/a&gt;. This PR creates blockwise implementations for the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_array&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_zarr&lt;/span&gt;&lt;/code&gt; functions.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Rick Zamora: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7628"&gt;Move DataFrame from_array and from_pandas to Blockwise #7628&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bruce Merry: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7686"&gt;Use BlockwiseDep for map_blocks with block_id or block_info #7686&lt;/a&gt;&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/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 70)&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="a-high-level-graph-for-map-overlap"&gt;
&lt;h1&gt;A high level graph for map overlap&lt;/h1&gt;
&lt;section id="id1"&gt;
&lt;h2&gt;Summary&lt;/h2&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/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 72); &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: “summary”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Investigating a high level graph for Dask’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt; is a project driven by &lt;a class="reference external" href="https://github.com/dask/dask/discussions/7404"&gt;user needs&lt;/a&gt;. People have told us that the time taken just to generate the task graph (before any actual computation takes place) can sometimes be a big user experience problem. So, we’re looking in to ways to improve it.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="id2"&gt;
&lt;h2&gt;Work to date&lt;/h2&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/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 76); &lt;em&gt;&lt;a href="#id2"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “work to date”.&lt;/p&gt;
&lt;/aside&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Merged PR by Genevieve Buckley: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7595"&gt;A HighLevelGraph abstract layer for map_overlap #7595&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This PR defers much of the computation involved in creating the Dask task graph, but does not does not reduce the total end-to-end computation time. Further optimization is therefore required.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="id3"&gt;
&lt;h2&gt;Ongoing work&lt;/h2&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/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 82); &lt;em&gt;&lt;a href="#id3"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “ongoing work”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Followup work includes:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Find number of tasks in overlap layer without materializing the layer #7788 https://github.com/dask/dask/issues/7788&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Implement cull method for ArrayOverlapLayer #7789 https://github.com/dask/dask/issues/7789 (culling is simplifying a Dask graph by removing unnecessary tasks)&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/2021/07/07/high-level-graphs.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="slicing-and-high-level-graphs"&gt;
&lt;h1&gt;Slicing and high level graphs&lt;/h1&gt;
&lt;section id="id4"&gt;
&lt;h2&gt;Summary&lt;/h2&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/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 91); &lt;em&gt;&lt;a href="#id4"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “summary”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Profiling &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt;, we saw that a lot of time is being spent in slicing operations. So, slicing was a logical next step to investigate possible performance improvements with high level graphs.&lt;/p&gt;
&lt;p&gt;Meanwhile, Rick Zamora has been working on the dataframe side of Dask, using high level graphs to improve dataframe slicing/selections.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="id5"&gt;
&lt;h2&gt;Work to date&lt;/h2&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/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 97); &lt;em&gt;&lt;a href="#id5"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “work to date”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;A couple of minor bugfixes/improvements:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Merged PR by Genevieve Buckley: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7787"&gt;SimpleShuffleLayer should compare parts_out with set(self.parts_out) #7787&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Merged PR by Genevieve Buckley: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7775"&gt;Make Layer get_output_keys officially an abstract method #7775&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
&lt;section id="id6"&gt;
&lt;h2&gt;Ongoing work&lt;/h2&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/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 105); &lt;em&gt;&lt;a href="#id6"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “ongoing work”.&lt;/p&gt;
&lt;/aside&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Rick Zamora: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7663"&gt;[WIP] Add DataFrameGetitemLayer to simplify HLG Optimizations #7663&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Genevieve Buckley: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7655"&gt;Array slicing HighLevelGraph layer #7655&lt;/a&gt;&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/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 111)&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="visualization"&gt;
&lt;h1&gt;Visualization&lt;/h1&gt;
&lt;section id="id7"&gt;
&lt;h2&gt;Summary&lt;/h2&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/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 113); &lt;em&gt;&lt;a href="#id7"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “summary”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;We’ve also put some work into making better visualizations for Dask objects (including high level graphs).&lt;/p&gt;
&lt;p&gt;Defining a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;_repr_html_&lt;/span&gt;&lt;/code&gt; method for your classes is a great way to get nice HTML output when you’re working with jupyter notebooks. You can read &lt;a class="reference external" href="http://matthewrocklin.com/blog/2019/07/04/html-repr"&gt;this post&lt;/a&gt; to see more neat HTML representations in other scientific python libraries.&lt;/p&gt;
&lt;p&gt;Dask already uses HTML representations in lots of places (like the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Array&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; classes). We now have new HTML representations for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HighLevelGraph&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Layer&lt;/span&gt;&lt;/code&gt; objects, as well as &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Scheduler&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client&lt;/span&gt;&lt;/code&gt; objects in Dask distributed.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="id8"&gt;
&lt;h2&gt;Work to date&lt;/h2&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/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 121); &lt;em&gt;&lt;a href="#id8"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “work to date”.&lt;/p&gt;
&lt;/aside&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Merged PR by Jacob Tomlinson: &lt;a class="reference external" href="https://github.com/dask/distributed/pull/4857"&gt;Add HTML repr to scheduler_info and incorporate into client and cluster reprs #4857&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Merged PR by Jacob Tomlinson: &lt;a class="reference external" href="https://github.com/dask/distributed/pull/4853"&gt;HTML reprs CLient.who_has &amp;amp; Client.has_what&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Merged PR by Genevieve Buckley: Implementation of HTML repr for HighLevelGraph layers #7763 https://github.com/dask/dask/pull/7763&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Merged PR by Genevieve Buckley &lt;a class="reference external" href="https://github.com/dask/dask/pull/7716"&gt;Automatically show graph visualization in jupyter notebooks #771&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Merged PR by Genevivee Buckley: &lt;a class="reference external" href="https://github.com/dask/dask/pull/7309"&gt;Adding chunks and type information to dask high level graphs #7309&lt;/a&gt;. This PR inserts extra information into the high level graph, so that we can create richer visualizations using this extra context later on.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
&lt;section id="example"&gt;
&lt;h2&gt;Example&lt;/h2&gt;
&lt;section id="before"&gt;
&lt;h3&gt;Before:&lt;/h3&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="o"&gt;&amp;lt;&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;highlevelgraph&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;HighLevelGraph&lt;/span&gt; &lt;span class="n"&gt;at&lt;/span&gt; &lt;span class="mh"&gt;0x7f9851b7e4f0&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="after-html-representation"&gt;
&lt;h3&gt;After (HTML representation):&lt;/h3&gt;
&lt;img src="/images/2021-highlevelgraph-html-repr.png" alt="HTML representation for a Dask high level graph" width="700" height="470"&gt;
&lt;/section&gt;
&lt;section id="after-text-only-representation"&gt;
&lt;h3&gt;After (text-only representation):&lt;/h3&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.datasets&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;timeseries&lt;/span&gt;

&lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;timeseries&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shuffle&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;id&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;shuffle&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;tasks&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="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="n"&gt;compute&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;HighLevelGraph&lt;/span&gt; &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt; &lt;span class="n"&gt;layers&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;
&lt;span class="o"&gt;&amp;lt;&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;highlevelgraph&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;HighLevelGraph&lt;/span&gt; &lt;span class="nb"&gt;object&lt;/span&gt; &lt;span class="n"&gt;at&lt;/span&gt; &lt;span class="mh"&gt;0x7fc259015b80&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt;
 &lt;span class="mf"&gt;0.&lt;/span&gt; &lt;span class="n"&gt;make&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;timeseries&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;94&lt;/span&gt;&lt;span class="n"&gt;aab6e7236cbd9828bcbfb35fe6caee&lt;/span&gt;
 &lt;span class="mf"&gt;1.&lt;/span&gt; &lt;span class="n"&gt;simple&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;shuffle&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;cd01443e43b7a6eb9810ad67992c40b6&lt;/span&gt;
 &lt;span class="mf"&gt;2.&lt;/span&gt; &lt;span class="n"&gt;head&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;simple&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;shuffle&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;cd01443e43b7a6eb9810ad67992c40b6&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This gives us a much more meaningful representation, and is already being used by developers working on high level graphs.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/07/07/high-level-graphs.md&lt;/span&gt;, line 160)&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&gt;
&lt;section id="documentation"&gt;
&lt;h1&gt;Documentation&lt;/h1&gt;
&lt;p&gt;Finally, the documentation around high level graphs is sparse. This is because they’re relatively new, and have also been undergoing quite a bit of change. However, this makes it difficult for people. We’re planning to improve the documentation, for both users and devlopers of Dask.&lt;/p&gt;
&lt;p&gt;If you’d like to follow these discussions, or help out, you can subscribe to the issues:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;For Dask users: &lt;a class="reference external" href="https://github.com/dask/dask/issues/7709"&gt;Update HighLevelGraph documentation #7709&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For Dask developers: &lt;a class="reference external" href="https://github.com/dask/dask/issues/7755"&gt;Document dev process around high level graphs #7755&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/07/07/high-level-graphs/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <published>2021-07-07T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/07/02/ragged-output/</id>
    <title>Ragged output, how to handle awkward shaped results</title>
    <updated>2021-07-02T00:00:00+00:00</updated>
    <author>
      <name>Genevieve Buckley</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/07/02/ragged-output.md&lt;/span&gt;, line 8)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;This blogpost explains some of the difficulties associated with distributed computation and ragged or irregularly shaped outputs. We present a recommended method for using Dask in these circumstances.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/07/02/ragged-output.md&lt;/span&gt;, line 12)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="background"&gt;
&lt;h1&gt;Background&lt;/h1&gt;
&lt;p&gt;Often, we come across workflows where analyzing the data involves searching for features (which may or may not be present) then computing some results from those features.
Because we don’t know ahead of time how many features will be found, we can expect the processing output size to vary.&lt;/p&gt;
&lt;p&gt;For distributed workloads, we need to split up the data, process it, and then recombine the results. That means ragged output can cause cause problems (like broadcasting errors) when Dask combines the output.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/07/02/ragged-output.md&lt;/span&gt;, line 19)&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="problem-constraints"&gt;
&lt;h1&gt;Problem constraints&lt;/h1&gt;
&lt;p&gt;In this blogpost, we’ll look at an example with the following constraints:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Input array data&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A processing function requiring overlap between chunks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The output returned&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/07/02/ragged-output.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="solution"&gt;
&lt;h1&gt;Solution&lt;/h1&gt;
&lt;p&gt;The simplest strategy is a two step process:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Expand the array chunks using the &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html?#dask.array.overlap.overlap"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;overlap&lt;/span&gt;&lt;/code&gt; function&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Use &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html#dask.array.map_blocks"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; with the &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html#dask.array.map_blocks"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;drop_axis&lt;/span&gt;&lt;/code&gt; keyword argument&lt;/a&gt;&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/2021/07/02/ragged-output.md&lt;/span&gt;, line 34)&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-code"&gt;
&lt;h1&gt;Example code&lt;/h1&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;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="n"&gt;arr&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;100&lt;/span&gt;&lt;span class="p"&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;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;50&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;&lt;span class="mi"&gt;50&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;  &lt;span class="c1"&gt;# example input data&lt;/span&gt;
&lt;span class="n"&gt;expanded&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;overlap&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;overlap&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;arr&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;depth&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="n"&gt;boundary&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;reflect&amp;quot;&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;expanded&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;processing_func&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;drop_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="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="nb"&gt;float&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;compute&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/2021/07/02/ragged-output.md&lt;/span&gt;, line 45)&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="multiple-output-types-supported"&gt;
&lt;h1&gt;Multiple output types supported&lt;/h1&gt;
&lt;p&gt;This pattern supports multiple types of output from the processing function, including:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;numpy arrays&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;pandas Series&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;pandas DataFrames&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;You can try this for yourself using any of the example processing functions below, generating dummy data output. Or, you can try out a function of your own.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Random length, 1D output returned&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;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="c1"&gt;# function returns numpy array&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;processing_func&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;random_length&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;7&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;np&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="n"&gt;random_length&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# function returns pandas series&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;processing_func&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;random_length&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;7&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;output_series&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;arange&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;random_length&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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Series&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;output_series&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# function returns pandas dataframe&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;processing_func&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;random_length&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;7&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;x_data&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;arange&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;random_length&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;y_data&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;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="n"&gt;random_length&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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;x&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;x_data&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;y&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;y_data&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/2021/07/02/ragged-output.md&lt;/span&gt;, line 79)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="why-can-t-i-use-map-overlap-or-reduction"&gt;
&lt;h1&gt;Why can’t I use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;reduction&lt;/span&gt;&lt;/code&gt;?&lt;/h1&gt;
&lt;p&gt;Ragged output sizes can cause &lt;a class="reference external" href="https://numpy.org/doc/stable/user/basics.broadcasting.html"&gt;broadcasting&lt;/a&gt; errors when the outputs are combined for some Dask functions.&lt;/p&gt;
&lt;p&gt;However, if ragged output sizes aren’t a constraint for your particular programming problem, then you can continue to use the Dask &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html?#dask.array.overlap.map_overlap"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; and &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html?#dask.array.reduction"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;reduction&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; functions as much as you like.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/07/02/ragged-output.md&lt;/span&gt;, line 85)&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="alternative-solution"&gt;
&lt;h1&gt;Alternative solution&lt;/h1&gt;
&lt;section id="dask-delayed"&gt;
&lt;h2&gt;Dask delayed&lt;/h2&gt;
&lt;p&gt;As an alternative solution, you can use &lt;a class="reference external" href="https://docs.dask.org/en/latest/delayed.html"&gt;Dask delayed&lt;/a&gt; (a tutorial is &lt;a class="reference external" href="https://tutorial.dask.org/01_dask.delayed.html"&gt;available here&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;Advantages:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Your processing function can have any type of output (it not restricted to numpy or pandas objects)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;There is more flexibility in the ways you can use Dask delayed.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Disadvantages:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;You will have to handle combining the outputs yourself.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You will have to be more careful about performance:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;For example, because the code below uses delayed in a list comprehension, it’s very important for performance reasons that we pass in the expected metadata. Fortunately, dask has a &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe-api.html#dask.dataframe.utils.make_meta"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;make_meta&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; function available.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You can read more about performance considerations for Dask delayed and &lt;a class="reference external" href="https://docs.dask.org/en/latest/delayed-best-practices.html"&gt;best practices here&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Example code:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;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="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="n"&gt;arr&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;20&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;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;10&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="nd"&gt;@dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&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;processing_func&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;# returns dummy dataframe output&lt;/span&gt;
    &lt;span class="n"&gt;random_length&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;10&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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;x&amp;#39;&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;arange&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;random_length&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="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="n"&gt;random_length&lt;/span&gt;&lt;span class="p"&gt;)})&lt;/span&gt;

&lt;span class="n"&gt;meta&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;utils&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;make_meta&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;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;int64&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;y&amp;#39;&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;int64&lt;/span&gt;&lt;span class="p"&gt;)])&lt;/span&gt;
&lt;span class="n"&gt;expanded&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;overlap&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;overlap&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;arr&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;depth&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="n"&gt;boundary&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;reflect&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;blocks&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;expanded&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_delayed&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ravel&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;results&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;processing_func&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;meta&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;b&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;blocks&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;concat&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;results&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/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/2021/07/02/ragged-output.md&lt;/span&gt;, line 129)&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="summing-up"&gt;
&lt;h1&gt;Summing up&lt;/h1&gt;
&lt;p&gt;That’s it! We’ve learned how to avoid common errors when working with processing functions returning ragged outputs. The method recommended here works well with multiple output types including: numpy arrays, pandas series, and pandas DataFrames.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/07/02/ragged-output/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <published>2021-07-02T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/06/25/dask-down-under/</id>
    <title>Dask Down Under</title>
    <updated>2021-06-25T00:00:00+00:00</updated>
    <author>
      <name>Nick Mortimer</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/25/dask-down-under.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;Dask Down Under was a special event held for the first time last month during the 2021 Dask Summit.
It featured talks, tutorials, and events tailored specifically for an Australian (and wider Oceania) audience.&lt;/p&gt;
&lt;p&gt;To get involved in the new Pangeo Oceania community group,
&lt;a class="reference external" href="https://confirmsubscription.com/h/j/E30A9F4EAC96EA73"&gt;register your interest 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/2021/06/25/dask-down-under.md&lt;/span&gt;, line 17)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="contents"&gt;
&lt;h1&gt;Contents&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#what-is-dask-down-under"&gt;&lt;span class="xref myst"&gt;What is Dask Down Under?&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#who-came"&gt;&lt;span class="xref myst"&gt;Who came?&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#watch-the-talks"&gt;&lt;span class="xref myst"&gt;Watch the talks&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#whats-next"&gt;&lt;span class="xref myst"&gt;What’s next? Here’s how to get involved!&lt;/span&gt;&lt;/a&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/2021/06/25/dask-down-under.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="what-is-dask-down-under"&gt;
&lt;h1&gt;What is Dask Down Under?&lt;/h1&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Dask down under is a chance for everyone in Oceania to forge links and build community here in our backyard. Dask down under we feature talks, tutorials and panel discussions on using Dask to accelerate research. All levels from beginner to expert are encouraged to attend.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;Dask Down Under involved two days of events:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;5 talks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;2 tutorials&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;1 panel discussion&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;1 meet and greet networking event&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/25/dask-down-under.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="who-came"&gt;
&lt;h1&gt;Who came?&lt;/h1&gt;
&lt;p&gt;There was a strong geoscience theme across Dask Down Under. This reflects the strong scientific community we have in these areas. People came from government organisations, universities, and industry.&lt;/p&gt;
&lt;p&gt;We expected most attendees would be based in the Asia-Pacific region, since those were the timezones targeted by these events.&lt;/p&gt;
&lt;p&gt;Unexpectedly, we also saw a lot of extra traffic at the talks on day one, likely from US timezones. Publicity from Dask Summit emails and tweets mentioning Dask Down Under resulted in a lot of people stopping by to watch. This more than doubled our live attendance during the first event. It was great to see so much interest coming from other parts of the world, too.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/25/dask-down-under.md&lt;/span&gt;, line 43)&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="watch-the-talks"&gt;
&lt;h1&gt;Watch the talks&lt;/h1&gt;
&lt;p&gt;You can watch the talks and tutorials from the Dask Dwon Under workshop on the Dask youtube channel.
The &lt;a class="reference external" href="https://www.youtube.com/playlist?list=PLJ0vO2F_f6OAXBfb_SAF2EbJve9k1vkQX"&gt;full playlist for the workshop is available here&lt;/a&gt;.&lt;/p&gt;
&lt;section id="panel-discussion"&gt;
&lt;h2&gt;Panel discussion&lt;/h2&gt;
&lt;p&gt;A panel discussion was held, bringing together a diverse group of users from novice to expert, academic to commercial. We hope this discussion will start a conversation about using Dask in Australia, how we build our community, contribute and stay in touch with the rest of the world. You can &lt;a class="reference external" href="https://www.youtube.com/watch?v=WlSw7rhwGrA"&gt;watch it here&lt;/a&gt;:&lt;/p&gt;
&lt;iframe width="900" height="506" src="https://www.youtube.com/watch?v=WlSw7rhwGrA" title="YouTube video player" frameborder="0" allow="accelerometer; autoplay; clipboard-write; encrypted-media; gyroscope; picture-in-picture" allowfullscreen&gt;&lt;/iframe&gt;
&lt;p&gt;Moderator: Draga Doncila Pop&lt;br /&gt;
Panelists:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Ben Leighton, CSIRO&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tisham Dhar, Geoscience Australia&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Genevieve Buckley, Dask life science fellow&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Hugo Bowne-Anderson, Coiled&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="invited-talks"&gt;
&lt;h2&gt;Invited talks&lt;/h2&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://www.youtube.com/playlist?list=PLJ0vO2F_f6OAXBfb_SAF2EbJve9k1vkQX"&gt;full playlist for the workshop is available here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Featured talks include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Draga Doncila Pop, &lt;a class="reference external" href="https://www.youtube.com/watch?v=10Ws59NGDaE&amp;amp;amp;list=PLJ0vO2F_f6OAXBfb_SAF2EbJve9k1vkQX&amp;amp;amp;index=2"&gt;Interactive visualization and near real-time analysis on out-of-core satellite images&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tisham Dhar: &lt;a class="reference external" href="https://www.youtube.com/watch?v=MderTABZvyA&amp;amp;amp;list=PLJ0vO2F_f6OAXBfb_SAF2EbJve9k1vkQX&amp;amp;amp;index=3"&gt;Dask DevOps for Remote Sensing&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kirill Kouzoubov: &lt;a class="reference external" href="https://www.youtube.com/watch?v=9-zBmUSk29Q&amp;amp;amp;list=PLJ0vO2F_f6OAXBfb_SAF2EbJve9k1vkQX&amp;amp;amp;index=4"&gt;Patterns for large scale temporal processing of geo-spatial data using Dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ben Leighton and Kim Opie: &lt;a class="reference external" href="https://www.youtube.com/watch?v=Fbh07T1K_IE&amp;amp;amp;list=PLJ0vO2F_f6OAXBfb_SAF2EbJve9k1vkQX&amp;amp;amp;index=6"&gt;Image Processing Using Dask - Using dask and skimage to identity vegetation morphology across the Australian landscape&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Nick Mortimer: &lt;a class="reference external" href="https://www.youtube.com/watch?v=YF_GNJdQRQ4&amp;amp;amp;list=PLJ0vO2F_f6OAXBfb_SAF2EbJve9k1vkQX&amp;amp;amp;index=7"&gt;Making the most of your schedule: From HPC to Local Cluster&lt;/a&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/2021/06/25/dask-down-under.md&lt;/span&gt;, line 74)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="what-s-next"&gt;
&lt;h1&gt;What’s next?&lt;/h1&gt;
&lt;p&gt;Here’s how you can get involved:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;Several people have discussed setting up a new Pangeo Oceania group. You can
&lt;a class="reference external" href="https://confirmsubscription.com/h/j/E30A9F4EAC96EA73"&gt;register your interest here&lt;/a&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;&amp;gt; Soon we&amp;#39;ll start holding regular Pangeo Oceania meetups for sharing information, support, training, and workflow advocacy across our region.  We look forward to you helping to shape the Pangeo Oceania community. And if you have a friend or colleague that should be here too, please share this sign-up link: http://bit.ly/Pangeo_email_signup
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The Python for Atmosphere and Ocean Science (PyAOS) provides information and resources to the user community: https://pyaos.github.io/
To keep the site up-to-date, the first ever PyAOS census is being conducted. It would be great if Python users in the atmosphere and/or ocean science community could take a few minutes to fill out the survey.
https://forms.gle/L84W7bsxmP86G3Ji9&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/06/25/dask-down-under/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="Australia" label="Australia"/>
    <category term="DaskSummit" label="Dask Summit"/>
    <category term="geoscience" label="geoscience"/>
    <published>2021-06-25T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/06/18/early-survey/</id>
    <title>Dask Survey 2021, early anecdotes</title>
    <updated>2021-06-18T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</name>
    </author>
    <content type="html">&lt;p&gt;The annual Dask user survey is under way and currently accepting responses at &lt;a class="reference external" href="https://dask.org/survey"&gt;dask.org/survey&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;This post provides a preview into early results, focusing on anecdotal responses.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/18/early-survey.md&lt;/span&gt;, line 12)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="motivation"&gt;

&lt;p&gt;The Dask user survey helps developers focus and prioritize our larger efforts.  It’s also a fascinating and rewarding dataset of anecdotal use cases of how people use Dask today.  Thank you to everyone who has participated so far, you make a difference.&lt;/p&gt;
&lt;p&gt;The survey is still open, and I encourage people to speak up about their experience.  This blogpost is intended to encourage participation by giving you a sense for how it affects development, and by sharing user stories provided within the survey.&lt;/p&gt;
&lt;p&gt;This article skips all of the quantitative data that we collect, and focuses in on direct feedback listed in the final comments.  For a more quantitative analysis see the posts from previous years by Tom at &lt;a class="reference external" href="https://blog.dask.org/2020/09/22/user_survey"&gt;2020 Dask User Survey Results&lt;/a&gt; and  &lt;a class="reference external" href="https://blog.dask.org/2019/08/05/user-survey"&gt;2019 Dask User Survey Results&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/2021/06/18/early-survey.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&gt;
&lt;section id="how-can-dask-improve"&gt;
&lt;h1&gt;How can Dask Improve?&lt;/h1&gt;
&lt;p&gt;In this post we’re going to look at answers to this one question. This was a long-form response field asking &lt;em&gt;“How can Dask Improve?”&lt;/em&gt;. Looking through some of the responses we see that a few of them fall into some common themes. I’ve grouped them here.&lt;/p&gt;
&lt;p&gt;In each section we’ll include raw responses, followed up with a few comments from me in response.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/18/early-survey.md&lt;/span&gt;, line 26)&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-documentation"&gt;
&lt;h1&gt;Intermediate Documentation&lt;/h1&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;More long-form content about the internals of Dask to understand when things don’t work and why. The “Hacking Dask” tutorial in the Dask 2021 summit was precisely the kind of content I really need, because 90% of my time with Dask is spent not understanding why I’m running out of memory and I feel like I’ve ready all the documentation pages 5 times already (although sometimes I also stumble upon a useful page I’ve never seen before).&lt;/p&gt;
&lt;p&gt;There’s also a dearth of documentation of intermediate topics like blockwise in dask.array. (I think I ended up reverse engineering how it worked from docs, GitHub issue comments, reading the code, and black-box reverse engineering with different functions before I finally “got it”.)&lt;/p&gt;
&lt;p&gt;Improve documentation and error messages to cover more of the 2nd-level problems that people run into beyond the first-level tutorial examples.&lt;/p&gt;
&lt;p&gt;more examples for complex concepts (passing metadata to custom functions, for example). more examples/support for using dask arrays and cupy.&lt;/p&gt;
&lt;p&gt;I think the hardest thing about Dask is debugging performance issues with dask delayed and complex mixing of other libraries and not knowing when things are being pickled or not. I am getting better at reading the performance reports, but I think that better documentation and tutorials surrounding understanding the reports would help me greater than new features. For example, make a tutorial that does some non-trivial dask-delayed work (ie not just computing a mean) that is written against best practices and show how the performance improves with each adopted best practice/explain why things were slow with each step. I think there could also be improvements to the performance reports to point out the slowest 5 parts of your code and what lines they are, and possibly relevant docs links.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;section id="response"&gt;
&lt;h2&gt;Response&lt;/h2&gt;
&lt;p&gt;I really like this theme.  We now have a solid community of intermediate-advanced Dask users that we should empower.  We usually write materials that target the broad base of beginning users, but maybe we should rethink this a bit.  There is a lot of good potential material that advanced users have around performance and debugging that could be fun to publish.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/18/early-survey.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&gt;
&lt;section id="documentation-organization"&gt;
&lt;h1&gt;Documentation Organization&lt;/h1&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Documentation website is sometimes confusing to navigate, better separation of API and examples would help. Maybe this can inspire: &lt;a class="reference external" href="https://documentation.divio.com/"&gt;https://documentation.divio.com/&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;I actually think Dask’s documentation is pretty good. But the docs could use some reorganizing – it is often difficult to find the relevant APIs. And there is an incredible amount of HPC insider knowledge that is required to launch a typical workflow - right now much of this knowledge is hidden in the github issues (which is great! but more of it could be pushed into the FAQs to make it more accessible).&lt;/p&gt;
&lt;p&gt;More detailed documentation and examples. Start to finish examples that do not assume I know very much (about Dask, command line tools, Cloud technologies, Kubernetes, etc.).&lt;/p&gt;
&lt;p&gt;I think an easier introduction to delayed/bags and additional examples for more complex use-cases could be helpful.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;section id="id1"&gt;
&lt;h2&gt;Response&lt;/h2&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/2021/06/18/early-survey.md&lt;/span&gt;, line 52); &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: “response”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;We get alternating praise and scorn for our documentation.  We have what I would call excellent &lt;em&gt;reference documentation&lt;/em&gt;.  In fact, if anyone wants to build a dynamic distributed task scheduler today I’m going to claim that distributed.dask.org is probably the most comprehensive reference out there.&lt;/p&gt;
&lt;p&gt;However, we lack good &lt;em&gt;narrative documentation&lt;/em&gt;, which is the concern raised by most of these comments. This is hard to do because Dask is used in so many &lt;em&gt;different user narratives&lt;/em&gt;.  It’s challenging to orient the Dask documentation around all of them simultaneously.&lt;/p&gt;
&lt;p&gt;I appreciated the direct reference in the first comment to a website with a framework.  In general I’d love to talk to people who lay out documentation semi-professionally and learn more.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/18/early-survey.md&lt;/span&gt;, line 60)&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="functionality"&gt;
&lt;h1&gt;Functionality&lt;/h1&gt;
&lt;p&gt;Here is a soup of various feature requests, there are a few themes among them&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Have a better pandas support (like multi-index), which can help me migrate my existing code to Dask.&lt;/p&gt;
&lt;p&gt;I’d like to see better support for actors. I think having a remote object is a common use case.&lt;/p&gt;
&lt;p&gt;Improve Dataframes - multi index!! More feature parity with Pandas API.&lt;/p&gt;
&lt;p&gt;Maybe a little less machine learning, more “classical” big data applications (CDF, PDEs, particle physics etc.). Not everything is map-reducable.&lt;/p&gt;
&lt;p&gt;Better database integration. Re-writing an SQL query in SQL Alchemy can be very impractical. Would also be great if there were better ways to ensure the process didn’t die from misjudging how much memory was needed per chunk.&lt;/p&gt;
&lt;p&gt;Better diagnostic tools; what operations are bottlenecking a task graph? Support for multiindex.&lt;/p&gt;
&lt;p&gt;I do work that regularly requires sorting a DataFrame by multiple columns. Pandas can do this single-core; H2O and Spark can do this multicore and distributed. But dask cannot sort_values() on multiple columns at all (such as &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.sort_values([&lt;/span&gt; &lt;span class="pre"&gt;&amp;quot;col1&amp;quot;,&lt;/span&gt; &lt;span class="pre"&gt;&amp;quot;col2&amp;quot;&lt;/span&gt; &lt;span class="pre"&gt;,&amp;quot;col3&amp;quot;&lt;/span&gt; &lt;span class="pre"&gt;],&lt;/span&gt; &lt;span class="pre"&gt;ascending=False)&lt;/span&gt;&lt;/code&gt;).&lt;/p&gt;
&lt;p&gt;Type-hints! It is very tedious using Dask in a huge ML-Application without even having the option to do some static type-checking.&lt;/p&gt;
&lt;p&gt;Additionally it is very frustrating that Dask tries to mimic Pandas API, but then 40% of the API doesn’t work (isn’t implemented), or deviates so far from the Pandas API that some parameters aren’t implemented. Only way to find out about that is to read the docs. With some typehints one could mitigate much of this trial-and-error process when switching from Pandas to Dask.&lt;/p&gt;
&lt;p&gt;It’s hard to track everything around dask!!! Actors are a bit unloved, but I find them super useful&lt;/p&gt;
&lt;p&gt;Type annotations for all methods for better IDE (VSCode) support&lt;/p&gt;
&lt;p&gt;I think the Actor model could use a little love&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;section id="id2"&gt;
&lt;h2&gt;Response&lt;/h2&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/2021/06/18/early-survey.md&lt;/span&gt;, line 88); &lt;em&gt;&lt;a href="#id2"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “response”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Interesting trends, not many that I would have expected&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;MultiIndex (well, this was expected)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Actors&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Type hinting for IDE support&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;SQL access&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/18/early-survey.md&lt;/span&gt;, line 97)&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="high-level-optimization"&gt;
&lt;h1&gt;High Level Optimization&lt;/h1&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Needs better physical data independence. Manual data chunking, memory management, query optimization are all a big hassle. Automate those more.&lt;/p&gt;
&lt;p&gt;Dask makes it easy for users with no parallel computing experience to scale up quickly (me), but we have no sense of how to judge our resource needs. It’d be great if Dask had some tools or tutorials that helped me judge the size of my problem (e.g. memory usage). These may already exist, but examples of how to do it may be hard to find.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/18/early-survey.md&lt;/span&gt;, line 103)&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="runtime-stability-and-advanced-troubleshooting"&gt;
&lt;h1&gt;Runtime Stability and Advanced Troubleshooting&lt;/h1&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Stability is the most important factor&lt;/p&gt;
&lt;p&gt;I have answered no to the Long Term Support version of dask but often the really great opportunities are those that arre on demand. The problem is that when these fixes are released, their not well advertised and something under the hood has changed. So, it ends up breaking something else or my particular knowledge of the workings are no longer correct. Dask maintainers have a bit of a weird clique and it can feel as a newbie or a learner that your talked down to or in reality. They don’t have the time to help someone. So they should probably have some more maintainers answering some of the more mundane questions via the blog or via some other method, Things we have seen people do wrong or having difficulty in . A bit of basic, a bit of intermediate and a bit of advanced. If the underlying dask API has changed, then these should be updated with new posts with updates of what has changed. Showing a breakdown of doing it the hard way. So people can see what is done step by step with standard workflows that work. Then vs dask, with less boilerplate and/or speed improvement. If there are places where speed isn’t improved. Show that the difference of where it doesnt work alongside the workflow where it might.&lt;/p&gt;
&lt;p&gt;We have long deployed dask clusters (weeks to months) and have noticed that they sometimes go into a wonky state. We’ve been unable to identify root cause(s). Redeployment is simple and easy when it does occur, but slightly annoying nonetheless.&lt;/p&gt;
&lt;p&gt;My biggest pain point is the scheduler, as I tend to spend time writing infrastructure to manage the scheduler and breaking apart / rewriting tasks graphs to minimize impact on the scheduler.&lt;/p&gt;
&lt;p&gt;As my answers make clear (and from previous conversations with Matt, James, and Genevieve) the biggest improvement I’d like to see is stable releases. Stable from both a runtime point of view (i.e. rock solid Dask distributed), and from an API point of view (so I don’t have to fix my code every couple of weeks). So a big +1 to LTS releases.&lt;/p&gt;
&lt;p&gt;Better error handling/descriptions of errors, better interoperability between (slightly) different versions&lt;/p&gt;
&lt;p&gt;If something goes wrong (in Dask, the batch system, or the interaction between Dask and the batch system), the problem is very opaque and difficult to diagnose. Dask needs significant additional documentation, and probably additional features, to make debugging easier and more transparent.&lt;/p&gt;
&lt;p&gt;Better ways of getting out logs of worker memory usage, especially after dask crashes/failures. Ways of getting performance reports written to log files, rather than html files which don’t write if the dask client process fails.&lt;/p&gt;
&lt;p&gt;Two big problems for me are when dask fails determining what when wrong and how to fix it.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;section id="id3"&gt;
&lt;h2&gt;Response&lt;/h2&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/2021/06/18/early-survey.md&lt;/span&gt;, line 123); &lt;em&gt;&lt;a href="#id3"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “response”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Stability definitely took a dive last December.  I’m feeling good right now though.  There is a lot of good work that should be merged in and released in the next few weeks that I think will significantly improve many of the common pain points.&lt;/p&gt;
&lt;p&gt;However, there are still many significant improvements yet to be made.  I in particular like the theme above in reporting and logging when things fail.  We’re ok at this today, but there is a lot of room for growth.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/18/early-survey.md&lt;/span&gt;, line 129)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="what-s-next"&gt;
&lt;h1&gt;What’s Next?&lt;/h1&gt;
&lt;p&gt;Do the views above fully express your thoughts on where Dask should go, or is there something missing?&lt;/p&gt;
&lt;p&gt;Share your perspective at &lt;a class="reference external" href="https://dask.org/survey"&gt;&lt;strong&gt;dask.org/survey&lt;/strong&gt;&lt;/a&gt;.  The whole process should take less than five minutes.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/06/18/early-survey/"/>
    <summary>The annual Dask user survey is under way and currently accepting responses at dask.org/survey.</summary>
    <published>2021-06-18T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/06/01/dask-distributed-user-journey/</id>
    <title>The evolution of a Dask Distributed user</title>
    <updated>2021-06-01T00:00:00+00:00</updated>
    <author>
      <name>Jacob Tomlinson (NVIDIA)</name>
    </author>
    <content type="html">&lt;p&gt;This week was the 2021 Dask Summit and &lt;a class="reference external" href="https://summit.dask.org/schedule/presentation/20/deploying-dask/"&gt;one of the workshops&lt;/a&gt; that we ran covered many deployment options for Dask Distributed.&lt;/p&gt;
&lt;p&gt;We covered local deployments, SSH, Hadoop, Kubernetes, the Cloud and managed services, but one question that came up a few times was “where do I start?”.&lt;/p&gt;
&lt;p&gt;I wanted to share the journey that I’ve seen many Dask users take in the hopes that you may recognize yourself as being somewhere along this path and it may inform you where to look next.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/01/dask-distributed-user-journey.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="in-the-beginning"&gt;

&lt;p&gt;As a user who is new to Dask you’re likely working your way through &lt;a class="reference external" href="https://docs.dask.org/en/latest/index.html"&gt;the documentation&lt;/a&gt; or perhaps &lt;a class="reference external" href="https://github.com/dask/dask-tutorial"&gt;a tutorial&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;We often introduce the concept of the distributed scheduler early on, but you don’t need it to get initial benefits from Dask. Switching from Pandas to Dask for larger than memory datasets is a common entry point and performs perfectly well using the default threaded 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="c1"&gt;# Switching from this&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/.../2018-*-*.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;account_id&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;balance&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="c1"&gt;# To this&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/.../2018-*-*.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;account_id&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;balance&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;But by the time you’re a few pages into the documentation you’re already being encouraged to create &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client()&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LocalCluster()&lt;/span&gt;&lt;/code&gt; objects.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;&lt;strong&gt;Note&lt;/strong&gt;: When you create a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client()&lt;/span&gt;&lt;/code&gt; with no arguments/config set Dask will launch a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LocalCluster()&lt;/span&gt;&lt;/code&gt; object for you under the hood. So often &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client()&lt;/span&gt;&lt;/code&gt; is equivalent to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client(LocalCluster())&lt;/span&gt;&lt;/code&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;This is a common area for users to stick around in, launch a local distributed scheduler and do your work maximising the resources on your local machine.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/.../2018-*-*.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;account_id&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;balance&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;/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/2021/06/01/dask-distributed-user-journey.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="breaking-free-from-your-machine"&gt;
&lt;h1&gt;Breaking free from your machine&lt;/h1&gt;
&lt;p&gt;Once you get used to task graphs and work scheduling you may begin thinking about how you can expand your computation beyond your local machine.&lt;/p&gt;
&lt;p&gt;Our code doesn’t really need to change much, we are already connecting a client and doing Dask work, all we need are more networked machines with the same user environments, data, etc.&lt;/p&gt;
&lt;p&gt;Personally I used to work in an organisation where every researcher was given a Linux desktop under their desk. These machines were on a LAN and had Active Directory and user home directories stored on a storage server. This meant you could sit down at any desk and log in and have a consistent experience. This also meant you could SSH to another machine on the network and your home directory would be there with all your files including your data and conda environments.&lt;/p&gt;
&lt;p&gt;This is a common setup in many organisations and it can be tempting to SSH onto the machines of folks who may not be fully utilising their machine and run your work there. And I’m sure you ask first right!&lt;/p&gt;
&lt;p&gt;Organisations may also have servers in racks designated for computational use and the setup will be similar. You can SSH onto them and home directories and data are available via network storage.&lt;/p&gt;
&lt;p&gt;With Dask Distributed you can start to expand your workload onto these machines using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;SSHCluster&lt;/span&gt;&lt;/code&gt;. All you need is your SSH keys set up so you can log into those machines without a password.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;SSHCluster&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;SSHCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="p"&gt;[&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;localhost&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;alices-desktop.lan&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;bobs-desktop.lan&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;team-server.lan&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;/data/.../2018-*-*.csv&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;account_id&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;balance&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now the same workload can run on all of the CPUs in our little ad-hoc cluster, using all the memory and pulling data from the same shared storage.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/01/dask-distributed-user-journey.md&lt;/span&gt;, line 84)&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="moving-to-a-compute-platform"&gt;
&lt;h1&gt;Moving to a compute platform&lt;/h1&gt;
&lt;p&gt;Using (and abusing) hardware like desktops and shared servers will get you reasonably far, but probably to the dismay of your IT team.&lt;/p&gt;
&lt;p&gt;Organisations who have many users trying to perform large compute workloads will probably be thinking about or already have some kind of platform that is designated for running this work.&lt;/p&gt;
&lt;p&gt;The platforms your organisation has will be the result of many somewhat arbitrary technology choices. What programming languages does your company use? What deals did vendors offer at the time of procurement? What skills do the current IT staff have? What did your CTO have for breakfast the day they chose a vendor?&lt;/p&gt;
&lt;p&gt;I’m not saying these decisions are made thoughtlessly, but the criteria that are considered are often orthogonal to how the resource will ultimately be used by you. At Dask we support whatever platform decisions your organisations make. We try to build deployment tools for as many popular platforms as we can including:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Hadoop via &lt;a class="reference external" href="https://github.com/dask/dask-yarn"&gt;dask-yarn&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kubernetes via &lt;a class="reference external" href="https://github.com/dask/dask-kubernetes"&gt;dask-kubernetes&lt;/a&gt; and the &lt;a class="reference external" href="https://github.com/dask/helm-chart"&gt;helm chart&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;HPC (with schedulers like SLURM, PBS and SGE) via &lt;a class="reference external" href="https://github.com/dask/dask-jobqueue"&gt;dask-jobqueue&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Cloud platforms (including AWS, Azure and GCP) with &lt;a class="reference external" href="https://github.com/dask/dask-cloudprovider"&gt;dask-cloudprovider&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;As a user within an organisations you may have been onboarded to one of these platforms. You’ve probably been given some credentials and a little training on how to launch jobs on it.&lt;/p&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-foo&lt;/span&gt;&lt;/code&gt; tools listed above are designed to sit on top of those platforms and submit jobs on your behalf as if they were individual compute jobs. But instead of submitting a Python script to the platform we submit Dask schedulers and workers and then connect to them to leverage the provisioned resource. Clusters on top of clusters.&lt;/p&gt;
&lt;p&gt;With this approach your IT team has full control over the compute resource. They can ensure folks get their fair share with quotas and queues. But you as a user gets the same Dask experience you are used to on your local machine.&lt;/p&gt;
&lt;p&gt;Your data may be in a slightly different place on these platforms though. Perhaps you are on the cloud and your data is in object storage for example. Thankfully we have tools built on &lt;a class="reference external" href="https://filesystem-spec.readthedocs.io/en/latest/"&gt;fsspec&lt;/a&gt; like &lt;a class="reference external" href="https://github.com/dask/s3fs"&gt;s3fs&lt;/a&gt; or &lt;a class="reference external" href="https://pypi.org/project/adlfs/"&gt;adlfs&lt;/a&gt; we can read this data in pretty much the same way. So still not much change to your workflow.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_cloudprovider.azure&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;AzureVMCluster&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;AzureVMCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;resource_group&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;lt;resource group&amp;gt;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                         &lt;span class="n"&gt;vnet&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;lt;vnet&amp;gt;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                         &lt;span class="n"&gt;security_group&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;lt;security group&amp;gt;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                         &lt;span class="n"&gt;n_workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;adl://.../2018-*-*.csv&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;account_id&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;balance&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;/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/2021/06/01/dask-distributed-user-journey.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="centralizing-your-dask-resources"&gt;
&lt;h1&gt;Centralizing your Dask resources&lt;/h1&gt;
&lt;p&gt;When your organisation gets enough folks adopting and using Dask it may be time for your IT team to step in and provide you with a managed service. Having many users submitting many ad-hoc clusters in a myriad of ways is likely to be less efficient than a centrally managed and more importantly ordained service from IT.&lt;/p&gt;
&lt;p&gt;The motivation to move to a managed service is often driven at the organisational level rather than by individuals. Once you’ve reached this stage of Dask usage you’re probably quite comfortable with your workflows and it may be inconvenient to change them. However the level of Dask deployment knowledge you’ve acquired to reach this stage is probably quite large, and as Dask usage at your organization grows it’s not practical to expect everyone to reach the same level of competency.&lt;/p&gt;
&lt;p&gt;At the end of the day being an expert in deploying distributed systems is probably not listed in your job description and you probably have something more important to be getting on with like data science, finance, physics, biology or whatever it is Dask is helping you do.&lt;/p&gt;
&lt;p&gt;You may also be feeling some pressure from IT. You are running clusters on top of clusters and to them your Dask cluster is a black box and this can make them comfortable as they are the ones responsible for this hardware. It is common to feel constrained by your IT team, I know because I’ve been a sysadmin and used to constrain folks. But the motivations of your IT team are good ones, they are trying to save the organisation money, make best use of limited resources and ultimately get the IT out of your way so that you can get on with your job. So lean into this, engage with them, share your Dask knowledge and offer to become a pilot user for whatever solution they end up building.&lt;/p&gt;
&lt;p&gt;One approach you could recommend they take is to deploy &lt;a class="reference external" href="https://gateway.dask.org/"&gt;Dask Gateway&lt;/a&gt;. This can be deployed by an administrator and provides a central hub which launches Dask clusters on behalf of users. It supports many types of authentication so it can hook into whatever your organisation uses and supports many of the same backend compute platforms that the standalone tools do, including Kubernetes, Hadoop and HPC.&lt;/p&gt;
&lt;p&gt;This will allow them to ensure security settings are correct and consistent across clusters. If you are using containers they probably want you to use some official images which are regularly updated and vulnerability scanned. It may also give them more insight into what types of workloads folks are running and plan future systems more accurately. By using Dask Gateway this puts the control and responsibility of these things onto their side of the fence.&lt;/p&gt;
&lt;p&gt;Users will need to authenticate with the gateway, but then can launch Dask clusters in a platform agnostic way.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_gateway&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;Gateway&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="n"&gt;gateway&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Gateway&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;address&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;http://daskgateway.myorg.com&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;auth&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;kerberos&amp;quot;&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;gateway&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;new_cluster&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;/data/.../2018-*-*.csv&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;account_id&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;balance&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Again reading your data requires some knowledge on how it is stored on the underlying compute platform you the gateway is using, but the changes required are minimal.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/01/dask-distributed-user-journey.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="managed-services"&gt;
&lt;h1&gt;Managed services&lt;/h1&gt;
&lt;p&gt;If your organisation is too small to have an IT team to manage this for you, or you just have a preference for managed services, there are startups popping up to provide this to you as a service including &lt;a class="reference external" href="https://coiled.io/"&gt;Coiled&lt;/a&gt; and &lt;a class="reference external" href="https://www.saturncloud.io/s/home/"&gt;Saturn Cloud&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/2021/06/01/dask-distributed-user-journey.md&lt;/span&gt;, line 160)&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-platforms"&gt;
&lt;h1&gt;Future platforms&lt;/h1&gt;
&lt;p&gt;Today the large cloud vendors have managed data science platforms including &lt;a class="reference external" href="https://aws.amazon.com/sagemaker/"&gt;AWS Sagemaker&lt;/a&gt;, &lt;a class="reference external" href="https://azure.microsoft.com/en-gb/services/machine-learning/"&gt;Azure Machine Learning&lt;/a&gt; and &lt;a class="reference external" href="https://cloud.google.com/vertex-ai"&gt;Google Cloud AI Platform&lt;/a&gt;. But these do not include Dask as a service.&lt;/p&gt;
&lt;p&gt;These cloud services are focussed on batch processing and machine learning today, but these clouds also have managed services for Spark and other compute cluster offerings. With Dask’s increasing popularity it wouldn’t surprise me if managed Dask services are released by these cloud vendors in the years to follow.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/06/01/dask-distributed-user-journey.md&lt;/span&gt;, line 166)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="summary"&gt;
&lt;h1&gt;Summary&lt;/h1&gt;
&lt;p&gt;One of the most powerful features of Dask is that your code can stay pretty much the same regardless of how big or complex the distributed compute cluster is. It scales from a single machine to thousands of servers with ease.&lt;/p&gt;
&lt;p&gt;But scaling up requires both user and organisational growth and folks already seem to be treading a common path on that journey.&lt;/p&gt;
&lt;p&gt;Hopefully this post will give you an idea of where you are on that path and where to jump to next. Whether you’re new to the community and discovering the power of multi-core computing or an old hand who is trying to wrangle hundreds of users who all love Dask, good luck!&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/06/01/dask-distributed-user-journey/"/>
    <summary>This week was the 2021 Dask Summit and one of the workshops that we ran covered many deployment options for Dask Distributed.</summary>
    <category term="Distributed" label="Distributed"/>
    <category term="Organisations" label="Organisations"/>
    <category term="Tools" label="Tools"/>
    <published>2021-06-01T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/05/25/user-survey/</id>
    <title>The 2021 Dask User Survey is out now</title>
    <updated>2021-05-25T00:00:00+00:00</updated>
    <author>
      <name>Genevieve Buckley</name>
    </author>
    <content type="html">&lt;p&gt;The Dask User Survey is out again! Tell us how you use Dask, and help us make it better for everyone.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://forms.gle/F7QSGpSHwBWu8NCg8"&gt;Click this link to take the survey&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/2021/05/25/user-survey.md&lt;/span&gt;, line 13)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="why-take-the-survey"&gt;

&lt;p&gt;Feedback from users is very important. It helps give us a clear picture who our users are and what is important to them. Your responses will inform prioritization for Dask development and improve the experience for the Dask community.&lt;/p&gt;
&lt;p&gt;We expect the survey to take no more than 5-10 minutes. It has the following short sections:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;How do you use Dask?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How could Dask improve?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;What other tools do you use with Dask?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Optional: What do you work on?&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/2021/05/25/user-survey.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="survey-results-from-previous-years"&gt;
&lt;h1&gt;Survey results from previous years&lt;/h1&gt;
&lt;p&gt;We will also publish answers to non-sensitive questions in our annual survey review to help keep everyone informed.&lt;/p&gt;
&lt;p&gt;You can see the results from previous user surveys here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2020/09/22/user_survey"&gt;2020 Dask User Survey Results&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2019/08/05/user-survey"&gt;2019 Dask User Survey Results&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/05/25/user-survey/"/>
    <summary>The Dask User Survey is out again! Tell us how you use Dask, and help us make it better for everyone.</summary>
    <category term="UserSurvey" label="User Survey"/>
    <published>2021-05-25T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/05/24/life-science-summit-workshop/</id>
    <title>Life sciences at the 2021 Dask Summit</title>
    <updated>2021-05-24T00:00:00+00:00</updated>
    <author>
      <name>Genevieve Buckley</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/05/24/life-science-summit-workshop.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;The Dask life science workshop ran as part of the 2021 Dask Summit. Lightning talks from this workshop are &lt;a class="reference external" href="https://www.youtube.com/playlist?list=PLJ0vO2F_f6OBAY6hjRHM_mIQ9yh32mWr0"&gt;available here&lt;/a&gt;, and you can read on for a summary of the event.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/05/24/life-science-summit-workshop.md&lt;/span&gt;, line 13)&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-is-the-dask-life-science-workshop"&gt;
&lt;h1&gt;What is the Dask life science workshop?&lt;/h1&gt;
&lt;p&gt;The Dask life science workshop ran as part of the 2021 Dask Summit. Currently many people in life sciences use Dask, but individual groups are relatively isolated from one another. This workshop gave us an opportunity to learn from each other, as well as opportunities to identify common frustrations and areas for improvement.&lt;/p&gt;
&lt;p&gt;The workshop involved:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Pre-recorded lightning talks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Interactive discussion times (accessible across timezones in Europe, Oceania, and the Americas)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Asynchronous text chat throughout the Dask Summit&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/05/24/life-science-summit-workshop.md&lt;/span&gt;, line 23)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="if-i-missed-it-how-can-i-catch-up"&gt;
&lt;h1&gt;If I missed it, how can I catch up?&lt;/h1&gt;
&lt;p&gt;If you missed the Dask Summit, you can catch up on YouTube.
There is a playlist of all the life science lightning talks &lt;a class="reference external" href="https://www.youtube.com/playlist?list=PLJ0vO2F_f6OBAY6hjRHM_mIQ9yh32mWr0"&gt;available here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;You can also join our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;#life-science&lt;/span&gt;&lt;/code&gt; channel on Slack:
&lt;a class="reference external" href="https://join.slack.com/t/dask/shared_invite/zt-mfmh7quc-nIrXL6ocgiUH2haLYA914g"&gt;Click here for an invitation link&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/2021/05/24/life-science-summit-workshop.md&lt;/span&gt;, line 31)&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-came"&gt;
&lt;h1&gt;Who came?&lt;/h1&gt;
&lt;p&gt;We invited attendees at the life science workshop to do a short Q&amp;amp;A about their work with Dask. This is a small subset of the people who joined us, many people came to the conference and did not do a Q&amp;amp;A.&lt;/p&gt;
&lt;p&gt;The responses give us an overview of the diversity of work people in the community are doing. In no particular order, here are some of those Q&amp;amp;As:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Name:&lt;/strong&gt; Tom White&lt;br /&gt;
&lt;strong&gt;Timezone:&lt;/strong&gt; EU/UK&lt;br /&gt;
&lt;strong&gt;What kind of science do you work on?&lt;/strong&gt; Statistical genetics&lt;br /&gt;
&lt;strong&gt;Something you’ve tried (or would like to try) with Dask?&lt;/strong&gt; Run per-row linear regressions at scale.&lt;br /&gt;
&lt;strong&gt;What do you want to do next with Dask?&lt;/strong&gt; Collaborative optimization of a public workflow (GWAS).&lt;br /&gt;
&lt;strong&gt;Lightning talk:&lt;/strong&gt; &lt;a class="reference external" href="https://www.youtube.com/watch?v=qt6YsHoPpZs&amp;amp;amp;list=PLJ0vO2F_f6OBAY6hjRHM_mIQ9yh32mWr0&amp;amp;amp;index=2"&gt;click here&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Name:&lt;/strong&gt; Giovanni Palla&lt;br /&gt;
&lt;strong&gt;Affiliation:&lt;/strong&gt; Helmholtz Center Munich&lt;br /&gt;
&lt;strong&gt;Timezone:&lt;/strong&gt; Europe&lt;br /&gt;
&lt;strong&gt;What kind of science do you work on?&lt;/strong&gt; Computational Biology and Spatial transcriptomics&lt;br /&gt;
&lt;strong&gt;Something you’ve tried (or would like to try) with Dask?&lt;/strong&gt; &lt;a class="reference external" href="http://image.dask.org/en/latest/"&gt;dask-image&lt;/a&gt; for image processing.&lt;br /&gt;
**What do you want to do next with Dask? Further integration with &lt;a class="reference external" href="https://squidpy.readthedocs.io/en/latest/"&gt;Squidpy&lt;/a&gt;.&lt;br /&gt;
**Lightning talk:** &lt;a class="reference external" href="https://www.youtube.com/watch?v=sGr7O8spfvE&amp;amp;amp;list=PLJ0vO2F_f6OBAY6hjRHM_mIQ9yh32mWr0&amp;amp;amp;index=8"&gt;click here&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Name:&lt;/strong&gt; Isaac Virshup&lt;br /&gt;
&lt;strong&gt;Affiliation:&lt;/strong&gt; University of Melbourne. Open source projects Scanpy and AnnData
&lt;strong&gt;Timezone:&lt;/strong&gt; AEST&lt;br /&gt;
&lt;strong&gt;What kind of science do you work on?&lt;/strong&gt; Single cell omics data.&lt;br /&gt;
&lt;strong&gt;Something you’ve tried (or would like to try) with Dask?&lt;/strong&gt;&lt;br /&gt;
I’ve used dask for some nested embarrassingly parallel calculations. Having an intelligent scheduler with good monitoring made this task as easy as it should be, especially compared with multiprocessing or joblib.&lt;br /&gt;
&lt;strong&gt;What do you want to do next with Dask?&lt;/strong&gt;&lt;br /&gt;
I would love to get AnnData, a container for working with single cell assays integrated with dask. Dataset sizes in this field are constantly increasing, and it would be good to be able to work with the coolest new dataset regardless of available RAM.&lt;br /&gt;
Since we rely heavily on sparse arrays, a key step towards this will be getting better sparse array support (CSC and CSR especially) inside dask. After all, it’s not great if our strategy for scaling out requires many times the total memory! As a maintainer, I’m interested in hearing people’s experience with distributing tools that integrate well with dask.&lt;br /&gt;
&lt;strong&gt;Lightning talk:&lt;/strong&gt; &lt;a class="reference external" href="https://www.youtube.com/watch?v=e8pWpRo5Ars&amp;amp;amp;list=PLJ0vO2F_f6OBAY6hjRHM_mIQ9yh32mWr0&amp;amp;amp;index=14"&gt;click here&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Name:&lt;/strong&gt; Anna Kreshuk&lt;br /&gt;
&lt;strong&gt;Affiliation:&lt;/strong&gt; European Molecular Biology Laboratory&lt;br /&gt;
&lt;strong&gt;Timezone:&lt;/strong&gt; CEST (GMT+2)&lt;br /&gt;
&lt;strong&gt;What kind of science do you work on?&lt;/strong&gt; Machine learning for microscopy image analysis.&lt;br /&gt;
&lt;strong&gt;Something you’ve tried (or would like to try) with Dask?&lt;/strong&gt; We run a lot of image processing workflows and want to see how Dask can be exploited in this context.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Name:&lt;/strong&gt; Beth Cimini&lt;br /&gt;
&lt;strong&gt;Affiliation:&lt;/strong&gt; Broad Institute&lt;br /&gt;
&lt;strong&gt;Timezone:&lt;/strong&gt; US-East&lt;br /&gt;
&lt;strong&gt;What kind of science do you work on?&lt;/strong&gt; User friendly image analysis tools for microscopy imaging.&lt;br /&gt;
&lt;strong&gt;Something you’ve tried (or would like to try) with Dask?&lt;/strong&gt; Making Dask work in CellProfiler, to make it easy to analyze big images in high throughput!&lt;br /&gt;
&lt;strong&gt;Lightning talk:&lt;/strong&gt; &lt;a class="reference external" href="https://www.youtube.com/playlist?list=PLJ0vO2F_f6OBAY6hjRHM_mIQ9yh32mWr0"&gt;click here&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Name:&lt;/strong&gt; Volker Hilsenstein&lt;br /&gt;
&lt;strong&gt;Affiliation:&lt;/strong&gt; EMBL / Alexandrov lab&lt;br /&gt;
&lt;strong&gt;Timezone:&lt;/strong&gt; Central European Summer Time&lt;br /&gt;
&lt;strong&gt;What kind of science do you work on?&lt;/strong&gt; Spatial Metabolomics, combining microscopy and mass spectrometry.&lt;br /&gt;
&lt;strong&gt;Something I would like to try with dask:&lt;/strong&gt; fusing large mosaics of individual images or image volumes for which affine transformation into a joint coordinate system are available.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Name:&lt;/strong&gt; Marvin Albert&lt;br /&gt;
&lt;strong&gt;Affiliation:&lt;/strong&gt; University of Zurich&lt;br /&gt;
&lt;strong&gt;Timezone:&lt;/strong&gt; UTC/GMT +2&lt;br /&gt;
&lt;strong&gt;What kind of science do you work on?&lt;/strong&gt; Life sciences / image analysis&lt;br /&gt;
&lt;strong&gt;Something you’ve tried (or would like to try) with Dask? What do you want to do next with Dask?&lt;/strong&gt; Parallelise / reduce the memory footprint of image processing tasks and define workflows that can run on different compute environments.&lt;br /&gt;
&lt;strong&gt;Lightning talk:&lt;/strong&gt; &lt;a class="reference external" href="https://www.youtube.com/watch?v=YIblUvonMvo&amp;amp;amp;list=PLJ0vO2F_f6OBAY6hjRHM_mIQ9yh32mWr0&amp;amp;amp;index=9"&gt;click here&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Name:&lt;/strong&gt; Jordao Bragantini&lt;br /&gt;
&lt;strong&gt;Affiliation:&lt;/strong&gt; CZ Biohub&lt;br /&gt;
&lt;strong&gt;Timezone:&lt;/strong&gt; Pacific Daylight Time (UTC -7)&lt;br /&gt;
&lt;strong&gt;What kind of science do you work on?&lt;/strong&gt; Light-sheet microscopy&lt;br /&gt;
&lt;strong&gt;Something you’ve tried (or would like to try) with Dask?&lt;/strong&gt; Image processing of very large data.&lt;br /&gt;
&lt;strong&gt;What do you want to do next with Dask?&lt;/strong&gt; Implement algorithms for cell segmentation.&lt;br /&gt;
&lt;strong&gt;Lightning talk:&lt;/strong&gt; &lt;a class="reference external" href="https://www.youtube.com/watch?v=xadb-oXMFKI&amp;amp;amp;list=PLJ0vO2F_f6OBAY6hjRHM_mIQ9yh32mWr0&amp;amp;amp;index=3"&gt;click here&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Name:&lt;/strong&gt; Josh Moore&lt;br /&gt;
&lt;strong&gt;Affiliation:&lt;/strong&gt; Open Microscopy Environment (OME)&lt;br /&gt;
&lt;strong&gt;Timezone:&lt;/strong&gt; CEST&lt;br /&gt;
&lt;strong&gt;What kind of science do you work on?&lt;/strong&gt; Bioimaging (infrastructure for RDM)&lt;br /&gt;
&lt;strong&gt;Something you’ve tried (or would like to try) with Dask?&lt;/strong&gt; Accessing large image (Zarr) volumes over HTTP, primarily.
What do you want to do next with Dask? Improve pre-fetching for typical usage patterns, possibly integrating multiscale data (i.e. google maps zooming)&lt;br /&gt;
&lt;strong&gt;Lightning talk:&lt;/strong&gt; &lt;a class="reference external" href="https://www.youtube.com/watch?v=6PerbQhcupM&amp;amp;amp;list=PLJ0vO2F_f6OBAY6hjRHM_mIQ9yh32mWr0&amp;amp;amp;index=1"&gt;click here&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Name:&lt;/strong&gt; Jackson Maxfield Brown&lt;br /&gt;
&lt;strong&gt;Timezone:&lt;/strong&gt; PST&lt;br /&gt;
&lt;strong&gt;What kind of science do you work in?&lt;/strong&gt; Cell biology, specifically microscopy and computational biology.&lt;br /&gt;
&lt;strong&gt;Something you’ve tried (or would like to try) with Dask?&lt;/strong&gt; Built a metadata aware / backed microscopy imaging reading library that uses Dask to read any size image w/ chunking by metadata dimension information. As well as TB-scale image processing pipelines using Dask + Prefect.&lt;br /&gt;
&lt;strong&gt;What do you want to do next with Dask?&lt;/strong&gt; Tighter integration with other libraries. I see cuCim from the RAPIDs team and would love to extend work with them to have a more general “bio-image-spec” so we can all play nicely together.&lt;br /&gt;
&lt;strong&gt;Lightning talk:&lt;/strong&gt; &lt;a class="reference external" href="https://www.youtube.com/watch?v=LNa_gGpSnvc&amp;amp;amp;list=PLJ0vO2F_f6OBAY6hjRHM_mIQ9yh32mWr0&amp;amp;amp;index=8"&gt;click here&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Name:&lt;/strong&gt; Gregory R. Lee&lt;br /&gt;
&lt;strong&gt;Affiliation:&lt;/strong&gt; Quansight&lt;br /&gt;
&lt;strong&gt;Timezone:&lt;/strong&gt; EST (UTC-5)&lt;br /&gt;
&lt;strong&gt;What kind of science do you work on?&lt;/strong&gt; Scientific software development (with a background doing research in magnetic resonance imaging).&lt;br /&gt;
&lt;strong&gt;Something you’ve tried (or would like to try) with Dask?&lt;/strong&gt;&lt;br /&gt;
In past research work, I used Dask primarily in two scenarios, both on a single workstation:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;To achieve multi-threading by processing image blocks in parallel on the CPU (e.g. like in dask-image)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Serial blockwise processing of large volumetric data on the GPU (i.e. CuPy arrays of 10-100 GB in size) to reduce peak memory requirements.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;strong&gt;What do you want to do next with Dask?&lt;/strong&gt;&lt;br /&gt;
Audit scikit-image functions to determine which can easily be accelerated using block-wise approaches as in dask-image. Ideally a subset of functions would work directly with dask-arrays as inputs rather than requiring users to learn about Dask’s map_overlap, etc. to use this feature.&lt;br /&gt;
&lt;strong&gt;Lightning talk:&lt;/strong&gt; &lt;a class="reference external" href="https://www.youtube.com/watch?v=vPorCnEhM6g&amp;amp;amp;list=PLJ0vO2F_f6OBAY6hjRHM_mIQ9yh32mWr0&amp;amp;amp;index=16"&gt;click 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/2021/05/24/life-science-summit-workshop.md&lt;/span&gt;, line 126)&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-s-next"&gt;
&lt;h1&gt;What’s next?&lt;/h1&gt;
&lt;p&gt;Dask is now considering holding “office hours” for the life science community. If we can find enough maintainers able to host one-hour Q&amp;amp;A sessions, then we’ll trial this for a short period of time.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/05/24/life-science-summit-workshop/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="DaskSummit" label="Dask Summit"/>
    <category term="lifescience" label="life science"/>
    <published>2021-05-24T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/05/21/stability/</id>
    <title>Stability of the Dask library</title>
    <updated>2021-05-21T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</name>
    </author>
    <content type="html">&lt;p&gt;Dask is moving fast these days. Sometimes we break things as a result.&lt;/p&gt;
&lt;p&gt;Historically this hasn’t been a problem, according to our survey last year
most users were fairly happy with Dask’s stability.&lt;/p&gt;
&lt;img src="/images/2020_survey/2020_27_0.png"&gt;
&lt;p&gt;However the last year has seen a lot of evolution of the project,
which in turn causes code churn.
This can cause friction for downstream users today,
but also means more-than-incremental changes for the future.
We’ve optimized a little bit for long-term growth over short-term stability.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/05/21/stability.md&lt;/span&gt;, line 21)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="motivation-for-change"&gt;

&lt;p&gt;There are two structural things driving some of these changes:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;An increase in computational scale&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;An increase in organizational scale&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/2021/05/21/stability.md&lt;/span&gt;, line 28)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="computational-scale"&gt;
&lt;h1&gt;Computational Scale&lt;/h1&gt;
&lt;p&gt;Dask today is used across a wider range of problems,
a more diverse set of hardware,
and at larger scales more routinely than before.&lt;/p&gt;
&lt;p&gt;Addressing this increase in scale across many dimensions has caused us to
redesign Dask’s internal infrastructure in several ways.&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;We’ve changed how Dask graphs are represented and communicated to the scheduler&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We’ve pulled out Dask’s internal state machines and made them more formalized&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We’ve rewritten large chunks of the scheduler in Cython&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We’ve overhauled how we serialize messages that go between all Dask servers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We’re now tracking memory with much finer granularity than we did before&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;… and more&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We’ve been doing all of these internal changes with minimal impact to the
myriad of downstream user communities (Xarray, Prefect, RAPIDS, XGBoost, …).
This is largely due to those downstream developer communities,
who help to identify, isolate, and work through the subtle tremors that occur
on the surface when we make these subsurface shifts.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/05/21/stability.md&lt;/span&gt;, line 50)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="organizational-scale"&gt;
&lt;h1&gt;Organizational scale&lt;/h1&gt;
&lt;p&gt;Historically Dask’s core was maintained by a relatively small set of people,
mostly at Anaconda.
There were dozens of developers that worked on various &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-foo&lt;/span&gt;&lt;/code&gt; projects, but
only a small group that thought about things like serialization, state
machines, and so on.
In particular I personally tracked every issue and knew the entire project.
Whenever a potential conflict arose I was usually able to identify it early.&lt;/p&gt;
&lt;p&gt;This has all changed dramatically.&lt;/p&gt;
&lt;p&gt;First, there are now several multi-company teams working on different parts of
Dask internals.&lt;/p&gt;
&lt;p&gt;Second, we’ve also taken some time to redesign parts of Dask internals to make them more maintainable.
Dask scheduling is like a finely made clock.
Historically parts of that clock were built and designed by individuals with a craftsman-like approach.
Now we’re redesigning things with more of a group mindset.
This results in more maintainable designs,
but it also means that we’re taking apart the clock and putting it back together.
It takes a little while to find all of the missing parts :)&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/05/21/stability.md&lt;/span&gt;, line 73)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="how-this-affects-you-today"&gt;
&lt;h1&gt;How this affects you today&lt;/h1&gt;
&lt;p&gt;This all started around when we switched to Calendar Versioning at the end of last year
(Dask version &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;2.30.1&lt;/span&gt;&lt;/code&gt; rolled over into &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;2020.12.0&lt;/span&gt;&lt;/code&gt; last December). You may
have noticed&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;an increased sensitivity to version mismatches (as we change the Dask
protocol different versions of Dask can no longer talk to each other well)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;releases with stability issues (2020.12 was particularly rough)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/community/issues/155"&gt;tighter pinning&lt;/a&gt; between dask and distributed versions during releases&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/2021/05/21/stability.md&lt;/span&gt;, line 84)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="how-this-will-affect-you"&gt;
&lt;h1&gt;How this will affect you&lt;/h1&gt;
&lt;p&gt;We’ve merged in a &lt;a class="reference external" href="https://github.com/dask/dask/pull/7620"&gt;PR&lt;/a&gt;
to change the default behavior when moving &lt;a class="reference external" href="https://docs.dask.org/en/latest/high-level-graphs.html"&gt;high level graphs&lt;/a&gt;
to the scheduler for Dask Dataframes. This should result in much
less delay when submitting large computations and almost no delay in
optimization. It also opens up a conduit for us to send &lt;em&gt;a lot&lt;/em&gt; more semantic
information to the scheduler about your computation, which can result in new
visualizations and smarter scheduling in the future.&lt;/p&gt;
&lt;p&gt;It will also probably break some things.&lt;/p&gt;
&lt;p&gt;To be clear, all tests pass among Dask, distributed, xarray, prefect, rapids,
and other downstream projects. We’ve done our homework here, but almost certainly we’ve missed something.&lt;/p&gt;
&lt;p&gt;This is only one of several larger changes happening in the coming months.
We appreciate your patience and your engagement as we make some of these larger shifts.
For better or worse end users are the final testing suite :)&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/05/21/stability/"/>
    <summary>Dask is moving fast these days. Sometimes we break things as a result.</summary>
    <published>2021-05-21T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/05/07/skeleton-analysis/</id>
    <title>Skeleton analysis</title>
    <updated>2021-05-07T00:00:00+00:00</updated>
    <author>
      <name>Genevieve Buckley</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/05/07/skeleton-analysis.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;In this blogpost, we show how to modify a skeleton network analysis with Dask to work with constrained RAM (eg: on your laptop). This makes it more accessible: it can run on a small laptop, instead of requiring access to a supercomputing cluster. Example code is also &lt;a class="reference external" href="https://github.com/GenevieveBuckley/distributed-skeleton-analysis/blob/main/distributed-skeleton-analysis-with-dask.ipynb"&gt;provided 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/2021/05/07/skeleton-analysis.md&lt;/span&gt;, line 13)&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="contents"&gt;
&lt;h1&gt;Contents&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#skeleton-structures-are-everywhere"&gt;&lt;span class="xref myst"&gt;Skeleton structures are everywhere&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#the-scientific-problem"&gt;&lt;span class="xref myst"&gt;The scientific problem&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#the-compute-problem"&gt;&lt;span class="xref myst"&gt;The compute problem&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#our-approach"&gt;&lt;span class="xref myst"&gt;Our approach&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#results"&gt;&lt;span class="xref myst"&gt;Results&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#limitations"&gt;&lt;span class="xref myst"&gt;Limitations&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#problems-encountered"&gt;&lt;span class="xref myst"&gt;Problems encountered&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#how-we-solved-them"&gt;&lt;span class="xref myst"&gt;How we solved them&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#problem-1-the-skeletonize-function-from-scikit-image-crashes-due-to-lack-of-ram"&gt;&lt;span class="xref myst"&gt;Problem 1: The skeletonize function from scikit-image crashes due to lack of RAM&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#problem-2-ragged-or-non-uniform-output-from-dask-array-chunks"&gt;&lt;span class="xref myst"&gt;Problem 2: Ragged or non-uniform output from Dask array chunks&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#problem-3-grabbing-the-image-chunks-with-an-overlap"&gt;&lt;span class="xref myst"&gt;Problem 3: Grabbing the image chunks with an overlap&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#problem-4-summary-statistics-with-skan"&gt;&lt;span class="xref myst"&gt;Problem 4: Summary statistics with skan&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#what's-next"&gt;&lt;span class="xref myst"&gt;What’s next&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#how-you-can-help"&gt;&lt;span class="xref myst"&gt;How you can help&lt;/span&gt;&lt;/a&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/2021/05/07/skeleton-analysis.md&lt;/span&gt;, line 30)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="skeleton-structures-are-everywhere"&gt;
&lt;h1&gt;Skeleton structures are everywhere&lt;/h1&gt;
&lt;p&gt;Lots of biological structures have a skeleton or network-like shape. We see these in all kinds of places, including:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;blood vessel branching&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;the branching of airways&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;neuron networks in the brain&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;the root structure of plants&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;the capillaries in leaves&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;… and many more&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Analysing the structure of these skeletons can give us important information about the biology of that system.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/05/07/skeleton-analysis.md&lt;/span&gt;, line 43)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="the-scientific-problem"&gt;
&lt;h1&gt;The scientific problem&lt;/h1&gt;
&lt;p&gt;For this bogpost, we will look at the blood vessels inside of a lung. This data was shared with us by &lt;a class="reference external" href="https://research.monash.edu/en/persons/marcus-kitchen"&gt;Marcus Kitchen&lt;/a&gt;, &lt;a class="reference external" href="https://hudson.org.au/researcher-profile/andrew-stainsby/"&gt;Andrew Stainsby&lt;/a&gt;, and their team of collaborators.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Skeleton network of blood vessels within a healthy lung" src="https://blog.dask.org/_images/skeleton-screenshot-crop.jpg" /&gt;&lt;/p&gt;
&lt;p&gt;This research group focusses on lung development.
We want to compare the blood vessels in a healthy lung, against a lung from a hernia model. In the hernia model the lung is underdeveloped, squashed, and smaller.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/05/07/skeleton-analysis.md&lt;/span&gt;, line 52)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="the-compute-problem"&gt;
&lt;h1&gt;The compute problem&lt;/h1&gt;
&lt;p&gt;These image volumes have a shape of roughtly 1000x1000x1000 pixels.
That doesn’t seem huge but given the high RAM consumption involved in processing the analysis, it crashes when running on a laptop.&lt;/p&gt;
&lt;p&gt;If you’re running out of RAM, there are two possible appoaches:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Get more RAM. Run things on a bigger computer, or move things to a supercomputing cluster. This has the advantage that you don’t need to rewrite your code, but it does require access to more powerful computer hardware.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Manage the RAM you’ve got. Dask is good for this. If we use Dask, and some reasonable chunking of our arrays, we can manage things so that we never hit the RAM ceiling and crash. This has the advantage that you don’t need to buy more computer hardware, but it will require re-writing some code.&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/2021/05/07/skeleton-analysis.md&lt;/span&gt;, line 63)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="our-approach"&gt;
&lt;h1&gt;Our approach&lt;/h1&gt;
&lt;p&gt;We took the second approach, using Dask so we can run our analysis on a small laptop with constrained RAM without crashing. This makes it more accessible, to more people.&lt;/p&gt;
&lt;p&gt;All the image pre-processing steps will be done with &lt;a class="reference external" href="http://image.dask.org/en/latest/"&gt;dask-image&lt;/a&gt;, and the &lt;a class="reference external" href="https://scikit-image.org/docs/dev/auto_examples/edges/plot_skeleton.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;skeletonize&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; function of &lt;a class="reference external" href="https://scikit-image.org/"&gt;scikit-image&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;We use &lt;a class="reference external" href="https://jni.github.io/skan/"&gt;skan&lt;/a&gt; as the backbone of our analysis pipeline. &lt;a class="reference external" href="https://jni.github.io/skan/"&gt;skan&lt;/a&gt; is a library for skeleton image analysis. Given a skeleton image, it can describe statistics of the branches. To make it fast, the library is accelerated with &lt;a class="reference external" href="https://numba.pydata.org/"&gt;numba&lt;/a&gt; (if you’re curious, you can hear more about that in &lt;a class="reference external" href="https://www.youtube.com/watch?v=0pUPNMglnaE"&gt;this talk&lt;/a&gt; and its &lt;a class="reference external" href="https://github.com/jni/skan-talk-scipy-2019"&gt;related notebook&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;There is an example notebook containing the full details of the skeleton analysis &lt;a class="reference external" href="https://github.com/GenevieveBuckley/distributed-skeleton-analysis/blob/main/distributed-skeleton-analysis-with-dask.ipynb"&gt;available here&lt;/a&gt;. You can read on to hear just the highlights.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/05/07/skeleton-analysis.md&lt;/span&gt;, line 73)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="results"&gt;
&lt;h1&gt;Results&lt;/h1&gt;
&lt;p&gt;The statistics from the blood vessel branches in the healthy and herniated lung shows clear differences between the two.&lt;/p&gt;
&lt;p&gt;Most striking is the difference in the number of blood vessel branches.
The herniated lung has less than 40% of the number of blood vessel branches in the healthy lung.&lt;/p&gt;
&lt;p&gt;There are also quantitative differences in the sizes of the blood vessels.
Here is a violin plot showing the distribution of the distances between the start and end points of each blood vessel branch. We can see that overall the blood vessel branches start and end closer together in the herniated lung. This is consistent with what we might expect, since the healthy lung is more well developed than the lung from the hernia model and the hernia has compressed that lung into a smaller overall space.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Violin plot comparing blood vessel thickness between a healthy and herniated lung" src="https://blog.dask.org/_images/compare-euclidean-distance.png" /&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;EDIT: This blogpost previously described the euclidean distance violin plot as measuring the thickness of the blood vessels. This is incorrect, and the mistake was not caught in the review process before publication. This post has been updated to correctly describe the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;euclidean-distance&lt;/span&gt;&lt;/code&gt; measuremet as the distance between the start and end of branches, as if you pulled a string taught between those points. An alternative measurement, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;branch-length&lt;/span&gt;&lt;/code&gt; describes the total branch length, including any winding twists and turns.&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/2021/05/07/skeleton-analysis.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="limitations"&gt;
&lt;h1&gt;Limitations&lt;/h1&gt;
&lt;p&gt;We rely on one big assumption: once skeletonized the reduced non-zero pixel data will fit into memory. While this holds true for datasets of this size (the cropped rabbit lung datasets are roughly 1000 x 1000 x 1000 pixels), it may not hold true for much larger data.&lt;/p&gt;
&lt;p&gt;Dask computation is also triggered at a few points through our prototype workflow. Ideally all computation would be delayed until the very final stage.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/05/07/skeleton-analysis.md&lt;/span&gt;, line 93)&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-encountered"&gt;
&lt;h1&gt;Problems encountered&lt;/h1&gt;
&lt;p&gt;This project was originally intended to be a quick &amp;amp; easy one. Famous last words!&lt;/p&gt;
&lt;p&gt;What I wanted to do was to put the image data in a Dask array, and then use the &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-overlap.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; function to do the image filtering, thresholding, skeletonizing, and skeleton analysis. What I soon found was that although the image filtering, thresholding, and skeletonization worked well, the skeleton analysis step had some problems:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Dask’s map_overlap function doesn’t handle ragged or non-uniformly shaped results from different image chunks very well, and…&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Internal function in the skan library were written in a way that was incompatible with distributed computation.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/05/07/skeleton-analysis.md&lt;/span&gt;, line 103)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="how-we-solved-them"&gt;
&lt;h1&gt;How we solved them&lt;/h1&gt;
&lt;section id="problem-1-the-skeletonize-function-from-scikit-image-crashes-due-to-lack-of-ram"&gt;
&lt;h2&gt;Problem 1: The skeletonize function from scikit-image crashes due to lack of RAM&lt;/h2&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://scikit-image.org/docs/dev/auto_examples/edges/plot_skeleton.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;skeletonize&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; function of &lt;a class="reference external" href="https://scikit-image.org/"&gt;scikit-image&lt;/a&gt; is very memory intensive, and was crashing on a laptop with 16GB RAM.&lt;/p&gt;
&lt;p&gt;We solved this by:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Putting our image data into a Dask array with &lt;a class="reference external" href="http://image.dask.org/en/latest/dask_image.imread.html"&gt;dask-image &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;imread&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://docs.dask.org/en/latest/array-chunks.html?highlight=rechunk#rechunking"&gt;Rechunking&lt;/a&gt; the Dask array. We need to change the chunk shapes from 2D slices to small cuboid volumes, so the next step in the computation is efficient. We can choose the overall size of the chunks so that we can stay under the memory threshold needed for skeletonize.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Finally, we run the &lt;a class="reference external" href="https://scikit-image.org/docs/dev/auto_examples/edges/plot_skeleton.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;skeletonize&lt;/span&gt;&lt;/code&gt; function&lt;/a&gt; on the Dask array chunks using the &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-overlap.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt; function&lt;/a&gt;. By limiting the size of the array chunks, we stay under our memory threshold!&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="problem-2-ragged-or-non-uniform-output-from-dask-array-chunks"&gt;
&lt;h2&gt;Problem 2: Ragged or non-uniform output from Dask array chunks&lt;/h2&gt;
&lt;p&gt;The skeleton analysis functions will return results with ragged or non-uniform length for each image chunk. This is unsurpising, because different chunks will have different numbers of non-zero pixels in our skeleton shape.&lt;/p&gt;
&lt;p&gt;When working with Dask arrays, there are two very commonly used functions: &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html#dask.array.map_blocks"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; and &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-overlap.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;. Here’s what happens when we try a function with ragged outputs with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt; versus &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="kn"&gt;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="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;100&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;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;10&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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;foo&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;# our dummy analysis function&lt;/span&gt;
    &lt;span class="n"&gt;random_length&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;7&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;np&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="n"&gt;random_length&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;With &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt;, everything works well:&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;result&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;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;foo&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;drop_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="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="c1"&gt;# this works well&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;But if we need some overlap for function &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;foo&lt;/span&gt;&lt;/code&gt; to work correctly, then we run into problems:&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;result&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;map_overlap&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;foo&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;depth&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;drop_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="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="c1"&gt;# incorrect results&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here, the first and last element of the results from foo are trimmed off before the results are concatenated, which we don’t want! Setting the keyword argument &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;trim=False&lt;/span&gt;&lt;/code&gt; would help avoid this problem, except then we get an error:&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;result&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;map_overlap&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;foo&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;depth&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;trim&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;drop_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="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="c1"&gt;# ValueError&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Unfortunately for us, it’s really important to have a 1 pixel overlap in our array chunks, so that we can tell if a skeleton branch is ending or continuing on into the next chunk.&lt;/p&gt;
&lt;p&gt;There’s some complexity in the way &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt; results are concatenated back together so rather than diving into that, a more straightforward solution is to use &lt;a class="reference external" href="https://docs.dask.org/en/latest/delayed.html"&gt;Dask delayed&lt;/a&gt; instead. &lt;a class="reference external" href="https://github.com/chrisroat"&gt;Chris Roat&lt;/a&gt; shows a nice example of how we can use &lt;a class="reference external" href="https://docs.dask.org/en/latest/delayed.html"&gt;Dask delayed&lt;/a&gt; in a list comprehension that is then concatenated with Dask (&lt;a class="reference external" href="https://github.com/dask/dask/issues/7589"&gt;link to original discussion&lt;/a&gt;).&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="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;20&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;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;10&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="nd"&gt;@dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&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;foo&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;size&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Make each dataframe a different size&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;x&amp;#39;&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;arange&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;size&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="n"&gt;np&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;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="o"&gt;+&lt;/span&gt;&lt;span class="n"&gt;size&lt;/span&gt;&lt;span class="p"&gt;)})&lt;/span&gt;

&lt;span class="n"&gt;meta&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;utils&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;make_meta&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;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;int64&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;y&amp;#39;&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;int64&lt;/span&gt;&lt;span class="p"&gt;)])&lt;/span&gt;
&lt;span class="n"&gt;blocks&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;to_delayed&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ravel&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# no overlap&lt;/span&gt;
&lt;span class="n"&gt;results&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;foo&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;meta&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;b&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;blocks&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;concat&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;results&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Warning:&lt;/strong&gt; It’s very important to pass in a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; keyword argument to the function &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_delayed&lt;/span&gt;&lt;/code&gt;. Without it, things will be extremely inefficient!&lt;/p&gt;
&lt;p&gt;If the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; keyword argument is not given, Dask will try and work out what it should be. Ordinarily that might be a good thing, but inside a list comprehension that means those tasks are computed slowly and sequentially before the main computation even begins, which is horribly inefficient. Since we know ahead of time what kinds of results we expect from our analysis function (we just don’t know the length of each set of results), we can use the &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe-api.html#dask.dataframe.utils.make_meta"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;utils.make_meta&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; function to help us here.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="problem-3-grabbing-the-image-chunks-with-an-overlap"&gt;
&lt;h2&gt;Problem 3: Grabbing the image chunks with an overlap&lt;/h2&gt;
&lt;p&gt;Now that we’re using &lt;a class="reference external" href="https://docs.dask.org/en/latest/delayed.html"&gt;Dask delayed&lt;/a&gt; to piece together our skeleton analysis results, it’s up to us to handle the array chunks overlap ourselves.&lt;/p&gt;
&lt;p&gt;We’ll do that by modifying Dask’s &lt;a class="reference external" href="https://github.com/dask/dask/blob/21aaf44d4d25bdba05951b85f3f2d943b823e82d/dask/array/core.py#L209-L225"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array.core.slices_from_chunks&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; function, into something that will be able to handle an overlap. Some special handling is required at the boundaries of the Dask array, so that we don’t try to slice past the edge of the array.&lt;/p&gt;
&lt;p&gt;Here’s what that looks like (&lt;a class="reference external" href="https://gist.github.com/GenevieveBuckley/decd23c22ee3417f7d78e87f791bc081"&gt;gist&lt;/a&gt;):&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;itertools&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;product&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.array.slicing&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;cached_cumsum&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;slices_from_chunks_overlap&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;array_shape&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;depth&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;cumdims&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;cached_cumsum&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;bds&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;initial_zero&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;bds&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

    &lt;span class="n"&gt;slices&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[]&lt;/span&gt;
    &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;starts&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;shapes&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="nb"&gt;zip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cumdims&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="n"&gt;inner_slices&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[]&lt;/span&gt;
        &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;dim&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;maxshape&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="nb"&gt;zip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;starts&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;shapes&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;array_shape&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
            &lt;span class="n"&gt;slice_start&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;
            &lt;span class="n"&gt;slice_stop&lt;/span&gt; &lt;span class="o"&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;dim&lt;/span&gt;
            &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;slice_start&lt;/span&gt; &lt;span class="o"&gt;&amp;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;slice_start&lt;/span&gt; &lt;span class="o"&gt;-=&lt;/span&gt; &lt;span class="n"&gt;depth&lt;/span&gt;
            &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;slice_stop&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;=&lt;/span&gt; &lt;span class="n"&gt;maxshape&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                &lt;span class="n"&gt;slice_stop&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="n"&gt;depth&lt;/span&gt;
            &lt;span class="n"&gt;inner_slices&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;slice&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;slice_start&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;slice_stop&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
        &lt;span class="n"&gt;slices&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inner_slices&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;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;product&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;slices&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now that we can slice an image chunk plus an extra pixel of overlap, all we need is a way to do that for all the chunks in an array. Drawing inspiration from this &lt;a class="reference external" href="https://github.com/dask/dask-image/blob/63543bf2f6553a8150f45289492bf614e1945ac0/dask_image/ndmeasure/__init__.py#L299-L303"&gt;block iteration&lt;/a&gt; we make a similar iterator.&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;block_iter&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;zip&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;ndindex&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;image&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;numblocks&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
    &lt;span class="nb"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;functools&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;partial&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;operator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;getitem&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;image&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
        &lt;span class="n"&gt;slices_from_chunks_overlap&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;image&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;image&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;depth&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="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;meta&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;utils&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;make_meta&lt;/span&gt;&lt;span class="p"&gt;([(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;row&amp;#39;&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;int64&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;col&amp;#39;&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;int64&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;data&amp;#39;&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;float64&lt;/span&gt;&lt;span class="p"&gt;)])&lt;/span&gt;
&lt;span class="n"&gt;intermediate_results&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;skeleton_graph_func&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;block&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;block&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;block_iter&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;results&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;concat&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;intermediate_results&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;results&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;results&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;drop_duplicates&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# we need to drop duplicates because it counts pixels in the overlapping region twice&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;With these results, we’re able to create the sparse skeleton graph.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="problem-4-summary-statistics-with-skan"&gt;
&lt;h2&gt;Problem 4: Summary statistics with skan&lt;/h2&gt;
&lt;p&gt;Skeleton branch statistics can be calculate with the &lt;a class="reference external" href="https://jni.github.io/skan/api/skan.csr.html#skan.csr.summarize"&gt;skan &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;summarize&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; function. The problem here is that the function expects a &lt;a class="reference external" href="https://jni.github.io/skan/api/skan.csr.html#skan.csr.Skeleton"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Skeleton&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; object instance, but initializing a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Skeleton&lt;/span&gt;&lt;/code&gt; object calls methods that are not compatible for distributed analysis.&lt;/p&gt;
&lt;p&gt;We’ll solve this problem by first initializing a &lt;a class="reference external" href="https://jni.github.io/skan/api/skan.csr.html#skan.csr.Skeleton"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Skeleton&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; object instance with a tiny dummy dataset, then overwriting the attributes of the skeleton object with our real results. This is a hack, but it lets us achieve our goal: summary branch statistics for our large dataset.&lt;/p&gt;
&lt;p&gt;First we make a Skeleton object instance with dummy 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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;skan._testdata&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;skeleton0&lt;/span&gt;

&lt;span class="n"&gt;skeleton_object&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Skeleton&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;skeleton0&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# initialize with dummy data&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Then we overwrite the attributes with the previously calculated results:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;skeleton_object&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;skeleton_image&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;
&lt;span class="n"&gt;skeleton_object&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;graph&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;
&lt;span class="n"&gt;skeleton_object&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;coordinates&lt;/span&gt;
&lt;span class="n"&gt;skeleton_object&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;degrees&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;
&lt;span class="n"&gt;skeleton_object&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;distances&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;
&lt;span class="o"&gt;...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Then finally we can calculate the summary branch statistics:&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;skan&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;summarize&lt;/span&gt;

&lt;span class="n"&gt;statistics&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;summarize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;skel_obj&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;statistics&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="pst-scrollable-table-container"&gt;&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr class="row-odd"&gt;&lt;th class="head text-right"&gt;&lt;p&gt;&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;skeleton-id&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;node-id-src&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;node-id-dst&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;branch-distance&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;branch-type&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;mean-pixel-value&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;stdev-pixel-value&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;image-coord-src-0&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;image-coord-src-1&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;image-coord-src-2&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;image-coord-dst-0&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;image-coord-dst-1&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;image-coord-dst-2&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;coord-src-0&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;coord-src-1&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;coord-src-2&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;coord-dst-0&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;coord-dst-1&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;coord-dst-2&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;euclidean-distance&lt;/p&gt;&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr class="row-even"&gt;&lt;td class="text-right"&gt;&lt;p&gt;0&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.474584&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.00262514&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;22&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;400&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;595&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;22&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;400&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;596&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;22&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;400&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;595&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;22&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;400&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;596&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td class="text-right"&gt;&lt;p&gt;1&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;3&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;9&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;8.19615&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.464662&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.00299629&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;37&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;400&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;622&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;43&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;392&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;590&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;37&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;400&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;622&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;43&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;392&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;590&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;33.5261&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-even"&gt;&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;3&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;10&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;11&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.483393&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.00771038&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;49&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;391&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;589&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;50&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;391&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;589&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;49&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;391&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;589&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;50&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;391&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;589&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td class="text-right"&gt;&lt;p&gt;3&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;5&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;13&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;19&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;6.82843&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.464325&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.0139064&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;52&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;389&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;588&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;55&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;385&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;588&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;52&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;389&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;588&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;55&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;385&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;588&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;5&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-even"&gt;&lt;td class="text-right"&gt;&lt;p&gt;4&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;7&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;21&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;23&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.45862&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.0104024&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;57&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;382&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;587&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;58&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;380&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;586&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;57&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;382&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;587&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;58&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;380&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;586&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2.44949&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;/div&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;statistics&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;describe&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="pst-scrollable-table-container"&gt;&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr class="row-odd"&gt;&lt;th class="head text-left"&gt;&lt;p&gt;&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;skeleton-id&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;node-id-src&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;node-id-dst&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;branch-distance&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;branch-type&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;mean-pixel-value&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;stdev-pixel-value&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;image-coord-src-0&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;image-coord-src-1&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;image-coord-src-2&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;image-coord-dst-0&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;image-coord-dst-1&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;image-coord-dst-2&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;coord-src-0&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;coord-src-1&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;coord-src-2&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;coord-dst-0&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;coord-dst-1&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;coord-dst-2&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-right"&gt;&lt;p&gt;euclidean-distance&lt;/p&gt;&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr class="row-even"&gt;&lt;td class="text-left"&gt;&lt;p&gt;count&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1095&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td class="text-left"&gt;&lt;p&gt;mean&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2089.38&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;11520.1&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;11608.6&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;22.9079&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2.00091&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.663422&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.0418607&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;591.939&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;430.303&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;377.409&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;594.325&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;436.596&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;373.419&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;591.939&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;430.303&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;377.409&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;594.325&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;436.596&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;373.419&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;190.13&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-even"&gt;&lt;td class="text-left"&gt;&lt;p&gt;std&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;636.377&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;6057.61&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;6061.18&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;24.2646&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.0302199&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.242828&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.0559064&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;174.04&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;194.499&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;97.0219&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;173.353&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;188.708&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;96.8276&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;174.04&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;194.499&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;97.0219&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;173.353&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;188.708&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;96.8276&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;151.171&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td class="text-left"&gt;&lt;p&gt;min&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.414659&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;6.79493e-06&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;22&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;39&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;116&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;22&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;39&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;114&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;22&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;39&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;116&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;22&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;39&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;114&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-even"&gt;&lt;td class="text-left"&gt;&lt;p&gt;25%&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1586&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;6215.5&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;6429.5&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1.73205&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.482&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.00710439&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;468.5&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;278.5&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;313&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;475&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;299.5&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;307&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;468.5&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;278.5&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;313&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;475&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;299.5&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;307&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;72.6946&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td class="text-left"&gt;&lt;p&gt;50%&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2431&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;11977&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;12010&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;16.6814&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.552626&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.0189069&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;626&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;405&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;388&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;627&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;410&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;381&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;626&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;405&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;388&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;627&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;410&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;381&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;161.059&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-even"&gt;&lt;td class="text-left"&gt;&lt;p&gt;75%&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2542.5&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;16526.5&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;16583&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;35.0433&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;2&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.768359&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.0528814&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;732&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;579&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;434&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;734&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;590&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;432&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;732&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;579&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;434&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;734&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;590&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;432&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;265.948&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td class="text-left"&gt;&lt;p&gt;max&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;8034&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;26820&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;26822&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;197.147&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;3&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;1.29687&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;0.357193&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;976&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;833&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;622&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;976&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;841&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;606&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;976&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;833&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;622&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;976&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;841&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;606&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-right"&gt;&lt;p&gt;737.835&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;/div&gt;
&lt;p&gt;Success!&lt;/p&gt;
&lt;p&gt;We’ve achieved distributed skeleton analysis with Dask.
You can see the example notebook containing the full details of the skeleton analysis &lt;a class="reference external" href="https://github.com/GenevieveBuckley/distributed-skeleton-analysis/blob/main/distributed-skeleton-analysis-with-dask.ipynb"&gt;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/2021/05/07/skeleton-analysis.md&lt;/span&gt;, line 294)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="what-s-next"&gt;
&lt;h1&gt;What’s next?&lt;/h1&gt;
&lt;p&gt;A good next step is modifing the &lt;a class="reference external" href="https://github.com/jni/skan"&gt;skan&lt;/a&gt; library code so that it directly supports distributed skeleton 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/2021/05/07/skeleton-analysis.md&lt;/span&gt;, line 298)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="how-you-can-help"&gt;
&lt;h1&gt;How you can help&lt;/h1&gt;
&lt;p&gt;If you’d like to get involved, there are a couple of options:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Try a similar analysis on your own data. The notebook with the full example code is &lt;a class="reference external" href="https://github.com/GenevieveBuckley/distributed-skeleton-analysis/blob/main/distributed-skeleton-analysis-with-dask.ipynb"&gt;available here&lt;/a&gt;. You can share or ask questions in the &lt;a class="reference external" href="https://join.slack.com/t/dask/shared_invite/zt-mfmh7quc-nIrXL6ocgiUH2haLYA914g"&gt;Dask slack&lt;/a&gt; or &lt;a class="reference internal" href="#twitter.com/dask_dev"&gt;&lt;span class="xref myst"&gt;on twitter&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Help add support for distributed skeleton analysis to skan. Head on over to the &lt;a class="reference external" href="https://github.com/jni/skan/issues/"&gt;skan issues page&lt;/a&gt; and leave a comment if you’d like to join in.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/05/07/skeleton-analysis/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="imaging" label="imaging"/>
    <category term="lifescience" label="life science"/>
    <category term="skan" label="skan"/>
    <category term="skeletonanalysis" label="skeleton analysis"/>
    <published>2021-05-07T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/03/29/apply-pretrained-pytorch-model/</id>
    <title>Dask with PyTorch for large scale image analysis</title>
    <updated>2021-03-29T00:00:00+00:00</updated>
    <author>
      <name>Genevieve Buckley</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/29/apply-pretrained-pytorch-model.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;This post explores applying a pre-trained &lt;a class="reference external" href="https://pytorch.org/"&gt;PyTorch&lt;/a&gt; model in parallel with Dask Array.&lt;/p&gt;
&lt;p&gt;We cover a simple example applying a pre-trained UNet to a stack of images to generate features for every pixel.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/29/apply-pretrained-pytorch-model.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&gt;
&lt;section id="a-worked-example"&gt;
&lt;h1&gt;A Worked Example&lt;/h1&gt;
&lt;p&gt;Let’s start with an example applying a pre-trained &lt;a class="reference external" href="https://arxiv.org/abs/1505.04597"&gt;UNet&lt;/a&gt; to a stack of light sheet microscopy data.&lt;/p&gt;
&lt;p&gt;In this example, we:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Load the image data from Zarr into a multi-chunked Dask array&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Load a pre-trained PyTorch model that featurizes images&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Construct a function to apply the model onto each chunk&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Apply that function across the Dask array with the dask.array.map_blocks function.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Store the result back into Zarr format&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;section id="step-1-load-the-image-data"&gt;
&lt;h2&gt;Step 1. Load the image data&lt;/h2&gt;
&lt;p&gt;First, we load the image data into a Dask array.&lt;/p&gt;
&lt;p&gt;The example dataset we’re using here is lattice lightsheet microscopy of the tail region of a zebrafish embryo. It is described in &lt;a class="reference external" href="http://dx.doi.org/10.1126/science.aaq1392"&gt;this Science paper&lt;/a&gt; (see Figure 4), and provided with permission from Srigokul Upadhyayula.&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Liu &lt;em&gt;et al.&lt;/em&gt; 2018 “Observing the cell in its native state: Imaging subcellular dynamics in multicellular organisms” &lt;em&gt;Science&lt;/em&gt;, Vol. 360, Issue 6386, eaaq1392 DOI: 10.1126/science.aaq1392 (&lt;a class="reference external" href="http://dx.doi.org/10.1126/science.aaq1392"&gt;link&lt;/a&gt;)&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;This is the same data that we analysed in our last &lt;a class="reference external" href="https://blog.dask.org/2019/08/09/image-itk"&gt;blogpost on Dask and ITK&lt;/a&gt;. You should note the similarities to that workflow even though we are now using new libaries and performing different analyses.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;cd&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;/Users/nicholassofroniew/Github/image-demos/data/LLSM&amp;#39;&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="c1"&gt;# Load our data&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="n"&gt;imgs&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;from_zarr&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;AOLLSM_m4_560nm.zarr&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;imgs&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;from&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;zarr&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;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;199&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;768&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1024&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="n"&gt;float32&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="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;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;768&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1024&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="step-2-load-a-pre-trained-pytorch-model"&gt;
&lt;h2&gt;Step 2. Load a pre-trained PyTorch model&lt;/h2&gt;
&lt;p&gt;Next, we load our pre-trained UNet model.&lt;/p&gt;
&lt;p&gt;This UNet model takes in an 2D image and returns a 2D x 16 array, where each pixel is now associate with a feature vector of length 16.&lt;/p&gt;
&lt;p&gt;We thank Mars Huang for training this particular UNet on a corpous of biological images to produce biologically relevant feature vectors, as part of his work on &lt;a class="reference external" href="https://github.com/transformify-plugins/segmentify"&gt;interactive bio-image segmentation&lt;/a&gt;. These features can then be used for more downstream image processing tasks such as image segmentation.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Load our pretrained UNet¶&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;torch&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;segmentify.model&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;UNet&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;layers&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_unet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot;Load a pretrained UNet model.&amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;

    &lt;span class="c1"&gt;# load in saved model&lt;/span&gt;
    &lt;span class="n"&gt;pth&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;torch&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;load&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;model_args&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pth&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;model_args&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
    &lt;span class="n"&gt;model_state&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pth&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;model_state&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
    &lt;span class="n"&gt;model&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;UNet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;model_args&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;load_state_dict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;model_state&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

    &lt;span class="c1"&gt;# remove last layer and activation&lt;/span&gt;
    &lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;segment&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;layers&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Identity&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;activate&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;layers&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Identity&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;eval&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;model&lt;/span&gt;

&lt;span class="n"&gt;model&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;load_unet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;HPA_3.pth&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="step-3-construct-a-function-to-apply-the-model-to-each-chunk"&gt;
&lt;h2&gt;Step 3. Construct a function to apply the model to each chunk&lt;/h2&gt;
&lt;p&gt;We make a function to apply our pre-trained UNet model to each chunk of the Dask array.&lt;/p&gt;
&lt;p&gt;Because Dask arrays are just made out of Numpy arrays which are easily converted to Torch arrays, we’re able to leverage the power of machine learning at scale.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Apply UNet featurization&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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;unet_featurize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;image&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;model&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;Featurize pixels in an image using pretrained UNet model.&lt;/span&gt;
&lt;span class="sd"&gt;    &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
    &lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;torch&lt;/span&gt;

    &lt;span class="c1"&gt;# Extract the 2D image data from the Dask array&lt;/span&gt;
    &lt;span class="c1"&gt;# Original Dask array dimensions were (time, z-slice, y, x)&lt;/span&gt;
    &lt;span class="n"&gt;img&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;image&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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

    &lt;span class="c1"&gt;# Put the data into a shape PyTorch expects&lt;/span&gt;
    &lt;span class="c1"&gt;# Expected dimensions are (Batch x Channel x Width x Height)&lt;/span&gt;
    &lt;span class="n"&gt;img&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

    &lt;span class="c1"&gt;# convert image to torch Tensor&lt;/span&gt;
    &lt;span class="n"&gt;img&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;torch&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Tensor&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;float&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

    &lt;span class="c1"&gt;# pass image through model&lt;/span&gt;
    &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;torch&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;no_grad&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
        &lt;span class="n"&gt;features&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;numpy&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

    &lt;span class="c1"&gt;# generate feature vectors (w,h,f)&lt;/span&gt;
    &lt;span class="n"&gt;features&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;transpose&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;features&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;&lt;span class="mi"&gt;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;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;span class="c1"&gt;# Add back the leading length-one dimensions&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;features&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;result&lt;/span&gt;

&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Note: Very observant readers might notice that the steps for extracting the 2D image data and then putting it into a shape PyTorch expects appear to be redundant. It is redundant for our particular example, but that might easily not have been the case.&lt;/p&gt;
&lt;p&gt;To explain this in more detail, the UNet expects 4D input, with dimensions &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(Batch&lt;/span&gt; &lt;span class="pre"&gt;x&lt;/span&gt; &lt;span class="pre"&gt;Channel&lt;/span&gt; &lt;span class="pre"&gt;x&lt;/span&gt; &lt;span class="pre"&gt;Width&lt;/span&gt; &lt;span class="pre"&gt;x&lt;/span&gt; &lt;span class="pre"&gt;Height)&lt;/span&gt;&lt;/code&gt;. The original Dask array dimensions were &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(time,&lt;/span&gt; &lt;span class="pre"&gt;z-slice,&lt;/span&gt; &lt;span class="pre"&gt;y,&lt;/span&gt; &lt;span class="pre"&gt;x)&lt;/span&gt;&lt;/code&gt;. In our example it just so happens those match in a way that makes removing and then adding the leading dimensions redundant, but depending on the shape of the original Dask array this might not have been true.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="step-4-apply-that-function-across-the-dask-array"&gt;
&lt;h2&gt;Step 4. Apply that function across the Dask array&lt;/h2&gt;
&lt;p&gt;Now we apply that function to the data in our Dask array using &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html?highlight=map_blocks#dask.array.map_blocks"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array.map_blocks&lt;/span&gt;&lt;/code&gt;&lt;/a&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="c1"&gt;# Apply UNet featurization&lt;/span&gt;
&lt;span class="n"&gt;out&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;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;unet_featurize&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;imgs&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;model&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="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;float32&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;1&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;imgs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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;imgs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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;16&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;new_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="n"&gt;out&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;unet_featurize&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;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;199&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;768&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1024&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;16&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="n"&gt;float32&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="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;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;768&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1024&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;16&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="step-5-store-the-result-back-into-zarr-format"&gt;
&lt;h2&gt;Step 5. Store the result back into Zarr format&lt;/h2&gt;
&lt;p&gt;Last, we store the result from the UNet model featurization as a zarr 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="c1"&gt;# Trigger computation and store&lt;/span&gt;
&lt;span class="n"&gt;out&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_zarr&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;AOLLSM_featurized.zarr&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;overwrite&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now we’ve saved our output, these features can be used for more downstream image processing tasks such as image segmentation.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="summing-up"&gt;
&lt;h2&gt;Summing up&lt;/h2&gt;
&lt;p&gt;Here we’ve shown how to apply a pre-trained PyTorch model to a Dask array of image data.&lt;/p&gt;
&lt;p&gt;Because our Dask array chunks are Numpy arrays, they can be easily converted to Torch arrays. This way, we’re able to leverage the power of machine learning at scale.&lt;/p&gt;
&lt;p&gt;This workflow was very similar to &lt;a class="reference external" href="https://blog.dask.org/2019/08/09/image-itk"&gt;our example&lt;/a&gt; using the dask.array.map_blocks function with ITK to perform image deconvolution. This shows you can easily adapt the same type of workflow to achieve many different types of analysis with Dask.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/03/29/apply-pretrained-pytorch-model/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="PyTorch" label="PyTorch"/>
    <category term="deeplearning" label="deep learning"/>
    <category term="imaging" label="imaging"/>
    <published>2021-03-29T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/03/19/image-segmentation/</id>
    <title>Image segmentation with Dask</title>
    <updated>2021-03-19T00:00:00+00:00</updated>
    <author>
      <name>Genevieve Buckley</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/19/image-segmentation.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;We look at how to create a basic image segmentation pipeline, using the &lt;a class="reference external" href="http://image.dask.org/en/latest/"&gt;dask-image&lt;/a&gt; library.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/19/image-segmentation.md&lt;/span&gt;, line 13)&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="contents"&gt;
&lt;h1&gt;Contents&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#just-show-me-the-code"&gt;&lt;span class="xref myst"&gt;Just show me the code&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#image-segmentation-pipeline"&gt;&lt;span class="xref myst"&gt;Image segmentation pipeline&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#set-up-your-python-environment"&gt;&lt;span class="xref myst"&gt;Set up your python environment&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#download-the-example-data"&gt;&lt;span class="xref myst"&gt;Download the example data&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#step-1-reading-in-data"&gt;&lt;span class="xref myst"&gt;Step 1: Reading in data&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#step-2-siltering-images"&gt;&lt;span class="xref myst"&gt;Step 2: Filtering images&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#step-3-segmenting-objects"&gt;&lt;span class="xref myst"&gt;Step 3: Segmenting objects&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#step-4-morphological-operations"&gt;&lt;span class="xref myst"&gt;Step 4: Morphological operations&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#step-5-measuring-objects"&gt;&lt;span class="xref myst"&gt;Step 5: Measuring objects&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#custom-functions"&gt;&lt;span class="xref myst"&gt;Custom functions&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#dask-map_overlap-and-map_blocks"&gt;&lt;span class="xref myst"&gt;Dask map_overlap and map_blocks&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#dask-delayed"&gt;&lt;span class="xref myst"&gt;Dask delayed decorator&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#scikit-image-apply_parallel-function"&gt;&lt;span class="xref myst"&gt;scikit-image apply_parallel&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#scaling-up-computation"&gt;&lt;span class="xref myst"&gt;Scaling up computation&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#bonus-content-using-arrays-on-gpu"&gt;&lt;span class="xref myst"&gt;Bonus content: using arrays on GPU&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#how-you-can-get-involved"&gt;&lt;span class="xref myst"&gt;How you can get involved&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The content of this blog post originally appeared as &lt;a class="reference external" href="https://github.com/genevieveBuckley/dask-image-talk-2020"&gt;a conference talk in 2020&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/2021/03/19/image-segmentation.md&lt;/span&gt;, line 34)&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="just-show-me-the-code"&gt;
&lt;h1&gt;Just show me the code&lt;/h1&gt;
&lt;p&gt;If you want to run this yourself, you’ll need to download the example data from the Broad Bioimage Benchmark Collection: https://bbbc.broadinstitute.org/BBBC039&lt;/p&gt;
&lt;p&gt;And install these requirements:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;image&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;=&lt;/span&gt;&lt;span class="mf"&gt;0.4.0&lt;/span&gt; &lt;span class="n"&gt;tifffile&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here’s our full pipeline:&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;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_image.imread&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;imread&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_image&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;ndfilters&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;ndmorph&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;ndmeasure&lt;/span&gt;

&lt;span class="n"&gt;images&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;data/BBBC039/images/*.tif&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;smoothed&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ndfilters&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;gaussian_filter&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sigma&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;span class="n"&gt;thresh&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ndfilters&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;threshold_local&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;smoothed&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;blocksize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;chunksize&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;threshold_images&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;smoothed&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;thresh&lt;/span&gt;
&lt;span class="n"&gt;structuring_element&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;array&lt;/span&gt;&lt;span class="p"&gt;([[[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="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="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="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="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="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="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;],&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;1&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="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]],&lt;/span&gt; &lt;span class="p"&gt;[[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;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="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="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="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="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]]])&lt;/span&gt;
&lt;span class="n"&gt;binary_images&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ndmorph&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;binary_closing&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;threshold_image&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;structure&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;structuring_element&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;label_images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;num_features&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ndmeasure&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;label&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;binary_image&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;index&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;np&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="n"&gt;num_features&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;area&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ndmeasure&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;area&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;label_images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;mean_intensity&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ndmeasure&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;images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;label_images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You can keep reading for a step by step walkthrough of this image segmentation pipeline, or you can skip ahead to the sections on &lt;a class="reference internal" href="#Custom-functions"&gt;&lt;span class="xref myst"&gt;custom functions&lt;/span&gt;&lt;/a&gt;, &lt;a class="reference internal" href="#Scaling-up-computation"&gt;&lt;span class="xref myst"&gt;scaling up computation&lt;/span&gt;&lt;/a&gt;, or &lt;a class="reference internal" href="#Bonus-content:-using-arrays-on-GPU"&gt;&lt;span class="xref myst"&gt;GPU acceleration&lt;/span&gt;&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/2021/03/19/image-segmentation.md&lt;/span&gt;, line 65)&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="image-segmentation-pipeline"&gt;
&lt;h1&gt;Image segmentation pipeline&lt;/h1&gt;
&lt;p&gt;Our basic image segmentation pipeline has five steps:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Reading in data&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Filtering images&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Segmenting objects&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Morphological operations&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Measuring objects&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;section id="set-up-your-python-environment"&gt;
&lt;h2&gt;Set up your python environment&lt;/h2&gt;
&lt;p&gt;Before we begin, we’ll need to set up our python virtual environment.&lt;/p&gt;
&lt;p&gt;At a minimum, you’ll need:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;image&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;=&lt;/span&gt;&lt;span class="mf"&gt;0.4.0&lt;/span&gt; &lt;span class="n"&gt;tifffile&lt;/span&gt; &lt;span class="n"&gt;matplotlib&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Optionally, you can also install the &lt;a class="reference external" href="https://napari.org/"&gt;napari&lt;/a&gt; image viewer to visualize the image segmentation.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;napari[all]&amp;quot;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;To use napari from IPython or jupyter, run the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;%gui&lt;/span&gt; &lt;span class="pre"&gt;qt&lt;/span&gt;&lt;/code&gt; magic in a cell before calling napari. See the &lt;a class="reference external" href="https://napari.org/"&gt;napari documentation&lt;/a&gt; for more details.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="download-the-example-data"&gt;
&lt;h2&gt;Download the example data&lt;/h2&gt;
&lt;p&gt;We’ll use the publically available image dataset &lt;a class="reference external" href="https://bbbc.broadinstitute.org/BBBC039"&gt;BBBC039&lt;/a&gt; Caicedo et al. 2018, available from the Broad Bioimage Benchmark Collection &lt;a class="reference external" href="http://dx.doi.org/10.1038/nmeth.2083"&gt;Ljosa et al., Nature Methods, 2012&lt;/a&gt;. You can download the dataset here: &lt;a class="reference external" href="https://bbbc.broadinstitute.org/BBBC039"&gt;https://bbbc.broadinstitute.org/BBBC039&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;img alt="Example image from the BBBC039 dataset, Broad Bioimage Benchmark Collection" src="https://blog.dask.org/_images/BBBC039-example-image.png" /&gt;&lt;/p&gt;
&lt;p&gt;These are fluorescence microscopy images, where we see the nuclei in individual cells.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="step-1-reading-in-data"&gt;
&lt;h2&gt;Step 1: Reading in data&lt;/h2&gt;
&lt;p&gt;Step one in our image segmentation pipeline is to read in the image data. We can do that with the &lt;a class="reference external" href="http://image.dask.org/en/latest/dask_image.imread.html"&gt;dask-image imread function&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;We pass the path to the folder full of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;*.tif&lt;/span&gt;&lt;/code&gt; images from our example 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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_image.imread&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;imread&lt;/span&gt;

&lt;span class="n"&gt;images&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;data/BBBC039/images/*.tif&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;&lt;img alt="HTML reprsentation of a Dask array" src="https://blog.dask.org/_images/dask-array-html-repr.png" /&gt;&lt;/p&gt;
&lt;p&gt;By default, each individual &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.tif&lt;/span&gt;&lt;/code&gt; file on disk has become one chunk in our Dask array.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="step-2-filtering-images"&gt;
&lt;h2&gt;Step 2: Filtering images&lt;/h2&gt;
&lt;p&gt;Denoising images with a small amount of blur can improve segmentation later on. This is a common first step in a lot of image segmentation pipelines. We can do this with the dask-image &lt;a class="reference external" href="http://image.dask.org/en/latest/dask_image.ndfilters.html#dask_image.ndfilters.gaussian_filter"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;gaussian_filter&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; function.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_image&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;ndfilters&lt;/span&gt;

&lt;span class="n"&gt;smoothed&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ndfilters&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;gaussian_filter&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sigma&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;

&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="step-3-segmenting-objects"&gt;
&lt;h2&gt;Step 3: Segmenting objects&lt;/h2&gt;
&lt;p&gt;Next, we want to separate the objects in our images from the background. There are lots of different ways we could do this. Because we have fluorescent microscopy images, we’ll use a thresholding method.&lt;/p&gt;
&lt;section id="absolute-threshold"&gt;
&lt;h3&gt;Absolute threshold&lt;/h3&gt;
&lt;p&gt;We could set an absolute threshold value, where we’d consider pixels with intensity values below this threshold to be part of the background.&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;absolute_threshold&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;smoothed&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;160&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Let’s have a look at these images with the napari image viewer. First we’ll need to use the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;%gui&lt;/span&gt; &lt;span class="pre"&gt;qt&lt;/span&gt;&lt;/code&gt; magic:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;gui&lt;/span&gt; &lt;span class="n"&gt;qt&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And now we can look a the images with napari:&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;napari&lt;/span&gt;

&lt;span class="n"&gt;viewer&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;napari&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Viewer&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;viewer&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;add_image&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;absolute_threshold&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;viewer&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;add_image&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;contrast_limits&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2000&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/2021-image-segmentation/napari-absolute-threshold.png" alt="Absolute threshold napari screenshot" width="700" height="476"&gt;
&lt;p&gt;But there’s a problem here.&lt;/p&gt;
&lt;p&gt;When we look at the results for different image frames, it becomes clear that there is no “one size fits all” we can use for an absolute threshold value. Some images in the dataset have quite bright backgrounds, others have fluorescent nuclei with low brightness. We’ll have to try a different kind of thresholding method.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="local-threshold"&gt;
&lt;h3&gt;Local threshold&lt;/h3&gt;
&lt;p&gt;We can improve the segmentation using a local thresholding method.&lt;/p&gt;
&lt;p&gt;If we calculate a threshold value independently for each image frame then we can avoid the problem caused by fluctuating background intensity between frames.&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;thresh&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ndfilters&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;threshold_local&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;smoothed&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;chunksize&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;threshold_images&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;smoothed&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;thresh&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="c1"&gt;# Let&amp;#39;s take a look at the images with napari&lt;/span&gt;
&lt;span class="n"&gt;viewer&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;add_image&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;threshold_images&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/2021-image-segmentation/napari-local-threshold.png" alt="Local threshold napari screenshot" width="700" height="476"&gt;
&lt;p&gt;The results here look much better, this is a much cleaner separation of nuclei from the background and it looks good for all the image frames.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="step-4-morphological-operations"&gt;
&lt;h2&gt;Step 4: Morphological operations&lt;/h2&gt;
&lt;p&gt;Now that we have a binary mask from our threshold, we can clean it up a bit with some morphological operations.&lt;/p&gt;
&lt;p&gt;Morphological operations are changes we make to the shape of structures a binary image. We’ll briefly describe some of the basic concepts here, but for a more detailed reference you can look at &lt;a class="reference external" href="https://docs.opencv.org/3.0-beta/doc/py_tutorials/py_imgproc/py_morphological_ops/py_morphological_ops.html"&gt;this excellent page of the OpenCV documentation&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Erosion&lt;/strong&gt; is an operation where the edges of structures in a binary image are eaten away, or eroded.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Example: Erosion of a binary image" src="https://blog.dask.org/_images/erosion.png" /&gt;&lt;/p&gt;
&lt;p&gt;Image credit: &lt;a class="reference external" href="https://docs.opencv.org/3.0-beta/doc/py_tutorials/py_imgproc/py_morphological_ops/py_morphological_ops.html"&gt;OpenCV documentation&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Dilation&lt;/strong&gt; is the opposite of an erosion. With dilation, the edges of structures in a binary image are expanded.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Example: Dilation of a binary image" src="https://blog.dask.org/_images/dilation.png" /&gt;&lt;/p&gt;
&lt;p&gt;Image credit: &lt;a class="reference external" href="https://docs.opencv.org/3.0-beta/doc/py_tutorials/py_imgproc/py_morphological_ops/py_morphological_ops.html"&gt;OpenCV documentation&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;We can combine morphological operations in different ways to get useful effects.&lt;/p&gt;
&lt;p&gt;A &lt;strong&gt;morphological opening&lt;/strong&gt; operation is an erosion, followed by a dilation.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Example: Morphological opening of a binary image" src="https://blog.dask.org/_images/opening.png" /&gt;&lt;/p&gt;
&lt;p&gt;Image credit: &lt;a class="reference external" href="https://docs.opencv.org/3.0-beta/doc/py_tutorials/py_imgproc/py_morphological_ops/py_morphological_ops.html"&gt;OpenCV documentation&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;In the example image above, we can see the left hand side has a noisy, speckled background. If the structuring element used for the morphological operations is larger than the size of the noisy speckles, they will disappear completely in the first erosion step. Then when it is time to do the second dilation step, there’s nothing left of the noise in the background to dilate. So we have removed the noise in the background, while the major structures we are interested in (in this example, the J shape) are restored almost perfectly.&lt;/p&gt;
&lt;p&gt;Let’s use this morphological opening trick to clean up the binary images in our segmentation pipeline.&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_image&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;ndmorph&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="n"&gt;structuring_element&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;array&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;
    &lt;span class="p"&gt;[[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;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="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="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="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="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="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;],&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;1&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="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]],&lt;/span&gt;
    &lt;span class="p"&gt;[[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;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="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="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="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="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]]])&lt;/span&gt;
&lt;span class="n"&gt;binary_images&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ndmorph&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;binary_opening&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;threshold_images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;structure&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;structuring_element&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You’ll notice here that we need to be a little bit careful about the structuring element. All our image frames are combined in a single Dask array, but we only want to apply the morphological operation independently to each frame.
To do this, we sandwich the default 2D structuring element between two layers of zeros. This means the neighbouring image frames have no effect on the result.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Default 2D structuring element&lt;/span&gt;

&lt;span class="p"&gt;[[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;],&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;1&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="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="step-5-measuring-objects"&gt;
&lt;h2&gt;Step 5: Measuring objects&lt;/h2&gt;
&lt;p&gt;The last step in any image processing pipeline is to make some kind of measurement. We’ll turn our binary mask into a label image, and then measure the intensity and size of those objects.&lt;/p&gt;
&lt;p&gt;For the sake of keeping the computation time in this tutorial nice and quick, we’ll measure only a small subset of the data. Let’s measure all the objects in the first three image frames (roughly 300 nuclei).&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_image&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;ndmeasure&lt;/span&gt;

&lt;span class="c1"&gt;# Create labelled mask&lt;/span&gt;
&lt;span class="n"&gt;label_images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;num_features&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ndmeasure&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;label&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;binary_images&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;structuring_element&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;index&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;np&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="n"&gt;num_features&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="mi"&gt;1&lt;/span&gt;  &lt;span class="c1"&gt;# [1, 2, 3, ...num_features]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here’s a screenshot of the label image generated from our mask.&lt;/p&gt;
&lt;img src="/images/2021-image-segmentation/napari-label-image.png" alt="Label image napari screenshot" width="700" height="476"&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;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Number of nuclei:&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;num_features&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;

&lt;span class="go"&gt;Number of nuclei: 271&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;section id="measure-objects-in-images"&gt;
&lt;h3&gt;Measure objects in images&lt;/h3&gt;
&lt;p&gt;The dask-image &lt;a class="reference external" href="http://image.dask.org/en/latest/dask_image.ndmeasure.html"&gt;ndmeasure subpackage&lt;/a&gt; includes a number of different measurement functions. In this example, we’ll choose to measure:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;The area in pixels of each object, and&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The average intensity of each object.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;area&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ndmeasure&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;area&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;images&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;label_images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;mean_intensity&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ndmeasure&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;images&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;label_images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="run-computation-and-plot-results"&gt;
&lt;h3&gt;Run computation and plot results&lt;/h3&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;matplotlib.pyplot&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;plt&lt;/span&gt;

&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scatter&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;area&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;mean_intensity&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;0.5&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;gca&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;update&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;dict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Area vs mean intensity&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;xlabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;Area (pixels)&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;Mean intensity&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;show&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="Matplotlib graph of dask-image measurement results: " src="https://blog.dask.org/_images/dask-image-matplotlib-output.png" /&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/19/image-segmentation.md&lt;/span&gt;, line 285)&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&gt;
&lt;section id="custom-functions"&gt;
&lt;h1&gt;Custom functions&lt;/h1&gt;
&lt;p&gt;What if you want to do something that isn’t included in the dask-image API? There are several options we can use to write custom functions.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;dask &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-overlap.html?highlight=map_overlap#dask.array.map_overlap"&gt;map_overlap&lt;/a&gt; / &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html?highlight=map_blocks#dask.array.map_blocks"&gt;map_blocks&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;dask &lt;a class="reference external" href="https://docs.dask.org/en/latest/delayed.html"&gt;delayed&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;scikit-image &lt;a class="reference external" href="https://scikit-image.org/docs/dev/api/skimage.util.html#skimage.util.apply_parallel"&gt;apply_parallel()&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;section id="dask-map-overlap-and-map-blocks"&gt;
&lt;h2&gt;Dask map_overlap and map_blocks&lt;/h2&gt;
&lt;p&gt;The Dask array &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-overlap.html#dask.array.map_overlap"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; and &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html#dask.array.map_blocks"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; are what is used to build most of the functions in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-image&lt;/span&gt;&lt;/code&gt;. You can use them yourself too. They will apply a function to each chunk in a Dask 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="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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;my_custom_function&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;args&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="c1"&gt;# ... does something really cool&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;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_overlap&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;my_custom_function&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;my_dask_array&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;args&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You can read more about &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-overlap.html"&gt;overlapping computations here&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-delayed"&gt;
&lt;h2&gt;Dask delayed&lt;/h2&gt;
&lt;p&gt;If you want more flexibility and fine-grained control over your computation, then you can use &lt;a class="reference external" href="https://docs.dask.org/en/latest/delayed.html"&gt;Dask delayed&lt;/a&gt;. You can get started &lt;a class="reference external" href="https://tutorial.dask.org/01_dask.delayed.html"&gt;with the Dask delayed tutorial here&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="scikit-image-apply-parallel-function"&gt;
&lt;h2&gt;scikit-image apply_parallel function&lt;/h2&gt;
&lt;p&gt;If you’re a person who does a lot of image processing in python, one tool you’re likely to already be using is &lt;a class="reference external" href="https://scikit-image.org/"&gt;scikit-image&lt;/a&gt;. I’d like to draw your attention to the &lt;a class="reference external" href="https://scikit-image.org/docs/dev/api/skimage.util.html?highlight=apply_parallel#skimage.util.apply_parallel"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;apply_parallel&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; function available in scikit-image. It uses &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map-overlap&lt;/span&gt;&lt;/code&gt;, and can be very helpful.&lt;/p&gt;
&lt;p&gt;It’s useful not only when when you have big data, but also in cases where your data fits into memory but the computation you want to apply to the data is memory intensive. This might cause you to exceed the available RAM, and &lt;a class="reference external" href="https://scikit-image.org/docs/dev/api/skimage.util.html?highlight=apply_parallel#skimage.util.apply_parallel"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;apply_parallel&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; is great for these situations too.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/19/image-segmentation.md&lt;/span&gt;, line 318)&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="scaling-up-computation"&gt;
&lt;h1&gt;Scaling up computation&lt;/h1&gt;
&lt;p&gt;When you want to scale up from a laptop onto a supercomputing cluster, you can use &lt;a class="reference external" href="https://distributed.dask.org/en/latest/"&gt;dask-distributed&lt;/a&gt; to handle that.&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.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="c1"&gt;# Setup a local cluster&lt;/span&gt;
&lt;span class="c1"&gt;# By default this sets up 1 worker per core&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;

&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;See the &lt;a class="reference external" href="https://distributed.dask.org/en/latest/"&gt;documentation here&lt;/a&gt; to get set up for your system.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/19/image-segmentation.md&lt;/span&gt;, line 334)&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="bonus-content-using-arrays-on-gpu"&gt;
&lt;h1&gt;Bonus content: using arrays on GPU&lt;/h1&gt;
&lt;p&gt;We’ve recently been adding GPU support to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-image&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;We’re able to add GPU support using a library called &lt;a class="reference external" href="https://cupy.dev/"&gt;CuPy&lt;/a&gt;. &lt;a class="reference external" href="https://cupy.dev/"&gt;CuPy&lt;/a&gt; is an array library with a numpy-like API, accelerated with NVIDIA CUDA. Instead of having Dask arrays which contain numpy chunks, we can have Dask arrays containing cupy chunks instead. This &lt;a class="reference external" href="https://blog.dask.org/2019/01/03/dask-array-gpus-first-steps"&gt;blogpost&lt;/a&gt; explains the benefits of GPU acceleration and gives some benchmarks for computations on CPU, a single GPU, and multiple GPUs.&lt;/p&gt;
&lt;section id="gpu-support-available-in-dask-image"&gt;
&lt;h2&gt;GPU support available in dask-image&lt;/h2&gt;
&lt;p&gt;From &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-image&lt;/span&gt;&lt;/code&gt; version 0.6.0, there is GPU array support for four of the six subpackages:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;imread&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;ndfilters&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;ndinterp&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;ndmorph&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Subpackages of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-image&lt;/span&gt;&lt;/code&gt; that do &lt;em&gt;not&lt;/em&gt; yet have GPU support are.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;ndfourier&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;ndmeasure&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We hope to add GPU support to these in the future.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="an-example"&gt;
&lt;h2&gt;An example&lt;/h2&gt;
&lt;p&gt;Here’s an example of an image convolution with Dask on the CPU:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# CPU example&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_image.ndfilters&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;convolve&lt;/span&gt;

&lt;span class="n"&gt;s&lt;/span&gt; &lt;span class="o"&gt;=&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="mi"&gt;10&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;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;np&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="nb"&gt;int&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;prod&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reshape&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;chunks&lt;/span&gt;&lt;span class="o"&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;w&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="n"&gt;a&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndim&lt;/span&gt; &lt;span class="o"&gt;*&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;dtype&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;float32&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;convolve&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;w&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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And here’s the same example of an image convolution with Dask on the GPU. The only thing necessary to change is the type of input 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="c1"&gt;# Same example moved to the GPU&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cupy&lt;/span&gt;  &lt;span class="c1"&gt;# &amp;lt;- import cupy instead of numpy (version &amp;gt;=7.7.0)&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_image.ndfilters&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;convolve&lt;/span&gt;

&lt;span class="n"&gt;s&lt;/span&gt; &lt;span class="o"&gt;=&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="mi"&gt;10&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;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;cupy&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="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;prod&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reshape&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;chunks&lt;/span&gt;&lt;span class="o"&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;# &amp;lt;- cupy dask array&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;cupy&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="n"&gt;a&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndim&lt;/span&gt; &lt;span class="o"&gt;*&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;# &amp;lt;- cupy array&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;convolve&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;w&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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You can’t mix arrays on the CPU and arrays on the GPU in the same computation. This is why the weights &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;w&lt;/span&gt;&lt;/code&gt; must be a cupy array in the second example above.&lt;/p&gt;
&lt;p&gt;Additionally, you can transfer data between the CPU and GPU. So in situations where the GPU speedup is larger than than cost associated with transferring data, this may be useful to do.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="reading-in-images-onto-the-gpu"&gt;
&lt;h2&gt;Reading in images onto the GPU&lt;/h2&gt;
&lt;p&gt;Generally, we want to start our image processing by reading in data from images stored on disk. We can use the &lt;a class="reference external" href="http://image.dask.org/en/latest/dask_image.imread.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;imread&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; function with the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;arraytype=cupy&lt;/span&gt;&lt;/code&gt; keyword argument to do 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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_image.imread&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;imread&lt;/span&gt;

&lt;span class="n"&gt;images&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;data/BBBC039/images/*.tif&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;images_on_gpu&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;data/BBBC039/images/*.tif&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;arraytype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;cupy&amp;quot;&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/2021/03/19/image-segmentation.md&lt;/span&gt;, line 404)&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="how-you-can-get-involved"&gt;
&lt;h1&gt;How you can get involved&lt;/h1&gt;
&lt;p&gt;Create and share your own segmentation or image processing workflows with Dask (&lt;a class="reference external" href="https://github.com/dask/dask-blog/issues/47"&gt;join the current discussion on segmentation&lt;/a&gt; or &lt;a class="reference external" href="https://github.com/dask/dask-blog/issues/new?assignees=&amp;amp;amp;labels=%5B%22type%2Ffeature%22%2C+%22needs-triage%22%5D&amp;amp;amp;template=feature-request.md"&gt;propose a new blogpost topic here&lt;/a&gt;)&lt;/p&gt;
&lt;p&gt;Contribute to adding GPU support to dask-image: https://github.com/dask/dask-image/issues/133&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/03/19/image-segmentation/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="imaging" label="imaging"/>
    <published>2021-03-19T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/03/11/dask_memory_usage/</id>
    <title>Measuring Dask memory usage with dask-memusage</title>
    <updated>2021-03-11T00:00:00+00:00</updated>
    <author>
      <name>&lt;a href="https://pythonspeed.com"&gt;Itamar Turner-Trauring&lt;/a&gt;</name>
    </author>
    <content type="html">&lt;p&gt;Using too much computing resources can get expensive when you’re scaling up in the cloud.&lt;/p&gt;
&lt;p&gt;To give a real example, I was working on the image processing pipeline for a spatial gene sequencing device, which could report not just which genes were being expressed but also where they were in a 3D volume of cells.
In order to get this information, a specialized microscope took snapshots of the cell culture or tissue, and the resulting data was run through a Dask pipeline.&lt;/p&gt;
&lt;p&gt;The pipeline was fairly slow, so I did some back-of-the-envelope math to figure out our computing costs would be once we started running more data for customers.
&lt;strong&gt;It turned out that we’d be using 70% of our revenue just paying for cloud computing!&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;Clearly I needed to optimize this code.&lt;/p&gt;
&lt;p&gt;When we think about the bottlenecks in large-scale computation, we often focus on CPU: we want to use more CPU cores in order to get faster results.
Paying for all that CPU can be expensive, as in this case, and I did successfully reduce CPU usage by quite a lot.&lt;/p&gt;
&lt;p&gt;But high memory usage was also a problem, and fixing that problem led me to build a series of tools, tools that can also help you optimize and reduce your Dask memory usage.&lt;/p&gt;
&lt;p&gt;In the rest of this article you will learn:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#problem"&gt;&lt;span class="xref myst"&gt;How high memory usage can drive up your computing costs&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How a tool called &lt;a class="reference external" href="https://github.com/itamarst/dask-memusage/"&gt;dask-memusage&lt;/a&gt; can help you &lt;a class="reference internal" href="#dask-memusage"&gt;&lt;span class="xref myst"&gt;find peak memory usage of the tasks in your Dask execution graph&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How to &lt;a class="reference internal" href="#fil"&gt;&lt;span class="xref myst"&gt;further pinpoint high memory usage&lt;/span&gt;&lt;/a&gt; using the &lt;a class="reference external" href="https://pythonspeed.com/fil"&gt;Fil memory profiler&lt;/a&gt;, so you can reduce memory usage.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/11/dask_memory_usage.md&lt;/span&gt;, line 30)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="the-problem-fixed-processing-chunks-and-a-high-memory-cpu-ratio-problem"&gt;

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


&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;calculate_top_10&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;file_path&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;Path&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;gc&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;collect&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# See notes below&lt;/span&gt;

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

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

    &lt;span class="c1"&gt;# Choose the top 10:&lt;/span&gt;
    &lt;span class="n"&gt;by_count&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;sorted&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;counts&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;items&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;key&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
    &lt;span class="n"&gt;sleep&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;0.1&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# See notes below&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;file_path&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;name&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;by_count&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;:])&lt;/span&gt;


&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;main&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;directory&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="c1"&gt;# Setup the calculation:&lt;/span&gt;

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

    &lt;span class="c1"&gt;# Create the task graph:&lt;/span&gt;
    &lt;span class="n"&gt;files&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;from_sequence&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Path&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;directory&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;iterdir&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
    &lt;span class="n"&gt;graph&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;files&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;calculate_top_10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;graph&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;visualize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;filename&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;example2.png&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;rankdir&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;TD&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

    &lt;span class="c1"&gt;# Run the calculations:&lt;/span&gt;
    &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;graph&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
        &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="c1"&gt;# ... do something with results ...&lt;/span&gt;


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

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

&lt;p&gt;Dask wants to better support the needs of life scientists. We’ve been getting to know the community, in order to better understand:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Who is out there?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;What kind of problems are they trying to solve?&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We’ve learned that:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Lots of people want more examples tailored to their specific scientifc domain.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Better integration of Dask into other software is considered very important.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Managing memory constraints when working with big data is a common pain point.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Our strategic plan for this year involves three parallel streams:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#infrastructure"&gt;&lt;span class="xref myst"&gt;INFRASTRUCTURE&lt;/span&gt;&lt;/a&gt; (60%) - improvements to Dask, or to other software with many life science users.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#outreach"&gt;&lt;span class="xref myst"&gt;OUTREACH&lt;/span&gt;&lt;/a&gt; (20%) - blogposts, talks, webinars, tutorials, and examples.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#applications"&gt;&lt;span class="xref myst"&gt;APPLICATIONS&lt;/span&gt;&lt;/a&gt; (20%) - the application of Dask to a specific life science problem, collaborating with individual labs or groups.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;If you still want to have your say, it’s not too late -
&lt;a class="reference external" href="https://t.co/0NeknSdrO9?amp=1"&gt;click this link to get in touch!&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/2021/03/04/the-life-science-community.md&lt;/span&gt;, line 31)&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="contents"&gt;
&lt;h1&gt;Contents&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#background"&gt;&lt;span class="xref myst"&gt;Background&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#what-we-learned"&gt;&lt;span class="xref myst"&gt;What we learned&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#from-dask-users"&gt;&lt;span class="xref myst"&gt;From Dask users&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#from-other-software-libraries"&gt;&lt;span class="xref myst"&gt;From other software libraries&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#opportunities-we-see"&gt;&lt;span class="xref myst"&gt;Opportunities we see&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#strategic-plan"&gt;&lt;span class="xref myst"&gt;Strategic plan&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#limitations"&gt;&lt;span class="xref myst"&gt;Limitations&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#methods"&gt;&lt;span class="xref myst"&gt;Methods&lt;/span&gt;&lt;/a&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/2021/03/04/the-life-science-community.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="background"&gt;
&lt;h1&gt;Background&lt;/h1&gt;
&lt;p&gt;Recently Dask &lt;a class="reference external" href="https://chanzuckerberg.com/eoss/proposals/"&gt;won some funding&lt;/a&gt; to hire a developer (&lt;a class="reference external" href="https://github.com/GenevieveBuckley/"&gt;Genevieve Buckley&lt;/a&gt;) to improve Dask specifically for life sciences.&lt;/p&gt;
&lt;p&gt;Working with scientists is a really great way to drive growth in open source projects. Both scientists and software developers benefit. Early on, Dask had a lot of success integrating with the geosciences community. It’d be great to see similar success for life sciences too.&lt;/p&gt;
&lt;p&gt;There are several areas of life science where we see Dask being used today:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Biological image processing&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Single cell analysis&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Statistical genetics&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;…and many more&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We’ve solicited feedback from the life science community, to come up with a strategic plan to direct our effort over the next year.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/04/the-life-science-community.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="what-we-learned"&gt;
&lt;h1&gt;What we learned&lt;/h1&gt;
&lt;section id="from-dask-users"&gt;
&lt;h2&gt;From Dask users&lt;/h2&gt;
&lt;p&gt;When we talked to individual Dask users, we heard a lot of similar themes in their comments.&lt;/p&gt;
&lt;p&gt;People wanted:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Better documentation and examples&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Better support for working with constrained resources&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Better interoperability with other software tools&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The most common request was for better documentation with more examples. People across many different areas of life science all said this could help them a lot. A corresponding challenge here is the multitude of different areas of life science, all of which require targeted documentation.&lt;/p&gt;
&lt;p&gt;GPU support was also commonly mentioned. Comments about GPUs fit into two of the categories above: GPU memory is often a constraint, and life scientists also want it to be easier to apply deep learning models to their data.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="from-other-software-libraries"&gt;
&lt;h2&gt;From other software libraries&lt;/h2&gt;
&lt;p&gt;We didn’t only talk with individual users of Dask. We also spoke to developers of scientific software projects.&lt;/p&gt;
&lt;section id="why-would-other-software-libraries-adopt-dask"&gt;
&lt;h3&gt;Why would other software libraries adopt Dask?&lt;/h3&gt;
&lt;p&gt;Software projects wanted to solve problems related to:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Easier deployment to distributed clusters&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Managing memory when processing large datasets&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Parallelization of existing functionality&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Dask is good at solving those kinds of problems, and might be a good solution for this.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="who-we-ve-talked-to"&gt;
&lt;h3&gt;Who we’ve talked to&lt;/h3&gt;
&lt;p&gt;Some of the software projects we spoke to include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://napari.org/"&gt;napari&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://pystatgen.github.io/sgkit/latest/"&gt;sgkit&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://scanpy.readthedocs.io/en/stable/"&gt;scanpy&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://squidpy.readthedocs.io/en/latest/"&gt;squidpy&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://www.ilastik.org/"&gt;ilastik&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://cellprofiler.org/"&gt;CellProfiler&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="current-status"&gt;
&lt;h3&gt;Current status&lt;/h3&gt;
&lt;p&gt;&lt;a class="reference external" href="https://napari.org/"&gt;napari&lt;/a&gt; is a python based image viewer. Dask is already well-integrated with napari. Areas for opportunity here include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Improved documentation about how to work efficiently with Dask arrays in napari.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Smarter caching of neighbouring image chunks to avoid lag.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Guides for how to create plugins for napari, so the community can grow.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;a class="reference external" href="https://pystatgen.github.io/sgkit/latest/"&gt;sgkit&lt;/a&gt; is a statistical genetics toolkit. Dask is already well-integrated with sgkit. The developers would like improved infrastructure in the main Dask repositories that they can benefit from. Wishlist items include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Better ways to understand how things like array chunks change as they move through a Dask computation.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Better high level graph visualizations. Graph visualizations showing all the low level operations can be overwhelming.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Better ways to identify poorly efficient areas in Dask computations.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stability when new versions of Dask are released&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Making it easier to run Dask in the cloud. They are currently using &lt;a class="reference external" href="https://github.com/dask/dask-cloudprovider"&gt;dask-cloudprovider&lt;/a&gt; and finding that very useful.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;a class="reference external" href="https://scanpy.readthedocs.io/en/stable/"&gt;scanpy&lt;/a&gt; is a library for single cell analysis in Python. It is built together with &lt;a class="reference external" href="https://anndata.readthedocs.io/en/latest/"&gt;anndata&lt;/a&gt;, an annotated data structure.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Data size is less of an issue for scanpy users, although anndata developers do think support for Dask would be a useful thing to add.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Support for sparse arrays is very important for these communities.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;a class="reference external" href="https://squidpy.readthedocs.io/en/latest/"&gt;squidpy&lt;/a&gt; is a tool for the analysis and visualization of spatial molecular data. It builds on top of scanpy and anndata. Because squidpy involves large imaging data on top of what we’d normally see for datasets in scanpy/anndata, this is a project with a large area of opportunity for Dask.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Integrating Dask with the squidpy ImageContainer class is a good first step towards handling large image data within the availabe RAM constraints.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;a class="reference external" href="https://www.ilastik.org/"&gt;ilastik&lt;/a&gt; does not currently use Dask at all. They are curious to see if Dask can make it easier to scale up from a single machine to a cluster.
Users generally train an ilastik model interactively, and then want to apply it to many images. This second step is often when people want an easy way to scale up the computing resources available.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://cellprofiler.org/"&gt;CellProfiler&lt;/a&gt; is a pipeline tool for image processing. They have briefly experimented with Dask before.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Primarily, they want to parallelize existing functionality.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Most common pipelines fall into three major “user stories” where focussing effort would make the most impact:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Image processing&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Object processing&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Measurements&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/04/the-life-science-community.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&gt;
&lt;/section&gt;
&lt;section id="opportunities-we-see"&gt;
&lt;h1&gt;Opportunities we see&lt;/h1&gt;
&lt;p&gt;Because large scientific software projects have many users, improvements here would be high value for the scientific community. This is a huge area of opportunity. We plan to collaborate with these developer communities as much as possible to drive this forward.&lt;/p&gt;
&lt;p&gt;Another area of opportunity is improving infrastructure for &lt;a class="reference external" href="https://github.com/dask/dask/issues/7141"&gt;high level graph visualizations&lt;/a&gt;. Power users and novices alike would benefit from better tools for identifying areas of inefficiencies in Dask computations.&lt;/p&gt;
&lt;p&gt;Finally, continuing to build support for Dask arrays with non-numpy chunks is also a high impact area of opportunity. In particular, support for sparse arrays, and support for arrays on the GPU were highlighted as very important to the life science community.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/04/the-life-science-community.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="strategic-direction"&gt;
&lt;h1&gt;Strategic direction&lt;/h1&gt;
&lt;p&gt;We’re going to manage this project with three parallel streams:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#infrastructure"&gt;&lt;span class="xref myst"&gt;INFRASTRUCTURE&lt;/span&gt;&lt;/a&gt; (60%)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#outreach"&gt;&lt;span class="xref myst"&gt;OUTREACH&lt;/span&gt;&lt;/a&gt; (20%)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#applications"&gt;&lt;span class="xref myst"&gt;APPLICATIONS&lt;/span&gt;&lt;/a&gt; (20%)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Each stream will likely have one primary project at any time, with many more queued. Within each stream, proposed projects will be ranked according to: level of impact, time commitment required, and the availability of other developer resources.&lt;/p&gt;
&lt;section id="infrastructure"&gt;
&lt;h2&gt;Infrastructure&lt;/h2&gt;
&lt;p&gt;Infrastructure projects are improvements to either:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Projects housed within the &lt;a class="reference external" href="https://github.com/dask/"&gt;Dask organisation&lt;/a&gt;, or&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Other software projects involving Dask with large numbers of life science users&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We’ll aim to spend around 60% of project effort on infrastructure.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="outreach"&gt;
&lt;h2&gt;Outreach&lt;/h2&gt;
&lt;p&gt;Outreach activities include blogposts, talks, webinars, tutorials, and creating examples for documentation. We aim to spend around 20% of project effort on outreach.&lt;/p&gt;
&lt;p&gt;If you have outreach ideas you want to share (perhaps you run a student group or popular meetup) then you can &lt;a class="reference external" href="https://docs.google.com/forms/d/e/1FAIpQLScBi8YOx3gGkL9rz8TsRTIZYiRha9qYOvXu4EZx9qGLtjLGCw/viewform?usp=sf_link"&gt;get in touch with us here&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="applications"&gt;
&lt;h2&gt;Applications&lt;/h2&gt;
&lt;p&gt;The final stream focusses on the application of Dask to a specific problem in life science.&lt;/p&gt;
&lt;p&gt;These projects generally involve collaborating with individual labs or group, and have an end goal of summarizing their workflow in a blogpost. This feeds back into our outreach, so others in the community can learn from it.&lt;/p&gt;
&lt;p&gt;Ideally these are short term projects, so we can showcase many different applications of Dask. We aim to spend around 20% of project effort on applications.&lt;/p&gt;
&lt;p&gt;If you use Dask and have an example in mind you’d like to share, then you can &lt;a class="reference external" href="https://docs.google.com/forms/d/e/1FAIpQLScBi8YOx3gGkL9rz8TsRTIZYiRha9qYOvXu4EZx9qGLtjLGCw/viewform?usp=sf_link"&gt;get in touch with us here&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="how-will-we-know-what-success-looks-like"&gt;
&lt;h2&gt;How will we know what success looks like?&lt;/h2&gt;
&lt;p&gt;The role of Dask Life Science Fellow has a very broad scope, so there are a lot of different ways we could be successful within this space.&lt;/p&gt;
&lt;p&gt;Some indicators of success are:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Bugs being clearly described, or bottlenecks clearly identified&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bug fixes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Improvements or new features made to Dask infrastructure&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Improvements or new features made in related project repositories&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Better integration or support for Dask made in related project repositories for life sciences&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Better documentation with examples tailored to specific areas of life science&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Blogposts written (ideally in collaboration with Dask users)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Talks given&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Webinars produced&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tutorials created&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We won’t have the time or the resources to do all the things, but we will be able to make an impact by focussing on a subset.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/04/the-life-science-community.md&lt;/span&gt;, line 196)&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="limitations"&gt;
&lt;h1&gt;Limitations&lt;/h1&gt;
&lt;p&gt;The information we discovered talking to the life science community is likely to be biased in a few different ways.&lt;/p&gt;
&lt;p&gt;My (Genevieve’s) network is strongest among imaging scientists, and among people in Australia. It’s much less strong for other fields in life science, as my original training is in physics.&lt;/p&gt;
&lt;p&gt;The Dask project has strong links to other open source python projects, including scientific software. The Dask developer community also has strong links from companies including NVIDIA, Quansight, and others. They are likely to be over-represented among the people we spoke to.&lt;/p&gt;
&lt;p&gt;It’s much harder to find people who aren’t using Dask at all yet but have problems that would be a good fit for it. These people are very unlikely to be, say following &lt;a class="reference external" href="https://twitter.com/dask_dev/"&gt;Dask on twitter&lt;/a&gt;, and probably won’t be aware that we’re looking for them.&lt;/p&gt;
&lt;p&gt;I don’t think there are any perfect solutions to these problems.
We’ve tried to mitigate these effects by using loose second and third degree connections to spread awareness, as well as posting in science public forums.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/04/the-life-science-community.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="methods"&gt;
&lt;h1&gt;Methods&lt;/h1&gt;
&lt;p&gt;We used a variety of approaches to gather feedback from the life science community.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;A &lt;a class="reference external" href="https://t.co/0NeknSdrO9?amp=1"&gt;short survey&lt;/a&gt; was created to gather comments&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It was advertised by the &lt;a class="reference external" href="https://twitter.com/dask_dev/"&gt;&amp;#64;dask_dev&lt;/a&gt; twitter account&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We asked related software projects consider retweeting for reach (&lt;a class="reference external" href="https://twitter.com/napari_imaging/status/1360090299901505543"&gt;example&lt;/a&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We posted in scientific Slack groups and online public forums&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We emailed other life scientists in our network, asking them to let their networks know too&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We contacted a number of life science researchers directly.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We contacted several other scientific software groups directly and spoke with the developers.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/04/the-life-science-community.md&lt;/span&gt;, line 221)&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="join-the-discussion"&gt;
&lt;h1&gt;Join the discussion&lt;/h1&gt;
&lt;p&gt;Come join us in the Dask slack! We have a #life-science channel so there’s a place to discuss things relevant to the Dask life science community. You can &lt;a class="reference external" href="https://join.slack.com/t/dask/shared_invite/zt-mfmh7quc-nIrXL6ocgiUH2haLYA914g"&gt;request an invite to the Slack here&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/03/04/the-life-science-community/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="imaging" label="imaging"/>
    <published>2021-03-04T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/03/03/summit/</id>
    <title>Dask User Summit 2021</title>
    <updated>2021-03-03T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</name>
    </author>
    <content type="html">&lt;p&gt;Dask is organizing a &lt;a class="reference external" href="https://summit.dask.org"&gt;user summit&lt;/a&gt; in mid-May.
This will be a remote event focused on bringing together developers and users of Dask and the distributed PyData stack in different domains.&lt;/p&gt;
&lt;p&gt;User Summits like this are particularly important for a project like Dask
which serves such a diverse set of use cases.
Dask’s user communities include industries like finance, government, health,
geoscience, imaging, machine learning, and more. These communities often have
very similar problems, but don’t often communicate with each other.&lt;/p&gt;
&lt;p&gt;User summits provide a venue for disparate domains to connect over shared
technology challenges. Often a solution designed for one domain is useful for
others. As technologists, this sharing is critical in order to promote
consistent and high quality software solutions across domains, rather than
silo’ed solutions.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/03/summit.md&lt;/span&gt;, line 23)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="history"&gt;

&lt;p&gt;We organized a summit a year ago, focusing mainly on developers.
This was a fantastic time and resulted in a surprising amount of consensus building and forward movement both in technological and domain-specific directions.&lt;/p&gt;
&lt;p&gt;For more on our summit last year, see &lt;a class="reference internal" href="#../../../2020/04/28/dask-summit.html"&gt;&lt;span class="xref myst"&gt;this post&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;&lt;img src="https://pbs.twimg.com/media/ERykEc9XUAEFq-L?format=jpg&amp;name=large"
     width="40%"&gt;
&lt;img src="https://pbs.twimg.com/media/ERzXhHnWAAE_zDA?format=jpg&amp;name=large"
    width="40%"&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/2021/03/03/summit.md&lt;/span&gt;, line 35)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="organization"&gt;
&lt;h1&gt;Organization&lt;/h1&gt;
&lt;p&gt;We’ve asked &lt;a class="reference external" href="https://numfocus.org"&gt;NumFOCUS&lt;/a&gt; to organize this event for us.
NumFOCUS runs the highly successful and community oriented PyData conference
series, and had great success with their remote-first PyData Global conference
late last year.&lt;/p&gt;
&lt;p&gt;Tickets are intended to be reasonably priced on a sliding scale, with assistance given to any in need.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/03/summit.md&lt;/span&gt;, line 44)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="open-cfp"&gt;
&lt;h1&gt;Open CFP&lt;/h1&gt;
&lt;p&gt;I would like to encourage people submit proposals to talk at &lt;a class="reference external" href="https://summit.dask.org"&gt;summit.dask.org&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;I would like to especially extend an invitation to those who are new to
the Dask community, or new to speaking in general. This year we’re especially
trying to highlight use cases of Dask, rather than developers pushing the
technology forward (although these talks are of course welcome as well).&lt;/p&gt;
&lt;p&gt;If you have an idea for a talk then please submit something and we’ll work
together on making it fit. Alternatively, if you have a colleague that you
think would enjoy or grow from speaking then I encourage you to encourage them
as well.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/03/summit.md&lt;/span&gt;, line 58)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="workshops"&gt;
&lt;h1&gt;Workshops&lt;/h1&gt;
&lt;p&gt;Finally, I’m excited about an experiment that we’re running this year with
&lt;em&gt;workshops&lt;/em&gt;. These are intended to be two-hour blocks of time dedicated to
a particular topic, organized by a specific community member (perhaps you?).
If you have a consistent theme for a set of 3-5 talks then this option gives
you the ability to curate and control a dedicated block of the conference. You
can invite your colleagues and collaborators. We’ll handle the conference
infrastructure while you handle the content.&lt;/p&gt;
&lt;p&gt;We stole this structure from workshops at larger academic conferences. We
think that it fits Dask well specifically because of the federated nature of
our community. We hope that it gives space for sub-communities to assemble and
better establish cohesive working groups.&lt;/p&gt;
&lt;p&gt;Themes in the past have included topics like Pangeo, RAPIDS, workflow
management, imaging, and 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/2021/03/03/summit.md&lt;/span&gt;, line 76)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="apply-to-speak"&gt;
&lt;h1&gt;Apply to speak&lt;/h1&gt;
&lt;p&gt;Again, I encourage you and your colleagues to submit applications to speak this
year in May. The proposal page is at
&lt;a class="reference external" href="https://summit.dask.org/present/#guidelines"&gt;https://summit.dask.org/present/#guidelines&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/03/03/summit/"/>
    <summary>Dask is organizing a user summit in mid-May.
This will be a remote event focused on bringing together developers and users of Dask and the distributed PyData stack in different domains.</summary>
    <published>2021-03-03T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2020/11/12/deconvolution/</id>
    <title>Image Analysis Redux</title>
    <updated>2020-11-12T00:00:00+00:00</updated>
    <author>
      <name>John Kirkham (NVIDIA) and Ben Zaitlen (NVIDIA)</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/11/12/deconvolution.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2019/08/09/image-itk"&gt;Last year&lt;/a&gt; we experimented with
Dask/ITK/Scikit-Image to perform large scale image analysis on a stack of 3D
images. Specifically, we looked at deconvolution, a common method to &lt;em&gt;deblur&lt;/em&gt;
images. Now, a year later, we return to these experiments with a better
understanding of how Dask and CuPy can interact, enhanced serialization
methods, and support from the open-source community. This post looks at the
following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Implementing a common deconvolution method for CPU + GPU&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Leveraging Dask to perform deconvolution on a larger dataset&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Exploring the results with the Napari image viewer&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/2020/11/12/deconvolution.md&lt;/span&gt;, line 23)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="image-analysis-redux"&gt;
&lt;h1&gt;Image Analysis Redux&lt;/h1&gt;
&lt;p&gt;Previously we used the &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Richardson%E2%80%93Lucy_deconvolution"&gt;Richardson Lucy
(RL)&lt;/a&gt;
deconvolution algorithm from ITK and
&lt;a class="reference external" href="https://github.com/scikit-image/scikit-image/blob/master/skimage/restoration/deconvolution.py#L329"&gt;Scikit-Image&lt;/a&gt;.
We left off at theorizing how GPUs could potentially help accelerate these
workflows. Starting with Scikit-Image’s implementation, we naively tried
replacing &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scipy.signal.convolve&lt;/span&gt;&lt;/code&gt; calls with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cupyx.scipy.ndimage.convolve&lt;/span&gt;&lt;/code&gt;,
and while performance improved, it did not improve &lt;em&gt;significantly&lt;/em&gt; – that is,
we did not get the 100X speed we were looking for.&lt;/p&gt;
&lt;p&gt;As it often turns out in mathematics a problem that can be inefficient to solve
in one representation can often be made more efficent by transforming the data
beforehand. In this new representation we can solve the same problem
(convolution in this case) more easily before transforming the result back into
a more familiar representation. When it comes to convolution, the
transformation we apply is called &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Fast_Fourier_transform"&gt;Fast-Fourier Transform
(FFT)&lt;/a&gt;. Once this is
applied we are able to convolve data using a simple multiplication.&lt;/p&gt;
&lt;p&gt;As it turns out this FFT transformation is extremely fast on both CPUs and
GPUs. Similarly the algorithm we can write with FFTs is accelerated. This is a
commonly used technique in the image processing field to speed up convolutions.
Despite the added step of doing FFTs, the cost of transformation + the cost of
the algorithm is still lower than performing the original algorithm in real
space. We (and others before us) found this was the case for Richardson Lucy
(on both CPUs and GPUs) and performance continued increasing when we
parallelized with Dask over multiple GPUs.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/11/12/deconvolution.md&lt;/span&gt;, line 53)&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-from-open-source"&gt;
&lt;h1&gt;Help from Open-Source&lt;/h1&gt;
&lt;p&gt;An FFT RL equivalent has been around for some time and the good folks at the
&lt;a class="reference external" href="https://sdo.gsfc.nasa.gov/mission/instruments.php"&gt;Solar Dynamics Observatory&lt;/a&gt;
built and shared a NumPy/CuPy implementation as part the &lt;a class="reference external" href="https://aiapy.readthedocs.io/en/v0.2.0/_modules/aiapy/psf/deconvolve.html"&gt;Atmospheric Imaging
Assembly&lt;/a&gt;
Python package (aiapy). We slightly modified their implementation to handle 3D
as well as 2D &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Point_spread_function"&gt;Point Spread
Functions&lt;/a&gt; and to take
advantage of
&lt;a class="reference external" href="https://numpy.org/neps/nep-0018-array-function-protocol.html"&gt;NEP-18&lt;/a&gt; for
convenient dispatching of NumPy and CuPy to NumPy and CuPy 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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;deconvolve&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;iterations&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="c1"&gt;# Pad PSF with zeros to match image shape&lt;/span&gt;
    &lt;span class="n"&gt;pad_l&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;pad_r&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;divmod&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;array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;&lt;span class="p"&gt;)&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;array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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;pad_r&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="n"&gt;pad_l&lt;/span&gt;
    &lt;span class="n"&gt;psf&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;pad&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;psf&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="nb"&gt;zip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pad_l&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;pad_r&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;constant&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;constant_values&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;# Recenter PSF at the origin&lt;/span&gt;
    &lt;span class="c1"&gt;# Needed to ensure PSF doesn&amp;#39;t introduce an offset when&lt;/span&gt;
    &lt;span class="c1"&gt;# convolving with image&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="n"&gt;psf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndim&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="n"&gt;psf&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;roll&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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="mi"&gt;2&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="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

    &lt;span class="c1"&gt;# Convolution requires FFT of the PSF&lt;/span&gt;
    &lt;span class="n"&gt;psf&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;fft&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rfftn&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

    &lt;span class="c1"&gt;# Perform deconvolution in-place on a copy of the image&lt;/span&gt;
    &lt;span class="c1"&gt;# (avoids changing the original)&lt;/span&gt;
    &lt;span class="n"&gt;img_decon&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;copy&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&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;_&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="n"&gt;iterations&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="n"&gt;ratio&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;img&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;fft&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;irfftn&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;fft&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rfftn&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img_decon&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;img_decon&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;fft&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;irfftn&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;fft&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rfftn&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ratio&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;conj&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;conj&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;img_decon&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;For a 1.3 GB image we measured the following:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;CuPy ~3 seconds for 20 iterations&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;NumPy ~36 seconds for 2 iterations&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We see 10x increase in speed for 10 times the number of iterations – very
close to our desired 100x speedup! Let’s explore how this implementation
performs with real biological data and 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/2020/11/12/deconvolution.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="define-a-dask-cluster-and-load-the-data"&gt;
&lt;h1&gt;Define a Dask Cluster and Load the Data&lt;/h1&gt;
&lt;p&gt;We were provided sample data from &lt;a class="reference external" href="https://www.nibib.nih.gov/about-nibib/staff/hari-shroff"&gt;Prof.
Shroff’s&lt;/a&gt; lab at the
NIH. The data originally was provided as a 3D TIFF file which we subsequently
converted to Zarr with a shape of (950, 2048, 2048).&lt;/p&gt;
&lt;p&gt;We start by creating a Dask cluster on a DGX2 (16 GPUs in a single node) and
loading the image stored in Zarr :&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/quasiben/3a638bb9a4f075ac9041bf66974ebb45"&gt;Example Notebook&lt;/a&gt;&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_cuda&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;LocalCUDACluster&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;

&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;rmm&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cupy&lt;/span&gt;&lt;span class="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;cp&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCUDACluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;local_directory&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;/tmp/bzaitlen&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;enable_nvlink&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;rmm_pool_size&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;26GB&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;run&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;cp&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cuda&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_allocator&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;rmm&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rmm_cupy_allocator&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;imgs&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;from_zarr&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;/public/NVMICROSCOPY/y1z1_C1_A.zarr/&amp;quot;&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;table&gt;
  &lt;thead&gt;
    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 7.97 GB &lt;/td&gt; &lt;td&gt; 8.39 MB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (950, 2048, 2048) &lt;/td&gt; &lt;td&gt; (1, 2048, 2048) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 951 Tasks &lt;/td&gt;&lt;td&gt; 950 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; uint16 &lt;/td&gt;&lt;td&gt; numpy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;/td&gt;
&lt;td&gt;
&lt;svg width="212" height="202" style="stroke:rgb(0,0,0);stroke-width:1" &gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="42" y2="32" style="stroke-width:2" /&gt;
  &lt;line x1="10" y1="120" x2="42" y2="152" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="10" y2="120" style="stroke-width:2" /&gt;
  &lt;line x1="12" y1="2" x2="12" y2="122" /&gt;
  &lt;line x1="14" y1="4" x2="14" y2="124" /&gt;
  &lt;line x1="16" y1="6" x2="16" y2="126" /&gt;
  &lt;line x1="18" y1="8" x2="18" y2="128" /&gt;
  &lt;line x1="20" y1="10" x2="20" y2="130" /&gt;
  &lt;line x1="22" y1="12" x2="22" y2="132" /&gt;
  &lt;line x1="24" y1="14" x2="24" y2="134" /&gt;
  &lt;line x1="26" y1="16" x2="26" y2="136" /&gt;
  &lt;line x1="28" y1="18" x2="28" y2="138" /&gt;
  &lt;line x1="30" y1="20" x2="30" y2="140" /&gt;
  &lt;line x1="32" y1="22" x2="32" y2="142" /&gt;
  &lt;line x1="34" y1="24" x2="34" y2="144" /&gt;
  &lt;line x1="36" y1="26" x2="36" y2="146" /&gt;
  &lt;line x1="38" y1="28" x2="38" y2="148" /&gt;
  &lt;line x1="41" y1="31" x2="41" y2="151" /&gt;
  &lt;line x1="42" y1="32" x2="42" y2="152" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 42.743566,32.743566 42.743566,152.743566 10.000000,120.000000" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="130" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="12" y1="2" x2="132" y2="2" /&gt;
  &lt;line x1="14" y1="4" x2="134" y2="4" /&gt;
  &lt;line x1="16" y1="6" x2="136" y2="6" /&gt;
  &lt;line x1="18" y1="8" x2="138" y2="8" /&gt;
  &lt;line x1="20" y1="10" x2="140" y2="10" /&gt;
  &lt;line x1="22" y1="12" x2="142" y2="12" /&gt;
  &lt;line x1="24" y1="14" x2="144" y2="14" /&gt;
  &lt;line x1="26" y1="16" x2="146" y2="16" /&gt;
  &lt;line x1="28" y1="18" x2="148" y2="18" /&gt;
  &lt;line x1="30" y1="20" x2="150" y2="20" /&gt;
  &lt;line x1="32" y1="22" x2="152" y2="22" /&gt;
  &lt;line x1="34" y1="24" x2="154" y2="24" /&gt;
  &lt;line x1="36" y1="26" x2="156" y2="26" /&gt;
  &lt;line x1="38" y1="28" x2="158" y2="28" /&gt;
  &lt;line x1="41" y1="31" x2="161" y2="31" /&gt;
  &lt;line x1="42" y1="32" x2="162" y2="32" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="42" y2="32" style="stroke-width:2" /&gt;
  &lt;line x1="130" y1="0" x2="162" y2="32" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 130.000000,0.000000 162.743566,32.743566 42.743566,32.743566" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="42" y1="32" x2="162" y2="32" style="stroke-width:2" /&gt;
  &lt;line x1="42" y1="152" x2="162" y2="152" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="42" y1="32" x2="42" y2="152" style="stroke-width:2" /&gt;
  &lt;line x1="162" y1="32" x2="162" y2="152" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="42.743566,32.743566 162.743566,32.743566 162.743566,152.743566 42.743566,152.743566" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="102.743566" y="172.743566" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;2048&lt;/text&gt;
&lt;text x="182.743566" y="92.743566" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(-90,182.743566,92.743566)"&gt;2048&lt;/text&gt;
&lt;text x="16.371783" y="156.371783" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(45,16.371783,156.371783)"&gt;950&lt;/text&gt;
&lt;/svg&gt;&lt;/p&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;From the Dask output above you can see the data is a z-stack of 950 images
where each slice is 2048x2048. For this data set, we can improve GPU
performance if we operate on larger chunks. Additionally, we need to ensure
the chunks are are least as big as the PSF which in this case is, (128, 128,
128). As we did our work on a DGX2, which has 16 GPUs, we can comfortably fit
the data and perform deconvolution on each GPU if we &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;rechunk&lt;/span&gt;&lt;/code&gt; the data
accordingly:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# chunk with respect to PSF shape (128, 128, 128)&lt;/span&gt;
&lt;span class="n"&gt;imgs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;imgs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&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;0&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;190&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;512&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;512&lt;/span&gt;&lt;span class="p"&gt;})&lt;/span&gt;
&lt;span class="n"&gt;imgs&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;td&gt;
&lt;table&gt;
  &lt;thead&gt;
    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 7.97 GB &lt;/td&gt; &lt;td&gt; 99.61 MB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (950, 2048, 2048) &lt;/td&gt; &lt;td&gt; (190, 512, 512) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 967 Tasks &lt;/td&gt;&lt;td&gt; 80 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; uint16 &lt;/td&gt;&lt;td&gt; numpy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;/td&gt;
&lt;p&gt;Next, we convert to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;float32&lt;/span&gt;&lt;/code&gt; as the data may not already be of floating point
type. Also 32-bit is a bit faster than 64-bit when computing and saves a bit on
memory. Below we map &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cupy.asarray&lt;/span&gt;&lt;/code&gt; onto each block of data. &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cupy.asarray&lt;/span&gt;&lt;/code&gt;
moves the data from host memory (NumPy) to the device/GPU (CuPy).&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;imgs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;imgs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;astype&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;float32&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;c_imgs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;imgs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cp&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;asarray&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;table&gt;
  &lt;thead&gt;
    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 15.94 GB &lt;/td&gt; &lt;td&gt; 199.23 MB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (950, 2048, 2048) &lt;/td&gt; &lt;td&gt; (190, 512, 512) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 80 Tasks &lt;/td&gt;&lt;td&gt; 80 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; float32 &lt;/td&gt;&lt;td&gt; cupy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;/td&gt;
&lt;td&gt;
&lt;svg width="212" height="202" style="stroke:rgb(0,0,0);stroke-width:1" &gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="42" y2="32" style="stroke-width:2" /&gt;
  &lt;line x1="10" y1="30" x2="42" y2="62" /&gt;
  &lt;line x1="10" y1="60" x2="42" y2="92" /&gt;
  &lt;line x1="10" y1="90" x2="42" y2="122" /&gt;
  &lt;line x1="10" y1="120" x2="42" y2="152" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="10" y2="120" style="stroke-width:2" /&gt;
  &lt;line x1="16" y1="6" x2="16" y2="126" /&gt;
  &lt;line x1="23" y1="13" x2="23" y2="133" /&gt;
  &lt;line x1="29" y1="19" x2="29" y2="139" /&gt;
  &lt;line x1="36" y1="26" x2="36" y2="146" /&gt;
  &lt;line x1="42" y1="32" x2="42" y2="152" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.0,0.0 42.74356617647059,32.74356617647059 42.74356617647059,152.74356617647058 10.0,120.0" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="130" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="16" y1="6" x2="136" y2="6" /&gt;
  &lt;line x1="23" y1="13" x2="143" y2="13" /&gt;
  &lt;line x1="29" y1="19" x2="149" y2="19" /&gt;
  &lt;line x1="36" y1="26" x2="156" y2="26" /&gt;
  &lt;line x1="42" y1="32" x2="162" y2="32" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="42" y2="32" style="stroke-width:2" /&gt;
  &lt;line x1="40" y1="0" x2="72" y2="32" /&gt;
  &lt;line x1="70" y1="0" x2="102" y2="32" /&gt;
  &lt;line x1="100" y1="0" x2="132" y2="32" /&gt;
  &lt;line x1="130" y1="0" x2="162" y2="32" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.0,0.0 130.0,0.0 162.74356617647058,32.74356617647059 42.74356617647059,32.74356617647059" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="42" y1="32" x2="162" y2="32" style="stroke-width:2" /&gt;
  &lt;line x1="42" y1="62" x2="162" y2="62" /&gt;
  &lt;line x1="42" y1="92" x2="162" y2="92" /&gt;
  &lt;line x1="42" y1="122" x2="162" y2="122" /&gt;
  &lt;line x1="42" y1="152" x2="162" y2="152" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="42" y1="32" x2="42" y2="152" style="stroke-width:2" /&gt;
  &lt;line x1="72" y1="32" x2="72" y2="152" /&gt;
  &lt;line x1="102" y1="32" x2="102" y2="152" /&gt;
  &lt;line x1="132" y1="32" x2="132" y2="152" /&gt;
  &lt;line x1="162" y1="32" x2="162" y2="152" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="42.74356617647059,32.74356617647059 162.74356617647058,32.74356617647059 162.74356617647058,152.74356617647058 42.74356617647059,152.74356617647058" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="102.743566" y="172.743566" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;2048&lt;/text&gt;
&lt;text x="182.743566" y="92.743566" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(-90,182.743566,92.743566)"&gt;2048&lt;/text&gt;
&lt;text x="16.371783" y="156.371783" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(45,16.371783,156.371783)"&gt;950&lt;/text&gt;
&lt;/svg&gt;&lt;/p&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;What we now have is a Dask array composed of 16 CuPy blocks of data. Notice
how Dask provides nice typing information in the SVG output. When we moved
from NumPy to CuPy, the block diagram above displays &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Type:&lt;/span&gt; &lt;span class="pre"&gt;cupy.ndarray&lt;/span&gt;&lt;/code&gt; –
this is a nice sanity check.&lt;/p&gt;
&lt;p&gt;The last piece we need before running the deconvolution is the PSF which should
also be loaded onto the GPU:&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;skimage.io&lt;/span&gt;

&lt;span class="n"&gt;psf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;/public/NVMICROSCOPY/PSF.tif&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;c_psf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;cp&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;asarray&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Lastly, we call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt; with the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;deconvolve&lt;/span&gt;&lt;/code&gt; function across the Dask
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;out&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;map_overlap&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;deconvolve&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;c_imgs&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;c_psf&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;iterations&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;meta&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;c_imgs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_meta&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;depth&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="nb"&gt;tuple&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;array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;c_psf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;//&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
    &lt;span class="n"&gt;boundary&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;periodic&amp;quot;&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;out&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/deconvolve.png"&gt;
    &lt;img src="/images/deconvolve.png" width="100%"&gt;&lt;/a&gt;
&lt;p&gt;The image above is taken from a mouse intestine.&lt;/p&gt;
&lt;p&gt;With Dask and multiple GPUs, we measured deconvolution of an 16GB image in ~30
seconds! But this is just the first step towards accelerated image science.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/11/12/deconvolution.md&lt;/span&gt;, line 386)&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="napari"&gt;
&lt;h1&gt;Napari&lt;/h1&gt;
&lt;p&gt;Deconvolution is just one operation and one tool, an image scientist or
microscopist will need. They will need other tools as they study the
underlying biology. Before getting to those next steps, they will need tools
to visualize the data. &lt;a class="reference external" href="https://napari.org/"&gt;Napari&lt;/a&gt;, a multi-dimensional image
viewer used in the PyData Bio ecosystem, is a good tool for visualizing this
data. As an experiment, we ran the same workflow on a local workstation with 2
Quadro RTX 8000 GPUs connected with NVLink. &lt;a class="reference external" href="https://gist.github.com/quasiben/02b3dabba8fb3415e40e685b3cb2ca4a"&gt;Example
Notebook&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;By adding a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt; call to our array, we can move our data &lt;em&gt;back&lt;/em&gt; from
GPU to CPU (device to host).&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;cupy_to_numpy&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cupy&lt;/span&gt;&lt;span class="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;cp&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;cp&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;asnumpy&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;np_out&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;out&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cupy_to_numpy&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;out&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/napari-deconv.png"&gt;
    &lt;img src="/images/napari-deconv.png" width="100%"&gt;&lt;/a&gt;
&lt;p&gt;When the user moves the slider on the Napari UI, we are instructing dask to the
following:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Load the data from disk onto the GPU (CuPy)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Compute the deconvolution&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Move back to the host (NumPy)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Render with Napari&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This has about a second latency which is great for a naive implementation! We
can improve this by adding caching, improving communications with
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt;, and optimizing the deconvolution kernel.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/11/12/deconvolution.md&lt;/span&gt;, line 423)&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;We have now shown with Dask + CuPy how one can perform Richardson-Lucy
Deconvolution. This required a minimal amount of code. Combining this with an
image viewer (Napari), we were able to inspect the data and our result. All of
this performed reasonably well by assembling PyData libraries: Dask, CuPy,
Zarr, and Napari with a new deconvolution kernel. Hopefully this provides you
a good template to get started analyzing your own data and demonstrates the
richness and easy expression of custom workflows. If you run into any
challenges, please reach out on &lt;a class="reference external" href="https://github.com/dask/dask/issues"&gt;the Dask issue
tracker&lt;/a&gt; and we would be happy to engage
with you :)&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2020/11/12/deconvolution/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <published>2020-11-12T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2020/09/22/user_survey/</id>
    <title>2020 Dask User Survey</title>
    <updated>2020-09-22T00:00:00+00:00</updated>
    <author>
      <name>Tom Augspurger</name>
    </author>
    <content type="html">&lt;style type="text/css"&gt;
table td {
    background: none;
}

table tr.even td {
    background: none;
}

table {
    text-shadow: none;
}

table tr:hover td {
    background: none;
}

&lt;/style&gt;
&lt;p&gt;This post presents the results of the 2020 Dask User Survey,
which ran earlier this summer. Thanks to everyone who took the time to fill out the survey!
These results help us better understand the Dask community and will guide future development efforts.&lt;/p&gt;
&lt;p&gt;The raw data, as well as the start of an analysis, can be found in this binder:&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://mybinder.org/v2/gh/dask/dask-examples/main?urlpath=%2Ftree%2Fsurveys%2F2020.ipynb"&gt;&lt;img alt="Binder" src="https://mybinder.org/badge_logo.svg" /&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Let us know if you find anything in the data.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/09/22/user_survey.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 id="highlights"&gt;

&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;We had 240 responses to the survey (slightly fewer than last year, which had about 260).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Overall, results look mostly similar to last year’s.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Our documentation has probably improved relative to last year&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Respondents care more about performance relative to last year.&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/2020/09/22/user_survey.md&lt;/span&gt;, line 45)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="new-questions"&gt;
&lt;h1&gt;New Questions&lt;/h1&gt;
&lt;p&gt;Most of the questions are the same as in 2019. We added a couple questions about deployment and dashboard usage. Let’s look at those first.&lt;/p&gt;
&lt;p&gt;Among respondents who use a Dask package to deploy a cluster (about 53% of respondents), there’s a wide spread of methods.&lt;/p&gt;
&lt;img src="/images/2020_survey/2020_3_0.png"&gt;
&lt;p&gt;Most people access the dashboard through a web browser. Those not using the dashboard are likely (hopefully) just using Dask on a single machine with the threaded scheduler (though the dashboard works fine on a single machine as well).&lt;/p&gt;
&lt;img src="/images/2020_survey/2020_5_0.png"&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/09/22/user_survey.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="learning-resources"&gt;
&lt;h1&gt;Learning Resources&lt;/h1&gt;
&lt;p&gt;Respondents’ learning material usage is farily similar to last year. The most notable differences are from
our survey form providing more options (our &lt;a class="reference external" href="https://www.youtube.com/channel/UCj9eavqmvwaCyKhIlu2GaoA"&gt;YouTube channel&lt;/a&gt; and “Gitter chat”). Other than that, &lt;a class="reference external" href="https://examples.dask.org"&gt;examples.dask.org&lt;/a&gt; might be relatively more popular.&lt;/p&gt;
&lt;img src="/images/2020_survey/2020_7_0.png"&gt;
&lt;p&gt;Just like last year, we’ll look at resource usage grouped by how often they use Dask.&lt;/p&gt;
&lt;img src="/images/2020_survey/2020_10_0.png"&gt;
&lt;p&gt;A few observations&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;GitHub issues are becoming relatively less popular, which perhaps reflects better documentation or stability (assuming people go to the issue tracker when they can’t find the answer in the docs or they hit a bug).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://examples.dask.org"&gt;https://examples.dask.org&lt;/a&gt; is notably now more popular among occasinal users.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;In response to last year’s survey, we invested time in making &lt;a class="reference external" href="https://tutorial.dask.org"&gt;https://tutorial.dask.org&lt;/a&gt; better, which we previously felt was lacking. Its usage is still about the same as last year’s (pretty popular), so it’s unclear whether we should dedicate additional focus there.&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/2020/09/22/user_survey.md&lt;/span&gt;, line 74)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="how-do-you-use-dask"&gt;
&lt;h1&gt;How do you use Dask?&lt;/h1&gt;
&lt;p&gt;API usage remains about the same as last year (recall that about 20 fewer people took the survey and people can select multiple, so relative differences are most interesting). We added new choices for RAPIDS, Prefect, and XGBoost, each of which are somewhat popular (in the neighborhood of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.Bag&lt;/span&gt;&lt;/code&gt;).&lt;/p&gt;
&lt;img src="/images/2020_survey/2020_12_0.png"&gt;
&lt;p&gt;About 65% of our users are using Dask on a cluster at least some of the time, which is similar to last year.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/09/22/user_survey.md&lt;/span&gt;, line 82)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="how-can-dask-improve"&gt;
&lt;h1&gt;How can Dask improve?&lt;/h1&gt;
&lt;p&gt;Respondents continue to say that more documentation and examples would be the most valuable improvements to the project.&lt;/p&gt;
&lt;p&gt;One interesting change comes from looking at “Which would help you most right now?” split by API group (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt;, etc.). Last year showed that “More examples” in my field was the most important for all API groups (first table below). But in 2020 there are some differences (second table below).&lt;/p&gt;
&lt;style  type="text/css" &gt;
    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row0_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row0_col1 {
            background-color:  #cacee5;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row0_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row0_col3 {
            background-color:  #f1ebf4;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row0_col4 {
            background-color:  #c4cbe3;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row1_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row1_col1 {
            background-color:  #3b92c1;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row1_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row1_col3 {
            background-color:  #62a2cb;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row1_col4 {
            background-color:  #bdc8e1;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row2_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row2_col1 {
            background-color:  #c2cbe2;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row2_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row2_col3 {
            background-color:  #94b6d7;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row2_col4 {
            background-color:  #e0dded;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row3_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row3_col1 {
            background-color:  #e6e2ef;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row3_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row3_col3 {
            background-color:  #ced0e6;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row3_col4 {
            background-color:  #c5cce3;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row4_col0 {
            background-color:  #dedcec;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row4_col1 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row4_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row4_col3 {
            background-color:  #1c7fb8;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row4_col4 {
            background-color:  #73a9cf;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row5_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row5_col1 {
            background-color:  #b4c4df;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row5_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row5_col3 {
            background-color:  #b4c4df;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row5_col4 {
            background-color:  #eee9f3;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row6_col0 {
            background-color:  #faf2f8;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row6_col1 {
            background-color:  #e7e3f0;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row6_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row6_col3 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8701b8_e96b_11ea_9e95_186590cd1c87row6_col4 {
            background-color:  #f4eef6;
            color:  #000000;
        }&lt;/style&gt;&lt;table id="T_0a8701b8_e96b_11ea_9e95_186590cd1c87" &gt;&lt;caption&gt;2019 normalized by row. Darker means that a higher proporiton of users of that API prefer that priority.&lt;/caption&gt;&lt;thead&gt;    &lt;tr&gt;        &lt;th class="index_name level0" &gt;Which would help you most right now?&lt;/th&gt;        &lt;th class="col_heading level0 col0" &gt;Bug fixes&lt;/th&gt;        &lt;th class="col_heading level0 col1" &gt;More documentation&lt;/th&gt;        &lt;th class="col_heading level0 col2" &gt;More examples in my field&lt;/th&gt;        &lt;th class="col_heading level0 col3" &gt;New features&lt;/th&gt;        &lt;th class="col_heading level0 col4" &gt;Performance improvements&lt;/th&gt;    &lt;/tr&gt;    &lt;tr&gt;        &lt;th class="index_name level0" &gt;Dask APIs&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;    &lt;/tr&gt;&lt;/thead&gt;&lt;tbody&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;            &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87level0_row0&amp;quot; class=&amp;quot;row_heading level0 row0&amp;quot; &amp;gt;Array&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row0_col0&amp;quot; class=&amp;quot;data row0 col0&amp;quot; &amp;gt;10&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row0_col1&amp;quot; class=&amp;quot;data row0 col1&amp;quot; &amp;gt;24&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row0_col2&amp;quot; class=&amp;quot;data row0 col2&amp;quot; &amp;gt;62&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row0_col3&amp;quot; class=&amp;quot;data row0 col3&amp;quot; &amp;gt;15&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row0_col4&amp;quot; class=&amp;quot;data row0 col4&amp;quot; &amp;gt;25&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87level0_row1&amp;quot; class=&amp;quot;row_heading level0 row1&amp;quot; &amp;gt;Bag&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row1_col0&amp;quot; class=&amp;quot;data row1 col0&amp;quot; &amp;gt;3&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row1_col1&amp;quot; class=&amp;quot;data row1 col1&amp;quot; &amp;gt;11&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row1_col2&amp;quot; class=&amp;quot;data row1 col2&amp;quot; &amp;gt;16&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row1_col3&amp;quot; class=&amp;quot;data row1 col3&amp;quot; &amp;gt;10&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row1_col4&amp;quot; class=&amp;quot;data row1 col4&amp;quot; &amp;gt;7&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87level0_row2&amp;quot; class=&amp;quot;row_heading level0 row2&amp;quot; &amp;gt;DataFrame&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row2_col0&amp;quot; class=&amp;quot;data row2 col0&amp;quot; &amp;gt;16&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row2_col1&amp;quot; class=&amp;quot;data row2 col1&amp;quot; &amp;gt;32&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row2_col2&amp;quot; class=&amp;quot;data row2 col2&amp;quot; &amp;gt;71&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row2_col3&amp;quot; class=&amp;quot;data row2 col3&amp;quot; &amp;gt;39&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row2_col4&amp;quot; class=&amp;quot;data row2 col4&amp;quot; &amp;gt;26&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87level0_row3&amp;quot; class=&amp;quot;row_heading level0 row3&amp;quot; &amp;gt;Delayed&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row3_col0&amp;quot; class=&amp;quot;data row3 col0&amp;quot; &amp;gt;16&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row3_col1&amp;quot; class=&amp;quot;data row3 col1&amp;quot; &amp;gt;22&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row3_col2&amp;quot; class=&amp;quot;data row3 col2&amp;quot; &amp;gt;55&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row3_col3&amp;quot; class=&amp;quot;data row3 col3&amp;quot; &amp;gt;26&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row3_col4&amp;quot; class=&amp;quot;data row3 col4&amp;quot; &amp;gt;27&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87level0_row4&amp;quot; class=&amp;quot;row_heading level0 row4&amp;quot; &amp;gt;Futures&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row4_col0&amp;quot; class=&amp;quot;data row4 col0&amp;quot; &amp;gt;12&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row4_col1&amp;quot; class=&amp;quot;data row4 col1&amp;quot; &amp;gt;9&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row4_col2&amp;quot; class=&amp;quot;data row4 col2&amp;quot; &amp;gt;25&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row4_col3&amp;quot; class=&amp;quot;data row4 col3&amp;quot; &amp;gt;20&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row4_col4&amp;quot; class=&amp;quot;data row4 col4&amp;quot; &amp;gt;17&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87level0_row5&amp;quot; class=&amp;quot;row_heading level0 row5&amp;quot; &amp;gt;ML&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row5_col0&amp;quot; class=&amp;quot;data row5 col0&amp;quot; &amp;gt;5&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row5_col1&amp;quot; class=&amp;quot;data row5 col1&amp;quot; &amp;gt;11&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row5_col2&amp;quot; class=&amp;quot;data row5 col2&amp;quot; &amp;gt;23&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row5_col3&amp;quot; class=&amp;quot;data row5 col3&amp;quot; &amp;gt;11&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row5_col4&amp;quot; class=&amp;quot;data row5 col4&amp;quot; &amp;gt;7&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87level0_row6&amp;quot; class=&amp;quot;row_heading level0 row6&amp;quot; &amp;gt;Xarray&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row6_col0&amp;quot; class=&amp;quot;data row6 col0&amp;quot; &amp;gt;8&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row6_col1&amp;quot; class=&amp;quot;data row6 col1&amp;quot; &amp;gt;11&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row6_col2&amp;quot; class=&amp;quot;data row6 col2&amp;quot; &amp;gt;34&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row6_col3&amp;quot; class=&amp;quot;data row6 col3&amp;quot; &amp;gt;7&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8701b8_e96b_11ea_9e95_186590cd1c87row6_col4&amp;quot; class=&amp;quot;data row6 col4&amp;quot; &amp;gt;9&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
&amp;lt;/tbody&amp;gt;&amp;lt;/table&amp;gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;style  type="text/css" &gt;
    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row0_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row0_col1 {
            background-color:  #f1ebf5;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row0_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row0_col3 {
            background-color:  #f5eef6;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row0_col4 {
            background-color:  #d0d1e6;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row1_col0 {
            background-color:  #f0eaf4;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row1_col1 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row1_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row1_col3 {
            background-color:  #f0eaf4;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row1_col4 {
            background-color:  #4c99c5;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row2_col0 {
            background-color:  #f5eff6;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row2_col1 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row2_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row2_col3 {
            background-color:  #fcf4fa;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row2_col4 {
            background-color:  #8eb3d5;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row3_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row3_col1 {
            background-color:  #ebe6f2;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row3_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row3_col3 {
            background-color:  #f5eff6;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row3_col4 {
            background-color:  #3d93c2;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row4_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row4_col1 {
            background-color:  #f5eef6;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row4_col2 {
            background-color:  #0567a2;
            color:  #f1f1f1;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row4_col3 {
            background-color:  #cacee5;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row4_col4 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row5_col0 {
            background-color:  #ede8f3;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row5_col1 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row5_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row5_col3 {
            background-color:  #c1cae2;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row5_col4 {
            background-color:  #80aed2;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row6_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row6_col1 {
            background-color:  #f8f1f8;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row6_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row6_col3 {
            background-color:  #c9cee4;
            color:  #000000;
        }    #T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row6_col4 {
            background-color:  #86b0d3;
            color:  #000000;
        }&lt;/style&gt;&lt;table id="T_0a8d3eac_e96b_11ea_9e95_186590cd1c87" &gt;&lt;caption&gt;2020 normalized by row. Darker means that a higher proporiton of users of that API prefer that priority.&lt;/caption&gt;&lt;thead&gt;    &lt;tr&gt;        &lt;th class="index_name level0" &gt;Which would help you most right now?&lt;/th&gt;        &lt;th class="col_heading level0 col0" &gt;Bug fixes&lt;/th&gt;        &lt;th class="col_heading level0 col1" &gt;More documentation&lt;/th&gt;        &lt;th class="col_heading level0 col2" &gt;More examples in my field&lt;/th&gt;        &lt;th class="col_heading level0 col3" &gt;New features&lt;/th&gt;        &lt;th class="col_heading level0 col4" &gt;Performance improvements&lt;/th&gt;    &lt;/tr&gt;    &lt;tr&gt;        &lt;th class="index_name level0" &gt;Dask APIs&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;    &lt;/tr&gt;&lt;/thead&gt;&lt;tbody&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;            &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87level0_row0&amp;quot; class=&amp;quot;row_heading level0 row0&amp;quot; &amp;gt;Array&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row0_col0&amp;quot; class=&amp;quot;data row0 col0&amp;quot; &amp;gt;12&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row0_col1&amp;quot; class=&amp;quot;data row0 col1&amp;quot; &amp;gt;16&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row0_col2&amp;quot; class=&amp;quot;data row0 col2&amp;quot; &amp;gt;56&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row0_col3&amp;quot; class=&amp;quot;data row0 col3&amp;quot; &amp;gt;15&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row0_col4&amp;quot; class=&amp;quot;data row0 col4&amp;quot; &amp;gt;23&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87level0_row1&amp;quot; class=&amp;quot;row_heading level0 row1&amp;quot; &amp;gt;Bag&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row1_col0&amp;quot; class=&amp;quot;data row1 col0&amp;quot; &amp;gt;7&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row1_col1&amp;quot; class=&amp;quot;data row1 col1&amp;quot; &amp;gt;5&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row1_col2&amp;quot; class=&amp;quot;data row1 col2&amp;quot; &amp;gt;24&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row1_col3&amp;quot; class=&amp;quot;data row1 col3&amp;quot; &amp;gt;7&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row1_col4&amp;quot; class=&amp;quot;data row1 col4&amp;quot; &amp;gt;16&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87level0_row2&amp;quot; class=&amp;quot;row_heading level0 row2&amp;quot; &amp;gt;DataFrame&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row2_col0&amp;quot; class=&amp;quot;data row2 col0&amp;quot; &amp;gt;24&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row2_col1&amp;quot; class=&amp;quot;data row2 col1&amp;quot; &amp;gt;21&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row2_col2&amp;quot; class=&amp;quot;data row2 col2&amp;quot; &amp;gt;67&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row2_col3&amp;quot; class=&amp;quot;data row2 col3&amp;quot; &amp;gt;22&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row2_col4&amp;quot; class=&amp;quot;data row2 col4&amp;quot; &amp;gt;41&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87level0_row3&amp;quot; class=&amp;quot;row_heading level0 row3&amp;quot; &amp;gt;Delayed&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row3_col0&amp;quot; class=&amp;quot;data row3 col0&amp;quot; &amp;gt;15&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row3_col1&amp;quot; class=&amp;quot;data row3 col1&amp;quot; &amp;gt;19&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row3_col2&amp;quot; class=&amp;quot;data row3 col2&amp;quot; &amp;gt;46&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row3_col3&amp;quot; class=&amp;quot;data row3 col3&amp;quot; &amp;gt;17&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row3_col4&amp;quot; class=&amp;quot;data row3 col4&amp;quot; &amp;gt;34&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87level0_row4&amp;quot; class=&amp;quot;row_heading level0 row4&amp;quot; &amp;gt;Futures&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row4_col0&amp;quot; class=&amp;quot;data row4 col0&amp;quot; &amp;gt;9&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row4_col1&amp;quot; class=&amp;quot;data row4 col1&amp;quot; &amp;gt;10&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row4_col2&amp;quot; class=&amp;quot;data row4 col2&amp;quot; &amp;gt;21&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row4_col3&amp;quot; class=&amp;quot;data row4 col3&amp;quot; &amp;gt;13&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row4_col4&amp;quot; class=&amp;quot;data row4 col4&amp;quot; &amp;gt;24&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87level0_row5&amp;quot; class=&amp;quot;row_heading level0 row5&amp;quot; &amp;gt;ML&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row5_col0&amp;quot; class=&amp;quot;data row5 col0&amp;quot; &amp;gt;6&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row5_col1&amp;quot; class=&amp;quot;data row5 col1&amp;quot; &amp;gt;4&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row5_col2&amp;quot; class=&amp;quot;data row5 col2&amp;quot; &amp;gt;21&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row5_col3&amp;quot; class=&amp;quot;data row5 col3&amp;quot; &amp;gt;9&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row5_col4&amp;quot; class=&amp;quot;data row5 col4&amp;quot; &amp;gt;12&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87level0_row6&amp;quot; class=&amp;quot;row_heading level0 row6&amp;quot; &amp;gt;Xarray&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row6_col0&amp;quot; class=&amp;quot;data row6 col0&amp;quot; &amp;gt;3&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row6_col1&amp;quot; class=&amp;quot;data row6 col1&amp;quot; &amp;gt;4&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row6_col2&amp;quot; class=&amp;quot;data row6 col2&amp;quot; &amp;gt;25&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row6_col3&amp;quot; class=&amp;quot;data row6 col3&amp;quot; &amp;gt;9&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_0a8d3eac_e96b_11ea_9e95_186590cd1c87row6_col4&amp;quot; class=&amp;quot;data row6 col4&amp;quot; &amp;gt;13&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
&amp;lt;/tbody&amp;gt;&amp;lt;/table&amp;gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Examples are again the most important (for all API groups except &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Futures&lt;/span&gt;&lt;/code&gt;). But “Performance improvements” is now the second-most important improvement (except for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Futures&lt;/span&gt;&lt;/code&gt; where it’s most important). How should we interpret this? A charitable interpretation is that Dask’s users are scaling to larger problems and are running into new scaling challenges. A less charitable interpretation is that our user’s workflows are the same but Dask is getting slower!&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/09/22/user_survey.md&lt;/span&gt;, line 422)&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-other-systems-do-you-use"&gt;
&lt;h1&gt;What other systems do you use?&lt;/h1&gt;
&lt;p&gt;SSH continues to be the most popular “cluster resource mananger”. This was the big surprise last year, so we put in some work to make it nicer. Aside from that, not much has changed.&lt;/p&gt;
&lt;img src="/images/2020_survey/2020_25_0.png"&gt;
&lt;p&gt;And Dask users are about as happy with its stability as last year.&lt;/p&gt;
&lt;img src="/images/2020_survey/2020_27_0.png"&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/09/22/user_survey.md&lt;/span&gt;, line 432)&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="takeaways"&gt;
&lt;h1&gt;Takeaways&lt;/h1&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Overall, most things are similar to last year.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Documentation, especially domain-specific examples, continues to be important. That said, our documentation is probably better than it was last year.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;More users are pushing Dask further. Investing in performance is likely to be valuable.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Thanks again to all the respondents!&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2020/09/22/user_survey/"/>
    <summary>This post presents the results of the 2020 Dask User Survey,
which ran earlier this summer. Thanks to everyone who took the time to fill out the survey!
These results help us better understand the Dask community and will guide future development efforts.</summary>
    <category term="UserSurvey" label="User Survey"/>
    <published>2020-09-22T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2020/08/31/helm_daskhub/</id>
    <title>Announcing the DaskHub Helm Chart</title>
    <updated>2020-08-31T00:00:00+00:00</updated>
    <author>
      <name>Tom Augspurger</name>
    </author>
    <content type="html">&lt;p&gt;Today we’re announcing the release of the
&lt;a class="reference external" href="https://github.com/dask/helm-chart/blob/master/daskhub/README.md"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;daskhub&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;
helm chart. This is a &lt;a class="reference external" href="https://helm.sh/"&gt;Helm&lt;/a&gt; chart to easily install
&lt;a class="reference external" href="https://jupyter.org/hub"&gt;JupyterHub&lt;/a&gt; and Dask for multiple users on a
Kubernetes Cluster. If you’re managing deployment for many people that needs
interactive, scalable computing (say for a class of students, a data science
team, or a research lab) then &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask/daskhub&lt;/span&gt;&lt;/code&gt; might be right for you.&lt;/p&gt;
&lt;p&gt;You can install &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask/daskhub&lt;/span&gt;&lt;/code&gt; on a Kubernetes cluster today with&lt;/p&gt;
&lt;div class="highlight-console notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="go"&gt;helm repo add dask https://helm.dask.org/&lt;/span&gt;
&lt;span class="go"&gt;helm repo update&lt;/span&gt;
&lt;span class="go"&gt;helm upgrade --install dhub dask/daskhub&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/2020/08/31/helm_daskhub.md&lt;/span&gt;, line 26)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="history"&gt;

&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask/daskhub&lt;/span&gt;&lt;/code&gt; helm chart is an evolution of the &lt;a class="reference external" href="http://pangeo.io/"&gt;Pangeo&lt;/a&gt;
helm chart, which came out of that community’s attempts to do big data
geoscience on the cloud. We’re very grateful to have years of experience using
Dask and JupyterHub together. Pangeo was always aware that there wasn’t anything
geoscience-specific to their Helm chart and so were eager to contribute it to
Dask to share the maintenance burden. In the process of moving it over to Dask’s
chart repository we took the opportunity to clean up a few rough edges.&lt;/p&gt;
&lt;p&gt;It’s interesting to read the &lt;a class="reference external" href="https://blog.dask.org/2018/01/22/pangeo-2"&gt;original
announcement&lt;/a&gt; of Pangeo’s JupyterHub
deployment. A lot has improved, and we hope that this helm chart assists more
groups in deploying JupyterHubs capable of scalable computations with 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/2020/08/31/helm_daskhub.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="details"&gt;
&lt;h1&gt;Details&lt;/h1&gt;
&lt;p&gt;Internally, the DaskHub helm chart is relatively simple combination of the
&lt;a class="reference external" href="https://github.com/jupyterhub/zero-to-jupyterhub-k8s"&gt;JupyterHub&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/dask/dask-gateway/"&gt;Dask
Gateway&lt;/a&gt; helm charts. The only additional
magic is some configuration to&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Register Dask Gateway as a &lt;a class="reference external" href="https://jupyterhub.readthedocs.io/en/stable/reference/services.html"&gt;JupyterHub
service&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Set environment variables to make using Dask Gateway easy for your users.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;With the default configuration, your users will be able to create and connect to
Dask Clusters, including their dashboards, with a simple&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_gateway&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;GatewayCluster&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;GatewayCluster&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;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Check out the
&lt;a class="reference external" href="https://docs.dask.org/en/latest/setup/kubernetes-helm.html"&gt;documentation&lt;/a&gt; for
details and let us know if you run into any difficulties.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2020/08/31/helm_daskhub/"/>
    <summary>Today we’re announcing the release of the
daskhub
helm chart. This is a Helm chart to easily install
JupyterHub and Dask for multiple users on a
Kubernetes Cluster. If you’re managing deployment for many people that needs
interactive, scalable computing (say for a class of students, a data science
team, or a research lab) then dask/daskhub might be right for you.</summary>
    <category term="DaskGateway" label="Dask Gateway"/>
    <category term="Deployment" label="Deployment"/>
    <category term="Helm" label="Helm"/>
    <published>2020-08-31T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2020/08/21/running-tutorials/</id>
    <title>Running tutorials</title>
    <updated>2020-08-21T00:00:00+00:00</updated>
    <author>
      <name>Jacob Tomlinson (NVIDIA)</name>
    </author>
    <content type="html">&lt;p&gt;For the last couple of months we’ve been running community tutorials every three weeks or so. The response from the community has been great and we’ve had 50-100 people at each 90 minute session.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/08/21/running-tutorials.md&lt;/span&gt;, line 12)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="why-should-open-source-projects-run-tutorials"&gt;

&lt;p&gt;The Dask team has historically run tutorials at conferences such as SciPy. With 2020 turning out the way that it has much of this content is being presented virtually this year. As more people are becoming accustomed to participating in virtual tutorials we felt it would be a good service to our community to start running regular virtual tutorials independent of conferences we may be attending or speaking at.&lt;/p&gt;
&lt;p&gt;Tutorials are great for open source projects as they appeal to multiple types of learner.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The tutorial material provides a great foundation for &lt;em&gt;written and visual learners&lt;/em&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Using an interactive tool like Jupyter Notebooks allows &lt;em&gt;kinesthetic learners&lt;/em&gt; to follow along and take their own paths.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Having an instructor run through the material in real time provides a spoken source for &lt;em&gt;auditory learners&lt;/em&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It’s also just fun to have a bunch of people from around the world participate in a live event. There is a greater sense of community.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Many open source projects provide documentation, some also make instructional videos on YouTube, but you really can’t beat a tutorial for producing a single set of content that is valuable to many users.&lt;/p&gt;
&lt;p&gt;The more users can share knowledge, information and skills with the more they are going to use and engage with the project. Having a great source of learning material is critical for converting interested newcomers to users and users to contributors.&lt;/p&gt;
&lt;p&gt;It is also great for the maintainers too. Dask is a large project made up of many open source repositories all with different functions. Each maintainer tends to participate in their specialist areas, but do not engage with everything on a day-to-day basis. Having maintainers run tutorials encourages them to increase their knowledge of areas they rarely touch in order to deliver the material, and this benefits the project as a whole.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/08/21/running-tutorials.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="how"&gt;
&lt;h1&gt;How&lt;/h1&gt;
&lt;p&gt;For the rest of this post we will discuss the preparation and logistics we have undertaken to provide our tutorials. Hopefully this will provide a blueprint for others waning to run similar activities.&lt;/p&gt;
&lt;section id="writing-the-material"&gt;
&lt;h2&gt;Writing the material&lt;/h2&gt;
&lt;p&gt;When starting to compile material is it important to consider a few questions; “Who is this for?”, “How long should it be?” and “What already exists today?”.&lt;/p&gt;
&lt;p&gt;For the Dask tutorial we were targeting users who were either new to Dask, or had been using it for a while but wanted to learn more about the wider project. Dask is a large project after all and there are many features that you may not discover when trying to solve your specific challenges with it.&lt;/p&gt;
&lt;p&gt;At large conferences is it quite normal to run a three hour tutorial, however when trying to schedule a tutorial as part of a person’s normal working day that is probably too much to ask of them. Folks are accustomed to scheduling in work meetings that are typically 30-60 minutes, but that may not be enough to run a tutorial. So we settled on 90 minutes, enough to get through a good amount of content, but not too long that folks will be put off.&lt;/p&gt;
&lt;p&gt;We already have an &lt;a class="reference external" href="https://github.com/dask/dask-tutorial"&gt;“official” tutorial&lt;/a&gt; which is designed to fill the three hours of a SciPy tutorial. This tutorial is also designed as a “Dask from first principals” style tutorial where we explore how Dask works and eventually scale up to how Dask implements familiar APIs like Numpy and Pandas. This is great for giving folks a thorough understanding of Dask but given that we decided on 90 minutes we may not want to start with low level code as we may run out of time before getting to general usage.&lt;/p&gt;
&lt;p&gt;While researching what already exists I was pointed to the &lt;a class="reference external" href="https://github.com/adbreind/dask-mini-2019"&gt;Mini Dask 2019 tutorial&lt;/a&gt; which was created for an &lt;a class="reference external" href="https://www.oreilly.com/live-training/courses/scale-your-python-processing-with-dask/0636920319573/"&gt;O’Reilly event&lt;/a&gt;. This tutorial starts with familiar APIs such as dataframes and arrays and eventually digs down into Dask fundamentals. As tutorial content like this is often licensed as open source and made available on GitHub it’s great to be able to build upon the work of others.&lt;/p&gt;
&lt;p&gt;The result of combining the two tutorials was the &lt;a class="reference external" href="https://github.com/jacobtomlinson/dask-video-tutorial-2020"&gt;Dask Video Tutorial 2020&lt;/a&gt;. It follows the same structure as the mini tutorial starting with high level APIs and digging further down. It also includes some new content on deployment and distributed methods.&lt;/p&gt;
&lt;section id="structuring-content"&gt;
&lt;h3&gt;Structuring content&lt;/h3&gt;
&lt;p&gt;To ensure this content targets the different learner types that we discussed earlier we need to ensure our content has a few things.&lt;/p&gt;
&lt;p&gt;As a foundation we should put together a series of pages/documents with a written version of the information we are trying to communicate for &lt;em&gt;written learners&lt;/em&gt;. We should also endeavor to include diagrams and pictures to illustrate this information for &lt;em&gt;visual learners&lt;/em&gt;.&lt;/p&gt;
&lt;p&gt;As we are sharing knowledge on an open source software project we should also make things as interactive as possible. Using Jupyter Notebooks as our document format means we can include many code examples which both provide written examples but are also editable and executable to empower &lt;em&gt;kinesthetic learners&lt;/em&gt; to feel how things work in practice.&lt;/p&gt;
&lt;p&gt;When the content is being delivered the instructor will be running through the content at the same time and narrating what they are doing for &lt;em&gt;auditory learners&lt;/em&gt;. It is important to try and structure things in a way where you explain each section of the content out loud, but without directly reading the text from the screen as that can be off-putting.&lt;/p&gt;
&lt;p&gt;We also want to ensure folks are taking things in, and labs are a great way to include small tests in the content. Having a section at the end of an example which is incomplete means that you can give the audience some time to try and figure things our for themselves. Some folks will be able to fill things in with no problems. For others they will hit errors or make mistakes, this is good for teaching how to debug and troubleshoot your project. And for those who are having awful flashbacks to pop-quizzes they can simply skip it without worrying that someone will check up on them.&lt;/p&gt;
&lt;p&gt;For each section of content you want to include in your tutorial I recommend you create a notebook with an explanation, an example and some things for the audience to figure out. Doing this for each section (in the Dask tutorial we had 9 sections) the audience will quickly become familiar with the process and be able to anticipate what is coming next. This will make them feel comfortable.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="hosting-the-material"&gt;
&lt;h2&gt;Hosting the material&lt;/h2&gt;
&lt;p&gt;Once you have put your material together you need to share it with your attendees.&lt;/p&gt;
&lt;p&gt;GitHub is a great place to put things, especially if you include an open license with it. For narrative tutorial content a creative commons license if often used which requires modifications to also be shared.&lt;/p&gt;
&lt;p&gt;As we have put our content together as Jupyter Notebooks we can use &lt;a class="reference external" href="https://mybinder.org/"&gt;Binder&lt;/a&gt; to make it possible for folks to run the material without having to download it locally or ensure their Python environment is set up correctly.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="choosing-a-video-platform"&gt;
&lt;h2&gt;Choosing a video platform&lt;/h2&gt;
&lt;p&gt;Next we have to decide how we will present the material. As this is a virtual tutorial we will want to use some kind of video conferences or streaming software.&lt;/p&gt;
&lt;p&gt;These tools tend to fall into two categories; private meetings with a tool like Zoom, Hangouts or Teams and public broadcasts on websites like YouTube or Twitch.&lt;/p&gt;
&lt;p&gt;Any of these options will likely be a good choice, they allow the presenter to share their video, audio and screen with participants and participants can communicate back with a range of tools.&lt;/p&gt;
&lt;p&gt;The main decision you will have to make is around whether you want to restrict numbers or not. The more interactivity you want to have in the tutorial the more you will need a handle on numbers. For our initial tutorials we wanted to enable participants to ask questions at any time and get a quick response, so we opted to use Zoom and limit our numbers to allow us to not get overwhelmed with questions. However if you want to present to as many people as possible and accept that you may not be able to address them all individually you may want to use a streaming platform instead.&lt;/p&gt;
&lt;p&gt;It is also possible to do both at the same time. Zoom can stream directly to YouTube for example. This can be useful if you want to open things to as many folks as possible, but also limit the interactivity to a select group (probably on a first-come-first-served basis). For the Dask tutorials we decided to not livestream and instead run multiple tutorials so that everyone gets an interactive experience, but we are fortunate to have the resources to do that.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="registering-attendees"&gt;
&lt;h2&gt;Registering attendees&lt;/h2&gt;
&lt;p&gt;There are a couple of reasons why you may wish to register attendees ahead of time.&lt;/p&gt;
&lt;p&gt;If you want to limit numbers you will certainly need some way to register people and put a cap on that number. But even if you are streaming generally you may want to get folks to register ahead of time as that allows you to send them reminder emails in the run up to the event, which likely will add more certainty to the attendance numbers.&lt;/p&gt;
&lt;p&gt;As our event was private we registered folks with &lt;a class="reference external" href="https://www.eventbrite.com/"&gt;Eventbrite&lt;/a&gt;. This allowed us to cap numbers and also schedule automated emails to act as a reminder but also share the details of the private Zoom meeting.&lt;/p&gt;
&lt;p&gt;When running the Dask tutorials we found about 50% of the folks who registered actually turned up, so we accounted for this an set out limit to around double the number we wanted.&lt;/p&gt;
&lt;p&gt;Here’s an example of the event details what we created:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;&lt;strong&gt;Event Title&lt;/strong&gt;: Dask Tutorial&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Organizer&lt;/strong&gt;: Presenter’s name&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Event Type&lt;/strong&gt;: Seminar or talk, Science and Technology, Online event&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Tags&lt;/strong&gt;: dask, pydata, python, tutorial&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Location&lt;/strong&gt;: Online event&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Date and time&lt;/strong&gt;: Single Event, add times&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Details&lt;/strong&gt;:&lt;/p&gt;
&lt;p&gt;Come learn about Dask at this online free tutorial provided by the Dask maintainers.&lt;/p&gt;
&lt;p&gt;This ninety minute course will mix overview discussion and demonstration by a leader in the Dask community, as well as interactive exercises in live notebook sessions for attendees. The computing environment will be provided.&lt;/p&gt;
&lt;p&gt;If you want to get a sample of similar content, take a look at https://tutorial.dask.org (although this tutorial will cover different material appropriate for this shorter session).&lt;/p&gt;
&lt;p&gt;We look forward to seeing you there!&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Image&lt;/strong&gt;: https://i.imgur.com/2i1tMNG.png&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Live video content&lt;/strong&gt;: NA&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Text and media&lt;/strong&gt;: NA&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Links to resources&lt;/strong&gt;:
Tutorial Content (Online Jupyter Notebooks)
https://github.com/jacobtomlinson/dask-video-tutorial-2020&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Ticket Cost&lt;/strong&gt;: Free&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Ticket Attendee limit&lt;/strong&gt;: 150 people&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;/section&gt;
&lt;section id="count-down-to-the-tutorial"&gt;
&lt;h2&gt;Count down to the tutorial&lt;/h2&gt;
&lt;p&gt;We also set up a series of automated emails. You can find this under &lt;strong&gt;Manage Attendees &amp;gt; Emails to Attendees&lt;/strong&gt; in the event management page.&lt;/p&gt;
&lt;p&gt;We scheduled emails for two days before, two hours before and 10 minutes before to let folks know where to go and another a few hours after to gather feedback. &lt;em&gt;We will discuss the feedback email shortly&lt;/em&gt;.&lt;/p&gt;
&lt;p&gt;You’ll need to ensure you have links to the materials and meeting location ready for this. In our case we pushed the content to GitHub and scheduled the Zoom call ahead of time.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Two days and two hours before&lt;/strong&gt;&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Hi Everyone!&lt;/p&gt;
&lt;p&gt;We look forward to seeing you &amp;lt;tomorrow|soon&amp;gt;. We wanted to share some important links with you to help you connect to the meeting.&lt;/p&gt;
&lt;p&gt;The materials for the course are available on GitHub here at the link below:&lt;/p&gt;
&lt;p&gt;&amp;lt;Link to materials&amp;gt;&lt;/p&gt;
&lt;p&gt;This repository contains Jupyter notebooks that we’ll go through together as a group. You do not need to install anything before the tutorial. We will run the notebooks on the online service, mybinder.org . All you need is a web connection.&lt;/p&gt;
&lt;p&gt;The meeting itself will be held by video call at the following Zoom link:&lt;/p&gt;
&lt;p&gt;&amp;lt;Zoom link and pin&amp;gt;&lt;/p&gt;
&lt;p&gt;We look forward to seeing you soon!&lt;/p&gt;
&lt;p&gt;&amp;lt;Organisers names&amp;gt;&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;&lt;strong&gt;Ten minutes before&lt;/strong&gt;&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Hi Everyone!&lt;/p&gt;
&lt;p&gt;We are about to get started. Here’s a final reminder of the meeting details.&lt;/p&gt;
&lt;p&gt;&amp;lt;Zoom link and pin&amp;gt;&lt;/p&gt;
&lt;p&gt;See you in a minute!&lt;/p&gt;
&lt;p&gt;&amp;lt;Organisers names&amp;gt;&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;&lt;strong&gt;Few hours after&lt;/strong&gt;&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Hi Everyone!&lt;/p&gt;
&lt;p&gt;Thank you so much for attending the Dask tutorial. We really hope you found it valuable.&lt;/p&gt;
&lt;p&gt;We would really appreciate it if you could answer a couple of quick feedback questions to help us improve things for next time.&lt;/p&gt;
&lt;p&gt;&amp;lt;Google form link &amp;gt;&lt;/p&gt;
&lt;p&gt;Also we want to remind you that the tutorial materials are always available on GitHub and you can run through them any time or share them with others.&lt;/p&gt;
&lt;p&gt;&amp;lt;Link to materials&amp;gt;&lt;/p&gt;
&lt;p&gt;Thanks,&lt;/p&gt;
&lt;p&gt;&amp;lt;Organisers names&amp;gt;&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;/section&gt;
&lt;section id="getting-the-word-out"&gt;
&lt;h2&gt;Getting the word out&lt;/h2&gt;
&lt;p&gt;Now that we have an Eventbrite page we need to tell people about it.&lt;/p&gt;
&lt;p&gt;You may already have existing channels where you can contact your community. For Dask we have an active twitter account with a good number of followers, so tweeting out the link to the event a couple of times the week running up to the tutorial was enough to fill the spaces.&lt;/p&gt;
&lt;p&gt;If you have a mailing list, or any other platform you will probably want to share it there.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="setting-up-the-call"&gt;
&lt;h2&gt;Setting up the call&lt;/h2&gt;
&lt;p&gt;Be sure to join the call ahead of the attendees. I would make sure this is at least before the final reminder email goes out. Personally I join 20 minutes or so before hand. This allows you to ensure the call is being recorded and that attendees were muted when they join.&lt;/p&gt;
&lt;p&gt;Consider the experience of the user’s here. They will have signed up for an event online, received a few emails with Zoom call details and then they will join the call. If there is no indication that they are in the right place within a few seconds they may become anxious.&lt;/p&gt;
&lt;p&gt;To combat this I tend to show some graphic which lets people know they are in the right place. You could either use a tool like &lt;a class="reference external" href="https://jacobtomlinson.dev/posts/2020/how-to-use-obs-studio-with-zoom-hangouts-teams-and-more-on-macos/"&gt;OBS with Zoom&lt;/a&gt; to create a custom scene or just share your screen with a simple slide saying something like “The Dask tutorial will start soon”.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;The only downside to sharing your screen is you can’t continue to use your computer in the run up to the tutorial.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;When we ran our first few tutorials we were also running our Dask user survey so also included a link to that on the waiting screen to give folks something to do.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="greeting-and-getting-folks-set-up"&gt;
&lt;h2&gt;Greeting and getting folks set up&lt;/h2&gt;
&lt;p&gt;Say hi on the hour and welcome everyone to the tutorial. But as the event is virtual folks will be late, so don’t kick off until around five minutes in, otherwise you’ll just get a flood of questions asking what’s going on.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="interactivity"&gt;
&lt;h2&gt;Interactivity&lt;/h2&gt;
&lt;p&gt;A fun thing to do during this waiting period is get everyone to introduce themselves in the chat. Say something like “Please say hi in that chat and give your name and where you are joining from”.&lt;/p&gt;
&lt;p&gt;This is nice feedback for you as the instructor to see where folks are joining from, but it also gives the attendees a sense of being in a room full of people. One of the benefits of an event like this is that it is interactive, so be sure to say hi back to people.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;I’m awful at pronouncing names correctly so I tend to list the places they said they are from instead. It still makes them feel like their message has been seen.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Once you’re ready to start introduce yourself and a general overview of the tutorial content. Then make use of any interaction tools you may have in your chat application. In zoom there are buttons that participants can click with labels like “go faster”, “go slower”, “yes” and “no”. These are great for getting feedback from the audience when running the tutorial, but it’s good to make sure everyone knows where they are and has a go at using them. I tend to explain where the buttons are and then ask questions like “have you managed to launch the binder?”, “have you used Dask before?” or “are you a Pandas user?”. You learn a little about your audience and they get familiar with the controls.&lt;/p&gt;
&lt;p&gt;Being interactive means you can also respond to user questions. In Dask tutorials we mute everyone by default and encourage folks to type in the text chat. We also have an additional instructor who is not delivering the material who is able to watch the chat and answer questions in real time. If they feel like a question/answer would be beneficial to the whole group they can unmute and interrupt the presenter in order to bubble it up. Be prepared for a wide range of questions from the chat, including topics that are not being actively covered in the tutorial. This is often the only time that attendees have real-time access to core maintainers.&lt;/p&gt;
&lt;p&gt;You may not have the resources to have two instructors for every tutorial, Dask is fortunate to have a strong maintainer team, so instead you may want to allocate breaks at the end of each section to answer questions. During the labs can be a good time to go back and review any questions.&lt;/p&gt;
&lt;p&gt;Interactivity is one of the big benefits a live tutorial has over a video.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="run-through-the-material"&gt;
&lt;h2&gt;Run through the material&lt;/h2&gt;
&lt;p&gt;Once you’re all set up and everyone is in it’s time to run through the material. Given the amount of preparation we did before hand to construct the material this is relatively straight forward. Everything is laid out in front of us and we just need to go through the motions of talking through it.&lt;/p&gt;
&lt;p&gt;I find it very helpful to have a list of the sections with timings written down that I can refer to in order to pace things.&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Overview of Dask with Dask Dataframe (10 mins)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Introductory Lab (10 mins) and results (5 mins)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask GUI and dashboards (10 mins)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask Array (10 mins)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask ML with lab (10 mins) and results (5 mins)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bags and Futures (10 mins)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Distributed (10 mins)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Wrapup and close (5 mins)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;As we have another instructor answering questions I tend to ignore the chat and run through each section as slowly as I can without going over time. Personally my default is to go too fast, so forcing myself to be slow but having some timings to keep me on track seems to work well. But you should do whatever works for you.&lt;/p&gt;
&lt;p&gt;During the labs I tend to mute my microphone and join in with answering questions on the chat.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="wrapping-things-up"&gt;
&lt;h2&gt;Wrapping things up&lt;/h2&gt;
&lt;p&gt;When you’re nearing the end it’s good to have some time for any final questions. People may want to ask things that they didn’t get a chance to earlier or have questions which haven’t fit in with any particular area.&lt;/p&gt;
&lt;p&gt;If you get complex questions or want to go in to depth you may want to offer to stay after and continue talking, but your attendees will appreciate you finishing at the scheduled time as they may have other things booked immediately after.&lt;/p&gt;
&lt;p&gt;It’s always good to leave folks with some extra resources, whether that is links to the documentation, community places they can learn more like a Gitter chat, etc.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="sharing-the-content-later"&gt;
&lt;h2&gt;Sharing the content later&lt;/h2&gt;
&lt;p&gt;Once you’re done it is also beneficial to upload a recording of the tutorial to YouTube. If you’ve livestreamed then this may happen automatically. If you used a tool like Zoom you’ll need to upload it to yourself.&lt;/p&gt;
&lt;p&gt;Anyone watching in the future won’t get the benefit of the interactivity, but should still be able to get much of the benefit from following through the material.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="gathering-feedback-and-planning-for-next-time"&gt;
&lt;h2&gt;Gathering feedback and planning for next time&lt;/h2&gt;
&lt;p&gt;The last thing for you to do is plan for next time. The Dask team have decided to run tutorials every month or so but rotate around timezones to try and cover as many users as possible. We’ve also discussed having special deep dive tutorials which follow the same length and format but dive into one topic in particular.&lt;/p&gt;
&lt;p&gt;To help you plan for future events you will likely want feedback from your participants. You can use tools like Google Forms to create a short questionnaire which you can send out to participants afterwards. In our experience about 20% of participants will fill in a survey that is 10 questions long.&lt;/p&gt;
&lt;p&gt;This feedback can be very helpful for making changes to the content or format. For example in our first tutorial we use OBS for both the intro screen and screen sharing throughout. However Zoom limits webcams to 720p and adds heavy compression, so the quality for participants was not good and 50% of the surveys mentioned poor video. In later tutorials we only used OBS for the intro screen and then used the built in screen sharing utility in Zoom which provided a better experience and no user reported any audio/video issues in the survey.&lt;/p&gt;
&lt;p&gt;Here are some examples of questions we asked and how they were answered for our tutorial.&lt;/p&gt;
&lt;section id="have-you-used-dask-before"&gt;
&lt;h3&gt;Have you used Dask before?&lt;/h3&gt;
&lt;p&gt;When writing our material we said we were “targeting users who were either new to Dask, or had been using it for a while but wanted to learn more about the wider project.”. Our feedback results confirm that we are hitting these groups.&lt;/p&gt;
&lt;p&gt;We could’ve been more specific and asked folks to rank their ability. But the more complex the questions the less likely folks will fill them out, so it’s a balancing act.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Forms response chart. Question title: Have you used Dask before? 39% no, 61% yes." src="https://i.imgur.com/T1loyeb.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="did-we-cover-all-the-topics-you-were-expecting-and-if-not-what-was-missing"&gt;
&lt;h3&gt;Did we cover all the topics you were expecting? And if not, what was missing?&lt;/h3&gt;
&lt;p&gt;Depending on the complexity of your project you may have to make compromises on what you can cover in the time you have. Dask is a large project and so we couldn’t cover everything, so we wanted to check we had covered the basics.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Forms response chart. Question title: Did we cover all the topics you expected? 22% no, 78% yes." src="https://i.imgur.com/la3dqrA.png" /&gt;&lt;/p&gt;
&lt;p&gt;Most of the feedback we had from folks who answered no were asking about advanced topics like Kubernetes, Google Cloud deployments, deep dives into internal workings, etc. I’m satisfied that this shouldn’t have been in this tutorial, but it adds weight to our plans to run deep dives in the future.&lt;/p&gt;
&lt;p&gt;Once useful bit of feedback we had here was “When should I use Dask and when should I stick with Pandas?”. This is something which definitely should be covered by an intro tutorial, so our material is clearly lacking here. As a result we can go back and make modifications and improve the content.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="how-was-the-pace"&gt;
&lt;h3&gt;How was the pace?&lt;/h3&gt;
&lt;p&gt;Setting the pace is hard. If you’re targeting a range of abilities then it’s easy to go too fast or slow for a big chunk of the attendees.&lt;/p&gt;
&lt;p&gt;Our feedback shows that folks were generally happy, but we are leaning on the side of being too fast. Given that we are filling our allocated time this probably indicates that we should cut a little content in order to slow things down.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Forms response chart. Question title: How was the pace? 70% Just right, 26% Too fast, 4% Too slow." src="https://i.imgur.com/mHPNmwp.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="which-sections-did-you-find-more-informative"&gt;
&lt;h3&gt;Which sections did you find more informative?&lt;/h3&gt;
&lt;p&gt;By asking what sections were most informative we can identify things to cut in future if we do need to slow things down. It also shows areas where we may want to spend more time and add more content.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Forms response chart. Question title: Which sections did you find more informative?" src="https://i.imgur.com/XLzSEw4.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="what-would-be-your-preferred-platform-for-a-tutorial-like-this"&gt;
&lt;h3&gt;What would be your preferred platform for a tutorial like this?&lt;/h3&gt;
&lt;p&gt;We had to make a decision on which video platform to use based on the criteria we discussed earlier. For our tutorials we chose Zoom. By doing a user survey we were able to check that this worked for people and also see if there is an alternative that folks prefer.&lt;/p&gt;
&lt;p&gt;Our results confirmed that folks were happy with Zoom. These results may be a little biased given that we used Zoom, but I’m confident that we can keep using it and folks will have a good experience.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Forms response chart. Question title: What would be your preferred platform for a tutorial like this? 70% Zoom, &amp;lt;5% for options including YouTube, Twitch, Jitsi, and No preference" src="https://i.imgur.com/fMxTZOK.png" /&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="would-you-recommend-the-tutorial-to-a-colleague"&gt;
&lt;h3&gt;Would you recommend the tutorial to a colleague?&lt;/h3&gt;
&lt;p&gt;The last thing to check is that folks had a good time. It gives you great pleasure as an instructor to see 100% of folks say they would recommend to a colleague.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;These results may be biased because if folks wouldn’t recommend it they probably wouldn’t bother to fill out a survey. But hey, I’ll take it!&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;img alt="Forms response chart. Question title: Would you recommend the tutorial to a colleague? 100% Yes." src="https://i.imgur.com/RzrXvfn.png" /&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/08/21/running-tutorials.md&lt;/span&gt;, line 318)&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&gt;
&lt;section id="wrap-up"&gt;
&lt;h1&gt;Wrap up&lt;/h1&gt;
&lt;p&gt;In this post we have covered why and how you can run community tutorials for open source projects.&lt;/p&gt;
&lt;p&gt;In summary you should run tutorials because:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;You can share knowledge with a range of people with different learning styles&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You can give back to your community&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You can grow your community&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You can improve maintainers knowledge of the whole project&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;And you can run a tutorial by following these steps:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Break your project into sections&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Write up interactive documents on each section with tools like Jupyter notebooks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Give people access to this content with services like Binder&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Manage attendees with services like Eventbrite&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Advertise your tutorial on social media&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Get everyone in a video meeting&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Make use of the interactive tools&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Deliver your material&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Gather feedback&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2020/08/21/running-tutorials/"/>
    <summary>For the last couple of months we’ve been running community tutorials every three weeks or so. The response from the community has been great and we’ve had 50-100 people at each 90 minute session.</summary>
    <category term="Community" label="Community"/>
    <category term="Tutorials" label="Tutorials"/>
    <published>2020-08-21T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2020/08/06/ray-tune/</id>
    <title>Comparing Dask-ML and Ray Tune's Model Selection Algorithms</title>
    <updated>2020-08-06T00:00:00+00:00</updated>
    <author>
      <name>&lt;a href="https://stsievert.com"&gt;Scott Sievert&lt;/a&gt; (University of Wisconsin–Madison)</name>
    </author>
    <content type="html">&lt;p&gt;Hyperparameter optimization is the process of deducing model parameters that
can’t be learned from data. This process is often time- and resource-consuming,
especially in the context of deep learning. A good description of this process
can be found at “&lt;a class="reference external" href="https://scikit-learn.org/stable/modules/grid_search.html"&gt;Tuning the hyper-parameters of an estimator&lt;/a&gt;,” and
the issues that arise are concisely summarized in Dask-ML’s documentation of
“&lt;a class="reference external" href="https://ml.dask.org/hyper-parameter-search.html"&gt;Hyper Parameter Searches&lt;/a&gt;.”&lt;/p&gt;
&lt;p&gt;There’s a host of libraries and frameworks out there to address this problem.
&lt;a class="reference external" href="https://scikit-learn.org/stable/modules/grid_search.html"&gt;Scikit-Learn’s module&lt;/a&gt; has been mirrored &lt;a class="reference external" href="https://ml.dask.org/hyper-parameter-search.html"&gt;in Dask-ML&lt;/a&gt; and
&lt;a class="reference external" href="https://automl.github.io/auto-sklearn/master/"&gt;auto-sklearn&lt;/a&gt;, both of which offer advanced hyperparameter optimization
techniques. Other implementations that don’t follow the Scikit-Learn interface
include &lt;a class="reference external" href="https://docs.ray.io/en/master/tune.html"&gt;Ray Tune&lt;/a&gt;, &lt;a class="reference external" href="https://www.automl.org/"&gt;AutoML&lt;/a&gt; and &lt;a class="reference external" href="https://medium.com/optuna/optuna-supports-hyperband-93b0cae1a137"&gt;Optuna&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://docs.ray.io"&gt;Ray&lt;/a&gt; recently provided a wrapper to &lt;a class="reference external" href="https://docs.ray.io/en/master/tune.html"&gt;Ray Tune&lt;/a&gt; that mirrors the Scikit-Learn
API called tune-sklearn (&lt;a class="reference external" href="https://docs.ray.io/en/master/tune/api_docs/sklearn.html"&gt;docs&lt;/a&gt;, &lt;a class="reference external" href="https://github.com/ray-project/tune-sklearn"&gt;source&lt;/a&gt;). &lt;a class="reference external" href="https://medium.com/distributed-computing-with-ray/gridsearchcv-2-0-new-and-improved-ee56644cbabf"&gt;The introduction&lt;/a&gt; of this library
states the following:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Cutting edge hyperparameter tuning techniques (Bayesian optimization, early
stopping, distributed execution) can provide significant speedups over grid
search and random search.&lt;/p&gt;
&lt;p&gt;However, the machine learning ecosystem is missing a solution that provides
users with the ability to leverage these new algorithms while allowing users
to stay within the Scikit-Learn API. In this blog post, we introduce
tune-sklearn [Ray’s tuning library] to bridge this gap. Tune-sklearn is a
drop-in replacement for Scikit-Learn’s model selection module with
state-of-the-art optimization features.&lt;/p&gt;
&lt;p&gt;—&lt;a class="reference external" href="https://medium.com/distributed-computing-with-ray/gridsearchcv-2-0-new-and-improved-ee56644cbabf"&gt;GridSearchCV 2.0 — New and Improved&lt;/a&gt;&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;This claim is inaccurate: for over a year Dask-ML has provided access to
“cutting edge hyperparameter tuning techniques” with a Scikit-Learn compatible
API. To correct their statement, let’s look at each of the features that Ray’s
tune-sklearn provides, and compare them to Dask-ML:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Here’s what [Ray’s] tune-sklearn has to offer:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Consistency with Scikit-Learn API&lt;/strong&gt; …&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Modern hyperparameter tuning techniques&lt;/strong&gt; …&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Framework support&lt;/strong&gt; …&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Scale up&lt;/strong&gt; … [to] multiple cores and even multiple machines.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;[Ray’s] Tune-sklearn is also &lt;strong&gt;fast&lt;/strong&gt;.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;Dask-ML’s model selection module has every one of the features:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Consistency with Scikit-Learn API:&lt;/strong&gt; Dask-ML’s model selection API
mirrors the Scikit-Learn model selection API.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Modern hyperparameter tuning techniques:&lt;/strong&gt; Dask-ML offers state-of-the-art
hyperparameter tuning techniques.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Framework support:&lt;/strong&gt; Dask-ML model selection supports many libraries
including Scikit-Learn, PyTorch, Keras, LightGBM and XGBoost.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Scale up:&lt;/strong&gt; Dask-ML supports distributed tuning (how could it not?) and
larger-than-memory datasets.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Dask-ML is also &lt;strong&gt;fast.&lt;/strong&gt; In “&lt;a class="reference internal" href="#speed"&gt;&lt;span class="xref myst"&gt;Speed&lt;/span&gt;&lt;/a&gt;” we show a benchmark between
Dask-ML, Ray and Scikit-Learn:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/2020-model-selection/n_workers=8.png" width="450px"
 /&gt;&lt;/p&gt;
&lt;p&gt;Only time-to-solution is relevant; all of these methods produce similar model
scores. See “&lt;a class="reference internal" href="#speed"&gt;&lt;span class="xref myst"&gt;Speed&lt;/span&gt;&lt;/a&gt;” for details.&lt;/p&gt;
&lt;p&gt;Now, let’s walk through the details on how to use Dask-ML to obtain the 5
features above.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/08/06/ray-tune.md&lt;/span&gt;, line 95)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="consistency-with-the-scikit-learn-api"&gt;

&lt;p&gt;&lt;em&gt;Dask-ML is consistent with the Scikit-Learn API.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Here’s how to use Scikit-Learn’s, Dask-ML’s and Ray’s tune-sklearn
hyperparameter optimization:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;## Trimmed example; see appendix for more detail&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;sklearn.model_selection&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;RandomizedSearchCV&lt;/span&gt;
&lt;span class="n"&gt;search&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;RandomizedSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&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;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&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_ml.model_selection&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;HyperbandSearchCV&lt;/span&gt;
&lt;span class="n"&gt;search&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;HyperbandSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&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;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;classes&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;

&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;tune_sklearn&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;TuneSearchCV&lt;/span&gt;
&lt;span class="n"&gt;search&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;TuneSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&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;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;classes&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The definitions of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;model&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;params&lt;/span&gt;&lt;/code&gt; follow the normal Scikit-Learn
definitions as detailed in the &lt;a class="reference internal" href="#full-example-usage"&gt;&lt;span class="xref myst"&gt;appendix&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Clearly, both Dask-ML and Ray’s tune-sklearn are Scikit-Learn compatible. Now
let’s focus on how each search performs and how it’s configured.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/08/06/ray-tune.md&lt;/span&gt;, line 126)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="modern-hyperparameter-tuning-techniques"&gt;
&lt;h1&gt;Modern hyperparameter tuning techniques&lt;/h1&gt;
&lt;p&gt;&lt;em&gt;Dask-ML offers state-of-the-art hyperparameter tuning techniques
in a Scikit-Learn interface.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://medium.com/distributed-computing-with-ray/gridsearchcv-2-0-new-and-improved-ee56644cbabf"&gt;The introduction&lt;/a&gt; of Ray’s tune-sklearn made this claim:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;tune-sklearn is the only
Scikit-Learn interface that allows you to easily leverage Bayesian
Optimization, HyperBand and other optimization techniques by simply toggling a few parameters.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;The state-of-the-art in hyperparameter optimization is currently
“&lt;a class="reference external" href="https://arxiv.org/pdf/1603.06560.pdf"&gt;Hyperband&lt;/a&gt;.” Hyperband reduces the amount of computation
required with a &lt;em&gt;principled&lt;/em&gt; early stopping scheme; past that, it’s the same as
Scikit-Learn’s popular &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomizedSearchCV&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Hyperband &lt;em&gt;works.&lt;/em&gt; As such, it’s very popular. After the introduction of
Hyperband in 2016 by Li et. al, &lt;a class="reference external" href="https://arxiv.org/pdf/1603.06560.pdf"&gt;the paper&lt;/a&gt; has been cited
&lt;a class="reference external" href="https://scholar.google.com/scholar?cites=10473284631669296057&amp;amp;amp;as_sdt=5,39&amp;amp;amp;sciodt=0,39&amp;amp;amp;hl=en"&gt;over 470 times&lt;/a&gt; and has been implemented in many different libraries
including &lt;a class="reference external" href="https://ml.dask.org/modules/generated/dask_ml.model_selection.HyperbandSearchCV.html#dask_ml.model_selection.HyperbandSearchCV"&gt;Dask-ML&lt;/a&gt;, &lt;a class="reference external" href="https://docs.ray.io/en/master/tune/api_docs/schedulers.html#asha-tune-schedulers-ashascheduler"&gt;Ray Tune&lt;/a&gt;, &lt;a class="reference external" href="https://keras-team.github.io/keras-tuner/documentation/tuners/#hyperband-class"&gt;keras-tune&lt;/a&gt;, &lt;a class="reference external" href="https://medium.com/optuna/optuna-supports-hyperband-93b0cae1a137"&gt;Optuna&lt;/a&gt;,
&lt;a class="reference external" href="https://www.automl.org/"&gt;AutoML&lt;/a&gt;,&lt;a class="footnote-reference brackets" href="#automl" id="id1" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;1&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt; and &lt;a class="reference external" href="https://nni.readthedocs.io/en/latest/Tuner/HyperbandAdvisor.html"&gt;Microsoft’s NNI&lt;/a&gt;. The original paper shows a
rather drastic improvement over all the relevant
implementations,&lt;a class="footnote-reference brackets" href="#hyperband-figs" id="id2" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;2&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt; and this drastic improvement persists in
follow-up works.&lt;a class="footnote-reference brackets" href="#follow-up" id="id3" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;3&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt; Some illustrative results from Hyperband are
below:&lt;/p&gt;
&lt;p&gt;&lt;img width="80%" src="/images/2020-model-selection/hyperband-fig-7-8.png"
 style="display: block; margin-left: auto; margin-right: auto;" /&gt;&lt;/p&gt;
&lt;div style="max-width: 80%; word-wrap: break-word;" style="text-align: center;"&gt;
&lt;p&gt;&lt;sup&gt;All algorithms are configured to do the same amount of work except “random
2x” which does twice as much work. “hyperband (finite)” is similar Dask-ML’s
default implementation, and “bracket s=4” is similar to Ray’s default
implementation. “random” is a random search. SMAC,&lt;a class="footnote-reference brackets" href="#smac" id="id4" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;4&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt;
spearmint,&lt;a class="footnote-reference brackets" href="#spearmint" id="id5" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;5&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt; and TPE&lt;a class="footnote-reference brackets" href="#tpe" id="id6" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;6&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt; are popular Bayesian algorithms. &lt;/sup&gt;&lt;/p&gt;
&lt;/div&gt;
&lt;p&gt;Hyperband is undoubtedly a “cutting edge” hyperparameter optimization
technique. Dask-ML and Ray offer Scikit-Learn implementations of this algorithm
that rely on similar implementations, and Dask-ML’s implementation also has a
&lt;a class="reference external" href="https://ml.dask.org/hyper-parameter-search.html#hyperband-parameters-rule-of-thumb"&gt;rule of thumb&lt;/a&gt; for configuration. Both Dask-ML’s and Ray’s documentation
encourages use of Hyperband.&lt;/p&gt;
&lt;p&gt;Ray does support using their Hyperband implementation on top of a technique
called Bayesian sampling. This changes the hyperparameter sampling scheme for
model initialization. This can be used in conjunction with Hyperband’s early
stopping scheme. Adding this option to Dask-ML’s Hyperband implementation is
future work for Dask-ML.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/08/06/ray-tune.md&lt;/span&gt;, line 222)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="framework-support"&gt;
&lt;h1&gt;Framework support&lt;/h1&gt;
&lt;p&gt;&lt;em&gt;Dask-ML model selection supports many libraries including Scikit-Learn, PyTorch, Keras, LightGBM and XGBoost.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Ray’s tune-sklearn supports these frameworks:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;tune-sklearn is used primarily for tuning
Scikit-Learn models, but it also supports and provides examples for many
other frameworks with Scikit-Learn wrappers such as Skorch (Pytorch),
KerasClassifiers (Keras), and XGBoostClassifiers (XGBoost).&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;Clearly, both Dask-ML and Ray support the many of the same libraries.&lt;/p&gt;
&lt;p&gt;However, both Dask-ML and Ray have some qualifications. Certain libraries don’t
offer an implementation of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt;,&lt;a class="footnote-reference brackets" href="#ray-pf" id="id7" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;7&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt; so not all of the modern
hyperparameter optimization techniques can be offered. Here’s a table comparing
different libraries and their support in Dask-ML’s model selection and Ray’s
tune-sklearn:&lt;/p&gt;
&lt;div class="pst-scrollable-table-container"&gt;&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr class="row-odd"&gt;&lt;th class="head text-center"&gt;&lt;p&gt;Model Library&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-center"&gt;&lt;p&gt;Dask-ML support&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-center"&gt;&lt;p&gt;Ray support&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-center"&gt;&lt;p&gt;Dask-ML: early stopping?&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-center"&gt;&lt;p&gt;Ray: early stopping?&lt;/p&gt;&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr class="row-even"&gt;&lt;td class="text-center"&gt;&lt;p&gt;&lt;a class="reference external" href="https://scikit-learn.org/"&gt;Scikit-Learn&lt;/a&gt;&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔*&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔*&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td class="text-center"&gt;&lt;p&gt;&lt;a class="reference external" href="https://pytorch.org/"&gt;PyTorch&lt;/a&gt; (via &lt;a class="reference external" href="https://skorch.readthedocs.io/"&gt;Skorch&lt;/a&gt;)&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-even"&gt;&lt;td class="text-center"&gt;&lt;p&gt;&lt;a class="reference external" href="https://keras.io/"&gt;Keras&lt;/a&gt; (via &lt;a class="reference external" href="https://github.com/adriangb/scikeras"&gt;SciKeras&lt;/a&gt;)&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔**&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔**&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td class="text-center"&gt;&lt;p&gt;&lt;a class="reference external" href="https://lightgbm.readthedocs.io/"&gt;LightGBM&lt;/a&gt;&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;❌&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;❌&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-even"&gt;&lt;td class="text-center"&gt;&lt;p&gt;&lt;a class="reference external" href="https://xgboost.ai/"&gt;XGBoost&lt;/a&gt;&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;✔&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;❌&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;❌&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;/div&gt;
&lt;p&gt;&lt;sup&gt;* Only for &lt;a class="reference external" href="https://scikit-learn.org/stable/modules/computing.html#incremental-learning"&gt;the models that implement &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;.&lt;/sup&gt;&lt;br&gt;
&lt;sup&gt;** Thanks to work by the Dask developers around &lt;a class="reference external" href="https://github.com/adriangb/scikeras/issues/24"&gt;scikeras#24&lt;/a&gt;.&lt;/sup&gt;&lt;/p&gt;
&lt;p&gt;By this measure, Dask-ML and Ray model selection have the same level of
framework support. Of course, Dask has tangential integration with LightGBM and
XGBoost through &lt;a class="reference external" href="https://ml.dask.org/xgboost.html"&gt;Dask-ML’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;xgboost&lt;/span&gt;&lt;/code&gt; module&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/dask/dask-lightgbm"&gt;dask-lightgbm&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/2020/08/06/ray-tune.md&lt;/span&gt;, line 272)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="scale-up"&gt;
&lt;h1&gt;Scale up&lt;/h1&gt;
&lt;p&gt;&lt;em&gt;Dask-ML supports distributed tuning (how could it not?), aka parallelization
across multiple machines/cores. In addition, it also supports
larger-than-memory data.&lt;/em&gt;&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;[Ray’s] Tune-sklearn leverages Ray Tune, a library for distributed
hyperparameter tuning, to efficiently and transparently parallelize cross
validation on multiple cores and even multiple machines.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;Naturally, Dask-ML also scales to multiple cores/machines because it relies on
Dask. Dask has wide support for &lt;a class="reference external" href="https://docs.dask.org/en/latest/setup.html"&gt;different deployment options&lt;/a&gt; that span
from your personal machine to supercomputers. Dask will very likely work on top
of any computing system you have available, including Kubernetes, SLURM, YARN
and Hadoop clusters as well as your personal machine.&lt;/p&gt;
&lt;p&gt;Dask-ML’s model selection also scales to larger-than-memory datasets, and is
thoroughly tested. Support for larger-than-memory data is untested in Ray, and
there are no examples detailing how to use Ray Tune with the distributed
dataset implementations in PyTorch/Keras.&lt;/p&gt;
&lt;p&gt;In addition, I have benchmarked Dask-ML’s model selection module to see how the
time-to-solution is affected by the number of Dask workers in “&lt;a class="reference external" href="https://blog.dask.org/2019/09/30/dask-hyperparam-opt"&gt;Better and
faster hyperparameter optimization with Dask&lt;/a&gt;.” That is, how does the
time to reach a particular accuracy scale with the number of workers &lt;span class="math notranslate nohighlight"&gt;\(P\)&lt;/span&gt;? At
first, it’ll scale like &lt;span class="math notranslate nohighlight"&gt;\(1/P\)&lt;/span&gt; but with large number of workers the serial
portion will dictate time to solution according to &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Amdahl%27s_law"&gt;Amdahl’s Law&lt;/a&gt;. Briefly, I
found Dask-ML’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt; speedup started to saturate around 24
workers for a particular search.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/08/06/ray-tune.md&lt;/span&gt;, line 311)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="speed"&gt;
&lt;h1&gt;Speed&lt;/h1&gt;
&lt;p&gt;&lt;em&gt;Both Dask-ML and Ray are much faster than Scikit-Learn.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Ray’s tune-sklearn runs some benchmarks in &lt;a class="reference external" href="https://medium.com/distributed-computing-with-ray/gridsearchcv-2-0-new-and-improved-ee56644cbabf"&gt;the introduction&lt;/a&gt; with the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GridSearchCV&lt;/span&gt;&lt;/code&gt; class found in Scikit-Learn and Dask-ML. A more fair benchmark
would be use Dask-ML’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt; because it is almost the same as the
algorithm in Ray’s tune-sklearn. To be specific, I’m interested in comparing
these methods:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Scikit-Learn’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomizedSearchCV&lt;/span&gt;&lt;/code&gt;. This is a popular implementation, one
that I’ve bootstrapped myself with a custom model.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask-ML’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt;. This is an early stopping technique for
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomizedSearchCV&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ray tune-sklearn’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;TuneSearchCV&lt;/span&gt;&lt;/code&gt;. This is a slightly different early
stopping technique than &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt;’s.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Each search is configured to perform the same task: sample 100 parameters and
train for no longer than 100 “epochs” or passes through the
data.&lt;a class="footnote-reference brackets" href="#random-search" id="id8" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;8&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt; Each estimator is configured as their respective
documentation suggests. Each search uses 8 workers with a single cross
validation split, and a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt; call takes one second with 50,000
examples. The complete setup can be found in &lt;a class="reference internal" href="#appendix"&gt;&lt;span class="xref myst"&gt;the appendix&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Here’s how long each library takes to complete the same search:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/2020-model-selection/n_workers=8.png" width="450px"
 /&gt;&lt;/p&gt;
&lt;p&gt;Notably, we didn’t improve the Dask-ML codebase for this benchmark, and ran the
code as it’s been for the last year.&lt;a class="footnote-reference brackets" href="#priority-impl" id="id9" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;9&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt; Regardless, it’s possible that
other artifacts from &lt;a class="reference external" href="http://matthewrocklin.com/blog/work/2017/03/09/biased-benchmarks"&gt;biased benchmarks&lt;/a&gt; crept into this benchmark.&lt;/p&gt;
&lt;p&gt;Clearly, Ray and Dask-ML offer similar performance for 8 workers when compared
with Scikit-Learn. To Ray’s credit, their implementation is ~15% faster than
Dask-ML’s with 8 workers. We suspect that this performance boost comes from the
fact that Ray implements an asynchronous variant of Hyperband. We should
investigate this difference between Dask and Ray, and how each balances the
tradeoffs, number FLOPs vs. time-to-solution. This will vary with the number
of workers: the asynchronous variant of Hyperband provides no benefit if used
with a single worker.&lt;/p&gt;
&lt;p&gt;Dask-ML reaches scores quickly in serial environments, or when the number of
workers is small. Dask-ML prioritizes fitting high scoring models: if there are
100 models to fit and only 4 workers available, Dask-ML selects the models that
have the highest score. This is most relevant in serial
environments;&lt;a class="footnote-reference brackets" href="#priority" id="id10" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;10&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt; see “&lt;a class="reference external" href="https://blog.dask.org/2019/09/30/dask-hyperparam-opt"&gt;Better and faster hyperparameter optimization
with Dask&lt;/a&gt;” for benchmarks. This feature is omitted from this
benchmark, which only focuses on time to solution.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/08/06/ray-tune.md&lt;/span&gt;, line 377)&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;Dask-ML and Ray offer the same features for model selection: state-of-the-art
features with a Scikit-Learn compatible API, and both implementations have
fairly wide support for different frameworks and rely on backends that can
scale to many machines.&lt;/p&gt;
&lt;p&gt;In addition, the Ray implementation has provided motivation for further
development, specifically on the following items:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Adding support for more libraries, including Keras&lt;/strong&gt; (&lt;a class="reference external" href="https://github.com/dask/dask-ml/issues/696"&gt;dask-ml#696&lt;/a&gt;,
&lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/713"&gt;dask-ml#713&lt;/a&gt;, &lt;a class="reference external" href="https://github.com/adriangb/scikeras/issues/24"&gt;scikeras#24&lt;/a&gt;). SciKeras is a Scikit-Learn wrapper for
Keras that (now) works with Dask-ML model selection because SciKeras models
implement the Scikit-Learn model API.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Better documenting the models that Dask-ML supports&lt;/strong&gt;
(&lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/699"&gt;dask-ml#699&lt;/a&gt;). Dask-ML supports any model that implement the
Scikit-Learn interface, and there are wrappers for Keras, PyTorch, LightGBM
and XGBoost. Now, &lt;a class="reference external" href="https://ml.dask.org"&gt;Dask-ML’s documentation&lt;/a&gt; prominently highlights this
fact.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The Ray implementation has also helped motivate and clarify future work.
Dask-ML should include the following implementations:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;A Bayesian sampling scheme for the Hyperband implementation&lt;/strong&gt; that’s
similar to Ray’s and BOHB’s (&lt;a class="reference external" href="https://github.com/dask/dask-ml/issues/697"&gt;dask-ml#697&lt;/a&gt;).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;A configuration of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt; that’s well-suited for
exploratory hyperparameter searches.&lt;/strong&gt; An initial implementation is in
&lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/532"&gt;dask-ml#532&lt;/a&gt;, which should be benchmarked against Ray.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Luckily, all of these pieces of development are straightforward modifications
because the Dask-ML model selection framework is pretty flexible.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Thank you &lt;a class="reference external" href="https://github.com/TomAugspurger"&gt;Tom Augspurger&lt;/a&gt;, &lt;a class="reference external" href="https://github.com/mrocklin"&gt;Matthew Rocklin&lt;/a&gt;, &lt;a class="reference external" href="https://github.com/jsignell"&gt;Julia Signell&lt;/a&gt;, and &lt;a class="reference external" href="https://github.com/quasiben"&gt;Benjamin
Zaitlen&lt;/a&gt; for your feedback, suggestions and edits.&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/08/06/ray-tune.md&lt;/span&gt;, line 427)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="appendix"&gt;
&lt;h1&gt;Appendix&lt;/h1&gt;
&lt;section id="benchmark-setup"&gt;
&lt;h2&gt;Benchmark setup&lt;/h2&gt;
&lt;p&gt;This is the complete setup for the benchmark between Dask-ML, Scikit-Learn and
Ray. Complete details can be found at
&lt;a class="reference external" href="https://github.com/stsievert/dask-hyperband-comparison"&gt;stsievert/dask-hyperband-comparison&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Let’s create a dummy model that takes 1 second for a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt; call with
50,000 examples. This is appropriate for this benchmark; we’re only interested
in the time required to finish the search, not how well the models do.
Scikit-learn, Ray and Dask-ML have have very similar methods of choosing
hyperparameters to evaluate; they differ in their early stopping techniques.&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;scipy.stats&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;uniform&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;sklearn.model_selection&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;make_classification&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;benchmark&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;ConstantFunction&lt;/span&gt;  &lt;span class="c1"&gt;# custom module&lt;/span&gt;

&lt;span class="c1"&gt;# This model sleeps for `latency * len(X)` seconds before&lt;/span&gt;
&lt;span class="c1"&gt;# reporting a score of `value`.&lt;/span&gt;
&lt;span class="n"&gt;model&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ConstantFunction&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;latency&lt;/span&gt;&lt;span class="o"&gt;=&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;50e3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;max_iter&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;max_iter&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;params&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;value&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;uniform&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;)}&lt;/span&gt;
&lt;span class="c1"&gt;# This dummy dataset mirrors the MNIST dataset&lt;/span&gt;
&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;make_classification&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_samples&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;60e3&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;n_features&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;784&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This model will take 2 minutes to train for 100 epochs (aka passes through the
data). Details can be found at &lt;a class="reference external" href="https://github.com/stsievert/dask-hyperband-comparison"&gt;stsievert/dask-hyperband-comparison&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Let’s configure our searches to use 8 workers with a single cross-validation
split:&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;sklearn.model_selection&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;RandomizedSearchCV&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;ShuffleSplit&lt;/span&gt;
&lt;span class="n"&gt;split&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ShuffleSplit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;test_size&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;0.2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n_splits&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;kwargs&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="n"&gt;cv&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;split&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;refit&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;search&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;RandomizedSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n_jobs&lt;/span&gt;&lt;span class="o"&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;n_iter&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;n_params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# 20.88 minutes&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_ml.model_selection&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;HyperbandSearchCV&lt;/span&gt;
&lt;span class="n"&gt;dask_search&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;HyperbandSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;test_size&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;0.2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;max_iter&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;max_iter&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;aggressiveness&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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;tune_sklearn&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;TuneSearchCV&lt;/span&gt;
&lt;span class="n"&gt;ray_search&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;TuneSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n_iter&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;n_params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;max_iters&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;max_iter&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;early_stopping&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;dask_search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# 2.93 minutes&lt;/span&gt;
&lt;span class="n"&gt;ray_search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# 2.49 minutes&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="full-example-usage"&gt;
&lt;h2&gt;Full example usage&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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;sklearn.linear_model&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;SGDClassifier&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;scipy.stats&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;uniform&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;loguniform&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;sklearn.datasets&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;make_classification&lt;/span&gt;
&lt;span class="n"&gt;model&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;SGDClassifier&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;params&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;alpha&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;loguniform&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;1e-5&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;1e-3&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;l1_ratio&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;uniform&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;)}&lt;/span&gt;
&lt;span class="n"&gt;X&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;make_classification&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;sklearn.model_selection&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;RandomizedSearchCV&lt;/span&gt;
&lt;span class="n"&gt;search&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;RandomizedSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&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;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&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_ml.model_selection&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;HyperbandSearchCV&lt;/span&gt;
&lt;span class="n"&gt;HyperbandSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&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;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;classes&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;

&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;tune_sklearn&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;TuneSearchCV&lt;/span&gt;
&lt;span class="n"&gt;search&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;TuneSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;model&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&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;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;classes&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;hr class="docutils" /&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/08/06/ray-tune.md&lt;/span&gt;, line 40)&lt;/p&gt;
&lt;p&gt;Duplicate reference definition: TSNE [myst.duplicate_def]&lt;/p&gt;
&lt;/aside&gt;
&lt;hr class="footnotes docutils" /&gt;
&lt;aside class="footnote-list brackets"&gt;
&lt;aside class="footnote brackets" id="automl" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id1"&gt;1&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;Their implementation of Hyperband in &lt;a class="reference external" href="https://github.com/automl/HpBandSter"&gt;HpBandSter&lt;/a&gt; is included in &lt;a class="reference external" href="https://www.automl.org/wp-content/uploads/2018/09/chapter7-autonet.pdf"&gt;Auto-PyTorch&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/automl/BOAH"&gt;BOAH&lt;/a&gt;.&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="hyperband-figs" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id2"&gt;2&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;See Figures 4, 7 and 8 in “&lt;a class="reference external" href="https://arxiv.org/pdf/1603.06560.pdf"&gt;Hyperband: A Novel Bandit-Based Approach to Hyperparameter Optimization&lt;/a&gt;.”&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="follow-up" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id3"&gt;3&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;See Figure 1 of &lt;a class="reference external" href="http://proceedings.mlr.press/v80/falkner18a/falkner18a.pdf"&gt;the BOHB paper&lt;/a&gt; and &lt;a class="reference external" href="https://arxiv.org/pdf/1801.01596.pdf"&gt;a paper&lt;/a&gt; from an augmented reality company.&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="smac" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id4"&gt;4&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;SMAC is described in “&lt;a class="reference external" href="https://www.cs.ubc.ca/~hutter/papers/10-TR-SMAC.pdf"&gt;Sequential Model-Based Optimization forGeneral Algorithm Configuration&lt;/a&gt;,” and is available &lt;a class="reference external" href="https://www.automl.org/automated-algorithm-design/algorithm-configuration/smac/"&gt;in AutoML&lt;/a&gt;.&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="spearmint" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id5"&gt;5&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;Spearmint is described in “&lt;a class="reference external" href="https://papers.nips.cc/paper/4522-practical-bayesian-optimization-of-machine-learning-algorithms.pdf"&gt;Practical Bayesian Optimization of MachineLearning Algorithms&lt;/a&gt;,” and is available in &lt;a class="reference external" href="https://github.com/HIPS/Spearmint"&gt;HIPS/spearmint&lt;/a&gt;.&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="tpe" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id6"&gt;6&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;TPE is described in Section 4 of “&lt;a class="reference external" href="http://papers.nips.cc/paper/4443-algorithms-for-hyper-parameter-optimization.pdf"&gt;Algorithms for Hyperparameter Optimization&lt;/a&gt;,” and is available &lt;a class="reference external" href="http://hyperopt.github.io/hyperopt/"&gt;through Hyperopt&lt;/a&gt;.&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="ray-pf" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id7"&gt;7&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;From &lt;a class="reference external" href="https://github.com/ray-project/tune-sklearn/blob/31f228e21ef632a89a74947252d8ad5323cbd043/README.md"&gt;Ray’s README.md&lt;/a&gt;: “If the estimator does not support &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt;, a warning will be shown saying early stopping cannot be done and it will simply run the cross-validation on Ray’s parallel back-end.”&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="random-search" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id8"&gt;8&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;I choose to benchmark random searches instead of grid searches because random searches produce better results because grid searches require estimating how important each parameter is; for more detail see “&lt;a class="reference external" href="http://www.jmlr.org/papers/volume13/bergstra12a/bergstra12a.pdf"&gt;Random Search for Hyperparameter Optimization&lt;/a&gt;” by Bergstra and Bengio.&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="priority-impl" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id9"&gt;9&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;Despite a relevant implementation in &lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/527"&gt;dask-ml#527&lt;/a&gt;.&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="priority" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id10"&gt;10&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;Because priority is meaningless if there are an infinite number of workers.&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="bohb-exps" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;11&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;Details are in “&lt;a class="reference external" href="http://proceedings.mlr.press/v80/falkner18a/falkner18a.pdf"&gt;BOHB: Robust and Efficient Hyperparameter Optimization at Scale&lt;/a&gt;.”&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="nlp-future" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;12&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;Future work is combining this with the Dask-ML’s Hyperband implementation.&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="openai" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;13&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;Computing &lt;a class="reference external" href="https://en.wikipedia.org/wiki/N-gram"&gt;n-grams&lt;/a&gt; requires a ton of memory and computation. For OpenAI, NLP preprocessing took 8 GPU-months! (&lt;a class="reference external" href="https://openai.com/blog/language-unsupervised/#drawbacks"&gt;source&lt;/a&gt;)&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="stopping" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;14&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;Hyperband’s theory answers “how many models should be stopped?” and “when should they be stopped?”&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="bohb-parallel" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;15&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;In Section 4.2 of &lt;a class="reference external" href="http://proceedings.mlr.press/v80/falkner18a/falkner18a.pdf"&gt;their paper&lt;/a&gt;.&lt;/p&gt;
&lt;/aside&gt;
&lt;/aside&gt;
</content>
    <link href="https://blog.dask.org/2020/08/06/ray-tune/"/>
    <summary>Hyperparameter optimization is the process of deducing model parameters that
can’t be learned from data. This process is often time- and resource-consuming,
especially in the context of deep learning. A good description of this process
can be found at “Tuning the hyper-parameters of an estimator,” and
the issues that arise are concisely summarized in Dask-ML’s documentation of
“Hyper Parameter Searches.”</summary>
    <category term="dask" label="dask"/>
    <category term="dask-ml" label="dask-ml"/>
    <category term="machine-learning" label="machine-learning"/>
    <category term="ray" label="ray"/>
    <published>2020-08-06T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2020/07/30/beginners-config/</id>
    <title>Configuring a Distributed Dask Cluster</title>
    <updated>2020-07-30T00:00:00+00:00</updated>
    <author>
      <name>Julia Signell (Saturn Cloud)</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;Configuring a Dask cluster can seem daunting at first, but the good news is that the Dask project has a lot of built in heuristics that try its best to anticipate and adapt to your workload based on the machine it is deployed on and the work it receives. Possibly for a long time you can get away with not configuring anything special at all. That being said, if you are looking for some tips to move on from using Dask locally, or have a Dask cluster that you are ready to optimize with some more in-depth configuration, these tips and tricks will help guide you and link you to the best Dask docs on the topic!&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/30/beginners-config.md&lt;/span&gt;, line 12)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="how-to-host-a-distributed-dask-cluster"&gt;

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

&lt;p&gt;Let’s dive in by covering the most straight forward way to setup a distributed Dask cluster.&lt;/p&gt;
&lt;section id="setup-scheduler-and-workers"&gt;
&lt;h2&gt;Setup scheduler and workers&lt;/h2&gt;
&lt;p&gt;Imagine we have three computers, we will call them &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;MachineA&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;MachineB&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;MachineC&lt;/span&gt;&lt;/code&gt;. Each of these machines has a functioning Python environment and we have &lt;a class="reference external" href="https://docs.dask.org/en/latest/install.html"&gt;installed Dask with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;conda&lt;/span&gt; &lt;span class="pre"&gt;install&lt;/span&gt; &lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;. If we want to pull them together into a Dask cluster we start by running a scheduler on &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;MachineA&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;div class="highlight-console notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;$ &lt;/span&gt;dask-scheduler
&lt;span class="go"&gt;distributed.scheduler - INFO - -----------------------------------------------&lt;/span&gt;
&lt;span class="go"&gt;distributed.scheduler - INFO - Local Directory: /tmp/scheduler-btqf8ve1&lt;/span&gt;
&lt;span class="go"&gt;distributed.scheduler - INFO - -----------------------------------------------&lt;/span&gt;
&lt;span class="go"&gt;distributed.scheduler - INFO - Clear task state&lt;/span&gt;
&lt;span class="go"&gt;distributed.scheduler - INFO -   Scheduler at: tcp://MachineA:8786&lt;/span&gt;
&lt;span class="go"&gt;distributed.scheduler - INFO -   dashboard at:               :8787&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Next we need to start a worker process on both &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;MachineB&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;MachineC&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;div class="highlight-console notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;$ &lt;/span&gt;dask-worker&lt;span class="w"&gt; &lt;/span&gt;tcp://MachineA:8786
&lt;span class="go"&gt;distributed.nanny - INFO -         Start Nanny at:    &amp;#39;tcp://127.0.0.1:51224&amp;#39;&lt;/span&gt;
&lt;span class="go"&gt;distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51225&lt;/span&gt;
&lt;span class="go"&gt;distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51225&lt;/span&gt;
&lt;span class="go"&gt;distributed.worker - INFO -          dashboard at:            127.0.0.1:51226&lt;/span&gt;
&lt;span class="go"&gt;distributed.worker - INFO - Waiting to connect to:        tcp://MachineA:8786&lt;/span&gt;
&lt;span class="go"&gt;distributed.worker - INFO - -------------------------------------------------&lt;/span&gt;
&lt;span class="go"&gt;distributed.worker - INFO -               Threads:                          4&lt;/span&gt;
&lt;span class="go"&gt;distributed.worker - INFO -                Memory:                    8.00 GB&lt;/span&gt;
&lt;span class="go"&gt;distributed.worker - INFO -       Local Directory:       /tmp/worker-h3wfwg7j&lt;/span&gt;
&lt;span class="go"&gt;distributed.worker - INFO - -------------------------------------------------&lt;/span&gt;
&lt;span class="go"&gt;distributed.worker - INFO -         Registered to:        tcp://MachineA:8786&lt;/span&gt;
&lt;span class="go"&gt;distributed.worker - INFO - -------------------------------------------------&lt;/span&gt;
&lt;span class="go"&gt;distributed.core - INFO - Starting established connection&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If we start a worker on both of our two space machines Dask will autodetect the resources on the machine and make them available to the scheduler. In the example above the worker has detected 4 CPU cores and 8GB of RAM. Therefore our scheduler has access to a total of 8 cores and 16GB of RAM and it will use these resources to run through the computation graph as quickly as possible. If we add more workers on more machines then the amount of resources available to the scheduler increases and computation times should get faster.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;&lt;strong&gt;Note&lt;/strong&gt;: While the scheduler machine probably has the same resources as the other two these will not be used in the computation.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Lastly we need to connect to our scheduler from our Python session.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;tcp://MachineA:8786&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Creating this &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client&lt;/span&gt;&lt;/code&gt; object within the Python global namespace means that any Dask code you execute will detect this and hand the computation off to the scheduler which will then execute on the workers.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="accessing-the-dashboard"&gt;
&lt;h2&gt;Accessing the dashboard&lt;/h2&gt;
&lt;p&gt;The Dask distributed scheduler also has a dashboard which can be opened in a web browser. As you can see in the output above the default location for this is on the scheduler machine at port &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;8787&lt;/span&gt;&lt;/code&gt;. So you should be able to navigate to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;http://MachineA:8787&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;a href="https://i.imgur.com/VzQIVpI.png"&gt;
&lt;img alt="Dask dashboard" src="https://i.imgur.com/VzQIVpI.png" width="100%" align="center"&gt;
&lt;/a&gt;
&lt;p&gt;If you are using Jupyter Lab as your Python environment you are also able to open individual plots from the dashboard as windows in Jupyter Lab with the &lt;a class="reference external" href="https://github.com/dask/dask-labextension"&gt;Dask Lab Extension&lt;/a&gt;.&lt;/p&gt;
&lt;a href="https://i.imgur.com/SNk6F0H.png"&gt;
&lt;img alt="Dask Lab Extension" src="https://i.imgur.com/SNk6F0H.png" width="100%" align="center"&gt;
&lt;/a&gt;
&lt;/section&gt;
&lt;section id="recap"&gt;
&lt;h2&gt;Recap&lt;/h2&gt;
&lt;p&gt;In this minimal example we have installed Dask on some machines, ran a distributed scheduler on one of them and workers on the others. We then connected to our cluster from our Python session and opened the dashboard to keep an eye on the cluster.&lt;/p&gt;
&lt;p&gt;What we haven’t covered is where these machines came from in the first place. In the rest of this post we will discuss the different ways that folks tend to run clusters out in the wild and give an overview of the various tools that exist to help you set up Dask clusters on a variety of infrastructure.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/23/current-state-of-distributed-dask-clusters.md&lt;/span&gt;, line 85)&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="cluster-requirements"&gt;
&lt;h1&gt;Cluster requirements&lt;/h1&gt;
&lt;p&gt;In order to run a Dask cluster you must be able to install Dask on a machine and start the scheduler and worker components. These machines need to be able to communicate via a network so that these components can speak to each other.&lt;/p&gt;
&lt;p&gt;You also need to be able to access the scheduler from your Python session via a network in order to connect the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client&lt;/span&gt;&lt;/code&gt; and access the dashboard.&lt;/p&gt;
&lt;p&gt;Lastly the Python environment in the Python session where you create your &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client&lt;/span&gt;&lt;/code&gt; must match the Python environment where the workers are running. This is because Dask uses &lt;a class="reference external" href="https://github.com/cloudpipe/cloudpickle"&gt;cloudpickle&lt;/a&gt; to serialize objects and send them to workers and to retrieve results. Therefore package versions must match in both locations.&lt;/p&gt;
&lt;p&gt;We will need to bear these requirements in mind as we discuss the different platforms that folks generally want to run Dask on.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/23/current-state-of-distributed-dask-clusters.md&lt;/span&gt;, line 95)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="cluster-types"&gt;
&lt;h1&gt;Cluster types&lt;/h1&gt;
&lt;p&gt;There are two “types” of clusters that I tend to see folks running. Fixed clusters and ephemeral clusters.&lt;/p&gt;
&lt;section id="fixed-clusters"&gt;
&lt;h2&gt;Fixed clusters&lt;/h2&gt;
&lt;p&gt;One common way of setting up a cluster is to run the scheduler and worker commands as described above, but leave them running indefinitely. For the purpose of this article I’ll refer to this as a “fixed cluster”. You may use something like &lt;a class="reference external" href="https://www.freedesktop.org/wiki/Software/systemd/"&gt;systemd&lt;/a&gt; or &lt;a class="reference external" href="http://supervisord.org/"&gt;supervisord&lt;/a&gt; to manage the processes and ensure they are always running on the machines. The Dask cluster can then be treated as a service.&lt;/p&gt;
&lt;p&gt;In this paradigm once a cluster is set up folks may start their Python session, connect their client to this existing cluster, do some work and disconnect again. They might later come back to that cluster and run further work. The cluster will sit idle in the meantime.&lt;/p&gt;
&lt;p&gt;It is also common in this paradigm for multiple users to share this single cluster, however this is not recommended as the Dask scheduler does not manage users or clients separately and work will be executed on a first come first served bases. Therefore we recommend that users use a cluster one at a time.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="ephemeral-clusters"&gt;
&lt;h2&gt;Ephemeral clusters&lt;/h2&gt;
&lt;p&gt;An ephemeral cluster is one which only exists for the duration of the work. In this case a user may SSH onto the machines, run the commands to set up the cluster, connect a client and perform work, then disconnect and exit the Dask processes. A basic way of doing this would be to create a bash script which calls &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ssh&lt;/span&gt;&lt;/code&gt; and sets up the cluster. You would run this script in the background while performing your work and then kill it once you are done. We will cover other implementations of this in the coming sections.&lt;/p&gt;
&lt;p&gt;Ephemeral clusters allow you to leverage a bunch of machines but free them up again when you are done. This is especially useful when you are using a system like a cloud service or a batch scheduler where you have limited credits, or are charged for provisioned resources.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/23/current-state-of-distributed-dask-clusters.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&gt;
&lt;section id="adaptivity"&gt;
&lt;h1&gt;Adaptivity&lt;/h1&gt;
&lt;p&gt;Ephemeral clusters are also generally easier to scale as you will likely have an automated mechanism for starting workers. The Dask scheduler maintains an estimate of how long it expects the outstanding work will take to complete. If the scheduler has a mechanism for starting and stopping workers then it will scale up the workers in an attempt to complete all outstanding work within 5 seconds. This is referred to as adaptive mode.&lt;/p&gt;
&lt;p&gt;The mechanisms for starting and stopping workers are added via plugins. Many of the implementations we are about to discuss include this logic.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/23/current-state-of-distributed-dask-clusters.md&lt;/span&gt;, line 119)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="connectivity"&gt;
&lt;h1&gt;Connectivity&lt;/h1&gt;
&lt;p&gt;Dask uses TCP to communicate between client, scheduler and workers by default. This means that all of these components must be on a TCP/IP network with open routes between the machines. Many connectivity problems step from firewalls or private networks blocking connections between certain components. An example of this would be running Dask on a cloud platform like AWS, but running the Python session and client on your laptop while sitting in a coffee shop using the free wifi. You must ensure you are able to route traffic between components, either by exposing the Dask cluster to the internet or by connecting your laptop to the private network via a VPN or tunnel.&lt;/p&gt;
&lt;p&gt;There is also &lt;a class="reference external" href="https://blog.dask.org/2019/06/09/ucx-dgx"&gt;ongoing work to add support for UCX&lt;/a&gt; to Dask, which will allow it to make use of InfiniBand or NVLink networks where they are available.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/23/current-state-of-distributed-dask-clusters.md&lt;/span&gt;, line 125)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="cluster-managers"&gt;
&lt;h1&gt;Cluster Managers&lt;/h1&gt;
&lt;p&gt;In the following section we are going to cover a range of cluster manager implementations which are available within the Dask community.&lt;/p&gt;
&lt;p&gt;In the Dask distributed codebase there is a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Cluster&lt;/span&gt;&lt;/code&gt; superclass which can be subclassed to build various cluster managers for different platforms. Members of the community have taken this and built their own packages which enable creating a Dask cluster on a specific platform, for example &lt;a class="reference external" href="https://kubernetes.io/"&gt;Kubernetes&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The design of these classes is that you import the cluster manager into your Python session and instantiate it. The object then handles starting the scheduler and worker processes on the target platform. You can then create a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client&lt;/span&gt;&lt;/code&gt; object as usual from that cluster object to connect to it.&lt;/p&gt;
&lt;p&gt;All of these cluster manager objects are ephemeral clusters, they only exist for the duration of the Python session and then will be cleaned up.&lt;/p&gt;
&lt;section id="local-cluster"&gt;
&lt;h2&gt;Local Cluster&lt;/h2&gt;
&lt;p&gt;Let’s start with one of the reference implementations of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Cluster&lt;/span&gt;&lt;/code&gt; from the Dask distributed codebase &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LocalCluster&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This cluster manager starts a scheduler on your local machine, and then starts a worker for every CPU core that it finds on the machine.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="ssh-cluster"&gt;
&lt;h2&gt;SSH Cluster&lt;/h2&gt;
&lt;p&gt;Another reference implementation is &lt;a class="reference external" href="https://docs.dask.org/en/latest/setup/ssh.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;SSHCluster&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;. This is one of the most pure and simple ways of using multiple machines with Dask distributed and is very similar to our initial example in this blog post.&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.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;SSHCluster&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;SSHCluster&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;MachineA&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;MachineB&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;MachineC&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The first argument here is a list of machines which we can SSH into and set up a Dask cluster on. The first machine in the list will be used as the scheduler and the rest as workers.&lt;/p&gt;
&lt;p&gt;As the scheduler will likely use far less resources than the workers you may even want to run that locally and make use of all three remote machines as workers.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;SSHCluster&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;localhost&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;MachineA&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;MachineB&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;MachineC&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="speccluster"&gt;
&lt;h2&gt;SpecCluster&lt;/h2&gt;
&lt;p&gt;The last implementation that is included in the core Dask distributed library is &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;SpecCluster&lt;/span&gt;&lt;/code&gt;. This is actually another superclass and is designed to be subclassed by other developers when building cluster managers. However it goes further than &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Cluster&lt;/span&gt;&lt;/code&gt; in expecting the developer to provide a full specification for schedulers and workers as Python classes. There is also a superclass called &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ProcessInterface&lt;/span&gt;&lt;/code&gt; which is designed to be used when creating those scheduler and worker classes.&lt;/p&gt;
&lt;p&gt;Having standard interfaces means a more consistent experience for users. Many of the cluster manager we will cover next use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;SpecCluster&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-kubernetes"&gt;
&lt;h2&gt;Dask Kubernetes&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/"&gt;Dask Kubernetes&lt;/a&gt; provides a cluster manager for &lt;a class="reference external" href="https://kubernetes.io/"&gt;Kubernetes&lt;/a&gt; called &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;KubeCluster&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Kubernetes provides high level APIs and abstract concepts for scheduling linux containers on a cluster of machines. It provides abstracted concepts for processes, containers, networks, storage, etc to empower better use of data centre scale resources.&lt;/p&gt;
&lt;p&gt;As a Dask user it generally doesn’t matter to you how your cluster is set up. But if you’ve been given access to a Kubernetes cluster by your organisation or institution you will need to understand those concepts in order to schedule your work on it.&lt;/p&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;KubeCluster&lt;/span&gt;&lt;/code&gt; cluster manager further abstracts away those concepts into the Dask terms we are familiar with.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_kubernetes&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;KubeCluster&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;KubeCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;cluster_specific_kwargs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In order for this code to work you will need to have &lt;a class="reference external" href="https://kubernetes.io/docs/tasks/access-application-cluster/access-cluster/"&gt;configured your Kubernetes credentials&lt;/a&gt;, in the same way that for the SSH example you will need to configure your keys.&lt;/p&gt;
&lt;p&gt;Your client will also need to be able to access the Dask scheduler, and you probably want to be able to open the dashboard in your browser. However Kubernetes uses an overlay network which means that the IP addresses assigned to the scheduler and workers are only routable within the cluster. This is fine for them talking to each other but means you wont be able to get in from the outside.&lt;/p&gt;
&lt;p&gt;One way around this is to ensure your Python session is also running inside the Kubernetes cluster. A popular way of setting up an interactive Python environment on Kubernetes is with &lt;a class="reference external" href="https://zero-to-jupyterhub.readthedocs.io/en/latest/"&gt;Zero to Jupyter Hub&lt;/a&gt;, which gives you access to &lt;a class="reference external" href="https://jupyter.org/"&gt;Jupyter&lt;/a&gt; notebooks running within the Kubernetes cluster.&lt;/p&gt;
&lt;p&gt;The alternative is exposing your scheduler to the external network. You can do this by &lt;a class="reference external" href="https://kubernetes.io/docs/tutorials/kubernetes-basics/expose/expose-intro/"&gt;exposing the Kubernetes &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Service&lt;/span&gt;&lt;/code&gt; object&lt;/a&gt; associated with the scheduler or by &lt;a class="reference external" href="https://kubernetes.io/docs/concepts/services-networking/ingress/"&gt;setting up and configuring an Ingress component&lt;/a&gt; for your Kubernetes cluster. Both of these options require some knowledge of Kubernetes.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-helm-chart"&gt;
&lt;h2&gt;Dask Helm chart&lt;/h2&gt;
&lt;p&gt;Another option for running Dask on a Kubernetes cluster is using the &lt;a class="reference external" href="https://github.com/dask/helm-chart"&gt;Dask Helm Chart&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;This is an example of a fixed cluster setup. Helm is a way of installing specific resources on a Kubernetes cluster, similar to a package manager like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;apt&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;yum&lt;/span&gt;&lt;/code&gt;. The Dask Helm chart includes a Jupyter notebook, a Dask scheduler and three Dask workers. The workers can be scaled manually by interacting with the Kubernetes API but not adaptively by the Dask scheduler itself.&lt;/p&gt;
&lt;p&gt;This feels like a different approach to what we’ve seen so far. It gives you a Dask cluster which is always available, and a Jupyter notebook to drive the cluster from. You then have to take your work to the cluster’s Jupyter session rather than spawning a cluster from your existing work place.&lt;/p&gt;
&lt;p&gt;One benefit of this approach is that because the Jupyter notebook has been set up as part of the cluster it already has the Lab Extension installed and also has been &lt;a class="reference external" href="https://github.com/dask/helm-chart/blob/f413647f90d6e278515b172c623977578a535aa2/dask/templates/dask-jupyter-deployment.yaml#L47-L48"&gt;pre-configured&lt;/a&gt; with the location of the Dask cluster. So unlike previous examples where you need to either give the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client&lt;/span&gt;&lt;/code&gt; the address of the scheduler or a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Cluster&lt;/span&gt;&lt;/code&gt; object, in this instance it will auto-detect the cluster from environment variables that are set by the Helm chart.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;

&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# The address is loaded from an environment variable&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;em&gt;&lt;strong&gt;Note:&lt;/strong&gt; If you call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client&lt;/span&gt;&lt;/code&gt; without any arguments in other situations where the scheduler location has not been configured it will automatically create a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LocalCluster&lt;/span&gt;&lt;/code&gt; object and use that.&lt;/em&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-jobqueue"&gt;
&lt;h2&gt;Dask Jobqueue&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-jobqueue"&gt;Dask Jobqueue&lt;/a&gt; is a set of cluster managers aimed at HPC users.&lt;/p&gt;
&lt;p&gt;When working as a researcher or academic with access to an HPC or Supercomputer you likely have to submit work to that machine via some kind of job queueing system. This is often in the form of a bash script which contains metadata about how much resource you need on the machine and the commands you want to run.&lt;/p&gt;
&lt;p&gt;Dask Jobqueue has cluster manager objects for &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Portable_Batch_System"&gt;PBS&lt;/a&gt;, &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Slurm_Workload_Manager"&gt;Slurm&lt;/a&gt;, and &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Oracle_Grid_Engine"&gt;SGE&lt;/a&gt;. When creating these cluster managers they will construct scripts for the batch scheduler based on your arguments and submit them using your default credentials.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_jobqueue&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;PBSCluster&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;PBSCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;cluster_specific_kwargs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;As batch systems like these often have a long wait time you may not immediately get access to your cluster object and scaling can be slow. Depending on the queueing policies it may be best to think of this as a fixed sized cluster. However if you have a responsive interactive queue then you can use this like any other autoscaling cluster manager.&lt;/p&gt;
&lt;p&gt;Again it is expected that your Python session is able to connect to the IP address of the scheduler. This may vary depending on your HPC centre setup as to how you can ensure this.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-yarn"&gt;
&lt;h2&gt;Dask Yarn&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="https://yarn.dask.org/en/latest/"&gt;Dask Yarn&lt;/a&gt; is a cluster manager for traditional &lt;a class="reference external" href="https://hadoop.apache.org/"&gt;Hadoop&lt;/a&gt; systems.&lt;/p&gt;
&lt;p&gt;Hadoop is a framework that allows for the distributed processing of large data sets across clusters of computers using simple programming models. It is a common piece of infrastrcture in Java/Scala ecosystems for processing large volumes of data. However you can also use the scheduling functionality called YARN to schedule Dask workers and leverage the underlying hardware resources.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_yarn&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;YarnCluster&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;YarnCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;cluster_specific_kwargs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Dask Yarn is only intended to be used from a Hadoop edge node which will have access to the internal network of the Hadoop cluster.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-cloudprovider"&gt;
&lt;h2&gt;Dask Cloudprovider&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="https://cloudprovider.dask.org/en/latest/"&gt;Dask Cloudprovider&lt;/a&gt; is a collection of cluster managers for leveraging cloud native APIs.&lt;/p&gt;
&lt;p&gt;Cloud providers such as &lt;a class="reference external" href="https://aws.amazon.com/"&gt;Amazon&lt;/a&gt;, &lt;a class="reference external" href="https://azure.microsoft.com/"&gt;Microsoft&lt;/a&gt; and &lt;a class="reference external" href="https://cloud.google.com/"&gt;Google&lt;/a&gt; have many APIs available for building and running various types of infrastructure. These range from traditional virtual servers running linux or Windows to higher level APIs that can execute small snippets of code on demand. They have batch systems, Hadoop systems, machine learning systems and more.&lt;/p&gt;
&lt;p&gt;The ideal scenario for running Dask on a cloud provider would be a service which would allow you to run the scheduler and worker with specified Python environments and then connect to them securely from the outside. Such a service doesn’t quite exist, but similar things do to varying degrees.&lt;/p&gt;
&lt;p&gt;One example is &lt;a class="reference external" href="https://aws.amazon.com/fargate/"&gt;AWS Fargate&lt;/a&gt; which is a managed container platform. You can run &lt;a class="reference external" href="https://www.docker.com/"&gt;Docker containers&lt;/a&gt; on demand which each have a unique IP address which can be public or private. This means we can run Dask scheduler and worker processes within a &lt;a class="reference external" href="https://github.com/dask/dask-docker"&gt;Dask container&lt;/a&gt; and connect to them from our Python session. This service is &lt;a class="reference external" href="https://aws.amazon.com/fargate/pricing/"&gt;billed per second&lt;/a&gt; for the requested resources, so makes most sense as an ephemeral service which has no cost when you aren’t using it.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_cloudprovider&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;FargateCluster&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;FargateCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;cluster_specific_kwargs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This cluster manager uses your &lt;a class="reference external" href="https://docs.aws.amazon.com/cli/latest/userguide/cli-chap-configure.html#cli-quick-configuration"&gt;AWS credentials&lt;/a&gt; to authenticate and request AWS resources on Fargate, and then connects your local session to the Dask cluster running on the cloud.&lt;/p&gt;
&lt;p&gt;There are even higher level services such as &lt;a class="reference external" href="https://aws.amazon.com/lambda/"&gt;AWS Lambda&lt;/a&gt; or &lt;a class="reference external" href="https://cloud.google.com/functions"&gt;Google Cloud Functions&lt;/a&gt; which allow you to execute code on demand and you are billed for the execution time of the code. These are referred to as “serverless” services as the servers are totally abstracted away. This would be perfect for out Dask cluster as you could submit the scheduler and workers as the code to run. &lt;strong&gt;However&lt;/strong&gt; when running these cloud functions it is not possible to get a network connection between them as they do not have routable IP addresses, so there is no way to set up a Dask cluster made of these executing functions. Maybe one day!&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-gateway"&gt;
&lt;h2&gt;Dask Gateway&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="https://gateway.dask.org/"&gt;Dask Gateway&lt;/a&gt; is a central service for managing Dask clusters. It provides a secure API which multiple users can communicate with to request Dask servers. It can spawn Dask clusters on Kubernetes, Yarn or batch systems.&lt;/p&gt;
&lt;p&gt;This tool is targeted at IT administrators who want to enable their users to create Dask clusters, but want to maintain some centralized control instead of each user creating their own thing. This can also be useful for tracking Dask usage and setting per user limits.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_gateway&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;GatewayCluster&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;GatewayCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;cluster_specific_kwargs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;For each user the commands for creating and using a gateway cluster are the same. It is down to the administrator to setup and manage the gateway server and configure &lt;a class="reference external" href="https://gateway.dask.org/authentication.html#"&gt;authentication via kerberos or Jupyter Hub&lt;/a&gt;. They should also provide &lt;a class="reference external" href="https://gateway.dask.org/configuration-user.html"&gt;configuration&lt;/a&gt; to their users so that Dask Gateway knows how to connect to the gateway server. In a large organisation or institution the IT department also likely provisions the machines that staff are using, and so should be able to drop configuration files onto users computers.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="local-cuda-cluster"&gt;
&lt;h2&gt;Local CUDA Cluster&lt;/h2&gt;
&lt;p&gt;The last cluster manager I’m going to cover is &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LocalCUDACluster&lt;/span&gt;&lt;/code&gt; from the &lt;a class="reference external" href="https://github.com/rapidsai/dask-cuda"&gt;Dask CUDA&lt;/a&gt; package.&lt;/p&gt;
&lt;p&gt;This is slightly different than the other cluster managers in that it is constructing a Dask cluster which is specifically optimised for a single piece of hardware. In this case it is targeting machines with GPUs ranging from your laptop with an onboard NVIDIA GPU to an &lt;a class="reference external" href="https://www.nvidia.com/en-gb/data-center/dgx-2/"&gt;NVIDIA DGX-2&lt;/a&gt; with multiple GPUs running in your datacentre.&lt;/p&gt;
&lt;p&gt;The cluster manager closely follows the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LocalCluster&lt;/span&gt;&lt;/code&gt; in that is creates resources locally on the current machine, but instead of creating one worker per CPU core it creates one per GPU. It also changes some of the configuration defaults to ensure good performance of GPU workloads.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_cuda&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;LocalCUDACluster&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCUDACluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;cluster_specific_kwargs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This package also has an alternative Dask worker bash command called &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-cuda-worker&lt;/span&gt;&lt;/code&gt; which also modified the defaults of the Dask worker to ensure it is optimised for GPU 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/2020/07/23/current-state-of-distributed-dask-clusters.md&lt;/span&gt;, line 309)&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="future"&gt;
&lt;h1&gt;Future&lt;/h1&gt;
&lt;p&gt;Now that we have laid out the current state of the Dask distributed cluster ecosystem let’s discuss where we could go next.&lt;/p&gt;
&lt;p&gt;As shown at the beginning a Dask cluster is a combination of scheduler, workers and client which enable distributed execution of Python functions. Setting up your own cluster on your own machines is straight forward, but there is such a variety of ways to provision infrastructure that we now have a number of ways of automating this.&lt;/p&gt;
&lt;p&gt;This variation opens up a number of questions about how we can improve things.&lt;/p&gt;
&lt;section id="do-we-need-more-fixed-cluster-options"&gt;
&lt;h2&gt;Do we need more fixed cluster options?&lt;/h2&gt;
&lt;p&gt;While covering the various cluster managers we only covered one fixed cluster implementation, the Helm chart. Is there a requirement for more fixed clusters? Examples may be &lt;a class="reference external" href="https://aws.amazon.com/cloudformation/"&gt;CloudFormation&lt;/a&gt; or &lt;a class="reference external" href="https://www.terraform.io/"&gt;Terraform&lt;/a&gt; templates which follow the same structure as the Helm chart, providing a Jupyter service, Dask scheduler and fixed number of workers.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="can-we-bridge-some-gaps"&gt;
&lt;h2&gt;Can we bridge some gaps?&lt;/h2&gt;
&lt;p&gt;Could the Dask Kubernetes cluster manager connect to an existing cluster that was built using the Helm chart to then perform adaptive scaling? I’ve been asked this a lot but it is currently unclear how to get to this position. The cluster manager and Helm chart use different Kubernetes resources to achieve the same goal, so some unification would be needed before this is possible.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="are-ephemeral-clusters-too-ephemeral"&gt;
&lt;h2&gt;Are ephemeral clusters too ephemeral?&lt;/h2&gt;
&lt;p&gt;Many of the cluster managers only exist for the duration of the Python session. However some like the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;YarnCluster&lt;/span&gt;&lt;/code&gt; allow you to disconnect and reconnect from the cluster. This allows you to treat a YARN cluster more like a fixed cluster.&lt;/p&gt;
&lt;p&gt;In other circumstances the Python session may have a timeout or limit and may be killed before the Dask cluster can complete its work. Would there be benefit to letting the Dask cluster continue to exist? With the Python session cleared up the client and futures will also be garbage collected. So perhaps not.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="can-we-manage-conda-environments-better"&gt;
&lt;h2&gt;Can we manage conda environments better?&lt;/h2&gt;
&lt;p&gt;Currently it is the responsibility of the person creating the cluster to ensure that the worker’s conda environment matches the one where the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client&lt;/span&gt;&lt;/code&gt; is going to be created. On fixed clusters this can be easier as the Python/Jupyter environment can be provided within the same set of systems. However on ephemeral clusters where you may be reaching into a cloud or batch system they may not match your laptop’s environment for example.&lt;/p&gt;
&lt;p&gt;Perhaps there could be integration between workers and conda to create dynamic environments on the fly. Exploring the performance impact of this would be interesting.&lt;/p&gt;
&lt;p&gt;Another option could be enabling users to start a remote Jupyter kernel on a worker. They wouldn’t have access to the same filesystem, but they would share a conda environment.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2020/07/23/current-state-of-distributed-dask-clusters/"/>
    <summary>Dask enables you to build up a graph of the computation you want to perform and then executes it in parallel for you. This is great for making best use of your computer’s hardware. It is also great when you want to expand beyond the limits of a single machine.</summary>
    <published>2020-07-23T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2020/07/21/faster-scheduling/</id>
    <title>Faster Scheduling</title>
    <updated>2020-07-21T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin (Coiled)</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/21/faster-scheduling.md&lt;/span&gt;, line 8)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;This post discusses Dask overhead costs for task scheduling,
and then lays out a rough plan for acceleration.&lt;/p&gt;
&lt;p&gt;This post is written for other maintainers, and often refers to internal
details. It is not intended for broad readability.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/21/faster-scheduling.md&lt;/span&gt;, line 16)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="how-does-this-problem-present"&gt;
&lt;h1&gt;How does this problem present?&lt;/h1&gt;
&lt;p&gt;When we submit large graphs there is a bit of a delay between us calling
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.compute()&lt;/span&gt;&lt;/code&gt; and work actually starting on the workers. In some cases, that
delay can affect usability and performance.&lt;/p&gt;
&lt;p&gt;Additionally, in far fewer cases, the gaps in between tasks can be an issue,
especially if those tasks are very short and for some reason can not be made
longer.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/21/faster-scheduling.md&lt;/span&gt;, line 26)&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-cares"&gt;
&lt;h1&gt;Who cares?&lt;/h1&gt;
&lt;p&gt;First, this is a problem that affects about 1-5% of Dask users. These are people
who want to process millions of tasks relatively quickly. Let’s list a few use
cases:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Xarray/Pangeo workloads at the 10-100TB scale&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;NVIDIA RAPIDS workloads on large tabular data (GPUs make computing fast, so other costs become
relatively larger)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Some mystery use cases inside of some hedge funds&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;It does not affect the everyday user, who processes 100GB to a few TB of data,
and doesn’t mind waiting 10s for things to start running.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/21/faster-scheduling.md&lt;/span&gt;, line 40)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="coarse-breakdown-of-costs"&gt;
&lt;h1&gt;Coarse breakdown of costs&lt;/h1&gt;
&lt;p&gt;When you call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x.sum().compute()&lt;/span&gt;&lt;/code&gt; a few things happen:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Graph generation:&lt;/strong&gt; Some Python code in a Dask collection library, like
dask array, calls the sum function, which generates a task graph on the
client side.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Graph Optimization:&lt;/strong&gt; We then optimize that graph, also on the client
side, in order to remove unnecessary work, fuse tasks, apply important high
level optimizations, and more.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Graph Serializtion:&lt;/strong&gt; We now pack up that graph in a form that can be
sent over to the scheduler.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Graph Communication:&lt;/strong&gt; We fire those bytes across a wire over to the
scheduler&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Scheduler.update_graph:&lt;/strong&gt; The scheduler receives these bytes, unpacks
them, and then updates its own internal data structures&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Scheduling:&lt;/strong&gt; The scheduler then assigns ready tasks to workers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Communicate to workers:&lt;/strong&gt; The scheduler sends out lots of smaller
messages to each of the workers with the tasks that they can perform&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Workers work:&lt;/strong&gt; The workers then perform this work, and start
communicating back and forth with the scheduler to receive new instructions&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Generally most people today are concerned with steps 1-6. Once things get out
to the workers and progress bars start moving people tend to care a bit less
(but not zero).&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/21/faster-scheduling.md&lt;/span&gt;, line 66)&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-do-other-people-do"&gt;
&lt;h1&gt;What do other people do?&lt;/h1&gt;
&lt;p&gt;Let’s look at a few things that other projects do, and see if there are things
that we can learn. These are commonly suggested, but there are challenges with
most of them.&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Rewrite the scheduler it in C++/Rust/C/Cython&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;Proposal: Python is slow. Want to make it faster? Don’t use Python. See
academic projects.&lt;/p&gt;
&lt;p&gt;Challenge: This makes sense for some parts of the pipeline above, but not
for others. It also makes it harder to attract maintainers.&lt;/p&gt;
&lt;p&gt;What we should consider: Some parts of the scheduler and optimization
algorithms could be written in a lower level language, maybe Cython. We’ll
need to be careful about maintainability.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Distributed scheduling&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;Proposal: The scheduler is slow, maybe have many schedulers? See Ray.&lt;/p&gt;
&lt;p&gt;Challenge: It’s actually really hard to make the kinds of decisions that
Dask has to make if scheduling state is spread on many computers.
Distributed scheduling works better when the workload is very either
uniform or highly decoupled.
Distributed scheduling is really attractive to people who like solving
interesting/hard problems.&lt;/p&gt;
&lt;p&gt;What we should consider: We can move some simple logic down to the workers.
We’ve already done this with the easy stuff though.
It’s not clear how much additional benefit there is here.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Build specialty scheduling around collections&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;Proposal: If Dask were to become just a dataframe library or just an array
computing library then it could special-case things more effectively. See
Spark, Mars, and others.&lt;/p&gt;
&lt;p&gt;Challenge: Yes, but Dask is not a dataframe library or an array library.
The three use cases we mention above are all very different.&lt;/p&gt;
&lt;p&gt;What we should consider: modules like dask array and dask dataframe should
develop high level query blocks, and we should endeavor to
communicate these subgraphs over the wire directly so that they are more
compact.&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/2020/07/21/faster-scheduling.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="what-should-we-actually-do"&gt;
&lt;h1&gt;What should we actually do?&lt;/h1&gt;
&lt;p&gt;Because our pipeline has many stages, each of which can be slow for different
reasons, we’ll have to do many things. Additionally, this is a hard problem
because changing one piece of the project at this level has repurcussions for
many other pieces. The rest of this post tries to lay out a consistent set of
changes. Let’s start with a summary:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;For Dask array/dataframe let’s use high level graphs more aggressively so
that we can communicate only abstract representations between the client
and scheduler.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;But this breaks low level graph optimizations, fuse, cull, and slice fusion
in particular. We can make these unnecessary with two changes:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;We can make high level graphs considerably smarter to handle cull and slice fusion&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We can move a bit more of the scheduling down to the workers to
replicate the advantages of low-level fusion there&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Then, once all of the graph manipulation happens on the scheduler, let’s
try to accelerate it, hopefully in a language that the current dev
community can understand, like Cython&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;At the same time in parallel, let’s take a look at our network stack&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We’ll go into these in more depth below&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/21/faster-scheduling.md&lt;/span&gt;, line 136)&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="graph-generation"&gt;
&lt;h1&gt;Graph Generation&lt;/h1&gt;
&lt;section id="high-level-graph-history"&gt;
&lt;h2&gt;High Level Graph History&lt;/h2&gt;
&lt;p&gt;A year or two ago we moved graph generation costs from user-code-typing time to
graph-optimization-time with high level 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="n"&gt;y&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="mi"&gt;1&lt;/span&gt;                 &lt;span class="c1"&gt;# graph generation used to happen here&lt;/span&gt;
&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;,)&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;optimize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;,)&lt;/span&gt;  &lt;span class="c1"&gt;# now it happens here&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This really improved usability, and also let us do some high level
optimizations which sometimes allowed us to skip some lower-level optimization
costs.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="can-we-push-this-further"&gt;
&lt;h2&gt;Can we push this further?&lt;/h2&gt;
&lt;p&gt;The first four stages of our pipeline happen on the client:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Graph generation:&lt;/strong&gt; Some Python code in a Dask collection library, like
dask array, calls the sum function, which generates a task graph on the
client side.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Graph Optimization:&lt;/strong&gt; We then optimize that graph, also on the client
side, in order to remove unnecessary work, fuse tasks, apply important high
level optimizations, and more.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Graph Serializtion:&lt;/strong&gt; We now pack up that graph in a form that can be
sent over to the scheduler.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Graph Communication:&lt;/strong&gt; We fire those bytes across a wire over to the
scheduler&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;If we’re able to stay with the high level graph representation through these
stages all the way until graph communication, then we can communicate a far
more compact representation up to the scheduler. We can drop a lot of these
costs, at least for the high level collection APIs (delayed and client.submit
would still be slow, client.map might be ok though).&lt;/p&gt;
&lt;p&gt;This has a couple of other nice benefits:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;User’s code won’t block, and we can alert the user immediately that we’re
on the job&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We’ve centralized costs in just the scheduler,
so there is now only one place where we might have to think about low-level code&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;(some conversation here: https://github.com/dask/distributed/issues/3872)&lt;/p&gt;
&lt;/section&gt;
&lt;section id="however-low-level-graph-optimizations-are-going-to-be-a-problem"&gt;
&lt;h2&gt;However, low-level graph optimizations are going to be a problem&lt;/h2&gt;
&lt;p&gt;In principle changing the distributed scheduler to accept a variety of graph
layer types is a tedious but straightforward problem. I’m not concerned.&lt;/p&gt;
&lt;p&gt;The bigger concern is what to do with low-level graph optimizations.
Today we have three of these that really matter:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Task fusion: this is what keeps your &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_parquet&lt;/span&gt;&lt;/code&gt; task merged with your
subsequent blockwise tasks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Culling: this is what makes &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.head()&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x[0]&lt;/span&gt;&lt;/code&gt; fast&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Slice fusion: this is why &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x[:100][5]&lt;/span&gt;&lt;/code&gt; works well&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In order for us to transmit abstract graph layers up to the scheduler, we need
to remove the need for these low level graph optimizations. I think that we
can do this with a combination of two approaches:&lt;/p&gt;
&lt;section id="more-clever-high-level-graph-manipulation"&gt;
&lt;h3&gt;More clever high level graph manipulation&lt;/h3&gt;
&lt;p&gt;We already do this a bit with blockwise, which has its own fusion, and which
removes much of the need for fusion generally. But other blockwise-like
operations, like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_*&lt;/span&gt;&lt;/code&gt; will probably have to join the Blockwise family.&lt;/p&gt;
&lt;p&gt;Getting culling to work properly may require us to teach each of the individual
graph layers how to track dependencies in each layer type and cull themselves.
This may get tricky.&lt;/p&gt;
&lt;p&gt;Slicing is doable, we just need someone to go in, grok all of the current
slicing optimizations, and make high level graph layers for these
computations. This would be a great project for a sharp masters student&lt;/p&gt;
&lt;/section&gt;
&lt;section id="send-speculative-tasks-to-the-workers"&gt;
&lt;h3&gt;Send speculative tasks to the workers&lt;/h3&gt;
&lt;p&gt;High level Blockwise fusion handles many of the use cases for low-level fusion,
but not all. For example I/O layers like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dd.read_parquet&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;da.from_zarr&lt;/span&gt;&lt;/code&gt;
aren’t fused at a high level.&lt;/p&gt;
&lt;p&gt;We can resolve this either by making them blockwise layers (this requires
expanding the blockwise abstraction, which may be hard) or alternatively we can
start sending not-yet-ready tasks to workers before all of their dependencies
are finished if we’re highly confident that we know where they’re going to go.
This would give us some of the same results of fusion, but would keep all of
the task types separate (which would be nice for diagnostics) and might still
give us some of the same performance benefits that we get from fusion.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="unpack-abstract-graph-layers-on-the-scheduler"&gt;
&lt;h2&gt;Unpack abstract graph layers on the scheduler&lt;/h2&gt;
&lt;p&gt;So after we’ve removed the need for low level optimizations, and we just send
the abstract graph layers up to the scheduler directly, we’ll need to teach the
scheduler how to unpack those graph layers.&lt;/p&gt;
&lt;p&gt;This is a little tricky because the Scheduler can’t run user Python code (for
security reasons). We’ll have to register layer types (like blockwise,
rechunk, dataframe shuffle) that the scheduler knows about and trusts ahead of
time. We’ll still always support custom layers, and these will be at the same
speed that they’ve always been, but hopefully there will be far less need for
these if we go all-in on high level layers.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/21/faster-scheduling.md&lt;/span&gt;, line 240)&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="rewrite-scheduler-in-low-level-language"&gt;
&lt;h1&gt;Rewrite scheduler in low-level language&lt;/h1&gt;
&lt;p&gt;Once most of the finicky bits are moved to the scheduler, we’ll have one place
where we can focus on low level graph state manipulation.&lt;/p&gt;
&lt;p&gt;Dask’s distributed scheduler is two things:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;A Tornado TCP application that receives signals from clients and workers
and send signals out to clients and workers&lt;/p&gt;
&lt;p&gt;This is async-heavy networking code&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A complex state machine internally that responds to those state changes&lt;/p&gt;
&lt;p&gt;This is a complex data structure heavy Python code&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;section id="networking"&gt;
&lt;h2&gt;Networking&lt;/h2&gt;
&lt;p&gt;Jim has an interesting project here that shows promise: https://github.com/jcrist/ery
Reducing latency between workers and the scheduler would be good, and would
help to accelerate stages 7-8 in the pipeline listed at the top of this post.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="state-machine"&gt;
&lt;h2&gt;State machine&lt;/h2&gt;
&lt;p&gt;Rewriting the state machine in some lower level language would be fine.
Ideally this would be in a language that was easy for the current maintainer
community to maintain, (Cython?) but we may also consider making a more firm
interface here that would allow other groups to experiment safely.&lt;/p&gt;
&lt;p&gt;There are some advantages to this (more experimentation by different groups)
but also some costs (splitting of core efforts and mismatches for users).
Also, I suspect that splitting out also probably means that we’ll probably lose the dashboard,
unless those other groups are very careful to expose the same state to Bokeh.&lt;/p&gt;
&lt;p&gt;There is more exploration to do here. Regardless I think that it probably makes
sense to try to isolate the state machine from the networking system.
Maybe this also makes it easier for people to profile in isolation.&lt;/p&gt;
&lt;p&gt;In speaking with a few different groups most people have expressed reservation
about having multiple different state machine codes. This was done in
MapReduce and Spark and resulted in difficult to maintain community dynamics.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/21/faster-scheduling.md&lt;/span&gt;, line 282)&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="high-level-graph-optimizations"&gt;
&lt;h1&gt;High Level Graph Optimizations&lt;/h1&gt;
&lt;p&gt;Once we have everything in smarter high level graph layers,
we will also be more ripe for optimization.&lt;/p&gt;
&lt;p&gt;We’ll need a better way to write down these optimizations with a separated
traversal system and a set of rules. A few of us have
written these things before, maybe it’s time we revisit 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/2020/07/21/faster-scheduling.md&lt;/span&gt;, line 291)&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-we-need"&gt;
&lt;h1&gt;What we need&lt;/h1&gt;
&lt;p&gt;This would require some effort, but I think that it would hit several high
profile problems at once. There are a few tricky things to get right:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;A framework for high level graph layers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;An optimization system for high level graph layers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Separation of the scheduler into two parts&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;For this I think that we’ll need people who are fairly familiar with Dask to do this right.&lt;/p&gt;
&lt;p&gt;And there there is a fair amount of follow-on work&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Build a hierarchy of layers for dask dataframe&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Build a hierarchy of layers for dask array&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Build optimizations for those to remove the need for low level graph
optimizations&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Rewrite core parts of the scheduler in Cython&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Experiment with the networking layer, maybe with a new Comm&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;I’ve been thinking about the right way to enact this change.
Historically most Dask changes over the past few years have been incremental or
peripheral, due to how burdened the maintainers are. There might be enough
pressure on this problem though that we can get some dedicated engineering
effort from a few organizations though, which might change how possible this is.
We’ve gotten 25% time from a few groups. I’m curious if we can gate 100% time
for some people for a few months.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2020/07/21/faster-scheduling/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <published>2020-07-21T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2020/07/17/scipy-2020-maintainers-track/</id>
    <title>Last Year in Review</title>
    <updated>2020-07-17T00:00:00+00:00</updated>
    <author>
      <name>Jacob Tomlinson (NVIDIA)</name>
    </author>
    <content type="html">&lt;p&gt;We recently enjoyed the 2020 SciPy conference from the comfort of our own homes this year. The 19th annual Scientific Computing with Python conference was a virtual conference this year due to the global pandemic. The annual SciPy Conference brought together over 1500 participants from industry, academia, and government to showcase their latest projects, learn from skilled users and developers, and collaborate on code development.&lt;/p&gt;
&lt;p&gt;As part of the maintainers track we presented an update on 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/2020/07/17/scipy-2020-maintainers-track.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="video"&gt;

&lt;p&gt;You can find the video on the SciPy YouTube channel. The Dask update runs from 0:00-19:30.&lt;/p&gt;
&lt;iframe width="560" height="315" src="https://www.youtube.com/embed/XC0M76CmzHg" frameborder="0" allow="accelerometer; autoplay; encrypted-media; gyroscope; picture-in-picture" allowfullscreen&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/2020/07/17/scipy-2020-maintainers-track.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&gt;
&lt;section id="slides"&gt;
&lt;h1&gt;Slides&lt;/h1&gt;
&lt;script async class="speakerdeck-embed" data-id="ae0f04df5b7341eaa3e2989221be1889" data-ratio="1.77777777777778" src="//speakerdeck.com/assets/embed.js"&gt;&lt;/script&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/07/17/scipy-2020-maintainers-track.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="talk-summary"&gt;
&lt;h1&gt;Talk Summary&lt;/h1&gt;
&lt;p&gt;Here’s a summary of the main topics covered in the talk. You can also check out the &lt;a class="reference external" href="https://threadreaderapp.com/thread/1280885850914553856.html"&gt;original thread on Twitter&lt;/a&gt;.&lt;/p&gt;
&lt;section id="community-overview"&gt;
&lt;h2&gt;Community overview&lt;/h2&gt;
&lt;p&gt;We’ve been trying to gauge the size of our community lately. The best proxy we have right now is the number of weekly visitors to the &lt;a class="reference external" href="https://docs.dask.org/en/latest/"&gt;Dask documentation&lt;/a&gt;. Which currently stands at around 10,000.&lt;/p&gt;
&lt;img alt="Dask documentation analytics showing growth to 10,000 weekly users over the last four years" src="https://pbs.twimg.com/media/EcaS9DpWkAEBaB4.jpg" style="width: 100%;" /&gt;
&lt;p&gt;Dask also came up in the &lt;a class="reference external" href="https://www.jetbrains.com/lp/devecosystem-2020/python/"&gt;Jetbrains Python developer survey&lt;/a&gt;. We were excited to see 5% of all the Python developers who filled out the survey said they use Dask. Which shows health in the PyData community as well as Dask.&lt;/p&gt;
&lt;img alt="Jetbrains survey results showing Dask used by 5% of Python users, beaten only by the Spark/hadoop ecosystem" src="https://pbs.twimg.com/media/EcaTTuiX0AIT2KB.jpg" style="width: 100%;" /&gt;
&lt;p&gt;We are running &lt;a class="reference external" href="https://dask.org/survey"&gt;our own survey&lt;/a&gt; at the moment. If you are a Dask user please take a few minutes to fill it out. We would really appreciate it.&lt;/p&gt;
&lt;img alt="Link to the Dask survey" src="https://pbs.twimg.com/media/EcaTlITXYAAVs-y.jpg" style="width: 100%;" /&gt;
&lt;/section&gt;
&lt;section id="community-events"&gt;
&lt;h2&gt;Community events&lt;/h2&gt;
&lt;p&gt;In February we had an in-person &lt;a class="reference external" href="https://blog.dask.org/2020/04/28/dask-summit"&gt;Dask Summit&lt;/a&gt; where a mixture of OSS maintainers and institutional users met. We had talks and workshops to help figure out our challenges and set our direction.&lt;/p&gt;
&lt;img alt="A room of attendees at the Dask summit" src="https://pbs.twimg.com/media/EcaUbHLXQAAHckq.jpg" style="width: 100%;" /&gt;
&lt;p&gt;The Dask community also has a &lt;a class="reference external" href="https://docs.dask.org/en/latest/support.html"&gt;monthly meeting&lt;/a&gt;! It is held on the first Thursday of the month at 10:00 US Central Time. If you’re a Dask user you are welcome to come to hear updates from maintainers and share what you’re working on.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="community-projects"&gt;
&lt;h2&gt;Community projects&lt;/h2&gt;
&lt;p&gt;There are many projects built on Dask. Looking at the preliminary results from the 2020 Dask survey shows some that are especially popular.&lt;/p&gt;
&lt;img alt="Graph showing the most popular projects built on Dask; Xarray, RAPIDS, XGBoost, Prefect and Iris" src="https://pbs.twimg.com/media/EcaVSHpX0AAMDYs.png" style="width: 100%;" /&gt;
&lt;p&gt;Let’s take a look at each of those.&lt;/p&gt;
&lt;section id="xarray"&gt;
&lt;h3&gt;Xarray&lt;/h3&gt;
&lt;p&gt;&lt;a class="reference external" href="https://xarray.pydata.org/en/stable/"&gt;Xarray&lt;/a&gt; allows you to work on multi-dimensional datasets that have supporting metadata arrays in a Pandas-like way.&lt;/p&gt;
&lt;img alt="Slide showing xarray code example" src="https://pbs.twimg.com/media/EcaVbOaXkAMQ4SU.jpg" style="width: 100%;" /&gt;
&lt;/section&gt;
&lt;section id="rapids"&gt;
&lt;h3&gt;RAPIDS&lt;/h3&gt;
&lt;p&gt;&lt;a class="reference external" href="https://rapids.ai/"&gt;RAPIDS&lt;/a&gt; is an open-source suite of GPU accelerated Python libraries. Using these tools you can execute end-to-end data science and analytics pipelines entirely on GPUs. All using familiar PyData APIs.&lt;/p&gt;
&lt;img alt="Slide showing RAPIDS dataframe code example" src="https://pbs.twimg.com/media/EcaWFfDXkAEX4B_.jpg" style="width: 100%;" /&gt;
&lt;/section&gt;
&lt;section id="blazingsql"&gt;
&lt;h3&gt;BlazingSQL&lt;/h3&gt;
&lt;p&gt;&lt;a class="reference external" href="https://blazingsql.com"&gt;BlazingSQL&lt;/a&gt; builds on RAPIDS and Dask to provide an open-source distributed, GPU accelerated SQL engine.&lt;/p&gt;
&lt;img alt="Slide showing BlazingSQL code example" src="https://pbs.twimg.com/media/EcaWW_CXsAM7XP7.jpg" style="width: 100%;" /&gt;
&lt;/section&gt;
&lt;section id="xgboost"&gt;
&lt;h3&gt;XGBoost&lt;/h3&gt;
&lt;p&gt;While &lt;a class="reference external" href="https://examples.dask.org/machine-learning/xgboost.html"&gt;XGBoost&lt;/a&gt; has been around for a long time you can now prepare your data on your Dask cluster and then bootstrap your XGBoost cluster on top of Dask and hand the distributed dataframes straight over.&lt;/p&gt;
&lt;img alt="Slide showing XGBoost code example" src="https://pbs.twimg.com/media/EcaXKlRWsAAjLYe.jpg" style="width: 100%;" /&gt;
&lt;/section&gt;
&lt;section id="prefect"&gt;
&lt;h3&gt;Prefect&lt;/h3&gt;
&lt;p&gt;&lt;a class="reference external" href="https://www.prefect.io/"&gt;Prefect&lt;/a&gt; is a workflow manager which is built on top of Dask’s scheduling engine. “Users organize Tasks into Flows, and Prefect takes care of the rest.”&lt;/p&gt;
&lt;img alt="Slide showing Prefect code example" src="https://pbs.twimg.com/media/EcaXlf-XYAEPY-Z.jpg" style="width: 100%;" /&gt;
&lt;/section&gt;
&lt;section id="iris"&gt;
&lt;h3&gt;Iris&lt;/h3&gt;
&lt;p&gt;&lt;a class="reference external" href="https://scitools.org.uk/iris/docs/latest/"&gt;Iris&lt;/a&gt;, part of the &lt;a class="reference external" href="https://scitools.org.uk"&gt;SciTools&lt;/a&gt; suite of tools, uses the CF data model giving you a format-agnostic interface for working with your data. It excels when working with multi-dimensional Earth Science data, where tabular representations become unwieldy and inefficient.&lt;/p&gt;
&lt;img alt="Slide showing Iris code example" src="https://pbs.twimg.com/media/EcaX3S9XsAAU-Sm.jpg" style="width: 100%;" /&gt;
&lt;/section&gt;
&lt;section id="more-tools"&gt;
&lt;h3&gt;More tools&lt;/h3&gt;
&lt;p&gt;These are the tools our community have told us they like so far. But if you use something which didn’t make the list then head to &lt;a class="reference external" href="https://dask.org/survey"&gt;our survey&lt;/a&gt; and let us know! According to PyPI there are many more out there.&lt;/p&gt;
&lt;img alt="Screenshot of PyPI showing 239 packages with Dask in their name" src="https://pbs.twimg.com/media/EcaYZmPWoAANYhr.jpg" style="width: 100%;" /&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="user-groups"&gt;
&lt;h2&gt;User groups&lt;/h2&gt;
&lt;p&gt;There are many user groups who use Dask. Everything from life sciences, geophysical sciences and beamline facilities to finance, retail and logistics. Check out the great &lt;a class="reference external" href="https://youtu.be/t_GRK4L-bnw"&gt;“Who uses Dask?” talk&lt;/a&gt; from &lt;a class="reference external" href="https://twitter.com/mrocklin"&gt;Matthew Rocklin&lt;/a&gt; for more info.&lt;/p&gt;
&lt;img alt="Screenshot 'Who uses Dask?' YouTube video" src="https://pbs.twimg.com/media/EcaYj2JXQAEvgV3.jpg" style="width: 100%;" /&gt;
&lt;/section&gt;
&lt;section id="for-profit-companies"&gt;
&lt;h2&gt;For profit companies&lt;/h2&gt;
&lt;p&gt;There has been an increase in for-profit companies building tools with Dask. Including &lt;a class="reference external" href="https://coiled.io/"&gt;Coiled Computing&lt;/a&gt;, &lt;a class="reference external" href="https://www.prefect.io/"&gt;Prefect&lt;/a&gt; and &lt;a class="reference external" href="https://www.saturncloud.io/s/"&gt;Saturn Cloud&lt;/a&gt;.&lt;/p&gt;
&lt;img alt="Slide describing the for-profit companies Coiled, Prefect and Saturn Cloud" src="https://pbs.twimg.com/media/EcaZOqgX0AABFpQ.jpg" style="width: 100%;" /&gt;
&lt;p&gt;We’ve also seen large companies like Microsoft’s &lt;a class="reference external" href="https://azure.microsoft.com/en-gb/services/machine-learning/"&gt;Azure ML&lt;/a&gt; team contributing a cluster manager to &lt;a class="reference external" href="https://cloudprovider.dask.org/en/latest/#azure"&gt;Dask Cloudprovider&lt;/a&gt;. This helps folks get up and running with Dask on AzureML quicker and easier.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="recent-improvements"&gt;
&lt;h2&gt;Recent improvements&lt;/h2&gt;
&lt;section id="communications"&gt;
&lt;h3&gt;Communications&lt;/h3&gt;
&lt;p&gt;Moving on to recent improvements there has been a lot of work to get &lt;a class="reference external" href="https://www.openucx.org/"&gt;Open UCX&lt;/a&gt; supported as a protocol in Dask. Which allows worker-worker communication to be accelerated vastly with hardware that supports &lt;a class="reference external" href="https://en.wikipedia.org/wiki/InfiniBand"&gt;Infiniband&lt;/a&gt; or &lt;a class="reference external" href="https://en.wikipedia.org/wiki/NVLink"&gt;NVLink&lt;/a&gt;.&lt;/p&gt;
&lt;img alt="Slide showing worker communication comparison between UCX/Infiniband and TCP with UCX being much faster" src="https://pbs.twimg.com/media/EcaaTxiXQAE4TD0.jpg" style="width: 100%;" /&gt;
&lt;p&gt;There have also been some &lt;a class="reference external" href="https://blogs.nvidia.com/blog/2020/06/22/big-data-analytics-tpcx-bb/"&gt;recent announcements&lt;/a&gt; around NVIDIA blowing away the TPCx-BB benchmark by outperforming the current leader by 20x. This is a huge success for all the open-source projects that were involved, including Dask.&lt;/p&gt;
&lt;img alt="Slide showing TPCx-BB benchmark results" src="https://pbs.twimg.com/media/EcabNUVWoAQGy8e.jpg" style="width: 100%;" /&gt;
&lt;/section&gt;
&lt;section id="dask-gateway"&gt;
&lt;h3&gt;Dask Gateway&lt;/h3&gt;
&lt;p&gt;We’ve seen increased adoption of &lt;a class="reference external" href="https://gateway.dask.org"&gt;Dask Gateway&lt;/a&gt;. Many institutions are using it as a way to provide their staff with on-demand Dask clusters.&lt;/p&gt;
&lt;img alt="Slide showing Dask Gateway overview" src="https://pbs.twimg.com/media/EcabpirWkAYtx-W.jpg" style="width: 100%;" /&gt;
&lt;/section&gt;
&lt;section id="cluster-map-plot-aka-pew-pew-pew"&gt;
&lt;h3&gt;Cluster map plot (aka ‘pew pew pew’)&lt;/h3&gt;
&lt;p&gt;The update that got the most 👏 feedback from the SciPy 2020 attendees was the Cluster Map Plot (known to maintainers as the “pew pew pew” plot). This plot shows a high-level overview of your Dask cluster scheduler and workers and the communication between them.&lt;/p&gt;
&lt;p&gt;&lt;video autoplay="" loop="" controls="" poster="https://pbs.twimg.com/tweet_video_thumb/EcacHRcXkAE53eI.jpg"&gt;&lt;source src="https://video.twimg.com/tweet_video/EcacHRcXkAE53eI.mp4" type="video/mp4"&gt;&lt;img alt="" src="https://pbs.twimg.com/tweet_video_thumb/EcacHRcXkAE53eI.jpg"&gt;&lt;/video&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="next-steps"&gt;
&lt;h2&gt;Next steps&lt;/h2&gt;
&lt;section id="high-level-graph-optimization"&gt;
&lt;h3&gt;High-level graph optimization&lt;/h3&gt;
&lt;p&gt;To wrap up with what Dask is going to be doing next we are going to be continuing to work on high-level graph optimization.&lt;/p&gt;
&lt;img alt="Slide showing High Level Graph documentation page" src="https://pbs.twimg.com/media/EcacZvfWsAIfqTz.jpg" style="width: 100%;" /&gt;
&lt;/section&gt;
&lt;section id="scheduler-performance"&gt;
&lt;h3&gt;Scheduler performance&lt;/h3&gt;
&lt;p&gt;With feedback from our community we are also going to be focussing on making the &lt;a class="reference external" href="https://github.com/dask/distributed/issues/3783"&gt;Dask scheduler more performant&lt;/a&gt;. There are a few things happening including a Rust implementation of the scheduler, dynamic task creation and ongoing benchmarking.&lt;/p&gt;
&lt;img alt="Scheduler performance tasks including a Rust implementation, benchmarking, dynamic tasks and Cython, PyPy and C experiments" src="https://pbs.twimg.com/media/Ecacr6pWoAEd4Tx.jpg" style="width: 100%;" /&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="chan-zuckerberg-foundation-maintainer-post"&gt;
&lt;h2&gt;Chan Zuckerberg Foundation maintainer post&lt;/h2&gt;
&lt;p&gt;Lastly I’m excited to share that with funding from the &lt;a class="reference external" href="https://chanzuckerberg.com/eoss/proposals/scaling-python-with-dask/"&gt;Chan Zuckerberg Foundation&lt;/a&gt;, Dask will be hiring a maintainer who will focus on growing usage in the biological sciences field. If that is of interest to you keep an eye on &lt;a class="reference external" href="https://twitter.com/dask_dev"&gt;our twitter account&lt;/a&gt; for more announcements.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2020/07/17/scipy-2020-maintainers-track/"/>
    <summary>We recently enjoyed the 2020 SciPy conference from the comfort of our own homes this year. The 19th annual Scientific Computing with Python conference was a virtual conference this year due to the global pandemic. The annual SciPy Conference brought together over 1500 participants from industry, academia, and government to showcase their latest projects, learn from skilled users and developers, and collaborate on code development.</summary>
    <category term="Community" label="Community"/>
    <category term="SciPy" label="SciPy"/>
    <category term="Talk" label="Talk"/>
    <published>2020-07-17T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2020/05/13/large-svds/</id>
    <title>Large SVDs</title>
    <updated>2020-05-13T00:00:00+00:00</updated>
    <author>
      <name>Alistair Miles (Oxford University)</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/05/13/large-svds.md&lt;/span&gt;, line 10)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;We perform Singular Value Decomposition (SVD) calculations on large datasets.&lt;/p&gt;
&lt;p&gt;We modify the computation both by using fully precise and approximate methods,
and by using both CPUs and GPUs.&lt;/p&gt;
&lt;p&gt;In the end we compute an approximate SVD of 200GB of simulated data and using a mutli-GPU machine in 15-20 seconds.
Then we run this from a dataset stored in the cloud
where we find that I/O is, predictably, a major bottleneck.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/05/13/large-svds.md&lt;/span&gt;, line 21)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="svd-the-simple-case"&gt;
&lt;h1&gt;SVD - The simple case&lt;/h1&gt;
&lt;p&gt;Dask arrays contain a relatively sophisticated SVD algorithm that works in the
tall-and-skinny or short-and-fat cases, but not so well in the roughly-square
case. It works by taking QR decompositions of each block of the array,
combining the R matrices, doing another smaller SVD on those, and then
performing some matrix multiplication to get back to the full result. It’s
numerically stable and decently fast, assuming that the intermediate R
matrices of the QR decompositions mostly fit in memory.&lt;/p&gt;
&lt;p&gt;The memory constraints here are that if you have an &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n&lt;/span&gt;&lt;/code&gt; by &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;m&lt;/span&gt;&lt;/code&gt; tall and
skinny array (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n&lt;/span&gt; &lt;span class="pre"&gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="pre"&gt;m&lt;/span&gt;&lt;/code&gt;) cut into &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;k&lt;/span&gt;&lt;/code&gt; blocks then you need to have about &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;m**2&lt;/span&gt; &lt;span class="pre"&gt;*&lt;/span&gt; &lt;span class="pre"&gt;k&lt;/span&gt;&lt;/code&gt; space. This is true in many cases, including typical PCA machine learning
workloads, where you have tabular data with a few columns (hundreds at most)
and many rows.&lt;/p&gt;
&lt;p&gt;It’s easy to use and quite robust.&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;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="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;10000000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;x&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table&gt;
&lt;tr&gt;
&lt;td&gt;
&lt;table&gt;  &lt;thead&gt;    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 1.60 GB &lt;/td&gt; &lt;td&gt; 100.00 MB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (10000000, 20) &lt;/td&gt; &lt;td&gt; (625000, 20) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 16 Tasks &lt;/td&gt;&lt;td&gt; 16 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; float64 &lt;/td&gt;&lt;td&gt; numpy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;&lt;/table&gt;
&lt;/td&gt;
&lt;td&gt;
&lt;svg width="75" height="170" style="stroke:rgb(0,0,0);stroke-width:1" &gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="0" y1="0" x2="25" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="0" y1="7" x2="25" y2="7" /&gt;
  &lt;line x1="0" y1="15" x2="25" y2="15" /&gt;
  &lt;line x1="0" y1="22" x2="25" y2="22" /&gt;
  &lt;line x1="0" y1="30" x2="25" y2="30" /&gt;
  &lt;line x1="0" y1="37" x2="25" y2="37" /&gt;
  &lt;line x1="0" y1="45" x2="25" y2="45" /&gt;
  &lt;line x1="0" y1="52" x2="25" y2="52" /&gt;
  &lt;line x1="0" y1="60" x2="25" y2="60" /&gt;
  &lt;line x1="0" y1="67" x2="25" y2="67" /&gt;
  &lt;line x1="0" y1="75" x2="25" y2="75" /&gt;
  &lt;line x1="0" y1="82" x2="25" y2="82" /&gt;
  &lt;line x1="0" y1="90" x2="25" y2="90" /&gt;
  &lt;line x1="0" y1="97" x2="25" y2="97" /&gt;
  &lt;line x1="0" y1="105" x2="25" y2="105" /&gt;
  &lt;line x1="0" y1="112" x2="25" y2="112" /&gt;
  &lt;line x1="0" y1="120" x2="25" y2="120" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="0" y1="0" x2="0" y2="120" style="stroke-width:2" /&gt;
  &lt;line x1="25" y1="0" x2="25" y2="120" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="0.000000,0.000000 25.412617,0.000000 25.412617,120.000000 0.000000,120.000000" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="12.706308" y="140.000000" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;20&lt;/text&gt;
&lt;text x="45.412617" y="60.000000" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(-90,45.412617,60.000000)"&gt;10000000&lt;/text&gt;
&lt;/svg&gt;&lt;/p&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&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;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;p&gt;This works fine in the short and fat case too (when you have far more columns
than rows) but we’re always going to assume that one of your dimensions is
unchunked, and that the other dimension has chunks that are quite a bit
longer, otherwise, things might not fit into 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/2020/05/13/large-svds.md&lt;/span&gt;, line 105)&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="approximate-svd"&gt;
&lt;h1&gt;Approximate SVD&lt;/h1&gt;
&lt;p&gt;If your dataset is large in both dimensions then the algorithm above won’t work
as is. However, if you don’t need exact results, or if you only need a few of
the components, then there are a number of excellent approximation algorithms.&lt;/p&gt;
&lt;p&gt;Dask array has one of these approximation algorithms implemented in the
&lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html#dask.array.linalg.svd_compressed"&gt;da.linalg.svd_compressed&lt;/a&gt;
function. And with it we can compute the approximate SVD of very large
matrices.&lt;/p&gt;
&lt;p&gt;We were recently working on a problem (explained below) and found that we were
still running out of memory when dealing with this algorithm. There were two
challenges that we ran into:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;The algorithm requires multiple passes over the data, but the Dask task
scheduler was keeping the input matrix in memory after it had been loaded once
in order to avoid recomputation.
Things still worked, but Dask had to move the data to disk and back
repeatedly, which reduced performance significantly.&lt;/p&gt;
&lt;p&gt;We resolved this by including explicit recomputation steps in the algorithm.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Related chunks of data would be loaded at different times, and so would
need to stick around longer than necessary to wait for their associated
chunks.&lt;/p&gt;
&lt;p&gt;We resolved this by engaging task fusion as an optimization pass.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Before diving further into the technical solution
we quickly provide the use case that was motivating this 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/2020/05/13/large-svds.md&lt;/span&gt;, line 137)&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="application-genomics"&gt;
&lt;h1&gt;Application - Genomics&lt;/h1&gt;
&lt;p&gt;Many studies are using genome sequencing to study genetic variation
between different individuals within a species. These includes
studies of human populations, but also other species such as mice,
mosquitoes or disease-causing parasites. These studies will, in
general, find a large number of sites in the genome sequence where
individuals differ from each other. For example, humans have more
than 100 million variable sites in the genome, and modern studies
like the &lt;a class="reference external" href="https://www.ukbiobank.ac.uk/"&gt;UK BioBank&lt;/a&gt; are working towards
sequencing the genomes of 1 million individuals or more.&lt;/p&gt;
&lt;p&gt;In diploid species like humans, mice or mosquitoes, each individual
carries two genome sequences, one inherited from each parent. At each
of the 100 million variable genome sites there will be two or more
“alleles” that a single genome might carry. One way to think about
this is via the &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Punnett_square"&gt;Punnett
square&lt;/a&gt;, which
represents the different possible genotypes that one individual might
carry at one of these variable sites:&lt;/p&gt;
&lt;td&gt;
&lt;img src="https://upload.wikimedia.org/wikipedia/commons/9/93/Punnett_Square_Genetic_Carriers.PNG" alt="punnet square" height="40%" width="40%"&gt;
&lt;/td&gt;
&lt;p&gt;In the above there are three possible genotypes: AA, Aa, and aa. For
computational genomics, these genotypes can be encoded as 0, 1, or 2.
In a study of a species with M genetic variants assayed in N
individual samples, we can represent these genotypes as an (M x N)
array of integers. For a modern human genetics study, the scale of
this array might approach (100 million x 1 million). (Although in
practice, the size of the first dimension (number of variants) can be
reduced somewhat, by at least an order of magnitude, because many
genetic variants will carry little information and/or be correlated
with each other.)&lt;/p&gt;
&lt;p&gt;These genetic differences are not random, but carry information about
patterns of genetic similarity and shared ancestry between
individuals, because of the way they have been inherited through many
generations. A common task is to perform a dimensionality reduction
analysis on these data, such as a &lt;a class="reference external" href="https://journals.plos.org/plosgenetics/article?id=10.1371/journal.pgen.0020190"&gt;principal components
analysis&lt;/a&gt;
(SVD), to identify genetic structure reflecting these differencies in
degree of shared ancestry. This is an essential part of discovering
genetic variants associated with different diseases, and for learning
more about the genetic history of populations and species.&lt;/p&gt;
&lt;p&gt;Reducing the time taken to compute an analysis such as SVD, like all
science, allows for exploring larger datasets and testing more
hypotheses in less time. Practically, this means not simply a fast
SVD but an accelerated pipeline end-to-end, from data loading to
analysis, to understanding.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;We want to run an experiment in less time than it takes to make a cup of tea&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/05/13/large-svds.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="performant-svds-w-dask"&gt;
&lt;h1&gt;Performant SVDs w/ Dask&lt;/h1&gt;
&lt;p&gt;Now that we have that scientific background, let’s transition back to talking about computation.&lt;/p&gt;
&lt;p&gt;To stop Dask from holding onto the data we intentionally trigger computation as
we build up the graph. This is a bit atypical in Dask calculations (we prefer
to have as much of the computation at once before computing) but useful given
the multiple-pass nature of this problem. This was a fairly easy change, and
is available in &lt;a class="reference external" href="https://github.com/dask/dask/pull/5041"&gt;dask/dask #5041&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Additionally, we found that it was helpful to turn on moderately wide task
fusion.&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;dask&lt;/span&gt;
&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;optimization.fuse.ave-width&amp;quot;&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;/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/2020/05/13/large-svds.md&lt;/span&gt;, line 210)&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="then-things-work-fine"&gt;
&lt;h1&gt;Then things work fine&lt;/h1&gt;
&lt;p&gt;We’re going to try this SVD on a few different choices of hardware including:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;A MacBook Pro&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A DGX-2, an NVIDIA worksation with 16 high-end GPUs and fast interconnect&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A twenty-node cluster on AWS&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;section id="macbook-pro"&gt;
&lt;h2&gt;Macbook Pro&lt;/h2&gt;
&lt;p&gt;We can happily perform an SVD on a 20GB array on a Macbook Pro&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;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="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="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;1_000_000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20_000&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;20_000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;5_000&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;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;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;v&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This call is no longer entirely lazy, and it recomputes &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x&lt;/span&gt;&lt;/code&gt; a couple times, but
it works, and it works using only a few GB of RAM on a consumer laptop.&lt;/p&gt;
&lt;p&gt;It takes around 2min 30s time to compute that on a laptop.
That’s great! It was super easy to try out, didn’t require any special
hardware or setup, and in many cases is fast enough.
By working locally we can iterate quickly.&lt;/p&gt;
&lt;p&gt;Now that things work, we can experiment with different hardware.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="adding-gpus-a-15-second-svd"&gt;
&lt;h2&gt;Adding GPUs (a 15 second SVD)&lt;/h2&gt;
&lt;p&gt;&lt;em&gt;Disclaimer: one of the authors (Ben Zaitlen) works for NVIDIA&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;We can dramatically increase performance by using a multi-GPU machine.
NVIDIA and other manufacturers now make machines with multiple GPUs co-located in the same physical box.
In the following section, we will run the calculations on a &lt;strong&gt;DGX2&lt;/strong&gt;, a machine with 16 GPUs and fast interconnect between the GPUs.&lt;/p&gt;
&lt;p&gt;Below is almost the same code, running in significantly less same time but we make the
following changes:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;We increase the size of the array by a factor of &lt;strong&gt;10x&lt;/strong&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We switch out NumPy for CuPy, a GPU NumPy implementation&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We use a sixteen-GPU DGX-2 machine with NVLink interconnects between GPUs (NVLink will dramatically decrease transfer time between workers)&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;On A DGX2 we can calculate an SVD on a 200GB Dask array between 10 to 15 seconds.&lt;/p&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://gist.github.com/quasiben/98ee254920837313946f621e103d41f4"&gt;full notebook is here&lt;/a&gt;,
but the relevant code snippets are 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="c1"&gt;# Some GPU specific setup&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_cuda&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cupy&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="n"&gt;rs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# Create the data and run the SVD as normal&lt;/span&gt;
&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;rs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;3&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;10_000_000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20_000&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;20_000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;5_000&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="s2"&gt;&amp;quot;uint8&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="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;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;seed&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;rs&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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;To see this run, we recommend viewing this screencast:&lt;/p&gt;
&lt;iframe width="560" height="315" src="https://www.youtube.com/embed/6hmt1gARqp0" frameborder="0" allow="accelerometer; autoplay; encrypted-media; gyroscope; picture-in-picture" allowfullscreen&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="read-dataset-from-disk"&gt;
&lt;h2&gt;Read dataset from Disk&lt;/h2&gt;
&lt;p&gt;While impressive, the computation above is mostly bound by generating random
data and then performing matrix calculations. GPUs are good at both of these
things.&lt;/p&gt;
&lt;p&gt;In practice though, our input array won’t be randomly generated, it will be
coming from some dataset stored on disk or increasingly more common, stored in the cloud.
To make things more realistic we perform a similar calculation with data
stored in a &lt;a class="reference external" href="https://zarr.readthedocs.io/en/stable/"&gt;Zarr format&lt;/a&gt;
in &lt;a class="reference external" href="https://cloud.google.com/storage"&gt;GCS&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;In this &lt;a class="reference external" href="https://gist.github.com/quasiben/e52bc740ae22ae321f30987c65998078"&gt;Zarr SVD example&lt;/a&gt;,
we load a 25GB GCS backed data set onto a DGX2,
run a few processing steps, then perform an SVD.
The combination of preprocessing and SVD calculations ran in 18.7 sec and the data loading took 14.3 seconds.&lt;/p&gt;
&lt;p&gt;Again, on a DGX2, from data loading to SVD we are running in time less than it would take to make a cup of tea.
However, the data loading can be accelerated.
From GCS we are reading into data into the main memory of the machine (host memory), uncompressing the zarr bits,
then moving the data from host memory to the GPU (device memory). Passing data back and forth between host and device memory can significantly decrease performance. Reading directly into the GPU, bypassing host memory, would improve the overall pipeline.&lt;/p&gt;
&lt;p&gt;And so we come back to a common lesson of high performance computing:&lt;/p&gt;
&lt;p&gt;&lt;em&gt;High performance computing isn’t about doing one thing exceedingly well, it’s
about doing nothing poorly&lt;/em&gt;.&lt;/p&gt;
&lt;p&gt;In this case GPUs made our computation fast enough that we now need to focus on
other components of our system, notably disk bandwidth, and a direct reader for
Zarr data to GPU memory.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="cloud"&gt;
&lt;h2&gt;Cloud&lt;/h2&gt;
&lt;p&gt;&lt;em&gt;Diclaimer: one of the authors (Matthew Rocklin) works for Coiled Computing&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;We can also run this on the cloud with any number of frameworks.
In this case we used the &lt;a class="reference external" href="https://coiled.io"&gt;Coiled Cloud&lt;/a&gt; service to deploy on AWS&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;coiled_cloud&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;Cloud&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Cluster&lt;/span&gt;
&lt;span class="n"&gt;cloud&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Cloud&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;cloud&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;create_cluster_type&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;organization&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;friends&amp;quot;&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="s2"&gt;&amp;quot;genomics&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;worker_cpu&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;worker_memory&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;16 GiB&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;worker_environment&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;OMP_NUM_THREADS&amp;quot;&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="s2"&gt;&amp;quot;OPENBLAS_NUM_THREADS&amp;quot;&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="c1"&gt;# &amp;quot;EXTRA_PIP_PACKAGES&amp;quot;: &amp;quot;zarr&amp;quot;&lt;/span&gt;
    &lt;span class="p"&gt;},&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Cluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;organization&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;friends&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;typename&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;genomics&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;n_workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;,&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# then proceed as normal&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Using 20 machines with a total of 80 CPU cores on a dataset that was 10x larger
than the MacBook pro example we were able to run in about the same amount of
time. This shows near optimal scaling for this problem, which is nice to see
given how complex the SVD calculation is.&lt;/p&gt;
&lt;p&gt;A screencast of this problem is viewable here&lt;/p&gt;
&lt;iframe width="560" height="315" src="https://www.youtube.com/embed/qaJcAvhgLy4" frameborder="0" allow="accelerometer; autoplay; encrypted-media; gyroscope; picture-in-picture" allowfullscreen&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/2020/05/13/large-svds.md&lt;/span&gt;, line 360)&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="compression"&gt;
&lt;h1&gt;Compression&lt;/h1&gt;
&lt;p&gt;One of the easiest ways for us to improve performance is to reduce the size of
this data through compression.
This data is highly compressible for two reasons:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;The real-world data itself has structure and repetition
(although the random play data does not)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We’re storing entries that take on only four values.
We’re using eight-bit integers when we only needed two-bit integers&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Let’s solve the second problem first.&lt;/p&gt;
&lt;section id="compression-with-bit-twiddling"&gt;
&lt;h2&gt;Compression with bit twiddling&lt;/h2&gt;
&lt;p&gt;Ideally Numpy would have a two-bit integer datatype.
Unfortunately it doesn’t, and this is hard because memory in computers is
generally thought of in full bytes.&lt;/p&gt;
&lt;p&gt;To work around this we can use bit arithmetic to shove four values into a single value
Here are functions that do that, assuming that our array is square,
and the last dimension is divisible by four.&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;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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;compress&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;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;out&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;zeros_like&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;shape&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
    &lt;span class="n"&gt;out&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;0&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;out&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;1&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;&amp;lt;&amp;lt;&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;
    &lt;span class="n"&gt;out&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;&amp;lt;&amp;lt;&lt;/span&gt; &lt;span class="mi"&gt;4&lt;/span&gt;
    &lt;span class="n"&gt;out&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;3&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;&amp;lt;&amp;lt;&lt;/span&gt; &lt;span class="mi"&gt;6&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;out&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;decompress&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="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;back&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;zeros_like&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="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="n"&gt;out&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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;out&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
    &lt;span class="n"&gt;back&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;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;out&lt;/span&gt; &lt;span class="o"&gt;&amp;amp;&lt;/span&gt; &lt;span class="mb"&gt;0b00000011&lt;/span&gt;
    &lt;span class="n"&gt;back&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;4&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;out&lt;/span&gt; &lt;span class="o"&gt;&amp;amp;&lt;/span&gt; &lt;span class="mb"&gt;0b00001100&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;
    &lt;span class="n"&gt;back&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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;out&lt;/span&gt; &lt;span class="o"&gt;&amp;amp;&lt;/span&gt; &lt;span class="mb"&gt;0b00110000&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;4&lt;/span&gt;
    &lt;span class="n"&gt;back&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;4&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;out&lt;/span&gt; &lt;span class="o"&gt;&amp;amp;&lt;/span&gt; &lt;span class="mb"&gt;0b11000000&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;6&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;back&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Then, we can use these functions along with Dask to store our data in a
compressed state, but lazily decompress on-demand.&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;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;compress&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;decompress&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;That’s it. We compress each block our data and store that in memory.
However the output variable that we have, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x&lt;/span&gt;&lt;/code&gt; will decompress each chunk before
we operate on it, so we don’t need to worry about handling compressed blocks.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="compression-with-zarr"&gt;
&lt;h2&gt;Compression with Zarr&lt;/h2&gt;
&lt;p&gt;A slightly more general but probably less efficient route would be to compress
our arrays with a proper compression library like Zarr.&lt;/p&gt;
&lt;p&gt;The example below shows how we do this in practice.&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;zarr&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numcodecs&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;Blosc&lt;/span&gt;
&lt;span class="n"&gt;compressor&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Blosc&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cname&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;lz4&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;clevel&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="n"&gt;shuffle&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;Blosc&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;BITSHUFFLE&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;


&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;zarr&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;compressor&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;compressor&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_blocks&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;array&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Additionally, if we’re using the dask-distributed scheduler then we want to
make sure that the Blosc compression library doesn’t use additional threads.
That way we don’t have parallel calls of a parallel library, which can cause
some contention&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;set_no_threads_blosc&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; Stop blosc from using multiple threads &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numcodecs&lt;/span&gt;
    &lt;span class="n"&gt;numcodecs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;blosc&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;use_threads&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="kc"&gt;False&lt;/span&gt;

&lt;span class="c1"&gt;# Run on all workers&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register_worker_plugin&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;set_no_threads_blosc&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This approach is more general, and probably a good trick to have up ones’
sleeve, but it also doesn’t work on GPUs, which in the end is why we ended up
going with the bit-twiddling approach one section above, which uses API that
was uniformly accessible within the Numpy and CuPy libraries.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/05/13/large-svds.md&lt;/span&gt;, line 452)&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="final-thoughts"&gt;
&lt;h1&gt;Final Thoughts&lt;/h1&gt;
&lt;p&gt;In this post we did a few things, all around a single important problems in
genomics.&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;We learned a bit of science&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We translated a science problem into a computational problem,
and in particular into a request to perform large singular value decompositions&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We used a canned algorithm in dask.array that performed pretty well,
assuming that we’re comfortable going over the array in a few passes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We then tried that algorithm on three architectures quickly&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;A Macbook Pro&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A multi-GPU machine&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A cluster in the cloud&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Finally we talked about some tricks to pack more data into the same memory
with compression&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This problem was nice in that we got to dive deep into a technical science
problem, and yet also try a bunch of architecture quickly to investigate
hardware choices that we might make in the future.&lt;/p&gt;
&lt;p&gt;We used several technologies here today, made by several different communities
and companies. It was great to see how they all worked together seamlessly to
provide a flexible-yet-consistent experience.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2020/05/13/large-svds/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="CuPy" label="CuPy"/>
    <category term="GPU" label="GPU"/>
    <category term="array" label="array"/>
    <published>2020-05-13T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2020/04/28/dask-summit/</id>
    <title>Dask Summit</title>
    <updated>2020-04-28T00:00:00+00:00</updated>
    <author>
      <name>Mike McCarty (Capital One Center for Machine Learning) and Matthew Rocklin (Coiled Computing)</name>
    </author>
    <content type="html">&lt;p&gt;In late February members of the Dask community gathered together in Washington, DC.
This was a mix of open source project maintainers
and active users from a broad range of institutions.
This post shares a summary of what happened at this workshop,
including slides, images, and lessons learned.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: this event happened just before the widespread effects of the COVID-19
outbreak in the US and Europe. We were glad to see each other, but wouldn’t recommend doing this today.&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/04/28/dask-summit.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="who-came"&gt;

&lt;p&gt;This was an invite-only event of fifty people, with a cap of three people per
organization. We intentionally invited an even mix of half people who
self-identified as open source maintainers, and half people who identified as
institutional users. We had attendees from academia, small startups, tech
companies, government institutions, and large enterprise. It was surprising
how much we all had in common.
We had attendees from the following companies:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Anaconda&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Berkeley Institute for Datascience&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Blue Yonder&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brookhaven National Lab&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Capital One&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Chan Zuckerberg Initiative&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Coiled Computing&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Columbia University&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;D. E. Shaw &amp;amp; Co.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Flatiron Health&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Howard Hughes Medial Institute, Janelia Research Campus&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Inria&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kitware&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Lawrence Berkeley National Lab&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Los Alamos National Laboratory&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;MetroStar Systems&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Microsoft&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;NIMH&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;NVIDIA&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;National Center for Atmospheric Research (NCAR)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;National Energy Research Scientific Computing (NERSC) Center&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Prefect&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Quansight&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Related Sciences&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Saturn Cloud&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Smithsonian Institution&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;SymphonyRM&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The HDF Group&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;USGS&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ursa Labs&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/2020/04/28/dask-summit.md&lt;/span&gt;, line 59)&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="objectives"&gt;
&lt;h1&gt;Objectives&lt;/h1&gt;
&lt;p&gt;The Dask community comes from a broad range of backgrounds.
It’s an odd bunch, all solving very different problems,
but all with a surprisingly common set of needs.
We’ve all known each other on GitHub for several years,
and have a long shared history, but many of us had never met in person.&lt;/p&gt;
&lt;p&gt;In hindsight, this workshop served two main purposes:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;It helped us to see that we were all struggling with the same problems
and so helped to form direction and motivate future work&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It helped us to create social bonds and collaborations that help us manage
the day to day challenges of building and maintaining community software
across organizations&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/2020/04/28/dask-summit.md&lt;/span&gt;, line 75)&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="structure"&gt;
&lt;h1&gt;Structure&lt;/h1&gt;
&lt;p&gt;We met for three days.&lt;/p&gt;
&lt;p&gt;On days 1-2 we started with quick talks from the attendees and followed with
afternoon working sessions.&lt;/p&gt;
&lt;p&gt;Talks were short around 10-15 minutes
(having only experts in the room meant that we could easily skip the introductory material)
and always had the same structure:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;A brief description of the domain that they’re in and why it’s important&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Example: We look at seismic readings from thousand of measurement devices around
the world to understand and predict catastrophic earthquakes&lt;/em&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How they use Dask to solve this problem&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Example: this means that we need to cross-correlate thousands of very
long timeseries. We use Xarray on AWS with some custom operations.&lt;/em&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;What is wrong with Dask, and what they would like to see improved&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Example: It turns out that our axes labels can grow larger than what
Xarray was designed for. Also, the task graph size for Dask can become a
limitation&lt;/em&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;These talks were structured into six sections:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Workflow and pipelines&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Deployment&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Imaging&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;General data analysis&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Performance and tooling&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Xarray&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We didn’t capture video, but we do have slides from each of the talks below.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/04/28/dask-summit.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="workflow-and-pipelines"&gt;
&lt;h1&gt;1: Workflow and Pipelines&lt;/h1&gt;
&lt;section id="blue-yonder"&gt;
&lt;h2&gt;Blue Yonder&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: ETL Pipelines for Machine Learning&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Florian Jetter&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Also attending:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Nefta Kanilmaz&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Lucas Rademaker&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vSk2zAnSmzpbz5BgK70mpPmeQeV4h1IkCQh-EU8KXrZFJQGHmlMTuHvln3CmOQVTg/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="360" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="prefect"&gt;
&lt;h2&gt;Prefect&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Prefect + Dask: Parallel / Distributed Workflows&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Chris White, CTO&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="//www.slideshare.net/slideshow/embed_code/key/4wiUwkDHmdzVTW" width="595" height="485" frameborder="0" marginwidth="0" marginheight="0" scrolling="no" style="border:1px solid #CCC; border-width:1px; margin-bottom:5px; max-width: 100%;" allowfullscreen&gt; &lt;/iframe&gt; &lt;div style="margin-bottom:5px"&gt; &lt;strong&gt; &lt;a href="//www.slideshare.net/ChrisWhite249/dask-prefect" title="Dask + Prefect" target="_blank"&gt;Dask + Prefect&lt;/a&gt; &lt;/strong&gt; from &lt;strong&gt;&lt;a href="https://www.slideshare.net/ChrisWhite249" target="_blank"&gt;Chris White&lt;/a&gt;&lt;/strong&gt; &lt;/div&gt;
&lt;/section&gt;
&lt;section id="symphonyrm"&gt;
&lt;h2&gt;SymphonyRM&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Dask and Prefect for Data Science in Healthcare&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenter: Joe Schmid, CTO&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vSCDbXrXtrL9vmA0hQ1NNk5DY0-3Azpcf9FbYgjoLuKV79vf_nm7wdUZl1NsL5DZqRmlUTP--u9HM56/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="366" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&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/2020/04/28/dask-summit.md&lt;/span&gt;, line 139)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="deployment"&gt;
&lt;h1&gt;2: Deployment&lt;/h1&gt;
&lt;section id="quansight"&gt;
&lt;h2&gt;Quansight&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Building Cloud-based Data Science Platforms with Dask&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Dharhas Pothina&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Also attending: - James Bourbeau - Dhavide Aruliah&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vSZ1fSrkWvzMPlx-f0qk7w2xj_uDp5q-Tg11S9UlynoohZV0VYjdFduDUrAdhptSYfpzFu9Wask1WSN/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="479" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="nvidia-and-microsoft-azure"&gt;
&lt;h2&gt;NVIDIA and Microsoft/Azure&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Native Cloud Deployment with Dask-Cloudprovider&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Jacob Tomlinson, Tom Drabas, and Code Peterson&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vT-B1c0r8MWMF8wvW4lNly-qmOCqhFqKdhshXnVql6UVkYQ-aGprY3Du0VH0PJBccOmM84ncw0lDV77/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="366" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="inria"&gt;
&lt;h2&gt;Inria&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: HPC Deployments with Dask-Jobqueue&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Loïc Esteve&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://lesteve.github.io/talks/2020-dask-jobqueue-dask-workshop/slides.html" frameborder="0" width="1000" height="800"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="anaconda"&gt;
&lt;h2&gt;Anaconda&lt;/h2&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Title: Dask Gateway&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Also attending: - Tom Augspurger - Eric Dill - Jonathan Helmus&lt;/p&gt;
&lt;style&gt;
    iframe {
        overflow:hidden;
    }
&lt;/style&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="http://jcrist.github.io/talks/dask_summit_2020/slides.html" frameborder="1" width="600" height="355" scrolling="no"&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/2020/04/28/dask-summit.md&lt;/span&gt;, line 176)&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="imaging"&gt;
&lt;h1&gt;3: Imaging&lt;/h1&gt;
&lt;section id="kitware"&gt;
&lt;h2&gt;Kitware&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Scientific Image Analysis and Visualization with ITK&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Matt McCormick&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vRz2SV2G-1LEXXCF0n9vugF13s7ABpLDT-yH3WtxQEOjt2FVHE7apl3nQhqkOiLeY9kSzM_Mrs6fJOk/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="366" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="id1"&gt;
&lt;h2&gt;Kitware&lt;/h2&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/2020/04/28/dask-summit.md&lt;/span&gt;, line 185); &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: “kitware”.&lt;/p&gt;
&lt;/aside&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Image processing with X-rays and electrons&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Marcus Hanwell&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vRT--l76IcSPlIP_N6ClUtm2ECZaxkvIGrBNyyoFmJNQu6kS6CilWoleIMCur2FQ7ZpEkkCsw7UXnRd/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="366" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="national-institutes-of-mental-health"&gt;
&lt;h2&gt;National Institutes of Mental Health&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Brain imaging&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: John Lee&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vTH1X0cSjozmCDvSQ8CtcxPPYejkLROC_b92W6uwznG5litWq_MwKJzUMnAQi0Prw/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="366" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="janelia-howard-hughes-medical-institute"&gt;
&lt;h2&gt;Janelia / Howard Hughes Medical Institute&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Spark, Dask, and FlyEM HPC&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Stuart Berg&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vSnZ-JgHAoAOUirqmLcI3GaKyC4oVo3vThZZ4oyx8vZjJ66An09JIhbcoy6k7ufTw/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="479" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&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/2020/04/28/dask-summit.md&lt;/span&gt;, line 206)&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="general-data-analysis"&gt;
&lt;h1&gt;4: General Data Analysis&lt;/h1&gt;
&lt;section id="brookhaven-national-labs"&gt;
&lt;h2&gt;Brookhaven National Labs&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Dask at DOE Light Sources&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Dan Allan&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vRd8PVHjW7Umjo1rUjR7XWDT95CcEoE_3jH-ceDHsN_lMv_4M2qnlFiFvtMl9SX0Eb1EFQTGkzUWCDy/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="366" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="d-e-shaw-group"&gt;
&lt;h2&gt;D.E. Shaw Group&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Dask at D.E. Shaw&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Akihiro Matsukawa&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="id2"&gt;
&lt;h2&gt;Anaconda&lt;/h2&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/2020/04/28/dask-summit.md&lt;/span&gt;, line 220); &lt;em&gt;&lt;a href="#id2"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “anaconda”.&lt;/p&gt;
&lt;/aside&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Dask Dataframes and Dask-ML summary&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vTs6nNsMkV92Uj4QUns1VB8pKlKSsRgUAGwvcbTOPqMazSAhxtawVNgb04YmHVFmb0z8-no-cdS8mE8/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="366" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&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/2020/04/28/dask-summit.md&lt;/span&gt;, line 227)&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="performance-and-tooling"&gt;
&lt;h1&gt;5: Performance and Tooling&lt;/h1&gt;
&lt;section id="berkeley-institute-for-data-science"&gt;
&lt;h2&gt;Berkeley Institute for Data Science&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Numpy APIs&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Sebastian Berg&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="ursa-labs"&gt;
&lt;h2&gt;Ursa Labs&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Arrow&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Joris Van den Bossche&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vQY3ubjCFkMcU_b8p2xmuXN8VVR1BxxSWZDe5Vy-ftnH2CstZILvTo2pRBv5R_VDk85rNjVoWew2AJl/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="366" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="nvidia"&gt;
&lt;h2&gt;NVIDIA&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: RAPIDS&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Keith Kraus&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Also attending: - Mike Beaumont - Richard Zamora&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vQiNrupzQlSqsu95AAHqIhU1V_iVUav_0WlIp4dXdSE6Izze1BL8mkFbIzg7p8CndEi9bjWaC2OVlyu/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="366" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="id3"&gt;
&lt;h2&gt;NVIDIA&lt;/h2&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/2020/04/28/dask-summit.md&lt;/span&gt;, line 249); &lt;em&gt;&lt;a href="#id3"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “nvidia”.&lt;/p&gt;
&lt;/aside&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: UCX&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Ben Zaitlen&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vRU-vsXsnXgeLKdmtWZkZVV_-mOojsNesCbQKJgmWkwSjxj5ZdwkmS6X4tOt3HpFrIOfNROSlV_8l84/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="366" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&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/2020/04/28/dask-summit.md&lt;/span&gt;, line 256)&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="xarray"&gt;
&lt;h1&gt;6: Xarray&lt;/h1&gt;
&lt;section id="usgs-and-ncar"&gt;
&lt;h2&gt;USGS and NCAR&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Dask in Pangeo&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Rich Signell and Anderson Banihirwe&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vStqGiQy6pJDYhRgF-BZylQussINK5BGlhnidOVCUECo_ebYqRH9cSY4e-2z7BfFFvTfvkqq_M1jXBX/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="366" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="lbnl"&gt;
&lt;h2&gt;LBNL&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Accelerating Experimental Science with Dask&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Matt Henderson&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://drive.google.com/file/d/1DVVzYmhkDhO2xs0tmxpPCkxx5c4o63bO/view"&gt;Slides&lt;/a&gt; - Fill too large to preview&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="lanl"&gt;
&lt;h2&gt;LANL&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Title: Seismic Analysis&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Presenters: Jonathan MacCarthy&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe src="https://docs.google.com/presentation/d/e/2PACX-1vSWAgKLxt1tBZxXjQfIRQNFPvAMFYZ-z0hkMy7euPnOHwO9pomH_gM8cKUTKXA68w/embed?start=false&amp;loop=false&amp;delayms=3000" frameborder="0" width="600" height="404" allowfullscreen="true" mozallowfullscreen="true" webkitallowfullscreen="true"&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/2020/04/28/dask-summit.md&lt;/span&gt;, line 278)&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="unstructured-time"&gt;
&lt;h1&gt;Unstructured Time&lt;/h1&gt;
&lt;p&gt;Having rapid fire talks in the morning, followed by unstructured time in the
afternoon was a productive combination. Below you’ll see pictures from
geo-scientists and quants talking about the same challenges, and library
maintainers from Pandas/Arrow/RAPDIS/Dask all working together on joint
solutions.&lt;/p&gt;
&lt;p&gt;&lt;img src="https://pbs.twimg.com/media/ERykEc9XUAEFq-L?format=jpg&amp;name=large"
     width="40%"&gt;
&lt;img src="https://pbs.twimg.com/media/ERzEcEeXkAU35sg?format=jpg&amp;name=large"
    width="40%"&gt;&lt;/p&gt;
&lt;p&gt;&lt;img src="https://pbs.twimg.com/media/ERyz7B5X0AIrDkn?format=jpg&amp;name=large"
    width="40%"&gt;
&lt;img src="https://pbs.twimg.com/media/ERzXhHnWAAE_zDA?format=jpg&amp;name=large"
    width="40%"&gt;&lt;/p&gt;
&lt;p&gt;&lt;img src="https://pbs.twimg.com/media/ERz3GDgXsAcE6Id?format=jpg&amp;name=large"
    width="40%"&gt;
&lt;img src="https://pbs.twimg.com/media/ERz4ur2WkAAGJwm?format=jpg&amp;name=large"
    width="40%"&gt;&lt;/p&gt;
&lt;p&gt;&lt;img src="https://pbs.twimg.com/media/ER0sZceUYAAF5fW?format=jpg&amp;name=large"
    width="40%"&gt;
&lt;img src="https://pbs.twimg.com/media/ER0yY2rX0AEFfXi?format=jpg&amp;name=large"
    width="40%"&gt;&lt;/p&gt;
&lt;p&gt;&lt;img src="https://pbs.twimg.com/media/ERyz98YWAAAmJbE?format=jpg&amp;name=large"
    width="40%"&gt;
&lt;img src="https://pbs.twimg.com/media/ERz5S2dWoAEhFHc?format=jpg&amp;name=large"
    width="40%"&gt;&lt;/p&gt;
&lt;p&gt;This unstructured time is a productive combination that we would recommend to
other technically diverse groups in the future. Engagement and productivity was
really high throughout the workshop.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/04/28/dask-summit.md&lt;/span&gt;, line 315)&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’s strength comes from this broad community of stakeholders.&lt;/p&gt;
&lt;p&gt;An early technical focus on simplicity and pragmatism allowed the project to be
quickly adopted within many different domains. As a result, the practitioners
within these domains are largely the ones driving the project forward today.
This Community Driven Development brings an incredible diversity of technical
and cultural challenges and experience that force the project to quickly evolve
in a way that is constrained towards pragmatism.&lt;/p&gt;
&lt;p&gt;There is still plenty of work to do.
Short term this workshop brought up many technical challenges that are shared
by all (simpler deployments, scheduling under task constraints, active memory
management). Longer term we need to welcome more people into this community,
both by increasing the diversity of domains, and the diversity of individuals
(the vast majority of attendees were white men in their thirties from the US
and western Europe).&lt;/p&gt;
&lt;p&gt;We’re in a good position to effect this change.
Dask’s recent growth has captured the attention of many different institutions.
Now is a critical time to be intentional about the projects growth to make sure
that the project and community continue to reflect a broad and ethical set of
principles.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/04/28/dask-summit.md&lt;/span&gt;, line 340)&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;section id="sponsors"&gt;
&lt;h2&gt;Sponsors&lt;/h2&gt;
&lt;p&gt;Without the support of our sponsors, this workshop would not have been possible.
Thanks to Anaconda, Capital One and NVIDIA for their support and generous
donations toward this event.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="organizers"&gt;
&lt;h2&gt;Organizers&lt;/h2&gt;
&lt;p&gt;Thank you very much to the organizers who took time from their busy schedules
and worked so hard to put together this event.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Brittany Treadway (Capital One)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Keith Kraus (NVIDIA)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin (Coiled Computing)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Mike Beaumont (NVIDIA)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Mike McCarty (Capital One)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Neia Woodson (Capital One)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jake Schmitt (Capital One)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist (Anaconda)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2020/04/28/dask-summit/"/>
    <summary>In late February members of the Dask community gathered together in Washington, DC.
This was a mix of open source project maintainers
and active users from a broad range of institutions.
This post shares a summary of what happened at this workshop,
including slides, images, and lessons learned.</summary>
    <published>2020-04-28T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2020/01/14/estimating-users/</id>
    <title>Estimating Users</title>
    <updated>2020-01-14T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</name>
    </author>
    <content type="html">&lt;p&gt;People often ask me &lt;em&gt;“How many people use Dask?”&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;As with any non-invasive open source software, the answer to this is
&lt;em&gt;“I don’t know”&lt;/em&gt;.&lt;/p&gt;
&lt;p&gt;There are many possible proxies for user counts, like downloads, GitHub stars,
and so on, but most of them are wildly incorrect.
As a project maintainer who tries to find employment for other maintainers,
I’m incentivized to take the highest number I can find,
but that is somewhat dishonest.
That number today is in the form of this likely false statement.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Dask has 50-100k daily downloads.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;This number comes from looking at the Python Package Index (PyPI)
(image from &lt;a class="reference external" href="https://pypistats.org"&gt;pypistats.org&lt;/a&gt;)&lt;/p&gt;
&lt;p&gt;&lt;a href="/images/dask-pypi-downloads-total.png"&gt;&lt;img src="/images/dask-pypi-downloads-total.png" width="100%" alt="Total Dask downloads"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;This is a huge number, but is almost certainly misleading.
Common sense tells us that there are not 100k new Dask users every day.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/01/14/estimating-users.md&lt;/span&gt;, line 31)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="bots-dominate-download-counts"&gt;

&lt;p&gt;If you dive in more deeply to numbers like these you will find that they are
almost entirely due to automated processes. For example, of Dask’s 100k new
users, a surprising number of them seem to be running Linux.&lt;/p&gt;
&lt;p&gt;&lt;a href="/images/linux-reigns.png"&gt;&lt;img src="/images/linux-reigns.png" width="100%" alt="Linux dominates download counts"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;While it’s true that Dask is frequently run on Linux because it is a
distributed library, it would be odd to see every machine in that deployment
individually &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pip&lt;/span&gt; &lt;span class="pre"&gt;install&lt;/span&gt; &lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt;. It’s more likely that these downloads are the
result of automated systems, rather than individual users.&lt;/p&gt;
&lt;p&gt;Anecdotally, if you get access to fine grained download data, one finds that a
small set of IPs dominate download counts. These tend to come mostly from
continuous integration services like Travis and Circle, are coming from AWS,
or are coming from a few outliers in the world (sometimes people in China try
to mirror everything)..&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/01/14/estimating-users.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="check-windows"&gt;
&lt;h1&gt;Check Windows&lt;/h1&gt;
&lt;p&gt;So, in an effort to avoid this effect we start looking at just Windows
downloads.&lt;/p&gt;
&lt;p&gt;&lt;a href="/images/dask-windows-downloads.png"&gt;&lt;img src="/images/dask-windows-downloads.png" width="100%" alt="Dask Monthly Windows Downloads"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;The magnitudes here seem more honest to me. These monthly numbers translate to
about 1000 downloads a day (perhaps multiplied by two or three for OSX and
Linux), which seems more in line with my expectations.&lt;/p&gt;
&lt;p&gt;However even this is strange. The structure doesn’t match my personal experience.
Why the big change in adoption in 2018?
What is the big spike in 2019?
Anecdotally maintainers did not notice a significant jump in users there.
Instead, we’ve experienced smooth continuous growth of adoption over time
(this is what most long-term software growth looks like).
It’s also odd that there hasn’t been continued growth since 2018. Anecdotally
Dask seems to have grown somewhat constantly over the last few years. Phase
transitions like these don’t match observed reality (at least in so far as I
personally have observed it).&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://nbviewer.jupyter.org/gist/mrocklin/ef6f9b6a649a6d78b2221d8fdeea5f2a"&gt;&lt;em&gt;Notebook for plot available here&lt;/em&gt;&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/2020/01/14/estimating-users.md&lt;/span&gt;, line 74)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="documentation-views"&gt;
&lt;h1&gt;Documentation views&lt;/h1&gt;
&lt;p&gt;My favorite metric is looking at weekly unique users to documentation.&lt;/p&gt;
&lt;p&gt;This is an over-estimate of users because many people look at the documentation
without using the project. This is also an under-estimate because many users
don’t consult our documentation on a weekly basis (oh I wish).&lt;/p&gt;
&lt;p&gt;&lt;a href="/images/dask-weekly-users.png"&gt;&lt;img src="/images/dask-weekly-users.png" width="100%" alt="Dask weekly users on documentation"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;This growth pattern matches my expectations and my experience with maintaining
a project that has steadily gained traction over several years.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Plot taken from Google Analytics&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/01/14/estimating-users.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 id="dependencies"&gt;
&lt;h1&gt;Dependencies&lt;/h1&gt;
&lt;p&gt;It’s also important to look at dependencies of a project. For example many
users in the earth and geo sciences use Dask through another project,
&lt;a class="reference external" href="https://xarray.pydata.org"&gt;Xarray&lt;/a&gt;. These users are much less likely to touch
Dask directly, but often use Dask as infrastructure underneath the Xarray
library. We should probably add in something like half of Xarray’s users as
well.&lt;/p&gt;
&lt;p&gt;&lt;a href="/images/xarray-weekly-users.png"&gt;&lt;img src="/images/xarray-weekly-users.png" width="100%" alt="Xarray weekly users on documentation"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Plot taken from Google Analytics, supplied by &lt;a class="reference external" href="https://joehamman.com/"&gt;Joe Hamman&lt;/a&gt; from Xarray&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2020/01/14/estimating-users.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="summary"&gt;
&lt;h1&gt;Summary&lt;/h1&gt;
&lt;p&gt;Dask has somewhere between 100k new users every day (download counts)
or something like 10k users total (weekly unique IPs). The 10k number sounds
more likely to me, maybe bumping up to 15k due to dependencies.
The fact is though that no one really knows.&lt;/p&gt;
&lt;p&gt;Judging the use of community maintained OSS is important as we try to value its
impact on society. This is also a fundamentally difficult problem.
I hope that this post helps to highlight how these numbers may be misleading,
and encourages us all to think more deeply about estimating impact.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2020/01/14/estimating-users/"/>
    <summary>People often ask me “How many people use Dask?”</summary>
    <published>2020-01-14T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/11/01/deployment-updates/</id>
    <title>Dask Deployment Updates</title>
    <updated>2019-11-01T00:00:00+00:00</updated>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/11/01/deployment-updates.md&lt;/span&gt;, line 7)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;Over the last six months many Dask developers have worked on making Dask easier
to deploy in a wide variety of situations. This post summarizes those
efforts, and provides links to ongoing 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/2019/11/01/deployment-updates.md&lt;/span&gt;, line 13)&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-we-mean-by-deployment"&gt;
&lt;h1&gt;What we mean by Deployment&lt;/h1&gt;
&lt;p&gt;In order to run Dask on a cluster, you need to setup a scheduler on one
machine:&lt;/p&gt;
&lt;div class="highlight-console notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;$ &lt;/span&gt;dask-scheduler
&lt;span class="go"&gt;Scheduler running at tcp://192.168.0.1&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And start Dask workers on many other machines&lt;/p&gt;
&lt;div class="highlight-console notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;$ &lt;/span&gt;dask-worker&lt;span class="w"&gt; &lt;/span&gt;tcp://192.168.0.1
&lt;span class="go"&gt;Waiting to connect to:       tcp://scheduler:8786&lt;/span&gt;

&lt;span class="gp"&gt;$ &lt;/span&gt;dask-worker&lt;span class="w"&gt; &lt;/span&gt;tcp://192.168.0.1
&lt;span class="go"&gt;Waiting to connect to:       tcp://scheduler:8786&lt;/span&gt;

&lt;span class="gp"&gt;$ &lt;/span&gt;dask-worker&lt;span class="w"&gt; &lt;/span&gt;tcp://192.168.0.1
&lt;span class="go"&gt;Waiting to connect to:       tcp://scheduler:8786&lt;/span&gt;

&lt;span class="gp"&gt;$ &lt;/span&gt;dask-worker&lt;span class="w"&gt; &lt;/span&gt;tcp://192.168.0.1
&lt;span class="go"&gt;Waiting to connect to:       tcp://scheduler:8786&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;For informal clusters people might do this manually, by logging into each
machine and running these commands themselves. However it’s much more common
to use a cluster resource manager such as Kubernetes, Yarn (Hadoop/Spark),
HPC batch schedulers (SGE, PBS, SLURM, LSF …), some cloud service or some custom system.&lt;/p&gt;
&lt;p&gt;As Dask is used by more institutions and used more broadly within those
institutions, making deployment smooth and natural becomes increasingly
important. This is so important in fact, that there have been seven separate
efforts to improve deployment in some regard or another by a few different
groups.&lt;/p&gt;
&lt;p&gt;We’ll briefly summarize and link to this work below, and then we’ll finish up
by talking about some internal design that helped to make this work more
consistent.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/11/01/deployment-updates.md&lt;/span&gt;, line 54)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="dask-ssh"&gt;
&lt;h1&gt;Dask-SSH&lt;/h1&gt;
&lt;p&gt;According to our user survey, the most common deployment mechanism was still
SSH. Dask has had a &lt;a class="reference external" href="https://docs.dask.org/en/latest/setup/ssh.html#command-line"&gt;command line dask-ssh
tool&lt;/a&gt; to make it
easier to deploy with SSH for some time. We recently updated this to also
include a Python interface, which provides more control.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;SSHCluster&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;SSHCluster&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="s2"&gt;&amp;quot;host1&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;host2&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;host3&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;host4&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;connect_options&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;known_hosts&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;worker_options&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;nthreads&amp;quot;&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="n"&gt;scheduler_options&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;port&amp;quot;&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="s2"&gt;&amp;quot;dashboard_address&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;:8797&amp;quot;&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This isn’t what we recommend for large institutions, but it can be helpful for
more informal groups who are just getting started.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/11/01/deployment-updates.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="dask-jobqueue-and-dask-kubernetes-rewrite"&gt;
&lt;h1&gt;Dask-Jobqueue and Dask-Kubernetes Rewrite&lt;/h1&gt;
&lt;p&gt;We’ve rewritten Dask-Jobqueue for SLURM/PBS/LSF/SGE cluster managers typically
found in HPC centers and Dask-Kubernetes. These now share a common codebase
along with Dask SSH, and so are much more consistent and hopefully bug free.&lt;/p&gt;
&lt;p&gt;Ideally users shouldn’t notice much difference with existing workloads,
but new features like asynchronous operation, integration with the Dask
JupyterLab extension, and so on are more consistently available. Also, we’ve
been able to unify development and reduce our maintenance burden considerably.&lt;/p&gt;
&lt;p&gt;The new version of Dask Jobqueue where these changes take place is 0.7.0, and
the work was done in &lt;a class="reference external" href="https://github.com/dask/dask-jobqueue/pull/307"&gt;dask/dask-jobqueue #307&lt;/a&gt;.
The new version of Dask Kubernetes is 0.10.0 and the work was done in
&lt;a class="reference external" href="https://github.com/dask/dask-kubernetes/pull/162"&gt;dask/dask-kubernetes #162&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/2019/11/01/deployment-updates.md&lt;/span&gt;, line 92)&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="dask-cloudprovider"&gt;
&lt;h1&gt;Dask-CloudProvider&lt;/h1&gt;
&lt;p&gt;For cloud deployments we generally recommend using a hosted Kubernetes or Yarn
service, and then using Dask-Kubernetes or Dask-Yarn on top of these.&lt;/p&gt;
&lt;p&gt;However, some institutions have made decisions or commitments to use
certain vendor specific technologies, and it’s more convenient to use APIs that
are more native to the particular cloud. The new package &lt;a class="reference external" href="https://cloudprovider.dask.org"&gt;Dask
Cloudprovider&lt;/a&gt; handles this today for Amazon’s
ECS API, which has been around for a long while and is more universally
accepted.&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_cloudprovider&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;ECSCluster&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ECSCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster_arn&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;arn:aws:ecs:&amp;lt;region&amp;gt;:&amp;lt;acctid&amp;gt;:cluster/&amp;lt;clustername&amp;gt;&amp;quot;&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_cloudprovider&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;FargateCluster&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;FargateCluster&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/11/01/deployment-updates.md&lt;/span&gt;, line 112)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="dask-gateway"&gt;
&lt;h1&gt;Dask-Gateway&lt;/h1&gt;
&lt;p&gt;In some cases users may not have access to the cluster manager. For example
the institution may not give all of their data science users access to the Yarn
or Kubernetes cluster. In this case the &lt;a class="reference external" href="https://gateway.dask.org"&gt;Dask-Gateway&lt;/a&gt;
project may be useful.
It can launch and manage Dask jobs,
and provide a proxy connection to these jobs if necessary.
It is typically deployed with elevated permissions but managed directly by IT,
giving them a point of greater control.&lt;/p&gt;
&lt;img src="https://gateway.dask.org/_images/architecture.svg" width="50%"&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/11/01/deployment-updates.md&lt;/span&gt;, line 125)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="gpus-and-dask-cuda"&gt;
&lt;h1&gt;GPUs and Dask-CUDA&lt;/h1&gt;
&lt;p&gt;While using Dask with multi-GPU deployments the &lt;a class="reference external" href="https://rapids.ai"&gt;NVIDIA
RAPIDS&lt;/a&gt; has needed the ability to specify increasingly
complex setups of Dask workers. They recommend the following deployment
strategy:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;One Dask-worker per GPU on a machine&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Specify the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;CUDA_VISIBLE_DEVICES&lt;/span&gt;&lt;/code&gt; environment variable to pin that worker
to that GPU&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;If your machine has multiple network interfaces then choose the network interface that has the best connection to that GPU&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;If your machine has multiple CPUs then set thread affinities to use the closest CPU&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;… and more&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;For this reason we wanted to specify these configurations in code, 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="n"&gt;specification&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;worker-0&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;cls&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;distributed&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Nanny&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;options&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;nthreads&amp;quot;&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="s2"&gt;&amp;quot;env&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;CUDA_VISIBLE_DEVICES&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;0,1,2,3&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt; &lt;span class="n"&gt;interface&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;ib0&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt;
    &lt;span class="p"&gt;},&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;worker-1&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;cls&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;distributed&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Nanny&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;options&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;nthreads&amp;quot;&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="s2"&gt;&amp;quot;env&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;CUDA_VISIBLE_DEVICES&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;1,2,3,0&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt; &lt;span class="n"&gt;interface&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;ib0&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt;
    &lt;span class="p"&gt;},&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;worker-2&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;cls&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;distributed&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Nanny&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;options&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;nthreads&amp;quot;&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="s2"&gt;&amp;quot;env&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;CUDA_VISIBLE_DEVICES&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;2,3,0,1&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt; &lt;span class="n"&gt;interface&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;ib1&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt;
    &lt;span class="p"&gt;},&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;worker-2&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;cls&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;distributed&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Nanny&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;options&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;nthreads&amp;quot;&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="s2"&gt;&amp;quot;env&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;CUDA_VISIBLE_DEVICES&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;3,0,1,2&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt; &lt;span class="n"&gt;interface&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;ib1&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt;
    &lt;span class="p"&gt;},&lt;/span&gt;
&lt;span class="p"&gt;}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And the new SpecCluster class to deploy these workers:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;SpecCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;specification&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We’ve used this technique in the
&lt;a class="reference external" href="https://github.com/rapidsai/dask-cuda"&gt;Dask-CUDA&lt;/a&gt; project to provide
convenient functions for deployment on multi-GPU systems.&lt;/p&gt;
&lt;p&gt;This class was generic enough that it ended up forming the base of the SSH,
Jobqueue, and Kubernetes solutions as well.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/11/01/deployment-updates.md&lt;/span&gt;, line 176)&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="standards-and-conventions"&gt;
&lt;h1&gt;Standards and Conventions&lt;/h1&gt;
&lt;p&gt;The solutions above are built by different teams that work in different companies.
This is great because those teams have hands-on experience with the
cluster managers in the wild, but has historically been somewhat challenging to
standardize user experience. This is particularly challenging when we build
other tools like IPython widgets or the Dask JupyterLab extension, which want
to interoperate with all of the Dask deployment solutions.&lt;/p&gt;
&lt;p&gt;The recent rewrite of Dask-SSH, Dask-Jobqueue, Dask-Kubernetes, and the new
Dask-Cloudprovider and Dask-CUDA libraries place them
all under the same &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.distributed.SpecCluster&lt;/span&gt;&lt;/code&gt; superclass. So we can expect a high degree of
uniformity from them. Additionally, all of the classes now match the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.distributed.Cluster&lt;/span&gt;&lt;/code&gt; interface, which standardizes things like
adaptivity, IPython widgets, logs, and some basic reporting.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Cluster&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;SpecCluster&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Kubernetes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;JobQueue (PBS/SLURM/LSF/SGE/Torque/Condor/Moab/OAR)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;SSH&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;CloudProvider (ECS)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;CUDA (LocalCUDACluster, DGX)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;LocalCluster&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Yarn&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Gateway&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/11/01/deployment-updates.md&lt;/span&gt;, line 203)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="future-work"&gt;
&lt;h1&gt;Future Work&lt;/h1&gt;
&lt;p&gt;There is still plenty to do. Here are some of the themes we’ve seen among
current development:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Move the Scheduler off to a separate job/pod/container in the network,
which is often helpful for complex networking situations&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Improve proxying of the dashboard in these situations&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Optionally separate the life-cycle of the cluster from the lifetime of the
Python process that requested the cluster&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Write up best practices how to compose GPU support generally with all of the cluster managers&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/11/01/deployment-updates/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <published>2019-11-01T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/10/08/df-groupby/</id>
    <title>DataFrame Groupby Aggregations</title>
    <updated>2019-10-08T00:00:00+00:00</updated>
    <author>
      <name>Benjamin Zaitlen &amp; James Bourbeau</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/10/08/df-groupby.md&lt;/span&gt;, line 10)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="groupby-aggregations-with-dask"&gt;

&lt;p&gt;In this post we’ll dive into how Dask computes groupby aggregations. These are commonly used operations for ETL and analysis in which we split data into groups, apply a function to each group independently, and then combine the results back together. In the PyData/R world this is often referred to as the split-apply-combine strategy (first coined by &lt;a class="reference external" href="https://www.jstatsoft.org/article/view/v040i01"&gt;Hadley Wickham&lt;/a&gt;) and is used widely throughout the &lt;a class="reference external" href="https://pandas.pydata.org/pandas-docs/stable/user_guide/groupby.html"&gt;Pandas ecosystem&lt;/a&gt;.&lt;/p&gt;
&lt;div align="center"&gt;
  &lt;a href="/images/split-apply-combine.png"&gt;
    &lt;img src="/images/split-apply-combine.png" width="80%" align="center"&gt;
  &lt;/a&gt;
  &lt;p align="center"&gt;&lt;i&gt;Image courtesy of swcarpentry.github.io&lt;/i&gt;&lt;/p&gt;
&lt;/div&gt;
&lt;p&gt;Dask leverages this idea using a similarly catchy name: apply-concat-apply or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;aca&lt;/span&gt;&lt;/code&gt; for short. Here we’ll explore the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;aca&lt;/span&gt;&lt;/code&gt; strategy in both simple and complex operations.&lt;/p&gt;
&lt;p&gt;First, recall that a Dask DataFrame is a &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe-design.html#internal-design"&gt;collection&lt;/a&gt; of DataFrame objects (e.g. each &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe-design.html#partitions"&gt;partition&lt;/a&gt; of a Dask DataFrame is a Pandas DataFrame). For example, let’s say we have the following Pandas DataFrame:&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;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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;dict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;a&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1&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="mi"&gt;3&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;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="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;99&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="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="n"&gt;b&lt;/span&gt;&lt;span class="o"&gt;=&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;3&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="mi"&gt;3&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;1&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;10&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;3&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;0&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="gp"&gt;... &lt;/span&gt;                       &lt;span class="n"&gt;c&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;4&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="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;5&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;9&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;44&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;33&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;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;
&lt;span class="go"&gt;     a   b   c&lt;/span&gt;
&lt;span class="go"&gt;0    1   1   2&lt;/span&gt;
&lt;span class="go"&gt;1    1   3   4&lt;/span&gt;
&lt;span class="go"&gt;2    2  10   5&lt;/span&gt;
&lt;span class="go"&gt;3    3   3   2&lt;/span&gt;
&lt;span class="go"&gt;4    3   2   3&lt;/span&gt;
&lt;span class="go"&gt;5    1   1   5&lt;/span&gt;
&lt;span class="go"&gt;6    1   3   2&lt;/span&gt;
&lt;span class="go"&gt;7    2  10   3&lt;/span&gt;
&lt;span class="go"&gt;8    3   3   9&lt;/span&gt;
&lt;span class="go"&gt;9    3   3   2&lt;/span&gt;
&lt;span class="go"&gt;10  99  12  44&lt;/span&gt;
&lt;span class="go"&gt;11  10   0  33&lt;/span&gt;
&lt;span class="go"&gt;12   1   9   2&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;To create a Dask DataFrame with three partitions from this data, we could partition &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df&lt;/span&gt;&lt;/code&gt; between the indices of: (0, 4), (5, 9), and (10, 12). We can perform this partitioning with Dask by using the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_pandas&lt;/span&gt;&lt;/code&gt; function with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;npartitions=3&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.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_pandas&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;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;The 3 partitions are simply 3 individual Pandas DataFrames:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;partitions&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;   a   b  c&lt;/span&gt;
&lt;span class="go"&gt;0  1   1  2&lt;/span&gt;
&lt;span class="go"&gt;1  1   3  4&lt;/span&gt;
&lt;span class="go"&gt;2  2  10  5&lt;/span&gt;
&lt;span class="go"&gt;3  3   3  2&lt;/span&gt;
&lt;span class="go"&gt;4  3   2  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/2019/10/08/df-groupby.md&lt;/span&gt;, line 66)&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="apply-concat-apply"&gt;
&lt;h1&gt;Apply-concat-apply&lt;/h1&gt;
&lt;p&gt;When Dask applies a function and/or algorithm (e.g. &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sum&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&lt;/span&gt;&lt;/code&gt;, etc.) to a Dask DataFrame, it does so by applying that operation to all the constituent partitions independently, collecting (or concatenating) the outputs into intermediary results, and then applying the operation again to the intermediary results to produce a final result. Internally, Dask re-uses the same apply-concat-apply methodology for many of its internal DataFrame calculations.&lt;/p&gt;
&lt;p&gt;Let’s break down how Dask computes &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ddf.groupby(['a',&lt;/span&gt; &lt;span class="pre"&gt;'b']).c.sum()&lt;/span&gt;&lt;/code&gt; by going through each step in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;aca&lt;/span&gt;&lt;/code&gt; process. We’ll begin by splitting our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df&lt;/span&gt;&lt;/code&gt; Pandas DataFrame into three partitions:&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;df_1&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&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="n"&gt;df_2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&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="mi"&gt;10&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;df_3&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&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="p"&gt;:]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;section id="apply"&gt;
&lt;h2&gt;Apply&lt;/h2&gt;
&lt;p&gt;Next we perform the same &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;groupby(['a',&lt;/span&gt; &lt;span class="pre"&gt;'b']).c.sum()&lt;/span&gt;&lt;/code&gt; operation on each of our three partitions:&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;sr1&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df_1&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&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="s1"&gt;&amp;#39;b&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;c&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;sr2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df_2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&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="s1"&gt;&amp;#39;b&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;c&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;sr3&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df_3&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&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="s1"&gt;&amp;#39;b&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;c&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;These operations each produce a Series with a &lt;a class="reference external" href="https://pandas.pydata.org/pandas-docs/stable/user_guide/advanced.html"&gt;MultiIndex&lt;/a&gt;:&lt;/p&gt;
&lt;table&gt;
  &lt;tr&gt;
    &lt;th&gt;
      &lt;pre&gt;
&gt;&gt;&gt; sr1
a  b
1  1     2
   3     4
2  10    5
3  2     3
   3     2
Name: c, dtype: int64
      &lt;/pre&gt;
    &lt;/th&gt;
    &lt;th&gt;
      &lt;pre&gt;
&gt;&gt;&gt; sr2
a  b
1  1      5
   3      2
2  10     3
3  3     11
Name: c, dtype: int64
      &lt;/pre&gt;
    &lt;/th&gt;
    &lt;th&gt;
      &lt;pre&gt;
&gt;&gt;&gt; sr3
a   b
1   9      2
10  0     33
99  12    44
Name: c, dtype: int64
      &lt;/pre&gt;
    &lt;/th&gt;
  &lt;/tr&gt;
&lt;/table&gt;
&lt;/section&gt;
&lt;section id="the-concat"&gt;
&lt;h2&gt;The conCat!&lt;/h2&gt;
&lt;p&gt;After the first &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;apply&lt;/span&gt;&lt;/code&gt;, the next step is to concatenate the intermediate &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sr1&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sr2&lt;/span&gt;&lt;/code&gt;, and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sr3&lt;/span&gt;&lt;/code&gt; results. This is fairly straightforward to do using the Pandas &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;concat&lt;/span&gt;&lt;/code&gt; function:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;sr_concat&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;concat&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;sr1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sr2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sr3&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;sr_concat&lt;/span&gt;
&lt;span class="go"&gt;a   b&lt;/span&gt;
&lt;span class="go"&gt;1   1      2&lt;/span&gt;
&lt;span class="go"&gt;    3      4&lt;/span&gt;
&lt;span class="go"&gt;2   10     5&lt;/span&gt;
&lt;span class="go"&gt;3   2      3&lt;/span&gt;
&lt;span class="go"&gt;    3      2&lt;/span&gt;
&lt;span class="go"&gt;1   1      5&lt;/span&gt;
&lt;span class="go"&gt;    3      2&lt;/span&gt;
&lt;span class="go"&gt;2   10     3&lt;/span&gt;
&lt;span class="go"&gt;3   3     11&lt;/span&gt;
&lt;span class="go"&gt;1   9      2&lt;/span&gt;
&lt;span class="go"&gt;10  0     33&lt;/span&gt;
&lt;span class="go"&gt;99  12    44&lt;/span&gt;
&lt;span class="go"&gt;Name: c, dtype: int64&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="apply-redux"&gt;
&lt;h2&gt;Apply Redux&lt;/h2&gt;
&lt;p&gt;Our final step is to apply the same &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;groupby(['a',&lt;/span&gt; &lt;span class="pre"&gt;'b']).c.sum()&lt;/span&gt;&lt;/code&gt; operation again on the concatenated &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sr_concat&lt;/span&gt;&lt;/code&gt; Series. However we no longer have columns &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;a&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;b&lt;/span&gt;&lt;/code&gt;, so how should we proceed?&lt;/p&gt;
&lt;p&gt;Zooming out a bit, our goal is to add the values in the column which have the same index. For example, there are two rows with the index &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(1,&lt;/span&gt; &lt;span class="pre"&gt;1)&lt;/span&gt;&lt;/code&gt; with corresponding values: 2, 5. So how can we groupby the indices with the same value? A MutliIndex uses &lt;a class="reference external" href="https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.MultiIndex.html#pandas.MultiIndex"&gt;levels&lt;/a&gt; to define what the value is at a give index. Dask &lt;a class="reference external" href="https://github.com/dask/dask/blob/973c6e1b2e38c2d9d6e8c75fb9b4ab7a0d07e6a7/dask/dataframe/groupby.py#L69-L75"&gt;determines&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/dask/dask/blob/973c6e1b2e38c2d9d6e8c75fb9b4ab7a0d07e6a7/dask/dataframe/groupby.py#L1065"&gt;uses these levels&lt;/a&gt; in the final apply step of the apply-concat-apply calculation. In our case, the level is &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;[0,&lt;/span&gt; &lt;span class="pre"&gt;1]&lt;/span&gt;&lt;/code&gt;, that is, we want both the index at the 0th level and the 1st level and if we group by both, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;0,&lt;/span&gt; &lt;span class="pre"&gt;1&lt;/span&gt;&lt;/code&gt;, we will have effectively grouped the same indices together:&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;sr_concat&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;level&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sum&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;th&gt;
      &lt;pre&gt;
&gt;&gt;&gt; total
a   b
1   1      7
    3      6
    9      2
2   10     8
3   2      3
    3     13
10  0     33
99  12    44
Name: c, dtype: int64
      &lt;/pre&gt;
    &lt;/th&gt;
    &lt;th&gt;
      &lt;pre&gt;
&gt;&gt;&gt; ddf.groupby(['a', 'b']).c.sum().compute()
a   b
1   1      7
    3      6
2   10     8
3   2      3
    3     13
1   9      2
10  0     33
99  12    44
Name: c, dtype: int64
      &lt;/pre&gt;
    &lt;/th&gt;
    &lt;th&gt;
      &lt;pre&gt;
&gt;&gt;&gt; df.groupby(['a', 'b']).c.sum()
a   b
1   1      7
    3      6
    9      2
2   10     8
3   2      3
    3     13
10  0     33
99  12    44
Name: c, dtype: int64
      &lt;/pre&gt;
    &lt;/th&gt;
  &lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;Additionally, we can easily examine the steps of this apply-concat-apply calculation by &lt;a class="reference external" href="https://docs.dask.org/en/latest/graphviz.html"&gt;visualizing the task graph&lt;/a&gt; for the computation:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&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="s1"&gt;&amp;#39;b&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;c&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;visualize&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/sum.svg"&gt;
  &lt;img src="/images/sum.svg" width="80%"&gt;
&lt;/a&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sum&lt;/span&gt;&lt;/code&gt; is rather a straight-forward calculation. What about something a bit more complex like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&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="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&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="s1"&gt;&amp;#39;b&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;c&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;visualize&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/mean.svg"&gt;
  &lt;img src="/images/mean.svg" width="80%"&gt;
&lt;/a&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Mean&lt;/span&gt;&lt;/code&gt; is a good example of an operation which doesn’t directly fit in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;aca&lt;/span&gt;&lt;/code&gt; model – concatenating &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&lt;/span&gt;&lt;/code&gt; values and taking the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&lt;/span&gt;&lt;/code&gt; again will yield incorrect results. Like any style of computation: vectorization, Map/Reduce, etc., we sometime need to creatively fit the computation to the style/mode. In the case of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;aca&lt;/span&gt;&lt;/code&gt; we can often break down the calculation into constituent parts. For &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&lt;/span&gt;&lt;/code&gt;, this would be &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sum&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;count&lt;/span&gt;&lt;/code&gt;:&lt;/p&gt;
&lt;div class="math notranslate nohighlight"&gt;
\[ \bar{x} = \frac{x_1+x_2+\cdots +x_n}{n}\]&lt;/div&gt;
&lt;p&gt;From the task graph above, we can see that two independent tasks for each partition: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;series-groupby-count-chunk&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;series-groupby-sum-chunk&lt;/span&gt;&lt;/code&gt;. The results are then aggregated into two final nodes: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;series-groupby-count-agg&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;series-groupby-sum-agg&lt;/span&gt;&lt;/code&gt; and then we finally calculate the mean: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;total&lt;/span&gt; &lt;span class="pre"&gt;sum&lt;/span&gt; &lt;span class="pre"&gt;/&lt;/span&gt; &lt;span class="pre"&gt;total&lt;/span&gt; &lt;span class="pre"&gt;count&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/10/08/df-groupby/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="dask" label="dask"/>
    <category term="dataframe" label="dataframe"/>
    <published>2019-10-08T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/09/30/dask-hyperparam-opt/</id>
    <title>Better and faster hyperparameter optimization with Dask</title>
    <updated>2019-09-30T00:00:00+00:00</updated>
    <author>
      <name>&lt;a href="http://stsievert.com"&gt;Scott Sievert&lt;/a&gt;</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;&lt;a class="reference external" href="https://stsievert.com"&gt;Scott Sievert&lt;/a&gt; wrote this post. The original post lives at
&lt;a class="reference external" href="https://stsievert.com/blog/2019/09/27/dask-hyperparam-opt/"&gt;https://stsievert.com/blog/2019/09/27/dask-hyperparam-opt/&lt;/a&gt; with better
styling. This work is supported by Anaconda, Inc.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://dask.org"&gt;Dask&lt;/a&gt;’s machine learning package, &lt;a class="reference external" href="https://ml.dask.org/"&gt;Dask-ML&lt;/a&gt; now implements Hyperband, an
advanced “hyperparameter optimization” algorithm that performs rather well.
This post will&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;describe “hyperparameter optimization”, a common problem in machine learning&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;describe Hyperband’s benefits and why it works&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;show how to use Hyperband via example alongside performance comparisons&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;In this post, I’ll walk through a practical example and highlight key portions
of the paper “&lt;a class="reference external" href="http://conference.scipy.org/proceedings/scipy2019/pdfs/scott_sievert.pdf"&gt;Better and faster hyperparameter optimization with Dask&lt;/a&gt;”, which is also
summarized in a &lt;a class="reference external" href="https://www.youtube.com/watch?v=x67K9FiPFBQ"&gt;~25 minute SciPy 2019 talk&lt;/a&gt;.&lt;/p&gt;
&lt;!--More--&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/09/30/dask-hyperparam-opt.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="problem"&gt;

&lt;p&gt;Machine learning requires data, an untrained model and “hyperparameters”, parameters that are chosen before training begins that
help with cohesion between the model and data. The user needs to specify values
for these hyperparameters in order to use the model. A good example is
adapting ridge regression or LASSO to the amount of noise in the
data with the regularization parameter.&lt;a class="footnote-reference brackets" href="#alpha" id="id1" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;1&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Model performance strongly depends on the hyperparameters provided. A fairly complex example is with a particular
visualization tool, &lt;a class="reference external" href="https://scikit-learn.org/stable/modules/generated/sklearn.manifold.TSNE.html"&gt;t-SNE&lt;/a&gt;. This tool requires (at least) three
hyperparameters and performance depends radically on the hyperparameters. In fact, the first section in “&lt;a class="reference external" href="https://distill.pub/2016/misread-tsne/"&gt;How to Use t-SNE
Effectively&lt;/a&gt;” is titled “Those hyperparameters really matter”.&lt;/p&gt;
&lt;p&gt;Finding good values for these hyperparameters is critical and has an entire
Scikit-learn documentation page, “&lt;a class="reference external" href="http://scikit-learn.org/stable/modules/grid_search.html"&gt;Tuning the hyperparameters of an
estimator&lt;/a&gt;.” Briefly, finding decent values of hyperparameters
is difficult and requires guessing or searching.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;How can these hyperparameters be found quickly and efficiently with an
advanced task scheduler like Dask?&lt;/strong&gt; Parallelism will pose some challenges, but
the Dask architecture enables some advanced algorithms.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: this post presumes knowledge of Dask basics. This material is covered in
Dask’s documentation on &lt;a class="reference external" href="https://docs.dask.org/en/latest/why.html"&gt;Why Dask?&lt;/a&gt;, a ~15 minute &lt;a class="reference external" href="https://www.youtube.com/watch?v=ods97a5Pzw0"&gt;video introduction to
Dask&lt;/a&gt;, a &lt;a class="reference external" href="https://www.youtube.com/watch?v=tQBovBvSDvA"&gt;video introduction to Dask-ML&lt;/a&gt; and &lt;a class="reference external" href="https://stsievert.com/blog/2016/09/09/dask-cluster/"&gt;a
blog post I wrote&lt;/a&gt; on my first use of Dask.&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/09/30/dask-hyperparam-opt.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="contributions"&gt;
&lt;h1&gt;Contributions&lt;/h1&gt;
&lt;p&gt;Dask-ML can quickly find high-performing hyperparameters. I will back this
claim with intuition and experimental evidence.&lt;/p&gt;
&lt;p&gt;Specifically, this is because
Dask-ML now
implements an algorithm introduced by Li et. al. in “&lt;a class="reference external" href="https://arxiv.org/pdf/1603.06560.pdf"&gt;Hyperband: A novel
bandit-based approach to hyperparameter optimization&lt;/a&gt;”.
Pairing of Dask and Hyperband enables some exciting new performance opportunities,
especially because Hyperband has a simple implementation and Dask is an
advanced task scheduler.&lt;a class="footnote-reference brackets" href="#first" id="id2" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;2&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Let’s go
through the basics of Hyperband then illustrate its use and performance with
an example.
This will highlight some key points of &lt;a class="reference external" href="http://conference.scipy.org/proceedings/scipy2019/pdfs/scott_sievert.pdf"&gt;the corresponding paper&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/2019/09/30/dask-hyperparam-opt.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="hyperband-basics"&gt;
&lt;h1&gt;Hyperband basics&lt;/h1&gt;
&lt;p&gt;The motivation for Hyperband is to find high performing hyperparameters with minimal
training. Given this goal, it makes sense to spend more time training high
performing models – why waste more time training time a model if it’s done poorly in the past?&lt;/p&gt;
&lt;p&gt;One method to spend more time on high performing models is to initialize many
models, start training all of them, and then stop training low performing models
before training is finished. That’s what Hyperband does. At the most basic
level, Hyperband is a (principled) early-stopping scheme for
&lt;a class="reference external" href="https://scikit-learn.org/stable/modules/generated/sklearn.model_selection.RandomizedSearchCV.html"&gt;RandomizedSearchCV&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Deciding when to stop the training of models depends on how strongly
the training data effects the score. There are two extremes:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;when only the training data matter&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;i.e., when the hyperparameters don’t influence the score at all&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;when only the hyperparameters matter&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;i.e., when the training data don’t influence the score at all&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Hyperband balances these two extremes by sweeping over how frequently
models are stopped. This sweep allows a mathematical proof that Hyperband
will find the best model possible with minimal &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt;
calls&lt;a class="footnote-reference brackets" href="#qual" id="id3" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;3&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Hyperband has significant parallelism because it has two “embarrassingly
parallel” for-loops – Dask can exploit this. Hyperband has been implemented
in Dask, specifically in Dask’s machine library Dask-ML.&lt;/p&gt;
&lt;p&gt;How well does it perform? Let’s illustrate via example. Some setup is required
before the performance comparison in &lt;em&gt;&lt;a class="reference internal" href="#performance"&gt;&lt;span class="xref myst"&gt;Performance&lt;/span&gt;&lt;/a&gt;&lt;/em&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/09/30/dask-hyperparam-opt.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="example"&gt;
&lt;h1&gt;Example&lt;/h1&gt;
&lt;p&gt;&lt;em&gt;Note: want to try &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt; out yourself? Dask has &lt;a class="reference external" href="https://examples.dask.org/machine-learning/hyperparam-opt.html"&gt;an example use&lt;/a&gt;.
It can even be run in-browser!&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;I’ll illustrate with a synthetic example. Let’s build a dataset with 4 classes:&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;experiment&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;make_circles&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="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;make_circles&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_classes&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;n_features&lt;/span&gt;&lt;span class="o"&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;n_informative&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;scatter&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="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;color&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img src="/images/2019-hyperband/synthetic/dataset.png"
style="max-width: 100%;"
width="200px" /&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: this content is pulled from
&lt;a class="reference external" href="https://github.com/stsievert/dask-hyperband-comparison"&gt;stsievert/dask-hyperband-comparison&lt;/a&gt;, or makes slight modifications.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Let’s build a fully connected neural net with 24 neurons for classification:&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;sklearn.neural_network&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;MLPClassifier&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;model&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;MLPClassifier&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Building the neural net with PyTorch is also possible&lt;a class="footnote-reference brackets" href="#skorch" id="id4" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;4&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt; (and what I used in development).&lt;/p&gt;
&lt;p&gt;This neural net’s behavior is dictated by 6 hyperparameters. Only one controls
the model of the optimal architecture (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;hidden_layer_sizes&lt;/span&gt;&lt;/code&gt;, the number of
neurons in each layer). The rest control finding the best model of that
architecture. Details on the hyperparameters are in the
&lt;em&gt;&lt;a class="reference internal" href="#appendix"&gt;&lt;span class="xref myst"&gt;Appendix&lt;/span&gt;&lt;/a&gt;&lt;/em&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="n"&gt;params&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;  &lt;span class="c1"&gt;# details in appendix&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;params&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;dict_keys([&amp;#39;hidden_layer_sizes&amp;#39;, &amp;#39;alpha&amp;#39;, &amp;#39;batch_size&amp;#39;, &amp;#39;learning_rate&amp;#39;&lt;/span&gt;
&lt;span class="go"&gt;           &amp;#39;learning_rate_init&amp;#39;, &amp;#39;power_t&amp;#39;, &amp;#39;momentum&amp;#39;])&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;hidden_layer_sizes&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;  &lt;span class="c1"&gt;# always 24 neurons&lt;/span&gt;
&lt;span class="go"&gt;[(24, ), (12, 12), (6, 6, 6, 6), (4, 4, 4, 4, 4, 4), (12, 6, 3, 3)]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;I choose these hyperparameters to have a complex search space that mimics the
searches performed for most neural networks. These searches typically involve
hyperparameters like “dropout”, “learning rate”, “momentum” and “weight
decay”.&lt;a class="footnote-reference brackets" href="#user-facing" id="id5" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;5&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt;
End users don’t care hyperparameters like these; they don’t change the
model architecture, only finding the best model of a particular architecture.&lt;/p&gt;
&lt;p&gt;How can high performing hyperparameter values be found quickly?&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/09/30/dask-hyperparam-opt.md&lt;/span&gt;, line 205)&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="finding-the-best-parameters"&gt;
&lt;h1&gt;Finding the best parameters&lt;/h1&gt;
&lt;p&gt;First, let’s look at the parameters required for Dask-ML’s implementation
of Hyperband (which is in the class &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt;).&lt;/p&gt;
&lt;section id="hyperband-parameters-rule-of-thumb"&gt;
&lt;h2&gt;Hyperband parameters: rule-of-thumb&lt;/h2&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt; has two inputs:&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;max_iter&lt;/span&gt;&lt;/code&gt;, which determines how many times to call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;the chunk size of the Dask array, which determines how many data each
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt; call receives.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;These fall out pretty naturally once it’s known how long to train the best
model and very approximately how many parameters to sample:&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;n_examples&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;50&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# 50 passes through dataset for best model&lt;/span&gt;
&lt;span class="n"&gt;n_params&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;299&lt;/span&gt;  &lt;span class="c1"&gt;# sample about 300 parameters&lt;/span&gt;

&lt;span class="c1"&gt;# inputs to hyperband&lt;/span&gt;
&lt;span class="n"&gt;max_iter&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;n_params&lt;/span&gt;
&lt;span class="n"&gt;chunk_size&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;n_examples&lt;/span&gt; &lt;span class="o"&gt;//&lt;/span&gt; &lt;span class="n"&gt;n_params&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The inputs to this rule-of-thumb are exactly what the user cares about:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;a measure of how complex the search space is (via &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n_params&lt;/span&gt;&lt;/code&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;how long to train the best model (via &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n_examples&lt;/span&gt;&lt;/code&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Notably, there’s no tradeoff between &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n_examples&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n_params&lt;/span&gt;&lt;/code&gt; like with
Scikit-learn’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomizedSearchCV&lt;/span&gt;&lt;/code&gt; because &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n_examples&lt;/span&gt;&lt;/code&gt; is only for &lt;em&gt;some&lt;/em&gt;
models, not for &lt;em&gt;all&lt;/em&gt; models. There’s more details on this
rule-of-thumb in the “Notes” section of the &lt;a class="reference external" href="https://ml.dask.org/modules/generated/dask_ml.model_selection.HyperbandSearchCV.html#dask_ml.model_selection.HyperbandSearchCV"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt;
docs&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;With these inputs a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt; object can easily be created.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="finding-the-best-performing-hyperparameters"&gt;
&lt;h2&gt;Finding the best performing hyperparameters&lt;/h2&gt;
&lt;p&gt;This model selection algorithm Hyperband is implemented in the class
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt;. Let’s create an instance of that class:&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_ml.model_selection&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;HyperbandSearchCV&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;search&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;HyperbandSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;est&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;max_iter&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;max_iter&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;aggressiveness&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;aggressiveness&lt;/span&gt;&lt;/code&gt; defaults to 3. &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;aggressiveness=4&lt;/span&gt;&lt;/code&gt; is chosen because this is an
&lt;em&gt;initial&lt;/em&gt; search; I know nothing about how this search space. Then, this search
should be more aggressive in culling off bad models.&lt;/p&gt;
&lt;p&gt;Hyperband hides some details from the user (which enables the mathematical
guarantees), specifically the details on the amount of training and
the number of models created. These details are available in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;metadata&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="n"&gt;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;metadata&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;n_models&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="go"&gt;378&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;metadata&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;partial_fit_calls&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="go"&gt;5721&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now that we have some idea on how long the computation will take, let’s ask it
to find the best set of hyperparameters:&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_ml.model_selection&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;train_test_split&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;X_test&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_test&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;train_test_split&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;y&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;chunk_size&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;y_train&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;chunk_size&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The dashboard will be active during this time&lt;a class="footnote-reference brackets" href="#dashboard" id="id6" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;6&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt;:&lt;/p&gt;
&lt;p&gt;
&lt;video width="600" style="max-width: 100%;" autoplay loop controls &gt;
  &lt;source src="/images/2019-hyperband/dashboard-compress.mp4" type="video/mp4" &gt;
  Your browser does not support the video tag.
&lt;/video&gt;
&lt;/p&gt;
&lt;p&gt;How well do these hyperparameters perform?&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;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_score_&lt;/span&gt;
&lt;span class="go"&gt;0.9019221418447483&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt; mirrors Scikit-learn’s API for &lt;a class="reference external" href="https://scikit-learn.org/stable/modules/generated/sklearn.model_selection.RandomizedSearchCV.html"&gt;RandomizedSearchCV&lt;/a&gt;, so it
has access to all the expected attributes and methods:&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;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_params_&lt;/span&gt;
&lt;span class="go"&gt;{&amp;quot;batch_size&amp;quot;: 64, &amp;quot;hidden_layer_sizes&amp;quot;: [6, 6, 6, 6], ...}&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;score&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_test&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_test&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;0.8989070100111217&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_model_&lt;/span&gt;
&lt;span class="go"&gt;MLPClassifier(...)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Details on the attributes and methods are in the &lt;a class="reference external" href="https://ml.dask.org/modules/generated/dask_ml.model_selection.HyperbandSearchCV.html"&gt;HyperbandSearchCV
documentation&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/2019/09/30/dask-hyperparam-opt.md&lt;/span&gt;, line 322)&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="performance"&gt;
&lt;h1&gt;Performance&lt;/h1&gt;
&lt;!--
Plot 1: how well does it do?
Plot 2: how does this scale?
Plot 3: what opportunities does Dask enable?
--&gt;
&lt;p&gt;I ran this 200 times on my personal laptop with 4 cores.
Let’s look at the distribution of final validation scores:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/2019-hyperband/synthetic/final-acc.svg"
style="max-width: 100%;"
 width="400px"/&gt;&lt;/p&gt;
&lt;p&gt;The “passive” comparison is really &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomizedSearchCV&lt;/span&gt;&lt;/code&gt; configured so it takes
an equal amount of work as &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt;. Let’s see how this does over
time:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/2019-hyperband/synthetic/val-acc.svg"
style="max-width: 100%;"
 width="400px"/&gt;&lt;/p&gt;
&lt;p&gt;This graph shows the mean score over the 200 runs with the solid line, and the
shaded region represents the &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Interquartile_range"&gt;interquartile range&lt;/a&gt;. The dotted green
line indicates the data required to train 4 models to completion.
“Passes through the dataset” is a good proxy
for “time to solution” because there are only 4 workers.&lt;/p&gt;
&lt;p&gt;This graph shows that &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt; will find parameters at least 3 times
quicker than &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomizedSearchCV&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;section id="dask-opportunities"&gt;
&lt;h2&gt;Dask opportunities&lt;/h2&gt;
&lt;p&gt;What opportunities does combining Hyperband and Dask create?
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt; has a lot of internal parallelism and Dask is an advanced task
scheduler.&lt;/p&gt;
&lt;p&gt;The most obvious opportunity involves job prioritization. Hyperband fits many
models in parallel and Dask might not have that
workers available. This means some jobs have to wait for other jobs
to finish. Of course, Dask can prioritize jobs&lt;a class="footnote-reference brackets" href="#prior" id="id7" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;7&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt; and choose which models
to fit first.&lt;/p&gt;
&lt;p&gt;Let’s assign the priority for fitting a certain model to be the model’s most
recent score. How does this prioritization scheme influence the score? Let’s
compare the prioritization schemes in
a single run of the 200 above:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/2019-hyperband/synthetic/priority.svg"
style="max-width: 100%;"
     width="400px" /&gt;&lt;/p&gt;
&lt;p&gt;These two lines are the same in every way except for
the prioritization scheme.
This graph compares the “high scores” prioritization scheme and the Dask’s
default prioritization scheme (“fifo”).&lt;/p&gt;
&lt;p&gt;This graph is certainly helped by the fact that is run with only 4 workers.
Job priority does not matter if every job can be run right away (there’s
nothing to assign priority too!).&lt;/p&gt;
&lt;/section&gt;
&lt;section id="amenability-to-parallelism"&gt;
&lt;h2&gt;Amenability to parallelism&lt;/h2&gt;
&lt;p&gt;How does Hyperband scale with the number of workers?&lt;/p&gt;
&lt;p&gt;I ran another separate experiment to measure. This experiment is described more in the &lt;a class="reference external" href="http://conference.scipy.org/proceedings/scipy2019/pdfs/scott_sievert.pdf"&gt;corresponding
paper&lt;/a&gt;, but the relevant difference is that a &lt;a class="reference external" href="https://pytorch.org/"&gt;PyTorch&lt;/a&gt; neural network is used
through &lt;a class="reference external" href="https://skorch.readthedocs.io/en/stable/"&gt;skorch&lt;/a&gt; instead of Scikit-learn’s MLPClassifier.&lt;/p&gt;
&lt;p&gt;I ran the &lt;em&gt;same&lt;/em&gt; experiment with a different number of Dask
workers.&lt;a class="footnote-reference brackets" href="#same" id="id8" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;8&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt; Here’s how &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt; scales:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/2019-hyperband/image-denoising/scaling-patience.svg" width="400px"
style="max-width: 100%;"
/&gt;&lt;/p&gt;
&lt;p&gt;Training one model to completion requires 243 seconds (which is marked by the
white line). This is a comparison with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;patience&lt;/span&gt;&lt;/code&gt;, which stops training models
if their scores aren’t increasing enough. Functionally, this is very useful
because the user might accidentally specify &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n_examples&lt;/span&gt;&lt;/code&gt; to be too large.&lt;/p&gt;
&lt;p&gt;It looks like the speedups start to saturate somewhere
between 16 and 24 workers, at least for this example.
Of course, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;patience&lt;/span&gt;&lt;/code&gt; doesn’t work as well for a large number of
workers.&lt;a class="footnote-reference brackets" href="#scale-worker" id="id9" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;9&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&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/2019/09/30/dask-hyperparam-opt.md&lt;/span&gt;, line 421)&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="future-work"&gt;
&lt;h1&gt;Future work&lt;/h1&gt;
&lt;p&gt;There’s some ongoing pull requests to improve &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt;. The most
significant of these involves tweaking some Hyperband internals so &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt;
works better with initial or very exploratory searches (&lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/532"&gt;dask/dask-ml #532&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;The biggest improvement I see is treating &lt;em&gt;dataset size&lt;/em&gt; as the scarce resource
that needs to be preserved instead of &lt;em&gt;training time&lt;/em&gt;. This would allow
Hyperband to work with any model, instead of only models that implement
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Serialization is an important part of the distributed Hyperband implementation
in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt;. Scikit-learn and PyTorch can easily handle this because
they support the Pickle protocol&lt;a class="footnote-reference brackets" href="#pickle-post" id="id10" role="doc-noteref"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;10&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/a&gt;, but
Keras/Tensorflow/MXNet present challenges. The use of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperbandSearchCV&lt;/span&gt;&lt;/code&gt; could
be increased by resolving this issue.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/09/30/dask-hyperparam-opt.md&lt;/span&gt;, line 444)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="appendix"&gt;
&lt;h1&gt;Appendix&lt;/h1&gt;
&lt;p&gt;I choose to tune 7 hyperparameters, which are&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;hidden_layer_sizes&lt;/span&gt;&lt;/code&gt;, which controls the activation function used at each
neuron&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;alpha&lt;/span&gt;&lt;/code&gt;, which controls the amount of regularization&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;More hyperparameters control finding the best neural network:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;batch_size&lt;/span&gt;&lt;/code&gt;, which controls the number of examples the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;optimizer&lt;/span&gt;&lt;/code&gt; uses to
approximate the gradient&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;learning_rate&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;learning_rate_init&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;power_t&lt;/span&gt;&lt;/code&gt;, which control some basic
hyperparameters for the SGD optimizer I’ll be using&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;momentum&lt;/span&gt;&lt;/code&gt;, a more advanced hyperparameter for SGD with Nesterov’s momentum.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;hr class="footnotes docutils" /&gt;
&lt;aside class="footnote-list brackets"&gt;
&lt;aside class="footnote brackets" id="alpha" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id1"&gt;1&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;Which amounts to choosing &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;alpha&lt;/span&gt;&lt;/code&gt; in Scikit-learn’s &lt;a class="reference external" href="https://scikit-learn.org/stable/modules/generated/sklearn.linear_model.Ridge.html"&gt;Ridge&lt;/a&gt; or &lt;a class="reference external" href="https://scikit-learn.org/stable/modules/generated/sklearn.linear_model.Lasso.html"&gt;LASSO&lt;/a&gt;&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="first" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id2"&gt;2&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;To the best of my knowledge, this is the first implementation of Hyperband with an advanced task scheduler&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="qual" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id3"&gt;3&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;More accurately, Hyperband will find close to the best model possible with &lt;span class="math notranslate nohighlight"&gt;\(N\)&lt;/span&gt; &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt; calls in expected score with high probability, where “close” means “within log terms of the upper bound on score”. For details, see Corollary 1 of the &lt;a class="reference external" href="http://conference.scipy.org/proceedings/scipy2019/pdfs/scott_sievert.pdf"&gt;corresponding paper&lt;/a&gt; or Theorem 5 of &lt;a class="reference external" href="https://arxiv.org/pdf/1603.06560.pdf"&gt;Hyperband’s paper&lt;/a&gt;.&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="skorch" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id4"&gt;4&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;through the Scikit-learn API wrapper &lt;a class="reference external" href="https://skorch.readthedocs.io/en/stable/"&gt;skorch&lt;/a&gt;&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="user-facing" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id5"&gt;5&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;There’s less tuning for adaptive step size methods like &lt;a class="reference external" href="https://arxiv.org/abs/1412.6980"&gt;Adam&lt;/a&gt; or &lt;a class="reference external" href="http://jmlr.org/papers/v12/duchi11a.html"&gt;Adagrad&lt;/a&gt;, but they might under-perform on the test data (see “&lt;a class="reference external" href="https://arxiv.org/abs/1705.08292"&gt;The Marginal Value of Adaptive Gradient Methods for Machine Learning&lt;/a&gt;”)&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="dashboard" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id6"&gt;6&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;But it probably won’t be this fast: the video is sped up by a factor of 3.&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="prior" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id7"&gt;7&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;See Dask’s documentation on &lt;a class="reference external" href="https://distributed.dask.org/en/latest/priority.html"&gt;Prioritizing Work&lt;/a&gt;&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="same" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id8"&gt;8&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;Everything is the same between different runs: the hyperparameters sampled, the model’s internal random state, the data passed for fitting. Only the number of workers varies.&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="scale-worker" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id9"&gt;9&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;There’s no time benefit to stopping jobs early if there are infinite workers; there’s never a queue of jobs waiting to be run&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="pickle-post" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;&lt;a role="doc-backlink" href="#id10"&gt;10&lt;/a&gt;&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;“&lt;a class="reference external" href="http://matthewrocklin.com/blog/work/2018/07/23/protocols-pickle"&gt;Pickle isn’t slow, it’s a protocol&lt;/a&gt;” by Matthew Rocklin&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="regularization" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;11&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;Performance comparison: Scikit-learn’s visualization of tuning a Support Vector Machine’s (SVM) regularization parameter: &lt;a class="reference external" href="https://scikit-learn.org/stable/auto_examples/svm/plot_svm_scale_c.html"&gt;Scaling the regularization parameter for SVMs&lt;/a&gt;&lt;/p&gt;
&lt;/aside&gt;
&lt;aside class="footnote brackets" id="new" role="doc-footnote"&gt;
&lt;span class="label"&gt;&lt;span class="fn-bracket"&gt;[&lt;/span&gt;12&lt;span class="fn-bracket"&gt;]&lt;/span&gt;&lt;/span&gt;
&lt;p&gt;It’s been around since 2016… and some call that “old news.”&lt;/p&gt;
&lt;/aside&gt;
&lt;/aside&gt;
</content>
    <link href="https://blog.dask.org/2019/09/30/dask-hyperparam-opt/"/>
    <summary>Scott Sievert wrote this post. The original post lives at
https://stsievert.com/blog/2019/09/27/dask-hyperparam-opt/ with better
styling. This work is supported by Anaconda, Inc.</summary>
    <category term="dask-ml" label="dask-ml"/>
    <category term="machine-learning" label="machine-learning"/>
    <published>2019-09-30T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/09/13/jupyter-on-dask/</id>
    <title>Co-locating a Jupyter Server and Dask Scheduler</title>
    <updated>2019-09-13T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</name>
    </author>
    <content type="html">&lt;p&gt;If you want, you can have Dask set up a Jupyter notebook server for you,
co-located with the Dask scheduler. There are many ways to do this, but this
blog post lists two.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/09/13/jupyter-on-dask.md&lt;/span&gt;, line 13)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="first-why-would-you-do-this"&gt;

&lt;p&gt;Sometimes people inside of large institutions have complex deployment pains.
It takes them a while to stand up a process running on a machine in their
cluster, with all of the appropriate networking ports open and such.
In that situation, it can sometimes be nice to do this just once, say for Dask,
rather than twice, say for Dask and for Jupyter.&lt;/p&gt;
&lt;p&gt;Probably in these cases people should invest in a long term solution like
&lt;a class="reference external" href="https://jupyter.org/hub"&gt;JupyterHub&lt;/a&gt;,
or one of its enterprise variants,
but this blogpost gives a couple of hacks in the meantime.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/09/13/jupyter-on-dask.md&lt;/span&gt;, line 26)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="hack-1-create-a-jupyter-server-from-a-python-function-call"&gt;
&lt;h1&gt;Hack 1: Create a Jupyter server from a Python function call&lt;/h1&gt;
&lt;p&gt;If your Dask scheduler is already running, connect to it with a Client and run
a Python function that starts up a Jupyter server.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;

&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;scheduler-address:8786&amp;quot;&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;start_juptyer_server&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;notebook.notebookapp&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;NotebookApp&lt;/span&gt;
    &lt;span class="n"&gt;app&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;NotebookApp&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;app&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;initialize&lt;/span&gt;&lt;span class="p"&gt;([])&lt;/span&gt;  &lt;span class="c1"&gt;# add command line args here if you want&lt;/span&gt;

&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;run_on_scheduler&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;start_jupyter_server&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If you have a complex networking setup (maybe you’re on the cloud or HPC and
had to open up a port explicitly) then you might want to install
&lt;a class="reference external" href="https://jupyter-server-proxy.readthedocs.io/en/latest/"&gt;jupyter-server-proxy&lt;/a&gt;
(which Dask also uses by default if installed), and then go to
&lt;a class="reference external" href="https://example.com"&gt;http://scheduler-address:8787/proxy/8888&lt;/a&gt; . The Dask dashboard can route your
connection to Jupyter (Jupyter is also kind enough to do the same for Dask if
it is the main service).&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/09/13/jupyter-on-dask.md&lt;/span&gt;, line 52)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="hack-2-preload-script"&gt;
&lt;h1&gt;Hack 2: Preload script&lt;/h1&gt;
&lt;p&gt;This is also a great opportunity to learn about the various ways of &lt;a class="reference external" href="https://docs.dask.org/en/latest/setup/custom-startup.html"&gt;adding
custom startup and teardown&lt;/a&gt;.
One such way, is a preload script 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="c1"&gt;# jupyter-preload.py&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;notebook.notebookapp&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;NotebookApp&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;dask_setup&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;app&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;NotebookApp&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;app&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;initialize&lt;/span&gt;&lt;span class="p"&gt;([])&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;dask-scheduler&lt;span class="w"&gt; &lt;/span&gt;--preload&lt;span class="w"&gt; &lt;/span&gt;jupyter-preload.py
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;That script will run at an appropriate time during scheduler startup. You can
also put this into configuration&lt;/p&gt;
&lt;div class="highlight-yaml notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nt"&gt;distributed&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;scheduler&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;preload&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="p p-Indicator"&gt;[&lt;/span&gt;&lt;span class="s"&gt;&amp;quot;/path/to/jupyter-preload.py&amp;quot;&lt;/span&gt;&lt;span class="p p-Indicator"&gt;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/09/13/jupyter-on-dask.md&lt;/span&gt;, line 80)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="really-though-you-should-use-something-else"&gt;
&lt;h1&gt;Really though, you should use something else&lt;/h1&gt;
&lt;p&gt;This is mostly a hack. If you’re at an institution then you should ask for
something like &lt;a class="reference external" href="https://jupyter.org/hub"&gt;JuptyerHub&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Or, you might also want to run this in a separate subprocess, so that Jupyter
and the Dask scheduler don’t collide with each other. This shouldn’t be so
much of a problem (they’re both pretty light weight), but isolating them
probably makes sense.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/09/13/jupyter-on-dask.md&lt;/span&gt;, line 90)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="thanks-nick"&gt;
&lt;h1&gt;Thanks Nick!&lt;/h1&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="https://github.com/bollwyvl"&gt;Nick Bollweg&lt;/a&gt;, who answered a &lt;a class="reference external" href="https://github.com/jupyter/notebook/issues/4873"&gt;questions on this topic here&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/09/13/jupyter-on-dask/"/>
    <summary>If you want, you can have Dask set up a Jupyter notebook server for you,
co-located with the Dask scheduler. There are many ways to do this, but this
blog post lists two.</summary>
    <category term="HPC" label="HPC"/>
    <published>2019-09-13T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/08/28/dask-on-summit/</id>
    <title>Dask on HPC: a case study</title>
    <updated>2019-08-28T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</name>
    </author>
    <content type="html">&lt;p&gt;Dask is deployed on traditional HPC machines with increasing frequency.
In the past week I’ve personally helped four different groups get set up.
This is a surprisingly individual process,
because every HPC machine has its own idiosyncrasies.
Each machine uses a job scheduler like SLURM/PBS/SGE/LSF/…, a network file
system, and fast interconnect, but each of those sub-systems have slightly
different policies on a machine-by-machine basis, which is where things get tricky.&lt;/p&gt;
&lt;p&gt;Typically we can solve these problems in about 30 minutes if we have both:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Someone familiar with the machine, like a power-user or an IT administrator&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Someone familiar with setting up Dask&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;These systems span a large range of scale. At different ends of this scale
this week I’ve seen both:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;A small in-house 24-node SLURM cluster for research work inside of a
bio-imaging lab&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Summit, the world’s most powerful supercomputer&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;In this post I’m going to share a few notes of what I went through in dealing
with Summit, which was particularly troublesome. Hopefully this gives a sense
for the kinds of situations that arise. These tips likely don’t apply to your
particular system, but hopefully they give a flavor of what can go wrong,
and the processes by which we track things down.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/28/dask-on-summit.md&lt;/span&gt;, line 35)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="power-architecture"&gt;

&lt;p&gt;First, Summit is an IBM PowerPC machine, meaning that packages compiled on
normal Intel chips won’t work. Fortunately, Anaconda maintains a download of
their distribution that works well with the Power architecture, so that gave me
a good starting point.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://www.anaconda.com/distribution/#linux"&gt;https://www.anaconda.com/distribution/#linux&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Packages do seem to be a few months older than for the normal distribution, but
I can live with that.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/28/dask-on-summit.md&lt;/span&gt;, line 47)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="install-dask-jobqueue-and-configure-basic-information"&gt;
&lt;h1&gt;Install Dask-Jobqueue and configure basic information&lt;/h1&gt;
&lt;p&gt;We need to tell Dask how many cores and how much memory is on each machine.
This process is fairly straightforward, is well documented at
&lt;a class="reference external" href="https://jobqueue.dask.org"&gt;jobqueue.dask.org&lt;/a&gt; with an informative screencast,
and even self-directing with error messages.&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_jobqueue&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;PBSCluster&lt;/span&gt;
&lt;span class="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;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;PBSCluster&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="ne"&gt;ValueError&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;You&lt;/span&gt; &lt;span class="n"&gt;must&lt;/span&gt; &lt;span class="n"&gt;specify&lt;/span&gt; &lt;span class="n"&gt;how&lt;/span&gt; &lt;span class="n"&gt;many&lt;/span&gt; &lt;span class="n"&gt;cores&lt;/span&gt; &lt;span class="n"&gt;to&lt;/span&gt; &lt;span class="n"&gt;use&lt;/span&gt; &lt;span class="n"&gt;per&lt;/span&gt; &lt;span class="n"&gt;job&lt;/span&gt; &lt;span class="n"&gt;like&lt;/span&gt; &lt;span class="err"&gt;``&lt;/span&gt;&lt;span class="n"&gt;cores&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;8&lt;/span&gt;&lt;span class="err"&gt;``&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;I’m going to skip this section for now because, generally, novice users are
able to handle this. For more information, consider watching this YouTube
video (30m).&lt;/p&gt;
&lt;iframe width="560" height="315"
        src="https://www.youtube.com/embed/FXsgmwpRExM?rel=0"
        frameborder="0" allow="autoplay; encrypted-media"
        allowfullscreen&gt;&lt;/iframe&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/28/dask-on-summit.md&lt;/span&gt;, line 69)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="invalid-operations-in-the-job-script"&gt;
&lt;h1&gt;Invalid operations in the job script&lt;/h1&gt;
&lt;p&gt;So we make a cluster object with all of our information, we call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.scale&lt;/span&gt;&lt;/code&gt; and
we get some error message from the job 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_jobqueue&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;LSFCluster&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LSFCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;cores&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;128&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;memory&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;600 GB&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;project&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;GEN119&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;walltime&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;00:30&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scale&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# ask for three nodes&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;Command:
bsub /tmp/tmp4874eufw.sh
stdout:

Typical usage:
  bsub [LSF arguments] jobscript
  bsub [LSF arguments] -Is $SHELL
  bsub -h[elp] [options]
  bsub -V

NOTES:
 * All jobs must specify a walltime (-W) and project id (-P)
 * Standard jobs must specify a node count (-nnodes) or -ln_slots. These jobs cannot specify a resource string (-R).
 * Expert mode jobs (-csm y) must specify a resource string and cannot specify -nnodes or -ln_slots.

stderr:
ERROR: Resource strings (-R) are not supported in easy mode. Please resubmit without a resource string.
ERROR: -n is no longer supported. Please request nodes with -nnodes.
ERROR: No nodes requested. Please request nodes with -nnodes.
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Dask-Jobqueue tried to generate a sensible job script from the inputs that you
provided, but the resource manager that you’re using may have additional
policies that are unique to that cluster. We debug this by looking at the
generated script, and comparing against scripts that are known to work on the
HPC machine.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;job_script&lt;/span&gt;&lt;span class="p"&gt;())&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="ch"&gt;#!/usr/bin/env bash&lt;/span&gt;

&lt;span class="c1"&gt;#BSUB -J dask-worker&lt;/span&gt;
&lt;span class="c1"&gt;#BSUB -P GEN119&lt;/span&gt;
&lt;span class="c1"&gt;#BSUB -n 128&lt;/span&gt;
&lt;span class="c1"&gt;#BSUB -R &amp;quot;span[hosts=1]&amp;quot;&lt;/span&gt;
&lt;span class="c1"&gt;#BSUB -M 600000&lt;/span&gt;
&lt;span class="c1"&gt;#BSUB -W 00:30&lt;/span&gt;
&lt;span class="nv"&gt;JOB_ID&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="si"&gt;${&lt;/span&gt;&lt;span class="nv"&gt;LSB_JOBID&lt;/span&gt;&lt;span class="p"&gt;%.*&lt;/span&gt;&lt;span class="si"&gt;}&lt;/span&gt;

/ccs/home/mrocklin/anaconda/bin/python&lt;span class="w"&gt; &lt;/span&gt;-m&lt;span class="w"&gt; &lt;/span&gt;distributed.cli.dask_worker&lt;span class="w"&gt; &lt;/span&gt;tcp://scheduler:8786&lt;span class="w"&gt; &lt;/span&gt;--nthreads&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="m"&gt;16&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;--nprocs&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="m"&gt;8&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;--memory-limit&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="m"&gt;75&lt;/span&gt;.00GB&lt;span class="w"&gt; &lt;/span&gt;--name&lt;span class="w"&gt; &lt;/span&gt;name&lt;span class="w"&gt; &lt;/span&gt;--nanny&lt;span class="w"&gt; &lt;/span&gt;--death-timeout&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="m"&gt;60&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;--interface&lt;span class="w"&gt; &lt;/span&gt;ib0&lt;span class="w"&gt; &lt;/span&gt;--interface&lt;span class="w"&gt; &lt;/span&gt;ib0
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;After comparing notes with existing scripts that we know to work on Summit,
we modify keywords to add and remove certain lines in the header.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LSFCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;cores&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;128&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;memory&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;500 GB&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;project&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;GEN119&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;walltime&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;00:30&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;job_extra&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;-nnodes 1&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;          &lt;span class="c1"&gt;# &amp;lt;--- new!&lt;/span&gt;
    &lt;span class="n"&gt;header_skip&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;-R&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;-n &amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;-M&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;  &lt;span class="c1"&gt;# &amp;lt;--- new!&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And when we call scale this seems to make LSF happy. It no longer dumps out
large error messages.&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;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scale&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# things seem to pass&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/28/dask-on-summit.md&lt;/span&gt;, line 153)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="workers-don-t-connect-to-the-scheduler"&gt;
&lt;h1&gt;Workers don’t connect to the Scheduler&lt;/h1&gt;
&lt;p&gt;So things seem fine from LSF’s perspective, but when we connect up a client to
our cluster we don’t see anything arriving.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&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;client&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;Client: scheduler=&amp;#39;tcp://10.41.0.34:41107&amp;#39; processes=0 cores=0&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Two things to check, have the jobs actually made it through the queue?
Typically we use a resource manager operation, like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;qstat&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;squeue&lt;/span&gt;&lt;/code&gt;, or
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;bjobs&lt;/span&gt;&lt;/code&gt; for this. Maybe our jobs are trapped in the queue?&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ bash
JOBID   USER       STAT   SLOTS    QUEUE       START_TIME    FINISH_TIME   JOB_NAME
600785  mrocklin   RUN    43       batch       Aug 26 13:11  Aug 26 13:41  dask-worker
600786  mrocklin   RUN    43       batch       Aug 26 13:11  Aug 26 13:41  dask-worker
600784  mrocklin   RUN    43       batch       Aug 26 13:11  Aug 26 13:41  dask-worker
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Nope, it looks like they’re in a running state. Now we go and look at their
logs. It can sometimes be tricky to track down the log files from your jobs,
but your IT administrator should know where they are. Often they’re where you
ran your job from, and have the Job ID in the filename.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ cat dask-worker.600784.err
distributed.worker - INFO -       Start worker at: tcp://128.219.134.81:44053
distributed.worker - INFO -          Listening to: tcp://128.219.134.81:44053
distributed.worker - INFO -          dashboard at:       128.219.134.81:34583
distributed.worker - INFO - Waiting to connect to: tcp://128.219.134.74:34153
distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO -               Threads:                         16
distributed.worker - INFO -                Memory:                   75.00 GB
distributed.worker - INFO -       Local Directory: /autofs/nccs-svm1_home1/mrocklin/worker-ybnhk4ib
distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO - Waiting to connect to: tcp://128.219.134.74:34153
distributed.worker - INFO - Waiting to connect to: tcp://128.219.134.74:34153
distributed.worker - INFO - Waiting to connect to: tcp://128.219.134.74:34153
distributed.worker - INFO - Waiting to connect to: tcp://128.219.134.74:34153
distributed.worker - INFO - Waiting to connect to: tcp://128.219.134.74:34153
distributed.worker - INFO - Waiting to connect to: tcp://128.219.134.74:34153
...
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So the worker processes have started, but they’re having difficulty connecting
to the scheduler. When we ask at IT administrator they identify the address
here as on the wrong network interface:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="mf"&gt;128.219.134.74&lt;/span&gt;  &lt;span class="o"&gt;&amp;lt;---&lt;/span&gt; &lt;span class="ow"&gt;not&lt;/span&gt; &lt;span class="n"&gt;accessible&lt;/span&gt; &lt;span class="n"&gt;network&lt;/span&gt; &lt;span class="n"&gt;address&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So we run &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ifconfig&lt;/span&gt;&lt;/code&gt;, and find the infiniband network interface, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ib0&lt;/span&gt;&lt;/code&gt;, which
is more broadly accessible.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LSFCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;cores&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;128&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;memory&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;500 GB&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;project&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;GEN119&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;walltime&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;00:30&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;job_extra&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;-nnodes 1&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
    &lt;span class="n"&gt;header_skip&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;-R&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;-n &amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;-M&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
    &lt;span class="n"&gt;interface&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;ib0&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;                    &lt;span class="c1"&gt;# &amp;lt;--- new!&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We try this out and still, no luck :(&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/28/dask-on-summit.md&lt;/span&gt;, line 227)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="interactive-nodes"&gt;
&lt;h1&gt;Interactive nodes&lt;/h1&gt;
&lt;p&gt;The expert user then says “Oh, our login nodes are pretty locked-down, lets try
this from an interactive compute node. Things tend to work better there”. We
run some arcane bash command (I’ve never seen two of these that look alike so
I’m going to omit it here), and things magically start working. Hooray!&lt;/p&gt;
&lt;p&gt;We run a tiny Dask computation just to prove that we can do some work.&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;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&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;1&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;11&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Actually, it turns out that we were eventually able to get things running from
the login nodes on Summit using a slightly different &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;bsub&lt;/span&gt;&lt;/code&gt; command in LSF, but
I’m going to omit details here because we’re fixing this in Dask and it’s
unlikely to affect future users (I hope?). Locked down login nodes remain a
common cause of no connections across a variety of systems. I’ll say something
like 30% of the systems that I interact with.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/28/dask-on-summit.md&lt;/span&gt;, line 249)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="ssh-tunneling"&gt;
&lt;h1&gt;SSH Tunneling&lt;/h1&gt;
&lt;p&gt;It’s important to get the dashboard up and running so that you can see what’s
going on. Typically we do this with SSH tunnelling. Most HPC people know how
to do this and it’s covered in the Youtube screencast above, so I’m going to
skip it here.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/28/dask-on-summit.md&lt;/span&gt;, line 256)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="jupyter-lab"&gt;
&lt;h1&gt;Jupyter Lab&lt;/h1&gt;
&lt;p&gt;Many interactive Dask users on HPC today are moving towards using JupyterLab.
This choice gives them a notebook, terminals, file browser, and Dask’s
dashboard all in a single web tab. This greatly reduces the number of times
they have to SSH in, and, with the magic of web proxies, means that they only
need to tunnel once.&lt;/p&gt;
&lt;p&gt;I conda installed JupyterLab and a proxy library, and then tried to
&lt;a class="reference external" href="https://github.com/dask/dask-labextension#installation"&gt;set up the Dask JupyterLab extension&lt;/a&gt;.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;conda&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;jupyterlab&lt;/span&gt;
&lt;span class="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;jupyter&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;server&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;proxy&lt;/span&gt;  &lt;span class="c1"&gt;# to route dashboard through Jupyter&amp;#39;s port&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Next, we’re going to install the
&lt;a class="reference external" href="https://github.com/dask/dask-labextension"&gt;Dask Labextension&lt;/a&gt; into JupyterLab
in order to get the Dask Dashboard directly into our Jupyter session..
For that, we need &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;nodejs&lt;/span&gt;&lt;/code&gt; in order to install things into JupyterLab.
I thought that this was going to be a pain, given the Power architecture, but
amazingly, this also seems to be in Anaconda’s default Power channel.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;mrocklin@login2.summit $ conda install nodejs  # Thanks conda packaging devs!
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Then I install Dask-Labextension, which is both a Python and a JavaScript
package:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;dask_labextension&lt;/span&gt;
&lt;span class="n"&gt;jupyter&lt;/span&gt; &lt;span class="n"&gt;labextension&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;labextension&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Then I set up a password for my Jupyter sessions&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;jupyter&lt;/span&gt; &lt;span class="n"&gt;notebook&lt;/span&gt; &lt;span class="n"&gt;password&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And run JupyterLab in a network friendly way&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;mrocklin@login2.summit $ jupyter lab --no-browser --ip=&amp;quot;login2&amp;quot;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And set up a single SSH tunnel from my home machine to the login node&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;# Be sure to match the login node&amp;#39;s hostname and the Jupyter port below

mrocklin@my-laptop $ ssh -L 8888:login2:8888 summit.olcf.ornl.gov
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;I can now connect to Jupyter from my laptop by navigating to
&lt;a class="reference external" href="http://localhost:8888"&gt;http://localhost:8888&lt;/a&gt; , run the cluster commands above in a notebook, and
things work great. Additionally, thanks to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;jupyter-server-proxy&lt;/span&gt;&lt;/code&gt;, Dask’s
dashboard is also available at &lt;a class="reference external" href="http://localhost:8888/proxy/####/status"&gt;http://localhost:8888/proxy/####/status&lt;/a&gt; , where
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;####&lt;/span&gt;&lt;/code&gt; is the port currently hosting Dask’s dashboard. You can probably find
this by looking at &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cluster.dashboard_link&lt;/span&gt;&lt;/code&gt;. It defaults to 8787, but if
you’ve started a bunch of Dask schedulers on your system recently it’s possible
that that port is taken up and so Dask had to resort to using random ports.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/28/dask-on-summit.md&lt;/span&gt;, line 320)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="configuration-files"&gt;
&lt;h1&gt;Configuration files&lt;/h1&gt;
&lt;p&gt;I don’t want to keep typing all of these commands, so now I put things into a
single configuration file, and plop that file into &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;~/.config/dask/summit.yaml&lt;/span&gt;&lt;/code&gt;
(any filename that ends in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.yaml&lt;/span&gt;&lt;/code&gt; will do).&lt;/p&gt;
&lt;div class="highlight-yaml notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nt"&gt;jobqueue&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;lsf&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;cores&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;128&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;processes&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;8&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;memory&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;500 GB&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;job-extra&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="s"&gt;&amp;quot;-nnodes&lt;/span&gt;&lt;span class="nv"&gt; &lt;/span&gt;&lt;span class="s"&gt;1&amp;quot;&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;interface&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;ib0&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;header-skip&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="s"&gt;&amp;quot;-R&amp;quot;&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="s"&gt;&amp;quot;-n&lt;/span&gt;&lt;span class="nv"&gt; &lt;/span&gt;&lt;span class="s"&gt;&amp;quot;&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="p p-Indicator"&gt;-&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="s"&gt;&amp;quot;-M&amp;quot;&lt;/span&gt;

&lt;span class="nt"&gt;labextension&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;factory&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;module&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="s"&gt;&amp;quot;dask_jobqueue&amp;quot;&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;class&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="s"&gt;&amp;quot;LSFCluster&amp;quot;&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;args&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="p p-Indicator"&gt;[]&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;      &lt;/span&gt;&lt;span class="nt"&gt;project&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;your-project-id&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/28/dask-on-summit.md&lt;/span&gt;, line 349)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="slow-worker-startup"&gt;
&lt;h1&gt;Slow worker startup&lt;/h1&gt;
&lt;p&gt;Now that things are easier to use I find myself using the system more, and some
other problems arise.&lt;/p&gt;
&lt;p&gt;I notice that it takes a long time to start up a worker. It seems to hang
intermittently during startup, so I add a few lines to
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed/__init__.py&lt;/span&gt;&lt;/code&gt; to print out the state of the main Python thread
every second, to see where this is happening:&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;threading&lt;/span&gt;&lt;span class="o"&gt;,&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;sys&lt;/span&gt;&lt;span class="o"&gt;,&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;time&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;.&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;profile&lt;/span&gt;

&lt;span class="n"&gt;main_thread&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;threading&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_ident&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;f&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="k"&gt;while&lt;/span&gt; &lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sleep&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;frame&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_current_frames&lt;/span&gt;&lt;span class="p"&gt;()[&lt;/span&gt;&lt;span class="n"&gt;main_thread&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
        &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;join&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;profile&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;call_stack&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;frame&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;thread&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;threading&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Thread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;target&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;daemon&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;thraed&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;start&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This prints out a traceback that brings us to this code in Dask:&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;if&lt;/span&gt; &lt;span class="n"&gt;is_locking_enabled&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="k"&gt;try&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_lock_path&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;os&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;join&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dir_path&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;DIR_LOCK_EXT&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="k"&gt;assert&lt;/span&gt; &lt;span class="ow"&gt;not&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;path&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;exists&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_lock_path&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;logger&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;debug&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Locking &lt;/span&gt;&lt;span class="si"&gt;%r&lt;/span&gt;&lt;span class="s2"&gt;...&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_lock_path&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="c1"&gt;# Avoid a race condition before locking the file&lt;/span&gt;
        &lt;span class="c1"&gt;# by taking the global lock&lt;/span&gt;
        &lt;span class="k"&gt;try&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;workspace&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_global_lock&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
                    &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_lock_file&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;locket&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;lock_file&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_lock_path&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
                    &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_lock_file&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;acquire&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;It looks like Dask is trying to use a file-based lock.
Unfortunately some NFS systems don’t like file-based locks, or handle them very
slowly. In the case of Summit, the home directory is actually mounted
read-only from the compute nodes, so a file-based lock will simply fail.
Looking up the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;is_locking_enabled&lt;/span&gt;&lt;/code&gt; function we see that it checks a
configuration value.&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;is_locking_enabled&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;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;distributed.worker.use-file-locking&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So we add that to our config file. At the same time I switch from the
forkserver to spawn multiprocessing method (I thought that this might help, but
it didn’t), which is relatively harmless.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;distributed&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
  &lt;span class="n"&gt;worker&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;multiprocessing&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;method&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;spawn&lt;/span&gt;
    &lt;span class="n"&gt;use&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;file&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;locking&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="kc"&gt;False&lt;/span&gt;

&lt;span class="n"&gt;jobqueue&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
  &lt;span class="n"&gt;lsf&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;cores&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;128&lt;/span&gt;
    &lt;span class="n"&gt;processes&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;8&lt;/span&gt;
    &lt;span class="n"&gt;memory&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;500&lt;/span&gt; &lt;span class="n"&gt;GB&lt;/span&gt;
    &lt;span class="n"&gt;job&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;extra&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
      &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;-nnodes 1&amp;quot;&lt;/span&gt;
    &lt;span class="n"&gt;interface&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;ib0&lt;/span&gt;
    &lt;span class="n"&gt;header&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;skip&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;-R&amp;quot;&lt;/span&gt;
    &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;-n &amp;quot;&lt;/span&gt;
    &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;-M&amp;quot;&lt;/span&gt;

&lt;span class="n"&gt;labextension&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
  &lt;span class="n"&gt;factory&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
     &lt;span class="n"&gt;module&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;dask_jobqueue&amp;#39;&lt;/span&gt;
     &lt;span class="n"&gt;class&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;LSFCluster&amp;#39;&lt;/span&gt;
     &lt;span class="n"&gt;args&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;[]&lt;/span&gt;
     &lt;span class="n"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
       &lt;span class="n"&gt;project&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;your&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;project&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="nb"&gt;id&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/28/dask-on-summit.md&lt;/span&gt;, line 435)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, 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;This post outlines many issues that I ran into when getting Dask to run on
one specific HPC system. These problems aren’t universal, so you may not run
into them, but they’re also not super-rare. Mostly my objective in writing
this up is to give people a sense of the sorts of problems that arise when
Dask and an HPC system interact.&lt;/p&gt;
&lt;p&gt;None of the problems above are that serious. They’ve all happened before and
they all have solutions that can be written down in a configuration file.
Finding what the problem is though can be challenging, and often requires the
combined expertise of individuals that are experienced with Dask and with that
particular HPC system.&lt;/p&gt;
&lt;p&gt;There are a few configuration files posted here
&lt;a class="reference external" href="https://jobqueue.dask.org/en/latest/configurations.html"&gt;jobqueue.dask.org/en/latest/configurations.html&lt;/a&gt;, which may be informative. The &lt;a class="reference external" href="https://github.com/dask/dask-jobqueue/issues"&gt;Dask Jobqueue issue tracker&lt;/a&gt; is also a fairly friendly place, full of both IT professionals and Dask experts.&lt;/p&gt;
&lt;p&gt;Also, as a reminder, you don’t need to have an HPC machine in order to use
Dask. Dask is conveniently deployable from other Cloud, Hadoop, and local
systems. See the &lt;a class="reference external" href="https://docs.dask.org/en/latest/setup.html"&gt;Dask setup
documentation&lt;/a&gt; for more
information.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/28/dask-on-summit.md&lt;/span&gt;, line 458)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="future-work-gpus"&gt;
&lt;h1&gt;Future work: GPUs&lt;/h1&gt;
&lt;p&gt;Summit is fast because it has a ton of GPUs. I’m going to work on that next,
but that will probably cover enough content to fill up a whole other blogpost :)&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/28/dask-on-summit.md&lt;/span&gt;, line 463)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="branches"&gt;
&lt;h1&gt;Branches&lt;/h1&gt;
&lt;p&gt;For anyone playing along at home (or on Summit). I’m operating from the
following development branches:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="github reference external" href="https://github.com/dask/distributed&amp;#64;master"&gt;dask/distributed&amp;#64;master&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/mrocklin/dask-jobqueue&amp;#64;spec-rewrite"&gt;mrocklin/dask-jobqueue&amp;#64;spec-rewrite&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Although hopefully within a month of writing this article, everything should be
in a nicely released state.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/08/28/dask-on-summit/"/>
    <summary>Dask is deployed on traditional HPC machines with increasing frequency.
In the past week I’ve personally helped four different groups get set up.
This is a surprisingly individual process,
because every HPC machine has its own idiosyncrasies.
Each machine uses a job scheduler like SLURM/PBS/SGE/LSF/…, a network file
system, and fast interconnect, but each of those sub-systems have slightly
different policies on a machine-by-machine basis, which is where things get tricky.</summary>
    <category term="HPC" label="HPC"/>
    <published>2019-08-28T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/08/09/image-itk/</id>
    <title>Dask and ITK for large scale image analysis</title>
    <updated>2019-08-09T00:00:00+00:00</updated>
    <author>
      <name>Matthew McCormick</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/09/image-itk.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;This post explores using the &lt;a class="reference external" href="https://www.itk.org"&gt;ITK&lt;/a&gt; suite of image processing utilities in parallel with Dask Array.&lt;/p&gt;
&lt;p&gt;We cover …&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;A simple but common example of applying deconvolution across a stack of 3d images&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tips on how to make these two libraries work well together&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Challenges that we ran into and opportunities for future improvements.&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/2019/08/09/image-itk.md&lt;/span&gt;, line 19)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="a-worked-example"&gt;
&lt;h1&gt;A Worked Example&lt;/h1&gt;
&lt;p&gt;Let’s start with a full example applying Richardson Lucy deconvolution to a
stack of light sheet microscopy data. This is the same data that we showed how
to load in our &lt;a class="reference external" href="https://blog.dask.org/2019/06/20/load-image-data"&gt;last blogpost on image loading&lt;/a&gt;.
You can &lt;a class="reference external" href="https://drive.google.com/drive/folders/13mpIfqspKTIINkfoWbFsVtFF8D7jbTqJ"&gt;access the data as tiff files from google drive here&lt;/a&gt;, and the access the &lt;a class="reference external" href="https://drive.google.com/drive/folders/13udO-h9epItG5MNWBp0VxBkKCllYBLQF"&gt;corresponding point spread function images here&lt;/a&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="c1"&gt;# Load our data from last time¶&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;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="n"&gt;imgs&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;from_zarr&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;AOLLSMData_m4_raw.zarr/&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;data&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table&gt;
&lt;tr&gt;
&lt;td&gt;
&lt;table&gt;  &lt;thead&gt;    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 188.74 GB &lt;/td&gt; &lt;td&gt; 316.15 MB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (3, 199, 201, 1024, 768) &lt;/td&gt; &lt;td&gt; (1, 1, 201, 1024, 768) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 598 Tasks &lt;/td&gt;&lt;td&gt; 597 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; uint16 &lt;/td&gt;&lt;td&gt; numpy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;&lt;/table&gt;
&lt;/td&gt;
&lt;td&gt;
&lt;svg width="404" height="206" style="stroke:rgb(0,0,0);stroke-width:1" &gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="0" y1="0" x2="45" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="0" y1="9" x2="45" y2="9" /&gt;
  &lt;line x1="0" y1="18" x2="45" y2="18" /&gt;
  &lt;line x1="0" y1="27" x2="45" y2="27" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="0" y1="0" x2="0" y2="27" style="stroke-width:2" /&gt;
  &lt;line x1="0" y1="0" x2="0" y2="27" /&gt;
  &lt;line x1="0" y1="0" x2="0" y2="27" /&gt;
  &lt;line x1="0" y1="0" x2="0" y2="27" /&gt;
  &lt;line x1="0" y1="0" x2="0" y2="27" /&gt;
  &lt;line x1="1" y1="0" x2="1" y2="27" /&gt;
  &lt;line x1="1" y1="0" x2="1" y2="27" /&gt;
  &lt;line x1="1" y1="0" x2="1" y2="27" /&gt;
  &lt;line x1="1" y1="0" x2="1" y2="27" /&gt;
  &lt;line x1="2" y1="0" x2="2" y2="27" /&gt;
  &lt;line x1="2" y1="0" x2="2" y2="27" /&gt;
  &lt;line x1="2" y1="0" x2="2" y2="27" /&gt;
  &lt;line x1="2" y1="0" x2="2" y2="27" /&gt;
  &lt;line x1="2" y1="0" x2="2" y2="27" /&gt;
  &lt;line x1="3" y1="0" x2="3" y2="27" /&gt;
  &lt;line x1="3" y1="0" x2="3" y2="27" /&gt;
  &lt;line x1="3" y1="0" x2="3" y2="27" /&gt;
  &lt;line x1="3" y1="0" x2="3" y2="27" /&gt;
  &lt;line x1="4" y1="0" x2="4" y2="27" /&gt;
  &lt;line x1="4" y1="0" x2="4" y2="27" /&gt;
  &lt;line x1="4" y1="0" x2="4" y2="27" /&gt;
  &lt;line x1="4" y1="0" x2="4" y2="27" /&gt;
  &lt;line x1="5" y1="0" x2="5" y2="27" /&gt;
  &lt;line x1="5" y1="0" x2="5" y2="27" /&gt;
  &lt;line x1="5" y1="0" x2="5" y2="27" /&gt;
  &lt;line x1="5" y1="0" x2="5" y2="27" /&gt;
  &lt;line x1="5" y1="0" x2="5" y2="27" /&gt;
  &lt;line x1="6" y1="0" x2="6" y2="27" /&gt;
  &lt;line x1="6" y1="0" x2="6" y2="27" /&gt;
  &lt;line x1="6" y1="0" x2="6" y2="27" /&gt;
  &lt;line x1="6" y1="0" x2="6" y2="27" /&gt;
  &lt;line x1="7" y1="0" x2="7" y2="27" /&gt;
  &lt;line x1="7" y1="0" x2="7" y2="27" /&gt;
  &lt;line x1="7" y1="0" x2="7" y2="27" /&gt;
  &lt;line x1="7" y1="0" x2="7" y2="27" /&gt;
  &lt;line x1="7" y1="0" x2="7" y2="27" /&gt;
  &lt;line x1="8" y1="0" x2="8" y2="27" /&gt;
  &lt;line x1="8" y1="0" x2="8" y2="27" /&gt;
  &lt;line x1="8" y1="0" x2="8" y2="27" /&gt;
  &lt;line x1="8" y1="0" x2="8" y2="27" /&gt;
  &lt;line x1="9" y1="0" x2="9" y2="27" /&gt;
  &lt;line x1="9" y1="0" x2="9" y2="27" /&gt;
  &lt;line x1="9" y1="0" x2="9" y2="27" /&gt;
  &lt;line x1="9" y1="0" x2="9" y2="27" /&gt;
  &lt;line x1="10" y1="0" x2="10" y2="27" /&gt;
  &lt;line x1="10" y1="0" x2="10" y2="27" /&gt;
  &lt;line x1="10" y1="0" x2="10" y2="27" /&gt;
  &lt;line x1="10" y1="0" x2="10" y2="27" /&gt;
  &lt;line x1="10" y1="0" x2="10" y2="27" /&gt;
  &lt;line x1="11" y1="0" x2="11" y2="27" /&gt;
  &lt;line x1="11" y1="0" x2="11" y2="27" /&gt;
  &lt;line x1="11" y1="0" x2="11" y2="27" /&gt;
  &lt;line x1="11" y1="0" x2="11" y2="27" /&gt;
  &lt;line x1="12" y1="0" x2="12" y2="27" /&gt;
  &lt;line x1="12" y1="0" x2="12" y2="27" /&gt;
  &lt;line x1="12" y1="0" x2="12" y2="27" /&gt;
  &lt;line x1="12" y1="0" x2="12" y2="27" /&gt;
  &lt;line x1="12" y1="0" x2="12" y2="27" /&gt;
  &lt;line x1="13" y1="0" x2="13" y2="27" /&gt;
  &lt;line x1="13" y1="0" x2="13" y2="27" /&gt;
  &lt;line x1="13" y1="0" x2="13" y2="27" /&gt;
  &lt;line x1="13" y1="0" x2="13" y2="27" /&gt;
  &lt;line x1="14" y1="0" x2="14" y2="27" /&gt;
  &lt;line x1="14" y1="0" x2="14" y2="27" /&gt;
  &lt;line x1="14" y1="0" x2="14" y2="27" /&gt;
  &lt;line x1="14" y1="0" x2="14" y2="27" /&gt;
  &lt;line x1="15" y1="0" x2="15" y2="27" /&gt;
  &lt;line x1="15" y1="0" x2="15" y2="27" /&gt;
  &lt;line x1="15" y1="0" x2="15" y2="27" /&gt;
  &lt;line x1="15" y1="0" x2="15" y2="27" /&gt;
  &lt;line x1="15" y1="0" x2="15" y2="27" /&gt;
  &lt;line x1="16" y1="0" x2="16" y2="27" /&gt;
  &lt;line x1="16" y1="0" x2="16" y2="27" /&gt;
  &lt;line x1="16" y1="0" x2="16" y2="27" /&gt;
  &lt;line x1="16" y1="0" x2="16" y2="27" /&gt;
  &lt;line x1="17" y1="0" x2="17" y2="27" /&gt;
  &lt;line x1="17" y1="0" x2="17" y2="27" /&gt;
  &lt;line x1="17" y1="0" x2="17" y2="27" /&gt;
  &lt;line x1="17" y1="0" x2="17" y2="27" /&gt;
  &lt;line x1="18" y1="0" x2="18" y2="27" /&gt;
  &lt;line x1="18" y1="0" x2="18" y2="27" /&gt;
  &lt;line x1="18" y1="0" x2="18" y2="27" /&gt;
  &lt;line x1="18" y1="0" x2="18" y2="27" /&gt;
  &lt;line x1="18" y1="0" x2="18" y2="27" /&gt;
  &lt;line x1="19" y1="0" x2="19" y2="27" /&gt;
  &lt;line x1="19" y1="0" x2="19" y2="27" /&gt;
  &lt;line x1="19" y1="0" x2="19" y2="27" /&gt;
  &lt;line x1="19" y1="0" x2="19" y2="27" /&gt;
  &lt;line x1="20" y1="0" x2="20" y2="27" /&gt;
  &lt;line x1="20" y1="0" x2="20" y2="27" /&gt;
  &lt;line x1="20" y1="0" x2="20" y2="27" /&gt;
  &lt;line x1="20" y1="0" x2="20" y2="27" /&gt;
  &lt;line x1="20" y1="0" x2="20" y2="27" /&gt;
  &lt;line x1="21" y1="0" x2="21" y2="27" /&gt;
  &lt;line x1="21" y1="0" x2="21" y2="27" /&gt;
  &lt;line x1="21" y1="0" x2="21" y2="27" /&gt;
  &lt;line x1="21" y1="0" x2="21" y2="27" /&gt;
  &lt;line x1="22" y1="0" x2="22" y2="27" /&gt;
  &lt;line x1="22" y1="0" x2="22" y2="27" /&gt;
  &lt;line x1="22" y1="0" x2="22" y2="27" /&gt;
  &lt;line x1="22" y1="0" x2="22" y2="27" /&gt;
  &lt;line x1="23" y1="0" x2="23" y2="27" /&gt;
  &lt;line x1="23" y1="0" x2="23" y2="27" /&gt;
  &lt;line x1="23" y1="0" x2="23" y2="27" /&gt;
  &lt;line x1="23" y1="0" x2="23" y2="27" /&gt;
  &lt;line x1="23" y1="0" x2="23" y2="27" /&gt;
  &lt;line x1="24" y1="0" x2="24" y2="27" /&gt;
  &lt;line x1="24" y1="0" x2="24" y2="27" /&gt;
  &lt;line x1="24" y1="0" x2="24" y2="27" /&gt;
  &lt;line x1="24" y1="0" x2="24" y2="27" /&gt;
  &lt;line x1="25" y1="0" x2="25" y2="27" /&gt;
  &lt;line x1="25" y1="0" x2="25" y2="27" /&gt;
  &lt;line x1="25" y1="0" x2="25" y2="27" /&gt;
  &lt;line x1="25" y1="0" x2="25" y2="27" /&gt;
  &lt;line x1="25" y1="0" x2="25" y2="27" /&gt;
  &lt;line x1="26" y1="0" x2="26" y2="27" /&gt;
  &lt;line x1="26" y1="0" x2="26" y2="27" /&gt;
  &lt;line x1="26" y1="0" x2="26" y2="27" /&gt;
  &lt;line x1="26" y1="0" x2="26" y2="27" /&gt;
  &lt;line x1="27" y1="0" x2="27" y2="27" /&gt;
  &lt;line x1="27" y1="0" x2="27" y2="27" /&gt;
  &lt;line x1="27" y1="0" x2="27" y2="27" /&gt;
  &lt;line x1="27" y1="0" x2="27" y2="27" /&gt;
  &lt;line x1="28" y1="0" x2="28" y2="27" /&gt;
  &lt;line x1="28" y1="0" x2="28" y2="27" /&gt;
  &lt;line x1="28" y1="0" x2="28" y2="27" /&gt;
  &lt;line x1="28" y1="0" x2="28" y2="27" /&gt;
  &lt;line x1="28" y1="0" x2="28" y2="27" /&gt;
  &lt;line x1="29" y1="0" x2="29" y2="27" /&gt;
  &lt;line x1="29" y1="0" x2="29" y2="27" /&gt;
  &lt;line x1="29" y1="0" x2="29" y2="27" /&gt;
  &lt;line x1="29" y1="0" x2="29" y2="27" /&gt;
  &lt;line x1="30" y1="0" x2="30" y2="27" /&gt;
  &lt;line x1="30" y1="0" x2="30" y2="27" /&gt;
  &lt;line x1="30" y1="0" x2="30" y2="27" /&gt;
  &lt;line x1="30" y1="0" x2="30" y2="27" /&gt;
  &lt;line x1="31" y1="0" x2="31" y2="27" /&gt;
  &lt;line x1="31" y1="0" x2="31" y2="27" /&gt;
  &lt;line x1="31" y1="0" x2="31" y2="27" /&gt;
  &lt;line x1="31" y1="0" x2="31" y2="27" /&gt;
  &lt;line x1="31" y1="0" x2="31" y2="27" /&gt;
  &lt;line x1="32" y1="0" x2="32" y2="27" /&gt;
  &lt;line x1="32" y1="0" x2="32" y2="27" /&gt;
  &lt;line x1="32" y1="0" x2="32" y2="27" /&gt;
  &lt;line x1="32" y1="0" x2="32" y2="27" /&gt;
  &lt;line x1="33" y1="0" x2="33" y2="27" /&gt;
  &lt;line x1="33" y1="0" x2="33" y2="27" /&gt;
  &lt;line x1="33" y1="0" x2="33" y2="27" /&gt;
  &lt;line x1="33" y1="0" x2="33" y2="27" /&gt;
  &lt;line x1="33" y1="0" x2="33" y2="27" /&gt;
  &lt;line x1="34" y1="0" x2="34" y2="27" /&gt;
  &lt;line x1="34" y1="0" x2="34" y2="27" /&gt;
  &lt;line x1="34" y1="0" x2="34" y2="27" /&gt;
  &lt;line x1="34" y1="0" x2="34" y2="27" /&gt;
  &lt;line x1="35" y1="0" x2="35" y2="27" /&gt;
  &lt;line x1="35" y1="0" x2="35" y2="27" /&gt;
  &lt;line x1="35" y1="0" x2="35" y2="27" /&gt;
  &lt;line x1="35" y1="0" x2="35" y2="27" /&gt;
  &lt;line x1="36" y1="0" x2="36" y2="27" /&gt;
  &lt;line x1="36" y1="0" x2="36" y2="27" /&gt;
  &lt;line x1="36" y1="0" x2="36" y2="27" /&gt;
  &lt;line x1="36" y1="0" x2="36" y2="27" /&gt;
  &lt;line x1="36" y1="0" x2="36" y2="27" /&gt;
  &lt;line x1="37" y1="0" x2="37" y2="27" /&gt;
  &lt;line x1="37" y1="0" x2="37" y2="27" /&gt;
  &lt;line x1="37" y1="0" x2="37" y2="27" /&gt;
  &lt;line x1="37" y1="0" x2="37" y2="27" /&gt;
  &lt;line x1="38" y1="0" x2="38" y2="27" /&gt;
  &lt;line x1="38" y1="0" x2="38" y2="27" /&gt;
  &lt;line x1="38" y1="0" x2="38" y2="27" /&gt;
  &lt;line x1="38" y1="0" x2="38" y2="27" /&gt;
  &lt;line x1="38" y1="0" x2="38" y2="27" /&gt;
  &lt;line x1="39" y1="0" x2="39" y2="27" /&gt;
  &lt;line x1="39" y1="0" x2="39" y2="27" /&gt;
  &lt;line x1="39" y1="0" x2="39" y2="27" /&gt;
  &lt;line x1="39" y1="0" x2="39" y2="27" /&gt;
  &lt;line x1="40" y1="0" x2="40" y2="27" /&gt;
  &lt;line x1="40" y1="0" x2="40" y2="27" /&gt;
  &lt;line x1="40" y1="0" x2="40" y2="27" /&gt;
  &lt;line x1="40" y1="0" x2="40" y2="27" /&gt;
  &lt;line x1="41" y1="0" x2="41" y2="27" /&gt;
  &lt;line x1="41" y1="0" x2="41" y2="27" /&gt;
  &lt;line x1="41" y1="0" x2="41" y2="27" /&gt;
  &lt;line x1="41" y1="0" x2="41" y2="27" /&gt;
  &lt;line x1="41" y1="0" x2="41" y2="27" /&gt;
  &lt;line x1="42" y1="0" x2="42" y2="27" /&gt;
  &lt;line x1="42" y1="0" x2="42" y2="27" /&gt;
  &lt;line x1="42" y1="0" x2="42" y2="27" /&gt;
  &lt;line x1="42" y1="0" x2="42" y2="27" /&gt;
  &lt;line x1="43" y1="0" x2="43" y2="27" /&gt;
  &lt;line x1="43" y1="0" x2="43" y2="27" /&gt;
  &lt;line x1="43" y1="0" x2="43" y2="27" /&gt;
  &lt;line x1="43" y1="0" x2="43" y2="27" /&gt;
  &lt;line x1="44" y1="0" x2="44" y2="27" /&gt;
  &lt;line x1="44" y1="0" x2="44" y2="27" /&gt;
  &lt;line x1="44" y1="0" x2="44" y2="27" /&gt;
  &lt;line x1="44" y1="0" x2="44" y2="27" /&gt;
  &lt;line x1="44" y1="0" x2="44" y2="27" /&gt;
  &lt;line x1="45" y1="0" x2="45" y2="27" /&gt;
  &lt;line x1="45" y1="0" x2="45" y2="27" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="0.000000,0.000000 45.378219,0.000000 45.378219,27.530335 0.000000,27.530335" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="22.689110" y="47.530335" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;199&lt;/text&gt;
&lt;text x="65.378219" y="13.765167" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(0,65.378219,13.765167)"&gt;3&lt;/text&gt;&lt;/p&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="115" y1="0" x2="141" y2="26" style="stroke-width:2" /&gt;
  &lt;line x1="115" y1="130" x2="141" y2="156" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="115" y1="0" x2="115" y2="130" style="stroke-width:2" /&gt;
  &lt;line x1="141" y1="26" x2="141" y2="156" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="115.000000,0.000000 141.720328,26.720328 141.720328,156.720328 115.000000,130.000000" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="115" y1="0" x2="212" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="141" y1="26" x2="239" y2="26" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="115" y1="0" x2="141" y2="26" style="stroke-width:2" /&gt;
  &lt;line x1="212" y1="0" x2="239" y2="26" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="115.000000,0.000000 212.500000,0.000000 239.220328,26.720328 141.720328,26.720328" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="141" y1="26" x2="239" y2="26" style="stroke-width:2" /&gt;
  &lt;line x1="141" y1="156" x2="239" y2="156" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="141" y1="26" x2="141" y2="156" style="stroke-width:2" /&gt;
  &lt;line x1="239" y1="26" x2="239" y2="156" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="141.720328,26.720328 239.220328,26.720328 239.220328,156.720328 141.720328,156.720328" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="190.470328" y="176.720328" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;768&lt;/text&gt;
&lt;text x="259.220328" y="91.720328" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(-90,259.220328,91.720328)"&gt;1024&lt;/text&gt;
&lt;text x="118.360164" y="163.360164" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(45,118.360164,163.360164)"&gt;201&lt;/text&gt;
&lt;/svg&gt;&lt;/p&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;This dataset has shape &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(3,&lt;/span&gt; &lt;span class="pre"&gt;199,&lt;/span&gt; &lt;span class="pre"&gt;201,&lt;/span&gt; &lt;span class="pre"&gt;1024,&lt;/span&gt; &lt;span class="pre"&gt;768)&lt;/span&gt;&lt;/code&gt;:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;3 fluorescence color channels,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;199 time points,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;201 z-slices,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;1024 pixels in the y dimension, and&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;768 pixels in the x dimension.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Load our Point Spread Function (PSF)&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.image&lt;/span&gt;
&lt;span class="n"&gt;psf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;image&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;AOLLSMData/m4/psfs_z0p1/*.tif&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)[:,&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table&gt;
&lt;tr&gt;
&lt;td&gt;
&lt;table&gt;  &lt;thead&gt;    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 2.48 MB &lt;/td&gt; &lt;td&gt; 827.39 kB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (3, 1, 101, 64, 64) &lt;/td&gt; &lt;td&gt; (1, 1, 101, 64, 64) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 6 Tasks &lt;/td&gt;&lt;td&gt; 3 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; uint16 &lt;/td&gt;&lt;td&gt; numpy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;&lt;/table&gt;
&lt;/td&gt;
&lt;td&gt;
&lt;svg width="402" height="208" style="stroke:rgb(0,0,0);stroke-width:1" &gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="0" y1="0" x2="27" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="0" y1="11" x2="27" y2="11" /&gt;
  &lt;line x1="0" y1="22" x2="27" y2="22" /&gt;
  &lt;line x1="0" y1="33" x2="27" y2="33" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="0" y1="0" x2="0" y2="33" style="stroke-width:2" /&gt;
  &lt;line x1="27" y1="0" x2="27" y2="33" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="0.000000,0.000000 27.530335,0.000000 27.530335,33.941765 0.000000,33.941765" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="13.765167" y="53.941765" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;1&lt;/text&gt;
&lt;text x="47.530335" y="16.970882" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(0,47.530335,16.970882)"&gt;3&lt;/text&gt;&lt;/p&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="97" y1="0" x2="173" y2="76" style="stroke-width:2" /&gt;
  &lt;line x1="97" y1="82" x2="173" y2="158" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="97" y1="0" x2="97" y2="82" style="stroke-width:2" /&gt;
  &lt;line x1="173" y1="76" x2="173" y2="158" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="97.000000,0.000000 173.470588,76.470588 173.470588,158.846826 97.000000,82.376238" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="97" y1="0" x2="179" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="173" y1="76" x2="255" y2="76" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="97" y1="0" x2="173" y2="76" style="stroke-width:2" /&gt;
  &lt;line x1="179" y1="0" x2="255" y2="76" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="97.000000,0.000000 179.376238,0.000000 255.846826,76.470588 173.470588,76.470588" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="173" y1="76" x2="255" y2="76" style="stroke-width:2" /&gt;
  &lt;line x1="173" y1="158" x2="255" y2="158" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="173" y1="76" x2="173" y2="158" style="stroke-width:2" /&gt;
  &lt;line x1="255" y1="76" x2="255" y2="158" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="173.470588,76.470588 255.846826,76.470588 255.846826,158.846826 173.470588,158.846826" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="214.658707" y="178.846826" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;64&lt;/text&gt;
&lt;text x="275.846826" y="117.658707" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(0,275.846826,117.658707)"&gt;64&lt;/text&gt;
&lt;text x="125.235294" y="140.611532" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(45,125.235294,140.611532)"&gt;101&lt;/text&gt;
&lt;/svg&gt;&lt;/p&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Convert data to float32 for computation¶&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="n"&gt;imgs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;imgs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;astype&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;float32&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="c1"&gt;# Note: the psf needs to be sampled with a voxel spacing&lt;/span&gt;
&lt;span class="c1"&gt;# consistent with the image&amp;#39;s sampling&lt;/span&gt;
&lt;span class="n"&gt;psf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;astype&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;float32&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="c1"&gt;# Apply Richardson-Lucy Deconvolution¶&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;richardson_lucy_deconvolution&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;iterations&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="w"&gt;    &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot; Apply deconvolution to a single chunk of data &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;itk&lt;/span&gt;

    &lt;span class="n"&gt;img&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;img&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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;  &lt;span class="c1"&gt;# remove leading two length-one dimensions&lt;/span&gt;
    &lt;span class="n"&gt;psf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;psf&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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;  &lt;span class="c1"&gt;# remove leading two length-one dimensions&lt;/span&gt;

    &lt;span class="n"&gt;image&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;itk&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;image_view_from_array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;   &lt;span class="c1"&gt;# Convert to ITK object&lt;/span&gt;
    &lt;span class="n"&gt;kernel&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;itk&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;image_view_from_array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Convert to ITK object&lt;/span&gt;

    &lt;span class="n"&gt;deconvolved&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;itk&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;richardson_lucy_deconvolution_image_filter&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
        &lt;span class="n"&gt;image&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="n"&gt;kernel_image&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;kernel&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="n"&gt;number_of_iterations&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;iterations&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;itk&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array_from_image&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;deconvolved&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Convert back to Numpy array&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;result&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;  &lt;span class="c1"&gt;# Add back the leading length-one dimensions&lt;/span&gt;

    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;result&lt;/span&gt;

&lt;span class="n"&gt;out&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;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;richardson_lucy_deconvolution&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;imgs&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;psf&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="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;float32&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="c1"&gt;# Create a local cluster of dask worker processes&lt;/span&gt;
&lt;span class="c1"&gt;# (this could also point to a distributed cluster if you have it)&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;threads_per_process&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;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# now dask operations use this cluster by default&lt;/span&gt;

&lt;span class="c1"&gt;# Trigger computation and store&lt;/span&gt;
&lt;span class="n"&gt;out&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_zarr&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;AOLLSMData_m4_raw.zarr&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;deconvolved&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;overwrite&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So in the example above we …&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Load data both from Zarr and TIFF files into multi-chunked Dask arrays&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Construct a function to apply an ITK routine onto each chunk&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Apply that function across the dask array with the &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html#dask.array.core.map_blocks"&gt;dask.array.map_blocks&lt;/a&gt; function.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Store the result back into Zarr format&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;From the perspective of an imaging scientist,
the new piece of technology here is the
&lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html#dask.array.core.map_blocks"&gt;dask.array.map_blocks&lt;/a&gt; function.
Given a Dask array composed of many NumPy arrays and a function, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt; applies that function across each block in parallel, returning a Dask array as a result.
It’s a great tool whenever you want to apply an operation across many blocks in a simple fashion.
Because Dask arrays are just made out of Numpy arrays it’s an easy way to
compose Dask with the rest of the Scientific Python ecosystem.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/09/image-itk.md&lt;/span&gt;, line 459)&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="building-the-right-function"&gt;
&lt;h1&gt;Building the right function&lt;/h1&gt;
&lt;p&gt;However in this case there are a few challenges to constructing the right Numpy
-&amp;gt; Numpy function, due to both idiosyncrasies in ITK and Dask Array. Let’s
look at our function again:&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;richardson_lucy_deconvolution&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;iterations&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="w"&gt;    &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot; Apply deconvolution to a single chunk of data &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;itk&lt;/span&gt;

    &lt;span class="n"&gt;img&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;img&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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;  &lt;span class="c1"&gt;# remove leading two length-one dimensions&lt;/span&gt;
    &lt;span class="n"&gt;psf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;psf&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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;  &lt;span class="c1"&gt;# remove leading two length-one dimensions&lt;/span&gt;

    &lt;span class="n"&gt;image&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;itk&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;image_view_from_array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;   &lt;span class="c1"&gt;# Convert to ITK object&lt;/span&gt;
    &lt;span class="n"&gt;kernel&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;itk&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;image_view_from_array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Convert to ITK object&lt;/span&gt;

    &lt;span class="n"&gt;deconvolved&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;itk&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;richardson_lucy_deconvolution_image_filter&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
        &lt;span class="n"&gt;image&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="n"&gt;kernel_image&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;kernel&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="n"&gt;number_of_iterations&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;iterations&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;itk&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array_from_image&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;deconvolved&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Convert back to Numpy array&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;result&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;  &lt;span class="c1"&gt;# Add back the leading length-one dimensions&lt;/span&gt;

    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;result&lt;/span&gt;

&lt;span class="n"&gt;out&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;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;richardson_lucy_deconvolution&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;imgs&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;psf&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="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;float32&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is longer than we would like.
Instead, we would have preferred to just use the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;itk&lt;/span&gt;&lt;/code&gt; function directly,
without all of the steps before and after.&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;deconvolved&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;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;itk&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;richardson_lucy_deconvolution_image_filter&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;imgs&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;What were the extra steps in our function and why were they necessary?&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Convert to and from ITK Image objects&lt;/strong&gt;: ITK functions don’t consume and
produce Numpy arrays, they consume and produce their own &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Image&lt;/span&gt;&lt;/code&gt; data
structure. There are convenient functions to convert back and forth,
so handling this is straightforward, but it does need to be handled each
time. See &lt;a class="reference external" href="https://github.com/InsightSoftwareConsortium/ITK/issues/1136"&gt;ITK #1136&lt;/a&gt; for a
feature request that would remove the need for this step.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Unpack and pack singleton dimensions&lt;/strong&gt;: Our Dask arrays have shapes like
the following:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;Array&lt;/span&gt; &lt;span class="n"&gt;Shape&lt;/span&gt;&lt;span class="p"&gt;:&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;199&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;201&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1024&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;768&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;Chunk&lt;/span&gt; &lt;span class="n"&gt;Shape&lt;/span&gt;&lt;span class="p"&gt;:&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;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;201&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1024&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;768&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt; function gets NumPy arrays of the chunk size,
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(1,&lt;/span&gt; &lt;span class="pre"&gt;1,&lt;/span&gt; &lt;span class="pre"&gt;201,&lt;/span&gt; &lt;span class="pre"&gt;1024,&lt;/span&gt; &lt;span class="pre"&gt;768)&lt;/span&gt;&lt;/code&gt;.
However, our ITK functions are meant to work on 3d arrays, not 5d arrays,
so we need to remove those first two dimensions.&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;img&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;img&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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;  &lt;span class="c1"&gt;# remove leading two length-one dimensions&lt;/span&gt;
&lt;span class="n"&gt;psf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;psf&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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;  &lt;span class="c1"&gt;# remove leading two length-one dimensions&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And then when we’re done, Dask expects to get back 5d arrays like what it
provided, so we add these singleton dimensions back in&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;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;  &lt;span class="c1"&gt;# Add back the leading length-one dimensions&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Again, this is straightforward for users who are accustomed to NumPy
slicing syntax, but does need to be done each time.
This adds some friction to our development process,
and is another step that can confuse users.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;But if you’re comfortable working around things like this,
then ITK and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt; can be a powerful combination
if you want to parallelize out ITK operations across a cluster.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/09/image-itk.md&lt;/span&gt;, line 541)&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="defining-a-dask-cluster"&gt;
&lt;h1&gt;Defining a Dask Cluster&lt;/h1&gt;
&lt;p&gt;Above we used &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.distributed.LocalCluster&lt;/span&gt;&lt;/code&gt; to set up 20 single-threaded
workers on our local workstation:&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;threads_per_process&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;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# now dask operations use this cluster by default&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If you had a distributed resource, this is where you would connect it.
You would swap out &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LocalCluster&lt;/span&gt;&lt;/code&gt; with one of
&lt;a class="reference external" href="https://docs.dask.org/en/latest/setup.html"&gt;Dask’s other deployment options&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Also, we found that we needed to use many single-threaded processes rather than
one multi-threaded process because ITK functions seem to still hold onto the
GIL. This is fine, we just need to be aware of it so that we set up our Dask
workers appropriately with one thread per process for maximum efficiency.
See &lt;a class="reference external" href="https://github.com/InsightSoftwareConsortium/ITK/issues/1134"&gt;ITK #1134&lt;/a&gt;
for an active Github issue on this 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/2019/08/09/image-itk.md&lt;/span&gt;, line 563)&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="serialization"&gt;
&lt;h1&gt;Serialization&lt;/h1&gt;
&lt;p&gt;We had some difficulty when using the ITK library across multiple processes,
because the library itself didn’t serialize well. (If you don’t understand
what that means, don’t worry). We solved a bit of this in
&lt;a class="reference external" href="https://github.com/InsightSoftwareConsortium/ITK/pull/1090"&gt;ITK #1090&lt;/a&gt;,
but some issues still remain.&lt;/p&gt;
&lt;p&gt;We got around this by including the import in the function rather than outside
of it.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;richardson_lucy_deconvolution&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;iterations&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;itk&lt;/span&gt;   &lt;span class="c1"&gt;# &amp;lt;--- we work around serialization issues by importing within the function&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;That way each task imports itk individually, and we sidestep this issue.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/09/image-itk.md&lt;/span&gt;, line 581)&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="trying-scikit-image"&gt;
&lt;h1&gt;Trying Scikit-Image&lt;/h1&gt;
&lt;p&gt;We also tried out the Richardson Lucy deconvolution operation in
&lt;a class="reference external" href="https://scikit-image.org/"&gt;Scikit-Image&lt;/a&gt;. Scikit-Image is known for being
more Scipy/Numpy native, but not always as fast as ITK. Our experience
confirmed this perception.&lt;/p&gt;
&lt;p&gt;First, we were glad to see that the scikit-image function worked with
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt; immediately without any packing/unpacking, dimensionality, or
serialization issues:&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;skimage.restoration&lt;/span&gt;

&lt;span class="n"&gt;out&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;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;restoration&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;richardson_lucy&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;imgs&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# just works&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So all of that converting to and from image objects or removing and adding
singleton dimensions isn’t necessary here.&lt;/p&gt;
&lt;p&gt;In terms of performance we were also happy to see that Scikit-Image released
the GIL, so we were able to get very high reported CPU utilization when using a
small number of multi-threaded processes. However, even though CPU utilization
was high, our parallel performance was poor enough that we stuck with the ITK
solution, warts and all. More information about this is available in
Github issue &lt;a class="reference external" href="https://github.com/scikit-image/scikit-image/issues/4083"&gt;scikit-image #4083&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: sequentially on a single chunk, ITK ran in around 2 minutes while
scikit-image ran in 3 minutes. It was only once we started parallelizing that
things became slow.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Regardless, our goal in this experiment was to see how well ITK and Dask
array played together. It was nice to see what smooth integration looks like,
if only to motivate future development in ITK+Dask relations.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/09/image-itk.md&lt;/span&gt;, line 616)&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="numba-gufuncs"&gt;
&lt;h1&gt;Numba GUFuncs&lt;/h1&gt;
&lt;p&gt;An alternative to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;da.map_blocks&lt;/span&gt;&lt;/code&gt; are Generalized Universal Functions (gufuncs)
These are functions that have many magical properties, one of which is that
they operate equally well on both NumPy and Dask arrays. If libraries like
ITK or Scikit-Image make their functions into gufuncs then they work without
users having to do anything special.&lt;/p&gt;
&lt;p&gt;The easiest way to implement gufuncs today is with Numba. I did this on our
wrapped &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;richardson_lucy&lt;/span&gt;&lt;/code&gt; function, just to show how it could work, in case
other libraries want to take this on in the future.&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;numba&lt;/span&gt;

&lt;span class="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;guvectorize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;float32[:,:,:], float32[:,:,:], float32[:,:,:]&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;  &lt;span class="c1"&gt;# we have to specify types&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;(i,j,k),(a,b,c)-&amp;gt;(i,j,k)&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;                          &lt;span class="c1"&gt;# and dimensionality explicitly&lt;/span&gt;
    &lt;span class="n"&gt;forceobj&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="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;richardson_lucy_deconvolution&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;psf&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="c1"&gt;# &amp;lt;---- no dimension unpacking!&lt;/span&gt;
    &lt;span class="n"&gt;iterations&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;
    &lt;span class="n"&gt;image&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;itk&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;image_view_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;ascontiguousarray&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
    &lt;span class="n"&gt;kernel&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;itk&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;image_view_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;ascontiguousarray&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

    &lt;span class="n"&gt;deconvolved&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;itk&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;richardson_lucy_deconvolution_image_filter&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
        &lt;span class="n"&gt;image&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;kernel_image&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;kernel&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;number_of_iterations&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;iterations&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="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;itk&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array_from_image&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;deconvolved&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# Now this function works natively on either NumPy or Dask arrays&lt;/span&gt;
&lt;span class="n"&gt;out&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;richardson_lucy_deconvolution&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;imgs&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;psf&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# &amp;lt;-- no map_blocks call!&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Note that we’ve both lost the dimension unpacking and the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt; call.
Our function now knows enough information about how it can broadcast that Dask
can do the parallelization without being told what to do explicitly.&lt;/p&gt;
&lt;p&gt;This adds some burden onto library maintainers,
but makes the user experience much more smooth.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/09/image-itk.md&lt;/span&gt;, line 658)&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="gpu-acceleration"&gt;
&lt;h1&gt;GPU Acceleration&lt;/h1&gt;
&lt;p&gt;When doing some user research on image processing and Dask, almost everyone we
interviewed said that they wanted faster deconvolution. This seemed to be a
major pain point. Now we know why. It’s both very common, and &lt;em&gt;very slow&lt;/em&gt;.&lt;/p&gt;
&lt;p&gt;Running deconvolution on a single chunk of this size takes around 2-4 minutes,
and we have hundreds of chunks in a single dataset. Multi-core parallelism can
help a bit here, but this problem may also be ripe for GPU acceleration.
Similar operations typically have 100x speedups on GPUs. This might be a more
pragmatic solution than scaling out to large distributed clusters.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/09/image-itk.md&lt;/span&gt;, line 670)&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-s-next"&gt;
&lt;h1&gt;What’s next?&lt;/h1&gt;
&lt;p&gt;This experiment both …&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Gives us an example&lt;/strong&gt; that other imaging scientists
can copy and modify to be effective with Dask and ITK together.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Highlights areas of improvement&lt;/strong&gt; where developers from the different
libraries can work to remove some of these rough interactions spots in the
future.&lt;/p&gt;
&lt;p&gt;It’s worth noting that Dask has done this with lots of libraries within the
Scipy ecosystem, including Pandas, Scikit-Image, Scikit-Learn, and others.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We’re also going to continue with our imaging experiment, while these technical
issues get worked out in the background. Next up, segmentation!&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/08/09/image-itk/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="imaging" label="imaging"/>
    <published>2019-08-09T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/08/05/user-survey/</id>
    <title>2019 Dask User Survey</title>
    <updated>2019-08-05T00:00:00+00:00</updated>
    <author>
      <name>Tom Augspurger</name>
    </author>
    <content type="html">&lt;style type="text/css"&gt;
table td {
    background: none;
}

table tr.even td {
    background: none;
}

table {
    text-shadow: none;
}

&lt;/style&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/05/user-survey.md&lt;/span&gt;, line 25)&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-user-survey-results"&gt;

&lt;p&gt;This notebook presents the results of the 2019 Dask User Survey,
which ran earlier this summer. Thanks to everyone who took the time to fill out the survey!
These results help us better understand the Dask community and will guide future development efforts.&lt;/p&gt;
&lt;p&gt;The raw data, as well as the start of an analysis, can be found in this binder:&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://mybinder.org/v2/gh/dask/dask-examples/main?urlpath=%2Ftree%2Fsurveys%2F2019.ipynb"&gt;&lt;img alt="Binder" src="https://mybinder.org/badge_logo.svg" /&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Let us know if you find anything in the data.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/05/user-survey.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="highlights"&gt;
&lt;h1&gt;Highlights&lt;/h1&gt;
&lt;p&gt;We had 259 responses to the survey. Overall, we found that the survey respondents really care about improved documentation, and ease of use (including ease of deployment), and scaling. While Dask brings together many different communities (big arrays versus big dataframes, traditional HPC users versus cloud-native resource managers), there was general agreement in what is most important for Dask.&lt;/p&gt;
&lt;p&gt;Now we’ll go through some individual items questions, highlighting particularly interesting results.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/05/user-survey.md&lt;/span&gt;, line 43)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="how-do-you-use-dask"&gt;
&lt;h1&gt;How do you use Dask?&lt;/h1&gt;
&lt;p&gt;For learning resources, almost every respondent uses the documentation.&lt;/p&gt;
&lt;p&gt;&lt;img alt="svg" src="https://blog.dask.org/_images/analyze_4_0.svg" /&gt;&lt;/p&gt;
&lt;p&gt;Most respondents use Dask at least occasionally. Fortunately we had a decent number of respondents who are just looking into Dask, yet still spent the time to take the survey.&lt;/p&gt;
&lt;p&gt;&lt;img alt="svg" src="https://blog.dask.org/_images/analyze_6_0.svg" /&gt;&lt;/p&gt;
&lt;p&gt;I’m curiuos about how learning resource usage changes as users become more experienced. We might expect those just looking into Dask to start with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;examples.dask.org&lt;/span&gt;&lt;/code&gt;, where they can try out Dask without installing anything.&lt;/p&gt;
&lt;p&gt;&lt;img alt="svg" src="https://blog.dask.org/_images/analyze_8_0.svg" /&gt;&lt;/p&gt;
&lt;p&gt;Overall, documentation is still the leader across user user groups.&lt;/p&gt;
&lt;p&gt;The usage of the &lt;a class="reference external" href="https://github.com/dask/dask-tutorial"&gt;Dask tutorial&lt;/a&gt; and the &lt;a class="reference internal" href="#examples.dask.org"&gt;&lt;span class="xref myst"&gt;dask examples&lt;/span&gt;&lt;/a&gt; are relatively consistent across groups. The primary difference between regular and new users is that regular users are more likely to engage on GitHub.&lt;/p&gt;
&lt;p&gt;From StackOverflow questions and GitHub issues, we have a vague idea about which parts of the library are used.
The survey shows that (for our respondents at least) DataFrame and Delayed are the most commonly used APIs.&lt;/p&gt;
&lt;p&gt;&lt;img alt="svg" src="https://blog.dask.org/_images/analyze_10_0.svg" /&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;About 65.49% of our respondests are using Dask on a Cluster.
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;But the majority of respondents &lt;em&gt;also&lt;/em&gt; use Dask on their laptop.
This highlights the importance of Dask scaling down, either for
prototyping with a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LocalCluster&lt;/span&gt;&lt;/code&gt;, or for out-of-core analysis
using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LocalCluster&lt;/span&gt;&lt;/code&gt; or one of the single-machine schedulers.&lt;/p&gt;
&lt;p&gt;&lt;img alt="svg" src="https://blog.dask.org/_images/analyze_13_0.svg" /&gt;&lt;/p&gt;
&lt;p&gt;Most respondents use Dask interactively, at least some of the time.&lt;/p&gt;
&lt;p&gt;&lt;img alt="svg" src="https://blog.dask.org/_images/analyze_15_0.svg" /&gt;&lt;/p&gt;
&lt;p&gt;Most repondents thought that more documentation and examples would be the most valuable improvements to the project. This is especially pronounced among new users. But even among those using Dask everyday more people thought that “More examples” is more valuable than “New features” or “Performance improvements”.&lt;/p&gt;
&lt;style  type="text/css" &gt;
    #T_820ef326_b488_11e9_ad41_186590cd1c87row0_col0 {
            background-color:  #3b92c1;
            color:  #000000;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row0_col1 {
            background-color:  #b4c4df;
            color:  #000000;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row0_col2 {
            background-color:  #dad9ea;
            color:  #000000;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row0_col3 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row0_col4 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row1_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row1_col1 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row1_col2 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row1_col3 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row1_col4 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row2_col0 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row2_col1 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row2_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row2_col3 {
            background-color:  #1b7eb7;
            color:  #000000;
        }    #T_820ef326_b488_11e9_ad41_186590cd1c87row2_col4 {
            background-color:  #589ec8;
            color:  #000000;
        }&lt;/style&gt;&lt;table id="T_820ef326_b488_11e9_ad41_186590cd1c87" &gt;&lt;caption&gt;Normalized by row. Darker means that a higher proporiton of users with that usage frequency prefer that priority.&lt;/caption&gt;&lt;thead&gt;    &lt;tr&gt;        &lt;th class="index_name level0" &gt;Which would help you most right now?&lt;/th&gt;        &lt;th class="col_heading level0 col0" &gt;Bug fixes&lt;/th&gt;        &lt;th class="col_heading level0 col1" &gt;More documentation&lt;/th&gt;        &lt;th class="col_heading level0 col2" &gt;More examples in my field&lt;/th&gt;        &lt;th class="col_heading level0 col3" &gt;New features&lt;/th&gt;        &lt;th class="col_heading level0 col4" &gt;Performance improvements&lt;/th&gt;    &lt;/tr&gt;    &lt;tr&gt;        &lt;th class="index_name level0" &gt;How often do you use Dask?&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;    &lt;/tr&gt;&lt;/thead&gt;&lt;tbody&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;            &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87level0_row0&amp;quot; class=&amp;quot;row_heading level0 row0&amp;quot; &amp;gt;Every day&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row0_col0&amp;quot; class=&amp;quot;data row0 col0&amp;quot; &amp;gt;9&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row0_col1&amp;quot; class=&amp;quot;data row0 col1&amp;quot; &amp;gt;11&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row0_col2&amp;quot; class=&amp;quot;data row0 col2&amp;quot; &amp;gt;25&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row0_col3&amp;quot; class=&amp;quot;data row0 col3&amp;quot; &amp;gt;22&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row0_col4&amp;quot; class=&amp;quot;data row0 col4&amp;quot; &amp;gt;23&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87level0_row1&amp;quot; class=&amp;quot;row_heading level0 row1&amp;quot; &amp;gt;Just looking for now&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row1_col0&amp;quot; class=&amp;quot;data row1 col0&amp;quot; &amp;gt;1&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row1_col1&amp;quot; class=&amp;quot;data row1 col1&amp;quot; &amp;gt;3&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row1_col2&amp;quot; class=&amp;quot;data row1 col2&amp;quot; &amp;gt;18&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row1_col3&amp;quot; class=&amp;quot;data row1 col3&amp;quot; &amp;gt;9&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row1_col4&amp;quot; class=&amp;quot;data row1 col4&amp;quot; &amp;gt;5&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87level0_row2&amp;quot; class=&amp;quot;row_heading level0 row2&amp;quot; &amp;gt;Occasionally&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row2_col0&amp;quot; class=&amp;quot;data row2 col0&amp;quot; &amp;gt;14&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row2_col1&amp;quot; class=&amp;quot;data row2 col1&amp;quot; &amp;gt;27&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row2_col2&amp;quot; class=&amp;quot;data row2 col2&amp;quot; &amp;gt;52&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row2_col3&amp;quot; class=&amp;quot;data row2 col3&amp;quot; &amp;gt;18&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_820ef326_b488_11e9_ad41_186590cd1c87row2_col4&amp;quot; class=&amp;quot;data row2 col4&amp;quot; &amp;gt;15&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
&amp;lt;/tbody&amp;gt;&amp;lt;/table&amp;gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Perhaps users of certain dask APIs feel differenlty from the group as a whole? We perform a similar analysis grouped by API use, rather than frequency of use.&lt;/p&gt;
&lt;style  type="text/css" &gt;
    #T_821479f4_b488_11e9_ad41_186590cd1c87row0_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row0_col1 {
            background-color:  #cacee5;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row0_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row0_col3 {
            background-color:  #f1ebf4;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row0_col4 {
            background-color:  #c4cbe3;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row1_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row1_col1 {
            background-color:  #3b92c1;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row1_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row1_col3 {
            background-color:  #62a2cb;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row1_col4 {
            background-color:  #bdc8e1;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row2_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row2_col1 {
            background-color:  #c2cbe2;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row2_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row2_col3 {
            background-color:  #94b6d7;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row2_col4 {
            background-color:  #e0dded;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row3_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row3_col1 {
            background-color:  #e6e2ef;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row3_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row3_col3 {
            background-color:  #ced0e6;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row3_col4 {
            background-color:  #c5cce3;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row4_col0 {
            background-color:  #dedcec;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row4_col1 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row4_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row4_col3 {
            background-color:  #1c7fb8;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row4_col4 {
            background-color:  #73a9cf;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row5_col0 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row5_col1 {
            background-color:  #b4c4df;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row5_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row5_col3 {
            background-color:  #b4c4df;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row5_col4 {
            background-color:  #eee9f3;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row6_col0 {
            background-color:  #faf2f8;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row6_col1 {
            background-color:  #e7e3f0;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row6_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row6_col3 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_821479f4_b488_11e9_ad41_186590cd1c87row6_col4 {
            background-color:  #f4eef6;
            color:  #000000;
        }&lt;/style&gt;&lt;table id="T_821479f4_b488_11e9_ad41_186590cd1c87" &gt;&lt;caption&gt;Normalized by row. Darker means that a higher proporiton of users of that API prefer that priority.&lt;/caption&gt;&lt;thead&gt;    &lt;tr&gt;        &lt;th class="index_name level0" &gt;Which would help you most right now?&lt;/th&gt;        &lt;th class="col_heading level0 col0" &gt;Bug fixes&lt;/th&gt;        &lt;th class="col_heading level0 col1" &gt;More documentation&lt;/th&gt;        &lt;th class="col_heading level0 col2" &gt;More examples in my field&lt;/th&gt;        &lt;th class="col_heading level0 col3" &gt;New features&lt;/th&gt;        &lt;th class="col_heading level0 col4" &gt;Performance improvements&lt;/th&gt;    &lt;/tr&gt;    &lt;tr&gt;        &lt;th class="index_name level0" &gt;Dask APIs&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;    &lt;/tr&gt;&lt;/thead&gt;&lt;tbody&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;            &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87level0_row0&amp;quot; class=&amp;quot;row_heading level0 row0&amp;quot; &amp;gt;Array&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row0_col0&amp;quot; class=&amp;quot;data row0 col0&amp;quot; &amp;gt;10&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row0_col1&amp;quot; class=&amp;quot;data row0 col1&amp;quot; &amp;gt;24&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row0_col2&amp;quot; class=&amp;quot;data row0 col2&amp;quot; &amp;gt;62&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row0_col3&amp;quot; class=&amp;quot;data row0 col3&amp;quot; &amp;gt;15&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row0_col4&amp;quot; class=&amp;quot;data row0 col4&amp;quot; &amp;gt;25&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87level0_row1&amp;quot; class=&amp;quot;row_heading level0 row1&amp;quot; &amp;gt;Bag&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row1_col0&amp;quot; class=&amp;quot;data row1 col0&amp;quot; &amp;gt;3&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row1_col1&amp;quot; class=&amp;quot;data row1 col1&amp;quot; &amp;gt;11&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row1_col2&amp;quot; class=&amp;quot;data row1 col2&amp;quot; &amp;gt;16&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row1_col3&amp;quot; class=&amp;quot;data row1 col3&amp;quot; &amp;gt;10&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row1_col4&amp;quot; class=&amp;quot;data row1 col4&amp;quot; &amp;gt;7&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87level0_row2&amp;quot; class=&amp;quot;row_heading level0 row2&amp;quot; &amp;gt;DataFrame&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row2_col0&amp;quot; class=&amp;quot;data row2 col0&amp;quot; &amp;gt;16&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row2_col1&amp;quot; class=&amp;quot;data row2 col1&amp;quot; &amp;gt;32&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row2_col2&amp;quot; class=&amp;quot;data row2 col2&amp;quot; &amp;gt;71&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row2_col3&amp;quot; class=&amp;quot;data row2 col3&amp;quot; &amp;gt;39&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row2_col4&amp;quot; class=&amp;quot;data row2 col4&amp;quot; &amp;gt;26&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87level0_row3&amp;quot; class=&amp;quot;row_heading level0 row3&amp;quot; &amp;gt;Delayed&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row3_col0&amp;quot; class=&amp;quot;data row3 col0&amp;quot; &amp;gt;16&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row3_col1&amp;quot; class=&amp;quot;data row3 col1&amp;quot; &amp;gt;22&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row3_col2&amp;quot; class=&amp;quot;data row3 col2&amp;quot; &amp;gt;55&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row3_col3&amp;quot; class=&amp;quot;data row3 col3&amp;quot; &amp;gt;26&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row3_col4&amp;quot; class=&amp;quot;data row3 col4&amp;quot; &amp;gt;27&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87level0_row4&amp;quot; class=&amp;quot;row_heading level0 row4&amp;quot; &amp;gt;Futures&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row4_col0&amp;quot; class=&amp;quot;data row4 col0&amp;quot; &amp;gt;12&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row4_col1&amp;quot; class=&amp;quot;data row4 col1&amp;quot; &amp;gt;9&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row4_col2&amp;quot; class=&amp;quot;data row4 col2&amp;quot; &amp;gt;25&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row4_col3&amp;quot; class=&amp;quot;data row4 col3&amp;quot; &amp;gt;20&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row4_col4&amp;quot; class=&amp;quot;data row4 col4&amp;quot; &amp;gt;17&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87level0_row5&amp;quot; class=&amp;quot;row_heading level0 row5&amp;quot; &amp;gt;ML&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row5_col0&amp;quot; class=&amp;quot;data row5 col0&amp;quot; &amp;gt;5&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row5_col1&amp;quot; class=&amp;quot;data row5 col1&amp;quot; &amp;gt;11&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row5_col2&amp;quot; class=&amp;quot;data row5 col2&amp;quot; &amp;gt;23&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row5_col3&amp;quot; class=&amp;quot;data row5 col3&amp;quot; &amp;gt;11&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row5_col4&amp;quot; class=&amp;quot;data row5 col4&amp;quot; &amp;gt;7&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87level0_row6&amp;quot; class=&amp;quot;row_heading level0 row6&amp;quot; &amp;gt;Xarray&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row6_col0&amp;quot; class=&amp;quot;data row6 col0&amp;quot; &amp;gt;8&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row6_col1&amp;quot; class=&amp;quot;data row6 col1&amp;quot; &amp;gt;11&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row6_col2&amp;quot; class=&amp;quot;data row6 col2&amp;quot; &amp;gt;34&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row6_col3&amp;quot; class=&amp;quot;data row6 col3&amp;quot; &amp;gt;7&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_821479f4_b488_11e9_ad41_186590cd1c87row6_col4&amp;quot; class=&amp;quot;data row6 col4&amp;quot; &amp;gt;9&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
&amp;lt;/tbody&amp;gt;&amp;lt;/table&amp;gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Nothing really stands out. The “futures” users (who we expect to be relatively advanced) may prioritize features and performance over documentation. But everyone agrees that more examples are the highest priority.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/05/user-survey.md&lt;/span&gt;, line 325)&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="common-feature-requests"&gt;
&lt;h1&gt;Common Feature Requests&lt;/h1&gt;
&lt;p&gt;For specific features, we made a list of things that we (as developers) thought might be important.&lt;/p&gt;
&lt;p&gt;&lt;img alt="svg" src="https://blog.dask.org/_images/analyze_22_0.svg" /&gt;&lt;/p&gt;
&lt;p&gt;The clearest standout is how many people thought “Better NumPy/Pandas support” was “most critical”. In hindsight, it’d be good to have a followup fill-in field to undertand what each respondent meant by that. The parsimonious interpretion is “cover more of the NumPy / pandas API”.&lt;/p&gt;
&lt;p&gt;“Ease of deployment” had a high proportion of “critical to me”. Again in hindsight, I notice a bit of ambiguity. Does this mean people want Dask to be easier to deploy? Or does this mean that Dask, which they currently find easy to deploy, is critically important? Regardless, we can prioritize simplicity in deployment.&lt;/p&gt;
&lt;p&gt;Relatively few respondents care about things like “Managing many users”, though we expect that this would be relatively popular among system administartors, who are a smaller population.&lt;/p&gt;
&lt;p&gt;And of course, we have people pushing Dask to its limits for whom “Improving scaling” is critically important.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/05/user-survey.md&lt;/span&gt;, line 339)&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-other-systems-do-you-use"&gt;
&lt;h1&gt;What other systems do you use?&lt;/h1&gt;
&lt;p&gt;A relatively high proportion of respondents use Python 3 (97% compared to 84% in the most recent &lt;a class="reference external" href="https://www.jetbrains.com/research/python-developers-survey-2018/"&gt;Python Developers Survey&lt;/a&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;3    97.29%
2     2.71%
Name: Python 2 or 3?, dtype: object
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We were a bit surprised to see that SSH is the most popular “cluster resource manager”.&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                                                       98
Kubernetes                                                73
HPC resource manager (SLURM, PBS, SGE, LSF or similar)    61
My workplace has a custom solution for this               23
I don&amp;#39;t know, someone else does this for me               16
Hadoop / Yarn / EMR                                       14
Name: If you use a cluster, how do you launch Dask? , dtype: int64
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;How does cluster-resource manager compare with API usage?&lt;/p&gt;
&lt;style  type="text/css" &gt;
    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col0 {
            background-color:  #056faf;
            color:  #f1f1f1;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col1 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col3 {
            background-color:  #034e7b;
            color:  #f1f1f1;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col4 {
            background-color:  #2685bb;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col5 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col6 {
            background-color:  #f2ecf5;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col0 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col1 {
            background-color:  #f7f0f7;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col2 {
            background-color:  #0771b1;
            color:  #f1f1f1;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col3 {
            background-color:  #0771b1;
            color:  #f1f1f1;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col4 {
            background-color:  #c5cce3;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col5 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col6 {
            background-color:  #79abd0;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col0 {
            background-color:  #8bb2d4;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col1 {
            background-color:  #b4c4df;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col3 {
            background-color:  #589ec8;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col4 {
            background-color:  #eee9f3;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col5 {
            background-color:  #8bb2d4;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col6 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col0 {
            background-color:  #4c99c5;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col1 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col3 {
            background-color:  #056dac;
            color:  #f1f1f1;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col4 {
            background-color:  #73a9cf;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col5 {
            background-color:  #d9d8ea;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col6 {
            background-color:  #f3edf5;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col0 {
            background-color:  #056ba9;
            color:  #f1f1f1;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col1 {
            background-color:  #fff7fb;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col2 {
            background-color:  #023858;
            color:  #f1f1f1;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col3 {
            background-color:  #1379b5;
            color:  #f1f1f1;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col4 {
            background-color:  #dfddec;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col5 {
            background-color:  #e8e4f0;
            color:  #000000;
        }    #T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col6 {
            background-color:  #f9f2f8;
            color:  #000000;
        }&lt;/style&gt;&lt;table id="T_8326d0f8_b488_11e9_ad41_186590cd1c87" &gt;&lt;thead&gt;    &lt;tr&gt;        &lt;th class="index_name level0" &gt;Dask APIs&lt;/th&gt;        &lt;th class="col_heading level0 col0" &gt;Array&lt;/th&gt;        &lt;th class="col_heading level0 col1" &gt;Bag&lt;/th&gt;        &lt;th class="col_heading level0 col2" &gt;DataFrame&lt;/th&gt;        &lt;th class="col_heading level0 col3" &gt;Delayed&lt;/th&gt;        &lt;th class="col_heading level0 col4" &gt;Futures&lt;/th&gt;        &lt;th class="col_heading level0 col5" &gt;ML&lt;/th&gt;        &lt;th class="col_heading level0 col6" &gt;Xarray&lt;/th&gt;    &lt;/tr&gt;    &lt;tr&gt;        &lt;th class="index_name level0" &gt;If you use a cluster, how do you launch Dask? &lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;        &lt;th class="blank" &gt;&lt;/th&gt;    &lt;/tr&gt;&lt;/thead&gt;&lt;tbody&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;            &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87level0_row0&amp;quot; class=&amp;quot;row_heading level0 row0&amp;quot; &amp;gt;Custom&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col0&amp;quot; class=&amp;quot;data row0 col0&amp;quot; &amp;gt;15&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col1&amp;quot; class=&amp;quot;data row0 col1&amp;quot; &amp;gt;6&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col2&amp;quot; class=&amp;quot;data row0 col2&amp;quot; &amp;gt;18&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col3&amp;quot; class=&amp;quot;data row0 col3&amp;quot; &amp;gt;17&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col4&amp;quot; class=&amp;quot;data row0 col4&amp;quot; &amp;gt;14&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col5&amp;quot; class=&amp;quot;data row0 col5&amp;quot; &amp;gt;6&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row0_col6&amp;quot; class=&amp;quot;data row0 col6&amp;quot; &amp;gt;7&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87level0_row1&amp;quot; class=&amp;quot;row_heading level0 row1&amp;quot; &amp;gt;HPC&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col0&amp;quot; class=&amp;quot;data row1 col0&amp;quot; &amp;gt;50&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col1&amp;quot; class=&amp;quot;data row1 col1&amp;quot; &amp;gt;13&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col2&amp;quot; class=&amp;quot;data row1 col2&amp;quot; &amp;gt;40&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col3&amp;quot; class=&amp;quot;data row1 col3&amp;quot; &amp;gt;40&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col4&amp;quot; class=&amp;quot;data row1 col4&amp;quot; &amp;gt;22&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col5&amp;quot; class=&amp;quot;data row1 col5&amp;quot; &amp;gt;11&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row1_col6&amp;quot; class=&amp;quot;data row1 col6&amp;quot; &amp;gt;30&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87level0_row2&amp;quot; class=&amp;quot;row_heading level0 row2&amp;quot; &amp;gt;Hadoop / Yarn / EMR&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col0&amp;quot; class=&amp;quot;data row2 col0&amp;quot; &amp;gt;7&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col1&amp;quot; class=&amp;quot;data row2 col1&amp;quot; &amp;gt;6&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col2&amp;quot; class=&amp;quot;data row2 col2&amp;quot; &amp;gt;12&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col3&amp;quot; class=&amp;quot;data row2 col3&amp;quot; &amp;gt;8&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col4&amp;quot; class=&amp;quot;data row2 col4&amp;quot; &amp;gt;4&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col5&amp;quot; class=&amp;quot;data row2 col5&amp;quot; &amp;gt;7&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row2_col6&amp;quot; class=&amp;quot;data row2 col6&amp;quot; &amp;gt;3&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87level0_row3&amp;quot; class=&amp;quot;row_heading level0 row3&amp;quot; &amp;gt;Kubernetes&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col0&amp;quot; class=&amp;quot;data row3 col0&amp;quot; &amp;gt;40&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col1&amp;quot; class=&amp;quot;data row3 col1&amp;quot; &amp;gt;18&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col2&amp;quot; class=&amp;quot;data row3 col2&amp;quot; &amp;gt;56&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col3&amp;quot; class=&amp;quot;data row3 col3&amp;quot; &amp;gt;47&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col4&amp;quot; class=&amp;quot;data row3 col4&amp;quot; &amp;gt;37&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col5&amp;quot; class=&amp;quot;data row3 col5&amp;quot; &amp;gt;26&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row3_col6&amp;quot; class=&amp;quot;data row3 col6&amp;quot; &amp;gt;21&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
        &amp;lt;tr&amp;gt;
                    &amp;lt;th id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87level0_row4&amp;quot; class=&amp;quot;row_heading level0 row4&amp;quot; &amp;gt;SSH&amp;lt;/th&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col0&amp;quot; class=&amp;quot;data row4 col0&amp;quot; &amp;gt;61&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col1&amp;quot; class=&amp;quot;data row4 col1&amp;quot; &amp;gt;23&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col2&amp;quot; class=&amp;quot;data row4 col2&amp;quot; &amp;gt;72&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col3&amp;quot; class=&amp;quot;data row4 col3&amp;quot; &amp;gt;58&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col4&amp;quot; class=&amp;quot;data row4 col4&amp;quot; &amp;gt;32&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col5&amp;quot; class=&amp;quot;data row4 col5&amp;quot; &amp;gt;30&amp;lt;/td&amp;gt;
                    &amp;lt;td id=&amp;quot;T_8326d0f8_b488_11e9_ad41_186590cd1c87row4_col6&amp;quot; class=&amp;quot;data row4 col6&amp;quot; &amp;gt;25&amp;lt;/td&amp;gt;
        &amp;lt;/tr&amp;gt;
&amp;lt;/tbody&amp;gt;&amp;lt;/table&amp;gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;HPC users are relatively heavy users of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt; and xarray.&lt;/p&gt;
&lt;p&gt;Somewhat surprisingly, Dask’s heaviest users find dask stable enough. Perhaps they’ve pushed past the bugs and found workarounds (percentages are normalized by row).&lt;/p&gt;
&lt;p&gt;&lt;img alt="svg" src="https://blog.dask.org/_images/analyze_32_0.svg" /&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/05/user-survey.md&lt;/span&gt;, line 525)&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="takeaways"&gt;
&lt;h1&gt;Takeaways&lt;/h1&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;We should prioritize improving and expanding our documentation and examples. This may be
accomplished by Dask maintainers seeking examples from the community. Many of the examples
on &lt;a class="reference external" href="https://examples.dask.org"&gt;https://examples.dask.org&lt;/a&gt; were developed by domain specialist who use Dask.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Improved scaling to larger problems is important, but we shouldn’t
sacrifice the single-machine usecase to get there.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Both interactive and batch workflows are important.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask’s various sub-communities are more similar than they are different.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Thanks again to all the respondents. We look forward to repeating this process to identify trends over time.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/08/05/user-survey/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="UserSurvey" label="User Survey"/>
    <published>2019-08-05T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/08/02/dask-2.2/</id>
    <title>Dask Release 2.2.0</title>
    <updated>2019-08-02T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;I’m pleased to announce the release of Dask version 2.2.
This is a significant release with bug fixes and new features.
The last blogged release was 2.0 on 2019-06-22.
This blogpost outlines notable changes since the last post.&lt;/p&gt;
&lt;p&gt;You can conda install Dask:&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 pip install from PyPI:&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[complete] --upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Full changelogs are available here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/blob/master/docs/source/changelog.rst"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/docs/source/changelog.rst"&gt;dask/distributed&lt;/a&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/2019/08/02/dask-2.2.md&lt;/span&gt;, line 26)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="notable-changes"&gt;

&lt;p&gt;As always there are too many changes to list here,
instead we’ll highlight a few that readers may find interesting,
or that break old behavior.
In particular we discuss the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Parquet rewrite&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Nicer HTML output for Clients and Logs&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Hyper-parameter selection with Hyperband in Dask-ML&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Move bytes I/O handling out of Dask to FSSpec&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;async/await everywhere, and cleaner setup for developers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A new SSH deployment solution&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/2019/08/02/dask-2.2.md&lt;/span&gt;, line 40)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="parquet-rewrite"&gt;
&lt;h1&gt;1 - Parquet Rewrite&lt;/h1&gt;
&lt;p&gt;Today Dask DataFrame can read and write Parquet data using either
&lt;a class="reference external" href="https://fastparquet.readthedocs.io"&gt;fastparquet&lt;/a&gt; or
&lt;a class="reference external" href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt;.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;/path/to/mydata.parquet&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;engine&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;arrow&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="c1"&gt;# or&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;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;/path/to/mydata.parquet&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;engine&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;fastparquet&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Supporting both libraries within Dask has been helpful for
users, but introduced some maintenance burden, especially given that each
library co-evolved with Dask dataframe over the years. The contract between
Dask Dataframe and these libraries was convoluted, making it difficult to
evolve swiftly.&lt;/p&gt;
&lt;p&gt;To address this we’ve formalized what Dask expects of Parquet reader/writers
into a more formal Parquet Engine contract. This keeps maintenance costs
low, enables independent development for each project, and allows for new
engines to emerge.&lt;/p&gt;
&lt;p&gt;Already a GPU-accelerated Parquet reader is available in a PR on the &lt;a class="reference external" href="https://github.com/rapidsai/cudf/pull/2368"&gt;RAPIDS
cuDF&lt;/a&gt; library.&lt;/p&gt;
&lt;p&gt;As a result, we’ve also been able to fix a number of long-standing bugs, and
improve the functionality with both engines.&lt;/p&gt;
&lt;p&gt;Some fun quotes from &lt;a class="reference external" href="https://github.com/birdsarah"&gt;Sarah Bird&lt;/a&gt; during development&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;I am currently testing this. So far so good. I can load my dataset in a few seconds with 1800 partitions. Game changing!&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;and&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;I am now successfully working on a dataset with 74,000 partitions and no metadata.
Opening dataset and df.head() takes 7 - 30s. (Presumably depending on whether s3fs cache is cold or not). THIS IS HUGE! This was literally impossible before.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;The API remains the same, but functionality should be smoother.&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="https://github.com/rjzamora"&gt;Rick Zamora&lt;/a&gt;, &lt;a class="reference external" href="https://github.com/martindurant"&gt;Martin
Durant&lt;/a&gt; for doing most of the work here and to
&lt;a class="reference external" href="https://github.com/birdsarah"&gt;Sarah Bird&lt;/a&gt;, &lt;a class="reference external" href="https://github.com/wesm"&gt;Wes
McKinney&lt;/a&gt;, and &lt;a class="reference external" href="https://github.com/mmccarty"&gt;Mike
McCarty&lt;/a&gt; for providing guidance and review.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/02/dask-2.2.md&lt;/span&gt;, line 88)&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="nicer-html-output-for-clients-and-logs"&gt;
&lt;h1&gt;2 - Nicer HTML output for Clients and Logs&lt;/h1&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table style="border: 2px solid white;"&gt;
&lt;tr&gt;
&lt;td style="vertical-align: top; border: 0px solid white"&gt;
&lt;h3 style="text-align: left;"&gt;Client&lt;/h3&gt;
&lt;ul style="text-align: left; list-style: none; margin: 0; padding: 0;"&gt;
  &lt;li&gt;&lt;b&gt;Scheduler: &lt;/b&gt;tcp://127.0.0.1:60275&lt;/li&gt;
  &lt;li&gt;&lt;b&gt;Dashboard: &lt;/b&gt;&lt;a href='http://127.0.0.1:8787/status' target='_blank'&gt;http://127.0.0.1:8787/status&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/td&gt;
&lt;td style="vertical-align: top; border: 0px solid white"&gt;
&lt;h3 style="text-align: left;"&gt;Cluster&lt;/h3&gt;
&lt;ul style="text-align: left; list-style:none; margin: 0; padding: 0;"&gt;
  &lt;li&gt;&lt;b&gt;Workers: &lt;/b&gt;4&lt;/li&gt;
  &lt;li&gt;&lt;b&gt;Cores: &lt;/b&gt;12&lt;/li&gt;
  &lt;li&gt;&lt;b&gt;Memory: &lt;/b&gt;17.18 GB&lt;/li&gt;
&lt;/ul&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;logs&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div markdown="0"&gt;
&lt;details&gt;
&lt;summary&gt;Scheduler&lt;/summary&gt;
&lt;pre&gt;&lt;code&gt;distributed.scheduler - INFO - Clear task state
distributed.scheduler - INFO -   Scheduler at:     tcp://127.0.0.1:60275
distributed.scheduler - INFO -   dashboard at:            127.0.0.1:8787
distributed.scheduler - INFO - Register tcp://127.0.0.1:60281
distributed.scheduler - INFO - Register tcp://127.0.0.1:60282
distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:60281
distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:60282
distributed.scheduler - INFO - Register tcp://127.0.0.1:60285
distributed.scheduler - INFO - Register tcp://127.0.0.1:60286
distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:60285
distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:60286
distributed.scheduler - INFO - Receive client connection: Client-6b6ba1d0-b3bd-11e9-9bd0-acde48001122&lt;/code&gt;&lt;/pre&gt;
&lt;/details&gt;
&lt;details&gt;
&lt;summary&gt;tcp://127.0.0.1:60281&lt;/summary&gt;
&lt;pre&gt;&lt;code&gt;distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:60281
distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:60281
distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:60275
distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO -               Threads:                          3
distributed.worker - INFO -                Memory:                    4.29 GB
distributed.worker - INFO -       Local Directory: /Users/mrocklin/workspace/dask/dask-worker-space/worker-c4_44fym
distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:60275
distributed.worker - INFO - -------------------------------------------------&lt;/code&gt;&lt;/pre&gt;
&lt;/details&gt;
&lt;details&gt;
&lt;summary&gt;tcp://127.0.0.1:60282&lt;/summary&gt;
&lt;pre&gt;&lt;code&gt;distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:60282
distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:60282
distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:60275
distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO -               Threads:                          3
distributed.worker - INFO -                Memory:                    4.29 GB
distributed.worker - INFO -       Local Directory: /Users/mrocklin/workspace/dask/dask-worker-space/worker-quu4taje
distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:60275
distributed.worker - INFO - -------------------------------------------------&lt;/code&gt;&lt;/pre&gt;
&lt;/details&gt;
&lt;details&gt;
&lt;summary&gt;tcp://127.0.0.1:60285&lt;/summary&gt;
&lt;pre&gt;&lt;code&gt;distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:60285
distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:60285
distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:60275
distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO -               Threads:                          3
distributed.worker - INFO -                Memory:                    4.29 GB
distributed.worker - INFO -       Local Directory: /Users/mrocklin/workspace/dask/dask-worker-space/worker-ll4cozug
distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:60275
distributed.worker - INFO - -------------------------------------------------&lt;/code&gt;&lt;/pre&gt;
&lt;/details&gt;
&lt;details&gt;&lt;summary&gt;tcp://127.0.0.1:60286&lt;/summary&gt;&lt;pre&gt;&lt;code&gt;distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:60286
distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:60286
distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:60275
distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO -               Threads:                          3
distributed.worker - INFO -                Memory:                    4.29 GB
distributed.worker - INFO -       Local Directory: /Users/mrocklin/workspace/dask/dask-worker-space/worker-lpbkkzj6
distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:60275
distributed.worker - INFO - -------------------------------------------------&lt;/code&gt;&lt;/pre&gt;&lt;/details&gt;
&lt;/div&gt;
&lt;p&gt;&lt;em&gt;Note: this looks better under any browser other than IE and Edge&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="https://github.com/jacobtomlinson"&gt;Jacob Tomlinson&lt;/a&gt; for this 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/2019/08/02/dask-2.2.md&lt;/span&gt;, line 191)&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="hyperparameter-selection-with-hyperband"&gt;
&lt;h1&gt;3 - Hyperparameter selection with HyperBand&lt;/h1&gt;
&lt;p&gt;Dask-ML 1.0 has been released with a new &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;HyperBandSearchCV&lt;/span&gt;&lt;/code&gt; meta-estimator for
hyper-parameter optimization. This can be used as an alternative to
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomizedSearchCV&lt;/span&gt;&lt;/code&gt; to find similar hyper-parameters in less time by not
wasting time on hyper-parameters that are not promising.&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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_ml.model_selection&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;HyperbandSearchCV&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_ml.datasets&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;make_classification&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;sklearn.linear_model&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;SGDClassifier&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="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;make_classification&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="mi"&gt;20&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;est&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;SGDClassifier&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;tol&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;1e-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;param_dist&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;alpha&amp;#39;&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;logspace&lt;/span&gt;&lt;span class="p"&gt;(&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="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;num&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="s1"&gt;&amp;#39;loss&amp;#39;&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;hinge&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;log&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;modified_huber&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;squared_hinge&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="s1"&gt;&amp;#39;average&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;False&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;search&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;HyperbandSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;est&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;param_dist&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;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;classes&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;unique&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_params_&lt;/span&gt;
&lt;span class="go"&gt;{&amp;#39;loss&amp;#39;: &amp;#39;log&amp;#39;, &amp;#39;average&amp;#39;: False, &amp;#39;alpha&amp;#39;: 0.0080502}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="http://github.com/stsievert"&gt;Scott Sievert&lt;/a&gt;.
You can see Scott talk about this topic in greater depth by watching his
&lt;a class="reference external" href="https://youtu.be/x67K9FiPFBQ"&gt;SciPy 2019 talk&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/2019/08/02/dask-2.2.md&lt;/span&gt;, line 220)&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="move-bytes-i-o-handling-out-of-dask-to-fsspec"&gt;
&lt;h1&gt;4 - Move bytes I/O handling out of Dask to FSSpec&lt;/h1&gt;
&lt;p&gt;We’ve spun out Dask’s internal code to read and write raw data to different
storage systems out to a separate project, &lt;a class="reference external" href="https://fsspec.readthedocs.io"&gt;fsspec&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Here is a small example:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;fsspec&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;fsspec&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;https://github.com/dask/dask/edit/master/README.rst&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="nb"&gt;print&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;read&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="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;fsspec&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;s3://bucket/myfile.csv&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&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="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;fsspec&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;hdfs:///path/to/myfile.csv&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&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="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;fsspec&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;gcs://bucket/myfile.csv&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Dask’s I/O infrastructure to read and write bytes from systems like
HDFS, S3, GCS, Azure, and other remote storage systems is arguably the most
uniform and comprehensive in Python today. Through tooling like
&lt;a class="reference external" href="https://s3fs.readthedocs.io"&gt;s3fs&lt;/a&gt;, &lt;a class="reference external" href="https://gcsfs.readthedocs.io"&gt;gcsfs&lt;/a&gt;,
and ~~hdfs3~~ &lt;a class="reference external" href="https://arrow.apache.org/docs/python/filesystems.html"&gt;pyarrow.hdfs&lt;/a&gt;,
it’s easy to read and write data in a Pythonic way to a
variety of remote storage systems.&lt;/p&gt;
&lt;p&gt;Early on we decided that we wanted this code to live outside of the mainline
Dask codebase, which is why they are independent projects.
This choice allowed other libraries, like Pandas, Zarr, and others to benefit
from this work, without a strict dependency on Dask.
However, there was still code within Dask that helped to unify them a bit.
We’ve moved this code out to an external project,
&lt;a class="reference external" href="https://filesystem-spec.readthedocs.io/en/latest"&gt;fsspec&lt;/a&gt; which includes all
of the centralization code that Dask used to provide, as well as a formal
specification for what a remote data system should look like in order to be
compatible. This also helps to unify efforts with other projects like Arrow.&lt;/p&gt;
&lt;p&gt;Special thanks to &lt;a class="reference external" href="https://github.com/martindurant"&gt;Martin Durant&lt;/a&gt; for
shepherding Dask’s I/O infrastructure over the years, and for doing the more
immediate work of splitting out &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;fsspec&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;You can read more about FSSpec and its transition out of Dask
&lt;a class="reference external" href="https://blog.dask.org/2019/07/23/extracting-fsspec-from-dask"&gt;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/2019/08/02/dask-2.2.md&lt;/span&gt;, line 269)&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="async-await-everywhere-and-cleaner-setup-for-developers"&gt;
&lt;h1&gt;5 - Async/Await everywhere, and cleaner setup for developers&lt;/h1&gt;
&lt;p&gt;In Dask 2.0 we dropped Python 2 support and now support only Python 3.5 and
above.
This allows us to adopt async and await syntax for concurrent execution rather
than an older coroutine based approach with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;yield&lt;/span&gt;&lt;/code&gt;. The differences here
started out as largely aesthetic, but triggered a number of substantive
improvements as we walked through the codebase cleaning things up. Starting
and stopping internal Scheduler, Worker, Nanny, and Client objects is now far
more uniform, reducing the presence of subtle bugs.&lt;/p&gt;
&lt;p&gt;This is discussed in more detail in the &lt;a class="reference external" href="https://docs.dask.org/en/latest/setup/python-advanced.html"&gt;Python API setup
documentation&lt;/a&gt; and
is encapsulated in this code example from those docs:&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;asyncio&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="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Worker&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;

&lt;span class="k"&gt;async&lt;/span&gt; &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;f&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="k"&gt;async&lt;/span&gt; &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;Scheduler&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="k"&gt;async&lt;/span&gt; &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;Worker&lt;/span&gt;&lt;span class="p"&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;address&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;w1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Worker&lt;/span&gt;&lt;span class="p"&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;address&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;w2&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
            &lt;span class="k"&gt;async&lt;/span&gt; &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;address&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;asynchronous&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                &lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&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;1&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;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt;
                &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;asyncio&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_event_loop&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;run_until_complete&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;As a result of this and other internal cleanup intermittent testing failures in
our CI have disappeared, and developer mood is high :)&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/08/02/dask-2.2.md&lt;/span&gt;, line 303)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="a-new-sshcluster"&gt;
&lt;h1&gt;6 - A new SSHCluster&lt;/h1&gt;
&lt;p&gt;We’ve added a second SSH cluster deployment solution. It looks like this:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;distributed.deploy.ssh2&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;SSHCluster&lt;/span&gt;  &lt;span class="c1"&gt;# this will move in future releases&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;SSHCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;hosts&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;host1&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;host2&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;host3&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;host4&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
    &lt;span class="c1"&gt;# hosts=[&amp;quot;localhost&amp;quot;] * 4  # if you want to try this out locally,&lt;/span&gt;
    &lt;span class="n"&gt;worker_kwargs&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;nthreads&amp;quot;&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;scheduler_kwargs&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{},&lt;/span&gt;
    &lt;span class="n"&gt;connect_kwargs&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;known_hosts&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;em&gt;Note that this object is experimental, and subject to change without notice&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;We worked on this for two reasons:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;Our user survey showed that a surprising number of people were deploying
Dask with SSH. Anecdotally they seem to be just SSHing into machines and
then using Dask’s normal &lt;a class="reference external" href="https://docs.dask.org/en/latest/setup/cli.html"&gt;Dask Command Line
Interface&lt;/a&gt;)&lt;/p&gt;
&lt;p&gt;We wanted a solution that was easier than this.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We’ve been trying to unify the code in the various deployment solutions
(like Kubernetes, SLURM, Yarn/Hadoop) to a central codebase, and having a
simple SSHCluster as a test case has proven valuable for testing and
experimentation.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;em&gt;Also note, Dask already has a
&lt;a class="reference external" href="https://docs.dask.org/en/latest/setup/ssh.html"&gt;dask-ssh&lt;/a&gt; solution today that is more mature&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;We expect that unification of deployment will be a central theme for the next
few months of 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/2019/08/02/dask-2.2.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="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;There have been two releases since the last time we had a release blogpost.
The following people contributed to the following repositories since the 2.0
release on June 30th:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Brett Naul&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Daniel Saxton&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;David Brochart&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Davis Bennett&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Elliott Sales de Andrade&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;GALI PREM SAGAR&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Bourbeau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Loïc Estève&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthias Bussonnier&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Natalya Rapstine&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Nick Becker&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Peter Andreas Entschev&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ralf Gommers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Richard (Rick) Zamora&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Sarah Bird&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Sean McKenna&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Willi Rath&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Xavier Holt&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;andrethrill&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;asmith26&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;msbrown47&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;tshatrov&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed"&gt;dask/distributed&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Christian Hudon&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Gabriel Sailer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jacob Tomlinson&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Bourbeau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Pierre Glaser&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Russ Bubley&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;tjb900&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-jobqueue"&gt;dask/dask-jobqueue&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Guillaume Eynard-Bontemps&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Leo Singer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Loïc Estève&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stuart Berg&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-examples"&gt;dask/dask-examples&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Chris White&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ian Rose&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-mpi"&gt;dask/dask-mpi&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Anderson Banihirwe&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kevin Paul&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-kubernetes"&gt;dask/dask-kubernetes&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml"&gt;dask/dask-ml&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Roman Yurchak&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-yarn"&gt;dask/dask-yarn&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Al Johri&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-examples"&gt;dask/dask-examples&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/08/02/dask-2.2/"/>
    <summary>I’m pleased to announce the release of Dask version 2.2.
This is a significant release with bug fixes and new features.
The last blogged release was 2.0 on 2019-06-22.
This blogpost outlines notable changes since the last post.</summary>
    <category term="release" label="release"/>
    <published>2019-08-02T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/07/23/extracting-fsspec-from-dask/</id>
    <title>Extracting fsspec from Dask</title>
    <updated>2019-07-23T00:00:00+00:00</updated>
    <author>
      <name>Martin Durant</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/07/23/extracting-fsspec-from-dask.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="tl-dr"&gt;

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


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

&lt;p&gt;As always, you can conda install Dask:&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 pip install from PyPI:&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 &amp;quot;dask[complete]&amp;quot; --upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Full changelogs are available here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/blob/master/docs/source/changelog.rst"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/docs/source/changelog.rst"&gt;dask/distributed&lt;/a&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/2019/06/22/dask-2.0.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="drop-support-for-python-2"&gt;
&lt;h1&gt;Drop support for Python 2&lt;/h1&gt;
&lt;p&gt;Python 2 reaches end of life in 2020, just six months away. Most major PyData
projects are dropping Python 2 support around now. See the &lt;a class="reference external" href="https://python3statement.org/"&gt;Python 3
Statement&lt;/a&gt; for more details about some of your
favorite projects.&lt;/p&gt;
&lt;p&gt;Python 2 users can continue to use older versions of Dask, which are in
widespread use today. Institutions looking for long term support of Dask in
Python 2 may wish to reach out to for-profit consulting companies, like
&lt;a class="reference external" href="https://www.quansight.com/"&gt;Quansight&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Dropping Python 2 will allow maintainers to spend more of their time fixing
bugs and developing new features. It will also allow the project to adopt more
modern development practices going forward.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/22/dask-2.0.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="small-breaking-changes"&gt;
&lt;h1&gt;Small breaking changes&lt;/h1&gt;
&lt;p&gt;We now include a list with a brief description of most of the breaking changes:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The distributed.bokeh module has moved to distributed.dashboard&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Various &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ncores&lt;/span&gt;&lt;/code&gt; keywords have been moved to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;nthreads&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Client.map/gather/scatter no longer accept iterators and Python queue
objects. Users can handle this themselves with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;submit&lt;/span&gt;&lt;/code&gt;/&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;as_completed&lt;/span&gt;&lt;/code&gt; or
can use the &lt;a class="reference external" href="https://github.com/python-streamz/streamz"&gt;Streamz&lt;/a&gt; library.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The worker &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;/main&lt;/span&gt;&lt;/code&gt; route has moved to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;/status&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Cluster.workers is now a dictionary mapping worker name to worker, rather
than a list as it was before&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/22/dask-2.0.md&lt;/span&gt;, line 70)&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-larger-fun-changes"&gt;
&lt;h1&gt;Some larger fun changes&lt;/h1&gt;
&lt;p&gt;We didn’t only break things. We also added some new things :)&lt;/p&gt;
&lt;section id="array-metadata"&gt;
&lt;h2&gt;Array metadata&lt;/h2&gt;
&lt;p&gt;Previously Dask Arrays were defined by their shape, chunkshape, and datatype,
like float, int, and so on.&lt;/p&gt;
&lt;p&gt;Now, Dask Arrays also know the type of their chunks. Historically this was
almost always a NumPy array, so it didn’t make sense to store, but now that
Dask Arrays are being used more frequently with sparse array chunks and GPU
array chunks we now maintain this information as well in a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;._meta&lt;/span&gt;&lt;/code&gt; attribute.
This is already how Dask dataframes work, so it should be familiar to advanced
users of that module.&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;eye&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;_meta&lt;/span&gt;
&lt;span class="go"&gt;array([], shape=(0, 0), dtype=float64)&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;sparse&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sparse&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;COO&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_numpy&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;s&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_meta&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;COO: shape=(0, 0), dtype=float64, nnz=0, fill_value=0.0&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This work was largely done by &lt;a class="reference external" href="https://github.com/pentschev"&gt;Peter Entschev&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="array-html-output"&gt;
&lt;h2&gt;Array HTML output&lt;/h2&gt;
&lt;p&gt;Dask arrays now print themselves nicely in Jupyter notebooks, showing a table
of information about their size and chunk size, and also a visual diagram of
their chunk structure.&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;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="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;10000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1000&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;table&gt;  &lt;thead&gt;    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 80.00 GB &lt;/td&gt; &lt;td&gt; 125.00 MB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (10000, 1000, 1000) &lt;/td&gt; &lt;td&gt; (250, 250, 250) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 640 Tasks &lt;/td&gt;&lt;td&gt; 640 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; float64 &lt;/td&gt;&lt;td&gt; numpy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;&lt;/table&gt;
&lt;/td&gt;
&lt;td&gt;
&lt;svg width="241" height="231" style="stroke:rgb(0,0,0);stroke-width:1" &gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="127" y2="117" style="stroke-width:2" /&gt;
  &lt;line x1="10" y1="16" x2="127" y2="133" /&gt;
  &lt;line x1="10" y1="32" x2="127" y2="149" /&gt;
  &lt;line x1="10" y1="48" x2="127" y2="165" /&gt;
  &lt;line x1="10" y1="64" x2="127" y2="181" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="10" y2="64" style="stroke-width:2" /&gt;
  &lt;line x1="12" y1="2" x2="12" y2="67" /&gt;
  &lt;line x1="15" y1="5" x2="15" y2="70" /&gt;
  &lt;line x1="18" y1="8" x2="18" y2="73" /&gt;
  &lt;line x1="21" y1="11" x2="21" y2="76" /&gt;
  &lt;line x1="24" y1="14" x2="24" y2="79" /&gt;
  &lt;line x1="27" y1="17" x2="27" y2="81" /&gt;
  &lt;line x1="30" y1="20" x2="30" y2="84" /&gt;
  &lt;line x1="33" y1="23" x2="33" y2="87" /&gt;
  &lt;line x1="36" y1="26" x2="36" y2="90" /&gt;
  &lt;line x1="39" y1="29" x2="39" y2="93" /&gt;
  &lt;line x1="42" y1="32" x2="42" y2="96" /&gt;
  &lt;line x1="45" y1="35" x2="45" y2="99" /&gt;
  &lt;line x1="48" y1="38" x2="48" y2="102" /&gt;
  &lt;line x1="51" y1="41" x2="51" y2="105" /&gt;
  &lt;line x1="54" y1="44" x2="54" y2="108" /&gt;
  &lt;line x1="57" y1="47" x2="57" y2="111" /&gt;
  &lt;line x1="60" y1="50" x2="60" y2="114" /&gt;
  &lt;line x1="62" y1="52" x2="62" y2="117" /&gt;
  &lt;line x1="65" y1="55" x2="65" y2="120" /&gt;
  &lt;line x1="68" y1="58" x2="68" y2="123" /&gt;
  &lt;line x1="71" y1="61" x2="71" y2="126" /&gt;
  &lt;line x1="74" y1="64" x2="74" y2="129" /&gt;
  &lt;line x1="77" y1="67" x2="77" y2="131" /&gt;
  &lt;line x1="80" y1="70" x2="80" y2="134" /&gt;
  &lt;line x1="83" y1="73" x2="83" y2="137" /&gt;
  &lt;line x1="86" y1="76" x2="86" y2="140" /&gt;
  &lt;line x1="89" y1="79" x2="89" y2="143" /&gt;
  &lt;line x1="92" y1="82" x2="92" y2="146" /&gt;
  &lt;line x1="95" y1="85" x2="95" y2="149" /&gt;
  &lt;line x1="98" y1="88" x2="98" y2="152" /&gt;
  &lt;line x1="101" y1="91" x2="101" y2="155" /&gt;
  &lt;line x1="104" y1="94" x2="104" y2="158" /&gt;
  &lt;line x1="107" y1="97" x2="107" y2="161" /&gt;
  &lt;line x1="110" y1="100" x2="110" y2="164" /&gt;
  &lt;line x1="112" y1="102" x2="112" y2="167" /&gt;
  &lt;line x1="115" y1="105" x2="115" y2="170" /&gt;
  &lt;line x1="118" y1="108" x2="118" y2="173" /&gt;
  &lt;line x1="121" y1="111" x2="121" y2="176" /&gt;
  &lt;line x1="124" y1="114" x2="124" y2="179" /&gt;
  &lt;line x1="127" y1="117" x2="127" y2="181" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 127.647059,117.647059 127.647059,181.975164 10.000000,64.328105" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="74" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="12" y1="2" x2="77" y2="2" /&gt;
  &lt;line x1="15" y1="5" x2="80" y2="5" /&gt;
  &lt;line x1="18" y1="8" x2="83" y2="8" /&gt;
  &lt;line x1="21" y1="11" x2="86" y2="11" /&gt;
  &lt;line x1="24" y1="14" x2="89" y2="14" /&gt;
  &lt;line x1="27" y1="17" x2="91" y2="17" /&gt;
  &lt;line x1="30" y1="20" x2="94" y2="20" /&gt;
  &lt;line x1="33" y1="23" x2="97" y2="23" /&gt;
  &lt;line x1="36" y1="26" x2="100" y2="26" /&gt;
  &lt;line x1="39" y1="29" x2="103" y2="29" /&gt;
  &lt;line x1="42" y1="32" x2="106" y2="32" /&gt;
  &lt;line x1="45" y1="35" x2="109" y2="35" /&gt;
  &lt;line x1="48" y1="38" x2="112" y2="38" /&gt;
  &lt;line x1="51" y1="41" x2="115" y2="41" /&gt;
  &lt;line x1="54" y1="44" x2="118" y2="44" /&gt;
  &lt;line x1="57" y1="47" x2="121" y2="47" /&gt;
  &lt;line x1="60" y1="50" x2="124" y2="50" /&gt;
  &lt;line x1="62" y1="52" x2="127" y2="52" /&gt;
  &lt;line x1="65" y1="55" x2="130" y2="55" /&gt;
  &lt;line x1="68" y1="58" x2="133" y2="58" /&gt;
  &lt;line x1="71" y1="61" x2="136" y2="61" /&gt;
  &lt;line x1="74" y1="64" x2="139" y2="64" /&gt;
  &lt;line x1="77" y1="67" x2="141" y2="67" /&gt;
  &lt;line x1="80" y1="70" x2="144" y2="70" /&gt;
  &lt;line x1="83" y1="73" x2="147" y2="73" /&gt;
  &lt;line x1="86" y1="76" x2="150" y2="76" /&gt;
  &lt;line x1="89" y1="79" x2="153" y2="79" /&gt;
  &lt;line x1="92" y1="82" x2="156" y2="82" /&gt;
  &lt;line x1="95" y1="85" x2="159" y2="85" /&gt;
  &lt;line x1="98" y1="88" x2="162" y2="88" /&gt;
  &lt;line x1="101" y1="91" x2="165" y2="91" /&gt;
  &lt;line x1="104" y1="94" x2="168" y2="94" /&gt;
  &lt;line x1="107" y1="97" x2="171" y2="97" /&gt;
  &lt;line x1="110" y1="100" x2="174" y2="100" /&gt;
  &lt;line x1="112" y1="102" x2="177" y2="102" /&gt;
  &lt;line x1="115" y1="105" x2="180" y2="105" /&gt;
  &lt;line x1="118" y1="108" x2="183" y2="108" /&gt;
  &lt;line x1="121" y1="111" x2="186" y2="111" /&gt;
  &lt;line x1="124" y1="114" x2="189" y2="114" /&gt;
  &lt;line x1="127" y1="117" x2="191" y2="117" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="127" y2="117" style="stroke-width:2" /&gt;
  &lt;line x1="26" y1="0" x2="143" y2="117" /&gt;
  &lt;line x1="42" y1="0" x2="159" y2="117" /&gt;
  &lt;line x1="58" y1="0" x2="175" y2="117" /&gt;
  &lt;line x1="74" y1="0" x2="191" y2="117" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 74.328105,0.000000 191.975164,117.647059 127.647059,117.647059" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="127" y1="117" x2="191" y2="117" style="stroke-width:2" /&gt;
  &lt;line x1="127" y1="133" x2="191" y2="133" /&gt;
  &lt;line x1="127" y1="149" x2="191" y2="149" /&gt;
  &lt;line x1="127" y1="165" x2="191" y2="165" /&gt;
  &lt;line x1="127" y1="181" x2="191" y2="181" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="127" y1="117" x2="127" y2="181" style="stroke-width:2" /&gt;
  &lt;line x1="143" y1="117" x2="143" y2="181" /&gt;
  &lt;line x1="159" y1="117" x2="159" y2="181" /&gt;
  &lt;line x1="175" y1="117" x2="175" y2="181" /&gt;
  &lt;line x1="191" y1="117" x2="191" y2="181" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="127.647059,117.647059 191.975164,117.647059 191.975164,181.975164 127.647059,181.975164" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="159.811111" y="201.975164" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;1000&lt;/text&gt;
&lt;text x="211.975164" y="149.811111" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(-90,211.975164,149.811111)"&gt;1000&lt;/text&gt;
&lt;text x="58.823529" y="143.151634" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(45,58.823529,143.151634)"&gt;10000&lt;/text&gt;
&lt;/svg&gt;&lt;/p&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;/section&gt;
&lt;section id="proxy-worker-dashboards-from-the-scheduler-dashboard"&gt;
&lt;h2&gt;Proxy Worker dashboards from the Scheduler dashboard&lt;/h2&gt;
&lt;p&gt;If you’ve used Dask.distributed they you’re probably familiar with Dask’s
scheduler dashboard, which shows the state of computations on the cluster with
a real-time interactive &lt;a class="reference external" href="https://bokeh.org"&gt;Bokeh&lt;/a&gt; dashboard. However you may
not be aware that Dask workers also have their own dashboard, which shows a
completely separate set of plots for the state of that individual worker.&lt;/p&gt;
&lt;p&gt;Historically these worker dashboards haven’t been as commonly used because it’s
hard to connect to them. Users don’t know their address, or network rules
don’t enable direct web connections. Fortunately, the scheduler dashboard is
now able to proxy a connection from the user to the worker dashbaord.&lt;/p&gt;
&lt;p&gt;You can access this by clicking on the “Info” tab and then selecting the
“dashboard” link next to any of the workers. You will need to also install
&lt;a class="reference external" href="https://github.com/jupyterhub/jupyter-server-proxy"&gt;jupyter-server-proxy&lt;/a&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 jupyter-server-proxy
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="https://github.com/quasiben"&gt;Ben Zaitlen&lt;/a&gt; for this fun addtition.
We hope that now that these plots are made more visible, people will invest
more into developing plots for them.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="black-everywhere"&gt;
&lt;h2&gt;Black everywhere&lt;/h2&gt;
&lt;p&gt;We now use the &lt;a class="reference external" href="https://black.readthedocs.io/"&gt;Black&lt;/a&gt; code formatter throughout
most Dask repositories. These repositories include pre-commit hooks, which we
recommend when developing on the project.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;cd&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;to&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;
&lt;span class="n"&gt;git&lt;/span&gt; &lt;span class="n"&gt;checkout&lt;/span&gt; &lt;span class="n"&gt;master&lt;/span&gt;
&lt;span class="n"&gt;git&lt;/span&gt; &lt;span class="n"&gt;pull&lt;/span&gt; &lt;span class="n"&gt;upstream&lt;/span&gt; &lt;span class="n"&gt;master&lt;/span&gt;

&lt;span class="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;pre&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;commit&lt;/span&gt;
&lt;span class="n"&gt;pre&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;commit&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Git will then call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;black&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;flake8&lt;/span&gt;&lt;/code&gt; whenever you attempt to commit 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/2019/06/22/dask-2.0.md&lt;/span&gt;, line 300)&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="dask-gateway"&gt;
&lt;h1&gt;Dask Gateway&lt;/h1&gt;
&lt;p&gt;We would also like to inform readers about the somewhat new &lt;a class="reference external" href="https://github.com/jcrist/dask-gateway"&gt;Dask
Gateway&lt;/a&gt; project that enables
institutions and IT to control many Dask clusters for a variety of users.&lt;/p&gt;
&lt;p&gt;&lt;img src="https://jcrist.github.io/dask-gateway/_images/architecture.svg"
     width="70%"
     alt="Dask Gateway"&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/22/dask-2.0.md&lt;/span&gt;, line 310)&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;There have been several releases since the last time we had a release blogpost.
The following people contributed to the following repositories since the 1.1.0
release on January 23rd:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;(Rick) Richard J Zamora&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Abhinav Ralhan&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Adam Beberg&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Alistair Miles&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Álvaro Abella Bascarán&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Anderson Banihirwe&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Aploium&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bart Broere&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Benjamin Zaitlen&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bouwe Andela&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brett Naul&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brian Chu&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bruce Merry&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Christian Hudon&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Cody Johnson&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dan O’Donovan&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Daniel Saxton&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Daniel Severo&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Danilo Horta&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dimplexion&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Elliott Sales de Andrade&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Endre Mark Borza&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Genevieve Buckley&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;George Sakkis&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Guillaume Lemaitre&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;HSR05&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Hameer Abbasi&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Henrique Ribeiro&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Henry Pinkard&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Hugo&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ian Bolliger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ian Rose&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Isaiah Norton&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Bourbeau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Janne Vuorela&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joe Corbett&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jorge Pessoa&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Julia Signell&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;JulianWgs&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Justin Poehnelt&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Justin Waugh&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ksenia Bobrova&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Lijo Jose&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Marco Neumann&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Mark Bell&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Michael Eaton&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Michał Jastrzębski&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Nathan Matare&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Nick Becker&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Paweł Kordek&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Peter Andreas Entschev&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Philipp Rudiger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Philipp S. Sommer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Roma Sokolov&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ross Petchler&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scott Sievert&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Shyam Saladi&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Søren Fuglede Jørgensen&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Thomas Zilio&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Yu Feng&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;aaronfowles&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;amerkel2&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;asmith26&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;btw08&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;gregrf&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;mbarkhau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;mcsoini&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;severo&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;tpanza&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed"&gt;dask/distributed&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Adam Beberg&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Benjamin Zaitlen&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brett Jurman&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brett Randall&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brian Chu&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Caleb&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Chris White&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Daniel Farrell&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Elliott Sales de Andrade&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;George Sakkis&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Bourbeau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;K.-Michael Aye&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Loïc Estève&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Magnus Nord&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Manuel Garrido&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Marco Neumann&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Mathieu Dugré&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matt Nicolls&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Michael Delgado&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Michael Spiegel&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Muammar El Khatib&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Nikos Tsaousis&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Olivier Grisel&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Peter Andreas Entschev&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Sam Grayson&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scott Sievert&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Torsten Wörtwein&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;amerkel2&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;condoratberlin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;deepthirajagopalan7&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;jukent&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;plbertrand&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml"&gt;dask/dask-ml&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Alejandro&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Florian Rohrer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Bourbeau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Julien Jerphanion&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Nathan Henrie&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Paul Vecchio&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ryan McCormick&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Saadullah Amin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scott Sievert&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Sriharsha Atyam&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-jobqueue"&gt;dask/dask-jobqueue&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Andrea Zonca&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Guillaume Eynard-Bontemps&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kyle Husmann&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Levi Naden&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Loïc Estève&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matyas Selmeci&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;ocaisa&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-kubernetes"&gt;dask/dask-kubernetes&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Brian Phillips&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jacob Tomlinson&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joe Hamman&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joseph Hamman&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Yuvi Panda&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;adam&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-examples"&gt;dask/dask-examples&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Christoph Deil&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Genevieve Buckley&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ian Rose&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthias Bussonnier&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Robert Sare&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Willi Rath&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-labextension"&gt;dask/dask-labextension&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Daniel Bast&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ian Rose&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Yuvi Panda&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/06/22/dask-2.0/"/>
    <summary>Please take the Dask User Survey for 2019.
Your reponse helps to prioritize future work.</summary>
    <category term="release" label="release"/>
    <published>2019-06-22T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/06/20/load-image-data/</id>
    <title>Load Large Image Data with Dask Array</title>
    <updated>2019-06-20T00:00:00+00:00</updated>
    <author>
      <name>John Kirkham</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/20/load-image-data.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;This post explores simple workflows to load large stacks of image data with Dask array.&lt;/p&gt;
&lt;p&gt;In particular, we start with a &lt;a class="reference external" href="https://drive.google.com/drive/folders/13mpIfqspKTIINkfoWbFsVtFF8D7jbTqJ"&gt;directory full of TIFF
files&lt;/a&gt;
of images like the following:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ $ ls raw/ | head
ex6-2_CamA_ch1_CAM1_stack0000_560nm_0000000msec_0001291795msecAbs_000x_000y_000z_0000t.tif
ex6-2_CamA_ch1_CAM1_stack0001_560nm_0043748msec_0001335543msecAbs_000x_000y_000z_0000t.tif
ex6-2_CamA_ch1_CAM1_stack0002_560nm_0087497msec_0001379292msecAbs_000x_000y_000z_0000t.tif
ex6-2_CamA_ch1_CAM1_stack0003_560nm_0131245msec_0001423040msecAbs_000x_000y_000z_0000t.tif
ex6-2_CamA_ch1_CAM1_stack0004_560nm_0174993msec_0001466788msecAbs_000x_000y_000z_0000t.tif
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;and show how to stitch these together into large lazy arrays
using the &lt;a class="reference external" href="https://image.dask.org/en/latest/"&gt;dask-image&lt;/a&gt; library&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_image&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_image&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;raw/*.tif&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;or by writing your own Dask delayed image reader function.&lt;/p&gt;
&lt;table&gt;
&lt;tr&gt;
&lt;td&gt;
&lt;table&gt;  &lt;thead&gt;    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 3.16 GB &lt;/td&gt; &lt;td&gt; 316.15 MB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (2010, 1024, 768) &lt;/td&gt; &lt;td&gt; (201, 1024, 768) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 30 Tasks &lt;/td&gt;&lt;td&gt; 10 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; uint16 &lt;/td&gt;&lt;td&gt; numpy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;&lt;/table&gt;
&lt;/td&gt;
&lt;td&gt;
&lt;svg width="176" height="181" style="stroke:rgb(0,0,0);stroke-width:1" &gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="80" y2="70" style="stroke-width:2" /&gt;
  &lt;line x1="10" y1="61" x2="80" y2="131" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="10" y2="61" style="stroke-width:2" /&gt;
  &lt;line x1="17" y1="7" x2="17" y2="68" /&gt;
  &lt;line x1="24" y1="14" x2="24" y2="75" /&gt;
  &lt;line x1="31" y1="21" x2="31" y2="82" /&gt;
  &lt;line x1="38" y1="28" x2="38" y2="89" /&gt;
  &lt;line x1="45" y1="35" x2="45" y2="96" /&gt;
  &lt;line x1="52" y1="42" x2="52" y2="103" /&gt;
  &lt;line x1="59" y1="49" x2="59" y2="110" /&gt;
  &lt;line x1="66" y1="56" x2="66" y2="117" /&gt;
  &lt;line x1="73" y1="63" x2="73" y2="124" /&gt;
  &lt;line x1="80" y1="70" x2="80" y2="131" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 80.588235,70.588235 80.588235,131.722564 10.000000,61.134328" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="55" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="17" y1="7" x2="62" y2="7" /&gt;
  &lt;line x1="24" y1="14" x2="69" y2="14" /&gt;
  &lt;line x1="31" y1="21" x2="77" y2="21" /&gt;
  &lt;line x1="38" y1="28" x2="84" y2="28" /&gt;
  &lt;line x1="45" y1="35" x2="91" y2="35" /&gt;
  &lt;line x1="52" y1="42" x2="98" y2="42" /&gt;
  &lt;line x1="59" y1="49" x2="105" y2="49" /&gt;
  &lt;line x1="66" y1="56" x2="112" y2="56" /&gt;
  &lt;line x1="73" y1="63" x2="119" y2="63" /&gt;
  &lt;line x1="80" y1="70" x2="126" y2="70" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="80" y2="70" style="stroke-width:2" /&gt;
  &lt;line x1="55" y1="0" x2="126" y2="70" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 55.850746,0.000000 126.438982,70.588235 80.588235,70.588235" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="80" y1="70" x2="126" y2="70" style="stroke-width:2" /&gt;
  &lt;line x1="80" y1="131" x2="126" y2="131" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="80" y1="70" x2="80" y2="131" style="stroke-width:2" /&gt;
  &lt;line x1="126" y1="70" x2="126" y2="131" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="80.588235,70.588235 126.438982,70.588235 126.438982,131.722564 80.588235,131.722564" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="103.513608" y="151.722564" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;768&lt;/text&gt;
&lt;text x="146.438982" y="101.155399" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(-90,146.438982,101.155399)"&gt;1024&lt;/text&gt;
&lt;text x="35.294118" y="116.428446" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(45,35.294118,116.428446)"&gt;2010&lt;/text&gt;
&lt;/svg&gt;&lt;/p&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;Some day we’ll eventually be able to perform complex calculations on this dask array.&lt;/p&gt;
&lt;p&gt;&lt;img src="https://raw.githubusercontent.com/mrocklin/raw-host/gh-pages/images/aollsm-index-1.jpg"
     width="45%"
     alt="Light Microscopy data rendered with NVidia IndeX"&gt;
&lt;img src="https://raw.githubusercontent.com/mrocklin/raw-host/gh-pages/images/aollsm-index-2.jpg"
     width="45%"
     alt="Light Microscopy data rendered with NVidia IndeX"&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Disclaimer: we’re not going to produce rendered images like the above in this
post. These were created with &lt;a class="reference external" href="https://developer.nvidia.com/index"&gt;NVidia
IndeX&lt;/a&gt;, a completely separate tool chain
from what is being discussed here. This post covers the first step of image
loading.&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/20/load-image-data.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="series-overview"&gt;
&lt;h1&gt;Series Overview&lt;/h1&gt;
&lt;p&gt;A common case in fields that acquire large amounts of imaging data is to write
out smaller acquisitions into many small files. These files can tile a larger
space, sub-sample from a larger time period, and may contain multiple channels.
The acquisition techniques themselves are often state of the art and constantly
pushing the envelope in term of how large a field of view can be acquired, at
what resolution, and what quality.&lt;/p&gt;
&lt;p&gt;Once acquired this data presents a number of challenges. Algorithms often
designed and tested to work on very small pieces of this data need to be scaled
up to work on the full dataset. It might not be clear at the outset what will
actually work and so exploration still plays a very big part of the whole
process.&lt;/p&gt;
&lt;p&gt;Historically this analytical process has involved a lot of custom code. Often
the analytical process is stitched together by a series of scripts possibly in
several different languages that write various intermediate results to disk.
Thanks to advances in modern tooling these process can be significantly
improved. In this series of blogposts, we will outline ways for image
scientists to leverage different tools to move towards a high level, friendly,
cohesive, interactive analytical pipeline.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/20/load-image-data.md&lt;/span&gt;, line 151)&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="post-overview"&gt;
&lt;h1&gt;Post Overview&lt;/h1&gt;
&lt;p&gt;This post in particular focuses on loading and managing large stacks of image
data in parallel from Python.&lt;/p&gt;
&lt;p&gt;Loading large image data can be a complex and often unique problem. Different
groups may choose to store this across many files on disk, a commodity or
custom database solution, or they may opt to store it in the cloud. Not all
datasets within the same group may be treated the same for a variety of
reasons. In short, this means loading data is a hard and expensive problem.&lt;/p&gt;
&lt;p&gt;Despite data being stored in many different ways, often groups want to reapply
the same analytical pipeline to these datasets. However if the data pipeline is
tightly coupled to a particular way of loading the data for later analytical
steps, it may be very difficult if not impossible to reuse an existing
pipeline. In other words, there is friction between the loading and analysis
steps, which frustrates efforts to make things reusable.&lt;/p&gt;
&lt;p&gt;Having a modular and general way to load data makes it easy to present data
stored differently in a standard way. Further having a standard way to present
data to analytical pipelines allows that part of the pipeline to focus on what
it does best, analysis! In general, this should decouple these to components in
a way that improves the experience of users involved in all parts of the
pipeline.&lt;/p&gt;
&lt;p&gt;We will use
&lt;a class="reference external" href="https://drive.google.com/drive/folders/13mpIfqspKTIINkfoWbFsVtFF8D7jbTqJ"&gt;image data&lt;/a&gt;
generously provided by
&lt;a class="reference external" href="https://scholar.google.com/citations?user=nxwNAEgAAAAJ&amp;amp;amp;hl=en"&gt;Gokul Upadhyayula&lt;/a&gt;
at the
&lt;a class="reference external" href="http://microscopy.berkeley.edu/"&gt;Advanced Bioimaging Center&lt;/a&gt;
at UC Berkeley and discussed in
&lt;a class="reference external" href="https://science.sciencemag.org/content/360/6386/eaaq1392"&gt;this paper&lt;/a&gt;
(&lt;a class="reference external" href="https://www.biorxiv.org/content/10.1101/243352v2"&gt;preprint&lt;/a&gt;),
though the workloads presented here should work for any kind of imaging data,
or array data generally.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/20/load-image-data.md&lt;/span&gt;, line 188)&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="load-image-data-with-dask"&gt;
&lt;h1&gt;Load image data with Dask&lt;/h1&gt;
&lt;p&gt;Let’s start again with our image data from the top of the post:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ $ ls /path/to/files/raw/ | head
ex6-2_CamA_ch1_CAM1_stack0000_560nm_0000000msec_0001291795msecAbs_000x_000y_000z_0000t.tif
ex6-2_CamA_ch1_CAM1_stack0001_560nm_0043748msec_0001335543msecAbs_000x_000y_000z_0000t.tif
ex6-2_CamA_ch1_CAM1_stack0002_560nm_0087497msec_0001379292msecAbs_000x_000y_000z_0000t.tif
ex6-2_CamA_ch1_CAM1_stack0003_560nm_0131245msec_0001423040msecAbs_000x_000y_000z_0000t.tif
ex6-2_CamA_ch1_CAM1_stack0004_560nm_0174993msec_0001466788msecAbs_000x_000y_000z_0000t.tif
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;section id="load-a-single-sample-image-with-scikit-image"&gt;
&lt;h2&gt;Load a single sample image with Scikit-Image&lt;/h2&gt;
&lt;p&gt;To load a single image, we use &lt;a class="reference external" href="https://scikit-image.org/"&gt;Scikit-Image&lt;/a&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;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="n"&gt;glob&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;glob&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;/path/to/files/raw/*.tif&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;len&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="go"&gt;597&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;imageio&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;sample&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;imageio&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&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="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;sample&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;(201, 1024, 768)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Each filename corresponds to some 3d chunk of a larger image. We can look at a
few 2d slices of this single 3d chunk to get some 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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;matplotlib.pyplot&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;plt&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;skimage.io&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;figure&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;figsize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&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;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sample&lt;/span&gt;&lt;span class="p"&gt;[:,&lt;/span&gt; &lt;span class="p"&gt;:,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img src="https://raw.githubusercontent.com/mrocklin/raw-host/gh-pages/images/aollsm-sample-1.png"
     width="60%"&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;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;figure&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;figsize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&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;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sample&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img src="https://raw.githubusercontent.com/mrocklin/raw-host/gh-pages/images/aollsm-sample-2.png"
     width="60%"&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;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;figure&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;figsize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&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;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sample&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="p"&gt;:])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img src="https://raw.githubusercontent.com/mrocklin/raw-host/gh-pages/images/aollsm-sample-3.png"
     width="60%"&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="investigate-filename-structure"&gt;
&lt;h2&gt;Investigate Filename Structure&lt;/h2&gt;
&lt;p&gt;These are slices from only one chunk of a much larger aggregate image.
Our interest here is combining the pieces into a large image stack.
It is common to see a naming structure in the filenames. Each
filename then may indicate a channel, time step, and spatial location with the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;&amp;lt;i&amp;gt;&lt;/span&gt;&lt;/code&gt; being some numeric values (possibly with units). Individual filenames may
have more or less information and may notate it differently than we have.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;mydata_ch&lt;/span&gt;&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt;&lt;span class="n"&gt;_&lt;/span&gt;&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;j&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt;&lt;span class="n"&gt;t_&lt;/span&gt;&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;k&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt;&lt;span class="n"&gt;x_&lt;/span&gt;&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;l&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt;&lt;span class="n"&gt;y_&lt;/span&gt;&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;m&lt;/span&gt;&lt;span class="o"&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;tif&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In principle with NumPy we might allocate a giant array and then iteratively
load images and place them into the giant 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;full_array&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;empty&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="o"&gt;...&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="o"&gt;...&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;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;sample&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dtype&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="n"&gt;img&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;imageio&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&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;index&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;get_location_from_filename&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="c1"&gt;# We need to write this function&lt;/span&gt;
    &lt;span class="n"&gt;full_array&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;:,&lt;/span&gt; &lt;span class="p"&gt;:,&lt;/span&gt; &lt;span class="p"&gt;:]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;img&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;However if our data is large then we can’t load it all into memory at once like
this into a single Numpy array, and instead we need to be a bit more clever to
handle it efficiently. One approach here is to use &lt;a class="reference external" href="https://dask.org"&gt;Dask&lt;/a&gt;,
which handles larger-than-memory workloads easily.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="lazily-load-images-with-dask-array"&gt;
&lt;h2&gt;Lazily load images with Dask Array&lt;/h2&gt;
&lt;p&gt;Now we learn how to lazily load and stitch together image data with Dask array.
We’ll start with simple examples first and then move onto the full example with
this more complex dataset afterwards.&lt;/p&gt;
&lt;p&gt;We can delay the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;imageio.imread&lt;/span&gt;&lt;/code&gt; calls with &lt;a class="reference external" href="https://docs.dassk.org/en/latest/delayed.html"&gt;Dask
Delayed&lt;/a&gt;.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="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="n"&gt;lazy_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;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;imageio&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&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="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="n"&gt;lazy_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_delayed&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;shape&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;sample&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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="n"&gt;sample&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dtype&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;lazy_arrays&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;em&gt;Note: here we’re assuming that all of the images have the same shape and dtype
as the sample file that we loaded above. This is not always the case. See the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask_image&lt;/span&gt;&lt;/code&gt; note below in the Future Work section for an alternative.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;We haven’t yet stitched these together. We have hundreds of single-chunk Dask
arrays, each of which lazily loads a single 3d chunk of data from disk. Lets look at a single 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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;lazy_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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table&gt;
&lt;tr&gt;
&lt;td&gt;
&lt;table&gt;  &lt;thead&gt;    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 316.15 MB &lt;/td&gt; &lt;td&gt; 316.15 MB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (201, 1024, 768) &lt;/td&gt; &lt;td&gt; (201, 1024, 768) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 2 Tasks &lt;/td&gt;&lt;td&gt; 1 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; uint16 &lt;/td&gt;&lt;td&gt; numpy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;&lt;/table&gt;
&lt;/td&gt;
&lt;td&gt;
&lt;svg width="174" height="194" style="stroke:rgb(0,0,0);stroke-width:1" &gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="34" y2="24" style="stroke-width:2" /&gt;
  &lt;line x1="10" y1="120" x2="34" y2="144" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="10" y2="120" style="stroke-width:2" /&gt;
  &lt;line x1="34" y1="24" x2="34" y2="144" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 34.664918,24.664918 34.664918,144.664918 10.000000,120.000000" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="100" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="34" y1="24" x2="124" y2="24" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="34" y2="24" style="stroke-width:2" /&gt;
  &lt;line x1="100" y1="0" x2="124" y2="24" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 100.000000,0.000000 124.664918,24.664918 34.664918,24.664918" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="34" y1="24" x2="124" y2="24" style="stroke-width:2" /&gt;
  &lt;line x1="34" y1="144" x2="124" y2="144" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="34" y1="24" x2="34" y2="144" style="stroke-width:2" /&gt;
  &lt;line x1="124" y1="24" x2="124" y2="144" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="34.664918,24.664918 124.664918,24.664918 124.664918,144.664918 34.664918,144.664918" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="79.664918" y="164.664918" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;768&lt;/text&gt;
&lt;text x="144.664918" y="84.664918" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(-90,144.664918,84.664918)"&gt;1024&lt;/text&gt;
&lt;text x="12.332459" y="152.332459" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(45,12.332459,152.332459)"&gt;201&lt;/text&gt;
&lt;/svg&gt;&lt;/p&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;This is a lazy 3-dimensional Dask array of a &lt;em&gt;single&lt;/em&gt; 300MB chunk of data.
That chunk is created by loading in a particular TIFF file. Normally Dask
arrays are composed of &lt;em&gt;many&lt;/em&gt; chunks. We can concatenate many of these
single-chunked Dask arrays into a multi-chunked Dask array with functions like
&lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html#dask.array.concatenate"&gt;da.concatenate&lt;/a&gt;
and
&lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html#dask.array.stack"&gt;da.stack&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Here we concatenate the first ten Dask arrays along a few axes, to get an
easier-to-understand picture of how this looks. Take a look both at how the
shape changes as we change the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;axis=&lt;/span&gt;&lt;/code&gt; parameter both in the table on the left
and the image on the right.&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;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;lazy_arrays&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;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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table&gt;
&lt;tr&gt;
&lt;td&gt;
&lt;table&gt;  &lt;thead&gt;    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 3.16 GB &lt;/td&gt; &lt;td&gt; 316.15 MB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (2010, 1024, 768) &lt;/td&gt; &lt;td&gt; (201, 1024, 768) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 30 Tasks &lt;/td&gt;&lt;td&gt; 10 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; uint16 &lt;/td&gt;&lt;td&gt; numpy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;&lt;/table&gt;
&lt;/td&gt;
&lt;td&gt;
&lt;svg width="176" height="181" style="stroke:rgb(0,0,0);stroke-width:1" &gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="80" y2="70" style="stroke-width:2" /&gt;
  &lt;line x1="10" y1="61" x2="80" y2="131" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="10" y2="61" style="stroke-width:2" /&gt;
  &lt;line x1="17" y1="7" x2="17" y2="68" /&gt;
  &lt;line x1="24" y1="14" x2="24" y2="75" /&gt;
  &lt;line x1="31" y1="21" x2="31" y2="82" /&gt;
  &lt;line x1="38" y1="28" x2="38" y2="89" /&gt;
  &lt;line x1="45" y1="35" x2="45" y2="96" /&gt;
  &lt;line x1="52" y1="42" x2="52" y2="103" /&gt;
  &lt;line x1="59" y1="49" x2="59" y2="110" /&gt;
  &lt;line x1="66" y1="56" x2="66" y2="117" /&gt;
  &lt;line x1="73" y1="63" x2="73" y2="124" /&gt;
  &lt;line x1="80" y1="70" x2="80" y2="131" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 80.588235,70.588235 80.588235,131.722564 10.000000,61.134328" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="55" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="17" y1="7" x2="62" y2="7" /&gt;
  &lt;line x1="24" y1="14" x2="69" y2="14" /&gt;
  &lt;line x1="31" y1="21" x2="77" y2="21" /&gt;
  &lt;line x1="38" y1="28" x2="84" y2="28" /&gt;
  &lt;line x1="45" y1="35" x2="91" y2="35" /&gt;
  &lt;line x1="52" y1="42" x2="98" y2="42" /&gt;
  &lt;line x1="59" y1="49" x2="105" y2="49" /&gt;
  &lt;line x1="66" y1="56" x2="112" y2="56" /&gt;
  &lt;line x1="73" y1="63" x2="119" y2="63" /&gt;
  &lt;line x1="80" y1="70" x2="126" y2="70" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="80" y2="70" style="stroke-width:2" /&gt;
  &lt;line x1="55" y1="0" x2="126" y2="70" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 55.850746,0.000000 126.438982,70.588235 80.588235,70.588235" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="80" y1="70" x2="126" y2="70" style="stroke-width:2" /&gt;
  &lt;line x1="80" y1="131" x2="126" y2="131" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="80" y1="70" x2="80" y2="131" style="stroke-width:2" /&gt;
  &lt;line x1="126" y1="70" x2="126" y2="131" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="80.588235,70.588235 126.438982,70.588235 126.438982,131.722564 80.588235,131.722564" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="103.513608" y="151.722564" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;768&lt;/text&gt;
&lt;text x="146.438982" y="101.155399" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(-90,146.438982,101.155399)"&gt;1024&lt;/text&gt;
&lt;text x="35.294118" y="116.428446" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(45,35.294118,116.428446)"&gt;2010&lt;/text&gt;
&lt;/svg&gt;&lt;/p&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&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;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;lazy_arrays&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;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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table&gt;
&lt;tr&gt;
&lt;td&gt;
&lt;table&gt;  &lt;thead&gt;    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 3.16 GB &lt;/td&gt; &lt;td&gt; 316.15 MB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (201, 10240, 768) &lt;/td&gt; &lt;td&gt; (201, 1024, 768) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 30 Tasks &lt;/td&gt;&lt;td&gt; 10 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; uint16 &lt;/td&gt;&lt;td&gt; numpy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;&lt;/table&gt;
&lt;/td&gt;
&lt;td&gt;
&lt;svg width="113" height="187" style="stroke:rgb(0,0,0);stroke-width:1" &gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="27" y2="17" style="stroke-width:2" /&gt;
  &lt;line x1="10" y1="12" x2="27" y2="29" /&gt;
  &lt;line x1="10" y1="24" x2="27" y2="41" /&gt;
  &lt;line x1="10" y1="36" x2="27" y2="53" /&gt;
  &lt;line x1="10" y1="48" x2="27" y2="65" /&gt;
  &lt;line x1="10" y1="60" x2="27" y2="77" /&gt;
  &lt;line x1="10" y1="72" x2="27" y2="89" /&gt;
  &lt;line x1="10" y1="84" x2="27" y2="101" /&gt;
  &lt;line x1="10" y1="96" x2="27" y2="113" /&gt;
  &lt;line x1="10" y1="108" x2="27" y2="125" /&gt;
  &lt;line x1="10" y1="120" x2="27" y2="137" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="10" y2="120" style="stroke-width:2" /&gt;
  &lt;line x1="27" y1="17" x2="27" y2="137" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 27.014952,17.014952 27.014952,137.014952 10.000000,120.000000" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="46" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="27" y1="17" x2="63" y2="17" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="27" y2="17" style="stroke-width:2" /&gt;
  &lt;line x1="46" y1="0" x2="63" y2="17" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 46.948234,0.000000 63.963186,17.014952 27.014952,17.014952" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="27" y1="17" x2="63" y2="17" style="stroke-width:2" /&gt;
  &lt;line x1="27" y1="29" x2="63" y2="29" /&gt;
  &lt;line x1="27" y1="41" x2="63" y2="41" /&gt;
  &lt;line x1="27" y1="53" x2="63" y2="53" /&gt;
  &lt;line x1="27" y1="65" x2="63" y2="65" /&gt;
  &lt;line x1="27" y1="77" x2="63" y2="77" /&gt;
  &lt;line x1="27" y1="89" x2="63" y2="89" /&gt;
  &lt;line x1="27" y1="101" x2="63" y2="101" /&gt;
  &lt;line x1="27" y1="113" x2="63" y2="113" /&gt;
  &lt;line x1="27" y1="125" x2="63" y2="125" /&gt;
  &lt;line x1="27" y1="137" x2="63" y2="137" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="27" y1="17" x2="27" y2="137" style="stroke-width:2" /&gt;
  &lt;line x1="63" y1="17" x2="63" y2="137" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="27.014952,17.014952 63.963186,17.014952 63.963186,137.014952 27.014952,137.014952" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="45.489069" y="157.014952" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;768&lt;/text&gt;
&lt;text x="83.963186" y="77.014952" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(-90,83.963186,77.014952)"&gt;10240&lt;/text&gt;
&lt;text x="8.507476" y="148.507476" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(45,8.507476,148.507476)"&gt;201&lt;/text&gt;
&lt;/svg&gt;&lt;/p&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&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;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;lazy_arrays&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;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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table&gt;
&lt;tr&gt;
&lt;td&gt;
&lt;table&gt;  &lt;thead&gt;    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 3.16 GB &lt;/td&gt; &lt;td&gt; 316.15 MB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (201, 1024, 7680) &lt;/td&gt; &lt;td&gt; (201, 1024, 768) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 30 Tasks &lt;/td&gt;&lt;td&gt; 10 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; uint16 &lt;/td&gt;&lt;td&gt; numpy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;&lt;/table&gt;
&lt;/td&gt;
&lt;td&gt;
&lt;svg width="197" height="108" style="stroke:rgb(0,0,0);stroke-width:1" &gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="27" y2="17" style="stroke-width:2" /&gt;
  &lt;line x1="10" y1="40" x2="27" y2="58" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="10" y2="40" style="stroke-width:2" /&gt;
  &lt;line x1="27" y1="17" x2="27" y2="58" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 27.988258,17.988258 27.988258,58.112379 10.000000,40.124121" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="10" y1="0" x2="130" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="27" y1="17" x2="147" y2="17" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="10" y1="0" x2="27" y2="17" style="stroke-width:2" /&gt;
  &lt;line x1="22" y1="0" x2="39" y2="17" /&gt;
  &lt;line x1="34" y1="0" x2="51" y2="17" /&gt;
  &lt;line x1="46" y1="0" x2="63" y2="17" /&gt;
  &lt;line x1="58" y1="0" x2="75" y2="17" /&gt;
  &lt;line x1="70" y1="0" x2="87" y2="17" /&gt;
  &lt;line x1="82" y1="0" x2="99" y2="17" /&gt;
  &lt;line x1="94" y1="0" x2="111" y2="17" /&gt;
  &lt;line x1="106" y1="0" x2="123" y2="17" /&gt;
  &lt;line x1="118" y1="0" x2="135" y2="17" /&gt;
  &lt;line x1="130" y1="0" x2="147" y2="17" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="10.000000,0.000000 130.000000,0.000000 147.988258,17.988258 27.988258,17.988258" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="27" y1="17" x2="147" y2="17" style="stroke-width:2" /&gt;
  &lt;line x1="27" y1="58" x2="147" y2="58" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="27" y1="17" x2="27" y2="58" style="stroke-width:2" /&gt;
  &lt;line x1="39" y1="17" x2="39" y2="58" /&gt;
  &lt;line x1="51" y1="17" x2="51" y2="58" /&gt;
  &lt;line x1="63" y1="17" x2="63" y2="58" /&gt;
  &lt;line x1="75" y1="17" x2="75" y2="58" /&gt;
  &lt;line x1="87" y1="17" x2="87" y2="58" /&gt;
  &lt;line x1="99" y1="17" x2="99" y2="58" /&gt;
  &lt;line x1="111" y1="17" x2="111" y2="58" /&gt;
  &lt;line x1="123" y1="17" x2="123" y2="58" /&gt;
  &lt;line x1="135" y1="17" x2="135" y2="58" /&gt;
  &lt;line x1="147" y1="17" x2="147" y2="58" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="27.988258,17.988258 147.988258,17.988258 147.988258,58.112379 27.988258,58.112379" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="87.988258" y="78.112379" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;7680&lt;/text&gt;
&lt;text x="167.988258" y="38.050318" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(-90,167.988258,38.050318)"&gt;1024&lt;/text&gt;
&lt;text x="8.994129" y="69.118250" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(45,8.994129,69.118250)"&gt;201&lt;/text&gt;
&lt;/svg&gt;&lt;/p&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;Or, if we wanted to make a new dimension, we would use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;da.stack&lt;/span&gt;&lt;/code&gt;. In this
case note that we’ve run out of easily visible dimensions, so you should take
note of the listed shape in the table input on the left more than the picture
on the right. Notice that we’ve stacked these 3d images into a 4d image.&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;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;lazy_arrays&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table&gt;
&lt;tr&gt;
&lt;td&gt;
&lt;table&gt;  &lt;thead&gt;    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 3.16 GB &lt;/td&gt; &lt;td&gt; 316.15 MB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (10, 201, 1024, 768) &lt;/td&gt; &lt;td&gt; (1, 201, 1024, 768) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 30 Tasks &lt;/td&gt;&lt;td&gt; 10 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; uint16 &lt;/td&gt;&lt;td&gt; numpy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;&lt;/table&gt;
&lt;/td&gt;
&lt;td&gt;
&lt;svg width="354" height="194" style="stroke:rgb(0,0,0);stroke-width:1" &gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="0" y1="0" x2="25" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="0" y1="25" x2="25" y2="25" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="0" y1="0" x2="0" y2="25" style="stroke-width:2" /&gt;
  &lt;line x1="2" y1="0" x2="2" y2="25" /&gt;
  &lt;line x1="5" y1="0" x2="5" y2="25" /&gt;
  &lt;line x1="7" y1="0" x2="7" y2="25" /&gt;
  &lt;line x1="10" y1="0" x2="10" y2="25" /&gt;
  &lt;line x1="12" y1="0" x2="12" y2="25" /&gt;
  &lt;line x1="15" y1="0" x2="15" y2="25" /&gt;
  &lt;line x1="17" y1="0" x2="17" y2="25" /&gt;
  &lt;line x1="20" y1="0" x2="20" y2="25" /&gt;
  &lt;line x1="22" y1="0" x2="22" y2="25" /&gt;
  &lt;line x1="25" y1="0" x2="25" y2="25" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="0.000000,0.000000 25.412617,0.000000 25.412617,25.412617 0.000000,25.412617" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="12.706308" y="45.412617" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;10&lt;/text&gt;
&lt;text x="45.412617" y="12.706308" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(0,45.412617,12.706308)"&gt;1&lt;/text&gt;&lt;/p&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="95" y1="0" x2="119" y2="24" style="stroke-width:2" /&gt;
  &lt;line x1="95" y1="120" x2="119" y2="144" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="95" y1="0" x2="95" y2="120" style="stroke-width:2" /&gt;
  &lt;line x1="119" y1="24" x2="119" y2="144" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="95.000000,0.000000 119.664918,24.664918 119.664918,144.664918 95.000000,120.000000" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="95" y1="0" x2="185" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="119" y1="24" x2="209" y2="24" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="95" y1="0" x2="119" y2="24" style="stroke-width:2" /&gt;
  &lt;line x1="185" y1="0" x2="209" y2="24" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="95.000000,0.000000 185.000000,0.000000 209.664918,24.664918 119.664918,24.664918" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="119" y1="24" x2="209" y2="24" style="stroke-width:2" /&gt;
  &lt;line x1="119" y1="144" x2="209" y2="144" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="119" y1="24" x2="119" y2="144" style="stroke-width:2" /&gt;
  &lt;line x1="209" y1="24" x2="209" y2="144" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="119.664918,24.664918 209.664918,24.664918 209.664918,144.664918 119.664918,144.664918" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="164.664918" y="164.664918" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;768&lt;/text&gt;
&lt;text x="229.664918" y="84.664918" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(-90,229.664918,84.664918)"&gt;1024&lt;/text&gt;
&lt;text x="97.332459" y="152.332459" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(45,97.332459,152.332459)"&gt;201&lt;/text&gt;
&lt;/svg&gt;&lt;/p&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;These are the common case situations, where you have a single axis along which
you want to stitch images together.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="full-example"&gt;
&lt;h2&gt;Full example&lt;/h2&gt;
&lt;p&gt;This works fine for combining along a single axis. However if we need to
combine across multiple we need to perform multiple concatenate steps.
Fortunately there is a simpler option &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html#dask.array.block"&gt;da.block&lt;/a&gt;, which can
concatenate along multiple axes at once if you give it a nested list of dask
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="n"&gt;a&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;block&lt;/span&gt;&lt;span class="p"&gt;([[&lt;/span&gt;&lt;span class="n"&gt;laxy_array_00&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;lazy_array_01&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
              &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;lazy_array_10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;lazy_array_11&lt;/span&gt;&lt;span class="p"&gt;]])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We now do the following:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Parse each filename to learn where it should live in the larger array&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;See how many files are in each of our relevant dimensions&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Allocate a NumPy object-dtype array of the appropriate size, where each
element of this array will hold a single-chunk Dask array&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Go through our filenames and insert the proper Dask array into the right
position&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;da.block&lt;/span&gt;&lt;/code&gt; on the result&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This code is a bit complex, but shows what this looks like in a real-world
setting&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Get various dimensions&lt;/span&gt;

&lt;span class="n"&gt;fn_comp_sets&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="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="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;comp&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;os&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;splitext&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="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;split&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;_&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)):&lt;/span&gt;
        &lt;span class="n"&gt;fn_comp_sets&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;setdefault&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="nb"&gt;set&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
        &lt;span class="n"&gt;fn_comp_sets&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;comp&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;fn_comp_sets&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;sorted&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;fn_comp_sets&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;values&lt;/span&gt;&lt;span class="p"&gt;()))&lt;/span&gt;

&lt;span class="n"&gt;remap_comps&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;
    &lt;span class="nb"&gt;dict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;reversed&lt;/span&gt;&lt;span class="p"&gt;,&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;fn_comp_sets&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;]))),&lt;/span&gt;
    &lt;span class="nb"&gt;dict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;reversed&lt;/span&gt;&lt;span class="p"&gt;,&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;fn_comp_sets&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="p"&gt;]&lt;/span&gt;

&lt;span class="c1"&gt;# Create an empty object array to organize each chunk that loads a TIFF&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;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;empty&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="nb"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;remap_comps&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;1&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;1&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="nb"&gt;object&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="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;zip&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;lazy_arrays&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;channel&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;fn&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;fn&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;_ch&amp;quot;&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="p"&gt;:]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;split&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;_&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
    &lt;span class="n"&gt;stack&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;fn&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;fn&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;_stack&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&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;split&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;_&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)[&lt;/span&gt;&lt;span class="mi"&gt;0&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;channel&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;stack&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="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="c1"&gt;# Stitch together the many blocks into a single array&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;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;block&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;tolist&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;table&gt;  &lt;thead&gt;    &lt;tr&gt;&lt;td&gt; &lt;/td&gt;&lt;th&gt; Array &lt;/th&gt;&lt;th&gt; Chunk &lt;/th&gt;&lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;&lt;th&gt; Bytes &lt;/th&gt;&lt;td&gt; 188.74 GB &lt;/td&gt; &lt;td&gt; 316.15 MB &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Shape &lt;/th&gt;&lt;td&gt; (3, 199, 201, 1024, 768) &lt;/td&gt; &lt;td&gt; (1, 1, 201, 1024, 768) &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Count &lt;/th&gt;&lt;td&gt; 2985 Tasks &lt;/td&gt;&lt;td&gt; 597 Chunks &lt;/td&gt;&lt;/tr&gt;
    &lt;tr&gt;&lt;th&gt; Type &lt;/th&gt;&lt;td&gt; uint16 &lt;/td&gt;&lt;td&gt; numpy.ndarray &lt;/td&gt;&lt;/tr&gt;
  &lt;/tbody&gt;&lt;/table&gt;
&lt;/td&gt;
&lt;td&gt;
&lt;svg width="386" height="194" style="stroke:rgb(0,0,0);stroke-width:1" &gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="0" y1="0" x2="41" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="0" y1="8" x2="41" y2="8" /&gt;
  &lt;line x1="0" y1="16" x2="41" y2="16" /&gt;
  &lt;line x1="0" y1="25" x2="41" y2="25" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="0" y1="0" x2="0" y2="25" style="stroke-width:2" /&gt;
  &lt;line x1="0" y1="0" x2="0" y2="25" /&gt;
  &lt;line x1="0" y1="0" x2="0" y2="25" /&gt;
  &lt;line x1="0" y1="0" x2="0" y2="25" /&gt;
  &lt;line x1="0" y1="0" x2="0" y2="25" /&gt;
  &lt;line x1="1" y1="0" x2="1" y2="25" /&gt;
  &lt;line x1="1" y1="0" x2="1" y2="25" /&gt;
  &lt;line x1="1" y1="0" x2="1" y2="25" /&gt;
  &lt;line x1="1" y1="0" x2="1" y2="25" /&gt;
  &lt;line x1="1" y1="0" x2="1" y2="25" /&gt;
  &lt;line x1="2" y1="0" x2="2" y2="25" /&gt;
  &lt;line x1="2" y1="0" x2="2" y2="25" /&gt;
  &lt;line x1="2" y1="0" x2="2" y2="25" /&gt;
  &lt;line x1="2" y1="0" x2="2" y2="25" /&gt;
  &lt;line x1="2" y1="0" x2="2" y2="25" /&gt;
  &lt;line x1="3" y1="0" x2="3" y2="25" /&gt;
  &lt;line x1="3" y1="0" x2="3" y2="25" /&gt;
  &lt;line x1="3" y1="0" x2="3" y2="25" /&gt;
  &lt;line x1="3" y1="0" x2="3" y2="25" /&gt;
  &lt;line x1="3" y1="0" x2="3" y2="25" /&gt;
  &lt;line x1="4" y1="0" x2="4" y2="25" /&gt;
  &lt;line x1="4" y1="0" x2="4" y2="25" /&gt;
  &lt;line x1="4" y1="0" x2="4" y2="25" /&gt;
  &lt;line x1="4" y1="0" x2="4" y2="25" /&gt;
  &lt;line x1="5" y1="0" x2="5" y2="25" /&gt;
  &lt;line x1="5" y1="0" x2="5" y2="25" /&gt;
  &lt;line x1="5" y1="0" x2="5" y2="25" /&gt;
  &lt;line x1="5" y1="0" x2="5" y2="25" /&gt;
  &lt;line x1="5" y1="0" x2="5" y2="25" /&gt;
  &lt;line x1="6" y1="0" x2="6" y2="25" /&gt;
  &lt;line x1="6" y1="0" x2="6" y2="25" /&gt;
  &lt;line x1="6" y1="0" x2="6" y2="25" /&gt;
  &lt;line x1="6" y1="0" x2="6" y2="25" /&gt;
  &lt;line x1="6" y1="0" x2="6" y2="25" /&gt;
  &lt;line x1="7" y1="0" x2="7" y2="25" /&gt;
  &lt;line x1="7" y1="0" x2="7" y2="25" /&gt;
  &lt;line x1="7" y1="0" x2="7" y2="25" /&gt;
  &lt;line x1="7" y1="0" x2="7" y2="25" /&gt;
  &lt;line x1="7" y1="0" x2="7" y2="25" /&gt;
  &lt;line x1="8" y1="0" x2="8" y2="25" /&gt;
  &lt;line x1="8" y1="0" x2="8" y2="25" /&gt;
  &lt;line x1="8" y1="0" x2="8" y2="25" /&gt;
  &lt;line x1="8" y1="0" x2="8" y2="25" /&gt;
  &lt;line x1="9" y1="0" x2="9" y2="25" /&gt;
  &lt;line x1="9" y1="0" x2="9" y2="25" /&gt;
  &lt;line x1="9" y1="0" x2="9" y2="25" /&gt;
  &lt;line x1="9" y1="0" x2="9" y2="25" /&gt;
  &lt;line x1="9" y1="0" x2="9" y2="25" /&gt;
  &lt;line x1="10" y1="0" x2="10" y2="25" /&gt;
  &lt;line x1="10" y1="0" x2="10" y2="25" /&gt;
  &lt;line x1="10" y1="0" x2="10" y2="25" /&gt;
  &lt;line x1="10" y1="0" x2="10" y2="25" /&gt;
  &lt;line x1="10" y1="0" x2="10" y2="25" /&gt;
  &lt;line x1="11" y1="0" x2="11" y2="25" /&gt;
  &lt;line x1="11" y1="0" x2="11" y2="25" /&gt;
  &lt;line x1="11" y1="0" x2="11" y2="25" /&gt;
  &lt;line x1="11" y1="0" x2="11" y2="25" /&gt;
  &lt;line x1="11" y1="0" x2="11" y2="25" /&gt;
  &lt;line x1="12" y1="0" x2="12" y2="25" /&gt;
  &lt;line x1="12" y1="0" x2="12" y2="25" /&gt;
  &lt;line x1="12" y1="0" x2="12" y2="25" /&gt;
  &lt;line x1="12" y1="0" x2="12" y2="25" /&gt;
  &lt;line x1="13" y1="0" x2="13" y2="25" /&gt;
  &lt;line x1="13" y1="0" x2="13" y2="25" /&gt;
  &lt;line x1="13" y1="0" x2="13" y2="25" /&gt;
  &lt;line x1="13" y1="0" x2="13" y2="25" /&gt;
  &lt;line x1="13" y1="0" x2="13" y2="25" /&gt;
  &lt;line x1="14" y1="0" x2="14" y2="25" /&gt;
  &lt;line x1="14" y1="0" x2="14" y2="25" /&gt;
  &lt;line x1="14" y1="0" x2="14" y2="25" /&gt;
  &lt;line x1="14" y1="0" x2="14" y2="25" /&gt;
  &lt;line x1="14" y1="0" x2="14" y2="25" /&gt;
  &lt;line x1="15" y1="0" x2="15" y2="25" /&gt;
  &lt;line x1="15" y1="0" x2="15" y2="25" /&gt;
  &lt;line x1="15" y1="0" x2="15" y2="25" /&gt;
  &lt;line x1="15" y1="0" x2="15" y2="25" /&gt;
  &lt;line x1="15" y1="0" x2="15" y2="25" /&gt;
  &lt;line x1="16" y1="0" x2="16" y2="25" /&gt;
  &lt;line x1="16" y1="0" x2="16" y2="25" /&gt;
  &lt;line x1="16" y1="0" x2="16" y2="25" /&gt;
  &lt;line x1="16" y1="0" x2="16" y2="25" /&gt;
  &lt;line x1="17" y1="0" x2="17" y2="25" /&gt;
  &lt;line x1="17" y1="0" x2="17" y2="25" /&gt;
  &lt;line x1="17" y1="0" x2="17" y2="25" /&gt;
  &lt;line x1="17" y1="0" x2="17" y2="25" /&gt;
  &lt;line x1="17" y1="0" x2="17" y2="25" /&gt;
  &lt;line x1="18" y1="0" x2="18" y2="25" /&gt;
  &lt;line x1="18" y1="0" x2="18" y2="25" /&gt;
  &lt;line x1="18" y1="0" x2="18" y2="25" /&gt;
  &lt;line x1="18" y1="0" x2="18" y2="25" /&gt;
  &lt;line x1="18" y1="0" x2="18" y2="25" /&gt;
  &lt;line x1="19" y1="0" x2="19" y2="25" /&gt;
  &lt;line x1="19" y1="0" x2="19" y2="25" /&gt;
  &lt;line x1="19" y1="0" x2="19" y2="25" /&gt;
  &lt;line x1="19" y1="0" x2="19" y2="25" /&gt;
  &lt;line x1="19" y1="0" x2="19" y2="25" /&gt;
  &lt;line x1="20" y1="0" x2="20" y2="25" /&gt;
  &lt;line x1="20" y1="0" x2="20" y2="25" /&gt;
  &lt;line x1="20" y1="0" x2="20" y2="25" /&gt;
  &lt;line x1="20" y1="0" x2="20" y2="25" /&gt;
  &lt;line x1="21" y1="0" x2="21" y2="25" /&gt;
  &lt;line x1="21" y1="0" x2="21" y2="25" /&gt;
  &lt;line x1="21" y1="0" x2="21" y2="25" /&gt;
  &lt;line x1="21" y1="0" x2="21" y2="25" /&gt;
  &lt;line x1="21" y1="0" x2="21" y2="25" /&gt;
  &lt;line x1="22" y1="0" x2="22" y2="25" /&gt;
  &lt;line x1="22" y1="0" x2="22" y2="25" /&gt;
  &lt;line x1="22" y1="0" x2="22" y2="25" /&gt;
  &lt;line x1="22" y1="0" x2="22" y2="25" /&gt;
  &lt;line x1="22" y1="0" x2="22" y2="25" /&gt;
  &lt;line x1="23" y1="0" x2="23" y2="25" /&gt;
  &lt;line x1="23" y1="0" x2="23" y2="25" /&gt;
  &lt;line x1="23" y1="0" x2="23" y2="25" /&gt;
  &lt;line x1="23" y1="0" x2="23" y2="25" /&gt;
  &lt;line x1="23" y1="0" x2="23" y2="25" /&gt;
  &lt;line x1="24" y1="0" x2="24" y2="25" /&gt;
  &lt;line x1="24" y1="0" x2="24" y2="25" /&gt;
  &lt;line x1="24" y1="0" x2="24" y2="25" /&gt;
  &lt;line x1="24" y1="0" x2="24" y2="25" /&gt;
  &lt;line x1="25" y1="0" x2="25" y2="25" /&gt;
  &lt;line x1="25" y1="0" x2="25" y2="25" /&gt;
  &lt;line x1="25" y1="0" x2="25" y2="25" /&gt;
  &lt;line x1="25" y1="0" x2="25" y2="25" /&gt;
  &lt;line x1="25" y1="0" x2="25" y2="25" /&gt;
  &lt;line x1="26" y1="0" x2="26" y2="25" /&gt;
  &lt;line x1="26" y1="0" x2="26" y2="25" /&gt;
  &lt;line x1="26" y1="0" x2="26" y2="25" /&gt;
  &lt;line x1="26" y1="0" x2="26" y2="25" /&gt;
  &lt;line x1="26" y1="0" x2="26" y2="25" /&gt;
  &lt;line x1="27" y1="0" x2="27" y2="25" /&gt;
  &lt;line x1="27" y1="0" x2="27" y2="25" /&gt;
  &lt;line x1="27" y1="0" x2="27" y2="25" /&gt;
  &lt;line x1="27" y1="0" x2="27" y2="25" /&gt;
  &lt;line x1="27" y1="0" x2="27" y2="25" /&gt;
  &lt;line x1="28" y1="0" x2="28" y2="25" /&gt;
  &lt;line x1="28" y1="0" x2="28" y2="25" /&gt;
  &lt;line x1="28" y1="0" x2="28" y2="25" /&gt;
  &lt;line x1="28" y1="0" x2="28" y2="25" /&gt;
  &lt;line x1="29" y1="0" x2="29" y2="25" /&gt;
  &lt;line x1="29" y1="0" x2="29" y2="25" /&gt;
  &lt;line x1="29" y1="0" x2="29" y2="25" /&gt;
  &lt;line x1="29" y1="0" x2="29" y2="25" /&gt;
  &lt;line x1="29" y1="0" x2="29" y2="25" /&gt;
  &lt;line x1="30" y1="0" x2="30" y2="25" /&gt;
  &lt;line x1="30" y1="0" x2="30" y2="25" /&gt;
  &lt;line x1="30" y1="0" x2="30" y2="25" /&gt;
  &lt;line x1="30" y1="0" x2="30" y2="25" /&gt;
  &lt;line x1="30" y1="0" x2="30" y2="25" /&gt;
  &lt;line x1="31" y1="0" x2="31" y2="25" /&gt;
  &lt;line x1="31" y1="0" x2="31" y2="25" /&gt;
  &lt;line x1="31" y1="0" x2="31" y2="25" /&gt;
  &lt;line x1="31" y1="0" x2="31" y2="25" /&gt;
  &lt;line x1="31" y1="0" x2="31" y2="25" /&gt;
  &lt;line x1="32" y1="0" x2="32" y2="25" /&gt;
  &lt;line x1="32" y1="0" x2="32" y2="25" /&gt;
  &lt;line x1="32" y1="0" x2="32" y2="25" /&gt;
  &lt;line x1="32" y1="0" x2="32" y2="25" /&gt;
  &lt;line x1="33" y1="0" x2="33" y2="25" /&gt;
  &lt;line x1="33" y1="0" x2="33" y2="25" /&gt;
  &lt;line x1="33" y1="0" x2="33" y2="25" /&gt;
  &lt;line x1="33" y1="0" x2="33" y2="25" /&gt;
  &lt;line x1="33" y1="0" x2="33" y2="25" /&gt;
  &lt;line x1="34" y1="0" x2="34" y2="25" /&gt;
  &lt;line x1="34" y1="0" x2="34" y2="25" /&gt;
  &lt;line x1="34" y1="0" x2="34" y2="25" /&gt;
  &lt;line x1="34" y1="0" x2="34" y2="25" /&gt;
  &lt;line x1="34" y1="0" x2="34" y2="25" /&gt;
  &lt;line x1="35" y1="0" x2="35" y2="25" /&gt;
  &lt;line x1="35" y1="0" x2="35" y2="25" /&gt;
  &lt;line x1="35" y1="0" x2="35" y2="25" /&gt;
  &lt;line x1="35" y1="0" x2="35" y2="25" /&gt;
  &lt;line x1="35" y1="0" x2="35" y2="25" /&gt;
  &lt;line x1="36" y1="0" x2="36" y2="25" /&gt;
  &lt;line x1="36" y1="0" x2="36" y2="25" /&gt;
  &lt;line x1="36" y1="0" x2="36" y2="25" /&gt;
  &lt;line x1="36" y1="0" x2="36" y2="25" /&gt;
  &lt;line x1="37" y1="0" x2="37" y2="25" /&gt;
  &lt;line x1="37" y1="0" x2="37" y2="25" /&gt;
  &lt;line x1="37" y1="0" x2="37" y2="25" /&gt;
  &lt;line x1="37" y1="0" x2="37" y2="25" /&gt;
  &lt;line x1="37" y1="0" x2="37" y2="25" /&gt;
  &lt;line x1="38" y1="0" x2="38" y2="25" /&gt;
  &lt;line x1="38" y1="0" x2="38" y2="25" /&gt;
  &lt;line x1="38" y1="0" x2="38" y2="25" /&gt;
  &lt;line x1="38" y1="0" x2="38" y2="25" /&gt;
  &lt;line x1="38" y1="0" x2="38" y2="25" /&gt;
  &lt;line x1="39" y1="0" x2="39" y2="25" /&gt;
  &lt;line x1="39" y1="0" x2="39" y2="25" /&gt;
  &lt;line x1="39" y1="0" x2="39" y2="25" /&gt;
  &lt;line x1="39" y1="0" x2="39" y2="25" /&gt;
  &lt;line x1="39" y1="0" x2="39" y2="25" /&gt;
  &lt;line x1="40" y1="0" x2="40" y2="25" /&gt;
  &lt;line x1="40" y1="0" x2="40" y2="25" /&gt;
  &lt;line x1="40" y1="0" x2="40" y2="25" /&gt;
  &lt;line x1="40" y1="0" x2="40" y2="25" /&gt;
  &lt;line x1="41" y1="0" x2="41" y2="25" /&gt;
  &lt;line x1="41" y1="0" x2="41" y2="25" /&gt;
  &lt;line x1="41" y1="0" x2="41" y2="25" /&gt;
  &lt;line x1="41" y1="0" x2="41" y2="25" /&gt;
  &lt;line x1="41" y1="0" x2="41" y2="25" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="0.000000,0.000000 41.887587,0.000000 41.887587,25.412617 0.000000,25.412617" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="20.943793" y="45.412617" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;199&lt;/text&gt;
&lt;text x="61.887587" y="12.706308" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(0,61.887587,12.706308)"&gt;3&lt;/text&gt;&lt;/p&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="111" y1="0" x2="135" y2="24" style="stroke-width:2" /&gt;
  &lt;line x1="111" y1="120" x2="135" y2="144" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="111" y1="0" x2="111" y2="120" style="stroke-width:2" /&gt;
  &lt;line x1="135" y1="24" x2="135" y2="144" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="111.000000,0.000000 135.664918,24.664918 135.664918,144.664918 111.000000,120.000000" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="111" y1="0" x2="201" y2="0" style="stroke-width:2" /&gt;
  &lt;line x1="135" y1="24" x2="225" y2="24" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="111" y1="0" x2="135" y2="24" style="stroke-width:2" /&gt;
  &lt;line x1="201" y1="0" x2="225" y2="24" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="111.000000,0.000000 201.000000,0.000000 225.664918,24.664918 135.664918,24.664918" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Horizontal lines --&gt;
  &lt;line x1="135" y1="24" x2="225" y2="24" style="stroke-width:2" /&gt;
  &lt;line x1="135" y1="144" x2="225" y2="144" style="stroke-width:2" /&gt;
  &lt;!-- Vertical lines --&gt;
  &lt;line x1="135" y1="24" x2="135" y2="144" style="stroke-width:2" /&gt;
  &lt;line x1="225" y1="24" x2="225" y2="144" style="stroke-width:2" /&gt;
  &lt;!-- Colored Rectangle --&gt;
  &lt;polygon points="135.664918,24.664918 225.664918,24.664918 225.664918,144.664918 135.664918,144.664918" style="fill:#ECB172A0;stroke-width:0"/&gt;
  &lt;!-- Text --&gt;
&lt;p&gt;&lt;text x="180.664918" y="164.664918" font-size="1.0rem" font-weight="100" text-anchor="middle" &gt;768&lt;/text&gt;
&lt;text x="245.664918" y="84.664918" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(-90,245.664918,84.664918)"&gt;1024&lt;/text&gt;
&lt;text x="113.332459" y="152.332459" font-size="1.0rem" font-weight="100" text-anchor="middle" transform="rotate(45,113.332459,152.332459)"&gt;201&lt;/text&gt;
&lt;/svg&gt;&lt;/p&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;That’s a 180 GB logical array, composed of around 600 chunks, each of size 300
MB. We can now do normal NumPy like computations on this array using &lt;a class="reference external" href="https://docs.dask.org/en/latest/array.html"&gt;Dask
Array&lt;/a&gt;, but we’ll save that for a
future post.&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="c1"&gt;# array computations would work fine, and would run in low memory&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="c1"&gt;# but we&amp;#39;ll save actual computation for future posts&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;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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/20/load-image-data.md&lt;/span&gt;, line 1056)&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="save-data"&gt;
&lt;h1&gt;Save Data&lt;/h1&gt;
&lt;p&gt;To simplify data loading in the future, we store this in a large chunked
array format like &lt;a class="reference external" href="https://zarr.readthedocs.io/"&gt;Zarr&lt;/a&gt; using the &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-api.html#dask.array.Array.to_zarr"&gt;to_zarr&lt;/a&gt;
method.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;a&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_zarr&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;mydata.zarr&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We may add additional information about the image data as &lt;a class="reference external" href="https://zarr.readthedocs.io/en/stable/tutorial.html#user-attributes"&gt;attributes&lt;/a&gt;. This
both makes things simpler for future users (they can read the full dataset with
a single line using &lt;a class="reference external" href="http://docs.dask.org/en/latest/array-api.html#dask.array.from_zarr"&gt;da.from_zarr&lt;/a&gt;) and much
more performant because Zarr is an &lt;em&gt;analysis ready format&lt;/em&gt; that is efficiently
encoded for computation.&lt;/p&gt;
&lt;p&gt;Zarr uses the &lt;a class="reference external" href="http://blosc.org/"&gt;Blosc&lt;/a&gt; library for compression by default.
For scientific imaging data, we can optionally pass compression options that provide
a good compression ratio to speed tradeoff and optimize compression
performance.&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;numcodecs&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;Blosc&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;to_zarr&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;mydata.zarr&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;compressor&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;Blosc&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cname&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;zstd&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;clevel&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="n"&gt;shuffle&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;Blosc&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;BITSHUFFLE&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/20/load-image-data.md&lt;/span&gt;, line 1082)&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;The workload above is generic and straightforward. It works well in simple
cases and also extends well to more complex cases, providing you’re willing to
write some for-loops and parsing code around your custom logic. It works on a
single small-scale laptop as well as a large HPC or Cloud cluster. If you have
a function that turns a filename into a NumPy array, you can generate large
lazy Dask array using that function, &lt;a class="reference external" href="https://docs.dask.org/en/latest/delayed.html"&gt;Dask
Delayed&lt;/a&gt; and &lt;a class="reference external" href="https://docs.dask.org/en/latest/array.html"&gt;Dask
Array&lt;/a&gt;.&lt;/p&gt;
&lt;section id="dask-image"&gt;
&lt;h2&gt;Dask Image&lt;/h2&gt;
&lt;p&gt;However, we can make things a bit easier for users if we specialize a bit. For
example the &lt;a class="reference external" href="https://image.dask.org/en/latest/"&gt;Dask Image&lt;/a&gt; library has a
parallel image reader function, which automates much of our work above in the
simple case.&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_image&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_image&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;raw/*.tif&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;Similarly libraries like &lt;a class="reference external" href="https://xarray.pydata.org/en/stable/"&gt;Xarray&lt;/a&gt; have
readers for other file formats, like GeoTIFF.&lt;/p&gt;
&lt;p&gt;As domains do more and more work like what we did above they tend to write down
common patterns into domain-specific libraries, which then increases the
accessibility and user base of these tools.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="gpus"&gt;
&lt;h2&gt;GPUs&lt;/h2&gt;
&lt;p&gt;If we have special hardware lying around like a few GPUs, we can move the data
over to it and perform computations with a library like CuPy, which mimics
NumPy very closely. Thus benefiting from the same operations listed above, but
with the added performance of GPUs behind 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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cupy&lt;/span&gt;&lt;span class="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;cp&lt;/span&gt;
&lt;span class="n"&gt;a_gpu&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;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cp&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;asarray&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="computation"&gt;
&lt;h2&gt;Computation&lt;/h2&gt;
&lt;p&gt;Finally, in future blogposts we plan to talk about how to compute on our large
Dask arrays using common image-processing workloads like overlapping stencil
functions, segmentation and deconvolution, and integrating with other libraries
like ITK.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/06/20/load-image-data/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="dask-image" label="dask-image"/>
    <category term="python" label="python"/>
    <category term="scikit-image" label="scikit-image"/>
    <category term="scipy" label="scipy"/>
    <published>2019-06-20T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/06/19/python-gpus-status-update/</id>
    <title>Python and GPUs: A Status Update</title>
    <updated>2019-06-19T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This blogpost was delivered in talk form at the recent &lt;a class="reference external" href="https://pasc19.pasc-conference.org/"&gt;PASC
2019&lt;/a&gt; conference.
&lt;a class="reference external" href="https://docs.google.com/presentation/d/e/2PACX-1vSajAH6FzgQH4OwOJD5y-t9mjF9tTKEeljguEsfcjavp18pL4LkpABy4lW2uMykIUvP2dC-1AmhCq6l/pub?start=false&amp;amp;amp;loop=false&amp;amp;amp;delayms=60000"&gt;Slides for that talk are
here&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/19/python-gpus-status-update.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="executive-summary"&gt;

&lt;p&gt;We’re improving the state of scalable GPU computing in Python.&lt;/p&gt;
&lt;p&gt;This post lays out the current status, and describes future work.
It also summarizes and links to several other more blogposts from recent months that drill down into different topics for the interested reader.&lt;/p&gt;
&lt;p&gt;Broadly we cover briefly the following categories:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Python libraries written in CUDA like CuPy and RAPIDS&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Python-CUDA compilers, specifically Numba&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scaling these libraries out with Dask&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Network communication with UCX&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Packaging with Conda&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/19/python-gpus-status-update.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="performance-of-gpu-accelerated-python-libraries"&gt;
&lt;h1&gt;Performance of GPU accelerated Python Libraries&lt;/h1&gt;
&lt;p&gt;Probably the easiest way for a Python programmer to get access to GPU
performance is to use a GPU-accelerated Python library. These provide a set of
common operations that are well tuned and integrate well together.&lt;/p&gt;
&lt;p&gt;Many users know libraries for deep learning like PyTorch and TensorFlow, but
there are several other for more general purpose computing. These tend to copy
the APIs of popular Python projects:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Numpy on the GPU: &lt;a class="reference external" href="https://cupy.chainer.org/"&gt;CuPy&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Numpy on the GPU (again): &lt;a class="reference external" href="https://github.com/google/jax"&gt;Jax&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Pandas on the GPU: &lt;a class="reference external" href="https://docs.rapids.ai/api/cudf/nightly/"&gt;RAPIDS cuDF&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scikit-Learn on the GPU: &lt;a class="reference external" href="https://docs.rapids.ai/api/cuml/nightly/"&gt;RAPIDS cuML&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;These libraries build GPU accelerated variants of popular Python
libraries like NumPy, Pandas, and Scikit-Learn. In order to better understand
the relative performance differences
&lt;a class="reference external" href="https://github.com/pentschev"&gt;Peter Entschev&lt;/a&gt; recently put together a
&lt;a class="reference external" href="https://github.com/pentschev/pybench"&gt;benchmark suite&lt;/a&gt; to help with comparisons.
He has produced the following image showing the relative speedup between GPU
and CPU:&lt;/p&gt;
&lt;style&gt;
.vega-actions a {
    margin-right: 12px;
    color: #757575;
    font-weight: normal;
    font-size: 13px;
}
.error {
    color: red;
}
&lt;/style&gt;
&lt;script type="text/javascript" src="https://cdn.jsdelivr.net/npm//vega@5"&gt;&lt;/script&gt;
&lt;script type="text/javascript" src="https://cdn.jsdelivr.net/npm//vega-lite@3.3.0"&gt;&lt;/script&gt;
&lt;script type="text/javascript" src="https://cdn.jsdelivr.net/npm//vega-embed@4"&gt;&lt;/script&gt;
&lt;div id="vis"&gt;&lt;/div&gt;
&lt;p&gt;There are lots of interesting results there.
Peter goes into more depth in this in &lt;a class="reference external" href="https://blog.dask.org/2019/06/27/single-gpu-cupy-benchmarks"&gt;his blogpost&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;More broadly though, we see that there is variability in performance.
Our mental model for what is fast and slow on the CPU doesn’t neccessarily
carry over to the GPU. Fortunately though, due consistent APIs, users that are
familiar with Python can easily experiment with GPU acceleration without
learning CUDA.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/19/python-gpus-status-update.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="numba-compiling-python-to-cuda"&gt;
&lt;h1&gt;Numba: Compiling Python to CUDA&lt;/h1&gt;
&lt;p&gt;&lt;em&gt;See also this &lt;a class="reference external" href="https://blog.dask.org/2019/04/09/numba-stencil"&gt;recent blogpost about Numba
stencils&lt;/a&gt; and the attached &lt;a class="reference external" href="https://gist.github.com/mrocklin/9272bf84a8faffdbbe2cd44b4bc4ce3c"&gt;GPU
notebook&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;The built-in operations in GPU libraries like CuPy and RAPIDS cover most common
operations. However, in real-world settings we often find messy situations
that require writing a little bit of custom code. Switching down to C/C++/CUDA
in these cases can be challenging, especially for users that are primarily
Python developers. This is where Numba can come in.&lt;/p&gt;
&lt;p&gt;Python has this same problem on the CPU as well. Users often couldn’t be
bothered to learn C/C++ to write fast custom code. To address this there are
tools like Cython or Numba, which let Python programmers write fast numeric
code without learning much beyond the Python language.&lt;/p&gt;
&lt;p&gt;For example, Numba accelerates the for-loop style code below about 500x on the
CPU, from slow Python speeds up to fast C/Fortran speeds.&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;numba&lt;/span&gt;  &lt;span class="c1"&gt;# We added these two lines for a 500x speedup&lt;/span&gt;

&lt;span class="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;jit&lt;/span&gt;    &lt;span class="c1"&gt;# We added these two lines for a 500x speedup&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;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="n"&gt;total&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&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="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="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;total&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="n"&gt;i&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;total&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The ability to drop down to low-level performant code without context switching
out of Python is useful, particularly if you don’t already know C/C++ or
have a compiler chain set up for you (which is the case for most Python users
today).&lt;/p&gt;
&lt;p&gt;This benefit is even more pronounced on the GPU. While many Python programmers
know a little bit of C, very few of them know CUDA. Even if they did, they
would probably have difficulty in setting up the compiler tools and development
environment.&lt;/p&gt;
&lt;p&gt;Enter &lt;a class="reference external" href="https://numba.pydata.org/numba-doc/dev/cuda/index.html"&gt;numba.cuda.jit&lt;/a&gt;
Numba’s backend for CUDA. Numba.cuda.jit allows Python users to author,
compile, and run CUDA code, written in Python, interactively without leaving a
Python session. Here is an image of writing a stencil computation that
smoothes a 2d-image all from within a Jupyter Notebook:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/numba.cuda.jit.png"
     width="100%"
     alt="Numba.cuda.jit in a Jupyter Notebook"&gt;&lt;/p&gt;
&lt;p&gt;Here is a simplified comparison of Numba CPU/GPU code to compare programming
style..
The GPU code gets a 200x speed improvement over a single CPU core.&lt;/p&gt;
&lt;section id="cpu-600-ms"&gt;
&lt;h2&gt;CPU – 600 ms&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="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;jit&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;_smooth&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;out&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;empty_like&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="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;1&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;shape&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mi"&gt;1&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;j&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;1&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;shape&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="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="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&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="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&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;j&lt;/span&gt; &lt;span class="o"&gt;+&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="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;j&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="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&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;j&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt;  &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&lt;/span&gt; &lt;span class="o"&gt;+&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt;  &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&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="n"&gt;i&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;j&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt;  &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&lt;/span&gt; &lt;span class="o"&gt;+&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt;  &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&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="n"&gt;i&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;j&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="mi"&gt;9&lt;/span&gt;

    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;out&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or if we use the fancy numba.stencil decorator …&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;stencil&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;_smooth&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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="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="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;x&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="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;1&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;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="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;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="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;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="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;1&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="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;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;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;1&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="mi"&gt;9&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="gpu-3-ms"&gt;
&lt;h2&gt;GPU – 3 ms&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="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cuda&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;jit&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;smooth_gpu&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;out&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="n"&gt;j&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;cuda&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;grid&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;n&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;=&lt;/span&gt; &lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="n"&gt;n&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt; &lt;span class="ow"&gt;and&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;=&lt;/span&gt; &lt;span class="n"&gt;j&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="n"&gt;m&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;out&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="n"&gt;j&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&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="n"&gt;i&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;j&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;x&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="n"&gt;j&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;x&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="n"&gt;j&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="n"&gt;i&lt;/span&gt;    &lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&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="n"&gt;i&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;j&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="mi"&gt;9&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Numba.cuda.jit has been out in the wild for years.
It’s accessible, mature, and fun to play with.
If you have a machine with a GPU in it and some curiosity
then we strongly recommend that you try it out.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;conda&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;numba&lt;/span&gt;
&lt;span class="c1"&gt;# or&lt;/span&gt;
&lt;span class="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;numba&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;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numba.cuda&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/19/python-gpus-status-update.md&lt;/span&gt;, line 186)&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="scaling-with-dask"&gt;
&lt;h1&gt;Scaling with Dask&lt;/h1&gt;
&lt;p&gt;As mentioned in previous blogposts
(
&lt;a class="reference external" href="https://blog.dask.org/2019/01/03/dask-array-gpus-first-steps"&gt;1&lt;/a&gt;,
&lt;a class="reference external" href="https://blog.dask.org/2019/01/13/dask-cudf-first-steps"&gt;2&lt;/a&gt;,
&lt;a class="reference external" href="https://blog.dask.org/2019/03/04/building-gpu-groupbys"&gt;3&lt;/a&gt;,
&lt;a class="reference external" href="https://blog.dask.org/2019/03/18/dask-nep18"&gt;4&lt;/a&gt;
)
we’ve been generalizing &lt;a class="reference external" href="https://dask.org"&gt;Dask&lt;/a&gt;, to operate not just with
Numpy arrays and Pandas dataframes, but with anything that looks enough like
Numpy (like &lt;a class="reference external" href="https://cupy.chainer.org/"&gt;CuPy&lt;/a&gt; or
&lt;a class="reference external" href="https://sparse.pydata.org/en/latest/"&gt;Sparse&lt;/a&gt; or
&lt;a class="reference external" href="https://github.com/google/jax"&gt;Jax&lt;/a&gt;) or enough like Pandas (like &lt;a class="reference external" href="https://docs.rapids.ai/api/cudf/nightly/"&gt;RAPIDS
cuDF&lt;/a&gt;)
to scale those libraries out too. This is working out well. Here is a brief
video showing Dask array computing an SVD in parallel, and seeing what happens
when we swap out the Numpy library for CuPy.&lt;/p&gt;
&lt;iframe width="560"
        height="315"
        src="https://www.youtube.com/embed/QyyxpzNPuIE?start=1046"
        frameborder="0"
        allow="accelerometer; autoplay; encrypted-media; gyroscope; picture-in-picture"
        allowfullscreen&gt;&lt;/iframe&gt;
&lt;p&gt;We see that there is about a 10x speed improvement on the computation. Most
importantly, we were able to switch between a CPU implementation and a GPU
implementation with a small one-line change, but continue using the
sophisticated algorithms with Dask Array, like it’s parallel SVD
implementation.&lt;/p&gt;
&lt;p&gt;We also saw a relative slowdown in communication. In general almost all
non-trivial Dask + GPU work today is becoming communication-bound. We’ve
gotten fast enough at computation that the relative importance of communication
has grown significantly. We’re working to resolve this with our next topic,
UCX.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/19/python-gpus-status-update.md&lt;/span&gt;, line 224)&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="communication-with-ucx"&gt;
&lt;h1&gt;Communication with UCX&lt;/h1&gt;
&lt;p&gt;&lt;em&gt;See &lt;a class="reference external" href="https://developer.download.nvidia.com/video/gputechconf/gtc/2019/video/S9679/s9679-ucx-python-a-flexible-communication-library-for-python-applications.mp4"&gt;this talk&lt;/a&gt; by &lt;a class="reference external" href="https://github.com/Akshay-Venkatesh"&gt;Akshay
Venkatesh&lt;/a&gt; or view &lt;a class="reference external" href="https://www.slideshare.net/MatthewRocklin/ucxpython-a-flexible-communication-library-for-python-applications"&gt;the
slides&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Also see &lt;a class="reference external" href="https://blog.dask.org/2019/06/09/ucx-dgx"&gt;this recent blogpost about UCX and
Dask&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;We’ve been integrating the &lt;a class="reference external" href="https://openucx.org"&gt;OpenUCX&lt;/a&gt; library into Python
with &lt;a class="reference external" href="https://github.com/rapidsai/ucx-py"&gt;UCX-Py&lt;/a&gt;. UCX provides uniform access
to transports like TCP, InfiniBand, shared memory, and NVLink. UCX-Py is the
first time that access to many of these transports has been easily accessible
from the Python language.&lt;/p&gt;
&lt;p&gt;Using UCX and Dask together we’re able to get significant speedups. Here is a
trace of the SVD computation from before both before and after adding UCX:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Before UCX&lt;/strong&gt;:&lt;/p&gt;
&lt;iframe src="https://matthewrocklin.com/raw-host/task_stream_lcc_dgx16.html" width="100%" height="200"&gt;&lt;/iframe&gt;
&lt;p&gt;&lt;strong&gt;After UCX&lt;/strong&gt;:&lt;/p&gt;
&lt;iframe src="https://matthewrocklin.com/raw-host/task_stream_dgx_dgx16.html" width="100%" height="200"&gt;&lt;/iframe&gt;
&lt;p&gt;There is still a great deal to do here though (the blogpost linked above has
several items in the Future Work section).&lt;/p&gt;
&lt;p&gt;People can try out UCX and UCX-Py with highly experimental conda packages:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;conda&lt;/span&gt; &lt;span class="n"&gt;create&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;n&lt;/span&gt; &lt;span class="n"&gt;ucx&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt; &lt;span class="n"&gt;conda&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;forge&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt; &lt;span class="n"&gt;jakirkham&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;label&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;ucx&lt;/span&gt; &lt;span class="n"&gt;cudatoolkit&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;9.2&lt;/span&gt; &lt;span class="n"&gt;ucx&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;proc&lt;/span&gt;&lt;span class="o"&gt;=*=&lt;/span&gt;&lt;span class="n"&gt;gpu&lt;/span&gt; &lt;span class="n"&gt;ucx&lt;/span&gt; &lt;span class="n"&gt;ucx&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;py&lt;/span&gt; &lt;span class="n"&gt;python&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;3.7&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We hope that this work will also affect non-GPU users on HPC systems with
Infiniband, or even users on consumer hardware due to the easy access to shared
memory communication.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/19/python-gpus-status-update.md&lt;/span&gt;, line 263)&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="packaging"&gt;
&lt;h1&gt;Packaging&lt;/h1&gt;
&lt;p&gt;In an &lt;a class="reference external" href="https://matthewrocklin.com/blog/work/2018/12/17/gpu-python-challenges"&gt;earlier blogpost&lt;/a&gt;
we discussed the challenges around installing the wrong versions of CUDA
enabled packages that don’t match the CUDA driver installed on the system.
Fortunately due to recent work from &lt;a class="reference external" href="https://github.com/seibert"&gt;Stan Seibert&lt;/a&gt;
and &lt;a class="reference external" href="https://github.com/msarahan"&gt;Michael Sarahan&lt;/a&gt; at Anaconda, Conda 4.7 now
has a special &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cuda&lt;/span&gt;&lt;/code&gt; meta-package that is set to the version of the installed
driver. This should make it much easier for users in the future to install the
correct package.&lt;/p&gt;
&lt;p&gt;Conda 4.7 was just releasead, and comes with many new features other than the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cuda&lt;/span&gt;&lt;/code&gt; meta-package. You can read more about it &lt;a class="reference external" href="https://www.anaconda.com/how-we-made-conda-faster-4-7/"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;conda&lt;/span&gt; &lt;span class="n"&gt;update&lt;/span&gt; &lt;span class="n"&gt;conda&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;There is still plenty of work to do in the packaging space today.
Everyone who builds conda packages does it their own way,
resulting in headache and heterogeneity.
This is largely due to not having centralized infrastructure
to build and test CUDA enabled packages,
like we have in &lt;a class="reference external" href="https://conda-forge.org"&gt;Conda Forge&lt;/a&gt;.
Fortunately, the Conda Forge community is working together with Anaconda and
NVIDIA to help resolve this, though that will likely take some 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/2019/06/19/python-gpus-status-update.md&lt;/span&gt;, line 290)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="summary"&gt;
&lt;h1&gt;Summary&lt;/h1&gt;
&lt;p&gt;This post gave an update of the status of some of the efforts behind GPU
computing in Python. It also provided a variety of links for future reading.
We include them below if you would like to learn more:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://docs.google.com/presentation/d/e/2PACX-1vSajAH6FzgQH4OwOJD5y-t9mjF9tTKEeljguEsfcjavp18pL4LkpABy4lW2uMykIUvP2dC-1AmhCq6l/pub?start=false&amp;amp;amp;loop=false&amp;amp;amp;delayms=60000"&gt;Slides&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Numpy on the GPU: &lt;a class="reference external" href="https://cupy.chainer.org/"&gt;CuPy&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Numpy on the GPU (again): &lt;a class="reference external" href="https://github.com/google/jax"&gt;Jax&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Pandas on the GPU: &lt;a class="reference external" href="https://docs.rapids.ai/api/cudf/nightly/"&gt;RAPIDS cuDF&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scikit-Learn on the GPU: &lt;a class="reference external" href="https://docs.rapids.ai/api/cuml/nightly/"&gt;RAPIDS cuML&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/pentschev/pybench"&gt;Benchmark suite&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/9272bf84a8faffdbbe2cd44b4bc4ce3c"&gt;Numba CUDA JIT notebook&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://developer.download.nvidia.com/video/gputechconf/gtc/2019/video/S9679/s9679-ucx-python-a-flexible-communication-library-for-python-applications.mp4"&gt;A talk on UCX&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org/2019/06/09/ucx-dgx"&gt;A blogpost on UCX and Dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://www.anaconda.com/how-we-made-conda-faster-4-7/"&gt;Conda 4.7&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;script&gt;
  var spec = {
  "config": {
    "view": {
      "width": 300,
      "height": 200
    },
    "mark": {
      "tooltip": null
    },
    "axis": {
      "grid": false,
      "labelColor": "#666666",
      "labelFontSize": 16,
      "titleColor": "#666666",
      "titleFontSize": 20
    },
    "axisX": {
      "labelAngle": -30,
      "labelColor": "#666666",
      "labelFontSize": 0,
      "titleColor": "#666666",
      "titleFontSize": 0
    },
    "header": {
      "labelAngle": -20,
      "labelColor": "#666666",
      "labelFontSize": 16,
      "titleColor": "#666666",
      "titleFontSize": 20
    },
    "legend": {
      "fillColor": "#fefefe",
      "labelColor": "#666666",
      "labelFontSize": 18,
      "padding": 10,
      "strokeColor": "gray",
      "titleColor": "#666666",
      "titleFontSize": 18
    }
  },
  "data": {
    "name": "data-4957f64f65957150f8029f7df2e6936f"
  },
  "facet": {
    "column": {
      "type": "nominal",
      "field": "operation",
      "sort": {
        "field": "speedup",
        "op": "sum",
        "order": "descending"
      },
      "title": "Operation"
    }
  },
  "spec": {
    "layer": [
      {
        "mark": {
          "type": "bar",
          "fontSize": 18,
          "opacity": 1.0
        },
        "encoding": {
          "color": {
            "type": "nominal",
            "field": "size",
            "scale": {
              "domain": [
                "800MB",
                "8MB"
              ],
              "range": [
                "#7306ff",
                "#36c9dd"
              ]
            },
            "title": "Array Size"
          },
          "x": {
            "type": "nominal",
            "field": "size"
          },
          "y": {
            "type": "quantitative",
            "axis": {
              "title": "GPU Speedup Over CPU"
            },
            "field": "speedup",
            "scale": {
              "domain": [
                0,
                1000
              ],
              "type": "symlog"
            },
            "stack": null
          }
        },
        "height": 300,
        "width": 50
      },
      {
        "layer": [
          {
            "mark": {
              "type": "text",
              "dy": -5
            },
            "encoding": {
              "color": {
                "type": "nominal",
                "field": "size",
                "scale": {
                  "domain": [
                    "800MB",
                    "8MB"
                  ],
                  "range": [
                    "#7306ff",
                    "#36c9dd"
                  ]
                },
                "title": "Array Size"
              },
              "text": {
                "type": "quantitative",
                "field": "speedup"
              },
              "x": {
                "type": "nominal",
                "field": "size"
              },
              "y": {
                "type": "quantitative",
                "axis": {
                  "title": "GPU Speedup Over CPU"
                },
                "field": "speedup",
                "scale": {
                  "domain": [
                    0,
                    1000
                  ],
                  "type": "symlog"
                },
                "stack": null
              }
            },
            "height": 300,
            "width": 50
          },
          {
            "mark": {
              "type": "text",
              "dy": 7
            },
            "encoding": {
              "color": {
                "type": "nominal",
                "field": "size",
                "scale": {
                  "domain": [
                    "800MB",
                    "8MB"
                  ],
                  "range": [
                    "#7306ff",
                    "#36c9dd"
                  ]
                },
                "title": "Array Size"
              },
              "text": {
                "type": "quantitative",
                "field": "speedup"
              },
              "x": {
                "type": "nominal",
                "field": "size"
              },
              "y": {
                "type": "quantitative",
                "axis": {
                  "title": "GPU Speedup Over CPU"
                },
                "field": "speedup",
                "scale": {
                  "domain": [
                    0,
                    1000
                  ],

                  "type": "symlog"
                },
                "stack": null
              }
            },
            "height": 300,
            "width": 50
          }
        ]
      }
    ]
  },
  "$schema": "https://vega.github.io/schema/vega-lite/v3.3.0.json",
  "datasets": {
    "data-4957f64f65957150f8029f7df2e6936f": [
      {
        "operation": "FFT",
        "speedup": 5.3,
        "shape0": 1000,
        "shape1": 1000,
        "shape": "1000x1000",
        "size": "8MB"
      },
      {
        "operation": "FFT",
        "speedup": 210.0,
        "shape0": 10000,
        "shape1": 10000,
        "shape": "10000x10000",
        "size": "800MB"
      },
      {
        "operation": "Sum",
        "speedup": 8.3,
        "shape0": 1000,
        "shape1": 1000,
        "shape": "1000x1000",
        "size": "8MB"
      },
      {
        "operation": "Sum",
        "speedup": 66.0,
        "shape0": 10000,
        "shape1": 10000,
        "shape": "10000x10000",
        "size": "800MB"
      },
      {
        "operation": "Standard Deviation",
        "speedup": 1.1,
        "shape0": 1000,
        "shape1": 1000,
        "shape": "1000x1000",
        "size": "8MB"
      },
      {
        "operation": "Standard Deviation",
        "speedup": 3.5,
        "shape0": 10000,
        "shape1": 10000,
        "shape": "10000x10000",
        "size": "800MB"
      },
      {
        "operation": "Elementwise",
        "speedup": 150.0,
        "shape0": 1000,
        "shape1": 1000,
        "shape": "1000x1000",
        "size": "8MB"
      },
      {
        "operation": "Elementwise",
        "speedup": 270.0,
        "shape0": 10000,
        "shape1": 10000,
        "shape": "10000x10000",
        "size": "800MB"
      },
      {
        "operation": "Matrix Multiplication",
        "speedup": 18.0,
        "shape0": 1000,
        "shape1": 1000,
        "shape": "1000x1000",
        "size": "8MB"
      },
      {
        "operation": "Matrix Multiplication",
        "speedup": 11.0,
        "shape0": 10000,
        "shape1": 10000,
        "shape": "10000x10000",
        "size": "800MB"
      },
      {
        "operation": "Array Slicing",
        "speedup": 3.6,
        "shape0": 1000,
        "shape1": 1000,
        "shape": "1000x1000",
        "size": "8MB"
      },
      {
        "operation": "Array Slicing",
        "speedup": 190.0,
        "shape0": 10000,
        "shape1": 10000,
        "shape": "10000x10000",
        "size": "800MB"
      },
      {
        "operation": "SVD",
        "speedup": 1.5,
        "shape0": 1000,
        "shape1": 1000,
        "shape": "1000x1000",
        "size": "8MB"
      },
      {
        "operation": "SVD",
        "speedup": 17.0,
        "shape0": 10000,
        "shape1": 1000,
        "shape": "10000x1000",
        "size": "800MB"
      },
      {
        "operation": "Stencil",
        "speedup": 5.1,
        "shape0": 1000,
        "shape1": 1000,
        "shape": "1000x1000",
        "size": "8MB"
      },
      {
        "operation": "Stencil",
        "speedup": 150.0,
        "shape0": 10000,
        "shape1": 10000,
        "shape": "10000x10000",
        "size": "800MB"
      }
    ]
  }
};

  var embedOpt = {"mode": "vega-lite"};

  function showError(el, error){
      el.innerHTML = ('&lt;div class="error" style="color:red;"&gt;'
                      + '&lt;p&gt;JavaScript Error: ' + error.message + '&lt;/p&gt;'
                      + "&lt;p&gt;This usually means there's a typo in your chart specification. "
                      + "See the javascript console for the full traceback.&lt;/p&gt;"
                      + '&lt;/div&gt;');
      throw error;
  }
  vegaEmbed("#vis", spec, embedOpt)
    .catch(error =&gt; showError(el, error));
&lt;/script&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/06/19/python-gpus-status-update/"/>
    <summary>This blogpost was delivered in talk form at the recent PASC
2019 conference.
Slides for that talk are
here.</summary>
    <category term="python" label="python"/>
    <category term="scipy" label="scipy"/>
    <published>2019-06-19T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/06/12/dask-on-hpc/</id>
    <title>Dask on HPC</title>
    <updated>2019-06-12T00:00:00+00:00</updated>
    <author>
      <name>Joe Hamman (NCAR)</name>
    </author>
    <content type="html">&lt;p&gt;We analyze large datasets on HPC systems with Dask, a parallel computing
library that integrates well with the existing Python software ecosystem, and
works comfortably with native HPC hardware.&lt;/p&gt;
&lt;p&gt;This article explains why this approach makes sense for us.
Our motivation is to share our experiences with our colleagues,
and to highlight opportunities for future work.&lt;/p&gt;
&lt;p&gt;We start with six reasons why we use Dask,
followed by seven issues that affect us today.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/12/dask-on-hpc.md&lt;/span&gt;, line 21)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="reasons-why-we-use-dask"&gt;

&lt;section id="ease-of-use"&gt;
&lt;h2&gt;1. Ease of use&lt;/h2&gt;
&lt;p&gt;Dask extends libraries like Numpy, Pandas, and Scikit-learn, which are well-known APIs for scientists and engineers. It also extends simpler APIs for
multi-node multiprocessing. This makes it easy for our existing user base to
get up to speed.&lt;/p&gt;
&lt;p&gt;By abstracting the parallelism away from the user/developer, our analysis tools can be written by computer science non-experts, such as the scientists
themselves, meaning that our software engineers can take on more of a supporting role than a leadership role.
Experience has shown that, with tools like Dask and Jupyter, scientists spend less time coding and more time thinking about science, as they should.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="smooth-hpc-integration"&gt;
&lt;h2&gt;2. Smooth HPC integration&lt;/h2&gt;
&lt;p&gt;With tools like &lt;a class="reference external" href="https://jobqueue.dask.org"&gt;Dask Jobqueue&lt;/a&gt; and &lt;a class="reference external" href="https://mpi.dask.org"&gt;Dask MPI&lt;/a&gt; there is no need of any boilerplate shell scripting code commonly found with job queueing systems.&lt;/p&gt;
&lt;p&gt;Dask interacts natively with our existing job schedulers (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;SLURM&lt;/span&gt;&lt;/code&gt;/&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;SGE&lt;/span&gt;&lt;/code&gt;/&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LSF&lt;/span&gt;&lt;/code&gt;/&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;PBS&lt;/span&gt;&lt;/code&gt;/…)
so there is no additional system to set up and manage between users and IT.
All the infrastructure that we need is already in place.&lt;/p&gt;
&lt;p&gt;Interactive analysis at scale is powerful, and lets
us use our existing infrastructure in new ways.
Auto scaling improves our occupancy and helps with acceptance by HPC operators / owners.
Dask’s resilience against the death of all or part of its workers offers new ways of leveraging job-preemption when co-locating classical HPC workloads with analytics jobs.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="aimed-for-scientific-processing"&gt;
&lt;h2&gt;3. Aimed for Scientific Processing&lt;/h2&gt;
&lt;p&gt;In addition to being integrated with the Scipy and PyData software ecosystems,
Dask is compatible with scientific data formats like HDF5, NetCDF, Parquet, and
so on. This is because Dask works with other libraries within the Python
ecosystem, like Xarray, which already have strong support for scientific data
formats and processing, and with C/C++/Fortran codes, such as is common for Python libraries.&lt;/p&gt;
&lt;p&gt;This native support is one of the major advantages that we’ve seen of Dask over Apache Spark.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="versatility-of-apis"&gt;
&lt;h2&gt;4. Versatility of APIs&lt;/h2&gt;
&lt;p&gt;And yet Dask is not designed for any particular workflow, but instead can
provide infrastructure to cover a variety of different problems within an
institution. Many different kinds of workloads are possible:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;You can easily handle Numpy arrays or Pandas Dataframes at scale, doing some numerical work or data analysis/cleaning,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You can handle any objects collection, like JSON files, text, or log files,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You can express more arbitrary task or job scheduling workloads with Dask Delayed, or real time and reactive processing with Dask Futures.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Dask covers and simplifies many of the wide range of HPC workflows we’ve seen over the years. Many workflows that were previously implemented using job arrays, simplified MPI (e.g. mpi4py) or plain bash scripts seem to be easier for our users with Dask.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="versatility-of-infrastructure"&gt;
&lt;h2&gt;5. Versatility of Infrastructure&lt;/h2&gt;
&lt;p&gt;Dask is compatible with laptops, servers, HPC systems, and cloud computing. The environment can change with very little code adaptation which reduces our burden to rewrite code as we migrate analysis between systems such as from a laptop to a supercomputer, or between a supercomputer and the cloud.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Local machines&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="c1"&gt;# HPC Job Schedulers&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_jobqueue&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;SLURMCluster&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;PBSCluster&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;SGECluster&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;SLURMCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;queue&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;default&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;project&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ABCD1234&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# Hadoop/Spark clusters&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_yarn&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;YARNCluster&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;YarnCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;environment&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;environment.tar.gz&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;worker_vcores&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="c1"&gt;# Cloud/Kubernetes clusters&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_kubernetes&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;KubeCluster&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;KubeCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pod_spec&lt;/span&gt;&lt;span class="o"&gt;=&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Dask is more than just a tool to us; it is a gateway to thinking about a completely different way of providing computing infrastructure to our users. Dask opens up the door to cloud computing technologies (such as elastic scaling and object storage) and makes us rethink what an HPC center should really look like.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="cost-and-collaboration"&gt;
&lt;h2&gt;6. Cost and Collaboration&lt;/h2&gt;
&lt;p&gt;Dask is free and open source, which means we do not have to rebalance our budget and staff to address the new immediate need of data analysis tools.
We don’t have to pay for licenses, and we have the ability to make changes to the code when necessary. The HPC community has good representation among Dask developers. It’s easy for us to participate and our concerns are well understood.&lt;/p&gt;
&lt;!-- WR: Mention quick response to new use cases / demands as another benefit of the collaborative
approach?  And hint towards dask-mpi here? --&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/12/dask-on-hpc.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&gt;
&lt;section id="what-needs-work"&gt;
&lt;h1&gt;What needs work&lt;/h1&gt;
&lt;section id="heterogeneous-resources-handling"&gt;
&lt;h2&gt;1. Heterogeneous resources handling&lt;/h2&gt;
&lt;p&gt;Often we want to include different kinds of HPC nodes in the same deployment.
This includes situations like the following:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Workers with low or high memory,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Workers with GPUs,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Workers from different node pools.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Dask provides some support for this heterogeneity already, but not enough.
We see two major opportunities for improvement.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Tools like Dask-Jobqueue should make it easier to manage multiple worker
pools within the same cluster. Currently the deployment solution assumes
homogeneity.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It should be easier for users to specify which parts of a computation
require different hardware. The solution today works, but requires more
detail from the user than is ideal.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="coarse-grained-diagnostics-and-history"&gt;
&lt;h2&gt;2. Coarse-Grained Diagnostics and History&lt;/h2&gt;
&lt;p&gt;Dask provides a number of profiling tools that deliver real-time diagnostics at the individual task-level, but there is no way today to analyze or profile your Dask application at a coarse-grained level, and no built-in way to track performance over long periods of time.&lt;/p&gt;
&lt;p&gt;Having more tools to analyze bulk performance would be helpful when making design decisions and future architecture choices.&lt;/p&gt;
&lt;p&gt;Having the ability to persist or store history of computations (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;compute()&lt;/span&gt;&lt;/code&gt; calls)
and tasks executed on a scheduler could be really helpful to track problems and potential performance improvements.&lt;/p&gt;
&lt;!-- JJH: One tangible idea here would be a benchmarking suite that helps users make decisions about how to use dask most effectively.
 --&gt;
&lt;/section&gt;
&lt;section id="scheduler-performance-on-large-graphs"&gt;
&lt;h2&gt;3. Scheduler Performance on Large Graphs&lt;/h2&gt;
&lt;p&gt;HPC users want to analyze Petabyte datasets on clusters of thousands of large nodes.&lt;/p&gt;
&lt;p&gt;While Dask can theoretically handle this scale, it does tend to slow down a bit,
reducing the pleasure of interactive large-scale computing. Handling millions of tasks can lead to tens of seconds latency before a computation actually starts. This is perfectly fine for our Dask batch jobs, but tends to make the interactive Jupyter users frustrated.&lt;/p&gt;
&lt;p&gt;Much of this slowdown is due to task-graph construction time and centralized scheduling, both of which can be accelerated through a variety of means. We expect that, with some cleverness, we can increase the scale at which Dask continues to run smoothly by another order of magnitude.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="launch-batch-jobs-with-mpi"&gt;
&lt;h2&gt;4. ~~Launch Batch Jobs with MPI~~&lt;/h2&gt;
&lt;p&gt;&lt;em&gt;This issue was resolved while we prepared this blogpost.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Most Dask workflows today are interactive. People log into a Jupyter notebook, import Dask, and then Dask asks the job scheduler (like SLURM, PBS, …) for resources dynamically. This is great because Dask is able to fit into small gaps in the schedule, release workers when they’re not needed, giving users a pleasant interactive experience while lessening the load on the cluster.&lt;/p&gt;
&lt;p&gt;However not all jobs are interactive. Often scientists want to submit a large job similar to how they submit MPI jobs. They submit a single job script with the necessary resources, walk away, and the resource manager runs that job when those resources become available (which may be many hours from now). While not as novel as the interactive workloads, these workloads are critical to common processes, and important to support.&lt;/p&gt;
&lt;p&gt;This point was raised by Kevin Paul at NCAR during discussion of this blogpost. Between when we started planning and when we released this blogpost Kevin had already solved the problem by prodiving &lt;a class="reference external" href="https://dask-mpi.readthedocs.org"&gt;dask-mpi&lt;/a&gt;, a project that makes it easy to launch Dask using normal &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mpirun&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mpiexec&lt;/span&gt;&lt;/code&gt; commands, making it easy to deploy Dask anywhere that MPI is deployable.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="more-data-formats"&gt;
&lt;h2&gt;5. More Data Formats&lt;/h2&gt;
&lt;p&gt;Dask works well today with bread-and-butter scientific data formats like HDF5, Grib, and NetCDF, as well as common data science formats like CSV, JSON, Parquet, ORC, and so on.&lt;/p&gt;
&lt;p&gt;However, the space of data formats is vast and Dask users find themselves struggling a little, or even solving the data ingestion problem manually for a number of common formats in different domains:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Remote sensing datasets: GeoTIFF, Jpeg2000,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Astronomical data: FITS, VOTable,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;… and so on&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Supporting these isn’t hard (indeed many of us have built our own support for them in Dask), but it would be handy to have a high quality centralized solution.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="link-with-deep-learning"&gt;
&lt;h2&gt;6. Link with Deep Learning&lt;/h2&gt;
&lt;p&gt;Many of our institutions are excited to leverage recent advances in deep learning and integrate powerful tools like Keras, TensorFlow, and PyTorch and powerful hardware like GPUs into our workflows.&lt;/p&gt;
&lt;p&gt;However, we often find that our data and architecture look a bit different from what we find in standard deep learning tutorials. We like using Dask for data ingestion, cleanup, and pre-processing, but would like to establish better practices and smooth tooling to transition from scientific workflows on HPC using Dask to deep learning as efficiently as possible.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;For more information, see &lt;a class="reference external" href="https://github.com/pangeo-data/pangeo/issues/567"&gt;this github
issue&lt;/a&gt; for an example topic.&lt;/em&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="more-calculation-guidelines"&gt;
&lt;h2&gt;7. More calculation guidelines&lt;/h2&gt;
&lt;p&gt;While there are means to analyse and diagnose computations interactively, and
a quite decent set of examples for Dask common calculations, trials and error appear to be the norm with big HPC computation before coming to optimized workflows.&lt;/p&gt;
&lt;p&gt;We should develop more guidelines and strategy on how to perform large scale computation, and we need to foster the community around Dask, which is already done in projects such as Pangeo. Note that these guidelines may be infrastructure dependent.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/06/12/dask-on-hpc/"/>
    <summary>We analyze large datasets on HPC systems with Dask, a parallel computing
library that integrates well with the existing Python software ecosystem, and
works comfortably with native HPC hardware.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2019-06-12T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/06/09/ucx-dgx/</id>
    <title>Experiments in High Performance Networking with UCX and DGX</title>
    <updated>2019-06-09T00:00:00+00:00</updated>
    <author>
      <name>Rick Zamora</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This post is about experimental and rapidly changing software.
Code examples in this post should not be relied upon to work in the future.&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/09/ucx-dgx.md&lt;/span&gt;, line 12)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

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

$ dask-worker ucx://127.0.0.1:8786
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ucx://127.0.0.1:8786&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/09/ucx-dgx.md&lt;/span&gt;, line 95)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="experiment"&gt;
&lt;h1&gt;Experiment&lt;/h1&gt;
&lt;p&gt;We modified our &lt;a class="reference external" href="https://github.com/mrocklin/dask-gpu-benchmarks/blob/master/cupy-svd.ipynb"&gt;SVD with Dask and CuPy
benchmark&lt;/a&gt;
benchmark to use the UCX protocol for inter-process communication and ran it on
half of a DGX machine, using four GPUs. Here is a minimal implementation of the
UCX-enabled code:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cupy&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;wait&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_cuda&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;DGX&lt;/span&gt;

&lt;span class="c1"&gt;# Define DGX cluster and client&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;DGX&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# Create random data&lt;/span&gt;
&lt;span class="n"&gt;rs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;rs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mi"&gt;1000000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

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

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

Legend:

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

&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Nanny&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;SpecCluster&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Scheduler&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;distributed.worker&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;TOTAL_MEMORY&lt;/span&gt;

&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_cuda.local_cuda_cluster&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;cuda_visible_devices&lt;/span&gt;


&lt;span class="k"&gt;class&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nc"&gt;CPUAffinity&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot; A Worker plugin to pin CPU affinity &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="fm"&gt;__init__&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cores&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cores&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;cores&lt;/span&gt;

    &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;setup&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;worker&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="n"&gt;os&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sched_setaffinity&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cores&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;


&lt;span class="n"&gt;affinity&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;  &lt;span class="c1"&gt;# See nvidia-smi topo -m&lt;/span&gt;
    &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
    &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
    &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
    &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
    &lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;79&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
    &lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;79&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
    &lt;span class="mi"&gt;6&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;79&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
    &lt;span class="mi"&gt;7&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;79&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
&lt;span class="p"&gt;}&lt;/span&gt;

&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;DGX&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;interface&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;ib&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;dashboard_address&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;:8787&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;threads_per_worker&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;silence_logs&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;
&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot; A Local Cluster for a DGX 1 machine&lt;/span&gt;

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

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

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

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

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

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

    &lt;span class="n"&gt;scheduler&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;cls&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;Scheduler&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;options&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
            &lt;span class="s2"&gt;&amp;quot;interface&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;interface&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;str&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;//&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
            &lt;span class="s2"&gt;&amp;quot;protocol&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;ucx&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="s2"&gt;&amp;quot;dashboard_address&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;dashboard_address&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="p"&gt;},&lt;/span&gt;
    &lt;span class="p"&gt;}&lt;/span&gt;

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

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

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

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

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

&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;wait&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;distributed.utils&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;format_time&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;format_bytes&lt;/span&gt;

&lt;span class="k"&gt;async&lt;/span&gt; &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;f&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;

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

            &lt;span class="c1"&gt;# Create a simple random array&lt;/span&gt;
            &lt;span class="n"&gt;rs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
            &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;rs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mi"&gt;40000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;40000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;128 MiB&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
            &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;npartitions&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;chunks&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
            &lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

            &lt;span class="c1"&gt;# Add X to its transpose, forcing computation&lt;/span&gt;
            &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;T&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
            &lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

            &lt;span class="c1"&gt;# Collect, aggregate, and print peer-to-peer bandwidths&lt;/span&gt;
            &lt;span class="n"&gt;incoming_logs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;run&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;dask_worker&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;dask_worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;incoming_transfer_log&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
            &lt;span class="n"&gt;bandwidths&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;defaultdict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
            &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;k&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;L&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;incoming_logs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;items&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
                &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;d&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;L&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;d&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;total&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1_000_000&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                        &lt;span class="n"&gt;bandwidths&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;k&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;d&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;who&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;d&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;bandwidth&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
            &lt;span class="n"&gt;bandwidths&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
                &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;w1&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;name&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                    &lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;w2&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;name&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;format_bytes&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;/s&amp;#39;&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;quantile&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;v&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mf"&gt;0.25&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;0.50&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;0.75&lt;/span&gt;&lt;span class="p"&gt;])]&lt;/span&gt;
                &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;w1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;w2&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;bandwidths&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;items&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
            &lt;span class="p"&gt;}&lt;/span&gt;
            &lt;span class="n"&gt;pprint&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;bandwidths&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;


&lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="vm"&gt;__name__&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;__main__&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;asyncio&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_event_loop&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;run_until_complete&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;em&gt;Note: most of this example is just getting back diagnostics, which can be
easily ignored. Also, you can drop the async/await code if you like. I think
that there should probably be more examples in the world using Dask with
async/await syntax, so I decided to leave it in.&lt;/em&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/06/09/ucx-dgx/"/>
    <summary>This post is about experimental and rapidly changing software.
Code examples in this post should not be relied upon to work in the future.</summary>
    <category term="python" label="python"/>
    <category term="scipy" label="scipy"/>
    <published>2019-06-09T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/04/09/numba-stencil/</id>
    <title>Composing Dask Array with Numba Stencils</title>
    <updated>2019-04-09T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</name>
    </author>
    <content type="html">&lt;p&gt;In this post we explore four array computing technologies, and how they
work together to achieve powerful results.&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Numba’s stencil decorator to craft localized compute kernels&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Numba’s Just-In-Time (JIT) compiler for array computing in Python&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask Array for parallelizing array computations across many chunks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;NumPy’s Generalized Universal Functions (gufuncs) to tie everything
together&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In the end we’ll show how a novice developer can write a small amount of Python
to efficiently compute localized computation on large amounts of data. In
particular we’ll write a simple function to smooth images and apply that in
parallel across a large stack of images.&lt;/p&gt;
&lt;p&gt;Here is the full code, we’ll dive into it piece by piece below.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numba&lt;/span&gt;

&lt;span class="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;stencil&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;_smooth&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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="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="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;x&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="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;1&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;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="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;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="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;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="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;1&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="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;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;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;1&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="mi"&gt;9&lt;/span&gt;


&lt;span class="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;guvectorize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="n"&gt;numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;int8&lt;/span&gt;&lt;span class="p"&gt;[:,&lt;/span&gt; &lt;span class="p"&gt;:],&lt;/span&gt; &lt;span class="n"&gt;numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;int8&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;(n, m) -&amp;gt; (n, m)&amp;#39;&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;smooth&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;out&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="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;_smooth&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;# If you want fake data&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="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;1000000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="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="s1"&gt;&amp;#39;auto&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="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="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;int8&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# If you have actual data&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_image&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_image&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/path/to/*.png&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;smooth&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;# dask.array&amp;lt;transpose, shape=(1000000, 1000, 1000), dtype=int8, chunksize=(125, 1000, 1000)&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Note: the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;smooth&lt;/span&gt;&lt;/code&gt; function above is more commonly referred to as the 2D mean filter in the image processing community.&lt;/p&gt;
&lt;p&gt;Now, lets break this down a bit&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/04/09/numba-stencil.md&lt;/span&gt;, line 59)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="numba-stencils"&gt;

&lt;p&gt;&lt;strong&gt;Docs:&lt;/strong&gt;: https://numba.pydata.org/numba-doc/dev/user/stencil.html&lt;/p&gt;
&lt;p&gt;Many array computing functions operate only on a local region of the array.
This is common in image processing, signals processing, simulation, the
solution of differential equations, anomaly detection, time series analysis,
and more. Typically we write code that 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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;_smooth&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;out&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;empty_like&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="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;1&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;shape&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mi"&gt;1&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;j&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;1&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;shape&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="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="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&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="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&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;j&lt;/span&gt; &lt;span class="o"&gt;+&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="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;j&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="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&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;j&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt;  &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&lt;/span&gt; &lt;span class="o"&gt;+&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt;  &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&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="n"&gt;i&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;j&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt;  &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&lt;/span&gt; &lt;span class="o"&gt;+&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt;  &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;j&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="n"&gt;i&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;j&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="mi"&gt;9&lt;/span&gt;

    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;out&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Or something similar. The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;numba.stencil&lt;/span&gt;&lt;/code&gt; decorator makes this a bit easier to
write down. You just write down what happens on every element, and Numba
handles the rest.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;stencil&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;_smooth&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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="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="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;x&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="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;1&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;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="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;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="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;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="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;1&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="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;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;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;1&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="mi"&gt;9&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/04/09/numba-stencil.md&lt;/span&gt;, line 92)&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="numba-jit"&gt;
&lt;h1&gt;Numba JIT&lt;/h1&gt;
&lt;p&gt;&lt;strong&gt;Docs:&lt;/strong&gt; http://numba.pydata.org/&lt;/p&gt;
&lt;p&gt;When we run this function on a NumPy array, we find that it is slow, operating
at Python speeds.&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;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;100&lt;/span&gt;&lt;span class="p"&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;timeit&lt;/span&gt; &lt;span class="n"&gt;_smooth&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;527&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt; &lt;span class="err"&gt;±&lt;/span&gt; &lt;span class="mf"&gt;44.1&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt; &lt;span class="n"&gt;per&lt;/span&gt; &lt;span class="n"&gt;loop&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt; &lt;span class="err"&gt;±&lt;/span&gt; &lt;span class="n"&gt;std&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt; &lt;span class="n"&gt;dev&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt; &lt;span class="n"&gt;of&lt;/span&gt; &lt;span class="mi"&gt;7&lt;/span&gt; &lt;span class="n"&gt;runs&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt; &lt;span class="n"&gt;loop&lt;/span&gt; &lt;span class="n"&gt;each&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;But if we JIT compile this function with Numba, then it runs more quickly.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;njit&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;smooth&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="n"&gt;_smooth&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;timeit&lt;/span&gt; &lt;span class="n"&gt;smooth&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="mf"&gt;70.8&lt;/span&gt; &lt;span class="n"&gt;µs&lt;/span&gt; &lt;span class="err"&gt;±&lt;/span&gt; &lt;span class="mf"&gt;6.38&lt;/span&gt; &lt;span class="n"&gt;µs&lt;/span&gt; &lt;span class="n"&gt;per&lt;/span&gt; &lt;span class="n"&gt;loop&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt; &lt;span class="err"&gt;±&lt;/span&gt; &lt;span class="n"&gt;std&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt; &lt;span class="n"&gt;dev&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt; &lt;span class="n"&gt;of&lt;/span&gt; &lt;span class="mi"&gt;7&lt;/span&gt; &lt;span class="n"&gt;runs&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt; &lt;span class="n"&gt;loop&lt;/span&gt; &lt;span class="n"&gt;each&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;For those counting, that’s over 1000x faster!&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: this function already exists as &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scipy.ndimage.uniform_filter&lt;/span&gt;&lt;/code&gt;, which
operates at the same speed.&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/04/09/numba-stencil.md&lt;/span&gt;, line 121)&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="dask-array"&gt;
&lt;h1&gt;Dask Array&lt;/h1&gt;
&lt;p&gt;&lt;strong&gt;Docs:&lt;/strong&gt; https://docs.dask.org/en/latest/array.html&lt;/p&gt;
&lt;p&gt;In these applications people often have many such arrays and they want to apply
this function over all of them. In principle they could do this with a for
loop.&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;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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;skimage.io&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;glob&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/path/to/*.png&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;img&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&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;out&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;smooth&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imsave&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;fn&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;.png&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;.out.png&amp;#39;&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If they wanted to then do this in parallel they would maybe use the
multiprocessing or concurrent.futures modules. If they wanted to do this
across a cluster then they could rewrite their code with PySpark or some other
system.&lt;/p&gt;
&lt;p&gt;Or, they could use Dask array, which will handle both the pipelining and the
parallelism (single machine or on a cluster) all while still looking mostly
like 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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_image&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_image&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/path/to/*.png&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# a large lazy array of all of our images&lt;/span&gt;
&lt;span class="n"&gt;y&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;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;smooth&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;int8&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;And then because each of the chunks of a Dask array are just NumPy arrays, we
can use the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt; function to apply this function across all of our
images, and then save them out.&lt;/p&gt;
&lt;p&gt;This is fine, but lets go a bit further, and discuss generalized universal
functions from NumPy.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/04/09/numba-stencil.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="generalized-universal-functions"&gt;
&lt;h1&gt;Generalized Universal Functions&lt;/h1&gt;
&lt;p&gt;&lt;strong&gt;Numba Docs:&lt;/strong&gt; https://numba.pydata.org/numba-doc/dev/user/vectorize.html&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;NumPy Docs:&lt;/strong&gt; https://docs.scipy.org/doc/numpy-1.16.0/reference/c-api.generalized-ufuncs.html&lt;/p&gt;
&lt;p&gt;A generalized universal function (gufunc) is a normal function that has been
annotated with typing and dimension information. For example we can redefine
our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;smooth&lt;/span&gt;&lt;/code&gt; function as a gufunc 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="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;guvectorize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="n"&gt;numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;int8&lt;/span&gt;&lt;span class="p"&gt;[:,&lt;/span&gt; &lt;span class="p"&gt;:],&lt;/span&gt; &lt;span class="n"&gt;numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;int8&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;(n, m) -&amp;gt; (n, m)&amp;#39;&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;smooth&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;out&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="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;_smooth&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;p&gt;This function knows that it consumes a 2d array of int8’s and produces a 2d
array of int8’s of the same dimensions.&lt;/p&gt;
&lt;p&gt;This sort of annotation is a small change, but it gives other systems like Dask
enough information to use it intelligently. Rather than call functions like
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_blocks&lt;/span&gt;&lt;/code&gt;, we can just use the function directly, as if our Dask Array was
just a very large 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="c1"&gt;# Before gufuncs&lt;/span&gt;
&lt;span class="n"&gt;y&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;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;smooth&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;int8&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# After gufuncs&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;smooth&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;p&gt;This is nice. If you write library code with gufunc semantics then that code
just works with systems like Dask, without you having to build in explicit
support for parallel computing. This makes the lives of users much easier.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/04/09/numba-stencil.md&lt;/span&gt;, line 200)&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="finished-result"&gt;
&lt;h1&gt;Finished result&lt;/h1&gt;
&lt;p&gt;Lets see the full example one more time.&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;numba&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="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;stencil&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;_smooth&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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="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="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;x&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="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;1&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;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="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;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="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;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="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;1&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="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;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;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;1&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="mi"&gt;9&lt;/span&gt;


&lt;span class="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;guvectorize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="n"&gt;numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;int8&lt;/span&gt;&lt;span class="p"&gt;[:,&lt;/span&gt; &lt;span class="p"&gt;:],&lt;/span&gt; &lt;span class="n"&gt;numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;int8&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;(n, m) -&amp;gt; (n, m)&amp;#39;&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;smooth&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;out&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="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;_smooth&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;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;1000000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="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="s1"&gt;&amp;#39;auto&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="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="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;int8&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;smooth&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;p&gt;This code is decently approachable by novice users. They may not understand
the internal details of gufuncs or Dask arrays or JIT compilation, but they can
probably copy-paste-and-modify the example above to suit their needs.&lt;/p&gt;
&lt;p&gt;The parts that they do want to change are easy to change, like the stencil
computation, and creating an array of their own data.&lt;/p&gt;
&lt;p&gt;This workflow is efficient and scalable, using low-level compiled code and
potentially clusters of thousands of computers.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/04/09/numba-stencil.md&lt;/span&gt;, line 236)&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-could-be-better"&gt;
&lt;h1&gt;What could be better&lt;/h1&gt;
&lt;p&gt;There are a few things that could make this workflow nicer.&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;It would be nice not to have to specify dtypes in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;guvectorize&lt;/span&gt;&lt;/code&gt;, but
instead specialize to types as they arrive.
&lt;a class="reference external" href="https://github.com/numba/numba/issues/2979"&gt;numba/numba #2979&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Support GPU accelerators for the stencil computations using
&lt;a class="reference external" href="https://numba.pydata.org/numba-doc/dev/cuda/index.html"&gt;numba.cuda.jit&lt;/a&gt;.
Stencil computations are obvious candidates for GPU acceleration, and this
is a good accessible point where novice users can specify what they want in
a way that is sufficiently constrained for automated systems to rewrite it
as CUDA somewhat easily.
&lt;a class="reference external" href="https://github.com/numba/numba/issues/3915"&gt;numba/numba 3915&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It would have been nicer to be able to apply the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;&amp;#64;guvectorize&lt;/span&gt;&lt;/code&gt; decorator
directly on top of the stencil function like this.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;guvectorize&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="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;stencil&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;average&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="o"&gt;...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Rather than have two functions.
&lt;a class="reference external" href="https://github.com/numba/numba/issues/3914"&gt;numba/numba #3914&lt;/a&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You may have noticed that our guvectorize function had to assign its result into an
out parameter.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;guvectorize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="n"&gt;numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;int8&lt;/span&gt;&lt;span class="p"&gt;[:,&lt;/span&gt; &lt;span class="p"&gt;:],&lt;/span&gt; &lt;span class="n"&gt;numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;int8&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;(n, m) -&amp;gt; (n, m)&amp;#39;&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;smooth&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;out&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="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;_smooth&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;p&gt;It would have been nicer, perhaps, to just return the output&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;smooth&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="n"&gt;_smooth&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;p&gt;&lt;a class="reference external" href="https://github.com/numba/numba/issues/3916"&gt;numba/numba #3916&lt;/a&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The dask-image library could use a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;imsave&lt;/span&gt;&lt;/code&gt; function&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-image/issues/110"&gt;dask/dask-image #110&lt;/a&gt;&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/2019/04/09/numba-stencil.md&lt;/span&gt;, line 290)&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="aspirational-result"&gt;
&lt;h1&gt;Aspirational Result&lt;/h1&gt;
&lt;p&gt;With all of these, we might then be able to write the code 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="c1"&gt;# This is aspirational&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;numba&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_image&lt;/span&gt;

&lt;span class="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;guvectorize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="n"&gt;numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;int8&lt;/span&gt;&lt;span class="p"&gt;[:,&lt;/span&gt; &lt;span class="p"&gt;:],&lt;/span&gt; &lt;span class="n"&gt;numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;int8&lt;/span&gt;&lt;span class="p"&gt;[:,&lt;/span&gt; &lt;span class="p"&gt;:])],&lt;/span&gt;
    &lt;span class="n"&gt;signature&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;(n, m) -&amp;gt; (n, m)&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;target&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;gpu&amp;#39;&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;stencil&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;smooth&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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="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="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;x&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="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;1&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;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="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;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="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;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="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;1&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="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;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;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;1&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="mi"&gt;9&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_image&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/path/to/*.png&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;smooth&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;dask_image&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imsave&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;/path/to/out/*.png&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/04/09/numba-stencil.md&lt;/span&gt;, line 316)&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="update-now-with-gpus"&gt;
&lt;h1&gt;Update: Now with GPUs!&lt;/h1&gt;
&lt;p&gt;After writing this blogpost I did a small update where I used
&lt;a class="reference external" href="https://numba.pydata.org/numba-doc/dev/cuda/index.html"&gt;numba.cuda.jit&lt;/a&gt;
to implement the same smooth function on a GPU to achieve a 200x speedup with
only a modest increase to code complexity.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/9272bf84a8faffdbbe2cd44b4bc4ce3c"&gt;That notebook is here&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/04/09/numba-stencil/"/>
    <summary>In this post we explore four array computing technologies, and how they
work together to achieve powerful results.</summary>
    <category term="dask" label="dask"/>
    <category term="numba" label="numba"/>
    <published>2019-04-09T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/03/27/dask-cuml/</id>
    <title>cuML and Dask hyperparameter optimization</title>
    <updated>2019-03-27T00:00:00+00:00</updated>
    <author>
      <name>Benjamin Zaitlen</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/03/27/dask-cuml.md&lt;/span&gt;, line 10)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="setup"&gt;

&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;DGX-1 Workstation&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Host Memory: 512 GB&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;GPU Tesla V100 x 8&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;cudf 0.6&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;cuml 0.6&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;dask 1.1.4&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/quasiben/a96ce952b7eb54356f7f8390319473e4"&gt;Jupyter notebook&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;strong&gt;TLDR; Hyper-parameter Optimization is functional but slow with cuML&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/2019/03/27/dask-cuml.md&lt;/span&gt;, line 22)&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="cuml-and-dask-hyper-parameter-optimization"&gt;
&lt;h1&gt;cuML and Dask Hyper-parameter Optimization&lt;/h1&gt;
&lt;p&gt;cuML is an open source GPU accelerated machine learning library primarily
developed at NVIDIA which mirrors the &lt;a class="reference external" href="https://scikit-learn.org/"&gt;Scikit-Learn API&lt;/a&gt;.
The current suite of algorithms includes GLMs, Kalman Filtering, clustering,
and dimensionality reduction. Many of these machine learning algorithms use
hyper-parameters. These are parameters used during the model training process
but are not “learned” during the training. Often these parameters are
coefficients or penalty thresholds and finding the “best” hyper parameter can be
computationally costly. In the PyData community, we often reach to Scikit-Learn’s
&lt;a class="reference external" href="https://scikit-learn.org/stable/modules/generated/sklearn.model_selection.GridSearchCV.html"&gt;GridSearchCV&lt;/a&gt;
or
&lt;a class="reference external" href="https://scikit-learn.org/stable/modules/generated/sklearn.model_selection.RandomizedSearchCV.html#sklearn.model_selection.RandomizedSearchCV"&gt;RandomizedSearchCV&lt;/a&gt;
for easy definition of the search space for hyper-parameters – this is called hyper-parameter
optimization. Within the Dask community, &lt;a class="reference external" href="https://dask-ml.readthedocs.io/en/latest/"&gt;Dask-ML&lt;/a&gt; has incrementally improved the efficiency of hyper-parameter optimization by leveraging both Scikit-Learn and Dask to use multi-core and
distributed schedulers: &lt;a class="reference external" href="https://dask-ml.readthedocs.io/en/latest/hyper-parameter-search.html"&gt;Grid and RandomizedSearch with DaskML&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;With the newly created drop-in replacement for Scikit-Learn, cuML, we experimented with Dask’s GridSearchCV. In the upcoming 0.6 release of cuML, the estimators are serializable and are functional within the Scikit-Learn/dask-ml framework, but slow compared with Scikit-Learn estimators. And while speeds are slow now, we know how to boost performance, have filed several issues, and hope to show performance gains in future releases.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;All code and timing measurements can be found in this &lt;a class="reference external" href="https://gist.github.com/quasiben/a96ce952b7eb54356f7f8390319473e4"&gt;Jupyter notebook&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/03/27/dask-cuml.md&lt;/span&gt;, line 43)&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="fast-fitting"&gt;
&lt;h1&gt;Fast Fitting!&lt;/h1&gt;
&lt;p&gt;cuML is fast! But finding that speed requires developing a bit of GPU knowledge and some
intuition. For example, there is a non-zero cost of moving data from device to GPU and, when data is “small” there are little to no performance gains. “Small”, currently might mean less than 100MB.&lt;/p&gt;
&lt;p&gt;In the following example we use the &lt;a class="reference external" href="https://scikit-learn.org/stable/modules/generated/sklearn.datasets.load_diabetes.html"&gt;diabetes data&lt;/a&gt;
set provided by sklearn and linearly fit the data with &lt;a class="reference external" href="https://scikit-learn.org/stable/modules/generated/sklearn.linear_model.Ridge.html"&gt;RidgeRegression&lt;/a&gt;&lt;/p&gt;
&lt;div class="math notranslate nohighlight"&gt;
\[ \min\limits_w ||y - Xw||^2_2 + alpha \* ||w||^2_2\]&lt;/div&gt;
&lt;p&gt;&lt;a class="reference external" href="https://scikit-learn.org/stable/auto_examples/linear_model/plot_ridge_path.html"&gt;&lt;strong&gt;alpha&lt;/strong&gt;&lt;/a&gt; is the hyper-parameter and we initially set to 1.&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;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cuml&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;Ridge&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;cumlRidge&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_ml.model_selection&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;dcv&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;sklearn&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;datasets&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;linear_model&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;sklearn.externals.joblib&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;parallel_backend&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;sklearn.model_selection&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;train_test_split&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;GridSearchCV&lt;/span&gt;

&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;X_test&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_test&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;train_test_split&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;diabetes&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;diabetes&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;target&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;test_size&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;0.2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;fit_intercept&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="kc"&gt;True&lt;/span&gt;
&lt;span class="n"&gt;normalize&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="kc"&gt;False&lt;/span&gt;
&lt;span class="n"&gt;alpha&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;array&lt;/span&gt;&lt;span class="p"&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;ridge&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;linear_model&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Ridge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;solver&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;cholesky&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cu_ridge&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;cumlRidge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;solver&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;eig&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;ridge&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cu_ridge&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The above ran with a single timing measurement of:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Scikit-Learn Ridge: 28 ms&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;cuML Ridge: 1.12 s&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;But the data is quite small, ~28KB. Increasing the size to ~2.8GB and re-running we see significant gains:&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;dup_ridge&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;linear_model&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Ridge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;solver&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;cholesky&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;dup_cu_ridge&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;cumlRidge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;solver&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;eig&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# move data from host to device&lt;/span&gt;
&lt;span class="n"&gt;record_data&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;fea&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="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;dup_data&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="n"&gt;dup_data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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;gdf_data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;cudf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;record_data&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;gdf_train&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;cudf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;dict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;train&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;dup_train&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

&lt;span class="c1"&gt;#sklearn&lt;/span&gt;
&lt;span class="n"&gt;dup_ridge&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dup_data&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;dup_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# cuml&lt;/span&gt;
&lt;span class="n"&gt;dup_cu_ridge&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;gdf_data&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;gdf_train&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;With new timing measurements of:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Scikit-Learn Ridge: 4.82 s ± 694 ms&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;cuML Ridge: 450 ms ± 47.6 ms&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;With more data we clearly see faster fitting times, but the time to move data to the GPU (through CUDF)
was 19.7s. This cost of data movement is one of the reasons why RAPIDS/cuDF was developed – keep data
on the GPU and avoid having to move back and forth.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/03/27/dask-cuml.md&lt;/span&gt;, line 108)&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="hyper-parameter-optimization-experiments"&gt;
&lt;h1&gt;Hyper-Parameter Optimization Experiments&lt;/h1&gt;
&lt;p&gt;So moving to the GPU can be costly, but once there, with larger data sizes, we gain significant performance
optimizations. Naively, we thought, “well, we have GPU machine learning, we have distributed hyper-parameter optimization…
we &lt;em&gt;should&lt;/em&gt; have distributed, GPU-accelerated, hyper-parameter optimization!”&lt;/p&gt;
&lt;p&gt;Scikit-Learn assumes a specific, but well defined API for estimators over which it will perform hyper-parameter
optimization. Most estimators/classifiers in Scikit-Learn look 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="k"&gt;class&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nc"&gt;DummyEstimator&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;BaseEstimator&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="fm"&gt;__init__&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="o"&gt;=...&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;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;X&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="o"&gt;...&lt;/span&gt;

    &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;predict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;X&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;score&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;X&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="o"&gt;...&lt;/span&gt;

    &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;get_params&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="o"&gt;...&lt;/span&gt;

    &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;set_params&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="o"&gt;...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;When we started experimenting with hyper-parameter optimization, we found a few API holes missing, these were
resolved, mostly handling matching argument structure and various getters/setters.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;get_params and set_params (&lt;a class="reference external" href="https://github.com/rapidsai/cuml/pull/271"&gt;#271&lt;/a&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;fix/clf-solver (&lt;a class="reference external" href="https://github.com/rapidsai/cuml/pull/318"&gt;#318&lt;/a&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;map fit_transform to sklearn implementation (&lt;a class="reference external" href="https://github.com/rapidsai/cuml/pull/330"&gt;#330&lt;/a&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Fea get params small changes (&lt;a class="reference external" href="https://github.com/rapidsai/cuml/pull/322"&gt;#322&lt;/a&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;With holes plugged up we tested again. Using the same diabetes data set, we are now performing hyper-parameter optimization
and searching over many alpha parameters for the best &lt;em&gt;scoring&lt;/em&gt; alpha.&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;params&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;alpha&amp;#39;&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;logspace&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="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="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;clf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;linear_model&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Ridge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;solver&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;cholesky&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cu_clf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;cumlRidge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;solver&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;eig&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;grid&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;GridSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;clf&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scoring&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;r2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;grid&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;cu_grid&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;GridSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cu_clf&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scoring&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;r2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cu_grid&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Again, reminding ourselves that the data is small ~28KB, we don’t expect to observe cuml performing faster than sklearn. Instead, we want to demonstrate functionality.&lt;/p&gt;
&lt;p&gt;Again, reminding ourselves that the data is small ~28KB, we don’t expect to observe cuml performing faster than Scikit-Learn. Instead, we
want to demonstrate functionality. Additionally, we also tried swapping out Dask-ML’s implementation of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GridSearchCV&lt;/span&gt;&lt;/code&gt;
(which adheres to the same API as Scikit-Learn) to use all of the GPUs we have available in parallel.&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;params&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;alpha&amp;#39;&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;logspace&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="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="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;clf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;linear_model&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Ridge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;solver&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;cholesky&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cu_clf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;cumlRidge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;normalize&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;solver&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;eig&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;grid&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dcv&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;GridSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;clf&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scoring&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;r2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;grid&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;cu_grid&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dcv&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;GridSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cu_clf&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scoring&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;r2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cu_grid&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Timing Measurements:&lt;/p&gt;
&lt;div class="pst-scrollable-table-container"&gt;&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr class="row-odd"&gt;&lt;th class="head"&gt;&lt;p&gt;GridSearchCV&lt;/p&gt;&lt;/th&gt;
&lt;th class="head"&gt;&lt;p&gt;sklearn-Ridge&lt;/p&gt;&lt;/th&gt;
&lt;th class="head"&gt;&lt;p&gt;cuml-ridge&lt;/p&gt;&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr class="row-even"&gt;&lt;td&gt;&lt;p&gt;&lt;strong&gt;Scikit-Learn&lt;/strong&gt;&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;88.4 ms ± 6.11 ms&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;6.51 s ± 132 ms&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td&gt;&lt;p&gt;&lt;strong&gt;Dask-ML&lt;/strong&gt;&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;873 ms ± 347 ms&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;740 ms ± 142 ms&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;/div&gt;
&lt;p&gt;Unsurprisingly, we see that GridSearchCV and Ridge Regression from Scikit-Learn is the fastest in this context.
There is cost to distributing work and data, and as we previously mentioned, moving data from host to device.&lt;/p&gt;
&lt;section id="how-does-performance-scale-as-we-scale-data"&gt;
&lt;h2&gt;How does performance scale as we scale data?&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="n"&gt;two_dup_data&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;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;vstack&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;1e2&lt;/span&gt;&lt;span class="p"&gt;)))&lt;/span&gt;
&lt;span class="n"&gt;two_dup_train&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;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;hstack&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;1e2&lt;/span&gt;&lt;span class="p"&gt;)))&lt;/span&gt;
&lt;span class="n"&gt;three_dup_data&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;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;vstack&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;1e3&lt;/span&gt;&lt;span class="p"&gt;)))&lt;/span&gt;
&lt;span class="n"&gt;three_dup_train&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;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;hstack&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;1e3&lt;/span&gt;&lt;span class="p"&gt;)))&lt;/span&gt;

&lt;span class="n"&gt;cu_grid&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dcv&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;GridSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cu_clf&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scoring&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;r2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cu_grid&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;two_dup_data&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;two_dup_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;cu_grid&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dcv&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;GridSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cu_clf&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scoring&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;r2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cu_grid&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;three_dup_data&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;three_dup_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;grid&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dcv&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;GridSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;clf&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scoring&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;r2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;grid&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;three_dup_data&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;three_dup_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Timing Measurements:&lt;/p&gt;
&lt;div class="pst-scrollable-table-container"&gt;&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr class="row-odd"&gt;&lt;th class="head"&gt;&lt;p&gt;Data (MB)&lt;/p&gt;&lt;/th&gt;
&lt;th class="head"&gt;&lt;p&gt;cuML+Dask-ML&lt;/p&gt;&lt;/th&gt;
&lt;th class="head"&gt;&lt;p&gt;sklearn+Dask-ML&lt;/p&gt;&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr class="row-even"&gt;&lt;td&gt;&lt;p&gt;2.8 MB&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;13.8s&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td&gt;&lt;p&gt;28 MB&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;1min 17s&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;4.87 s&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;/div&gt;
&lt;p&gt;cuML + dask-ml (Distributed GridSearchCV) does significantly &lt;em&gt;worse&lt;/em&gt; as data sizes increase! Why? Primarily, two reasons:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Non optimized movement of data between host and device compounded by N devices and the size of
the parameter space&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scoring methods are not implemented in with cuML&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Below is the Dask graph for the GridSearch&lt;/p&gt;
&lt;p&gt;
  &lt;a href="/images/cuml_grid.svg"&gt;
    &lt;img src="/images/cuml_grid.svg" width="90%"&gt;
  &lt;/a&gt;
&lt;/p&gt;
&lt;p&gt;There are 50 (cv=5 times 10 parameters for alpha) instances of chunking up our test data set and scoring performance. That means 50 times we are moving data back forth between host and device for fitting and 50 times for scoring. That’s not great, but it’s also very solvable – build scoring functions for GPUs!&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/03/27/dask-cuml.md&lt;/span&gt;, line 230)&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="immediate-future-work"&gt;
&lt;h1&gt;Immediate Future Work&lt;/h1&gt;
&lt;p&gt;We know the problems, GH Issues have been filed, and we are working on these issues – come help!&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Built In Scorers (&lt;a class="reference external" href="https://github.com/rapidsai/cuml/issues/242"&gt;#242&lt;/a&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;DeviceNDArray as input data (&lt;a class="reference external" href="https://github.com/rapidsai/cuml/issues/369"&gt;#369&lt;/a&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Communication with UCX (&lt;a class="reference external" href="https://github.com/dask/distributed/issues/2344"&gt;#2344&lt;/a&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/03/27/dask-cuml/"/>
    <summary>Document headings start at H3, not H1 [myst.header]</summary>
    <category term="GPU" label="GPU"/>
    <category term="RAPIDS" label="RAPIDS"/>
    <category term="dask" label="dask"/>
    <published>2019-03-27T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/03/18/dask-nep18/</id>
    <title>Dask and the __array_function__ protocol</title>
    <updated>2019-03-18T00:00:00+00:00</updated>
    <author>
      <name>Peter Andreas Entschev</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/03/18/dask-nep18.md&lt;/span&gt;, line 10)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;Dask is versatile for analytics parallelism, but there is still one issue to
leverage it to a broader spectrum: allowing it to transparently work with
&lt;a class="reference external" href="https://www.numpy.org/"&gt;NumPy&lt;/a&gt;-like libraries. We have previously discussed
how to work with
&lt;a class="reference external" href="http://blog.dask.org/2019/01/03/dask-array-gpus-first-steps"&gt;GPU Dask Arrays&lt;/a&gt;,
but limited to the scope of the array’s member methods sharing a NumPy-like
interface, for example the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.sum()&lt;/span&gt;&lt;/code&gt; method, thus, calling general functionality
from NumPy’s library wasn’t still possible. NumPy recently addressed this issue
in &lt;a class="reference external" href="https://www.numpy.org/neps/nep-0018-array-function-protocol.html"&gt;NEP-18&lt;/a&gt;
with the introduction of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&lt;/span&gt;&lt;/code&gt; protocol. In short, the
protocol allows a NumPy function call to dispatch the appropriate NumPy-like
library implementation, depending on the array type given as input, thus
allowing Dask to remain agnostic of such libraries, internally calling just the
NumPy function, which automatically handles dispatching of the appropriate
library implementation, for example,
&lt;a class="reference external" href="https://cupy.chainer.org/"&gt;CuPy&lt;/a&gt; or &lt;a class="reference external" href="https://sparse.pydata.org/"&gt;Sparse&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;To understand what’s the end goal of this change, consider the following
example:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;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="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;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;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;d&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;from_array&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;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="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;np&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;d&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now consider we want to speedup the SVD computation of a Dask array and offload
that work to a CUDA-capable GPU, we ultimately want to simply replace the NumPy
array &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x&lt;/span&gt;&lt;/code&gt; by a CuPy array and let NumPy do its magic via
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&lt;/span&gt;&lt;/code&gt; protocol and dispatch the appropriate CuPy linear algebra
operations under the hood:&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;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cupy&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;

&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&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;d&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;from_array&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;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="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;np&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;d&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We could do the same for a Sparse array, or any other NumPy-like array that
supports the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&lt;/span&gt;&lt;/code&gt; protocol and the computation that we are
trying to perform. In the next section, we will take a look at potential
performance benefits that the protocol helps leveraging.&lt;/p&gt;
&lt;p&gt;Note that the features described in this post are still experimental, some
still under development and review. For a more detailed discussion on the
actual progress of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&lt;/span&gt;&lt;/code&gt;, please refer to the &lt;a class="reference internal" href="#issues"&gt;&lt;span class="xref myst"&gt;Issues section&lt;/span&gt;&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/2019/03/18/dask-nep18.md&lt;/span&gt;, line 70)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="performance"&gt;
&lt;h1&gt;Performance&lt;/h1&gt;
&lt;p&gt;Before going any further, assume the following hardware is utilized for all
performance results described in this entire post:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;CPU: 6-core (12-threads) Intel Core i7-7800X &amp;#64; 3.50GHz&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Main memory: 16 GB&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;GPU: NVIDIA Quadro GV100&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;OpenBLAS 0.2.18&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;cuBLAS 9.2.174&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;cuSOLVER 9.2.148&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Let’s now check an example to see potential performance benefits of the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&lt;/span&gt;&lt;/code&gt; protocol with Dask when using CuPy as a backend. Let’s
start by creating all the arrays that we will use for computing an SVD later.
Please note that my focus here is how Dask can leverage compute performance,
therefore I’m ignoring in this example the time spent on creating or copying
the arrays between CPU and GPU.&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;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cupy&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;

&lt;span class="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;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;10000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array&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;dx&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;from_array&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;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;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;dy&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;from_array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="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;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;asarray&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Seen above we have four arrays:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x&lt;/span&gt;&lt;/code&gt;: a NumPy array in main memory;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;y&lt;/span&gt;&lt;/code&gt;: a CuPy array in GPU memory;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dx&lt;/span&gt;&lt;/code&gt;: a NumPy array wrapped in a Dask array;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dy&lt;/span&gt;&lt;/code&gt;: a &lt;em&gt;copy&lt;/em&gt; of a CuPy array wrapped in a Dask array; wrapping a CuPy
array in a Dask array as a view (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;asarray=True&lt;/span&gt;&lt;/code&gt;) is not supported yet.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;section id="compute-svd-on-a-numpy-array"&gt;
&lt;h2&gt;Compute SVD on a NumPy array&lt;/h2&gt;
&lt;p&gt;We can then start by computing the SVD of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x&lt;/span&gt;&lt;/code&gt; using NumPy, thus, it’s
processed on CPU in a single thread:&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;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;np&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;p&gt;The timing information I obtained after that looks like the following:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;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;3&lt;/span&gt;&lt;span class="nb"&gt;min&lt;/span&gt; &lt;span class="mi"&gt;10&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;347&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;3&lt;/span&gt;&lt;span class="nb"&gt;min&lt;/span&gt; &lt;span class="mi"&gt;11&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;3&lt;/span&gt;&lt;span class="nb"&gt;min&lt;/span&gt; &lt;span class="mi"&gt;11&lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Over 3 minutes seems a bit too slow, so now the question is: Can we do better,
and more importantly, without having to change our entire code?&lt;/p&gt;
&lt;p&gt;The answer to this question is: Yes, we can.&lt;/p&gt;
&lt;p&gt;Let’s look now at other results.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="compute-svd-on-the-numpy-array-wrapped-in-dask-array"&gt;
&lt;h2&gt;Compute SVD on the NumPy array wrapped in Dask array&lt;/h2&gt;
&lt;p&gt;First, of all, this is what you had to do &lt;em&gt;before&lt;/em&gt; the introduction of the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&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="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;dx&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The code above might have been very prohibitive for several projects, since one
needs to call the proper library dispatcher in addition to passing the correct
array. In other words, one would need to find all NumPy calls in the code and
replace those by the correct library’s function call, depending on the input
array type. After &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&lt;/span&gt;&lt;/code&gt;, the same NumPy function can be
called, using the Dask array &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dx&lt;/span&gt;&lt;/code&gt; as input:&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;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;np&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;dx&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Note: Dask defers computation of results until its consumption, therefore we
need to call the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.compute()&lt;/span&gt;&lt;/code&gt; function on result arrays to actually compute
them.&lt;/p&gt;
&lt;p&gt;Let’s now take a look at the timing information:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;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;1&lt;/span&gt;&lt;span class="nb"&gt;min&lt;/span&gt; &lt;span class="mi"&gt;23&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;460&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;1&lt;/span&gt;&lt;span class="nb"&gt;min&lt;/span&gt; &lt;span class="mi"&gt;23&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;13&lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now, without changing any code, besides the wrapping of the NumPy array as a
Dask array, we can see a speedup of 2x. Not too bad. But let’s go back to our
previous question: Can we do better?&lt;/p&gt;
&lt;/section&gt;
&lt;section id="compute-svd-on-the-cupy-array"&gt;
&lt;h2&gt;Compute SVD on the CuPy array&lt;/h2&gt;
&lt;p&gt;We can do the same as for the Dask array now and simply call NumPy’s SVD
function on the CuPy array &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;y&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;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;np&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;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The timing information we get now is the following:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;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;17.3&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.81&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;19.1&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;19.1&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We now see a 4-5x speedup with no change in internal calls whatsoever! This is
exactly the sort of benefit that we expect to leverage with the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&lt;/span&gt;&lt;/code&gt; protocol, speeding up existing code, for free!&lt;/p&gt;
&lt;p&gt;Let’s go back to our original question one last time: Can we do better?&lt;/p&gt;
&lt;/section&gt;
&lt;section id="compute-svd-on-the-cupy-array-wrapped-in-dask-array"&gt;
&lt;h2&gt;Compute SVD on the CuPy array wrapped in Dask array&lt;/h2&gt;
&lt;p&gt;We can now take advantage of the benefits of Dask data chunk splitting &lt;em&gt;and&lt;/em&gt;
the CuPy GPU implementation, in an attempt to keep our GPU busy as much as
possible, this remains as simple as:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;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;np&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;dy&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;For which we get the following timing:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;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;8.97&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;653&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;9.62&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;9.45&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Giving us another 2x speedup over the single-threaded CuPy SVD computing.&lt;/p&gt;
&lt;p&gt;To conclude, we started from over 3 minutes and are now down to under 10
seconds by simply dispatching the work on a different array.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/03/18/dask-nep18.md&lt;/span&gt;, line 214)&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="application"&gt;
&lt;h1&gt;Application&lt;/h1&gt;
&lt;p&gt;We will now talk a bit about potential applications of the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&lt;/span&gt;&lt;/code&gt; protocol. For this, we will discuss the
&lt;a class="reference external" href="https://dask-glm.readthedocs.io/"&gt;Dask-GLM&lt;/a&gt; library, used for fitting
Generalized Linear Models on large datasets. It’s built on top of Dask and
offers an API compatible with &lt;a class="reference external" href="https://scikit-learn.org/"&gt;scikit-learn&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Before the introduction of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&lt;/span&gt;&lt;/code&gt; protocol, we would need
to rewrite most of its internal implementation for each and every NumPy-like
library that we wished to use as a backend, therefore, we would need a
specialization of the implementation for Dask, another for CuPy and yet
another for Sparse. Now, thanks to all the functionality that these libraries
share through compatible interface, we don’t have to change the implementation
at all, we simply pass a different array type as input, as simple as that.&lt;/p&gt;
&lt;section id="example-with-scikit-learn"&gt;
&lt;h2&gt;Example with scikit-learn&lt;/h2&gt;
&lt;p&gt;To demonstrate the ability we acquired, let’s consider the following
scikit-learn example (based on the original example
&lt;a class="reference external" href="https://scikit-learn.org/stable/auto_examples/linear_model/plot_ols.html#sphx-glr-auto-examples-linear-model-plot-ols-py"&gt;here&lt;/a&gt;):&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;matplotlib.pyplot&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;plt&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;sklearn.linear_model&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;LinearRegression&lt;/span&gt;

&lt;span class="n"&gt;N&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;

&lt;span class="c1"&gt;# x from 0 to N&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;N&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;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;40000&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="c1"&gt;# y = a*x + b with noise&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;0.5&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="mf"&gt;1.0&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;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="n"&gt;size&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;shape&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# create a linear regression model&lt;/span&gt;
&lt;span class="n"&gt;est&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LinearRegression&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We can then fit the model,&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;est&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;and obtain its time measurements:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;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;3.4&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt; &lt;span class="n"&gt;ns&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;3.4&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="mf"&gt;2.3&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We can then use it for prediction on some test 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="c1"&gt;# predict y from the data&lt;/span&gt;
&lt;span class="n"&gt;x_new&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;linspace&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;N&lt;/span&gt;&lt;span class="p"&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;y_new&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;est&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;predict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x_new&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;newaxis&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And also check its time measurements:&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;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.16&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;680&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;1.84&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="mf"&gt;1.58&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And finally plot the results:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# plot the results&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;figure&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;figsize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;4&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;ax&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;axes&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scatter&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;linewidth&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="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;plot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x_new&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_new&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;color&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;black&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_facecolor&lt;/span&gt;&lt;span class="p"&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="mf"&gt;1.0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;0.42&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

&lt;span class="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_xlabel&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;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_ylabel&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="n"&gt;ax&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;tight&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;show&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/dask-nep18-linreg.png"&gt;
&lt;/section&gt;
&lt;section id="example-with-dask-glm"&gt;
&lt;h2&gt;Example with Dask-GLM&lt;/h2&gt;
&lt;p&gt;The only thing we have to change from the code before is the first block, where
we import libraries and create 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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_glm.estimators&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;LinearRegression&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;matplotlib.pyplot&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;plt&lt;/span&gt;

&lt;span class="n"&gt;N&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;

&lt;span class="c1"&gt;# x from 0 to N&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;N&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;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;40000&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="c1"&gt;# y = a*x + b with noise&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;0.5&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="mf"&gt;1.0&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;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="n"&gt;size&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;shape&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# create a linear regression model&lt;/span&gt;
&lt;span class="n"&gt;est&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LinearRegression&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;solver&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;lbfgs&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;The rest of the code and also the plot look alike the previous scikit-learn
example, so we’re ommitting those here for brevity. Note also that we could
have called &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LinearRegression()&lt;/span&gt;&lt;/code&gt; without any arguments, but for this example
we chose the
&lt;a class="reference external" href="https://docs.scipy.org/doc/scipy/reference/optimize.minimize-lbfgsb.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;lbfgs&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;
solver, that converges reasonably fast.&lt;/p&gt;
&lt;p&gt;We can also have a look at the timing results for fitting, followed by those
for predicting with Dask-GLM:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Fitting&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;9.66&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;116&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;9.78&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="mf"&gt;8.94&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;

&lt;span class="c1"&gt;# Predicting&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;130&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;327&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;457&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;1.06&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If instead we want to use CuPy to compute, we have to change only 3 lines,
importing &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cupy&lt;/span&gt;&lt;/code&gt; instead of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;numpy&lt;/span&gt;&lt;/code&gt;, and the two lines where we create the
random arrays, replacing them to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cupy.random&lt;/span&gt;&lt;/code&gt; insted of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.random&lt;/span&gt;&lt;/code&gt;. The
block should then look like this:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cupy&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_glm.estimators&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;LinearRegression&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;matplotlib.pyplot&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;plt&lt;/span&gt;

&lt;span class="n"&gt;N&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;

&lt;span class="c1"&gt;# x from 0 to N&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;N&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mi"&gt;40000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

&lt;span class="c1"&gt;# y = a*x + b with noise&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;0.5&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="mf"&gt;1.0&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;normal&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="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="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# create a linear regression model&lt;/span&gt;
&lt;span class="n"&gt;est&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LinearRegression&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;solver&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;lbfgs&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;And the timing results we obtain in this scenario are:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Fitting&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;151&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;40.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;191&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;190&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;

&lt;span class="c1"&gt;# Predicting&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.91&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;778&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;2.69&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="mf"&gt;1.37&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;For the simple example chosen for this post, scikit-learn outperforms Dask-GLM
using both NumPy and CuPy arrays. There may exist several reasons that
contribute to this, and while we didn’t dive deep into understanding the exact
reasons and their extent, we could cite some likely possibilities:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;scikit-learn may be using solvers that converge faster;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask-GLM is entirely built on top of Dask, while scikit-learn may be
heavily optimized internally;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Too many synchronization steps and data transfer could occur for small
datasets with CuPy.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="performance-for-different-dask-glm-solvers"&gt;
&lt;h2&gt;Performance for different Dask-GLM solvers&lt;/h2&gt;
&lt;p&gt;To verify how Dask-GLM with NumPy arrays would compare with CuPy arrays, we
also did some logistic regression benchmarking of Dask-GLM solvers. The results
below were obtained from a training dataset with 10&lt;sup&gt;2&lt;/sup&gt;,
10&lt;sup&gt;3&lt;/sup&gt;, …, 10&lt;sup&gt;6&lt;/sup&gt; features of 100 dimensions, and matching
number of test features.&lt;/p&gt;
&lt;p&gt;Note: we are intentionally omitting results for Dask arrays, as we have
identified a &lt;a class="reference external" href="https://github.com/dask/dask-glm/issues/78"&gt;potential bug&lt;/a&gt; that
causes Dask arrays not to converge.&lt;/p&gt;
&lt;img src="/images/dask-nep18-fitting.png"&gt;
&lt;p&gt;From the results observed in the graphs above we can see that CuPy can be one
order of magnitude faster than NumPy for fitting with any of the Dask-GLM
solvers. Please note also that both axis are given in logarithmic scale for
easier visualization.&lt;/p&gt;
&lt;p&gt;Another interesting effect that can be seen is how converging may take longer
for small number of samples. However, as we would normally hope for, compute
time required to converge scales linearly to the number of samples.&lt;/p&gt;
&lt;img src="/images/dask-nep18-prediction.png"&gt;
&lt;p&gt;Prediction with CuPy, as seen above, can be proportionally much faster than
NumPy, staying mostly constant for all solvers, and around 3-4 orders of
magnitude faster.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/03/18/dask-nep18.md&lt;/span&gt;, line 418)&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="issues"&gt;
&lt;h1&gt;&lt;a name="issues"&gt;&lt;/a&gt;Issues&lt;/h1&gt;
&lt;p&gt;In this section we describe the work that has been done and is still ongoing
since February, 2019, towards enabling the features described in previous
sections. If you are not interested in all the details, feel free to completely
skip this.&lt;/p&gt;
&lt;section id="fixed-issues"&gt;
&lt;h2&gt;Fixed Issues&lt;/h2&gt;
&lt;p&gt;Since early February, 2019, substantial progress has been made towards deeper
support of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&lt;/span&gt;&lt;/code&gt; protocol in the different projects,
this trend is still going on and will continue in March. Below we see a list
of issues that have been fixed or are in the process of review:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&lt;/span&gt;&lt;/code&gt; protocol dependencies fixed in
&lt;a class="reference external" href="https://github.com/cupy/cupy/issues/2029"&gt;CuPy PR #2029&lt;/a&gt;;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask issues using CuPy backend with mean() and moment()
&lt;a class="reference external" href="https://github.com/dask/dask/issues/4481"&gt;Dask Issue #4481&lt;/a&gt;, fixed in
&lt;a class="reference external" href="https://github.com/dask/dask/pull/4513"&gt;Dask PR #4513&lt;/a&gt; and
&lt;a class="reference external" href="https://github.com/dask/dask/pull/4519"&gt;Dask PR #4519&lt;/a&gt;;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Replace in SciPy the aliased NumPy functions that may not be available in
libraries like CuPy, fixed in
&lt;a class="reference external" href="https://github.com/scipy/scipy/pull/9888"&gt;SciPy PR #9888&lt;/a&gt;;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Allow creation of arbitrary shaped arrays, using the input array as
reference for the new array to be created, under review in
&lt;a class="reference external" href="https://github.com/numpy/numpy/issues/13043"&gt;NumPy PR #13043&lt;/a&gt;;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Multithreading with CuPy first identified in
&lt;a class="reference external" href="https://github.com/dask/dask/issues/4487"&gt;Dask Issue #4487&lt;/a&gt;,
&lt;a class="reference external" href="https://github.com/cupy/cupy/issues/2045"&gt;CuPy Issue #2045&lt;/a&gt; and
&lt;a class="reference external" href="https://github.com/cupy/cupy/issues/1109"&gt;CuPy Issue #1109&lt;/a&gt;, now under
review in &lt;a class="reference external" href="https://github.com/cupy/cupy/pull/2053"&gt;CuPy PR #2053&lt;/a&gt;;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Calling Dask’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;flatnonzero()&lt;/span&gt;&lt;/code&gt; on CuPy array missing &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cupy.compress()&lt;/span&gt;&lt;/code&gt;,
first identified in
&lt;a class="reference external" href="https://github.com/dask/dask/issues/4497"&gt;Dask Issue #4497&lt;/a&gt;, under review
in &lt;a class="reference external" href="https://github.com/dask/dask/pull/4548"&gt;Dask PR #4548&lt;/a&gt;,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask support for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&lt;/span&gt;&lt;/code&gt;, under review in
&lt;a class="reference external" href="https://github.com/dask/dask/pull/4567"&gt;Dask PR #4567&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="known-issues"&gt;
&lt;h2&gt;Known Issues&lt;/h2&gt;
&lt;p&gt;Currently, one of the biggest issues we are tackling relates to the
&lt;a class="reference external" href="https://github.com/dask/dask/issues/4490"&gt;Dask issue #4490&lt;/a&gt; we first
identified when calling Dask’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;diag()&lt;/span&gt;&lt;/code&gt; on a CuPy array. This requires some
change on the Dask &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Array&lt;/span&gt;&lt;/code&gt; class, and subsequent changes throughout large
parts of the Dask codebase. I will not go into too much detail here, but the
way we are handling this issue is by adding a new attribute &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;_meta&lt;/span&gt;&lt;/code&gt; to Dask
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Array&lt;/span&gt;&lt;/code&gt; in replacement of the simple &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dtype&lt;/span&gt;&lt;/code&gt; that currently exists. This
new attribute will not only hold the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dtype&lt;/span&gt;&lt;/code&gt; information, but also an empty
array of the backend type used to create the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Array&lt;/span&gt;&lt;/code&gt; in the first place, thus
allowing us to internally reconstruct arrays of the backend type, without
having to know explicitly whether it’s a NumPy, CuPy, Sparse or any other
NumPy-like array. For additional details, please refer to &lt;a class="reference external" href="https://github.com/dask/dask/issues/2977"&gt;Dask Issue
#2977&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;We have identified some more issues with ongoing discussions:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Using Sparse as a Dask backend, discussed in
&lt;a class="reference external" href="https://github.com/dask/dask/issues/4523"&gt;Dask Issue #4523&lt;/a&gt;;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Calling Dask’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;fix()&lt;/span&gt;&lt;/code&gt; on CuPy array depends on &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_wrap__&lt;/span&gt;&lt;/code&gt;,
discussed in &lt;a class="reference external" href="https://github.com/dask/dask/issues/4496"&gt;Dask Issue #4496&lt;/a&gt;
and &lt;a class="reference external" href="https://github.com/cupy/cupy/issues/589"&gt;CuPy Issue #589&lt;/a&gt;;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Allow coercing of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_function__&lt;/span&gt;&lt;/code&gt;, discussed in
&lt;a class="reference external" href="https://github.com/numpy/numpy/issues/12974"&gt;NumPy Issue #12974&lt;/a&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/2019/03/18/dask-nep18.md&lt;/span&gt;, line 482)&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="future-work"&gt;
&lt;h1&gt;Future Work&lt;/h1&gt;
&lt;p&gt;There are several possibilities for a richer experience with Dask, some of which
could be very interesting in the short and mid-term are:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Use &lt;a class="reference external" href="https://github.com/rapidsai/dask-cudf"&gt;Dask-cuDF&lt;/a&gt; alongside with
Dask-GLM to present interesting realistic applications of the whole
ecosystem;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;More comprehensive examples and benchmarks for Dask-GLM;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Support for &lt;a class="reference external" href="https://scikit-learn.org/stable/modules/linear_model.html"&gt;more models in
Dask-GLM&lt;/a&gt;;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A deeper look into the Dask-GLM versus scikit-learn performance;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Profile CuPy’s performance of matrix-matrix multiplication operations
(GEMM), compare to matrix-vector multiplication operations (GEMV) for
distributed Dask operation.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/03/18/dask-nep18/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="CuPy" label="CuPy"/>
    <category term="Dask" label="Dask"/>
    <category term="Dask-GLM" label="Dask-GLM"/>
    <category term="Sparse" label="Sparse"/>
    <published>2019-03-18T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/03/04/building-gpu-groupbys/</id>
    <title>Building GPU Groupby-Aggregations for Dask</title>
    <updated>2019-03-04T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/03/04/building-gpu-groupbys.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;We’ve sufficiently aligned Dask DataFrame and cuDF to get groupby aggregations
like the following to work well.&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;y&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This post describes the kind of work we had to do as a model for future
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/2019/03/04/building-gpu-groupbys.md&lt;/span&gt;, line 21)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="plan"&gt;
&lt;h1&gt;Plan&lt;/h1&gt;
&lt;p&gt;As outlined in a previous post, &lt;a class="reference internal" href="#../../../2019/01/13/dask-cudf-first-steps.html"&gt;&lt;span class="xref myst"&gt;Dask, Pandas, and GPUs: first
steps&lt;/span&gt;&lt;/a&gt;, our plan to produce
distributed GPU dataframes was to combine &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe.html"&gt;Dask
DataFrame&lt;/a&gt; with
&lt;a class="reference external" href="https://rapids.ai"&gt;cudf&lt;/a&gt;. In particular, we had to&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;change Dask DataFrame so that it would parallelize not just around the
Pandas DataFrames that it works with today, but around anything that looked
enough like a Pandas DataFrame&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;change cuDF so that it would look enough like a Pandas DataFrame to fit
within the algorithms in Dask DataFrame&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/03/04/building-gpu-groupbys.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="changes"&gt;
&lt;h1&gt;Changes&lt;/h1&gt;
&lt;p&gt;On the Dask side this mostly meant replacing&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Replacing &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;isinstance(df,&lt;/span&gt; &lt;span class="pre"&gt;pd.DataFrame)&lt;/span&gt;&lt;/code&gt; checks with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;is_dataframe_like(df)&lt;/span&gt;&lt;/code&gt;
checks (after defining a suitable
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;is_dataframe_like&lt;/span&gt;&lt;/code&gt;/&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;is_series_like&lt;/span&gt;&lt;/code&gt;/&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;is_index_like&lt;/span&gt;&lt;/code&gt; functions&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Avoiding some more exotic functionality in Pandas, and instead trying to
use more common functionality that we can expect to be in most DataFrame
implementations&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;On the cuDF side this means making dozens of tiny changes to align the cuDF API
to the Pandas API, and to add in missing features.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Dask Changes:&lt;/strong&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4359"&gt;Remove explicit pandas checks and provide cudf lazy registration #4359&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4375"&gt;Replace isinstance(…, pandas) with is_dataframe_like #4375&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4395"&gt;Add has_parallel_type&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4396"&gt;Lazily register more cudf functions and move to backends file #4396&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4418"&gt;Avoid checking against types in is_dataframe_like #4418&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4470"&gt;Replace cudf-specific code with dask-cudf import #4470&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4482"&gt;Avoid groupby.agg(callable) in groupby-var #4482&lt;/a&gt; – this one is notable in that by simplifying our Pandas usage we actually got a significant speedup on the Pandas side.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;cuDF Changes:&lt;/strong&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/529"&gt;Build DataFrames from CUDA array libraries #529&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/534"&gt;Groupby AttributeError&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/556"&gt;Support comparison operations on Indexes #556&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/568"&gt;Support byte ranges in read_csv (and other formats) #568&lt;/a&gt;:w&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/824"&gt;Allow “df.index = some_index” #824&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/828"&gt;Support indexing on groupby objects #828&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/831"&gt;Support df.reset_index(drop=True) #831&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/879"&gt;Add Series.groupby #879&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/880"&gt;Support Dataframe/Series groupby level=0 #880&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/900"&gt;Implement division on DataFrame objects #900&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/934"&gt;Groupby objects aren’t indexable by column names #934&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/937"&gt;Support comparisons on index operations #937&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/944"&gt;Add DataFrame.rename #944&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/967"&gt;Set the index of a dataframe/series #967&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/968"&gt;Support concat(…, axis=1) #968&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/969"&gt;Support indexing with a pandas index from columns #969&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/970"&gt;Support indexing a dataframe with another boolean dataframe #970&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;I don’t really expect anyone to go through all of those issues, but my hope is
that by skimming over the issue titles people will get a sense for the kinds of
changes we’re making here. It’s a large number of small things.&lt;/p&gt;
&lt;p&gt;Also, kudos to &lt;a class="reference external" href="https://github.com/thomcom"&gt;Thomson Comer&lt;/a&gt; who solved most of
the cuDF issues above.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/03/04/building-gpu-groupbys.md&lt;/span&gt;, line 83)&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="there-are-still-some-pending-issues"&gt;
&lt;h1&gt;There are still some pending issues&lt;/h1&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/1055"&gt;Square Root #1055&lt;/a&gt;, needed for groupby-std&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/483"&gt;cuDF needs multi-index support for columns #483&lt;/a&gt;, needed for:&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;gropuby&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;agg&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="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;mean&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;], &amp;#39;&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;: [&amp;#39;&lt;/span&gt;&lt;span class="nb"&gt;min&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;, &amp;#39;&lt;/span&gt;&lt;span class="nb"&gt;max&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;]})&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/03/04/building-gpu-groupbys.md&lt;/span&gt;, line 92)&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="but-things-mostly-work"&gt;
&lt;h1&gt;But things mostly work&lt;/h1&gt;
&lt;p&gt;But generally things work pretty well today:&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_cudf&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask_cudf&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;yellow_tripdata_2016-*.csv&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="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;passenger_count&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;trip_distance&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;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="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;cudf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Series&lt;/span&gt; &lt;span class="n"&gt;nrows&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt; &lt;span class="o"&gt;&amp;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;_&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_pandas&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;Out&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;0&lt;/span&gt;    &lt;span class="mf"&gt;0.625424&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;    &lt;span class="mf"&gt;4.976895&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;    &lt;span class="mf"&gt;4.470014&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;    &lt;span class="mf"&gt;5.955262&lt;/span&gt;
&lt;span class="mi"&gt;4&lt;/span&gt;    &lt;span class="mf"&gt;4.328076&lt;/span&gt;
&lt;span class="mi"&gt;5&lt;/span&gt;    &lt;span class="mf"&gt;3.079661&lt;/span&gt;
&lt;span class="mi"&gt;6&lt;/span&gt;    &lt;span class="mf"&gt;2.998077&lt;/span&gt;
&lt;span class="mi"&gt;7&lt;/span&gt;    &lt;span class="mf"&gt;3.147452&lt;/span&gt;
&lt;span class="mi"&gt;8&lt;/span&gt;    &lt;span class="mf"&gt;5.165570&lt;/span&gt;
&lt;span class="mi"&gt;9&lt;/span&gt;    &lt;span class="mf"&gt;5.916169&lt;/span&gt;
&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;float64&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/03/04/building-gpu-groupbys.md&lt;/span&gt;, line 119)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="experience"&gt;
&lt;h1&gt;Experience&lt;/h1&gt;
&lt;p&gt;First, most of this work was handled by the cuDF developers (which may be
evident from the relative lengths of the issue lists above). When we started
this process it felt like a never-ending stream of tiny issues. We weren’t
able to see the next set of issues until we had finished the current set.
Fortunately, most of them were pretty easy to fix. Additionally, as we went
on, it seemed to get a bit easier over time.&lt;/p&gt;
&lt;p&gt;Additionally, lots of things work other than groupby-aggregations as a result
of the changes above. From the perspective of someone accustomed to Pandas,
The cuDF library is starting to feel more reliable. We hit missing
functionality less frequently when using cuDF on other operations.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/03/04/building-gpu-groupbys.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="what-s-next"&gt;
&lt;h1&gt;What’s next?&lt;/h1&gt;
&lt;p&gt;More recently we’ve been working on the various join/merge operations in Dask
DataFrame like indexed joins on a sorted column, joins between large and small
dataframes (a common special case) and so on. Getting these algorithms from
the mainline Dask DataFrame codebase to work with cuDF is resulting in a
similar set of issues to what we saw above with groupby-aggregations, but so
far the list is much smaller. We hope that this is a trend as we continue on
to other sets of functionality into the future like I/O, time-series
operations, rolling windows, and so on.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/03/04/building-gpu-groupbys/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="GPU" label="GPU"/>
    <category term="RAPIDS" label="RAPIDS"/>
    <category term="dataframe" label="dataframe"/>
    <published>2019-03-04T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/01/31/dask-mpi-experiment/</id>
    <title>Running Dask and MPI programs together</title>
    <updated>2019-01-31T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/31/dask-mpi-experiment.md&lt;/span&gt;, line 10)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;We present an experiment on how to pass data from a loosely coupled parallel
computing system like Dask to a tightly coupled parallel computing system like
MPI.&lt;/p&gt;
&lt;p&gt;We give motivation and a complete digestible example.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/193a9671f1536b9d13524214798da4a8"&gt;Here is a gist of the code and results&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/2019/01/31/dask-mpi-experiment.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&gt;
&lt;section id="motivation"&gt;
&lt;h1&gt;Motivation&lt;/h1&gt;
&lt;p&gt;&lt;em&gt;Disclaimer: Nothing in this post is polished or production ready. This is an
experiment designed to start conversation. No long-term support is offered.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;We often get the following question:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;How do I use Dask to pre-process my data,
but then pass those results to a traditional MPI application?&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;You might want to do this because you’re supporting legacy code written
in MPI, or because your computation requires tightly coupled parallelism of the
sort that only MPI can deliver.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/31/dask-mpi-experiment.md&lt;/span&gt;, line 34)&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="first-solution-write-to-disk"&gt;
&lt;h1&gt;First solution: Write to disk&lt;/h1&gt;
&lt;p&gt;The simplest thing to do of course is to write your Dask results to disk and
then load them back from disk with MPI. Given the relative cost of your
computation to data loading, this might be a great choice.&lt;/p&gt;
&lt;p&gt;For the rest of this blogpost we’re going to assume that it’s not.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/31/dask-mpi-experiment.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="second-solution"&gt;
&lt;h1&gt;Second solution&lt;/h1&gt;
&lt;p&gt;We have a trivial MPI library written in &lt;a class="reference external" href="https://mpi4py.readthedocs.io/en/stable/"&gt;MPI4Py&lt;/a&gt;
where each rank just prints out all the data that it was given. In principle
though it could call into C++ code, and do arbitrary MPI things.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# my_mpi_lib.py&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;mpi4py&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;MPI&lt;/span&gt;

&lt;span class="n"&gt;comm&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;MPI&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;COMM_WORLD&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;print_data_and_rank&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;chunks&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="w"&gt;    &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot; Fake function that mocks out how an MPI function should operate&lt;/span&gt;

&lt;span class="sd"&gt;    -   It takes in a list of chunks of data that are present on this machine&lt;/span&gt;
&lt;span class="sd"&gt;    -   It does whatever it wants to with this data and MPI&lt;/span&gt;
&lt;span class="sd"&gt;        Here for simplicity we just print the data and print the rank&lt;/span&gt;
&lt;span class="sd"&gt;    -   Maybe it returns something&lt;/span&gt;
&lt;span class="sd"&gt;    &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="n"&gt;rank&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;comm&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Get_rank&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;chunk&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;on rank:&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;rank&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;chunk&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;sum&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;chunk&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="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;chunk&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In our dask program we’re going to use Dask normally to load in data, do some
preprocessing, and then hand off all of that data to each MPI rank, which will
call the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;print_data_and_rank&lt;/span&gt;&lt;/code&gt; function above to initialize the MPI
computation.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# my_dask_script.py&lt;/span&gt;

&lt;span class="c1"&gt;# Set up Dask workers from within an MPI job using the dask_mpi project&lt;/span&gt;
&lt;span class="c1"&gt;# See https://dask-mpi.readthedocs.io/en/latest/&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_mpi&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;initialize&lt;/span&gt;
&lt;span class="n"&gt;initialize&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;futures_of&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="c1"&gt;# Use Dask Array to &amp;quot;load&amp;quot; data (actually just create random data here)&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="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;100000000&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="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# Find out where data is on each worker&lt;/span&gt;
&lt;span class="c1"&gt;# TODO: This could be improved on the Dask side to reduce boiler plate&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;first&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;collections&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;defaultdict&lt;/span&gt;
&lt;span class="n"&gt;key_to_part_dict&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="nb"&gt;str&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;part&lt;/span&gt;&lt;span class="o"&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;part&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;part&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;futures_of&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;who_has&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;who_has&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;worker_map&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;defaultdict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;key&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;workers&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;who_has&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;worker_map&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;first&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;key_to_part_dict&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="c1"&gt;# Call an MPI-enabled function on the list of data present on each worker&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;my_mpi_lib&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;print_data_and_rank&lt;/span&gt;

&lt;span class="n"&gt;futures&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;print_data_and_rank&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;list_of_parts&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;worker&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;worker&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;list_of_parts&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;worker_map&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;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;close&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 can call this mix of Dask and an MPI program using normal &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mpirun&lt;/span&gt;&lt;/code&gt; or
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mpiexec&lt;/span&gt;&lt;/code&gt; commands.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;mpirun&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;np&lt;/span&gt; &lt;span class="mi"&gt;5&lt;/span&gt; &lt;span class="n"&gt;python&lt;/span&gt; &lt;span class="n"&gt;my_dask_script&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;py&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/31/dask-mpi-experiment.md&lt;/span&gt;, line 126)&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;So MPI started up and ran our script.
The &lt;a class="reference external" href="https://dask-mpi.readthedocs.io/en/latest/"&gt;dask-mpi&lt;/a&gt; project set a Dask
scheduler on rank 0, runs our client code on rank 1, and then runs a bunch of workers on ranks 2+.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Rank 0: Runs a Dask scheduler&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Rank 1: Runs our script&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ranks 2+: Run Dask workers&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Our script then created a Dask array, though presumably here it would read in
data from some source, do more complex Dask manipulations before continuing on.&lt;/p&gt;
&lt;p&gt;We then wait until all of the Dask work has finished and is in a quiet state.
We then query the state in the scheduler to find out where all of that data
lives. That’s this code here:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Find out where data is on each worker&lt;/span&gt;
&lt;span class="c1"&gt;# TODO: This could be improved on the Dask side to reduce boiler plate&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;first&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;collections&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;defaultdict&lt;/span&gt;
&lt;span class="n"&gt;key_to_part_dict&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="nb"&gt;str&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;part&lt;/span&gt;&lt;span class="o"&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;part&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;part&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;futures_of&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;who_has&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;who_has&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;worker_map&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;defaultdict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;key&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;workers&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;who_has&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;worker_map&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;first&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;key_to_part_dict&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Admittedly, this code is gross, and not particularly friendly or obvious to
non-Dask experts (or even Dask experts themselves, I had to steal this from the
&lt;a class="reference external" href="http://ml.dask.org/xgboost.html"&gt;Dask XGBoost project&lt;/a&gt;, which does
the same trick).&lt;/p&gt;
&lt;p&gt;But after that we just call our MPI library’s initialize function,
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;print_data_and_rank&lt;/span&gt;&lt;/code&gt; on all of our data using Dask’s
&lt;a class="reference external" href="http://docs.dask.org/en/latest/futures.html"&gt;Futures interface&lt;/a&gt;.
That function gets the data directly from local memory (the Dask workers and
MPI ranks are in the same process), and does whatever the MPI application
wants.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/31/dask-mpi-experiment.md&lt;/span&gt;, line 168)&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;This could be improved in a few ways:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;The “gross” code referred to above could probably be placed into some
library code to make this pattern easier for people to use.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ideally the Dask part of the computation wouldn’t also have to be managed
by MPI, but could maybe start up MPI on its own.&lt;/p&gt;
&lt;p&gt;You could imagine Dask running on something like Kubernetes doing highly
dynamic work, scaling up and down as necessary. Then it would get to a
point where it needed to run some MPI code so it would, itself, start up
MPI on its worker processes and run the MPI application on its data.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We haven’t really said anything about resilience here. My guess is that
this isn’t hard to do (Dask has plenty of mechanisms to build complex
inter-task relationships) but I didn’t solve it above.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/193a9671f1536b9d13524214798da4a8"&gt;Here is a gist of the code and results&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/01/31/dask-mpi-experiment/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="MPI" label="MPI"/>
    <published>2019-01-31T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/01/29/cudf-joins/</id>
    <title>Single-Node Multi-GPU Dataframe Joins</title>
    <updated>2019-01-29T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/29/cudf-joins.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;We experiment with single-node multi-GPU joins using cuDF and Dask. We find
that the in-GPU computation is faster than communication. We also present
context and plans for near-future work, including improving high performance
communication in Dask with UCX.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/6e2c33c33b32bc324e3965212f202f66"&gt;Here is a notebook of the experiment in this post&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/2019/01/29/cudf-joins.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&gt;
&lt;section id="introduction"&gt;
&lt;h1&gt;Introduction&lt;/h1&gt;
&lt;p&gt;In a recent post we showed how Dask + cuDF could accelerate reading CSV files
using multiple GPUs in parallel. That operation quickly became bound by the
speed of our disk after we added a few GPUs. Now we try a very different kind
of operation, multi-GPU joins.&lt;/p&gt;
&lt;p&gt;This workload can be communication-heavy, especially if the column on which we
are joining is not sorted nicely, and so provides a good example on the other
extreme from parsing CSV.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/29/cudf-joins.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="benchmark"&gt;
&lt;h1&gt;Benchmark&lt;/h1&gt;
&lt;section id="construct-random-data-using-the-cpu"&gt;
&lt;h2&gt;Construct random data using the CPU&lt;/h2&gt;
&lt;p&gt;Here we use Dask array and Dask dataframe to construct two random tables with a
shared &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;id&lt;/span&gt;&lt;/code&gt; column. We can play with the number of rows of each table and the
number of keys to make the join challenging in a variety of ways.&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;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="n"&gt;n_rows&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;1000000000&lt;/span&gt;
&lt;span class="n"&gt;n_keys&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;5000000&lt;/span&gt;

&lt;span class="n"&gt;left&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;concat&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;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="n"&gt;n_rows&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_dask_dataframe&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;x&amp;#39;&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;n_keys&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="n"&gt;n_rows&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_dask_dataframe&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;id&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;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="n"&gt;n_rows&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;10000000&lt;/span&gt;

&lt;span class="n"&gt;right&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;concat&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;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="n"&gt;n_rows&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_dask_dataframe&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;y&amp;#39;&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;n_keys&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="n"&gt;n_rows&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_dask_dataframe&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;id&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;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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="send-to-the-gpus"&gt;
&lt;h2&gt;Send to the GPUs&lt;/h2&gt;
&lt;p&gt;We have two Dask dataframes composed of many Pandas dataframes of our random
data. We now map the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cudf.from_pandas&lt;/span&gt;&lt;/code&gt; function across these to make a Dask
dataframe of cuDF dataframes.&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;dask&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cudf&lt;/span&gt;

&lt;span class="n"&gt;gleft&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;left&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_partitions&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cudf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_pandas&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;gright&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;right&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_partitions&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cudf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_pandas&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;gleft&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;gright&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;gleft&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;gright&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# persist data in device memory&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;What’s nice here is that there wasn’t any special
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask_pandas_dataframe_to_dask_cudf_dataframe&lt;/span&gt;&lt;/code&gt; function. Dask composed nicely
with cuDF. We didn’t need to do anything special to support it.&lt;/p&gt;
&lt;p&gt;We’ll also persisted the data in device memory.&lt;/p&gt;
&lt;p&gt;After this, simple operations are easy and fast and use our eight GPUs.&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;gleft&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;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="c1"&gt;# this takes 250ms&lt;/span&gt;
&lt;span class="go"&gt;500004719.254711&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="join"&gt;
&lt;h2&gt;Join&lt;/h2&gt;
&lt;p&gt;We’ll use standard Pandas syntax to merge the datasets, persist the result in
RAM, and then wait&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;out&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;gleft&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;merge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;gright&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;on&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;id&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;  &lt;span class="c1"&gt;# this is lazy&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/29/cudf-joins.md&lt;/span&gt;, line 95)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="profile-and-analyze-results"&gt;
&lt;h1&gt;Profile and analyze results&lt;/h1&gt;
&lt;p&gt;We now look at the Dask diagnostic plots for this computation.&lt;/p&gt;
&lt;section id="task-stream-and-communication"&gt;
&lt;h2&gt;Task stream and communication&lt;/h2&gt;
&lt;p&gt;When we look at Dask’s task stream plot we see that each of our eight threads
(each of which manages a single GPU) spent most of its time in communication
(red is communication time). The actual merge and concat tasks are quite fast
relative to the data transfer time.&lt;/p&gt;
&lt;iframe src="https://matthewrocklin.com/raw-host/dask-cudf-joins.html"
        width="800"
        height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;That’s not too surprising. For this computation I’ve turned off any attempt to
communicate between devices (more on this below) so the data is being moved
from the GPU to the CPU memory, then serialized and put onto a TCP socket.
We’re moving tens of GB on a single machine, so we’re seeing about 1GB/s total
throughput of the system, which is typical for TCP-on-localhost in Python.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="flamegraph-of-computation"&gt;
&lt;h2&gt;Flamegraph of computation&lt;/h2&gt;
&lt;p&gt;We can also look more deeply at the computational costs in Dask’s
flamegraph-style plot. This shows which lines of our functions were taking up
the most time (down to the Python level at least).&lt;/p&gt;
&lt;iframe src="http://matthewrocklin.com/raw-host/dask-cudf-join-profile.html"
        width="800"
        height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;This &lt;a class="reference external" href="http://www.brendangregg.com/flamegraphs.html"&gt;Flame graph&lt;/a&gt; shows which
lines of cudf code we spent time on while computing (excluding the main
communication costs mentioned above). It may be interesting for those trying
to further optimize performance. It shows that most of our costs are in memory
allocation. Like communication, this has actually also been fixed in RAPIDS’
optional memory management pool, it just isn’t default yet, so I didn’t use it
here.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/29/cudf-joins.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&gt;
&lt;section id="plans-for-efficient-communication"&gt;
&lt;h1&gt;Plans for efficient communication&lt;/h1&gt;
&lt;p&gt;The cuDF library actually has a decent approach to single-node multi-GPU
communication that I’ve intentionally turned off for this experiment. That
approach cleverly used Dask to communicate device pointer information using
Dask’s normal channels (this is small and fast) and then used that information
to initiate a side-channel communication for the bulk of the data. This
approach was effective, but somewhat fragile. I’m inclined to move on for it
in favor of …&lt;/p&gt;
&lt;p&gt;UCX. The &lt;a class="reference external" href="http://www.openucx.org/"&gt;UCX&lt;/a&gt; project provides a single API that
wraps around several transports like TCP, Infiniband, shared memory, and also
GPU-specific transports. UCX claims to find the best way to communicate data
between two points given the hardware available. If Dask were able to use this
for communication then it would provide both efficient GPU-to-GPU communication
on a single machine, and also efficient inter-machine communication when
efficient networking hardware like Infiniband was present, even outside the
context of GPUs.&lt;/p&gt;
&lt;p&gt;There is some work we need to do here:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;We need to make a Python wrapper around UCX&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We need to make an optional &lt;a class="reference external" href="https://distributed.dask.org/en/latest/communications.html"&gt;Dask Comm&lt;/a&gt;
around this ucx-py library that allows users to specify endpoints like
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ucx://path-to-scheduler&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We need to make Python memoryview-like objects that refer to device memory&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;…&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This work is already in progress by &lt;a class="reference external" href="https://github.com/Akshay-Venkatesh"&gt;Akshay
Vekatesh&lt;/a&gt;, who works on UCX, and &lt;a class="reference external" href="https://tomaugspurger.github.io/"&gt;Tom
Augspurger&lt;/a&gt; a core Dask/Pandas developer. I
suspect that they’ll write about it soon. I’m looking forward to seeing what
comes of it, both for Dask and for high performance Python generally.&lt;/p&gt;
&lt;p&gt;It’s worth pointing out that this effort won’t just help GPU users. It should
help anyone on advanced networking hardware, including the mainstream
scientific HPC community.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/29/cudf-joins.md&lt;/span&gt;, line 172)&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;Summary&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/2019/01/29/cudf-joins.md&lt;/span&gt;, line 172); &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: “summary”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Single-node Mutli-GPU joins have a lot of promise. In fact, earlier RAPIDS
developers got this running much faster than I was able to do above through the
clever communication tricks I briefly mentioned. The main purpose of this post
is to provide a benchmark for joins that we can use in the future, and to
highlight when communication can be essential in parallel computing.&lt;/p&gt;
&lt;p&gt;Now that GPUs have accelerated the computation time of each of our chunks of
work we increasingly find that other systems become the bottleneck. We didn’t
care as much about communication before because computational costs were
comparable. Now that computation is an order of magnitude cheaper, other
aspects of our stack become much more important.&lt;/p&gt;
&lt;p&gt;I’m looking forward to seeing where this goes.&lt;/p&gt;
&lt;section id="come-help"&gt;
&lt;h2&gt;Come help!&lt;/h2&gt;
&lt;p&gt;If the work above sounds interesting to you then come help!
There is a lot of low-hanging and high impact work to do.&lt;/p&gt;
&lt;p&gt;If you’re interested in being paid to focus more on these topics, then consider
applying for a job. NVIDIA’s RAPIDS team is looking to hire engineers for Dask
development with GPUs and other data analytics library development projects.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://nvidia.wd5.myworkdayjobs.com/en-US/NVIDIAExternalCareerSite/job/US-TX-Austin/Senior-Library-Software-Engineer---RAPIDS_JR1919608-1"&gt;Senior Library Software Engineer - RAPIDS&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/01/29/cudf-joins/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="GPU" label="GPU"/>
    <category term="dataframe" label="dataframe"/>
    <published>2019-01-29T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/01/23/dask-1.1.0/</id>
    <title>Dask Release 1.1.0</title>
    <updated>2019-01-23T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;I’m pleased to announce the release of Dask version 1.1.0. This is a major
release with bug fixes and new features. The last release was 1.0.0 on
2018-11-29.
This blogpost outlines notable changes since the last release.&lt;/p&gt;
&lt;p&gt;You can conda install Dask:&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 pip install from PyPI:&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[complete] --upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Full changelogs are available here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/blob/master/docs/source/changelog.rst"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/docs/source/changelog.rst"&gt;dask/distributed&lt;/a&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/2019/01/23/dask-1.1.0.md&lt;/span&gt;, line 26)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="notable-changes"&gt;

&lt;p&gt;A lot of work has happened over the last couple months, and we encourage people
to look through the changelog to get a sense of the kinds of incremental
changes that developers are working on.&lt;/p&gt;
&lt;p&gt;There are also a few notable changes in this release that we’ll highlight here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Support for the recent Numpy 1.16 and Pandas 0.24 releases&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Support for Pandas Extension Arrays (see &lt;a class="reference internal" href="../2019/01/22/dask-extension-arrays/"&gt;&lt;span class="doc std std-doc"&gt;Tom Augspurger’s post on the topic&lt;/span&gt;&lt;/a&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;High level graph in Dask dataframe and operator fusion in simple cases&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Increased support for other libraries that look enough like Numpy and Pandas
to work within Dask Array/Dataframe&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/23/dask-1.1.0.md&lt;/span&gt;, line 40)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="support-for-numpy-1-16-and-pandas-0-24"&gt;
&lt;h1&gt;Support for Numpy 1.16 and Pandas 0.24&lt;/h1&gt;
&lt;p&gt;Both Numpy and Pandas have been evolving quickly over the last few months.
We’re excited about the changes to extensibility arriving in both libraries.
The Dask array/dataframe submodules have been updated to work well with these
recent changes.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/23/dask-1.1.0.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="pandas-extension-arrays"&gt;
&lt;h1&gt;Pandas Extension Arrays&lt;/h1&gt;
&lt;p&gt;In particular Dask Dataframe supports Pandas Extension arrays,
meaning that it’s easier to use third party Pandas packages like CyberPandas or
Fletcher in parallel with Dask Dataframe.&lt;/p&gt;
&lt;p&gt;For more information see &lt;a class="reference internal" href="../2019/01/22/dask-extension-arrays/"&gt;&lt;span class="doc std std-doc"&gt;Tom Augspurger’s post&lt;/span&gt;&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/2019/01/23/dask-1.1.0.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="high-level-graphs-in-dask-dataframe"&gt;
&lt;h1&gt;High Level Graphs in Dask Dataframe&lt;/h1&gt;
&lt;p&gt;For a while Dask array has had some high level graphs for “atop” operations
(elementwise, broadcasting, transpose, tensordot, reductions), which allow for
reduced overhead and task fusion on computations within this class.&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;y&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;exp&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;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;T&lt;/span&gt;  &lt;span class="c1"&gt;# These operations get fused to a single task&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We’ve renamed &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;atop&lt;/span&gt;&lt;/code&gt; to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;blockwise&lt;/span&gt;&lt;/code&gt; to be a bit more generic, and have also
started applying it to Dask Dataframe, which helps to reduce overhead
substantially when doing computations with many simple operations.&lt;/p&gt;
&lt;p&gt;This still needs to be improved to increase the class of cases where it works,
but we’re already seeing nice speedups on previously unseen workloads.&lt;/p&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;da.atop&lt;/span&gt;&lt;/code&gt; function has been deprecated in favor of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;da.blockwise&lt;/span&gt;&lt;/code&gt;. There
is now also a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dd.blockwise&lt;/span&gt;&lt;/code&gt; which shares a common code path.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/23/dask-1.1.0.md&lt;/span&gt;, line 75)&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="non-pandas-dataframe-and-non-numpy-array-types"&gt;
&lt;h1&gt;Non-Pandas dataframe and Non-Numpy array types&lt;/h1&gt;
&lt;p&gt;We’re working to make Dask a bit more agnostic to the types of in-memory array
and dataframe objects that it can manipulate. Rather than having Dask Array be
a grid of Numpy arrays and Dask Dataframe be a sequence of Pandas dataframes,
we’re relaxing that constraint to a grid of &lt;em&gt;Numpy-like&lt;/em&gt; arrays and a sequence
of &lt;em&gt;Pandas-like&lt;/em&gt; dataframes.&lt;/p&gt;
&lt;p&gt;This is an ongoing effort that has targetted alternate backends like
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scipy.sparse&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pydata/sparse&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cupy&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cudf&lt;/span&gt;&lt;/code&gt; and other systems.&lt;/p&gt;
&lt;p&gt;There were some recent posts on
&lt;a class="reference internal" href="../2019/01/03/dask-array-gpus-first-steps/"&gt;&lt;span class="doc std std-doc"&gt;arrays&lt;/span&gt;&lt;/a&gt; and
&lt;a class="reference internal" href="../2019/01/13/dask-cudf-first-steps/"&gt;&lt;span class="doc std std-doc"&gt;dataframes&lt;/span&gt;&lt;/a&gt; that show proofs of
concept for this with GPUs.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/23/dask-1.1.0.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="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;There have been several releases since the last time we had a release blogpost.
The following people contributed to the dask/dask repository since the 0.19.0
release on September 5th:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Anderson Banihirwe&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Antonino Ingargiola&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Armin Berres&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bart Broere&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Carlos Valiente&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Daniel Li&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Daniel Saxton&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;David Hoese&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Diane Trout&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Damien Garaud&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Elliott Sales de Andrade&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Eric Wolak&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Gábor Lipták&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Guido Imperiale&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Guillaume Eynard-Bontemps&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Itamar Turner-Trauring&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Bourbeau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jan Koch&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Javad&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jendrik Jördening&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jonathan Fraine&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Johnnie Gray&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Julia Signell&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Justin Dennison&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;M. Farrajota&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Marco Neumann&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Mark Harfouche&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Markus Gonser&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthias Bussonnier&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Mina Farid&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Paul Vecchio&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Prabakaran Kumaresshan&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Rahul Vaidya&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stephan Hoyer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stuart Berg&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;TakaakiFuruse&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Takahiro Kojima&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Yu Feng&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Zhenqing Li&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&amp;#64;milesial&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&amp;#64;samc0de&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&amp;#64;slnguyen&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The following people contributed to the dask/distributed repository since the 0.19.0
release on September 5th:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Adam Klein&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brett Naul&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Daniel Farrell&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Diane Trout&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dirk Petersen&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Eric Ma&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Gaurav Sheni&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Guillaume Eynard-Bontemps&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Loïc Estève&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Marius van Niekerk&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Michael Wheeler&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;MikeG&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;NotSqrt&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Peter Killick&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Roy Wedge&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Russ Bubley&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stephan Hoyer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&amp;#64;tjb900&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Rochette&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&amp;#64;fjetter&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/01/23/dask-1.1.0/"/>
    <summary>I’m pleased to announce the release of Dask version 1.1.0. This is a major
release with bug fixes and new features. The last release was 1.0.0 on
2018-11-29.
This blogpost outlines notable changes since the last release.</summary>
    <category term="release" label="release"/>
    <published>2019-01-23T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/01/22/dask-extension-arrays/</id>
    <title>Extension Arrays in Dask DataFrame</title>
    <updated>2019-01-22T00:00:00+00:00</updated>
    <author>
      <name>Tom Augspurger</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://anaconda.com"&gt;Anaconda Inc&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/22/dask-extension-arrays.md&lt;/span&gt;, line 11)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;Dask DataFrame works well with pandas’ new Extension Array interface, including
third-party extension arrays. This lets Dask&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;easily support pandas’ new extension arrays, like their new &lt;a class="reference external" href="http://pandas.pydata.org/pandas-docs/version/0.24/whatsnew/v0.24.0.html#optional-integer-na-support"&gt;nullable integer
array&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;support third-party extension array arrays, like &lt;a class="reference external" href="https://cyberpandas.readthedocs.io"&gt;cyberpandas’s&lt;/a&gt;
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;IPArray&lt;/span&gt;&lt;/code&gt;&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/2019/01/22/dask-extension-arrays.md&lt;/span&gt;, line 21)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="background"&gt;
&lt;h1&gt;Background&lt;/h1&gt;
&lt;p&gt;Pandas 0.23 introduced the &lt;a class="reference external" href="http://pandas.pydata.org/pandas-docs/version/0.24/extending.html#extension-types"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ExtensionArray&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;, a way to store things other
than a simple NumPy array in a DataFrame or Series. Internally pandas uses this
for data types that aren’t handled natively by NumPy like datetimes with
timezones, Categorical, or (the new!) nullable integer 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="n"&gt;s&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Series&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;date_range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;2000&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;periods&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;tz&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;US/Central&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;
&lt;span class="go"&gt;0   2000-01-01 00:00:00-06:00&lt;/span&gt;
&lt;span class="go"&gt;1   2000-01-02 00:00:00-06:00&lt;/span&gt;
&lt;span class="go"&gt;2   2000-01-03 00:00:00-06:00&lt;/span&gt;
&lt;span class="go"&gt;3   2000-01-04 00:00:00-06:00&lt;/span&gt;
&lt;span class="go"&gt;dtype: datetime64[ns, US/Central]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe&lt;/span&gt;&lt;/code&gt; has always supported the extension types that pandas defines.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_pandas&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;npartitions&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="go"&gt;Dask Series Structure:&lt;/span&gt;
&lt;span class="go"&gt;npartitions=2&lt;/span&gt;
&lt;span class="go"&gt;0    datetime64[ns, US/Central]&lt;/span&gt;
&lt;span class="go"&gt;2                           ...&lt;/span&gt;
&lt;span class="go"&gt;3                           ...&lt;/span&gt;
&lt;span class="go"&gt;dtype: datetime64[ns, US/Central]&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: from_pandas, 2 tasks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/22/dask-extension-arrays.md&lt;/span&gt;, line 52)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="the-challenge"&gt;
&lt;h1&gt;The Challenge&lt;/h1&gt;
&lt;p&gt;Newer versions of pandas allow third-party libraries to write custom extension
arrays. These arrays can be placed inside a DataFrame or Series, and work
just as well as any extension array defined within pandas itself. However,
third-party extension arrays provide a slight challenge for Dask.&lt;/p&gt;
&lt;p&gt;Recall: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe&lt;/span&gt;&lt;/code&gt; is lazy. We use a familiar pandas-like API to build up
a task graph, rather than executing immediately. But if Dask DataFrame is lazy,
then how do things like the following work?&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;A&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="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="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;3&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_pandas&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;npartitions&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;
&lt;span class="go"&gt;Index([&amp;#39;B&amp;#39;], dtype=&amp;#39;object&amp;#39;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ddf[['B']]&lt;/span&gt;&lt;/code&gt; (lazily) selects the column &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;'B'&lt;/span&gt;&lt;/code&gt; from the dataframe. But accessing
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.columns&lt;/span&gt;&lt;/code&gt; &lt;em&gt;immediately&lt;/em&gt; returns a pandas Index object with just the selected
columns.&lt;/p&gt;
&lt;p&gt;No real computation has happened (you could just as easily swap out the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_pandas&lt;/span&gt;&lt;/code&gt; for a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dd.read_parquet&lt;/span&gt;&lt;/code&gt; on a larger-than-memory dataset, and the
behavior would be the same). Dask is able to do these kinds of “metadata-only”
computations, where the output depends only on the columns and the dtypes,
without executing the task graph. Internally, Dask does this by keeping a pair
of dummy pandas DataFrames on each Dask DataFrame.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_meta&lt;/span&gt;
&lt;span class="go"&gt;Empty DataFrame&lt;/span&gt;
&lt;span class="go"&gt;Columns: [A, B]&lt;/span&gt;
&lt;span class="go"&gt;Index: []&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_meta_nonempty&lt;/span&gt;
&lt;span class="go"&gt;ddf._meta_nonempty&lt;/span&gt;
&lt;span class="go"&gt;   A  B&lt;/span&gt;
&lt;span class="go"&gt;0  1  1&lt;/span&gt;
&lt;span class="go"&gt;1  1  1&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We need the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;_meta_nonempty&lt;/span&gt;&lt;/code&gt;, since some operations in pandas behave differently
on an Empty DataFrame than on a non-empty one (either by design or,
occasionally, a bug in pandas).&lt;/p&gt;
&lt;p&gt;The issue with third-party extension arrays is that Dask doesn’t know what
values to put in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;_meta_nonempty&lt;/span&gt;&lt;/code&gt;. We’re quite happy to do it for each NumPy
dtype and each of pandas’ own extension dtypes. But any third-party library
could create an ExtensionArray for any type, and Dask would have no way of
knowing what’s a valid value for it.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/22/dask-extension-arrays.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="the-solution"&gt;
&lt;h1&gt;The Solution&lt;/h1&gt;
&lt;p&gt;Rather than Dask guessing what values to use for the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;_meta_nonempty&lt;/span&gt;&lt;/code&gt;, extension
array authors (or users) can register their extension dtype with Dask. Once
registered, Dask will be able to generate the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;_meta_nonempty&lt;/span&gt;&lt;/code&gt;, and things
should work fine from there. For example, we can register the dummy &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DecimalArray&lt;/span&gt;&lt;/code&gt;
that pandas uses for testing (this isn’t part of pandas’ public API) with Dask.&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;decimal&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;Decimal&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;pandas.tests.extension.decimal&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;DecimalArray&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;DecimalDtype&lt;/span&gt;

&lt;span class="c1"&gt;# The actual registration that would be done in the 3rd-party library&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe.extensions&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;make_array_nonempty&lt;/span&gt;


&lt;span class="nd"&gt;@make_array_nonempty&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;DecimalDtype&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;_&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dtype&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;DecimalArray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_from_sequence&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;NaN&amp;#39;&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="n"&gt;dtype&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now users of that extension type can place those arrays inside a Dask DataFrame
or Series.&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;A&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;DecimalArray&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;1.0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;2.0&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;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;3.0&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;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_pandas&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&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;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;
&lt;span class="go"&gt;Dask DataFrame Structure:&lt;/span&gt;
&lt;span class="go"&gt;                     A&lt;/span&gt;
&lt;span class="go"&gt;npartitions=1&lt;/span&gt;
&lt;span class="go"&gt;0              decimal&lt;/span&gt;
&lt;span class="go"&gt;2                  ...&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: from_pandas, 1 tasks&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dtypes&lt;/span&gt;
&lt;span class="go"&gt;A    decimal&lt;/span&gt;
&lt;span class="go"&gt;dtype: object&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And from there, the usual operations just as they would in pandas.&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;random&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;choices&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;A&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;DecimalArray&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;choices&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;1.0&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;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;2.0&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;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="gp"&gt;... &lt;/span&gt;                   &lt;span class="s2"&gt;&amp;quot;B&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;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;choice&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;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;100&lt;/span&gt;&lt;span class="p"&gt;,))})&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_pandas&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&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="go"&gt;In [35]: ddf.groupby(&amp;quot;A&amp;quot;).B.mean().compute()&lt;/span&gt;
&lt;span class="go"&gt;Out[35]:&lt;/span&gt;
&lt;span class="go"&gt;A&lt;/span&gt;
&lt;span class="go"&gt;1.0    1.50&lt;/span&gt;
&lt;span class="go"&gt;2.0    1.48&lt;/span&gt;
&lt;span class="go"&gt;Name: B, dtype: float64&lt;/span&gt;

&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/22/dask-extension-arrays.md&lt;/span&gt;, line 165)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="the-real-lesson"&gt;
&lt;h1&gt;The Real Lesson&lt;/h1&gt;
&lt;p&gt;It’s neat that Dask now supports extension arrays. But to me, the exciting thing
is just how little work this took. The
&lt;a class="reference external" href="https://github.com/dask/dask/pull/4379/files"&gt;PR&lt;/a&gt; implementing support for
third-party extension arrays is quite short, just defining the object that
third-parties register with, and using it to generate the data when dtype is
detected. Supporting the three new extension arrays in pandas 0.24.0
(&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;IntegerArray&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;PeriodArray&lt;/span&gt;&lt;/code&gt;, and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;IntervalArray&lt;/span&gt;&lt;/code&gt;), takes a handful of lines
of code&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@make_array_nonempty&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&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;Interval&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;_&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dtype&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;IntervalArray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_breaks&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;closed&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;closed&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;


&lt;span class="nd"&gt;@make_array_nonempty&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&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;Period&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;_&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dtype&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;period_array&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mi"&gt;2000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2001&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;freq&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;freq&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;


&lt;span class="nd"&gt;@make_array_nonempty&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;_IntegerDtype&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;_&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dtype&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;integer_array&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;None&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="n"&gt;dtype&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Dask benefits directly from improvements made to pandas. Dask didn’t have to
build out a new parallel extension array interface, and reimplement all the new
extension arrays using the parallel interface. We just re-used what pandas
already did, and it fits into the existing Dask structure.&lt;/p&gt;
&lt;p&gt;For third-party extension array authors, like &lt;a class="reference external" href="https://cyberpandas.readthedocs.io"&gt;cyberpandas&lt;/a&gt;, the
work is similarly minimal. They don’t need to re-implement everything from the
ground up, just to play well with Dask.&lt;/p&gt;
&lt;p&gt;This highlights the importance of one of the Dask project’s core values: working
with the community. If you visit &lt;a class="reference external" href="https://dask.org"&gt;dask.org&lt;/a&gt;, you’ll see
phrases like&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Integrates with existing projects&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;and&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Built with the broader community&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;At the start of Dask, the developers &lt;em&gt;could&lt;/em&gt; have gone off and re-written pandas
or NumPy from scratch to be parallel friendly (though we’d probably still be
working on that part today, since that’s such a massive undertaking). Instead,
the Dask developers worked with the community, occasionally nudging it in
directions that would help out dask. For example, many places in pandas &lt;a class="reference external" href="http://matthewrocklin.com/blog/work/2015/03/10/PyData-GIL"&gt;held
the GIL&lt;/a&gt;, preventing
thread-based parallelism. Rather than abandoning pandas, the Dask and pandas
developers worked together to release the GIL where possible when it was a
bottleneck for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe&lt;/span&gt;&lt;/code&gt;. This benefited Dask and anyone else trying to
do thread-based parallelism with pandas DataFrames.&lt;/p&gt;
&lt;p&gt;And now, when pandas introduces new features like nullable integers,
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe&lt;/span&gt;&lt;/code&gt; just needs to register it as an extension type and immediately
benefits from it. And third-party extension array authors can do the same for
their extension arrays.&lt;/p&gt;
&lt;p&gt;If you’re writing an ExtensionArray, make sure to add it to the &lt;a class="reference external" href="http://pandas.pydata.org/pandas-docs/version/0.24/ecosystem.html#extension-data-types"&gt;pandas
ecosystem&lt;/a&gt; page, and register it with Dask!&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/01/22/dask-extension-arrays/"/>
    <summary>This work is supported by Anaconda Inc</summary>
    <category term="dask" label="dask"/>
    <category term="dataframe" label="dataframe"/>
    <published>2019-01-22T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/01/13/dask-cudf-first-steps/</id>
    <title>Dask, Pandas, and GPUs: first steps</title>
    <updated>2019-01-13T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/13/dask-cudf-first-steps.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;We’re building a distributed GPU Pandas dataframe out of
&lt;a class="reference external" href="https://github.com/rapidsai/cudf"&gt;cuDF&lt;/a&gt; and
&lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe.html"&gt;Dask Dataframe&lt;/a&gt;.
This effort is young.&lt;/p&gt;
&lt;p&gt;This post describes the current situation,
our general approach,
and gives examples of what does and doesn’t work today.
We end with some notes on scaling performance.&lt;/p&gt;
&lt;p&gt;You can also view the experiment in this post as
&lt;a class="reference external" href="https://gist.github.com/mrocklin/4b1b80d1ae07ec73f75b2a19c8e90e2e"&gt;a notebook&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;And here is a table of results:&lt;/p&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
  &lt;tr&gt;
    &lt;th&gt;Architecture&lt;/th&gt;
    &lt;th&gt;Time&lt;/th&gt;
    &lt;th&gt;Bandwidth&lt;/th&gt;
  &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt; Single CPU Core &lt;/th&gt;
      &lt;td&gt; 3min 14s &lt;/td&gt;
      &lt;td&gt; 50 MB/s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; Eight CPU Cores &lt;/th&gt;
      &lt;td&gt; 58s &lt;/td&gt;
      &lt;td&gt; 170 MB/s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; Forty CPU Cores &lt;/th&gt;
      &lt;td&gt; 35s &lt;/td&gt;
      &lt;td&gt; 285 MB/s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; One GPU &lt;/th&gt;
      &lt;td&gt; 11s &lt;/td&gt;
      &lt;td&gt; 900 MB/s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; Eight GPUs &lt;/th&gt;
      &lt;td&gt; 5s &lt;/td&gt;
      &lt;td&gt; 2000 MB/s &lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/13/dask-cudf-first-steps.md&lt;/span&gt;, line 63)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="building-blocks-cudf-and-dask"&gt;
&lt;h1&gt;Building Blocks: cuDF and Dask&lt;/h1&gt;
&lt;p&gt;Building a distributed GPU-backed dataframe is a large endeavor.
Fortunately we’re starting on a good foundation and
can assemble much of this system from existing components:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;The &lt;a class="reference external" href="https://github.com/rapidsai/cudf"&gt;cuDF&lt;/a&gt; library aims to implement the
Pandas API on the GPU. It gets good speedups on standard operations like
reading CSV files, filtering and aggregating columns, joins, and so on.&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;cudf&lt;/span&gt;  &lt;span class="c1"&gt;# looks and feels like Pandas, but runs on the GPU&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;cudf&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;myfile.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;name&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Alice&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;id&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;value&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;cuDF is part of the growing &lt;a class="reference external" href="https://rapids.ai"&gt;RAPIDS initiative&lt;/a&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe.html"&gt;Dask Dataframe&lt;/a&gt;
library provides parallel algorithms around the Pandas API. It composes
large operations like distributed groupbys or distributed joins from a task
graph of many smaller single-node groupbys or joins accordingly (and many
&lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe-api.html"&gt;other operations&lt;/a&gt;).&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;  &lt;span class="c1"&gt;# looks and feels like Pandas, but runs in parallel&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;dd&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;myfile.*.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;name&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Alice&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;id&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;value&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The &lt;a class="reference external" href="https://distributed.dask.org"&gt;Dask distributed task scheduler&lt;/a&gt;
provides general-purpose parallel execution given complex task graphs.
It’s good for adding multi-node computing into an existing codebase.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Given these building blocks,
our approach is to make the cuDF API close enough to Pandas that
we can reuse the Dask Dataframe 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/2019/01/13/dask-cudf-first-steps.md&lt;/span&gt;, line 105)&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="benefits-and-challenges-to-this-approach"&gt;
&lt;h1&gt;Benefits and Challenges to this approach&lt;/h1&gt;
&lt;p&gt;This approach has a few benefits:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;We get to reuse the parallel algorithms found in Dask Dataframe originally designed for Pandas.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It consolidates the development effort within a single codebase so that
future effort spent on CPU Dataframes benefits GPU Dataframes and vice
versa. Maintenance costs are shared.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;By building code that works equally with two DataFrame implementations
(CPU and GPU) we establish conventions and protocols that will
make it easier for other projects to do the same, either with these two
Pandas-like libraries, or with future Pandas-like libraries.&lt;/p&gt;
&lt;p&gt;This approach also aims to demonstrate that the ecosystem should support Pandas-like
libraries, rather than just Pandas. For example, if
(when?) the Arrow library develops a computational system then we’ll be in
a better place to roll that in as well.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;When doing any refactor we tend to clean up existing code.&lt;/p&gt;
&lt;p&gt;For example, to make dask dataframe ready for a new GPU Parquet reader
we end up &lt;a class="reference external" href="https://github.com/dask/dask/pull/4336"&gt;refactoring and simplifying our Parquet I/O logic&lt;/a&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The approach also has some drawbacks. Namely, it places API pressure on cuDF to match Pandas so:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;Slight differences in API now cause larger problems, such as these:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/251"&gt;Join column ordering differs rapidsai/cudf #251&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/483#issuecomment-453218151"&gt;Groupby aggregation column ordering differs rapidsai/cudf #483&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;cuDF has some pressure on it to repeat what some believe to be mistakes in
the Pandas API.&lt;/p&gt;
&lt;p&gt;For example, cuDF today supports missing values arguably more sensibly than
Pandas. Should cuDF have to revert to the old way of doing things
just to match Pandas semantics? Dask Dataframe will probably need
to be more flexible in order to handle evolution and small differences
in semantics.&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/2019/01/13/dask-cudf-first-steps.md&lt;/span&gt;, line 146)&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="alternatives"&gt;
&lt;h1&gt;Alternatives&lt;/h1&gt;
&lt;p&gt;We could also write a new dask-dataframe-style project around cuDF that deviates
from the Pandas/Dask Dataframe API. Until recently this
has actually been the approach, and the
&lt;a class="reference external" href="https://github.com/rapidsai/dask-cudf"&gt;dask-cudf&lt;/a&gt; project did exactly this.
This was probably a good choice early on to get started and prototype things.
The project was able to implement a wide range of functionality including
groupby-aggregations, joins, and so on using
&lt;a class="reference external" href="https://docs.dask.org/en/latest/delayed.html"&gt;dask delayed&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;We’re redoing this now on top of dask dataframe though, which means that we’re
losing some functionality that dask-cudf already had, but hopefully the
functionality that we add now will be more stable and established on a firmer
base.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/13/dask-cudf-first-steps.md&lt;/span&gt;, line 162)&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="status-today"&gt;
&lt;h1&gt;Status Today&lt;/h1&gt;
&lt;p&gt;Today very little works, but what does is decently smooth.&lt;/p&gt;
&lt;p&gt;Here is a simple example that reads some data from many CSV files,
picks out a column,
and does some aggregations.&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_cuda&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;LocalCUDACluster&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_cudf&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCUDACluster&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# runs on eight local GPUs&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;gdf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask_cudf&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;data/nyc/many/*.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# wrap around many CSV files&lt;/span&gt;

&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;gdf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;passenger_count&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="mi"&gt;184464740&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;em&gt;Also note, NYC Taxi ridership is significantly less than it was a few years ago&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/13/dask-cudf-first-steps.md&lt;/span&gt;, line 186)&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-i-m-excited-about-in-the-example-above"&gt;
&lt;h1&gt;What I’m excited about in the example above&lt;/h1&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;All of the infrastructure surrounding the cuDF code, like the cluster setup,
diagnostics, JupyterLab environment, and so on, came for free, like any
other new Dask project.&lt;/p&gt;
&lt;p&gt;Here is an image of my JupyterLab setup&lt;/p&gt;
&lt;a href="https://matthewrocklin.com/blog/images/dask-cudf-environment.png"&gt;
  &lt;img src="https://matthewrocklin.com/blog/images/dask-cudf-environment.png"
       alt="Dask + CUDA + cuDF JupyterLab environment"
       width="70%"&gt;
&lt;/a&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df&lt;/span&gt;&lt;/code&gt; object is actually just a normal Dask Dataframe. We didn’t have to
write new &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__repr__&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__add__&lt;/span&gt;&lt;/code&gt;, or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.sum()&lt;/span&gt;&lt;/code&gt; implementations, and probably
many functions we didn’t think about work well today (though also many
don’t).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We’re tightly integrated and more connected to other systems. For example, if
we wanted to convert our dask-cudf-dataframe to a dask-pandas-dataframe then
we would just use the cuDF &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;to_pandas&lt;/span&gt;&lt;/code&gt; function:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_partitions&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cudf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_pandas&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We don’t have to write anything special like a separate &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.to_dask_dataframe&lt;/span&gt;&lt;/code&gt;
method or handle other special cases.&lt;/p&gt;
&lt;p&gt;Dask parallelism is orthogonal to the choice of CPU or GPU.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It’s easy to switch hardware. By avoiding separate &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-cudf&lt;/span&gt;&lt;/code&gt; code paths
it’s easier to add cuDF to an existing Dask+Pandas codebase to run on GPUs,
or to remove cuDF and use Pandas if we want our code to be runnable without GPUs.&lt;/p&gt;
&lt;p&gt;There are more examples of this in the scaling section below.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/13/dask-cudf-first-steps.md&lt;/span&gt;, line 224)&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-s-wrong-with-the-example-above"&gt;
&lt;h1&gt;What’s wrong with the example above&lt;/h1&gt;
&lt;p&gt;In general the answer is &lt;strong&gt;many small things&lt;/strong&gt;.&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cudf.read_csv&lt;/span&gt;&lt;/code&gt; function doesn’t yet support reading chunks from a
single CSV file, and so doesn’t work well with very large CSV files. We
had to split our large CSV files into many smaller CSV files first with
normal Dask+Pandas:&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;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&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;few-large/*.csv&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;repartition&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;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;to_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;many-small/*.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;(See &lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/568"&gt;rapidsai/cudf #568&lt;/a&gt;)&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Many operations that used to work in dask-cudf like groupby-aggregations
and joins no longer work. We’re going to need to slightly modify many cuDF
APIs over the next couple of months to more closely match their Pandas
equivalents.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;I ran the timing cell twice because it currently takes a few seconds to
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;import&lt;/span&gt; &lt;span class="pre"&gt;cudf&lt;/span&gt;&lt;/code&gt; today.
&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/627"&gt;rapidsai/cudf #627&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We had to make Dask Dataframe a bit more flexible and assume less about its
constituent dataframes being exactly Pandas dataframes. (see
&lt;a class="reference external" href="https://github.com/dask/dask/pull/4359"&gt;dask/dask #4359&lt;/a&gt; and
&lt;a class="reference external" href="https://github.com/dask/dask/pull/4375"&gt;dask/dask #4375&lt;/a&gt; for examples).
I suspect that there will by many more small changes like
these necessary in the future.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;These problems are representative of dozens more similar issues. They are
all fixable and indeed, many are actively being fixed today by the &lt;a class="reference external" href="https://github.com/rapidsai/cudf/graphs/contributors"&gt;good folks
working on RAPIDS&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/2019/01/13/dask-cudf-first-steps.md&lt;/span&gt;, line 262)&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="near-term-schedule"&gt;
&lt;h1&gt;Near Term Schedule&lt;/h1&gt;
&lt;p&gt;The RAPIDS group is currently busy working to release 0.5, which includes some
of the fixes necessary to run the example above, and also many unrelated
stability improvements. This will probably keep them busy for a week or two
during which I don’t expect to see much Dask + cuDF work going on other than
planning.&lt;/p&gt;
&lt;p&gt;After that, Dask parallelism support will be a top priority, so
I look forward to seeing some rapid progress here.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/13/dask-cudf-first-steps.md&lt;/span&gt;, line 273)&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="scaling-results"&gt;
&lt;h1&gt;Scaling Results&lt;/h1&gt;
&lt;p&gt;In &lt;a class="reference internal" href="../2019/01/03/dask-array-gpus-first-steps/"&gt;&lt;span class="doc std std-doc"&gt;my last post about combining Dask Array with CuPy&lt;/span&gt;&lt;/a&gt;,
a GPU-accelerated Numpy,
we saw impressive speedups from using many GPUs on a simple problem that
manipulated some simple random data.&lt;/p&gt;
&lt;section id="dask-array-cupy-on-random-data"&gt;
&lt;h2&gt;Dask Array + CuPy on Random Data&lt;/h2&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
  &lt;tr&gt;
    &lt;th&gt;Architecture&lt;/th&gt;
    &lt;th&gt;Time&lt;/th&gt;
  &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt; Single CPU Core &lt;/th&gt;
      &lt;td&gt; 2hr 39min &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; Forty CPU Cores &lt;/th&gt;
      &lt;td&gt; 11min 30s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; One GPU &lt;/th&gt;
      &lt;td&gt; 1 min 37s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; Eight GPUs &lt;/th&gt;
      &lt;td&gt; 19s &lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;That exercise was easy to scale because it was almost entirely bound by the
computation of creating random data.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-dataframe-cudf-on-csv-data"&gt;
&lt;h2&gt;Dask DataFrame + cuDF on CSV data&lt;/h2&gt;
&lt;p&gt;We did a similar study on the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_csv&lt;/span&gt;&lt;/code&gt; example above, which is bound mostly
by reading CSV data from disk and then parsing it. You can see a notebook
available
&lt;a class="reference external" href="https://gist.github.com/mrocklin/4b1b80d1ae07ec73f75b2a19c8e90e2e"&gt;here&lt;/a&gt;. We
have similar (though less impressive) numbers to present.&lt;/p&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
  &lt;tr&gt;
    &lt;th&gt;Architecture&lt;/th&gt;
    &lt;th&gt;Time&lt;/th&gt;
    &lt;th&gt;Bandwidth&lt;/th&gt;
  &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt; Single CPU Core &lt;/th&gt;
      &lt;td&gt; 3min 14s &lt;/td&gt;
      &lt;td&gt; 50 MB/s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; Eight CPU Cores &lt;/th&gt;
      &lt;td&gt; 58s &lt;/td&gt;
      &lt;td&gt; 170 MB/s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; Forty CPU Cores &lt;/th&gt;
      &lt;td&gt; 35s &lt;/td&gt;
      &lt;td&gt; 285 MB/s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; One GPU &lt;/th&gt;
      &lt;td&gt; 11s &lt;/td&gt;
      &lt;td&gt; 900 MB/s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; Eight GPUs &lt;/th&gt;
      &lt;td&gt; 5s &lt;/td&gt;
      &lt;td&gt; 2000 MB/s &lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;&lt;em&gt;The bandwidth numbers were computed by noting that the data was around 10 GB on disk&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/13/dask-cudf-first-steps.md&lt;/span&gt;, line 359)&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="analysis"&gt;
&lt;h1&gt;Analysis&lt;/h1&gt;
&lt;p&gt;First, I want to emphasize again that it’s easy to test a wide variety of
architectures using this setup because of the Pandas API compatibility between
all of the different projects. We’re seeing a wide range of performance (40x
span) across a variety of different hardware with a wide range of cost points.&lt;/p&gt;
&lt;p&gt;Second, note that this problem scales less well than our
&lt;a class="reference internal" href="../2019/01/03/dask-array-gpus-first-steps/"&gt;&lt;span class="doc std std-doc"&gt;previous example with CuPy&lt;/span&gt;&lt;/a&gt;,
both on CPU and GPU.
I suspect that this is because this example is also bound by I/O and not just
number-crunching. While the jump from single-CPU to single-GPU is large, the
jump from single-CPU to many-CPU or single-GPU to many-GPU is not as large as
we would have liked. For GPUs for example we got around a 2x speedup when we
added 8x as many GPUs.&lt;/p&gt;
&lt;p&gt;At first one might think that this is because we’re saturating disk read speeds.
However two pieces of evidence go against that guess:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;NVIDIA folks familiar with my current hardware inform me that they’re able to get
much higher I/O throughput when they’re careful&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The CPU scaling is similarly poor, despite the fact that it’s obviously not
reaching full I/O bandwidth&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Instead, it’s likely that we’re just not treating our disks and IO pipelines
carefully.&lt;/p&gt;
&lt;p&gt;We might consider working to think more carefully about data locality within a
single machine. Alternatively, we might just choose to use a smaller machine,
or many smaller machines. My team has been asking me to start playing with
some cheaper systems than a DGX, I may experiment with those soon. It may be
that for data-loading and pre-processing workloads the previous wisdom of “pack
as much computation as you can into a single box” no longer holds
(without us doing more work that is).&lt;/p&gt;
&lt;section id="come-help"&gt;
&lt;h2&gt;Come help&lt;/h2&gt;
&lt;p&gt;If the work above sounds interesting to you then come help!
There is a lot of low-hanging and high impact work to do.&lt;/p&gt;
&lt;p&gt;If you’re interested in being paid to focus more on these topics, then consider
applying for a job. NVIDIA’s RAPIDS team is looking to hire engineers for Dask
development with GPUs and other data analytics library development projects.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://nvidia.wd5.myworkdayjobs.com/en-US/NVIDIAExternalCareerSite/job/US-TX-Austin/Senior-Library-Software-Engineer---RAPIDS_JR1919608-1"&gt;Senior Library Software Engineer - RAPIDS&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/01/13/dask-cudf-first-steps/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="GPU" label="GPU"/>
    <category term="Pandas" label="Pandas"/>
    <published>2019-01-13T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/01/03/dask-array-gpus-first-steps/</id>
    <title>GPU Dask Arrays, first steps</title>
    <updated>2019-01-03T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</name>
    </author>
    <content type="html">&lt;p&gt;The following code creates and manipulates 2 TB of randomly generated 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="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="n"&gt;rs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;rs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;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="mi"&gt;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;500000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;500000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;10000&lt;/span&gt;&lt;span class="p"&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;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="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="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;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;threads&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;On a single CPU, this computation takes two hours.&lt;/p&gt;
&lt;p&gt;On an eight-GPU single-node system this computation takes nineteen seconds.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/03/dask-array-gpus-first-steps.md&lt;/span&gt;, line 24)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="combine-dask-array-with-cupy"&gt;

&lt;p&gt;Actually this computation isn’t that impressive.
It’s a simple workload,
for which most of the time is spent creating and destroying random data.
The computation and communication patterns are simple,
reflecting the simplicity commonly found in data processing workloads.&lt;/p&gt;
&lt;p&gt;What &lt;em&gt;is&lt;/em&gt; impressive is that we were able to create a distributed parallel GPU
array quickly by composing these four existing libraries:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://cupy.chainer.org/"&gt;CuPy&lt;/a&gt; provides a partial implementation of
Numpy on the GPU.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://docs.dask.org/en/latest/array.html"&gt;Dask Array&lt;/a&gt; provides chunked
algorithms on top of Numpy-like libraries like Numpy and CuPy.&lt;/p&gt;
&lt;p&gt;This enables us to operate on more data than we could fit in memory
by operating on that data in chunks.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The &lt;a class="reference external" href="https://distributed.dask.org"&gt;Dask distributed&lt;/a&gt; task scheduler runs
those algorithms in parallel, easily coordinating work across many CPU
cores.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The &lt;a class="reference external" href="https://github.com/rapidsai/dask-cuda"&gt;Dask CUDA&lt;/a&gt; to extend Dask
distributed with GPU support.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;These tools already exist. We had to connect them together with a small amount
of glue code and minor modifications. By mashing these tools together we can
quickly build and switch between different architectures to explore what is
best for our application.&lt;/p&gt;
&lt;p&gt;For this example we relied on the following changes upstream:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/cupy/cupy/pull/1689"&gt;cupy/cupy #1689: Support Numpy arrays as seeds in RandomState&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4041"&gt;dask/dask #4041 Make da.RandomState accessible to other modules&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/2432"&gt;dask/distributed #2432: Add LocalCUDACluster&lt;/a&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/2019/01/03/dask-array-gpus-first-steps.md&lt;/span&gt;, line 62)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="comparison-among-single-multi-cpu-gpu"&gt;
&lt;h1&gt;Comparison among single/multi CPU/GPU&lt;/h1&gt;
&lt;p&gt;We can now easily run some experiments on different architectures. This is
easy because …&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;We can switch between CPU and GPU by switching between Numpy and CuPy.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We can switch between single/multi-CPU-core and single/multi-GPU
by switching between Dask’s different task schedulers.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;These libraries allow us to quickly judge the costs of this computation for
the following hardware choices:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Single-threaded CPU&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Multi-threaded CPU with 40 cores (80 H/T)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Single-GPU&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Multi-GPU on a single machine with 8 GPUs&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We present code for these four choices below,
but first,
we present a table of results.&lt;/p&gt;
&lt;section id="results"&gt;
&lt;h2&gt;Results&lt;/h2&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
  &lt;tr&gt;
    &lt;th&gt;Architecture&lt;/th&gt;
    &lt;th&gt;Time&lt;/th&gt;
  &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt; Single CPU Core &lt;/th&gt;
      &lt;td&gt; 2hr 39min &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; Forty CPU Cores &lt;/th&gt;
      &lt;td&gt; 11min 30s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; One GPU &lt;/th&gt;
      &lt;td&gt; 1 min 37s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; Eight GPUs &lt;/th&gt;
      &lt;td&gt; 19s &lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;/section&gt;
&lt;section id="setup"&gt;
&lt;h2&gt;Setup&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cupy&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;

&lt;span class="c1"&gt;# generate chunked dask arrays of mamy numpy random arrays&lt;/span&gt;
&lt;span class="n"&gt;rs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;rs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;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="mi"&gt;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;500000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;500000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;10000&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

&lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;nbytes&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;# 2 TB&lt;/span&gt;
&lt;span class="c1"&gt;# 2000.0&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="cpu-timing"&gt;
&lt;h2&gt;CPU timing&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="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;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="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="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;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;single-threaded&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;x&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="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="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;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;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;threads&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="single-gpu-timing"&gt;
&lt;h2&gt;Single GPU timing&lt;/h2&gt;
&lt;p&gt;We switch from CPU to GPU by changing our data source to generate CuPy arrays
rather than NumPy arrays. Everything else should more or less work the same
without special handling for CuPy.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;(This actually isn’t true yet, many things in dask.array will break for
non-NumPy arrays, but we’re working on it actively both within Dask, within
NumPy, and within the GPU array libraries. Regardless, everything in this
example works fine.)&lt;/em&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="c1"&gt;# generate chunked dask arrays of mamy cupy random arrays&lt;/span&gt;
&lt;span class="n"&gt;rs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# &amp;lt;-- we specify cupy here&lt;/span&gt;
&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;rs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;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="mi"&gt;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;500000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;500000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;10000&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="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;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="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="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;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;single-threaded&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="multi-gpu-timing"&gt;
&lt;h2&gt;Multi GPU timing&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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_cuda&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;LocalCUDACluster&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCUDACluster&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="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;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="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="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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And again, here are the results:&lt;/p&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
  &lt;tr&gt;
    &lt;th&gt;Architecture&lt;/th&gt;
    &lt;th&gt;Time&lt;/th&gt;
  &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt; Single CPU Core &lt;/th&gt;
      &lt;td&gt; 2hr 39min &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; Forty CPU Cores &lt;/th&gt;
      &lt;td&gt; 11min 30s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; One GPU &lt;/th&gt;
      &lt;td&gt; 1 min 37s &lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt; Eight GPUs &lt;/th&gt;
      &lt;td&gt; 19s &lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;First, this is my first time playing with an 40-core system. I was surprised
to see that many cores. I was also pleased to see that Dask’s normal threaded
scheduler happily saturates many cores.&lt;/p&gt;
&lt;img src="https://matthewrocklin.com/blog/images/python-gil-8000-percent.png" width="100%"&gt;
&lt;p&gt;Although later on it did dive down to around 5000-6000%, and if you do the math
you’ll see that we’re not getting a 40x speedup. My &lt;em&gt;guess&lt;/em&gt; is that
performance would improve if we were to play with some mixture of threads and
processes, like having ten processes with eight threads each.&lt;/p&gt;
&lt;p&gt;The jump from the biggest multi-core CPU to a single GPU is still an order of
magnitude though. The jump to multi-GPU is another order of magnitude, and
brings the computation down to 19s, which is short enough that I’m willing to
wait for it to finish before walking away from my computer.&lt;/p&gt;
&lt;p&gt;Actually, it’s quite fun to watch on the dashboard (especially after you’ve
been waiting for three hours for the sequential solution to run):&lt;/p&gt;
&lt;blockquote class="imgur-embed-pub"
            lang="en"
            data-id="a/6hkPPwA"&gt;
&lt;a href="//imgur.com/6hkPPwA"&gt;&lt;/a&gt;
&lt;/blockquote&gt;
&lt;script async src="//s.imgur.com/min/embed.js" charset="utf-8"&gt;&lt;/script&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/03/dask-array-gpus-first-steps.md&lt;/span&gt;, line 221)&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="conclusion"&gt;
&lt;h1&gt;Conclusion&lt;/h1&gt;
&lt;p&gt;This computation was simple, but the range in architecture just explored was
extensive. We swapped out the underlying architecture from CPU to GPU (which
had an entirely different codebase) and tried both multi-core CPU parallelism
as well as multi-GPU many-core parallelism.&lt;/p&gt;
&lt;p&gt;We did this in less than twenty lines of code, making this experiment something
that an undergraduate student or other novice could perform at home.
We’re approaching a point where experimenting with multi-GPU systems is
approachable to non-experts (at least for array computing).&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/57be0ca4143974e6015732d0baacc1cb"&gt;Here is a notebook for the experiment above&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/2019/01/03/dask-array-gpus-first-steps.md&lt;/span&gt;, line 235)&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="room-for-improvement"&gt;
&lt;h1&gt;Room for improvement&lt;/h1&gt;
&lt;p&gt;We can work to expand the computation above in a variety of directions.
There is a ton of work we still have to do to make this reliable.&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Use more complex array computing workloads&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;The Dask Array algorithms were designed first around Numpy. We’ve only
recently started making them more generic to other kinds of arrays (like
GPU arrays, sparse arrays, and so on). As a result there are still many
bugs when exploring these non-Numpy workloads.&lt;/p&gt;
&lt;p&gt;For example if you were to switch &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sum&lt;/span&gt;&lt;/code&gt; for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&lt;/span&gt;&lt;/code&gt; in the computation above
you would get an error because our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&lt;/span&gt;&lt;/code&gt; computation contains an easy to
fix error that assumes Numpy arrays exactly.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Use Pandas and cuDF instead of Numpy and CuPy&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;The cuDF library aims to reimplement the Pandas API on the GPU,
much like how CuPy reimplements the NumPy API.
Using Dask DataFrame with cuDF will require some work on both sides,
but is quite doable.&lt;/p&gt;
&lt;p&gt;I believe that there is plenty of low-hanging fruit here.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Improve and move LocalCUDACluster&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LocalCUDAClutster&lt;/span&gt;&lt;/code&gt; class used above is an experimental &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Cluster&lt;/span&gt;&lt;/code&gt; type
that creates as many workers locally as you have GPUs, and assigns each
worker to prefer a different GPU. This makes it easy for people to load
balance across GPUs on a single-node system without thinking too much about
it. This appears to be a common pain-point in the ecosystem today.&lt;/p&gt;
&lt;p&gt;However, the LocalCUDACluster probably shouldn’t live in the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask/distributed&lt;/span&gt;&lt;/code&gt; repository (it seems too CUDA specific) so will probably
move to some dask-cuda repository. Additionally there are still many
questions about how to handle concurrency on top of GPUs, balancing between
CPU cores and GPU cores, and so on.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Multi-node computation&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;There’s no reason that we couldn’t accelerate computations like these
further by using multiple multi-GPU nodes. This is doable today with
manual setup, but we should also improve the existing deployment solutions
&lt;a class="reference external" href="https://kubernetes.dask.org"&gt;dask-kubernetes&lt;/a&gt;,
&lt;a class="reference external" href="https://yarn.dask.org"&gt;dask-yarn&lt;/a&gt;, and
&lt;a class="reference external" href="https://jobqueue.dask.org"&gt;dask-jobqueue&lt;/a&gt;, to make this easier for
non-experts who want to use a cluster of multi-GPU resources.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Expense&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;The machine I ran this on is expensive. Well, it’s nowhere close to as
expensive to own and operate as a traditional cluster that you would need
for these kinds of results, but it’s still well beyond the price point of a
hobbyist or student.&lt;/p&gt;
&lt;p&gt;It would be useful to run this on a more budget system to get a sense of
the tradeoffs on more reasonably priced systems. I should probably also
learn more about provisioning GPUs on the cloud.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;section id="come-help"&gt;
&lt;h2&gt;Come help!&lt;/h2&gt;
&lt;p&gt;If the work above sounds interesting to you then come help!
There is a lot of low-hanging and high impact work to do.&lt;/p&gt;
&lt;p&gt;If you’re interested in being paid to focus more on these topics, then consider
applying for a job. The NVIDIA corporation is hiring around the use of Dask
with GPUs.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://nvidia.wd5.myworkdayjobs.com/en-US/NVIDIAExternalCareerSite/job/US-TX-Austin/Senior-Library-Software-Engineer---RAPIDS_JR1919608-1"&gt;Senior Library Software Engineer - RAPIDS&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;That’s a fairly generic posting. If you’re interested the posting doesn’t seem
to fit then please apply anyway and we’ll tweak things.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/01/03/dask-array-gpus-first-steps/"/>
    <summary>The following code creates and manipulates 2 TB of randomly generated data.</summary>
    <category term="GPU" label="GPU"/>
    <category term="array" label="array"/>
    <category term="cupy" label="cupy"/>
    <published>2019-01-03T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/11/29/version-1.0/</id>
    <title>Dask Version 1.0</title>
    <updated>2018-11-29T00:00:00+00:00</updated>
    <author>
      <name>the Dask Team</name>
    </author>
    <content type="html">&lt;p&gt;We are pleased to announce the release of Dask version 1.0.0!&lt;/p&gt;
&lt;p&gt;Usually in release blogposts we outline important features and changes since
the last major version. Because of the 1.0 version number, this post will be a
bit different. Instead we’ll talk about what this version number means to us,
and discuss the broader context of Dask projects more generally.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/11/29/version-1.0.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="what-1-0-means-to-us"&gt;

&lt;p&gt;Version 1.0 software means different things to different groups.
In some communities it might mean …&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The first version of a package&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;When a package is first ready for production use&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;When a package has reached API stability&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;…&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;It is common in the PyData ecosystem to wait a &lt;em&gt;long time&lt;/em&gt; before releasing a
version 1.0. For example neither Pandas nor Scikit-Learn, arguably two of the
most well used PyData packages in production, have yet declared a 1.0 version
number (today they are at versions 0.23 and 0.20 respectively). And yet each
package is widely used in production by organizations that demand high degrees
of stability.&lt;/p&gt;
&lt;p&gt;Dask is not as API-stable as Pandas or Scikit-Learn, but it’s pretty close.
The project rarely invents new APIs, instead preferring to implement
pre-existing APIs (like the NumPy/Pandas/Scikit-Learn APIs) or standard language
protocols (like async-await, concurrent.futures, Queues, Locks, and so on).
Additionally, Dask is well used in production today across sectors ranging from
risk-tolerant industries like startups and quantitative finance shops, to
risk-averse institutions like banks, large enterprises, and governments.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;When we say that Dask has reached 1.0 we mean that it is ready to be used in
production. We are late in saying this. This happened a long time ago.&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/2018/11/29/version-1.0.md&lt;/span&gt;, line 44)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="development-will-continue-as-before"&gt;
&lt;h1&gt;Development will continue as before&lt;/h1&gt;
&lt;p&gt;Dask is living software that exists in a rapidly evolving space. Nothing is
changing about our internal stability practices. We will continue to add new
features, deprecate old ones, and fix bugs with the same policies. We always
try to minimize negative effects on users when making these internal changes
while maximizing the speed at which we can deliver new bugfixes and features.
This is hard and requires care, but we believe that we’ve done this decently in
the past so hopefully you haven’t noticed much. We will continue to operate
the same way into the future.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;The 1.0 version change does not affect our development cycle.
There are no LTS versions beyond what we already provide.&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/2018/11/29/version-1.0.md&lt;/span&gt;, line 58)&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="different-dask-packages-move-at-different-speeds"&gt;
&lt;h1&gt;Different Dask packages move at different speeds&lt;/h1&gt;
&lt;p&gt;Dask is able to evolve and experiment rapidly while maintaining a stable core
because it is split into sub-packages, each of which evolves independently, has
its own maintainers, its own versions, and its own release cycle. Some Dask
subprojects have had versions above 1.0 for a long time, while others are still
unstable.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Dask’s version number is hard to define today because it is composed of so
many independent efforts by different groups. This is similar to situation in
Jupyter, or in the Numeric Python ecosystem itself.&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/2018/11/29/version-1.0.md&lt;/span&gt;, line 70)&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;Finally, we’re grateful to everyone who has contributed to the project over the
years, either by contributing code, reviews, documentation, discussion, bug
reports, well written questions and answers, visual designs, and well wishes.
This means a lot to us.&lt;/p&gt;
&lt;p&gt;Today there are dozens of &lt;a class="reference external" href="https://pypi.org/search/?q=dask"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-*&lt;/span&gt;&lt;/code&gt; packages on
PyPI&lt;/a&gt; that support thousands of users and
several more that incorporate Dask for parallelism. We’re thankful to play a
role in such a vibrant community.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/11/29/version-1.0/"/>
    <summary>We are pleased to announce the release of Dask version 1.0.0!</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-11-29T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/10/08/Dask-Jobqueue/</id>
    <title>Dask-jobqueue</title>
    <updated>2018-10-08T00:00:00+00:00</updated>
    <author>
      <name>Joe Hamman</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This work was done in collaboration with &lt;a class="reference external" href="https://github.com/mrocklin"&gt;Matthew Rocklin&lt;/a&gt; (Anaconda), Jim Edwards (NCAR), &lt;a class="reference external" href="https://github.com/guillaumeeb"&gt;Guillaume Eynard-Bontemps&lt;/a&gt; (CNES), and &lt;a class="reference external" href="https://github.com/lesteve"&gt;Loïc Estève&lt;/a&gt; (INRIA), and is supported, in part, by the US National Science Foundation &lt;a class="reference external" href="https://www.earthcube.org/"&gt;Earth Cube program&lt;/a&gt;. The dask-jobqueue package is a spinoff of the &lt;a class="reference external" href="https://medium.com/pangeo"&gt;Pangeo Project&lt;/a&gt;. This blogpost was previously published &lt;a class="reference external" href="https://medium.com/pangeo/dask-jobqueue-d7754e42ca53"&gt;here&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;TLDR;&lt;/strong&gt; &lt;em&gt;Dask-jobqueue&lt;/em&gt; allows you to seamlessly deploy &lt;a class="reference external" href="https://dask.org/"&gt;dask&lt;/a&gt; on HPC clusters that use a variety of job queuing systems such as PBS, Slurm, SGE, or LSF. Dask-jobqueue provides a &lt;em&gt;Pythonic&lt;/em&gt; user interface that manages dask workers/clusters through the submission, execution, and deletion of individual jobs on a HPC system. It gives users the ability to interactively scale workloads across large HPC systems; turning an interactive &lt;a class="reference external" href="http://jupyter.org/"&gt;Jupyter&lt;/a&gt; Notebook into a powerful tool for scalable computation on very large datasets.&lt;/p&gt;
&lt;p&gt;Install with:&lt;/p&gt;
&lt;div class="highlight-bash notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;conda&lt;span class="w"&gt; &lt;/span&gt;install&lt;span class="w"&gt; &lt;/span&gt;-c&lt;span class="w"&gt; &lt;/span&gt;conda-forge&lt;span class="w"&gt; &lt;/span&gt;dask-jobqueue
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or&lt;/p&gt;
&lt;div class="highlight-bash notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;pip&lt;span class="w"&gt; &lt;/span&gt;install&lt;span class="w"&gt; &lt;/span&gt;dask-jobqueue
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And checkout the dask-jobqueue documentation: &lt;a class="reference external" href="http://jobqueue.dask.org"&gt;http://jobqueue.dask.org&lt;/a&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/10/08/Dask-Jobqueue.md&lt;/span&gt;, line 28)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="introduction"&gt;

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

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

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

&lt;p&gt;We recently changed how we organize and connect Dask’s documentation.
Our approach may prove useful for other umbrella projects that spread
documentation across many different builds and sites.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/27/docs-refactor.md&lt;/span&gt;, line 17)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="dask-splits-documentation-into-many-pages"&gt;
&lt;h1&gt;Dask splits documentation into many pages&lt;/h1&gt;
&lt;p&gt;Dask’s documentation is split into several different websites, each managed by
a different team for a different sub-project:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask.pydata.org"&gt;dask.pydata.org&lt;/a&gt; : Main site&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://distributed.readthedocs.org"&gt;distributed.readthedocs.org&lt;/a&gt; : Distributed scheduler&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-ml.readthedocs.io"&gt;dask-ml.readthedocs.io&lt;/a&gt; : Dask for machine learning&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-kubernetes.readthedocs.io"&gt;dask-kubernetes.readthedocs.io&lt;/a&gt; : Dask on Kubernetes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-jobqueue.readthedocs.io"&gt;dask-jobqueue.readthedocs.io&lt;/a&gt; : Dask on HPC systems&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-yarn.readthedocs.io"&gt;dask-yarn.readthedocs.io&lt;/a&gt; : Dask on Hadoop systems&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-examples.readthedocs.io"&gt;dask-examples.readthedocs.io&lt;/a&gt; : Examples that use Dask&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://matthewrocklin.com/blog"&gt;matthewrocklin.com/blog&lt;/a&gt;,
&lt;a class="reference external" href="https://jcrist.github.io"&gt;jcrist.github.io&lt;/a&gt;,
&lt;a class="reference external" href="https://tomaugspurger.github.io"&gt;tomaugspurger.github.io&lt;/a&gt;,
&lt;a class="reference external" href="https://martindurant.github.io/blog"&gt;martindurant.github.io/blog&lt;/a&gt; :
Developers’ personal blogs&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This split in documentation matches the split in development teams. Each of
sub-project’s team manages its own docs in its own way. They release at their
own pace and make their own decisions about technology. This makes it much
more likely that developers maintain the documentation as they develop and
change software libraries.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;We make it easy to write documentation. This choice causes many different documentation systems to emerge.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;This approach is common. A web search for Jupyter Documentation yields the
following list:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://jupyter.readthedocs.io/en/latest/"&gt;jupyter.readthedocs.io/en/latest/&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://jupyter-notebook.readthedocs.io/en/stable/"&gt;jupyter-notebook.readthedocs.io/en/stable/&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://jupyter.org/"&gt;jupyter.org/&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://jupyterhub.readthedocs.io/en/stable/"&gt;jupyterhub.readthedocs.io/en/stable/&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://nteract.io/"&gt;nteract.io/&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://ipython.org/"&gt;ipython.org/&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Different teams developing semi-independently create different web pages. This
is inevitable. Asking a large distributed team to coordinate on a single
cohesive website adds substantial friction, which results in worse
documentation coverage.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/27/docs-refactor.md&lt;/span&gt;, line 58)&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="problem"&gt;
&lt;h1&gt;Problem&lt;/h1&gt;
&lt;p&gt;However, while using separate websites results in excellent coverage, it
also fragments the documentation. This makes it harder for users to smoothly
navigate between sites and discover appropriate content.&lt;/p&gt;
&lt;p&gt;Monolithic documentation is good for readers,
modular documentation is good for writers.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/27/docs-refactor.md&lt;/span&gt;, line 67)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="our-solutions"&gt;
&lt;h1&gt;Our Solutions&lt;/h1&gt;
&lt;p&gt;Over the last month we took steps to connect our documentation and make it more
cohesive, while still enabling independent development. This post outlines the
following steps:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Organize under a single domain, dask.org&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Develop a sphinx template project for uniform style&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Include a cross-project navbar in addition to the within-project
table-of-contents&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We did some other things along the way that we find useful, but are probably
more specific to just Dask.&lt;/p&gt;
&lt;ol class="arabic simple" start="4"&gt;
&lt;li&gt;&lt;p&gt;We moved this blog to blog.dask.org&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We improved our example notebooks to host both a static site and also a live Binder&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/27/docs-refactor.md&lt;/span&gt;, line 84)&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="organize-under-a-single-domains-dask-org"&gt;
&lt;h1&gt;1: Organize under a single domains, Dask.org&lt;/h1&gt;
&lt;p&gt;Previously we had some documentation under &lt;a class="reference external" href="https://rtfd.org"&gt;readthedocs&lt;/a&gt;,
some under the &lt;a class="reference external" href="https://dask.pydata.org"&gt;dask.pydata.org&lt;/a&gt; subdomain (thanks
NumFOCUS!) and some pages on personal websites, like
&lt;a class="reference external" href="https://matthewrocklin.com/blog"&gt;matthewrocklin.com/blog&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;While looking for a new dask domain to host all of our content we noticed that
&lt;a class="reference external" href="https://dask.org"&gt;dask.org&lt;/a&gt; redirected to
&lt;a class="reference external" href="https://anaconda.org"&gt;anaconda.org&lt;/a&gt;, and were pleased to learn that someone at
&lt;a class="reference external" href="https://anaconda.com"&gt;Anaconda Inc&lt;/a&gt; had the foresight to register the domain
early on.&lt;/p&gt;
&lt;p&gt;Anaconda was happy to transfer ownership of the domain to NumFOCUS, who helps
us to maintain it now. Now all of our documentation is available under that
single domain as subdomains:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask.org"&gt;dask.org&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://docs.dask.org"&gt;docs.dask.org&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://distributed.dask.org"&gt;distributed.dask.org&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://ml.dask.org"&gt;ml.dask.org&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://kubernetes.dask.org"&gt;kubernetes.dask.org&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://yarn.dask.org"&gt;yarn.dask.org&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://jobqueue.dask.org"&gt;jobqueue.dask.org&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://examples.dask.org"&gt;examples.dask.org&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://examples.dask.org"&gt;stories.dask.org&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://blog.dask.org"&gt;blog.dask.org&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This uniformity means that the thing you want is probably at that-thing.dask.org, which is a bit easier to guess than otherwise.&lt;/p&gt;
&lt;p&gt;Many thanks to &lt;a class="reference external" href="https://andy.terrel.us/"&gt;Andy Terrel&lt;/a&gt; and &lt;a class="reference external" href="https://tomaugspurger.github.io"&gt;Tom
Augspurger&lt;/a&gt; for managing this move, and to
Anaconda for generously donating the domain.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/27/docs-refactor.md&lt;/span&gt;, line 118)&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="cross-project-navigation-bar"&gt;
&lt;h1&gt;2: Cross-project Navigation Bar&lt;/h1&gt;
&lt;p&gt;We wanted a way for readers to quickly discover the other sites that were
available to them. All of our sites have side-navigation-bars to help readers
navigate within a particular site, but now they also have a top-navigation-bar
to help them navigate between projects.&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/docs-navbar-sidebar.png"
     width="100%"
     alt="adding a navbar to dask docs"&gt;&lt;/p&gt;
&lt;p&gt;This navigation bar is managed independently from all of the documentation projects at
our new Sphinx theme.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/27/docs-refactor.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="dask-sphinx-theme"&gt;
&lt;h1&gt;3: Dask Sphinx Theme&lt;/h1&gt;
&lt;p&gt;To give a uniform sense of style we developed our own Sphinx HTML theme. This
inherits from ReadTheDocs’ theme, but with changed styling to match Dask color
and visual style. We publish this theme as a &lt;a class="reference external" href="https://pypi.org/project/dask-sphinx-theme/"&gt;package on
PyPI&lt;/a&gt; that all of our projects’
Sphinx builds can import and use if they want. We can change style in this one
package and publish to PyPI and all of the projects will pick up those changes
on their next build without having to copy stylesheets around to different
repositories.&lt;/p&gt;
&lt;p&gt;This allows several different projects to evolve content (which they care
about) and build process separately from style (which they typically don’t care
as much about). We have a single style sheet that gets used everywhere easily.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/27/docs-refactor.md&lt;/span&gt;, line 147)&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="move-dask-blogging-to-blog-dask-org"&gt;
&lt;h1&gt;4: Move Dask Blogging to blog.dask.org&lt;/h1&gt;
&lt;p&gt;Previously most announcements about Dask were written and published from one of
the maintainers’ personal blogs. This split information about the project and
made it hard for people to discover good content. There also wasn’t a good way
for a community member to suggest a blog for distribution to the general
community, other than by starting their own.&lt;/p&gt;
&lt;p&gt;Now we have an official blog at &lt;a class="reference external" href="https://blog.dask.org"&gt;blog.dask.org&lt;/a&gt; which
serves files submitted to
&lt;a class="reference external" href="https://github.com/dask/dask-blog"&gt;github.com/dask/dask-blog&lt;/a&gt;. These posts
are simple markdown files that should be easy for people to generate. For
example the source for this post is available at
&lt;a class="reference external" href="https://github.com/dask/dask-blog/blob/gh-pages/_posts/2018-09-27-docs-refactor.md"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;github.com/dask/dask-blog/blob/gh-pages/_posts/2018-09-27-docs-refactor.md&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;We encourage community members to share posts about work they’ve done with Dask
by submitting pull requests to that repository.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/27/docs-refactor.md&lt;/span&gt;, line 165)&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="host-examples-as-both-static-html-and-live-binder-sessions"&gt;
&lt;h1&gt;5: Host Examples as both static HTML and live Binder sessions&lt;/h1&gt;
&lt;p&gt;The Dask community maintains a set of example notebooks that show people how to
use Dask in a variety of ways. These notebooks live at
&lt;a class="reference external" href="https://github.com/dask/dask-examples"&gt;github.com/dask/dask-examples&lt;/a&gt; and are
easy for users to download and run.&lt;/p&gt;
&lt;p&gt;To get more value from these notebooks we now expose them in two additional
ways:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;As static HTML at &lt;a class="reference external" href="https://examples.dask.org"&gt;examples.dask.org&lt;/a&gt;, rendered
with the &lt;a class="reference external" href="https://nbsphinx.readthedocs.io/en/latest/"&gt;nbsphinx&lt;/a&gt; plugin.&lt;/p&gt;
&lt;p&gt;Seeing them statically rendered and being able to quickly navigate between
them really increases the pleasure of exploring them. We hope that this
encourages users to explore more broadly.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;As live-runnable notebooks on the cloud using &lt;a class="reference external" href="https://mybinder.org"&gt;mybinder.org&lt;/a&gt;.
You can play with any of these notebooks by clicking on this button:
&lt;a class="reference external" href="https://mybinder.org/v2/gh/dask/dask-examples/main?urlpath=lab"&gt;&lt;img alt="Binder" src="https://mybinder.org/badge.svg" /&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;This allows people to explore more deeply. Also, because we’ve connected
up the Dask JupyterLab extension to this environment, users get an
immediate instinctual experience of what parallel computing feels like (if
you haven’t used the dask dashboard during computation you really should
give that link a try).&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Now that these examples get much more exposure we hope that this encourages
community members to submit new examples. We hope that by providing
infrastructure more content creators will come as well.&lt;/p&gt;
&lt;p&gt;We also encourage other projects to take a look at what we’ve done in
&lt;a class="reference external" href="https://github.com/dask/dask-examples"&gt;github.com/dask/dask-examples&lt;/a&gt;. We
think that this model might be broadly useful across other projects.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/27/docs-refactor.md&lt;/span&gt;, line 200)&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;Thank you for reading. We hope that this post pushes readers to re-explore
Dask’s documentation, and that it pushes developers to consider some of the
approaches above for their own projects.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/09/27/docs-refactor/"/>
    <summary>This work is supported by Anaconda Inc</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-09-27T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/09/17/dask-dev/</id>
    <title>Dask Development Log</title>
    <updated>2018-09-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://anaconda.com"&gt;Anaconda Inc&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;To increase transparency I’m trying to blog more often about the current work
going on around Dask and related projects. Nothing here is ready for
production. This blogpost is written in haste, so refined polish should not be
expected.&lt;/p&gt;
&lt;p&gt;Since the last update in the &lt;a class="reference internal" href="../2018/09/05/dask-0.19.0/"&gt;&lt;span class="doc std std-doc"&gt;0.19.0 release blogpost&lt;/span&gt;&lt;/a&gt; two weeks ago we’ve seen activity in the following areas:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Update Dask examples to use JupyterLab on Binder&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Render Dask examples into static HTML pages for easier viewing&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Consolidate and unify disparate documentation&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Retire the &lt;a class="reference external" href="https://hdfs3.readthedocs.io/en/latest/"&gt;hdfs3 library&lt;/a&gt; in favor of the solution in &lt;a class="reference external" href="https://arrow.apache.org/docs/python/filesystems.html"&gt;Apache Arrow&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Continue work on hyper-parameter selection for incrementally trained models&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Publish two small bugfix releases&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Blogpost from the Pangeo community about combining Binder with Dask&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Skein/Yarn Update&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/17/dask-dev.md&lt;/span&gt;, line 26)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="update-dask-examples-to-use-jupyterlab-extension"&gt;

&lt;p&gt;The new &lt;a class="reference external" href="https://github.com/dask/dask-labextension"&gt;dask-labextension&lt;/a&gt; embeds
Dask’s dashboard plots into a JupyterLab session so that you can get easy
access to information about your computations from Jupyter directly. This was
released a few weeks ago as part of the previous release post.&lt;/p&gt;
&lt;p&gt;However since then we’ve hooked this up to our live examples system that lets
users try out Dask on a small cloud instance using
&lt;a class="reference external" href="https://mybinder.org"&gt;mybinder.org&lt;/a&gt;. If you want to try out Dask and
JupyterLab together then head here:&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://mybinder.org/v2/gh/dask/dask-examples/main?urlpath=lab"&gt;&lt;img alt="Binder" src="https://mybinder.org/badge.svg" /&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="https://github.com/ian-r-rose"&gt;Ian Rose&lt;/a&gt; for managing this.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/17/dask-dev.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="render-dask-examples-as-static-documentation"&gt;
&lt;h1&gt;2: Render Dask Examples as static documentation&lt;/h1&gt;
&lt;p&gt;Using the &lt;a class="reference external" href="https://nbsphinx.readthedocs.io/en/0.3.5/"&gt;nbsphinx&lt;/a&gt; Sphinx
extension to automatically run and render Jupyter Notebooks we’ve turned our
live examples repository into static documentation for easy viewing.&lt;/p&gt;
&lt;p&gt;These examples are currently available at
&lt;a class="reference external" href="https://dask.org/dask-examples/"&gt;https://dask.org/dask-examples/&lt;/a&gt; but will
soon be available at &lt;a class="reference external" href="https://dask.org/dask-examples/"&gt;examples.dask.org&lt;/a&gt; and
from the navbar at all dask pages.&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="https://tomaugspurger.github.io/"&gt;Tom Augspurger&lt;/a&gt; for putting this
together.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/17/dask-dev.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="consolidate-documentation-under-a-single-org-and-style"&gt;
&lt;h1&gt;3: Consolidate documentation under a single org and style&lt;/h1&gt;
&lt;p&gt;Dask documentation is currently spread out in many small hosted sites, each
associated to a particular subpackage like dask-ml, dask-kubernetes,
dask-distributed, etc.. This eases development (developers are encouraged to
modify documentation as they modify code) but results in a fragmented
experience because users don’t know how to discover and efficiently explore our
full documentation.&lt;/p&gt;
&lt;p&gt;To resolve this we’re doing two things:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;Moving all sites under the dask.org domain&lt;/p&gt;
&lt;p&gt;Anaconda Inc, the company that employs several of the Dask developers
(myself included) recently donated the domain &lt;a class="reference external" href="http://dask.org"&gt;dask.org&lt;/a&gt;
to NumFOCUS. We’ve been slowly moving over all of our independent sites to
use that location for our documentation.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Develop a uniform Sphinx theme &lt;a class="reference external" href="http://github.com/dask/dask-sphinx-theme"&gt;dask-sphinx-theme&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;This has both uniform styling and also includes a navbar that gets
automatically shared between the projects. The navbar makes it easy to
discover and explore content and is something that we can keep up-to-date
in a single repository.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;You can see how this works by going to any of the Dask sites, like
&lt;a class="reference external" href="http://docs.dask.org/en/latest/docs.html"&gt;docs.dask.org&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="https://tomaugspurger.github.io/"&gt;Tom Augspurger&lt;/a&gt; for managing this
work and &lt;a class="reference external" href="http://andy.terrel.us/"&gt;Andy Terrel&lt;/a&gt; for patiently handling things on
the NumFOCUS side and domain name side.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/17/dask-dev.md&lt;/span&gt;, line 88)&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="retire-the-hdfs3-library"&gt;
&lt;h1&gt;4: Retire the hdfs3 library&lt;/h1&gt;
&lt;p&gt;For years the Dask community has maintained the
&lt;a class="reference external" href="https://hdfs3.readthedocs.io/en/latest/"&gt;hdfs3&lt;/a&gt; library that allows for native
access to the Hadoop file system from Python. This used Pivotal’s libhdfs3
library written in C++ and was, for a long while the only performant way to
maturely manipulate HDFS from Python.&lt;/p&gt;
&lt;p&gt;Since then though PyArrow has developed efficient bindings to the standard
libhdfs library and exposed it through their Pythonic &lt;a class="reference external" href="https://arrow.apache.org/docs/python/filesystems.html#hadoop-file-system-hdfs"&gt;file system
interface&lt;/a&gt;,
which is fortunately Dask-compatible.&lt;/p&gt;
&lt;p&gt;We’ve been telling people to use the Arrow solution for a while now and thought
we’d now do so officially
(see &lt;a class="reference external" href="https://github.com/dask/hdfs3/pull/170"&gt;dask/hdfs3 #170&lt;/a&gt;). As of the
last bugfix release Dask will use Arrow by default and, while the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;hdfs3&lt;/span&gt;&lt;/code&gt;
library is still available, Dask maintainers probably won’t spend much time on
it in the future.&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="https://hdfs3.readthedocs.io/en/latest/"&gt;Martin Durant&lt;/a&gt; for building
and maintaining HDFS3 over all this 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/2018/09/17/dask-dev.md&lt;/span&gt;, line 111)&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="hyper-parameter-selection-for-incrementally-trained-models"&gt;
&lt;h1&gt;5: Hyper-parameter selection for incrementally trained models&lt;/h1&gt;
&lt;p&gt;In Dask-ML we continue to work on hyper-parameter selection for models that
implement the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt; API. We’ve built algorithms and infrastructure to
handle this well, and are currently fine tuning API, parameter names, etc..&lt;/p&gt;
&lt;p&gt;If you have any interest in this process, come on over to &lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/356"&gt;dask/dask-ml #356&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="https://tomaugspurger.github.io/"&gt;Tom Augspurger&lt;/a&gt; and &lt;a class="reference external" href="https://stsievert.com/"&gt;Scott
Sievert&lt;/a&gt; for this 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/2018/09/17/dask-dev.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="two-small-bugfix-releases"&gt;
&lt;h1&gt;6: Two small bugfix releases&lt;/h1&gt;
&lt;p&gt;We’ve been trying to increase the frequency of bugfix releases while things are
stable. Since our last writing there have been two minor bugfix releases. You
can read more about them here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/blob/master/docs/source/changelog.rst"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/docs/source/changelog.rst"&gt;dask/distributed&lt;/a&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/2018/09/17/dask-dev.md&lt;/span&gt;, line 131)&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="binder-dask"&gt;
&lt;h1&gt;7: Binder + Dask&lt;/h1&gt;
&lt;p&gt;The Pangeo community has done work to integrate Binder with Dask and has
written about the process here: &lt;a class="reference external" href="https://medium.com/pangeo/pangeo-meets-binder-2ea923feb34f"&gt;Pangeo meets Binder&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="http://joehamman.com/"&gt;Joe Hamman&lt;/a&gt; for this work and the blogpost.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/17/dask-dev.md&lt;/span&gt;, line 138)&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="skein-yarn-update"&gt;
&lt;h1&gt;8: Skein/Yarn Update&lt;/h1&gt;
&lt;p&gt;The Dask-Yarn connection to deploy Dask on Hadoop clusters uses a library
&lt;a class="reference external" href="https://jcrist.github.io/skein/"&gt;Skein&lt;/a&gt; to easily manage Yarn jobs from
Python.&lt;/p&gt;
&lt;p&gt;Skein has seen a lot of activity over the last few weeks, including the
following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;A Web UI for the project. See &lt;a class="reference external" href="https://github.com/jcrist/skein/pull/68"&gt;jcrist/skein #68&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A Tensorflow on Yarn project from Criteo that uses Skein. See
&lt;a class="reference external" href="https://github.com/criteo/tf-yarn"&gt;github.com/criteo/tf-yarn&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This work is mostly managed by &lt;a class="reference external" href="http://jcrist.github.io/"&gt;Jim Crist&lt;/a&gt; and other
Skein contributors.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/09/17/dask-dev/"/>
    <summary>This work is supported by Anaconda Inc</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-09-17T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/09/05/dask-0.19.0/</id>
    <title>Dask Release 0.19.0</title>
    <updated>2018-09-05T00: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://anaconda.com"&gt;Anaconda Inc.&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;I’m pleased to announce the release of Dask version 0.19.0. This is a major
release with bug fixes and new features. The last release was 0.18.2 on July
23rd. This blogpost outlines notable changes since the last release blogpost
for 0.18.0 on June 14th.&lt;/p&gt;
&lt;p&gt;You can conda install Dask:&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 pip install from PyPI:&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[complete] --upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Full changelogs are available here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/blob/master/docs/source/changelog.rst"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/docs/source/changelog.rst"&gt;dask/distributed&lt;/a&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/2018/09/05/dask-0.19.0.md&lt;/span&gt;, line 28)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="notable-changes"&gt;

&lt;p&gt;A ton of work has happened over the past two months, but most of the changes
are small and diffuse. Stability, feature parity with upstream libraries (like
Numpy and Pandas), and performance have all significantly improved, but in ways
that are difficult to condense into blogpost form.&lt;/p&gt;
&lt;p&gt;That being said, here are a few of the more exciting changes in the new
release.&lt;/p&gt;
&lt;section id="python-versions"&gt;
&lt;h2&gt;Python Versions&lt;/h2&gt;
&lt;p&gt;We’ve dropped official support for Python 3.4 and added official support for
Python 3.7.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="deploy-on-hadoop-clusters"&gt;
&lt;h2&gt;Deploy on Hadoop Clusters&lt;/h2&gt;
&lt;p&gt;Over the past few months &lt;a class="reference external" href="https://jcrist.github.io/"&gt;Jim Crist&lt;/a&gt; has bulit a
suite of tools to deploy applications on YARN, the primary cluster manager used
in Hadoop clusters.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://conda.github.io/conda-pack/"&gt;Conda-pack&lt;/a&gt;: packs up Conda
environments for redistribution to distributed clusters, especially when
Python or Conda may not be present.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://jcrist.github.io/skein/"&gt;Skein&lt;/a&gt;: easily launches and manages YARN
applications from non-JVM systems&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-yarn.readthedocs.io/en/latest/"&gt;Dask-Yarn&lt;/a&gt;: a thin library
around Skein to launch and manage Dask clusters&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Jim has written about Skein and Dask-Yarn in two recent blogposts:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://jcrist.github.io/dask-on-yarn"&gt;jcrist.github.io/dask-on-yarn&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://jcrist.github.io/introducing-skein.html"&gt;jcrist.github.io/introducing-skein.html&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="implement-actors"&gt;
&lt;h2&gt;Implement Actors&lt;/h2&gt;
&lt;p&gt;Some advanced workloads want to directly manage and mutate state on workers. A
task-based framework like Dask can be forced into this kind of workload using
long-running-tasks, but it’s an uncomfortable experience.&lt;/p&gt;
&lt;p&gt;To address this we’ve added an experimental Actors framework to Dask alongside
the standard task-scheduling system. This provides reduced latencies, removes
scheduling overhead, and provides the ability to directly mutate state on a
worker, but loses niceties like resilience and diagnostics.
The idea to adopt Actors was shamelessly stolen from the &lt;a class="reference external" href="http://ray.readthedocs.io/en/latest/"&gt;Ray Project&lt;/a&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="k"&gt;class&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nc"&gt;Counter&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="fm"&gt;__init__&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;n&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&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;increment&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;n&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;
        &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;n&lt;/span&gt;

&lt;span class="n"&gt;counter&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Counter&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;actor&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&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;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;counter&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;increment&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;future&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You can read more about actors in the &lt;a class="reference external" href="https://distributed.readthedocs.io/en/latest/actors.html"&gt;Actors documentation&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dashboard-improvements"&gt;
&lt;h2&gt;Dashboard improvements&lt;/h2&gt;
&lt;p&gt;The Dask dashboard is a critical tool to understand distributed performance.
There are a few accessibility issues that trip up beginning users that we’ve
addressed in this release.&lt;/p&gt;
&lt;section id="save-task-stream-plots"&gt;
&lt;h3&gt;Save task stream plots&lt;/h3&gt;
&lt;p&gt;You can now save a task stream record by wrapping a computation in the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;get_task_stream&lt;/span&gt;&lt;/code&gt; context manager.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;get_task_stream&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;processes&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&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&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;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;datasets&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;timeseries&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;get_task_stream&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;plot&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;save&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;filename&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;my-task-stream.html&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;ts&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;std&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;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;ts&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;
&lt;span class="go"&gt;[{&amp;#39;key&amp;#39;: &amp;quot;(&amp;#39;make-timeseries-edc372a35b317f328bf2bb5e636ae038&amp;#39;, 0)&amp;quot;,&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;nbytes&amp;#39;: 8175440,&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;startstops&amp;#39;: [(&amp;#39;compute&amp;#39;, 1535661384.2876947, 1535661384.3366017)],&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;status&amp;#39;: &amp;#39;OK&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;thread&amp;#39;: 139754603898624,&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;worker&amp;#39;: &amp;#39;inproc://192.168.50.100/15417/2&amp;#39;},&lt;/span&gt;

&lt;span class="go"&gt;  ...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This gives you the start and stop time of every task on every worker done
during that time. It also saves that data as an HTML file that you can share
with others. This is very valuable for communicating performance issues within
a team. I typically upload the HTML file as a gist and then share it with
rawgit.com&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ gist my-task-stream.html
https://gist.github.com/f48a121bf03c869ec586a036296ece1a
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;iframe src="https://rawgit.com/mrocklin/f48a121bf03c869ec586a036296ece1a/raw/d2c1a83d5dc62996eeabca495d5284e324d71d0c/my-task-stream.html" width="800" height="400"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="robust-to-different-screen-sizes"&gt;
&lt;h3&gt;Robust to different screen sizes&lt;/h3&gt;
&lt;p&gt;The Dashboard’s layout was designed to be used on a single screen, side-by-side
with a Jupyter notebook. This is how many Dask developers operate when working
on a laptop, however it is not how many users operate for one of two reasons:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;They are working in an office setting where they have several screens&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;They are new to Dask and uncomfortable splitting their screen into two
halves&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In these cases the styling of the dashboard becomes odd. Fortunately, &lt;a class="reference external" href="https://github.com/canavandl"&gt;Luke
Canavan&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/dsludwig"&gt;Derek
Ludwig&lt;/a&gt; recently improved the CSS for the
dashboard considerably, allowing it to switch between narrow and wide screens.
Here is a snapshot.&lt;/p&gt;
&lt;p&gt;&lt;a href="/images/dashboard-widescreen.png"&gt;&lt;img src="/images/dashboard-widescreen.png" width="70%"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="jupyter-lab-extension"&gt;
&lt;h3&gt;Jupyter Lab Extension&lt;/h3&gt;
&lt;p&gt;You can now embed Dashboard panes directly within Jupyter Lab using the newly
updated &lt;a class="reference external" href="https://github.com/dask/dask-labextension/"&gt;dask-labextension&lt;/a&gt;.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;jupyter&lt;/span&gt; &lt;span class="n"&gt;labextension&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;labextension&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This allows you to layout your own dashboard directly within JupyterLab. You
can combine plots from different pages, control their sizing, and so on. You
will need to provide the address of the dashboard server
(&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;http://localhost:8787&lt;/span&gt;&lt;/code&gt; by default on local machines) but after that
everything should persist between sessions. Now when I open up JupyterLab and
start up a Dask Client, I get this:&lt;/p&gt;
&lt;p&gt;&lt;a href="/images/dashboard-jupyterlab.png"&gt;&lt;img src="/images/dashboard-jupyterlab.png" width="70%"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="https://github.com/ian-r-rose"&gt;Ian Rose&lt;/a&gt; for doing most of the work
here.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/05/dask-0.19.0.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&gt;
&lt;/section&gt;
&lt;section id="outreach"&gt;
&lt;h1&gt;Outreach&lt;/h1&gt;
&lt;section id="dask-stories"&gt;
&lt;h2&gt;Dask Stories&lt;/h2&gt;
&lt;p&gt;People who use Dask have been writing about their experiences at &lt;a class="reference external" href="https://dask-stories.readthedocs.io/en/latest/"&gt;Dask
Stories&lt;/a&gt;. In the last couple
months the following people have written about and contributed their experience:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-stories.readthedocs.io/en/latest/sidewalk-labs.html"&gt;Civic Modelling at Sidewalk Labs&lt;/a&gt; by &lt;a class="reference external" href="https://github.com/bnaul"&gt;Brett Naul&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-stories.readthedocs.io/en/latest/mosquito-sequencing.html"&gt;Genome Sequencing for Mosquitoes&lt;/a&gt; by &lt;a class="reference external" href="http://alimanfoo.github.io/about/"&gt;Alistair Miles&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-stories.readthedocs.io/en/latest/fullspectrum.html"&gt;Lending and Banking at Full Spectrum&lt;/a&gt; by &lt;a class="reference external" href="https://www.linkedin.com/in/hussainsultan/"&gt;Hussain Sultan&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-stories.readthedocs.io/en/latest/icecube-cosmic-rays.html"&gt;Detecting Cosmic Rays at IceCube&lt;/a&gt; by &lt;a class="reference external" href="https://github.com/jrbourbeau"&gt;James Bourbeau&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-stories.readthedocs.io/en/latest/pangeo.html"&gt;Large Data Earth Science at Pangeo&lt;/a&gt; by &lt;a class="reference external" href="http://rabernat.github.io/"&gt;Ryan Abernathey&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-stories.readthedocs.io/en/latest/hydrologic-modeling.html"&gt;Hydrological Modelling at the National Center for Atmospheric Research&lt;/a&gt; by &lt;a class="reference external" href="http://joehamman.com/about/"&gt;Joe Hamman&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-stories.readthedocs.io/en/latest/network-modeling.html"&gt;Mobile Networks Modeling&lt;/a&gt; by &lt;a class="reference external" href="https://www.linkedin.com/in/lalwanisameer/"&gt;Sameer Lalwani&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-stories.readthedocs.io/en/latest/satellite-imagery.html"&gt;Satellite Imagery Processing at the Space Science and Engineering Center&lt;/a&gt; by &lt;a class="reference external" href="http://github.com/djhoese"&gt;David Hoese&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;These stories help people understand where Dask is and is not applicable, and
provide useful context around how it gets used in practice. We welcome further
contributions to this project. It’s very valuable to the broader community.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-examples"&gt;
&lt;h2&gt;Dask Examples&lt;/h2&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://github.com/dask/dask-examples"&gt;Dask-Examples repository&lt;/a&gt; maintains
easy-to-run examples using Dask on a small machine, suitable for an entry-level
laptop or for a small cloud instance. These are hosted on
&lt;a class="reference external" href="https://mybinder.org"&gt;mybinder.org&lt;/a&gt; and are integrated into our documentation.
A number of new examples have arisen recently, particularly in machine
learning. We encourage people to try them out by clicking the link below.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://mybinder.org/v2/gh/dask/dask-examples/main"&gt;&lt;img alt="Binder" src="https://mybinder.org/badge.svg" /&gt;&lt;/a&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/09/05/dask-0.19.0.md&lt;/span&gt;, line 210)&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="other-projects"&gt;
&lt;h1&gt;Other Projects&lt;/h1&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;The &lt;a class="reference external" href="https://dask-image.readthedocs.io/en/latest/"&gt;dask-image&lt;/a&gt; project was
recently released. It includes a number of image processing routines around
dask arrays.&lt;/p&gt;
&lt;p&gt;This project is mostly maintained by &lt;a class="reference external" href="https://github.com/jakirkham"&gt;John Kirkham&lt;/a&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-ml.readthedocs.io/en/latest/"&gt;Dask-ML&lt;/a&gt; saw a recent bugfix release&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The &lt;a class="reference external" href="http://epistasislab.github.io/tpot/"&gt;TPOT&lt;/a&gt; library for automated
machine learning recently published a new release that adds Dask support to
parallelize their model training. More information is available on the
&lt;a class="reference external" href="http://epistasislab.github.io/tpot/using/#parallel-training-with-dask"&gt;TPOT documentation&lt;/a&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/2018/09/05/dask-0.19.0.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="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;Since June 14th, the following people have contributed to the following repositories:&lt;/p&gt;
&lt;p&gt;The core Dask repository for parallel algorithms:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Anderson Banihirwe&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Andre Thrill&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Aurélien Ponte&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Christoph Moehl&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Cloves Almeida&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Daniel Rothenberg&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Danilo Horta&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Davis Bennett&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Elliott Sales de Andrade&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Eric Bonfadini&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;GPistre&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;George Sakkis&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Guido Imperiale&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Hans Moritz Günther&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Henrique Ribeiro&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Hugo&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Irina Truong&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Itamar Turner-Trauring&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jacob Tomlinson&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Bourbeau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jan Margeta&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Javad&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jeremy Chen&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joe Hamman&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John Mrziglod&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Julia Signell&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Marco Rossi&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Mark Harfouche&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matt Lee&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Mike Neish&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Robert Sare&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scott Sievert&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stephan Hoyer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tobias de Jong&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;WZY&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Yu Feng&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Yuval Langer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;minebogy&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;nmiles2718&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;rtobar&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The dask/distributed repository for distributed computing:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Anderson Banihirwe&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Aurélien Ponte&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bartosz Marcinkowski&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dave Hirschfeld&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Derek Ludwig&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dror Birkman&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Guillaume EB&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jacob Tomlinson&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joe Hamman&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Loïc Estève&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Luke Canavan&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Marius van Niekerk&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matt Nicolls&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Mike DePalatis&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Olivier Grisel&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Phil Tooley&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ray Bell&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Yu Feng&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The dask/dask-examples repository for easy-to-run examples:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Albert DeFusco&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dan Vatterott&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Guillaume EB&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scott Sievert&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;mholtzscher&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/09/05/dask-0.19.0/"/>
    <summary>This work is supported by Anaconda Inc.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-09-05T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/08/28/dataframe-performance-high-level/</id>
    <title>High level performance of Pandas, Dask, Spark, and Arrow</title>
    <updated>2018-08-28T00: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://anaconda.com"&gt;Anaconda Inc&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/08/28/dataframe-performance-high-level.md&lt;/span&gt;, line 10)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="question"&gt;

&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;How does Dask dataframe performance compare to Pandas? Also, what about
Spark dataframes and what about Arrow? How do they compare?&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;I get this question every few weeks. This post is to avoid repetition.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/08/28/dataframe-performance-high-level.md&lt;/span&gt;, line 17)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="caveats"&gt;
&lt;h1&gt;Caveats&lt;/h1&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;This answer is likely to change over time. I’m writing this in August 2018&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;This question and answer are very high level.
More technical answers are possible, but not contained here.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/08/28/dataframe-performance-high-level.md&lt;/span&gt;, line 23)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="answers"&gt;
&lt;h1&gt;Answers&lt;/h1&gt;
&lt;section id="pandas"&gt;
&lt;h2&gt;Pandas&lt;/h2&gt;
&lt;p&gt;If you’re coming from Python and have smallish datasets then Pandas is the
right choice. It’s usable, widely understood, efficient, and well maintained.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="benefits-of-parallelism"&gt;
&lt;h2&gt;Benefits of Parallelism&lt;/h2&gt;
&lt;p&gt;The performance benefit (or drawback) of using a parallel dataframe like Dask
dataframes or Spark dataframes over Pandas will differ based on the kinds of
computations you do:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;If you’re doing small computations then Pandas is always the right choice.
The administrative costs of parallelizing will outweigh any benefit.
You should not parallelize if your computations are taking less than, say,
100ms.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For simple operations like filtering, cleaning, and aggregating large data
you should expect linear speedup by using a parallel dataframes.&lt;/p&gt;
&lt;p&gt;If you’re on a 20-core computer you might expect a 20x speedup. If you’re
on a 1000-core cluster you might expect a 1000x speedup, assuming that you
have a problem big enough to spread across 1000 cores. As you scale up
administrative overhead will increase, so you should expect the speedup to
decrease a bit.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For complex operations like distributed joins it’s more complicated. You
might get linear speedups like above, or you might even get slowdowns.
Someone experienced in database-like computations and parallel computing
can probably predict pretty well which computations will do well.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;However, configuration may be required. Often people find that parallel
solutions don’t meet expectations when they first try them out. Unfortunately
most distributed systems require some configuration to perform optimally.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="there-are-other-options-to-speed-up-pandas"&gt;
&lt;h2&gt;There are other options to speed up Pandas&lt;/h2&gt;
&lt;p&gt;Many people looking to speed up Pandas don’t need parallelism. There are often
several other tricks like encoding text data, using efficient file formats,
avoiding groupby.apply, and so on that are more effective at speeding up Pandas
than switching to parallelism.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="comparing-apache-spark-and-dask"&gt;
&lt;h2&gt;Comparing Apache Spark and Dask&lt;/h2&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Assuming that yes, I do want parallelism, should I choose Apache Spark, or Dask dataframes?&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;This is often decided more by cultural preferences (JVM vs Python,
all-in-one-tool vs integration with other tools) than performance differences,
but I’ll try to outline a few things here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Spark dataframes will be much better when you have large SQL-style queries
(think 100+ line queries) where their query optimizer can kick in.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask dataframes will be much better when queries go beyond typical database
queries. This happens most often in time series, random access, and other
complex computations.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Spark will integrate better with JVM and data engineering technology.
Spark will also come with everything pre-packaged. Spark is its own
ecosystem.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask will integrate better with Python code. Dask is designed to integrate
with other libraries and pre-existing systems. If you’re coming from an
existing Pandas-based workflow then it’s usually much easier to evolve to
Dask.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Generally speaking for most operations you’ll be fine using either one. People
often choose between Pandas/Dask and Spark based on cultural preference.
Either they have people that really like the Python ecosystem, or they have
people that really like the Spark ecosystem.&lt;/p&gt;
&lt;p&gt;Dataframes are also only a small part of each project. Spark and Dask both do
many other things that aren’t dataframes. For example Spark has a graph
analysis library, Dask doesn’t. Dask supports multi-dimensional arrays, Spark
doesn’t. Spark is generally higher level and all-in-one while Dask is
lower-level and focuses on integrating into other tools.&lt;/p&gt;
&lt;p&gt;For more information, see &lt;a class="reference external" href="http://dask.pydata.org/en/latest/spark.html"&gt;Dask’s “Comparison to Spark documentation”&lt;/a&gt;
or &lt;a class="reference external" href="https://www.youtube.com/watch?v=jR0Y7NqKJs8&amp;amp;amp;list=PLJ0vO2F_f6OAE1xiEUE7DwMFWbdLCbN3P&amp;amp;amp;index=11&amp;amp;amp;t=413s"&gt;this interview with Steppingblocks&lt;/a&gt;, a data analytics company, on why they switched from Spark to Dask.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="apache-arrow"&gt;
&lt;h2&gt;Apache Arrow&lt;/h2&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;What about Arrow? Is Arrow faster than Pandas?&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;This question doesn’t quite make sense… &lt;em&gt;yet&lt;/em&gt;.&lt;/p&gt;
&lt;p&gt;Arrow is not a replacement for Pandas. Today Arrow is useful to people
building &lt;em&gt;systems&lt;/em&gt; and not to analysts directly like Pandas. Arrow is used to
move data between different computational systems and file formats. Arrow does
not do computation today, but is commonly used as a component in other
libraries that do do computation. For example, if you use Pandas or Spark or
Dask today you may be using Arrow without knowing it. Today Arrow is more
useful for other libraries than it is to end-users.&lt;/p&gt;
&lt;p&gt;However, this is likely to change in the future. Arrow developers plan
to write computational code around Arrow that we would expect to be faster than
the code in either Pandas or Spark. This is probably a year or two away
though. There will probably be some effort to make this semi-compatible with
Pandas, but it’s much too early to tell.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/08/28/dataframe-performance-high-level/"/>
    <summary>This work is supported by Anaconda Inc</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-08-28T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/08/07/incremental-saga/</id>
    <title>Building SAGA optimization for Dask arrays</title>
    <updated>2018-08-07T00: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="https://www.ethz.ch/en.html"&gt;ETH Zurich&lt;/a&gt;, &lt;a class="reference external" href="http://anaconda.com"&gt;Anaconda
Inc&lt;/a&gt;, and the &lt;a class="reference external" href="https://bids.berkeley.edu/"&gt;Berkeley Institute for Data
Science&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;At a recent Scikit-learn/Scikit-image/Dask sprint at BIDS, &lt;a class="reference external" href="http://fa.bianp.net"&gt;Fabian Pedregosa&lt;/a&gt; (a
machine learning researcher and Scikit-learn developer) and Matthew
Rocklin (Dask core developer) sat down together to develop an implementation of the incremental optimization algorithm
&lt;a class="reference external" href="https://arxiv.org/pdf/1407.0202.pdf"&gt;SAGA&lt;/a&gt; on parallel Dask datasets. The result is a sequential algorithm that can be run on any dask array, and so allows the data to be stored on disk or even distributed among different machines.&lt;/p&gt;
&lt;p&gt;It was interesting both to see how the algorithm performed and also to see
the ease and challenges to run a research algorithm on a Dask distributed dataset.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/08/07/incremental-saga.md&lt;/span&gt;, line 20)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="start"&gt;

&lt;p&gt;We started with an initial implementation that Fabian had written for Numpy
arrays using Numba. The following code solves an optimization problem of the form&lt;/p&gt;
&lt;div class="math notranslate nohighlight"&gt;
\[
min_x \sum_{i=1}^n f(a_i^t x, b_i)
\]&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numba&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;njit&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;sklearn.linear_model.sag&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_auto_step_size&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;sklearn.utils.extmath&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;row_norms&lt;/span&gt;

&lt;span class="nd"&gt;@njit&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;deriv_logistic&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;p&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="c1"&gt;# derivative of logistic loss&lt;/span&gt;
    &lt;span class="c1"&gt;# same as in lightning (with minus sign)&lt;/span&gt;
    &lt;span class="n"&gt;p&lt;/span&gt; &lt;span class="o"&gt;*=&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;p&lt;/span&gt; &lt;span class="o"&gt;&amp;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;phi&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;1.&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;1&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;exp&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;p&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="n"&gt;exp_t&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;exp&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;p&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;phi&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;exp_t&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;1.&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;exp_t&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;phi&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;y&lt;/span&gt;

&lt;span class="nd"&gt;@njit&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;SAGA&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="n"&gt;step_size&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;max_iter&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="w"&gt;  &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
&lt;span class="sd"&gt;  SAGA algorithm&lt;/span&gt;

&lt;span class="sd"&gt;  A : n_samples x n_features numpy array&lt;/span&gt;
&lt;span class="sd"&gt;  b : n_samples numpy array with values -1 or 1&lt;/span&gt;
&lt;span class="sd"&gt;  &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;

    &lt;span class="n"&gt;n_samples&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n_features&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;shape&lt;/span&gt;
    &lt;span class="n"&gt;memory_gradient&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;zeros&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_samples&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;gradient_average&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;zeros&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_features&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;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;zeros&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_features&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# vector of coefficients&lt;/span&gt;
    &lt;span class="n"&gt;step_size&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;0.3&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;get_auto_step_size&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;row_norms&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;squared&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;max&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="s1"&gt;&amp;#39;log&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;False&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;_&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="n"&gt;max_iter&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="c1"&gt;# sample randomly&lt;/span&gt;
        &lt;span class="n"&gt;idx&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;arange&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;memory_gradient&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;size&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;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;shuffle&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;idx&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="c1"&gt;# .. inner iteration ..&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="n"&gt;idx&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
            &lt;span class="n"&gt;grad_i&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;deriv_logistic&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;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="n"&gt;A&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="n"&gt;b&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="c1"&gt;# .. update coefficients ..&lt;/span&gt;
            &lt;span class="n"&gt;delta&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;grad_i&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;memory_gradient&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;A&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="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;-=&lt;/span&gt; &lt;span class="n"&gt;step_size&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;delta&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;gradient_average&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

            &lt;span class="c1"&gt;# .. update memory terms ..&lt;/span&gt;
            &lt;span class="n"&gt;gradient_average&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;grad_i&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;memory_gradient&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;A&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="n"&gt;n_samples&lt;/span&gt;
            &lt;span class="n"&gt;memory_gradient&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;grad_i&lt;/span&gt;

        &lt;span class="c1"&gt;# monitor convergence&lt;/span&gt;
        &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;gradient norm:&amp;#39;&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;linalg&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;norm&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;gradient_average&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;x&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This implementation is a simplified version of the &lt;a class="reference external" href="https://github.com/openopt/copt/blob/master/copt/randomized.py"&gt;SAGA
implementation&lt;/a&gt;
that Fabian uses regularly as part of his research, and that assumes that &lt;span class="math notranslate nohighlight"&gt;\(f\)&lt;/span&gt; is the &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Loss_functions_for_classification#Logistic_loss"&gt;logistic loss&lt;/a&gt;, i.e., &lt;span class="math notranslate nohighlight"&gt;\(f(z) = \log(1 + \exp(-z))\)&lt;/span&gt;. It can be used to solve problems with other values of &lt;span class="math notranslate nohighlight"&gt;\(f\)&lt;/span&gt; by overwriting the function &lt;code&gt;deriv_logistic&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;We wanted to apply it across a parallel Dask array by applying it to each chunk of the Dask array, a smaller Numpy array, one at a time, carrying along a set of parameters along the way.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/08/07/incremental-saga.md&lt;/span&gt;, line 91)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="development-process"&gt;
&lt;h1&gt;Development Process&lt;/h1&gt;
&lt;p&gt;In order to better understand the challenges of writing Dask algorithms, Fabian
did most of the actual coding to start. Fabian is good example of a researcher who
knows how to program well and how to design ML algorithms, but has no direct
exposure to the Dask library. This was an educational opportunity both for
Fabian and for Matt. Fabian learned how to use Dask, and Matt learned how to
introduce Dask to researchers like Fabian.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/08/07/incremental-saga.md&lt;/span&gt;, line 100)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="step-1-build-a-sequential-algorithm-with-pure-functions"&gt;
&lt;h1&gt;Step 1: Build a sequential algorithm with pure functions&lt;/h1&gt;
&lt;p&gt;To start we actually didn’t use Dask at all, instead, Fabian modified his implementation in a few ways:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;It should operate over a list of Numpy arrays. A list of Numpy arrays is similar to a Dask array, but simpler.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It should separate blocks of logic into separate functions, these will
eventually become tasks, so they should be sizable chunks of work. In this
case, this led to the creating of the function &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;_chunk_saga&lt;/span&gt;&lt;/code&gt; that
performs an iteration of the SAGA algorithm on a subset of the data.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;These functions should not modify their inputs, nor should they depend on
global state. All information that those functions require (like
the parameters that we’re learning in our algorithm) should be
explicitly provided as inputs.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;These requested modifications affect performance a bit, we end up making more
copies of the parameters and more copies of intermediate state. In terms of
programming difficulty this took a bit of time (around a couple hours) but is a
straightforward task that Fabian didn’t seem to find challenging or foreign.&lt;/p&gt;
&lt;p&gt;These changes resulted in the following code:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numba&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;njit&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;sklearn.utils.extmath&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;row_norms&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;sklearn.linear_model.sag&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_auto_step_size&lt;/span&gt;


&lt;span class="nd"&gt;@njit&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;_chunk_saga&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="n"&gt;n_samples&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;f_deriv&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;memory_gradient&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;gradient_average&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;step_size&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="c1"&gt;# Make explicit copies of inputs&lt;/span&gt;
    &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;copy&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;gradient_average&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;gradient_average&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;copy&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;memory_gradient&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;memory_gradient&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;copy&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

    &lt;span class="c1"&gt;# Sample randomly&lt;/span&gt;
    &lt;span class="n"&gt;idx&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;arange&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;memory_gradient&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;size&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;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;shuffle&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;idx&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

    &lt;span class="c1"&gt;# .. inner iteration ..&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="n"&gt;idx&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;grad_i&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;f_deriv&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;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="n"&gt;A&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="n"&gt;b&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="c1"&gt;# .. update coefficients ..&lt;/span&gt;
        &lt;span class="n"&gt;delta&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;grad_i&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;memory_gradient&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;A&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="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;-=&lt;/span&gt; &lt;span class="n"&gt;step_size&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;delta&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;gradient_average&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="c1"&gt;# .. update memory terms ..&lt;/span&gt;
        &lt;span class="n"&gt;gradient_average&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;grad_i&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;memory_gradient&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;A&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="n"&gt;n_samples&lt;/span&gt;
        &lt;span class="n"&gt;memory_gradient&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;grad_i&lt;/span&gt;

    &lt;span class="k"&gt;return&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;memory_gradient&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;gradient_average&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;full_saga&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="n"&gt;max_iter&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;callback&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
&lt;span class="sd"&gt;  data: list of (A, b), where A is a n_samples x n_features&lt;/span&gt;
&lt;span class="sd"&gt;  numpy array and b is a n_samples numpy array&lt;/span&gt;
&lt;span class="sd"&gt;  &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="n"&gt;n_samples&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;
    &lt;span class="k"&gt;for&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="ow"&gt;in&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;n_samples&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;shape&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;n_features&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="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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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;memory_gradients&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;zeros&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;shape&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;for&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="ow"&gt;in&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;gradient_average&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;zeros&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_features&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;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;zeros&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_features&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

    &lt;span class="n"&gt;steps&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_auto_step_size&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;row_norms&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;squared&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;max&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="s1"&gt;&amp;#39;log&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;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="ow"&gt;in&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;step_size&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;0.3&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;min&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;steps&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;_&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="n"&gt;max_iter&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="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="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;data&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;memory_gradients&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="n"&gt;gradient_average&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;_chunk_saga&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="n"&gt;n_samples&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;deriv_logistic&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;memory_gradients&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="n"&gt;gradient_average&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;step_size&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;callback&lt;/span&gt; &lt;span class="ow"&gt;is&lt;/span&gt; &lt;span class="ow"&gt;not&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
            &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;callback&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;data&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;x&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/2018/08/07/incremental-saga.md&lt;/span&gt;, line 180)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="step-2-apply-dask-delayed"&gt;
&lt;h1&gt;Step 2: Apply dask.delayed&lt;/h1&gt;
&lt;p&gt;Once functions neither modified their inputs nor relied on global state we went
over a &lt;a class="reference external" href="https://mybinder.org/v2/gh/dask/dask-examples/main?urlpath=%2Ftree%2Fdelayed.ipynb"&gt;dask.delayed example&lt;/a&gt;,
and then applied the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;&amp;#64;dask.delayed&lt;/span&gt;&lt;/code&gt; decorator to the functions that
Fabian had written. Fabian did this at first in about five minutes and to our
mutual surprise, things actually worked&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nout&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="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;---- New&lt;/span&gt;
&lt;span class="nd"&gt;@njit&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;_chunk_saga&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="n"&gt;n_samples&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;f_deriv&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;memory_gradient&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;gradient_average&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;step_size&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;full_saga&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="n"&gt;max_iter&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;callback&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;n_samples&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;
    &lt;span class="k"&gt;for&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="ow"&gt;in&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;n_samples&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;shape&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;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="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="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;---- New&lt;/span&gt;

    &lt;span class="o"&gt;...&lt;/span&gt;

    &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;_&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="n"&gt;max_iter&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="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="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;data&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;memory_gradients&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="n"&gt;gradient_average&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;_chunk_saga&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="n"&gt;n_samples&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;deriv_logistic&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;memory_gradients&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="n"&gt;gradient_average&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;step_size&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;cb&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;callback&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;data&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;        &lt;span class="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;---- Changed&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;cb&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cb&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;                 &lt;span class="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;---- New&lt;/span&gt;
        &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cb&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;However, they didn’t work &lt;em&gt;that well&lt;/em&gt;. When we took a look at the dask
dashboard we find that there is a lot of dead space, a sign that we’re still
doing a lot of computation on the client side.&lt;/p&gt;
&lt;a href="/images/saga-1.png"&gt;
  &lt;img src="/images/saga-1.png" width="90%"&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/2018/08/07/incremental-saga.md&lt;/span&gt;, line 221)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="step-3-diagnose-and-add-more-dask-delayed-calls"&gt;
&lt;h1&gt;Step 3: Diagnose and add more dask.delayed calls&lt;/h1&gt;
&lt;p&gt;While things worked, they were also fairly slow. If you notice the
dashboard plot above you’ll see that there is plenty of white in between
colored rectangles. This shows that there are long periods where none of the
workers is doing any work.&lt;/p&gt;
&lt;p&gt;This is a common sign that we’re mixing work between the workers (which shows
up on the dashbaord) and the client. The solution to this is usually more
targetted use of dask.delayed. Dask delayed is trivial to start using, but
does require some experience to use well. It’s important to keep track of
which operations and variables are delayed and which aren’t. There is some
cost to mixing between them.&lt;/p&gt;
&lt;p&gt;At this point Matt stepped in and added delayed in a few more places and the
dashboard plot started looking cleaner.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nout&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="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;---- New&lt;/span&gt;
&lt;span class="nd"&gt;@njit&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;_chunk_saga&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="n"&gt;n_samples&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;f_deriv&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;memory_gradient&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;gradient_average&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;step_size&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;full_saga&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="n"&gt;max_iter&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;callback&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;n_samples&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;
    &lt;span class="k"&gt;for&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="ow"&gt;in&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;n_samples&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;shape&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;n_features&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="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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="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="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;---- New&lt;/span&gt;
    &lt;span class="n"&gt;memory_gradients&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&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;zeros&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;shape&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;for&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="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;         &lt;span class="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;---- Changed&lt;/span&gt;
    &lt;span class="n"&gt;gradient_average&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&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;zeros&lt;/span&gt;&lt;span class="p"&gt;)(&lt;/span&gt;&lt;span class="n"&gt;n_features&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;#  Changed&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&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;zeros&lt;/span&gt;&lt;span class="p"&gt;)(&lt;/span&gt;&lt;span class="n"&gt;n_features&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;          &lt;span class="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;---- Changed&lt;/span&gt;

    &lt;span class="n"&gt;steps&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;get_auto_step_size&lt;/span&gt;&lt;span class="p"&gt;)(&lt;/span&gt;
                &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;row_norms&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;squared&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;max&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="s1"&gt;&amp;#39;log&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
             &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;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="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;                    &lt;span class="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;---- Changed&lt;/span&gt;
    &lt;span class="n"&gt;step_size&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;0.3&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&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;min&lt;/span&gt;&lt;span class="p"&gt;)(&lt;/span&gt;&lt;span class="n"&gt;steps&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;   &lt;span class="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;---- Changed&lt;/span&gt;

    &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;_&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="n"&gt;max_iter&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="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="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;data&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;memory_gradients&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="n"&gt;gradient_average&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;_chunk_saga&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="n"&gt;n_samples&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;deriv_logistic&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;memory_gradients&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="n"&gt;gradient_average&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;step_size&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;cb&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;callback&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;data&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;        &lt;span class="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;---- Changed&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;memory_gradients&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;gradient_average&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;step_size&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cb&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; \
            &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;memory_gradients&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;gradient_average&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;step_size&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cb&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# New&lt;/span&gt;
        &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cb&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;# &amp;lt;&amp;lt;&amp;lt;---- changed&lt;/span&gt;

    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/saga-2.png"&gt;
  &lt;img src="/images/saga-2.png" width="90%"&gt;
&lt;/a&gt;
&lt;p&gt;From a dask perspective this now looks good. We see that one &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt;
call is active at any given time with no large horizontal gaps between
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt; calls. We’re not getting any parallelism (this is just a
sequential algorithm) but we don’t have much dead space. The model seems to
jump between the various workers, processing on a chunk of data before moving
on to new 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/2018/08/07/incremental-saga.md&lt;/span&gt;, line 285)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="step-4-profile"&gt;
&lt;h1&gt;Step 4: Profile&lt;/h1&gt;
&lt;p&gt;The dashboard image above gives confidence that our algorithm is operating as
it should. The block-sequential nature of the algorithm comes out cleanly, and
the gaps between tasks are very short.&lt;/p&gt;
&lt;p&gt;However, when we look at the profile plot of the computation across all of our
cores (Dask constantly runs a profiler on all threads on all workers to get
this information) we see that most of our time is spent compiling Numba code.&lt;/p&gt;
&lt;a href="/images/saga-profile.png"&gt;
  &lt;img src="/images/saga-profile.png" width="100%"&gt;
&lt;/a&gt;
&lt;p&gt;We started a conversation for this on the &lt;a class="reference external" href="https://github.com/numba/numba/issues/3026"&gt;numba issue
tracker&lt;/a&gt; which has since been
resolved. That same computation over the same time now looks like this:&lt;/p&gt;
&lt;a href="/images/saga-3.png"&gt;
  &lt;img src="/images/saga-3.png" width="90%"&gt;
&lt;/a&gt;
&lt;p&gt;The tasks, which used to take seconds, now take tens of milliseconds, so we can
process through many more chunks in the same amount of 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/2018/08/07/incremental-saga.md&lt;/span&gt;, line 310)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, 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;This was a useful experience to build an interesting algorithm. Most of the
work above took place in an afternoon. We came away from this activity
with a few tasks of our own:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Build a normal Scikit-Learn style estimator class for this algorithm
so that people can use it without thinking too much about delayed objects,
and can instead just use dask arrays or dataframes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Integrate some of Fabian’s research on this algorithm that improves performance with
&lt;a class="reference external" href="https://arxiv.org/pdf/1707.06468.pdf"&gt;sparse data and in multi-threaded environments&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Think about how to improve the learning experience so that dask.delayed can
teach new users how to use it correctly&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/08/07/incremental-saga.md&lt;/span&gt;, line 324)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="links"&gt;
&lt;h1&gt;Links&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/5282dcf47505e2a1d214fd15c7da0ec3"&gt;Notebooks for different stages of SAGA+Dask implementation&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/scisprints/2018_05_sklearn_skimage_dask"&gt;Scikit-Learn/Image + Dask Sprint issue tracker&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/scisprints/2018_05_sklearn_skimage_dask"&gt;Paper on SAGA algorithm&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/openopt/copt/blob/master/copt/randomized.py"&gt;Fabian’s more fully featured non-Dask SAGA implementation&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/numba/numba/issues/3026"&gt;Numba issue on repeated deserialization&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/08/07/incremental-saga/"/>
    <summary>This work is supported by ETH Zurich, Anaconda
Inc, and the Berkeley Institute for Data
Science</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-08-07T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/08/02/dask-dev/</id>
    <title>Dask Development Log</title>
    <updated>2018-08-02T00: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://anaconda.com"&gt;Anaconda Inc&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;To increase transparency I’m trying to blog more often about the current work
going on around Dask and related projects. Nothing here is ready for
production. This blogpost is written in haste, so refined polish should not be
expected.&lt;/p&gt;
&lt;p&gt;Over the last two weeks we’ve seen activity in the following areas:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;An experimental Actor solution for stateful processing&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Machine learning experiments with hyper-parameter selection and parameter
servers.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Development of more preprocessing transformers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Statistical profiling of the distributed scheduler’s internal event loop
thread and internal optimizations&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A new release of dask-yarn&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A new narrative on dask-stories about modelling mobile networks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Support for LSF clusters in dask-jobqueue&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Test suite cleanup for intermittent failures&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/08/02/dask-dev.md&lt;/span&gt;, line 28)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="stateful-processing-with-actors"&gt;

&lt;p&gt;Some advanced workloads want to directly manage and mutate state on workers. A
task-based framework like Dask can be forced into this kind of workload using
long-running-tasks, but it’s an uncomfortable experience. To address this
we’ve been adding an experimental Actors framework to Dask alongside the
standard task-scheduling system. This provides reduced latencies, removes
scheduling overhead, and provides the ability to directly mutate state on a
worker, but loses niceties like resilience and diagnostics.&lt;/p&gt;
&lt;p&gt;The idea to adopt Actors was shamelessly stolen from the &lt;a class="reference external" href="http://ray.readthedocs.io/en/latest/"&gt;Ray Project&lt;/a&gt; :)&lt;/p&gt;
&lt;p&gt;Work for Actors is happening in &lt;a class="reference external" href="https://github.com/dask/distributed/pull/2133"&gt;dask/distributed #2133&lt;/a&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="k"&gt;class&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nc"&gt;Counter&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="fm"&gt;__init__&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;n&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&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;increment&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;n&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;
        &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;n&lt;/span&gt;

&lt;span class="n"&gt;counter&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Counter&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;actor&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&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;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;counter&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;increment&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;future&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/08/02/dask-dev.md&lt;/span&gt;, line 58)&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="machine-learning-experiments"&gt;
&lt;h1&gt;Machine learning experiments&lt;/h1&gt;
&lt;section id="hyper-parameter-optimization-on-incrementally-trained-models"&gt;
&lt;h2&gt;Hyper parameter optimization on incrementally trained models&lt;/h2&gt;
&lt;p&gt;Many Scikit-Learn-style estimators feature a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt; method that enables
incremental training on batches of data. This is particularly well suited for
systems like Dask array or Dask dataframe, that are built from many batches of
Numpy arrays or Pandas dataframes. It’s a nice fit because all of the
computational algorithm work is already done in Scikit-Learn, Dask just has to
administratively move models around to data and call scikit-learn (or other
machine learning models that follow the fit/transform/predict/score API). This
approach provides a nice community interface between parallelism and machine
learning developers.&lt;/p&gt;
&lt;p&gt;However, this training is inherently sequential because the model only trains
on one batch of data at a time. We’re leaving a lot of processing power on the
table.&lt;/p&gt;
&lt;p&gt;To address this we can combine incremental training with hyper-parameter
selection and train several models on the same data at the same time. This is
often required anyway, and lets us be more efficient with our computation.&lt;/p&gt;
&lt;p&gt;However there are many ways to do incremental training with hyper-parameter
selection, and the right algorithm likely depends on the problem at hand.
This is an active field of research and so it’s hard for a general project like
Dask to pick and implement a single method that works well for everyone. There
is probably a handful of methods that will be necessary with various options on
them.&lt;/p&gt;
&lt;p&gt;To help experimentation here we’ve been experimenting with some lower-level
tooling that we think will be helpful in a variety of cases. This accepts a
policy from the user as a Python function that gets scores from recent
evaluations, and asks for how much further to progress on each set of
hyper-parameters before checking in again. This allows us to model a few
common situations like random search with early stopping conditions, successive
halving, and variations of those easily without having to write any Dask code:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/288"&gt;dask/dask-ml #288&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/4c95bd26d15281d82e0bf2d27632e294"&gt;Notebook showing a few approaches&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/stsievert/c675b3a237a60efbd01dcb112e29115b"&gt;Another notebook showing convergence&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This work is done by &lt;a class="reference external" href="http://github.com/stsievert"&gt;Scott Sievert&lt;/a&gt; and myself&lt;/p&gt;
&lt;p&gt;&lt;img src="https://user-images.githubusercontent.com/1320475/43540881-7184496a-95b8-11e8-975a-96c2f17ee269.png"
     width="70%"
     alt="Successive halving and random search"&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="parameter-servers"&gt;
&lt;h2&gt;Parameter Servers&lt;/h2&gt;
&lt;p&gt;To improve the speed of training large models &lt;a class="reference external" href="https://github.com/stsievert"&gt;Scott
Sievert&lt;/a&gt; has been using Actors (mentioned above)
to develop simple examples for parameter servers. These are helping to
identify and motivate performance and diagnostic improvements improvements
within Dask itself:&lt;/p&gt;
&lt;script src="https://gist.github.com/ff8a1df9300a82f15a2704e913469522.js"&gt;&lt;/script&gt;
&lt;p&gt;These parameter servers manage the communication of models produced by
different workers, and leave the computation to the underlying deep learning
library. This is ongoing work.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dataframe-preprocessing-transformers"&gt;
&lt;h2&gt;Dataframe Preprocessing Transformers&lt;/h2&gt;
&lt;p&gt;We’ve started to orient some of the Dask-ML work around case studies. Our
first, written by &lt;a class="reference external" href="https://github.com/stsievert"&gt;Scott Sievert&lt;/a&gt;, uses the
Criteo dataset for ads. It’s a good example of a combined dense/sparse dataset
that can be somewhat large (around 1TB). The first challenge we’re running
into is preprocessing. These have lead to a few preprocessing improvements:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/310"&gt;Label Encoder supports Pandas Categorical dask/dask-ml #310&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/11"&gt;Add Imputer with mean and median strategies dask/dask-ml #11&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/313"&gt;Ad OneHotEncoder dask/dask-ml #313&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/122"&gt;Add Hashing Vectorizer dask/dask-ml #122&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/315"&gt;Add ColumnTransformer dask/dask-ml #315&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Some of these are also based off of improved dataframe handling features in the
upcoming 0.20 release for Scikit-Learn.&lt;/p&gt;
&lt;p&gt;This work is done by
&lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/122"&gt;Roman Yurchak&lt;/a&gt;,
&lt;a class="reference external" href="https://github.com/jrbourbeau"&gt;James Bourbeau&lt;/a&gt;,
&lt;a class="reference external" href="https://github.com/daniel-severo"&gt;Daniel Severo&lt;/a&gt;, and
&lt;a class="reference external" href="https://github.com/TomAugspurger"&gt;Tom Augspurger&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="profiling-the-main-thread"&gt;
&lt;h2&gt;Profiling the main thread&lt;/h2&gt;
&lt;p&gt;Profiling concurrent code is hard. Traditional profilers like CProfile become
confused by passing control between all of the different coroutines. This
means that we haven’t done a very comprehensive job of profiling and tuning the
distributed scheduler and workers. Statistical profilers on the other hand
tend to do a bit better. We’ve taken the statistical profiler that we usually
use on Dask worker threads (available in the dashboard on the “Profile” tab)
and have applied it to the central administrative threads running the Tornado
event loop as well. This has highlighted a few issues that we weren’t able to
spot before, and should hopefully result in reduced overhead in future
releases.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/2144"&gt;dask/distributed #2144&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://stackoverflow.com/questions/51582394/which-functions-are-free-when-profiling-tornado-asyncio"&gt;stackoverflow.com/questions/51582394/which-functions-are-free-when-profiling-tornado-asyncio&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;img src="https://user-images.githubusercontent.com/306380/43368136-4574f46c-930d-11e8-9d5b-6f4b4f6aeffe.png"
     width="70%"
     alt="Profile of event loop thread"&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="new-release-of-dask-yarn"&gt;
&lt;h2&gt;New release of Dask-Yarn&lt;/h2&gt;
&lt;p&gt;There is a new release of &lt;a class="reference external" href="http://dask-yarn.readthedocs.io/en/latest"&gt;Dask-Yarn&lt;/a&gt;
and the underlying library for managing Yarn jobs,
&lt;a class="reference external" href="https://jcrist.github.io/skein/"&gt;Skein&lt;/a&gt;. These include a number of bug-fixes
and improved concurrency primitives for YARN applications. The new features are
documented &lt;a class="reference external" href="https://jcrist.github.io/skein/key-value-store.html"&gt;here&lt;/a&gt;, and were
implemented in &lt;a class="reference external" href="https://github.com/jcrist/skein/pull/40"&gt;jcrist/skein #40&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;This work was done by &lt;a class="reference external" href="https://jcrist.github.io/"&gt;Jim Crist&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="support-for-lsf-clusters-in-dask-jobqueue"&gt;
&lt;h2&gt;Support for LSF clusters in Dask-Jobqueue&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="http://dask-jobqueue.readthedocs.io/en/latest/"&gt;Dask-jobqueue&lt;/a&gt; supports Dask
use on traditional HPC cluster managers like SGE, SLURM, PBS, and others.
We’ve recently &lt;a class="reference external" href="http://dask-jobqueue.readthedocs.io/en/latest/generated/dask_jobqueue.LSFCluster.html#dask_jobqueue.LSFCluster"&gt;added support for LSF clusters&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Work was done in &lt;a class="reference external" href="https://github.com/dask/dask-jobqueue/pull/78"&gt;dask/dask-jobqueue #78&lt;/a&gt; by &lt;a class="reference external" href="https://github.com/raybellwaves"&gt;Ray Bell&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="new-dask-story-on-mobile-networks"&gt;
&lt;h2&gt;New Dask Story on mobile networks&lt;/h2&gt;
&lt;p&gt;The &lt;a class="reference external" href="http://dask-stories.readthedocs.io/en/latest/"&gt;Dask Stories&lt;/a&gt;
repository holds narrative about how people use Dask.
&lt;a class="reference external" href="https://www.linkedin.com/in/lalwanisameer/"&gt;Sameer Lalwani&lt;/a&gt;
recently added a story about using Dask to
&lt;a class="reference external" href="http://dask-stories.readthedocs.io/en/latest/network-modeling.html"&gt;model mobile communication networks&lt;/a&gt;.
It’s worth a read.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="test-suite-cleanup"&gt;
&lt;h2&gt;Test suite cleanup&lt;/h2&gt;
&lt;p&gt;The dask.distributed test suite has been suffering from intermittent failures
recently. These are tests that fail very infrequently, and so are hard to
catch when writing them, but show up when future unrelated PRs run the test
suite on continuous integration and get failures. They add friction to the
development process, but are expensive to track down (testing distributed
systems is hard).&lt;/p&gt;
&lt;p&gt;We’re taking a bit of time this week to track these down. Progress here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/2146"&gt;dask/distributed #2146&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/2152"&gt;dask/distributed #2152&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/08/02/dask-dev/"/>
    <summary>This work is supported by Anaconda Inc</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-08-02T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/07/23/protocols-pickle/</id>
    <title>Pickle isn't slow, it's a protocol</title>
    <updated>2018-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://anaconda.com"&gt;Anaconda Inc&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr:&lt;/strong&gt; &lt;em&gt;Pickle isn’t slow, it’s a protocol. Protocols are important for
ecosystems.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;A recent Dask issue showed that using Dask with PyTorch was
slow because sending PyTorch models between Dask workers took a long time
(&lt;a class="reference external" href="https://github.com/dask/dask-ml/issues/281"&gt;Dask GitHub issue&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;This turned out to be because serializing PyTorch models with pickle was very
slow (1 MB/s for GPU based models, 50 MB/s for CPU based models). There is no
architectural reason why this needs to be this slow. Every part of the
hardware pipeline is much faster than this.&lt;/p&gt;
&lt;p&gt;We could have fixed this in Dask by special-casing PyTorch models (Dask has
it’s own optional serialization system for performance), but being good
ecosystem citizens, we decided to raise the performance problem in an issue
upstream (&lt;a class="reference external" href="https://github.com/pytorch/pytorch/issues/9168"&gt;PyTorch Github
issue&lt;/a&gt;). This resulted in a
five-line-fix to PyTorch that turned a 1-50 MB/s serialization bandwidth into a
1 GB/s bandwidth, which is more than fast enough for many use cases (&lt;a class="reference external" href="https://github.com/pytorch/pytorch/pull/9184"&gt;PR to
PyTorch&lt;/a&gt;).&lt;/p&gt;
&lt;div class="highlight-diff notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;    def __reduce__(self):
&lt;span class="gd"&gt;-        return type(self), (self.tolist(),)&lt;/span&gt;
&lt;span class="gi"&gt;+        b = io.BytesIO()&lt;/span&gt;
&lt;span class="gi"&gt;+        torch.save(self, b)&lt;/span&gt;
&lt;span class="gi"&gt;+        return (_load_from_bytes, (b.getvalue(),))&lt;/span&gt;


&lt;span class="gi"&gt;+def _load_from_bytes(b):&lt;/span&gt;
&lt;span class="gi"&gt;+    return torch.load(io.BytesIO(b))&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Thanks to the PyTorch maintainers this problem was solved pretty easily.
PyTorch tensors and models now serialize efficiently in Dask or in &lt;em&gt;any other
Python library&lt;/em&gt; that might want to use them in distributed systems like
PySpark, IPython parallel, Ray, or anything else without having to add
special-case code or do anything special. We didn’t solve a Dask problem, we
solved an ecosystem problem.&lt;/p&gt;
&lt;p&gt;However before we solved this problem we discussed things a bit. This comment
stuck with me:&lt;/p&gt;
&lt;a href="https://github.com/pytorch/pytorch/issues/9168#issuecomment-402514019"&gt;
  &lt;img src="/images/pytorch-pickle-is-slow-comment.png"
     alt="Github Image of maintainer saying that PyTorch's pickle implementation is slow"
     width="100%"&gt;&lt;/a&gt;
&lt;p&gt;This comment contains two beliefs that are both very common, and that I find
somewhat counter-productive:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Pickle is slow&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You should use our specialized methods instead&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;I’m sort of picking on the PyTorch maintainers here a bit (sorry!) but I’ve
found that they’re quite widespread, so I’d like to address them here.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/07/23/protocols-pickle.md&lt;/span&gt;, line 67)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="pickle-is-slow"&gt;

&lt;p&gt;Pickle is &lt;em&gt;not&lt;/em&gt; slow. Pickle is a protocol. &lt;em&gt;We&lt;/em&gt; implement pickle. If it’s slow
then it is &lt;em&gt;our&lt;/em&gt; fault, not Pickle’s.&lt;/p&gt;
&lt;p&gt;To be clear, there are many reasons not to use Pickle.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;It’s not cross-language&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It’s not very easy to parse&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It doesn’t provide random access&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It’s insecure&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;etc..&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;So you shouldn’t store your data or create public services using Pickle, but
for things like moving data on a wire it’s a great default choice if you’re
moving strictly from Python processes to Python processes in a trusted and
uniform environment.&lt;/p&gt;
&lt;p&gt;It’s great because it’s as fast as you can make it (up a a memory copy) and
other libraries in the ecosystem can use it without needing to special case
your code into theirs.&lt;/p&gt;
&lt;p&gt;This is the change we did for PyTorch.&lt;/p&gt;
&lt;div class="highlight-diff notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;    def __reduce__(self):
&lt;span class="gd"&gt;-        return type(self), (self.tolist(),)&lt;/span&gt;
&lt;span class="gi"&gt;+        b = io.BytesIO()&lt;/span&gt;
&lt;span class="gi"&gt;+        torch.save(self, b)&lt;/span&gt;
&lt;span class="gi"&gt;+        return (_load_from_bytes, (b.getvalue(),))&lt;/span&gt;


&lt;span class="gi"&gt;+def _load_from_bytes(b):&lt;/span&gt;
&lt;span class="gi"&gt;+    return torch.load(io.BytesIO(b))&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The slow part wasn’t Pickle, it was the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.tolist()&lt;/span&gt;&lt;/code&gt; call within &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__reduce__&lt;/span&gt;&lt;/code&gt;
that converted a PyTorch tensor into a list of Python ints and floats. I
suspect that the common belief of “Pickle is just slow” stopped anyone else
from investigating the poor performance here. I was surprised to learn that a
project as active and well maintained as PyTorch hadn’t fixed this already.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;As a reminder, you can implement the pickle protocol by providing the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__reduce__&lt;/span&gt;&lt;/code&gt; method on your class. The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__reduce__&lt;/span&gt;&lt;/code&gt; function returns a
loading function and sufficient arguments to reconstitute your object. Here we
used torch’s existing save/load functions to create a bytestring that we could
pass around.&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/2018/07/23/protocols-pickle.md&lt;/span&gt;, line 115)&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="just-use-our-specialized-option"&gt;
&lt;h1&gt;Just use our specialized option&lt;/h1&gt;
&lt;p&gt;Specialized options can be great. They can have nice APIs with many options,
they can tune themselves to specialized communication hardware if it exists
(like RDMA or NVLink), and so on. But people need to learn about them first, and
learning about them can be hard in two ways.&lt;/p&gt;
&lt;section id="hard-for-users"&gt;
&lt;h2&gt;Hard for users&lt;/h2&gt;
&lt;p&gt;Today we use a large and rapidly changing set of libraries. It’s hard
for users to become experts in all of them. Increasingly we rely on new
libraries making it easy for us by adhering to standard APIs, providing
informative error messages that lead to good behavior, and so on..&lt;/p&gt;
&lt;/section&gt;
&lt;section id="hard-for-other-libraries"&gt;
&lt;h2&gt;Hard for other libraries&lt;/h2&gt;
&lt;p&gt;Other libraries that need to interact &lt;em&gt;definitely&lt;/em&gt; won’t read the
documentation, and even if they did it’s not sensible for every library to
special case every other library’s favorite method to turn their objects into
bytes. Ecosystems of libraries depend strongly on the presence of protocols
and a strong consensus around implementing them consistently and efficiently.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/07/23/protocols-pickle.md&lt;/span&gt;, line 137)&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="sometimes-specialized-options-are-appropriate"&gt;
&lt;h1&gt;Sometimes Specialized Options are Appropriate&lt;/h1&gt;
&lt;p&gt;There &lt;em&gt;are&lt;/em&gt; good reasons to support specialized options. Sometimes you need
more than 1GB/s bandwidth. While this is rare in general (very few pipelines
process faster than 1GB/s/node), it is true in the particular case of PyTorch
when they are doing parallel training on a single machine with multiple
processes. Soumith (PyTorch maintainer) writes the following:&lt;/p&gt;
&lt;p&gt;When sending Tensors over multiprocessing, our custom serializer actually
shortcuts them through shared memory, i.e. it moves the underlying Storages
to shared memory and restores the Tensor in the other process to point to the
shared memory. We did this for the following reasons:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Speed:&lt;/strong&gt; we save on memory copies, especially if we amortize the cost of
moving a Tensor to shared memory before sending it into the multiprocessing
Queue. The total cost of actually moving a Tensor from one process to another
ends up being O(1), and independent of the Tensor’s size&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Sharing:&lt;/strong&gt; If Tensor A and Tensor B are views of each other, once we
serialize and send them, we want to preserve this property of them being
views. This is critical for neural-nets where it’s common to re-view the
weights / biases and use them for another. With the default pickle solution,
this property is actually lost.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/07/23/protocols-pickle/"/>
    <summary>This work is supported by Anaconda Inc</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-07-23T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/07/17/dask-dev/</id>
    <title>Dask Development Log, Scipy 2018</title>
    <updated>2018-07-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://anaconda.com"&gt;Anaconda Inc&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;To increase transparency I’m trying to blog more often about the current work
going on around Dask and related projects. Nothing here is ready for
production. This blogpost is written in haste, so refined polish should not be
expected.&lt;/p&gt;
&lt;p&gt;Last week many Dask developers gathered for the annual SciPy 2018 conference.
As a result, very little work was completed, but many projects were started or
discussed. To reflect this change in activity this blogpost will highlight
possible changes and opportunities for readers to further engage in
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/2018/07/17/dask-dev.md&lt;/span&gt;, line 21)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="dask-on-hpc-machines"&gt;

&lt;p&gt;The &lt;a class="reference external" href="https://dask-jobqueue.readthedocs.io/"&gt;dask-jobqueue&lt;/a&gt; project was a hit at
the conference. Dask-jobqueue helps people launch Dask on traditional job
schedulers like PBS, SGE, SLURM, Torque, LSF, and others that are commonly
found on high performance computers. These are &lt;em&gt;very common&lt;/em&gt; among scientific,
research, and high performance machine learning groups but commonly a bit hard
to use with anything other than MPI.&lt;/p&gt;
&lt;p&gt;This project came up in the &lt;a class="reference external" href="https://youtu.be/2rgD5AJsAbE"&gt;Pangeo talk&lt;/a&gt;,
lightning talks, and the Dask Birds of a Feather session.&lt;/p&gt;
&lt;p&gt;During sprints a number of people came up and we went through the process of
configuring Dask on common supercomputers like Cheyenne, Titan, and Cori. This
process usually takes around fifteen minutes and will likely be the subject of
a future blogpost. We published known-good configurations for these clusters
on our &lt;a class="reference external" href="http://dask-jobqueue.readthedocs.io/en/latest/configurations.html"&gt;configuration documentation&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Additionally, there is a &lt;a class="reference external" href="https://github.com/jupyterhub/batchspawner/issues/101"&gt;JupyterHub
issue&lt;/a&gt; to improve
documentation on best practices to deploy JupyterHub on these machines. The
community has done this well a few times now, and it might be time to write up
something for everyone else.&lt;/p&gt;
&lt;section id="get-involved"&gt;
&lt;h2&gt;Get involved&lt;/h2&gt;
&lt;p&gt;If you have access to a supercomputer then please try things out. There is a
30-minute Youtube video screencast on the
&lt;a class="reference external" href="https://dask-jobqueue.readthedocs.io/"&gt;dask-jobqueue&lt;/a&gt; documentation that should
help you get started.&lt;/p&gt;
&lt;p&gt;If you are an administrator on a supercomputer you might consider helping to
build a configuration file and place it in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;/etc/dask&lt;/span&gt;&lt;/code&gt; for your users. You
might also want to get involved in the &lt;a class="reference external" href="http://dask-jobqueue.readthedocs.io/en/latest/configurations.html"&gt;JupyterHub on
HPC&lt;/a&gt;
conversation.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/07/17/dask-dev.md&lt;/span&gt;, line 58)&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="dask-scikit-learn-talk"&gt;
&lt;h1&gt;Dask / Scikit-learn talk&lt;/h1&gt;
&lt;p&gt;Olivier Grisel and Tom Augspurger prepared and delivered a great talk on the
current state of the new Dask-ML project.&lt;/p&gt;
&lt;iframe width="560" height="315" src="https://www.youtube.com/embed/ccfsbuqsjgI"
frameborder="0" allow="autoplay; encrypted-media" allowfullscreen&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/2018/07/17/dask-dev.md&lt;/span&gt;, line 66)&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="mybinder-and-bokeh-servers"&gt;
&lt;h1&gt;MyBinder and Bokeh Servers&lt;/h1&gt;
&lt;p&gt;Not a Dask change, but Min Ragan-Kelley showed how to run services through
&lt;a class="reference external" href="https://mybinder.org/"&gt;mybinder.org&lt;/a&gt; that are not only Jupyter. As an example,
here is a repository that deploys a Bokeh server application with a single
click.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/minrk/binder-bokeh-server"&gt;Github repository&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://mybinder.org/v2/gh/minrk/binder-bokeh-server/master?urlpath=%2Fproxy%2F5006%2Fbokeh-app"&gt;Binder link&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;I think that by composing with Binder Min effectively just created the
free-to-use hosted Bokeh server service. Presumably this same model could be
easily adapted to other applications just as easily.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/07/17/dask-dev.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="dask-and-automated-machine-learning-with-tpot"&gt;
&lt;h1&gt;Dask and Automated Machine Learning with TPOT&lt;/h1&gt;
&lt;p&gt;Dask and TPOT developers are discussing paralellizing the
automatic-machine-learning tool &lt;a class="reference external" href="http://epistasislab.github.io/tpot/"&gt;TPOT&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;TPOT uses genetic algorithms to search over a space of scikit-learn style
pipelines to automatically find a decently performing pipeline and model. This
involves a fair amount of computation which Dask can help to parallelize out to
multiple machines.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/EpistasisLab/tpot/issues/304"&gt;Issue: EpistasisLab/tpot #304&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/EpistasisLab/tpot/pull/730"&gt;PR: EpistasisLab/tpot #730&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe width="560" height="315" src="https://www.youtube.com/embed/QrJlj0VCHys"
frameborder="0" allow="autoplay; encrypted-media" allowfullscreen&gt;&lt;/iframe&gt;
&lt;section id="id1"&gt;
&lt;h2&gt;Get involved&lt;/h2&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/2018/07/17/dask-dev.md&lt;/span&gt;, line 96); &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: “get involved”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Trivial things work now, but to make this efficient we’ll need to dive in a bit
more deeply. Extending that pull request to dive within pipelines would be a
good task if anyone wants to get involved. This would help to share
intermediate results between pipelines.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/07/17/dask-dev.md&lt;/span&gt;, line 103)&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="dask-and-scikit-optimize"&gt;
&lt;h1&gt;Dask and Scikit-Optimize&lt;/h1&gt;
&lt;p&gt;Among various features, &lt;a class="reference external" href="https://scikit-optimize.github.io/"&gt;Scikit-optimize&lt;/a&gt;
offers a &lt;a class="reference external" href="https://scikit-optimize.github.io/#skopt.BayesSearchCV"&gt;BayesSearchCV&lt;/a&gt;
object that is like Scikit-Learn’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GridSearchCV&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomSearchCV&lt;/span&gt;&lt;/code&gt;, but is a
bit smarter about how to choose new parameters to test given previous results.
Hyper-parameter optimization is a low-hanging fruit for Dask-ML workloads today,
so we investigated how the project might help here.&lt;/p&gt;
&lt;p&gt;So far we’re just experimenting using Scikit-Learn/Dask integration through
joblib to see what opportunities there are. Dicussion among Dask and
Scikit-Optimize developers is happening here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml/issues/300"&gt;Issue: dask/dask-ml #300&lt;/a&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/2018/07/17/dask-dev.md&lt;/span&gt;, line 118)&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="centralize-pydata-scipy-tutorials-on-binder"&gt;
&lt;h1&gt;Centralize PyData/Scipy tutorials on Binder&lt;/h1&gt;
&lt;p&gt;We’re putting a bunch of the PyData/Scipy tutorials on Binder, and hope to
embed snippets of Youtube videos into the notebooks themselves.&lt;/p&gt;
&lt;p&gt;This effort lives here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://pydata-tutorials.readthedocs.io"&gt;pydata-tutorials.readthedocs.io&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;section id="motivation"&gt;
&lt;h2&gt;Motivation&lt;/h2&gt;
&lt;p&gt;The PyData and SciPy community delivers tutorials as part of most conferences.
This activity generates both educational Jupyter notebooks and explanatory
videos that teach people how to use the ecosystem.&lt;/p&gt;
&lt;p&gt;However, this content isn’t very discoverable &lt;em&gt;after&lt;/em&gt; the conference. People
can search on Youtube for their topic of choice and hopefully find a link to
the notebooks to download locally, but this is a somewhat noisy process. It’s
not clear which tutorial to choose and it’s difficult to match up the video
with the notebooks during exercises.
We’re probably not getting as much value out of these resources as we could be.&lt;/p&gt;
&lt;p&gt;To help increase access we’re going to try a few things:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Produce a centralized website with links to recent tutorials delivered for
each topic&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ensure that those notebooks run easily on Binder&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Embed sections of the talk on Youtube within each notebook so that the
explanation of the section is tied to the exercises&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
&lt;section id="id2"&gt;
&lt;h2&gt;Get involved&lt;/h2&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/2018/07/17/dask-dev.md&lt;/span&gt;, line 148); &lt;em&gt;&lt;a href="#id2"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “get involved”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;This only really works long-term under a community maintenance model. So far
we’ve only done a few hours of work and there is still plenty to do in the
following tasks:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Find good tutorials for inclusion&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ensure that they work well on &lt;a class="reference external" href="https://mybinder.org/"&gt;mybinder.org&lt;/a&gt;&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;are self-contained and don’t rely on external scripts to run&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;have an environment.yml or requirements.txt&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;don’t require a lot of resources&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Find video for the tutorial&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Submit a pull request to the tutorial repository that embeds a link to the
youtube talk at the top cell of the notebook at the proper time for each
notebook&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/07/17/dask-dev.md&lt;/span&gt;, line 164)&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="dask-actors-and-ray"&gt;
&lt;h1&gt;Dask, Actors, and Ray&lt;/h1&gt;
&lt;p&gt;I really enjoyed the &lt;a class="reference external" href="https://youtu.be/D_oz7E4v-U0"&gt;talk on Ray&lt;/a&gt; another
distributed task scheduler for Python. I suspect that Dask will steal ideas
for &lt;a class="reference external" href="https://github.com/dask/distributed/issues/2109"&gt;actors for stateful operation&lt;/a&gt;.
I hope that Ray takes on ideas for using standard Python interfaces so that
more of the community can adopt it more quickly. I encourage people to check
out the talk and give Ray a try. It’s pretty slick.&lt;/p&gt;
&lt;iframe width="560" height="315" src="https://www.youtube.com/embed/D_oz7E4v-U0"
frameborder="0" allow="autoplay; encrypted-media" allowfullscreen&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/2018/07/17/dask-dev.md&lt;/span&gt;, line 176)&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="planning-conversations-for-dask-ml"&gt;
&lt;h1&gt;Planning conversations for Dask-ML&lt;/h1&gt;
&lt;p&gt;Dask and Scikit-learn developers had the opportunity to sit down again and
raise a number of issues to help plan near-term development. This focused
mostly around building important case studies to motivate future development,
and identifying algorithms and other projects to target for near-term
integration.&lt;/p&gt;
&lt;section id="case-studies"&gt;
&lt;h2&gt;Case Studies&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml/issues/302"&gt;What is the purpose of a case study: dask/dask-ml #302&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml/issues/295"&gt;Case study: Sparse Criteo Dataset: dask/dask-ml #295&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml/issues/296"&gt;Case study: Large scale text classification: dask/dask-ml #296&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml/issues/297"&gt;Case study: Transfer learning from pre-trained model: dask/dask-ml #297&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="algorithms"&gt;
&lt;h2&gt;Algorithms&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml/issues/299"&gt;Gradient boosted trees with Numba: dask/dask-ml #299&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-ml/issues/300"&gt;Parallelize Scikit-Optimize for hyperparameter optimization: dask/dask-ml #300&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="id3"&gt;
&lt;h2&gt;Get involved&lt;/h2&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/2018/07/17/dask-dev.md&lt;/span&gt;, line 196); &lt;em&gt;&lt;a href="#id3"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “get involved”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;We could use help in building out case studies to drive future development in
the project. There are also several algorithmic places to get involved.
Dask-ML is a young and fast-moving project with many opportunities for new
developers to get involved.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/07/17/dask-dev.md&lt;/span&gt;, line 203)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="dask-and-umap-for-low-dimensional-embeddings"&gt;
&lt;h1&gt;Dask and UMAP for low-dimensional embeddings&lt;/h1&gt;
&lt;p&gt;Leland McKinnes gave a great talk &lt;a class="reference external" href="https://youtu.be/nq6iPZVUxZU"&gt;Uniform Manifold Approximation and
Projection for Dimensionality Reduction&lt;/a&gt; in which
he lays out a well founded algorithm for dimensionality reduction, similar to
PCA or T-SNE, but with some nice properties. He worked together with some Dask
developers where we identified some challenges due to dask array slicing with
random-ish slices.&lt;/p&gt;
&lt;p&gt;A proposal to fix this problem lives here, if anyone wants a fun problem to work on:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/issues/3409#issuecomment-405254656"&gt;dask/dask #3409 (comment)&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;iframe width="560" height="315" src="https://www.youtube.com/embed/nq6iPZVUxZU"
frameborder="0" allow="autoplay; encrypted-media" allowfullscreen&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/2018/07/17/dask-dev.md&lt;/span&gt;, line 219)&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="dask-stories"&gt;
&lt;h1&gt;Dask stories&lt;/h1&gt;
&lt;p&gt;We soft-launched &lt;a class="reference external" href="http://dask-stories.readthedocs.io/en/latest/"&gt;Dask Stories&lt;/a&gt;
a webpage and project to collect user and share stories about how people use
Dask in practice. We’re also delivering a separate blogpost about this today.&lt;/p&gt;
&lt;p&gt;See blogpost: &lt;a class="reference internal" href="../2018/07/16/dask-stories/"&gt;&lt;span class="doc std std-doc"&gt;Who uses Dask?&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;If you use Dask and want to share your story we would absolutely welcome your
experience. Having people like yourself share how they use Dask is incredibly
important for the project.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/07/17/dask-dev/"/>
    <summary>This work is supported by Anaconda Inc</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-07-17T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/07/16/dask-stories/</id>
    <title>Who uses Dask?</title>
    <updated>2018-07-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://anaconda.com"&gt;Anaconda Inc&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;People often ask general questions like “Who uses Dask?” or more specific
questions like the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;For what applications do people use Dask dataframe?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How many machines do people often use with Dask?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How far does Dask scale?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Does dask get used on imaging data?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Does anyone use Dask with Kubernetes/Yarn/SGE/Mesos/… ?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Does anyone in the insurance industry use Dask?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;…&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This yields interesting and productive conversations where new users can dive
into historical use cases which informs their choices if and how they use the
project in the future.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;New users can learn a lot from existing users.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;To further enable this conversation we’ve made a new tiny project,
&lt;a class="reference external" href="https://dask-stories.readthedocs.io"&gt;dask-stories&lt;/a&gt;. This is a small
documentation page where people can submit how they use Dask and have that
published for others to see.&lt;/p&gt;
&lt;p&gt;To seed this site six generous users have written down how their group uses
Dask. You can read about them here:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask-stories.readthedocs.io/en/latest/sidewalk-labs.html"&gt;Sidewalk Labs: Civic Modeling&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask-stories.readthedocs.io/en/latest/mosquito-sequencing.html"&gt;Genome Sequencing for Mosquitoes&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask-stories.readthedocs.io/en/latest/fullspectrum.html"&gt;Full Spectrum: Credit and Banking&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask-stories.readthedocs.io/en/latest/icecube-cosmic-rays.html"&gt;Ice Cube: Detecting Cosmic Rays&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask-stories.readthedocs.io/en/latest/pangeo.html"&gt;Pangeo: Earth Science&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask-stories.readthedocs.io/en/latest/hydrologic-modeling.html"&gt;NCAR: Hydrologic Modeling&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We’ve focused on a few questions, available in &lt;a class="reference external" href="http://dask-stories.readthedocs.io/en/latest/template.html"&gt;our
template&lt;/a&gt; that
focus on problems over technology, and include negative as well as positive
feedback to get a complete picture.&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Who am I?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;What problem am I trying to solve?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How Dask helps?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;What pain points did I run into with Dask?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;What technology do I use around Dask?&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/07/16/dask-stories.md&lt;/span&gt;, line 53)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="easy-to-contribute"&gt;

&lt;p&gt;Contributions to this site are simple Markdown documents submitted as pull
requests to
&lt;a class="reference external" href="https://github.com/dask/dask-stories"&gt;github.com/dask/dask-stories&lt;/a&gt;. The site
is then built with ReadTheDocs and updated immediately. We tried to make this
as smooth and familiar to our existing userbase as possible.&lt;/p&gt;
&lt;p&gt;This is important. Sharing real-world experiences like this are probably more
valuable than code contributions to the Dask project at this stage. Dask is
more technically mature than it is well-known. Users look to other users to
help them understand a project (think of every time you’ve Googled for “&lt;em&gt;some
tool&lt;/em&gt; in &lt;em&gt;some topic&lt;/em&gt;”)&lt;/p&gt;
&lt;p&gt;If you use Dask today in an interesting way then please share your story.
The world would love to hear your voice.&lt;/p&gt;
&lt;p&gt;If you maintain another project you might consider implementing the same model.
I hope that this proves successful enough for other projects in the ecosystem
to reuse.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/07/16/dask-stories/"/>
    <summary>This work is supported by Anaconda Inc</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-07-16T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/07/08/dask-dev/</id>
    <title>Dask Development Log</title>
    <updated>2018-07-08T00: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://anaconda.com"&gt;Anaconda Inc&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;To increase transparency I’m trying to blog more often about the current work
going on around Dask and related projects. Nothing here is ready for
production. This blogpost is written in haste, so refined polish should not be
expected.&lt;/p&gt;
&lt;p&gt;Current efforts for June 2018 in Dask and Dask-related projects include
the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Yarn Deployment&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;More examples for machine learning&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Incremental machine learning&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;HPC Deployment configuration&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/07/08/dask-dev.md&lt;/span&gt;, line 23)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="yarn-deployment"&gt;

&lt;p&gt;Dask developers often get asked &lt;em&gt;How do I deploy Dask on my Hadoop/Spark/Hive
cluster?&lt;/em&gt;. We haven’t had a very good answer until recently.&lt;/p&gt;
&lt;p&gt;Most Hadoop/Spark/Hive clusters are actually &lt;em&gt;Yarn&lt;/em&gt; clusters. Yarn is the most
common cluster manager used by many clusters that are typically used to run
Hadoop/Spark/Hive jobs including any cluster purchased from a vendor like
Cloudera or Hortonworks. If your application can run on Yarn then it can be a
first class citizen here.&lt;/p&gt;
&lt;p&gt;Unfortunately Yarn has really only been accessible through a Java API, and so
has been difficult for Dask to interact with. That’s changing now with a few
projects, including:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask-yarn.readthedocs.io"&gt;dask-yarn&lt;/a&gt;: an easy way to launch Dask on
Yarn clusters&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://jcrist.github.io/skein/"&gt;skein&lt;/a&gt;: an easy way to launch generic
services on Yarn clusters (this is primarily what backs dask-yarn)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://conda.github.io/conda-pack/"&gt;conda-pack&lt;/a&gt;: an easy way to bundle
together a conda package into a redeployable environment, such as is useful
when launching Python applications on Yarn&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This work is all being done by &lt;a class="reference external" href="http://jcrist.github.io/"&gt;Jim Crist&lt;/a&gt; who is, I
believe, currently writing up a blogpost about the topic at large. Dask-yarn
was soft-released last week though, so people should give it a try and report
feedback on the &lt;a class="reference external" href="https://github.com/dask/dask-yarn"&gt;dask-yarn issue tracker&lt;/a&gt;.
If you ever wanted direct help on your cluster, now is the right time because
Jim is working on this actively and is not yet drowned in user requests so
generally has a fair bit of time to investigate particular cases.&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_yarn&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;YarnCluster&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="c1"&gt;# Create a cluster where each worker has two cores and eight GB of memory&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;YarnCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;environment&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;environment.tar.gz&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                      &lt;span class="n"&gt;worker_vcores&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="n"&gt;worker_memory&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;8GB&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="c1"&gt;# Scale out to ten such workers&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scale&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# Connect to the cluster&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/07/08/dask-dev.md&lt;/span&gt;, line 69)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="more-examples-for-machine-learning"&gt;
&lt;h1&gt;More examples for machine learning&lt;/h1&gt;
&lt;p&gt;Dask maintains a Binder of simple examples that show off various ways to use
the project. This allows people to click a link on the web and quickly be
taken to a Jupyter notebook running on the cloud. It’s a fun way to quickly
experience and learn about a new project.&lt;/p&gt;
&lt;p&gt;Previously we had a single example for arrays, dataframes, delayed, machine
learning, etc.&lt;/p&gt;
&lt;p&gt;Now &lt;a class="reference external" href="https://stsievert.com/"&gt;Scott Sievert&lt;/a&gt; is expanding the examples within
the machine learning section. He has submitted the following two so far:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://mybinder.org/v2/gh/dask/dask-examples/main?urlpath=%2Ftree%2Fmachine-learning%2Fincremental.ipynb"&gt;Incremental training with Scikit-Learn and large datasets&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://mybinder.org/v2/gh/dask/dask-examples/main?urlpath=%2Ftree%2Fmachine-learning%2Fxgboost.ipynb"&gt;Dask and XGBoost&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;I believe he’s planning on more. If you use
&lt;a class="reference external" href="http://dask-ml.readthedocs.io/en/latest/"&gt;dask-ml&lt;/a&gt; and have recommendations or
want to help, you might want to engage in the &lt;a class="reference external" href="https://github.com/dask/dask-ml/issues/new"&gt;dask-ml issue
tracker&lt;/a&gt; or &lt;a class="reference external" href="https://github.com/dask/dask-examples/issues/new"&gt;dask-examples issue
tracker&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/07/08/dask-dev.md&lt;/span&gt;, line 91)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="incremental-training"&gt;
&lt;h1&gt;Incremental training&lt;/h1&gt;
&lt;p&gt;The incremental training mentioned as an example above is also new-ish. This
is a Scikit-Learn style meta-estimator that wraps around other estimators that
support the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;partial_fit&lt;/span&gt;&lt;/code&gt; method. It enables training on large datasets in an
incremental or batchwise fashion.&lt;/p&gt;
&lt;section id="before"&gt;
&lt;h2&gt;Before&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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;sklearn.linear_model&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;SGDClassifier&lt;/span&gt;

&lt;span class="n"&gt;sgd&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;SGDClassifier&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="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="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;filename&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="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&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="n"&gt;X&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;

    &lt;span class="n"&gt;sgd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;partial_fit&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;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="after"&gt;
&lt;h2&gt;After&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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;sklearn.linear_model&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;SGDClassifier&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_ml.wrappers&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;Incremental&lt;/span&gt;

&lt;span class="n"&gt;sgd&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;SGDClassifier&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;inc&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Incremental&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sgd&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.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&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;X&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;
&lt;span class="n"&gt;inc&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="analysis"&gt;
&lt;h2&gt;Analysis&lt;/h2&gt;
&lt;p&gt;From a parallel computing perspective this is a very simple and un-sexy way of
doing things. However my understanding is that it’s also quite pragmatic. In
a distributed context we leave a lot of possible computation on the table (the
solution is inherently sequential) but it’s fun to see the model jump around
the cluster as it absorbs various chunks of data and then moves on.&lt;/p&gt;
&lt;p&gt;&lt;img src="https://user-images.githubusercontent.com/1320475/42237033-2bddf11e-7eec-11e8-88c5-5f0ebd2fb4df.png"
     width="70%"
     alt="Incremental training with Dask-ML"&gt;&lt;/p&gt;
&lt;p&gt;There’s ongoing work on how best to combine this with other work like pipelines
and hyper-parameter searches to fill in the extra computation.&lt;/p&gt;
&lt;p&gt;This work was primarily done by &lt;a class="reference external" href="https://tomaugspurger.github.io/"&gt;Tom Augspurger&lt;/a&gt;
with help from &lt;a class="reference external" href="https://stsievert.com/"&gt;Scott Sievert&lt;/a&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/07/08/dask-dev.md&lt;/span&gt;, line 148)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="dask-user-stories"&gt;
&lt;h1&gt;Dask User Stories&lt;/h1&gt;
&lt;p&gt;Dask developers are often asked “Who uses Dask?”. This is a hard question to
answer because, even though we’re inundated with thousands of requests for
help from various companies and research groups, it’s never fully clear who
minds having their information shared with others.&lt;/p&gt;
&lt;p&gt;We’re now trying to crowdsource this information in a more explicit way by
having users tell their own stories. Hopefully this helps other users in their
field understand how Dask can help and when it might (or might not) be useful
to them.&lt;/p&gt;
&lt;p&gt;We originally collected this information in a &lt;a class="reference external" href="https://goo.gl/forms/JEebEFTOPrWa3P4h1"&gt;Google
Form&lt;/a&gt; but have since then moved it to a
&lt;a class="reference external" href="https://github.com/mrocklin/dask-stories"&gt;Github repository&lt;/a&gt;. Eventually
we’ll publish this as a &lt;a class="reference external" href="https://github.com/mrocklin/dask-stories/issues/7"&gt;proper web
site&lt;/a&gt; and include it in our
documentation.&lt;/p&gt;
&lt;p&gt;If you use Dask and want to share your story this is a great way to contribute
to the project. Arguably Dask needs more help with spreading the word than it
does with technical solutions.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/07/08/dask-dev.md&lt;/span&gt;, line 171)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="hpc-deployments"&gt;
&lt;h1&gt;HPC Deployments&lt;/h1&gt;
&lt;p&gt;The &lt;a class="reference external" href="http://dask-jobqueue.readthedocs.io/en/latest/"&gt;Dask Jobqueue&lt;/a&gt; package for
deploying Dask on traditional HPC machines is nearing another release. We’ve
changed around a lot of the parameters and configuration options in order to
improve the onboarding experience for new users. It has been going very
smoothly in recent engagements with new groups, but will mean a breaking
change for existing users of the sub-project.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/07/08/dask-dev/"/>
    <summary>This work is supported by Anaconda Inc</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-07-08T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/06/26/dask-scaling-limits/</id>
    <title>Dask Scaling Limits</title>
    <updated>2018-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://anaconda.com"&gt;Anaconda Inc.&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/06/26/dask-scaling-limits.md&lt;/span&gt;, line 10)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="history"&gt;

&lt;p&gt;For the first year of Dask’s life it focused exclusively on single node
parallelism. We felt then that efficiently supporting 100+GB datasets on
personal laptops or 1TB datasets on large workstations was a sweet spot for
productivity, especially when avoiding the pain of deploying and configuring
distributed systems. We still believe in the efficiency of single-node
parallelism, but in the years since, Dask has extended itself to support larger
distributed systems.&lt;/p&gt;
&lt;p&gt;After that first year, Dask focused equally on both single-node and distributed
parallelism. We maintain &lt;a class="reference external" href="http://dask.pydata.org/en/latest/scheduling.html"&gt;two entirely separate
schedulers&lt;/a&gt;, one optimized for
each case. This allows Dask to be very simple to use on single machines, but
also scale up to thousand-node clusters and 100+TB datasets when needed with
the same API.&lt;/p&gt;
&lt;p&gt;Dask’s distributed system has a single central scheduler and many distributed
workers. This is a common architecture today that scales out to a few thousand
nodes. Roughly speaking Dask scales about the same as a system like Apache
Spark, but less well than a high-performance system like MPI.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/06/26/dask-scaling-limits.md&lt;/span&gt;, line 32)&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="an-example"&gt;
&lt;h1&gt;An Example&lt;/h1&gt;
&lt;p&gt;Most Dask examples in blogposts or talks are on modestly sized datasets,
usually in the 10-50GB range. This, combined with Dask’s history with
medium-data on single-nodes may have given people a more humble impression of
Dask than is appropriate.&lt;/p&gt;
&lt;p&gt;As a small nudge, here is an example using Dask to interact with 50 36-core
nodes on an artificial terabyte dataset.&lt;/p&gt;
&lt;iframe width="700"
        height="394"
        src="https://www.youtube.com/embed/nH_AQo8WdKw"
        frameborder="0"
        allow="autoplay; encrypted-media"
        allowfullscreen&gt;&lt;/iframe&gt;
&lt;p&gt;This is a common size for a typical modestly sized Dask cluster. We usually
see Dask deployment sizes either in the tens of machines (usually with Hadoop
style or ad-hoc enterprise clusters), or in the few-thousand range (usually
with high performance computers or cloud deployments). We’re showing the
modest case here just due to lack of resources. Everything in that example
should work fine scaling out a couple extra orders of magnitude.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/06/26/dask-scaling-limits.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="challenges-to-scaling-out"&gt;
&lt;h1&gt;Challenges to Scaling Out&lt;/h1&gt;
&lt;p&gt;For the rest of the article we’ll talk about common causes that we see today
that get in the way of scaling out. These are collected from experience
working both with people in the open source community, as well as private
contracts.&lt;/p&gt;
&lt;section id="simple-map-reduce-style"&gt;
&lt;h2&gt;Simple Map-Reduce style&lt;/h2&gt;
&lt;p&gt;If you’re doing simple map-reduce style parallelism then things will be pretty
smooth out to a large number of nodes. However, there are still some
limitations to keep in mind:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;The scheduler will have at least one, and possibly a few connections open
to each worker. You’ll want to ensure that your machines can have many
open file handles at once. Some Linux distributions cap this at 1024 by
default, but it is easy to change.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The scheduler has an overhead of around 200 microseconds per task.
So if each task takes one second then your scheduler can saturate 5000
cores, but if each task takes only 100ms then your scheduler can only
saturate around 500 cores, and so on. Task duration imposes an inversely
proportional constraint on scaling.&lt;/p&gt;
&lt;p&gt;If you want to scale larger than this then your tasks will need to
start doing more work in each task to avoid overhead. Often this involves
moving inner for loops within tasks rather than spreading them out to many
tasks.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
&lt;section id="more-complex-algorithms"&gt;
&lt;h2&gt;More complex algorithms&lt;/h2&gt;
&lt;p&gt;If you’re doing more complex algorithms (which is common among Dask users) then
many more things can break along the way. High performance computing isn’t
about doing any one thing well, it’s about doing &lt;em&gt;nothing badly&lt;/em&gt;. This section
lists a few issues that arise for larger deployments:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;Dask collection algorithms may be suboptimal.&lt;/p&gt;
&lt;p&gt;The parallel algorithms in Dask-array/bag/dataframe/ml are &lt;em&gt;pretty&lt;/em&gt; good,
but as Dask scales out to larger clusters and its algorithms are used by
more domains we invariably find that small corners of the API fail beyond a
certain point. Luckily these are usually pretty easy to fix after they are
reported.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The graph size may grow too large for the scheduler&lt;/p&gt;
&lt;p&gt;The metadata describing your computation has to all fit on a single
machine, the Dask scheduler. This metadata, the task graph, can grow big
if you’re not careful. It’s nice to have a scheduler process with at least
a few gigabytes of memory if you’re going to be processing million-node
task graphs. A task takes up around 1kB of memory &lt;em&gt;if&lt;/em&gt; you’re careful to
avoid closing over any unnecessary local data.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The graph serialization time may become annoying for interactive use&lt;/p&gt;
&lt;p&gt;Again, if you have million node task graphs you’re going to be serializaing
them up and passing them from the client to the scheduler. This is &lt;em&gt;fine&lt;/em&gt;,
assuming they fit at both ends, but can take up some time and limit
interactivity. If you press &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;compute&lt;/span&gt;&lt;/code&gt; and nothing shows up on the
dashboard for a minute or two, this is what’s happening.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The interactive dashboard plots stop being as useful&lt;/p&gt;
&lt;p&gt;Those beautiful plots on the dashboard were mostly designed for deployments
with 1-100 nodes, but not 1000s. Seeing the start and stop time of every
task of a million-task computation just isn’t something that our brains can
fully understand.&lt;/p&gt;
&lt;p&gt;This is something that we would like to improve. If anyone out there is
interested in scalable performance diagnostics, please get involved.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Other components that you rely on, like distributed storage, may also start
to break&lt;/p&gt;
&lt;p&gt;Dask provides users more power than they’re accustomed to.
It’s easy for them to accidentally clobber some other component of their
systems, like distributed storage, a local database, the network, and so
on, with too many requests.&lt;/p&gt;
&lt;p&gt;Many of these systems provide abstractions that are very well tested and
stable for normal single-machine use, but that quickly become brittle when
you have a thousand machines acting on them with the full creativity of a
novice user. Dask provies some primitives like distributed locks and
queues to help control access to these resources, but it’s on the user to
use them well and not break things.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/06/26/dask-scaling-limits.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&gt;
&lt;section id="conclusion"&gt;
&lt;h1&gt;Conclusion&lt;/h1&gt;
&lt;p&gt;Dask scales happily out to tens of nodes, like in the example above, or to
thousands of nodes, which I’m not showing here simply due to lack of resources.&lt;/p&gt;
&lt;p&gt;Dask provides this scalability while still maintaining the flexibility and
freedom to build custom systems that has defined the project since it began.
However, the combination of scalability and freedom makes it hard for Dask to
fully protect users from breaking things. It’s much easier to protect users
when you can constrain what they can do. When users stick to standard
workflows like Dask dataframe or Dask array they’ll probably be ok, but when
operating with full creativity at the thousand-node scale some expertise will
invariably be necessary. We try hard to provide the diagnostics and tools
necessary to investigate issues and control operation. The project is getting
better at this every day, in large part due to some expert users out there.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/06/26/dask-scaling-limits.md&lt;/span&gt;, line 158)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="a-call-for-examples"&gt;
&lt;h1&gt;A Call for Examples&lt;/h1&gt;
&lt;p&gt;Do you use Dask on more than one machine to do interesting work?
We’d love to hear about it either in the comments below, or in this &lt;a class="reference external" href="https://goo.gl/forms/ueIMoGl6ZPl529203"&gt;online
form&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/06/26/dask-scaling-limits/"/>
    <summary>This work is supported by Anaconda Inc.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-06-26T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/06/14/dask-0.18.0/</id>
    <title>Dask Release 0.18.0</title>
    <updated>2018-06-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://anaconda.com"&gt;Anaconda Inc.&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;I’m pleased to announce the release of Dask version 0.18.0. This is a major
release with breaking changes and new features.
The last release was 0.17.5 on May 4th.
This blogpost outlines notable changes since the last release blogpost for
0.17.2 on March 21st.&lt;/p&gt;
&lt;p&gt;You can conda install Dask:&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 pip install from PyPI:&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[complete] --upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Full changelogs are available here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/blob/master/docs/source/changelog.rst"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/docs/source/changelog.rst"&gt;dask/distributed&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We list some breaking changes below, followed up by changes that are less
important, but still fun.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/06/14/dask-0.18.0.md&lt;/span&gt;, line 32)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="context"&gt;

&lt;p&gt;The Dask core library is nearing a 1.0 release.
Before that happens, we need to do some housecleaning.
This release starts that process,
replaces some existing interfaces,
and builds up some needed infrastructure.
Almost all of the changes in this release include clean deprecation warnings,
but future releases will remove the old functionality, so now would be a good
time to check in.&lt;/p&gt;
&lt;p&gt;As happens with any release that starts breaking things,
many other smaller breaks get added on as well.
I’m personally very happy with this release because many aspects of using Dask
now feel a lot cleaner, however heavy users of Dask will likely experience
mild friction. Hopefully this post helps explain some of the larger changes.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/06/14/dask-0.18.0.md&lt;/span&gt;, line 49)&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="notable-breaking-changes"&gt;
&lt;h1&gt;Notable Breaking changes&lt;/h1&gt;
&lt;section id="centralized-configuration"&gt;
&lt;h2&gt;Centralized configuration&lt;/h2&gt;
&lt;p&gt;Taking full advantage of Dask sometimes requires user configuration, especially
in a distributed setting. This might be to control logging verbosity, specify
cluster configuration, provide credentials for security, or any of several
other options that arise in production.&lt;/p&gt;
&lt;p&gt;We’ve found that different computing cultures like to specify configuration in
several different ways:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Configuration files&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Environment variables&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Directly within Python code&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Previously this was handled with a variety of different solutions among the
different dask subprojects. The dask-distributed project had one system,
dask-kubernetes had another, and so on.&lt;/p&gt;
&lt;p&gt;Now we centralize configuration in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.config&lt;/span&gt;&lt;/code&gt; module, which collects
configuration from config files, environment variables, and runtime code, and
makes it centrally available to all Dask subprojects. A number of Dask
subprojects (dask.distributed,
&lt;a class="reference external" href="http://dask-kubernetes.readthedocs.io/en/latest/"&gt;dask-kubernetes&lt;/a&gt;, and
&lt;a class="reference external" href="http://dask-jobqueue.readthedocs.io/en/latest/"&gt;dask-jobqueue&lt;/a&gt;), are being
co-released at the same time to take advantage of this.&lt;/p&gt;
&lt;p&gt;If you were actively using Dask.distributed’s configuration files some things
have changed:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;The configuration is now namespaced and more heavily nested. Here is an
example from the &lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/distributed/distributed.yaml"&gt;dask.distributed default config
file&lt;/a&gt;
today:&lt;/p&gt;
&lt;div class="highlight-yaml notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nt"&gt;distributed&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;version&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;2&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;scheduler&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;allowed-failures&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;3&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="c1"&gt;# number of retries before a task is considered bad&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;work-stealing&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;True&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="c1"&gt;# workers should steal tasks from each other&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;worker-ttl&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;null&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="c1"&gt;# like &amp;#39;60s&amp;#39;. Workers must heartbeat faster than this&lt;/span&gt;

&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;worker&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;multiprocessing-method&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;forkserver&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;use-file-locking&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;True&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The default configuration location has moved from &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;~/.dask/config.yaml&lt;/span&gt;&lt;/code&gt; to
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;~/.config/dask/distributed.yaml&lt;/span&gt;&lt;/code&gt;, where it will live along side several
other files like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;kubernetes.yaml&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;jobqueue.yaml&lt;/span&gt;&lt;/code&gt;, and so on.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;However, your old configuration files will still be found and their values
will be used appropriately. We don’t make any attempt to migrate your old
config values to the new location though. You may want to delete the
auto-generated &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;~/.dask/config.yaml&lt;/span&gt;&lt;/code&gt; file at some point, if you felt like being
particularly clean.&lt;/p&gt;
&lt;p&gt;You can learn more about Dask’s configuration in &lt;a class="reference external" href="http://dask.pydata.org/en/latest/configuration.html"&gt;Dask’s configuration
documentation&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="replaced-the-common-get-keyword-with-scheduler"&gt;
&lt;h2&gt;Replaced the common get= keyword with scheduler=&lt;/h2&gt;
&lt;p&gt;Dask can execute code with a variety of scheduler backends based on threads,
processes, single-threaded execution, or distributed clusters.&lt;/p&gt;
&lt;p&gt;Previously, users selected between these backends using the somewhat
generically named &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;get=&lt;/span&gt;&lt;/code&gt; keyword:&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;x&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;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;threaded&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;x&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;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;multiprocessing&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;x&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;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;local&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_sync&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We’ve replaced this with a newer, and hopefully more clear, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scheduler=&lt;/span&gt;&lt;/code&gt; keyword:&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;x&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;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;threads&amp;#39;&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;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;processes&amp;#39;&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;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;single-threaded&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;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;get=&lt;/span&gt;&lt;/code&gt; keyword has been deprecated and will raise a warning. It will be
removed entirely on the next major release.&lt;/p&gt;
&lt;p&gt;For more information, see &lt;a class="reference external" href="http://dask.pydata.org/en/latest/scheduling.html"&gt;documentation on selecting different schedulers&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="replaced-dask-set-options-with-dask-config-set"&gt;
&lt;h2&gt;Replaced dask.set_options with dask.config.set&lt;/h2&gt;
&lt;p&gt;Related to the configuration changes, we now include runtime state in the
configuration. Previously people used to set runtime state with the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.set_options&lt;/span&gt;&lt;/code&gt; context manager. Now we recommend using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.config.set&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="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_options&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;threads&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;  &lt;span class="c1"&gt;# Before&lt;/span&gt;
    &lt;span class="o"&gt;...&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;threads&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;  &lt;span class="c1"&gt;# After&lt;/span&gt;
    &lt;span class="o"&gt;...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.set_options&lt;/span&gt;&lt;/code&gt; function is now an alias to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.config.set&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="removed-the-dask-array-learn-subpackage"&gt;
&lt;h2&gt;Removed the dask.array.learn subpackage&lt;/h2&gt;
&lt;p&gt;This was unadvertised and saw very little use. All functionality (and much
more) is now available in &lt;a class="reference external" href="http://dask-ml.readthedocs.io/en/latest/"&gt;Dask-ML&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="other"&gt;
&lt;h2&gt;Other&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;We’ve removed the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;token=&lt;/span&gt;&lt;/code&gt; keyword from map_blocks and moved the
functionality to the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;name=&lt;/span&gt;&lt;/code&gt; keyword.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.distributed.worker_client&lt;/span&gt;&lt;/code&gt; automatically rejoins the threadpool when
you close the context manager.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The Dask.distributed protocol now interprets msgpack arrays as tuples
rather than lists.&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/2018/06/14/dask-0.18.0.md&lt;/span&gt;, line 168)&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="fun-new-features"&gt;
&lt;h1&gt;Fun new features&lt;/h1&gt;
&lt;section id="arrays"&gt;
&lt;h2&gt;Arrays&lt;/h2&gt;
&lt;section id="generalized-universal-functions"&gt;
&lt;h3&gt;Generalized Universal Functions&lt;/h3&gt;
&lt;p&gt;Dask.array now supports Numpy-style
&lt;a class="reference external" href="https://docs.scipy.org/doc/numpy-1.13.0/reference/c-api.generalized-ufuncs.html"&gt;Generalized Universal Functions (gufuncs)&lt;/a&gt;
transparently.
This means that you can apply normal Numpy GUFuncs, like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;eig&lt;/span&gt;&lt;/code&gt; in the example
below, directly onto a Dask 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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="kn"&gt;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="c1"&gt;# Apply a Numpy GUFunc, eig, directly onto a Dask array&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="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;10&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="mi"&gt;10&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;2&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="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;w&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;np&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;_umath_linalg&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;eig&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;output_dtypes&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;float&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="nb"&gt;float&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="c1"&gt;# w and v are dask arrays with eig applied along the latter two axes&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Numpy has gufuncs of many of its internal functions, but they haven’t
yet decided to switch these out to the public API.
Additionally we can define GUFuncs with other projects, like Numba:&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;numba&lt;/span&gt;

&lt;span class="nd"&gt;@numba&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;vectorize&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;float64&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;float64&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;float64&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;f&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;y&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;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;y&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;f&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;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# if x and y are dask arrays, then z will be too&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;What I like about this is that Dask and Numba developers didn’t coordinate
at all on this feature, it’s just that they both support the Numpy GUFunc
protocol, so you get interactions like this for free.&lt;/p&gt;
&lt;p&gt;For more information see &lt;a class="reference external" href="http://dask.pydata.org/en/latest/array-gufunc.html"&gt;Dask’s GUFunc documentation&lt;/a&gt;. This work was done by &lt;a class="reference external" href="https://github.com/magonser"&gt;Markus Gonser (&amp;#64;magonser)&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="new-auto-value-for-rechunking"&gt;
&lt;h3&gt;New “auto” value for rechunking&lt;/h3&gt;
&lt;p&gt;Dask arrays now accept a value, “auto”, wherever a chunk value would previously
be accepted. This asks Dask to rechunk those dimensions to achieve a good
default chunk size.&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;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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="c1"&gt;# single chunk in this dimension&lt;/span&gt;
  &lt;span class="c1"&gt;# 1: 100e6 / x.dtype.itemsize / x.shape[0],  # before we had to calculate manually&lt;/span&gt;
    &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;      &lt;span class="c1"&gt;# Now we allow this dimension to respond to get ideal chunk size&lt;/span&gt;
&lt;span class="p"&gt;})&lt;/span&gt;

&lt;span class="c1"&gt;# or&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;from_array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;img&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This also checks the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;array.chunk-size&lt;/span&gt;&lt;/code&gt; config value for optimal chunk sizes&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;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;array.chunk-size&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;&amp;#39;128MiB&amp;#39;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;To be clear, this doesn’t support “automatic chunking”, which is a very hard
problem in general. Users still need to be aware of their computations and how
they want to chunk, this just makes it marginally easier to make good
decisions.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="algorithmic-improvements"&gt;
&lt;h3&gt;Algorithmic improvements&lt;/h3&gt;
&lt;p&gt;Dask.array gained a full &lt;a class="reference external" href="http://dask.pydata.org/en/latest/array-api.html#dask.array.einsum"&gt;einsum&lt;/a&gt; implementation thanks to &lt;a class="reference external" href="https://github.com/sjperkins"&gt;Simon Perkins&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Also, Dask.array’s QR decompositions has become nicer in two ways:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;They support &lt;a class="reference external" href="http://dask.pydata.org/en/latest/array-api.html#dask.array.linalg.sfqr"&gt;short-and-fat arrays&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The &lt;a class="reference external" href="http://dask.pydata.org/en/latest/array-api.html#dask.array.linalg.tsqr"&gt;tall-and-skinny&lt;/a&gt;
variant now operates more robustly in less memory. Here is a friendly GIF
of execution:&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;img src="https://user-images.githubusercontent.com/306380/41350133-175cac7e-6ee0-11e8-9a0e-785c6e846409.gif" width="40%"&gt;
&lt;p&gt;This work is greatly appreciated and was done by &lt;a class="reference external" href="https://github.com/convexset"&gt;Jeremy Chan&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Native support for the &lt;a class="reference external" href="http://zarr.readthedocs.io/en/stable/"&gt;Zarr format&lt;/a&gt; for
chunked n-dimensional arrays landed thanks to &lt;a class="reference external" href="https://github.com/martindurant"&gt;Martin
Durant&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/jakirkham"&gt;John A
Kirkham&lt;/a&gt;. Zarr has been especially useful due to
its speed, simple spec, support of the full NetCDF style conventions, and
amenability to cloud storage.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="dataframes-and-pandas-0-23"&gt;
&lt;h2&gt;Dataframes and Pandas 0.23&lt;/h2&gt;
&lt;p&gt;As usual, Dask Dataframes had many small improvements. Of note is continued
compatibility with the just-released Pandas 0.23, and some new data ingestion
formats.&lt;/p&gt;
&lt;p&gt;Dask.dataframe is consistent with changes in the recent Pandas 0.23 release
thanks to &lt;a class="reference external" href="https://github.com/TomAugspurger"&gt;Tom Augspurger&lt;/a&gt;.&lt;/p&gt;
&lt;section id="orc-support"&gt;
&lt;h3&gt;Orc support&lt;/h3&gt;
&lt;p&gt;Dask.dataframe has grown a reader for the &lt;a class="reference external" href="https://orc.apache.org/"&gt;Apache ORC&lt;/a&gt; format.&lt;/p&gt;
&lt;p&gt;Orc is a format for tabular data storage that is common in the Hadoop ecosystem.
The new
&lt;a class="reference external" href="http://dask.pydata.org/en/latest/dataframe-api.html#dask.dataframe.read_hdf"&gt;dd.read_orc&lt;/a&gt;
function parallelizes around similarly new ORC functionality within PyArrow .
Thanks to &lt;a class="reference external" href="https://github.com/jcrist"&gt;Jim Crist&lt;/a&gt; for the work on the Arrow side
and &lt;a class="reference external" href="https://github.com/martindurant"&gt;Martin Durant&lt;/a&gt; for parallelizing it with
Dask.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="read-json-support"&gt;
&lt;h3&gt;Read_json support&lt;/h3&gt;
&lt;p&gt;Dask.dataframe now has also grown a reader for JSON files.&lt;/p&gt;
&lt;p&gt;The &lt;a class="reference external" href="http://dask.pydata.org/en/latest/dataframe-api.html#dask.dataframe.read_json"&gt;dd.read_json&lt;/a&gt;
function matches most of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pandas.read_json&lt;/span&gt;&lt;/code&gt; API.&lt;/p&gt;
&lt;p&gt;This came about shortly after a recent &lt;a class="reference external" href="https://www.youtube.com/watch?v=X4YHGKj3V5M"&gt;PyCon 2018 talk comparing Spark and
Dask dataframe&lt;/a&gt; where &lt;a class="reference external" href="https://github.com/j-bennet"&gt;Irina
Truong&lt;/a&gt; mentioned that it was missing. Thanks to
&lt;a class="reference external" href="https://github.com/martindurant"&gt;Martin Durant&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/j-bennet"&gt;Irina
Truong&lt;/a&gt; for this contribution.&lt;/p&gt;
&lt;p&gt;See the &lt;a class="reference external" href="http://dask.pydata.org/en/latest/dataframe-create.html"&gt;dataframe data ingestion documentation&lt;/a&gt;
for more information about JSON, ORC, or any of the other formats
supported by Dask.dataframe.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="joblib"&gt;
&lt;h2&gt;Joblib&lt;/h2&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://pythonhosted.org/joblib/"&gt;Joblib&lt;/a&gt; library for parallel computing within
Scikit-Learn has had a &lt;a class="reference external" href="http://dask-ml.readthedocs.io/en/latest/joblib.html"&gt;Dask backend&lt;/a&gt;
for a while now. While it has always been pretty easy to use, it’s now
becoming much easier to use well without much expertise. After using this in
practice for a while together with the Scikit-Learn developers, we’ve
identified and smoothed over a number of usability issues. These changes will
only be fully available after the next Scikit-Learn release (hopefully soon) at
which point we’ll probably release a new blogpost dedicated to the 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/2018/06/14/dask-0.18.0.md&lt;/span&gt;, line 310)&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="related-projects"&gt;
&lt;h1&gt;Related projects&lt;/h1&gt;
&lt;p&gt;This release is timed with the following packages:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;dask&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;distributed&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;dask-kubernetes&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;There is also a new repository for deploying applications on YARN (a job
scheduler common in Hadoop environments) called
&lt;a class="reference external" href="https://jcrist.github.io/skein/"&gt;skein&lt;/a&gt;. Early adopters welcome.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/06/14/dask-0.18.0.md&lt;/span&gt;, line 322)&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;Since March 21st, the following people have contributed to the following repositories:&lt;/p&gt;
&lt;p&gt;The core Dask repository for parallel algorithms:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Andrethrill&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Beomi&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brendan Martin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Christopher Ren&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Guido Imperiale&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Diane Trout&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;fjetter&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Frederick&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Henry Doupe&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Bourbeau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jeremy Chen&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John A Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jon Mease&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jörg Dietrich&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kevin Mader&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ksenia Bobrova&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Larsr&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Marc Pfister&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Markus Gonser&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matt Lee&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Pierre-Bartet&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scott Sievert&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Simon Perkins&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stefan van der Walt&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stephan Hoyer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Uwe L. Korn&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Yu Feng&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The dask/distributed repository for distributed computing:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Bmaisonn&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Grant Jenks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Henry Doupe&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Irene Rodriguez&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Irina Truong&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John A Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joseph Atkins-Turkish&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kenneth Koski&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Loïc Estève&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Marius van Niekerk&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Olivier Grisel&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Russ Bubley&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tony Lorenzo&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The dask-kubernetes repository for deploying Dask on Kubernetes&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Brendan Martin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;J Gerard&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Olivier Grisel&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Yuvi Panda&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The dask-jobqueue repository for deploying Dask on HPC job schedulers&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Guillaume Eynard-Bontemps&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;jgerardsimcock&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joseph Hamman&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Loïc Estève&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ray Bell&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Rich Signell&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Shawn Taylor&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Spencer Clark&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The dask-ml repository for scalable machine learning:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Christopher Ren&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jeremy Chen&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scott Sievert&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;section id="id1"&gt;
&lt;h2&gt;Acknowledgements&lt;/h2&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/2018/06/14/dask-0.18.0.md&lt;/span&gt;, line 407); &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: “acknowledgements”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Thanks to Scott Sievert and James Bourbeau for their help editing this article.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/06/14/dask-0.18.0/"/>
    <summary>This work is supported by Anaconda Inc.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-06-14T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/05/27/beyond-numpy/</id>
    <title>Beyond Numpy Arrays in Python</title>
    <updated>2018-05-27T00:00:00+00:00</updated>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/05/27/beyond-numpy.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="executive-summary"&gt;

&lt;p&gt;In recent years Python’s array computing ecosystem has grown organically to support
GPUs, sparse, and distributed arrays.
This is wonderful and a great example of the growth that can occur in decentralized open source development.&lt;/p&gt;
&lt;p&gt;However to solidify this growth and apply it across the ecosystem we now need to do some central planning
to move from a pair-wise model where packages need to know about each other
to an ecosystem model where packages can negotiate by developing and adhering to community-standard protocols.&lt;/p&gt;
&lt;p&gt;With moderate effort we can define a subset of the Numpy API that works well across all of them,
allowing the ecosystem to more smoothly transition between hardware.
This post describes the opportunities and challenges to accomplish this.&lt;/p&gt;
&lt;p&gt;We start by discussing two kinds of libraries:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Libraries that &lt;em&gt;implement&lt;/em&gt; the Numpy API&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Libraries that &lt;em&gt;consume&lt;/em&gt; the Numpy API and build new functionality on top
of it&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/05/27/beyond-numpy.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="libraries-that-implement-the-numpy-api"&gt;
&lt;h1&gt;Libraries that Implement the Numpy API&lt;/h1&gt;
&lt;p&gt;The Numpy array is one of the foundations of the numeric Python ecosystem,
and serves as the standard model for similar libraries in other languages.
Today it is used to analyze satellite and biomedical imagery, financial models,
genomes, oceans and the atmosphere, super-computer simulations,
and data from thousands of other domains.&lt;/p&gt;
&lt;p&gt;However, Numpy was designed several years ago,
and its implementation is no longer optimal for some modern hardware,
particularly multi-core workstations, many-core GPUs, and distributed clusters.&lt;/p&gt;
&lt;p&gt;Fortunately other libraries implement the Numpy array API on these other architectures:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://cupy.chainer.org/"&gt;CuPy&lt;/a&gt;: implements the Numpy API on GPUs with CUDA&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://sparse.pydata.org/"&gt;Sparse&lt;/a&gt;: implements the Numpy API for sparse arrays that are mostly zeros&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask.pydata.org/"&gt;Dask array&lt;/a&gt;: implements the Numpy API in parallel for multi-core workstations or distributed clusters&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;So even when the Numpy implementation is no longer ideal,
the &lt;em&gt;Numpy API&lt;/em&gt; lives on in successor projects.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: the Numpy implementation remains ideal most of the time.
Dense in-memory arrays are still the common case.
This blogpost is about the minority of cases where Numpy is not ideal&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;So today we can write code similar code between all of
Numpy, GPU, sparse, and parallel 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="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="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;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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Runs on a single CPU&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;T&lt;/span&gt;&lt;span class="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;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;log&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&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;y&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;y&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="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;z&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cupy&lt;/span&gt;&lt;span class="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;cp&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;cp&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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Runs on a GPU&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;T&lt;/span&gt;&lt;span class="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;cp&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;log&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&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;y&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;y&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="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;z&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;get&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.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="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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Runs on many CPUs&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;T&lt;/span&gt;&lt;span class="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;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;log&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&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;y&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;y&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="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;z&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;compute&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;

&lt;span class="o"&gt;...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Additionally, each of the deep learning frameworks
(TensorFlow, PyTorch, MXNet)
has a Numpy-like thing that is &lt;em&gt;similar-ish&lt;/em&gt; to Numpy’s API,
but definitely not trying to be an exact match.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/05/27/beyond-numpy.md&lt;/span&gt;, line 84)&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="libraries-that-consume-and-extend-the-numpy-api"&gt;
&lt;h1&gt;Libraries that consume and extend the Numpy API&lt;/h1&gt;
&lt;p&gt;At the same time as the development of Numpy APIs for different hardware,
many libraries today build algorithmic functionality on top of the Numpy API:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://xarray.pydata.org/en/stable/"&gt;XArray&lt;/a&gt;
for labeled and indexed collections of arrays&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/hips/autograd"&gt;Autograd&lt;/a&gt; and
&lt;a class="reference external" href="https://github.com/google/tangent/"&gt;Tangent&lt;/a&gt;:
for automatic differentiation&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://tensorly.org/stable/index.html"&gt;TensorLy&lt;/a&gt;
for higher order array factorizations&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask.pydata.org"&gt;Dask array&lt;/a&gt;
which coordinates many Numpy-like arrays into a logical parallel array&lt;/p&gt;
&lt;p&gt;(dask array both &lt;em&gt;consumes&lt;/em&gt; and &lt;em&gt;implements&lt;/em&gt; the Numpy API)&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://optimized-einsum.readthedocs.io/en/latest/"&gt;Opt Einsum&lt;/a&gt;
for more efficient einstein summation operations&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;…&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;These projects and more enhance array computing in Python,
building on new features beyond what Numpy itself provides.&lt;/p&gt;
&lt;p&gt;There are also projects like Pandas, Scikit-Learn, and SciPy,
that use Numpy’s in-memory internal representation.
We’re going to ignore these libraries for this blogpost
and focus on those libraries that only use the high-level Numpy API
and not the low-level representation.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/05/27/beyond-numpy.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="opportunities-and-challenges"&gt;
&lt;h1&gt;Opportunities and Challenges&lt;/h1&gt;
&lt;p&gt;Given the two groups of projects:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;New libraries that &lt;em&gt;implement&lt;/em&gt; the Numpy API
(CuPy, Sparse, Dask array)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;New libraries that &lt;em&gt;consume&lt;/em&gt; and &lt;em&gt;extend&lt;/em&gt; the Numpy API
(XArray, Autograd/tangent, TensorLy, Einsum)&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We want to use them together, applying Autograd to CuPy, TensorLy to Sparse,
and so on, including all future implementations that might follow.
This is challenging.&lt;/p&gt;
&lt;p&gt;Unfortunately,
while all of the array implementations APIs are &lt;em&gt;very similar&lt;/em&gt; to Numpy’s API,
they use different 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="n"&gt;numpy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sin&lt;/span&gt; &lt;span class="ow"&gt;is&lt;/span&gt; &lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sin&lt;/span&gt;
&lt;span class="go"&gt;False&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This creates problems for the consumer libraries,
because now they need to switch out which functions they use
depending on which array-like objects they’ve been given.&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;f&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;if&lt;/span&gt; &lt;span class="nb"&gt;isinstance&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;numpy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&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;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sin&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;elif&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="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&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;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sin&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;elif&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;

&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Today each array project implements a custom plugin system
that they use to switch between some of the array options.
Links to these plugin mechanisms are below if you’re interested:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/pydata/xarray/blob/c346d3b7bcdbd6073cf96fdeb0710467a284a611/xarray/core/duck_array_ops.py"&gt;xarray/core/duck_array_ops.py&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/tensorly/tensorly/tree/af0700af61ca2cd104e90755d5e5033e23fd4ec4/tensorly/backend"&gt;tensorly/backend&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/HIPS/autograd/blob/bd3f92fcd4d66424be5fb6b6d3a7f9195c98eebf/autograd/numpy/numpy_vspaces.py"&gt;autograd/numpy/numpy_vspaces.py&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/google/tangent/blob/bc64848bba964c632a6da4965fb91f2f61a3cdd4/tangent/template.py"&gt;tangent/template.py&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/blob/8f164773cb3717b3c5ad856341205f605b8404cf/dask/array/core.py#L59-L62"&gt;dask/array/core.py#L59-L62&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dgasmith/opt_einsum/blob/32c1b0adb50511da1b86dc98bcf169d79b44efce/opt_einsum/backends.py"&gt;opt_einsum/backends.py&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;For example XArray can use either Numpy arrays or Dask arrays.
This has been hugely beneficial to users of that project,
which today seamlessly transition from small in-memory datasets on their laptops
to 100TB datasets on clusters,
all using the same programming model.
However when considering adding sparse or GPU arrays to XArray’s plugin system,
it quickly became clear that this would be expensive today.&lt;/p&gt;
&lt;p&gt;Building, maintaining, and extending these plugin mechanisms is &lt;em&gt;costly&lt;/em&gt;.
The plugin systems in each project are not alike,
so any new array implementation has to go to each library and build the same code several times.
Similarly, any new algorithmic library must build plugins to every ndarray implementation.
Each library has to explicitly import and understand each other library,
and has to adapt as those libraries change over time.
This coverage is not complete,
and so users lack confidence that their applications are portable between hardware.&lt;/p&gt;
&lt;p&gt;Pair-wise plugin mechanisms make sense for a single project,
but are not an efficient choice for the full ecosystem.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/05/27/beyond-numpy.md&lt;/span&gt;, line 181)&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="solutions"&gt;
&lt;h1&gt;Solutions&lt;/h1&gt;
&lt;p&gt;I see two solutions today:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Build a new library that holds dispatch-able versions of all of the relevant Numpy functions
and convince everyone to use it instead of Numpy internally&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Build this dispatch mechanism into Numpy itself&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Each has challenges.&lt;/p&gt;
&lt;section id="build-a-new-centralized-plugin-library"&gt;
&lt;h2&gt;Build a new centralized plugin library&lt;/h2&gt;
&lt;p&gt;We can build a new library,
here called &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;arrayish&lt;/span&gt;&lt;/code&gt;,
that holds dispatch-able versions of all of the relevant Numpy functions.
We then convince everyone to use it instead of Numpy internally.&lt;/p&gt;
&lt;p&gt;So in each array-like library’s codebase we write code 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="c1"&gt;# inside numpy&amp;#39;s codebase&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;arrayish&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="nd"&gt;@arrayish&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sin&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;numpy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;numpy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sin&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="nd"&gt;@arrayish&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cos&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;numpy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;numpy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cos&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="nd"&gt;@arrayish&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dot&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;numpy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;numpy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;numpy&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="o"&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="c1"&gt;# inside cupy&amp;#39;s codebase&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;arrayish&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cupy&lt;/span&gt;
&lt;span class="nd"&gt;@arrayish&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sin&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sin&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="nd"&gt;@arrayish&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cos&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cos&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="nd"&gt;@arrayish&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dot&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ndarray&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dot&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="o"&gt;...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;and so on for Dask, Sparse, and any other Numpy-like libraries.&lt;/p&gt;
&lt;p&gt;In all of the algorithm libraries (like XArray, autograd, TensorLy, …)
we use arrayish instead of 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="c1"&gt;# inside XArray&amp;#39;s codebase&lt;/span&gt;
&lt;span class="c1"&gt;# import numpy&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;arrayish&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;numpy&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is the same plugin solution as before,
but now we build a community standard plugin system
that hopefully all of the projects can agree to use.&lt;/p&gt;
&lt;p&gt;This reduces the big &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n&lt;/span&gt; &lt;span class="pre"&gt;by&lt;/span&gt; &lt;span class="pre"&gt;m&lt;/span&gt;&lt;/code&gt; cost of maintaining several plugin systems,
to a more manageable &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n&lt;/span&gt; &lt;span class="pre"&gt;plus&lt;/span&gt; &lt;span class="pre"&gt;m&lt;/span&gt;&lt;/code&gt; cost of using a single plugin system in each library.
This centralized project would also benefit, perhaps,
from being better maintained than any individual project is likely to do on its own.&lt;/p&gt;
&lt;p&gt;However this has costs:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;Getting many different projects to agree on a new standard is hard&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Algorithmic projects will need to start using arrayish internally,
adding new imports 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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;arrayish&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;numpy&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And this wll certainly cause some complications interally&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Someone needs to build an maintain the central infrastructure&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/hameerabbasi"&gt;Hameer Abbasi&lt;/a&gt;
put together a rudimentary prototype for arrayish here:
&lt;a class="reference external" href="https://github.com/hameerabbasi/arrayish"&gt;github.com/hameerabbasi/arrayish&lt;/a&gt;.
There has been some discussion about this topic, using XArray+Sparse as an example, in
&lt;a class="reference external" href="https://github.com/pydata/sparse/issues/1"&gt;pydata/sparse #1&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dispatch-from-within-numpy"&gt;
&lt;h2&gt;Dispatch from within Numpy&lt;/h2&gt;
&lt;p&gt;Alternatively, the central dispatching mechanism could live within Numpy itself.&lt;/p&gt;
&lt;p&gt;Numpy functions could learn to hand control over to their arguments,
allowing the array implementations to take over when possible.
This would allow existing Numpy code to work on externally developed array implementations.&lt;/p&gt;
&lt;p&gt;There is precedent for this.
The &lt;a class="reference external" href="https://docs.scipy.org/doc/numpy/reference/arrays.classes.html#numpy.class.__array_ufunc__"&gt;&lt;strong&gt;array_ufunc&lt;/strong&gt;&lt;/a&gt; protocol
allows any class that defines the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_ufunc__&lt;/span&gt;&lt;/code&gt; method
to take control of any Numpy ufunc like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.sin&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.exp&lt;/span&gt;&lt;/code&gt;.
Numpy reductions like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.sum&lt;/span&gt;&lt;/code&gt; already look for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.sum&lt;/span&gt;&lt;/code&gt; methods on their arguments and defer to them if possible.&lt;/p&gt;
&lt;p&gt;Some array projects, like Dask and Sparse, already implement the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_ufunc__&lt;/span&gt;&lt;/code&gt; protocol.
There is also &lt;a class="reference external" href="https://github.com/cupy/cupy/pull/1247"&gt;an open PR for CuPy&lt;/a&gt;.
Here is an example showing Numpy functions on Dask arrays cleanly.&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="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;10&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;# A Dask array&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;sum&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;exp&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;# Apply Numpy function to a Dask array&lt;/span&gt;
&lt;span class="go"&gt;dask.array&amp;lt;sum-aggregate, shape=(), dtype=float64, chunksize=()&amp;gt;  # get a Dask array&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;em&gt;I recommend that all Numpy-API compatible array projects implement the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_ufunc__&lt;/span&gt;&lt;/code&gt; protocol.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;This works for many functions, but not all.
Other operations like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;tensordot&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;concatenate&lt;/span&gt;&lt;/code&gt;, and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;stack&lt;/span&gt;&lt;/code&gt;
occur frequently in algorithmic code but are not covered here.&lt;/p&gt;
&lt;p&gt;This solution avoids the community challenges of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;arrayish&lt;/span&gt;&lt;/code&gt; solution above.
Everyone is accustomed to aligning themselves to Numpy’s decisions,
and relatively little code would need to be rewritten.&lt;/p&gt;
&lt;p&gt;The challenge with this approach is that historically
Numpy has moved more slowly than the rest of the ecosystem.
For example the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_ufunc__&lt;/span&gt;&lt;/code&gt; protocol mentioned above
was discussed for several years before it was merged.
Fortunately Numpy has recently
&lt;a class="reference external" href="https://www.numfocus.org/blog/numpy-receives-first-ever-funding-thanks-to-moore-foundation"&gt;received&lt;/a&gt;
&lt;a class="reference external" href="https://bids.berkeley.edu/news/bids-receives-sloan-foundation-grant-contribute-numpy-development"&gt;funding&lt;/a&gt;
to help it make changes like this more rapidly.
The full time developers hired under this funding have just started though,
and it’s not clear how much of a priority this work is for them at first.&lt;/p&gt;
&lt;p&gt;For what it’s worth I’d prefer to see this Numpy protocol solution take hold.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/05/27/beyond-numpy.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&gt;
&lt;section id="final-thoughts"&gt;
&lt;h1&gt;Final Thoughts&lt;/h1&gt;
&lt;p&gt;In recent years Python’s array computing ecosystem has grown organically to support
GPUs, sparse, and distributed arrays.
This is wonderful and a great example of the growth that can occur in decentralized open source development.&lt;/p&gt;
&lt;p&gt;However to solidify this growth and apply it across the ecosystem we now need to do some central planning
to move from a pair-wise model where packages need to know about each other
to an ecosystem model where packages can negotiate by developing and adhering to community-standard protocols.&lt;/p&gt;
&lt;p&gt;The community has done this transition before
(Numeric + Numarray -&amp;gt; Numpy, the Scikit-Learn fit/predict API, etc..)
usually with surprisingly positive results.&lt;/p&gt;
&lt;p&gt;The open questions I have today are the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;How quickly can Numpy adapt to this demand for protocols
while still remaining stable for its existing role as foundation of the ecosystem&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;What algorithmic domains can be written in a cross-hardware way
that depends only on the high-level Numpy API,
and doesn’t require specialization at the data structure level.
Clearly some domains exist (XArray, automatic differentiation),
but how common are these?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Once a standard protocol is in place,
what other array-like implementations might arise?
In-memory compression? Probabilistic? Symbolic?&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/05/27/beyond-numpy.md&lt;/span&gt;, line 339)&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="update"&gt;
&lt;h1&gt;Update&lt;/h1&gt;
&lt;p&gt;After discussing this topic at the
&lt;a class="reference external" href="https://scisprints.github.io/#may-numpy-developer-sprint"&gt;May NumPy Developer Sprint&lt;/a&gt;
at &lt;a class="reference external" href="https://bids.berkeley.edu/"&gt;BIDS&lt;/a&gt;
a few of us have drafted a Numpy Enhancement Proposal (NEP)
&lt;a class="reference external" href="https://github.com/numpy/numpy/pull/11189"&gt;available here&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/05/27/beyond-numpy/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-05-27T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/03/21/dask-0.17.2/</id>
    <title>Dask Release 0.17.2</title>
    <updated>2018-03-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://anaconda.com"&gt;Anaconda Inc.&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;I’m pleased to announce the release of Dask version 0.17.2. This is a minor
release with new features and stability improvements.
This blogpost outlines notable changes since the 0.17.0 release on February
12th.&lt;/p&gt;
&lt;p&gt;You can conda install Dask:&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 pip install from PyPI:&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[complete] --upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Full changelogs are available here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/blob/master/docs/source/changelog.rst"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/docs/source/changelog.rst"&gt;dask/distributed&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Some notable changes follow:&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/03/21/dask-0.17.2.md&lt;/span&gt;, line 32)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="tornado-5-0"&gt;

&lt;p&gt;Tornado is a popular framework for concurrent network programming that Dask
relies on heavily. Tornado recently released a major version update that
included both some major features for Dask as well as a couple of bugs.&lt;/p&gt;
&lt;p&gt;The new &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;IOStream.read_into&lt;/span&gt;&lt;/code&gt; method allows Dask communications (or anyone using
this API) to move large datasets more efficiently over the network with
fewer copies. This enables Dask to take advantage of high performance
networking available on modern super-computers. On the Cheyenne system, where
we tested this, we were able to get the full 3GB/s bandwidth available through
the Infiniband network with this change (when using a few worker processes).&lt;/p&gt;
&lt;p&gt;Many thanks to &lt;a class="reference external" href="https://github.com/pitrou"&gt;Antoine Pitrou&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/bdarnell"&gt;Ben
Darnell&lt;/a&gt; for their efforts on this.&lt;/p&gt;
&lt;p&gt;At the same time there were some unforeseen issues in the update to Tornado 5.0.
More pervasive use of bytearrays over bytes caused issues with compression
libraries like Snappy and Python 2 that were not expecting these types. There
is a brief window in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed.__version__&lt;/span&gt; &lt;span class="pre"&gt;==&lt;/span&gt; &lt;span class="pre"&gt;1.21.3&lt;/span&gt;&lt;/code&gt; that enables this
functionality if Tornado 5.0 is present but will misbehave if Snappy is also
present.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/03/21/dask-0.17.2.md&lt;/span&gt;, line 55)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="http-file-system"&gt;
&lt;h1&gt;HTTP File System&lt;/h1&gt;
&lt;p&gt;Dask leverages a &lt;a class="reference external" href="https://github.com/dask/dask/issues/2880"&gt;file-system-like protocol&lt;/a&gt;
for access to remote data.
This is what makes commands like the following work:&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;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://...&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;hdfs://...&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;gcs://...&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 have now added http and https file systems for reading data directly from
web servers. These also support random access if the web server supports range
queries.&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;https://...&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;As with S3, HDFS, GCS, … you can also use these tools outside of Dask
development. Here we read the first twenty bytes of the Pandas license:&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.bytes.http&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;HTTPFileSystem&lt;/span&gt;
&lt;span class="n"&gt;http&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;HTTPFileSystem&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;http&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;https://raw.githubusercontent.com/pandas-dev/pandas/master/LICENSE&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="nb"&gt;print&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;read&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="sa"&gt;b&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;BSD 3-Clause License&amp;#39;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="https://github.com/martindurant"&gt;Martin Durant&lt;/a&gt; who did this work
and manages Dask’s byte handling generally. See &lt;a class="reference external" href="http://dask.pydata.org/en/latest/remote-data-services.html"&gt;remote data documentation&lt;/a&gt; for more information.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/03/21/dask-0.17.2.md&lt;/span&gt;, line 94)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="fixed-a-correctness-bug-in-dask-dataframe-s-shuffle"&gt;
&lt;h1&gt;Fixed a correctness bug in Dask dataframe’s shuffle&lt;/h1&gt;
&lt;p&gt;We identified and resolved a correctness bug in dask.dataframe’s shuffle that
resulted in some rows being dropped during complex operations like joins and
groupby-applies with many partitions.&lt;/p&gt;
&lt;p&gt;See &lt;a class="reference external" href="https://github.com/dask/dask/pull/3201"&gt;dask/dask #3201&lt;/a&gt; for more information.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/03/21/dask-0.17.2.md&lt;/span&gt;, line 102)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="cluster-super-class-and-intelligent-adaptive-deployments"&gt;
&lt;h1&gt;Cluster super-class and intelligent adaptive deployments&lt;/h1&gt;
&lt;p&gt;There are many Python subprojects that help you deploy Dask on different
cluster resource managers like Yarn, SGE, Kubernetes, PBS, and more. These
have all converged to have more-or-less the same API that we have now combined
into a consistent interface that downstream projects can inherit from in
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed.deploy.Cluster&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Now that we have a consistent interface we have started to invest more in
improving the interface and intelligence of these systems as a group. This
includes both pleasant IPython widgets like the following:&lt;/p&gt;
&lt;img src="/images/dask-kubernetes-widget.png" width="70%"&gt;
&lt;p&gt;as well as improved logic around adaptive deployments. Adaptive deployments
allow clusters to scale themselves automatically based on current workload. If
you have recently submitted a lot of work the scheduler will estimate its
duration and ask for an appropriate number of workers to finish the computation
quickly. When the computation has finished the scheduler will release the
workers back to the system to free up resources.&lt;/p&gt;
&lt;p&gt;The logic here has improved substantially including the following:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;You can specify minimum and maximum limits on your adaptivity&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The scheduler estimates computation duration and asks for workers
appropriately&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;There is some additional delay in giving back workers to avoid hysteresis,
or cases where we repeatedly ask for and return workers&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/2018/03/21/dask-0.17.2.md&lt;/span&gt;, line 131)&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-projects"&gt;
&lt;h1&gt;Related projects&lt;/h1&gt;
&lt;p&gt;Some news from related projects:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The young daskernetes project was renamed to &lt;a class="reference external" href="http://dask-kubernetes.readthedocs.io/en/latest/"&gt;dask-kubernetes&lt;/a&gt;. This displaces a previous project (that had not been released) for launching Dask on Google Cloud Platform. That project has been renamed to &lt;a class="reference external" href="https://github.com/dask/dask-gke"&gt;dask-gke&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A new project, &lt;a class="reference external" href="https://github.com/dask/dask-jobqueue/"&gt;dask-jobqueue&lt;/a&gt; was
started to handle launching Dask clusters on traditional batch queuing
systems like PBS, SLURM, SGE, TORQUE, etc.. This projet grew out of the &lt;a class="reference external" href="https://pangeo-data.github.io/"&gt;Pangeo&lt;/a&gt; collaboration&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A Dask Helm chart has been added to Helm’s stable channel&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/2018/03/21/dask-0.17.2.md&lt;/span&gt;, line 141)&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;The following people contributed to the dask/dask repository since the 0.17.0
release on February 12h:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Anderson Banihirwe&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dan Collins&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dieter Weber&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Gabriele Lanaro&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Bourbeau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Julien Lhermitte&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Max Epstein&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;nkhadka&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;okkez&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Pangeran Bottor&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Rich Postelnik&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scott M. Edenbaum&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Simon Perkins&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Thrasibule&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tor E Hagemann&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Uwe L. Korn&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Wes Roach&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The following people contributed to the dask/distributed repository since the
1.21.0 release on February 12th:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Alexander Ford&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Andy Jones&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brett Naul&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joe Hamman&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Loïc Estève&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matti Lyra&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Sven Kreiss&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Thrasibule&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/03/21/dask-0.17.2/"/>
    <summary>This work is supported by Anaconda Inc.
and the Data Driven Discovery Initiative from the Moore
Foundation.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-03-21T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/02/28/minimal-bug-reports/</id>
    <title>Craft Minimal Bug Reports</title>
    <updated>2018-02-28T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;Following up on a post on &lt;a class="reference internal" href="#../../../2016/08/25/supporting-users"&gt;&lt;span class="xref myst"&gt;supporting users in open source&lt;/span&gt;&lt;/a&gt;
this post lists some suggestions on how to ask a maintainer to help you with a problem.&lt;/p&gt;
&lt;p&gt;You don’t have to follow these suggestions. They are optional.
They make it more likely that a project maintainer will spend time helping you.
It’s important to remember that their willingness to support you for free is optional too.&lt;/p&gt;
&lt;p&gt;Crafting minimal bug reports is essential for the life and maintenance of community-driven open source projects.
Doing this well is an incredible service to the community.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/02/28/minimal-bug-reports.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="minimal-complete-verifiable-examples"&gt;

&lt;p&gt;I strongly recommend following Stack Overflow’s guidelines on &lt;a class="reference external" href="https://stackoverflow.com/help/mcve"&gt;Minimal Complete Verifiable Exmamples&lt;/a&gt;. I’ll include brief highlights here:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;… code should be …&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Minimal – Use as little code as possible that still produces the same problem&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Complete – Provide all parts needed to reproduce the problem&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Verifiable – Test the code you’re about to provide to make sure it reproduces the problem&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;Lets be clear, this is &lt;em&gt;hard&lt;/em&gt; and takes time.&lt;/p&gt;
&lt;p&gt;As a question-asker I find that creating an MCVE often takes 10-30 minutes for a simple problem.
Fortunately this work is usually straightforward,
even if I don’t know very much about the package I’m having trouble with.
Most of the work to create a minimal example is about removing all of the code that was specific to my application,
and as the question-asker I am probably the most qualified person to do that.&lt;/p&gt;
&lt;p&gt;When answering questions I often point people to StackOverflow’s MCVE document.
They sometimes come back with a better-but-not-yet-minimal example.
This post clarifies a few common issues.&lt;/p&gt;
&lt;p&gt;As an running example I’m going to use Pandas dataframe 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/2018/02/28/minimal-bug-reports.md&lt;/span&gt;, line 44)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="don-t-post-data"&gt;
&lt;h1&gt;Don’t post data&lt;/h1&gt;
&lt;p&gt;You shouldn’t post the file that you’re working with.
Instead, try to see if you can reproduce the problem with just a few lines of data rather than the whole thing.&lt;/p&gt;
&lt;p&gt;Having to download a file, unzip it, etc. make it much less likely that someone will actually run your example in their free time.&lt;/p&gt;
&lt;section id="don-t"&gt;
&lt;h2&gt;Don’t&lt;/h2&gt;
&lt;p&gt;I’ve uploaded my data to Dropbox and you can get it here: &lt;a class="reference external" href="https://example.com"&gt;my-data.csv.gz&lt;/a&gt;&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;my-data.csv.gz&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="do"&gt;
&lt;h2&gt;Do&lt;/h2&gt;
&lt;p&gt;You should be able to copy-paste the following to get enough of my data to cause the problem:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pandas&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pd&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;account-start&amp;#39;&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;2017-02-03&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;2017-03-03&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;2017-01-01&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                   &lt;span class="s1"&gt;&amp;#39;client&amp;#39;&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;Alice Anders&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Bob Baker&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Charlie Chaplin&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                   &lt;span class="s1"&gt;&amp;#39;balance&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;1432.32&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;10.43&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;30000.00&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                   &lt;span class="s1"&gt;&amp;#39;db-id&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;1234&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2424&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;251&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                   &lt;span class="s1"&gt;&amp;#39;proxy-id&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;525&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1525&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2542&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                   &lt;span class="s1"&gt;&amp;#39;rank&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;52&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;525&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;32&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;/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/2018/02/28/minimal-bug-reports.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&gt;
&lt;section id="actually-don-t-include-your-data-at-all"&gt;
&lt;h1&gt;Actually don’t include your data at all&lt;/h1&gt;
&lt;p&gt;Actually, your data probably has lots of information that is very specific to
your application. Your eyes gloss over it but a maintainer doesn’t know what
is relevant and what isn’t, so it will take them time to digest it if you
include it. Instead see if you can reproduce your same failure with artificial
or random data.&lt;/p&gt;
&lt;section id="id1"&gt;
&lt;h2&gt;Don’t&lt;/h2&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/2018/02/28/minimal-bug-reports.md&lt;/span&gt;, line 84); &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: “don’t”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Here is enough of my data to reproduce the problem&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pandas&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pd&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;account-start&amp;#39;&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;2017-02-03&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;2017-03-03&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;2017-01-01&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                   &lt;span class="s1"&gt;&amp;#39;client&amp;#39;&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;Alice Anders&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Bob Baker&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Charlie Chaplin&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                   &lt;span class="s1"&gt;&amp;#39;balance&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;1432.32&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;10.43&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;30000.00&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                   &lt;span class="s1"&gt;&amp;#39;db-id&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;1234&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2424&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;251&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                   &lt;span class="s1"&gt;&amp;#39;proxy-id&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;525&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1525&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2542&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                   &lt;span class="s1"&gt;&amp;#39;rank&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;52&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;525&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;32&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="id2"&gt;
&lt;h2&gt;Do&lt;/h2&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/2018/02/28/minimal-bug-reports.md&lt;/span&gt;, line 100); &lt;em&gt;&lt;a href="#id2"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “do”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;My actual problem is about finding the best ranked employee over a certain time period,
but we can reproduce the problem with this simpler dataset.
Notice that the dates are &lt;em&gt;out of order&lt;/em&gt; in this data (2000-01-02 comes after 2000-01-03).
I found that this was critical to reproducing the error.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pandas&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pd&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;account-start&amp;#39;&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;2000-01-01&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;2000-01-03&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;2000-01-02&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                   &lt;span class="s1"&gt;&amp;#39;db-id&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;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;name&amp;#39;&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;Alice&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Bob&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Charlie&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;As we shrink down our example problem we often discover a lot about what causes the problem.
This discovery is valuable
and something that only the question-asker is capable of doing efficiently.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/02/28/minimal-bug-reports.md&lt;/span&gt;, line 118)&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="see-how-small-you-can-make-things"&gt;
&lt;h1&gt;See how small you can make things&lt;/h1&gt;
&lt;p&gt;To make it even easier, see how small you can make your data.
For example if working with tabular data (like Pandas),
then how many columns do you actually need to reproduce the failure?
How many rows do you actually need to reproduce the failure?
Do the columns need to be named as you have them now or could they be just “A” and “B”
or descriptive of the types within?&lt;/p&gt;
&lt;section id="id3"&gt;
&lt;h2&gt;Do&lt;/h2&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/2018/02/28/minimal-bug-reports.md&lt;/span&gt;, line 127); &lt;em&gt;&lt;a href="#id3"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “do”.&lt;/p&gt;
&lt;/aside&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pandas&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pd&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;datetime&amp;#39;&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;2000-01-03&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;2000-01-02&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                   &lt;span class="s1"&gt;&amp;#39;id&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;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;/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/2018/02/28/minimal-bug-reports.md&lt;/span&gt;, line 135)&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="remove-unnecessary-steps"&gt;
&lt;h1&gt;Remove unnecessary steps&lt;/h1&gt;
&lt;p&gt;Is every line in your example absolutely necessary to reproduce the error?
If you’re able to delete a line of code then please do.
Because you already understand your problem you are &lt;em&gt;much more efficient&lt;/em&gt; at doing this than the maintainer is.
They probably know more about the tool, but you know more about your code.&lt;/p&gt;
&lt;section id="id4"&gt;
&lt;h2&gt;Don’t&lt;/h2&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/2018/02/28/minimal-bug-reports.md&lt;/span&gt;, line 142); &lt;em&gt;&lt;a href="#id4"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “don’t”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;The groupby step below is raising a warning that I don’t understand&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value&lt;/span&gt; &lt;span class="o"&gt;&amp;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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fillna&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;y&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="c1"&gt;# &amp;lt;-- this produces the error&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="id5"&gt;
&lt;h2&gt;Do&lt;/h2&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/2018/02/28/minimal-bug-reports.md&lt;/span&gt;, line 155); &lt;em&gt;&lt;a href="#id5"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “do”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;The groupby step below is raising a warning that I don’t understand&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;y&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="c1"&gt;# &amp;lt;-- this produces the error&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/2018/02/28/minimal-bug-reports.md&lt;/span&gt;, line 165)&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="use-syntax-highlighting"&gt;
&lt;h1&gt;Use Syntax Highlighting&lt;/h1&gt;
&lt;p&gt;When using Github you can enclose code blocks in triple-backticks (the
character on the top-left of your keyboard on US-standard QWERTY keyboards).
It looks like this:&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;```python
x = 1
```
&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/2018/02/28/minimal-bug-reports.md&lt;/span&gt;, line 175)&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="provide-complete-tracebacks"&gt;
&lt;h1&gt;Provide complete tracebacks&lt;/h1&gt;
&lt;p&gt;You know all of that stuff between your code and the exception that is hard to
make sense of? You should include it.&lt;/p&gt;
&lt;section id="id6"&gt;
&lt;h2&gt;Don’t&lt;/h2&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/2018/02/28/minimal-bug-reports.md&lt;/span&gt;, line 180); &lt;em&gt;&lt;a href="#id6"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “don’t”.&lt;/p&gt;
&lt;/aside&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;I get a ZeroDivisionError from the following code:

```python
def div(x, y):
    return x / y

div(1, 0)
```
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="id7"&gt;
&lt;h2&gt;Do&lt;/h2&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/2018/02/28/minimal-bug-reports.md&lt;/span&gt;, line 191); &lt;em&gt;&lt;a href="#id7"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “do”.&lt;/p&gt;
&lt;/aside&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;I get a ZeroDivisionError from the following code:

```python
def div(x, y):
    return x / y

div(1, 0)
```

```python-traceback
ZeroDivisionError                         Traceback (most recent call last)
&amp;lt;ipython-input-4-7b96263abbfa&amp;gt; in &amp;lt;module&amp;gt;()
----&amp;gt; 1 div(1, 0)

&amp;lt;ipython-input-3-7685f97b4ce5&amp;gt; in div(x, y)
      1 def div(x, y):
----&amp;gt; 2     return x / y
      3

ZeroDivisionError: division by zero
```
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If the traceback is long that’s ok. If you really want to be clean you can put
it in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;&amp;lt;details&amp;gt;&lt;/span&gt;&lt;/code&gt; brackets.&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;I get a ZeroDivisionError from the following code:

```python
def div(x, y):
    return x / y

div(1, 0)
```

### Traceback

&amp;lt;details&amp;gt;

```python
ZeroDivisionError                         Traceback (most recent call last)
&amp;lt;ipython-input-4-7b96263abbfa&amp;gt; in &amp;lt;module&amp;gt;()
----&amp;gt; 1 div(1, 0)

&amp;lt;ipython-input-3-7685f97b4ce5&amp;gt; in div(x, y)
      1 def div(x, y):
----&amp;gt; 2     return x / y
      3

ZeroDivisionError: division by zero
```

&amp;lt;/details&amp;gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="ask-questions-in-public-places"&gt;
&lt;h2&gt;Ask Questions in Public Places&lt;/h2&gt;
&lt;p&gt;When raising issues you often have a few possible locations:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;GitHub issue tracker&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stack Overflow&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Project mailing list&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Project Chat room&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;E-mail maintainers directly (never do this)&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Different projects handle this differently, but they usually have a page on
their documentation about where to go for help. This is often labeled
“Community”, “Support” or “Where to ask for help”. Here are the
recommendations from the
&lt;a class="reference external" href="https://pandas.pydata.org/community.html"&gt;Pandas community&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Generally it’s good to ask questions where many maintainers can see your
question and help, and where other users can find your question and answer if
they encounter a similar bug in the future.&lt;/p&gt;
&lt;p&gt;While your goal may be to solve your problem, the maintainer’s goal is likely
to create a record of how to solve problems like yours. This helps many more
users who will have a similar problem in the future, see your well-crafted bug
report, and learn from the resulting conversation.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="my-personal-preferences"&gt;
&lt;h2&gt;My personal preferences&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;For user questions like “What is the right way to do X?” I prefer Stack Overflow.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For bug reports like “I did X, I’m pretty confident that it should work, but I
get this error” I prefer Github issues&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For general chit-chat I prefer Gitter, though actually, I personally spend
almost no time in gitter because it isn’t easily searchable by future
users. If you’ve asked me a question in Gitter I will almost certainly
not respond to it, except to direct you to github, stack overflow, or this
blogpost.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;I only like personal e-mail if someone is proposing to fund or seriously
support the project in some way&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;But again, different projects do this differently and have different policies.
You should check the documentation of the project you’re dealing with to learn
how they like to support users.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/02/28/minimal-bug-reports/"/>
    <summary>Following up on a post on supporting users in open source
this post lists some suggestions on how to ask a maintainer to help you with a problem.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2018-02-28T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/02/12/dask-0.17.0/</id>
    <title>Dask Release 0.17.0</title>
    <updated>2018-02-12T00: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://anaconda.com"&gt;Anaconda Inc.&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;I’m pleased to announce the release of Dask version 0.17.0. This a significant
major release with new features, breaking changes, and stability improvements.
This blogpost outlines notable changes since the 0.16.0 release on November
21st.&lt;/p&gt;
&lt;p&gt;You can conda install Dask:&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 -c conda-forge
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or pip install from PyPI:&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[complete] --upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Full changelogs are available here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/blob/master/docs/source/changelog.rst"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/docs/source/changelog.rst"&gt;dask/distributed&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Some notable changes follow.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/02/12/dask-0.17.0.md&lt;/span&gt;, line 32)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="deprecations"&gt;

&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Removed &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe.rolling_*&lt;/span&gt;&lt;/code&gt; methods, which were previously deprecated both in dask.dataframe and in pandas. These are replaced with the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;rolling.*&lt;/span&gt;&lt;/code&gt; namespace&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We’ve generally stopped maintenance of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-ec2&lt;/span&gt;&lt;/code&gt; project to launch dask clusters on Amazon’s EC2 using Salt. We generally recommend kubernetes instead both for Amazon’s EC2, and for Google and Azure as well&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="http://dask.pydata.org/en/latest/setup/kubernetes.html"&gt;dask.pydata.org/en/latest/setup/kubernetes.html&lt;/a&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Internal state of the distributed scheduler has changed significantly. This may affect advanced users who were inspecting this state for debugging or diagnostics.&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/2018/02/12/dask-0.17.0.md&lt;/span&gt;, line 41)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="task-ordering"&gt;
&lt;h1&gt;Task Ordering&lt;/h1&gt;
&lt;p&gt;As Dask encounters more complex problems from more domains
we continually run into problems where its current heuristics do not perform optimally.
This release includes a rewrite of our static task prioritization heuristics.
This will improve Dask’s ability to traverse complex computations
in a way that keeps memory use low.&lt;/p&gt;
&lt;p&gt;To aid debugging we also integrated these heuristics into the GraphViz-style plots
that come from the visualize method.&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;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="o"&gt;...&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;visualize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;color&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;order&amp;#39;&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&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="https://user-images.githubusercontent.com/306380/35012109-86df75fa-fad6-11e7-9fa8-a43a697a4a17.png"&gt;
  &lt;img src="https://user-images.githubusercontent.com/306380/35012109-86df75fa-fad6-11e7-9fa8-a43a697a4a17.png"
     width="80%"
     align="center"&gt;&lt;/a&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/3066"&gt;dask/dask #3066&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/3057"&gt;dask/dask #3057&lt;/a&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/2018/02/12/dask-0.17.0.md&lt;/span&gt;, line 66)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="nested-joblib"&gt;
&lt;h1&gt;Nested Joblib&lt;/h1&gt;
&lt;p&gt;Dask supports parallelizing Scikit-Learn
by extending Scikit-Learn’s underlying library for parallelism,
&lt;a class="reference external" href="http://tomaugspurger.github.io/distributed-joblib.html"&gt;Joblib&lt;/a&gt;.
This allows Dask to distribute &lt;em&gt;some&lt;/em&gt; SKLearn algorithms across a cluster
just by wrapping them with a context manager.&lt;/p&gt;
&lt;p&gt;This relationship has been strengthened,
and particular attention has been focused
when nesting one parallel computation within another,
such as occurs when you train a parallel estimator, like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomForest&lt;/span&gt;&lt;/code&gt;,
within another parallel computation, like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GridSearchCV&lt;/span&gt;&lt;/code&gt;.
Previously this would result in spawning too many threads/processes
and generally oversubscribing hardware.&lt;/p&gt;
&lt;p&gt;Due to recent combined development within both Joblib and Dask,
these sorts of situations can now be resolved efficiently by handing them off to Dask,
providing speedups even in single-machine cases:&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;sklearn.externals&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;joblib&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;distributed.joblib&lt;/span&gt;  &lt;span class="c1"&gt;# register the dask joblib backend&lt;/span&gt;

&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;est&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ParallelEstimator&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;gs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;GridSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;est&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;joblib&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;parallel_backend&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;dask&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;gs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;See Tom Augspurger’s recent post with more details about this work:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://tomaugspurger.github.io/distributed-joblib.html"&gt;http://tomaugspurger.github.io/distributed-joblib.html&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/joblib/joblib/pull/595"&gt;joblib/joblib #595&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/1705"&gt;dask/distributed #1705&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/joblib/joblib/pull/613"&gt;joblib/joblib #613&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="https://github.com/TomAugspurger"&gt;Tom Augspurger&lt;/a&gt;,
&lt;a class="reference external" href="https://github.com/jcrist"&gt;Jim Crist&lt;/a&gt;, and
&lt;a class="reference external" href="https://github.com/ogrisel"&gt;Olivier Grisel&lt;/a&gt; who did most of this 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/2018/02/12/dask-0.17.0.md&lt;/span&gt;, line 111)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="scheduler-internal-refactor"&gt;
&lt;h1&gt;Scheduler Internal Refactor&lt;/h1&gt;
&lt;p&gt;The distributed scheduler has been significantly refactored to change it from a forest of dictionaries:&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;priority&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;a&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="s1"&gt;&amp;#39;b&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="s1"&gt;&amp;#39;c&amp;#39;&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;dependencies&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;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="s1"&gt;&amp;#39;b&amp;#39;&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="s1"&gt;&amp;#39;c&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;c&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;nbytes&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;a&amp;#39;&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="s1"&gt;&amp;#39;b&amp;#39;&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="s1"&gt;&amp;#39;c&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;28&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;To a bunch of objects:&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;tasks&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;a&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;Task&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;priority&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;nbytes&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;dependencies&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;b&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;Task&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;priority&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="n"&gt;nbytes&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;dependencies&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;c&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;Task&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;c&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;priority&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="n"&gt;nbytes&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;28&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;dependencies&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[])}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;(there is &lt;em&gt;much&lt;/em&gt; more state than what is listed above,
but hopefully the examples above are clear.)&lt;/p&gt;
&lt;p&gt;There were a few motivations for this:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;We wanted to try out Cython and PyPy, for which objects like this might be more effective than dictionaries.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We believe that this is probably a bit easier for developers new to the schedulers to understand. The proliferation of state dictionaries was not highly discoverable.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Goal one ended up not working out.
We have not yet been able to make the scheduler significantly faster under Cython or PyPy with this new layout. There is even a slight memory increase with these changes.
However we have been happy with the results in code readability, and we hope that others find this useful as well.&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="https://github.com/pitrou"&gt;Antoine Pitrou&lt;/a&gt;,
who did most of the work here.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/02/12/dask-0.17.0.md&lt;/span&gt;, line 144)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="user-priorities"&gt;
&lt;h1&gt;User Priorities&lt;/h1&gt;
&lt;p&gt;You can now submit tasks with different priorities.&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;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&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="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;priority&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;# Higher priority preferred&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&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="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;priority&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;# Lower priority happens later&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;To be clear, Dask has always had priorities, they just weren’t easily user-settable.
Higher priorities are given precedence. The default priority for all tasks is zero.
You can also submit priorities for collections (like arrays and dataframes)&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;priority&lt;/span&gt;&lt;span class="o"&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;# give this computation higher priority.&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/1651"&gt;dask/distributed #1651&lt;/a&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/2018/02/12/dask-0.17.0.md&lt;/span&gt;, line 163)&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-projects"&gt;
&lt;h1&gt;Related projects&lt;/h1&gt;
&lt;p&gt;Several related projects are also undergoing releases:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;Tornado is updating to version 5.0 (there is a beta out now).
This is a major change that will put Tornado on the Asyncio event loop in Python 3.
It also includes many performance enhancements for high-bandwidth networks.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bokeh 0.12.14 was just released.&lt;/p&gt;
&lt;p&gt;Note that you will need to update Dask to work with this version of Bokeh&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://daskernetes.readthedocs.io/en/latest/"&gt;Daskernetes&lt;/a&gt;, a new project for launching Dask on Kubernetes clusters&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/2018/02/12/dask-0.17.0.md&lt;/span&gt;, line 176)&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;The following people contributed to the dask/dask repository since the 0.16.0
release on November 14th:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Albert DeFusco&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Apostolos Vlachopoulos&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;castalheiro&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Bourbeau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jon Mease&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ian Hopkinson&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jakub Nowacki&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John A Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joseph Lin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Keisuke Fujii&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martijn Arts&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Markus Gonser&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Nir&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Rich Signell&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Roman Yurchak&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;S. Andrew Sheppard&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;sephib&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stephan Hoyer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Uwe L. Korn&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Wei Ji&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Xander Johnson&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The following people contributed to the dask/distributed repository since the
1.20.0 release on November 14th:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Alexander Ford&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brett Naul&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brian Broll&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bruce Merry&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Cornelius Riemenschneider&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Daniel Li&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kelvin Yang&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Min RK&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;rqx&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Russ Bubley&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scott Sievert&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Xander Johnson&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/02/12/dask-0.17.0/"/>
    <summary>This work is supported by Anaconda Inc.
and the Data Driven Discovery Initiative from the Moore
Foundation.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-02-12T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/02/09/credit-models-with-dask/</id>
    <title>Credit Modeling with Dask</title>
    <updated>2018-02-09T00:00:00+00:00</updated>
    <author>
      <name>Richard Postelnik</name>
    </author>
    <content type="html">&lt;p&gt;This post explores a real-world use case calculating complex credit models in Python using Dask.
It is an example of a complex parallel system that is well outside of the traditional “big data” workloads.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/02/09/credit-models-with-dask.md&lt;/span&gt;, line 13)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="this-is-a-guest-post"&gt;

&lt;p&gt;Hi All,&lt;/p&gt;
&lt;p&gt;This is a guest post from &lt;a class="reference external" href="https://github.com/postelrich"&gt;Rich Postelnik&lt;/a&gt;,
an Anaconda employee who works with a large retail bank on their credit modeling system.
They’re doing interesting work with Dask to manage complex computations
(see task graph below).
This is a nice example of using Dask for complex problems that are neither a big dataframe nor a big array, but are still highly parallel.
Rich was kind enough to write up this description of their problem and share it here.&lt;/p&gt;
&lt;p&gt;Thanks Rich!&lt;/p&gt;
&lt;a href="/images/credit_models/simple-model.svg"&gt;
  &lt;img src="/images/credit_models/simple-model.svg"
       alt="zoomed model section"
       width="100%"&gt;&lt;/a&gt;
&lt;p&gt;&lt;em&gt;This is cross-posted at &lt;a class="reference external" href="https://www.anaconda.com/blog/developer-blog/credit-modeling-with-dask/"&gt;Anaconda’s Developer Blog&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;P.S. If others have similar solutions and would like to share them I’d love to host those on this blog as well.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/02/09/credit-models-with-dask.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="the-problem"&gt;
&lt;h1&gt;The Problem&lt;/h1&gt;
&lt;p&gt;When applying for a loan, like a credit card, mortgage, auto loan, etc., we want to estimate the likelihood of default and the profit (or loss) to be gained. Those models are composed of a complex set of equations that depend on each other. There can be hundreds of equations each of which could have up to 20 inputs and yield 20 outputs. That is a lot of information to keep track of! We want to avoid manually keeping track of the dependencies, as well as messy code like the following Python function:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;final_equation&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inputs&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;out1&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;equation1&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inputs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;out2_1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;out2_2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;out2_3&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;equation2&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inputs&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;out1&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;out3_1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;out3_2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;equation3&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;out2_3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;out1&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="o"&gt;...&lt;/span&gt;
    &lt;span class="n"&gt;out_final&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;equation_n&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inputs&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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;out_final&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This boils down to a dependency and ordering problem known as task scheduling.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/02/09/credit-models-with-dask.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="dags-to-the-rescue"&gt;
&lt;h1&gt;DAGs to the rescue&lt;/h1&gt;
&lt;img style="margin: 0 auto; display: block;" src="/images/credit_models/snatch.jpg" alt="snatch joke"&gt;
&lt;p&gt;A &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Directed_acyclic_graph"&gt;directed acyclic graph&lt;/a&gt; (DAG) is commonly used to solve task scheduling problems. Dask is a library for delayed task computation that makes use of directed graphs at its core. &lt;a class="reference external" href="http://dask.pydata.org/en/latest/delayed.html"&gt;dask.delayed&lt;/a&gt; is a simple decorator that turns a Python function into a graph vertex. If I pass the output from one delayed function as a parameter to another delayed function, Dask creates a directed edge between them. Let’s look at an 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="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;x&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&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;y&lt;/span&gt;

&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;add&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="mi"&gt;4&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So here we have a function to add two numbers together. Let’s see what happens when we wrap it with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.delayed&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="n"&gt;add&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;left&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;add&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;1&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;left&lt;/span&gt;
&lt;span class="go"&gt;Delayed(&amp;#39;add-f6204fac-b067-40aa-9d6a-639fc719c3ce&amp;#39;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;add&lt;/span&gt;&lt;/code&gt; now returns a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Delayed&lt;/span&gt;&lt;/code&gt; object. We can pass this as an argument back into our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.delayed&lt;/span&gt;&lt;/code&gt; function to start building out a chain of computation.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;right&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;add&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;1&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;four&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;left&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;right&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;four&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;4&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;four&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;visualize&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Below we can see how the DAG starts to come together.&lt;/p&gt;
&lt;img style="margin: 0 auto; display: block;" src="/images/credit_models/four.png" alt="four graph"&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/02/09/credit-models-with-dask.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 id="mock-credit-example"&gt;
&lt;h1&gt;Mock credit example&lt;/h1&gt;
&lt;p&gt;Let’s assume I’m a mortgage bank and have 10 people applying for a mortgage. I want to estimate the group’s average likelihood to default based on years of credit history and income.&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;hist_yrs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;incomes&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="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;Let’s also assume that default is a function of the incremented years history and half the years experience. While this could be written like:&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;default&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;hist&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;income&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;hist&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="mi"&gt;2&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;income&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 know in the future that I will need the incremented history for another calculation and want to be able to reuse the code as well as avoid doing the computation twice. Instead, I can break those functions out:&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;delayed&lt;/span&gt;

&lt;span class="nd"&gt;@delayed&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;increment&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="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;

&lt;span class="nd"&gt;@delayed&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;halve&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;

&lt;span class="nd"&gt;@delayed&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;default&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;hist&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;income&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;hist&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="n"&gt;income&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Note how I wrapped the functions with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;delayed&lt;/span&gt;&lt;/code&gt;. Now instead of returning a number these functions will return a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Delayed&lt;/span&gt;&lt;/code&gt; object. Even better is that these functions can also take &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Delayed&lt;/span&gt;&lt;/code&gt; objects as inputs. It is this passing of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Delayed&lt;/span&gt;&lt;/code&gt; objects as inputs to other &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;delayed&lt;/span&gt;&lt;/code&gt; functions that allows Dask to construct the task graph. I can now call these functions on my data in the style of normal Python code:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;inc_hist&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;increment&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n&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;n&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;hist_yrs&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;halved_income&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;halve&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n&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;n&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;income&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;estimated_default&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;default&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;hist&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;income&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;hist&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;income&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="nb"&gt;zip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inc_hist&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;halved_income&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If you look at these variables, you will see that nothing has actually been calculated yet. They are all lists of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Delayed&lt;/span&gt;&lt;/code&gt; objects.&lt;/p&gt;
&lt;p&gt;Now, to get the average, I could just take the sum of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;estimated_default&lt;/span&gt;&lt;/code&gt; but I want this to scale (and make a more interesting graph) so let’s do a merge-style reduction.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@delayed&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;agg&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;y&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;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;y&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;merge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;seq&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;seq&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;seq&lt;/span&gt;
    &lt;span class="n"&gt;middle&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;seq&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="n"&gt;left&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;merge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;seq&lt;/span&gt;&lt;span class="p"&gt;[:&lt;/span&gt;&lt;span class="n"&gt;middle&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
    &lt;span class="n"&gt;right&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;merge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;seq&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;middle&lt;/span&gt;&lt;span class="p"&gt;:])&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="ow"&gt;not&lt;/span&gt; &lt;span class="n"&gt;right&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;left&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;agg&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;left&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;right&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;default_sum&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;merge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;estimated_defaults&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;At this point &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;default_sum&lt;/span&gt;&lt;/code&gt; is a list of length 1 and that first element is the sum of estimated default for all applicants. To get the average, we divide by the number of applicants and call compute:&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_default&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;default_sum&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mi"&gt;10&lt;/span&gt;
&lt;span class="n"&gt;avg_default&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;# 40.75&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;To see the computation graph that Dask will use, we call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;visualize&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;avg_default&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;visualize&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img src="/images/credit_models/dummy_graph.png"
     style="margin: 0 auto; display: block;"
     alt="default graph"
     width="100%"&gt;&lt;/p&gt;
&lt;p&gt;And that is how Dask can be used to construct a complex system of equations with reusable intermediary calculations.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/02/09/credit-models-with-dask.md&lt;/span&gt;, line 173)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="how-we-used-dask-in-practice"&gt;
&lt;h1&gt;How we used Dask in practice&lt;/h1&gt;
&lt;p&gt;For our credit modeling problem, we used Dask to make a custom data structure to represent the individual equations. Using the default example above, this looked something 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="k"&gt;class&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nc"&gt;Default&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Equation&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;inputs&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;inc_hist&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;halved_income&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
    &lt;span class="n"&gt;outputs&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;defaults&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

    &lt;span class="nd"&gt;@delayed&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;equation&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;inc_hist&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;halved_income&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;inc_hist&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="n"&gt;halved_income&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This allows us to write each equation as its own isolated function and mark its inputs and outputs. With this set of equation objects, we can determine the order of computation (with a &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Topological_sorting"&gt;topological sort&lt;/a&gt;) and let Dask handle the graph generation and computation. This eliminates the onerous task of manually passing around the arguments in the code base. Below is an example task graph for one particular model that the bank actually does.&lt;/p&gt;
&lt;a href="/images/credit_models/simple.svg"&gt;
  &lt;img src="/images/credit_models/simple.svg"
       alt="calc task graph"
       width="100%"&gt;
  &lt;/a&gt;
&lt;p&gt;This graph was a bit too large to render with the normal &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;my_task.visualize()&lt;/span&gt;&lt;/code&gt; method, so instead we rendered it with &lt;a class="reference external" href="https://gephi.org"&gt;Gephi&lt;/a&gt; to make the pretty colored graph above. The chaotic upper region of this graph is the individual equation calculations. Zooming in we can see the entry point, our input pandas DataFrame, as the large orange circle at the top and how it gets fed into many of the equations.&lt;/p&gt;
&lt;a href="/images/credit_models/simple-model.svg"&gt;
  &lt;img src="/images/credit_models/simple-model.svg"
       alt="zoomed model section"
       width="100%"&gt;&lt;/a&gt;
&lt;p&gt;The output of the model is about 100 times the size of the input so we do some aggregation at the end via tree reduction. This accounts for the more structured bottom half of the graph. The large green node at the bottom is our final output.&lt;/p&gt;
&lt;a href="/images/credit_models/simple-agg.svg"&gt;
  &lt;img src="/images/credit_models/simple-agg.svg"
       alt="zoomed agg section"
       width="100%"&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/2018/02/09/credit-models-with-dask.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="final-thoughts"&gt;
&lt;h1&gt;Final Thoughts&lt;/h1&gt;
&lt;p&gt;With our Dask-based data structure, we spend more of our time writing model code rather than maintenance of the engine itself. This allows a clean separation between our analysts that design and write our models, and our computational system that runs them. Dask also offers a number of advantages not covered above. For example, with Dask you also get access to &lt;a class="reference external" href="https://distributed.readthedocs.io/en/latest/web.html"&gt;diagnostics&lt;/a&gt; such as time spent running each task and resources used. Also, you can easily distribute your computation with &lt;a class="reference external" href="https://distributed.readthedocs.io/en/latest/"&gt;dask distributed&lt;/a&gt; with relative ease. Now if I want to run our model across larger-than-memory data or on a distributed cluster, we don’t have to worry about rewriting our code to incorporate something like Spark. Finally, Dask allows you to give pandas-capable business analysts or less technical folks access to large datasets with the &lt;a class="reference external" href="http://dask.pydata.org/en/latest/dataframe.html"&gt;dask dataframe&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/02/09/credit-models-with-dask.md&lt;/span&gt;, line 213)&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="full-example"&gt;
&lt;h1&gt;Full Example&lt;/h1&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;delayed&lt;/span&gt;


&lt;span class="nd"&gt;@delayed&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;increment&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="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;


&lt;span class="nd"&gt;@delayed&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;halve&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;


&lt;span class="nd"&gt;@delayed&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;default&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;hist&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;income&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;hist&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="n"&gt;income&lt;/span&gt;


&lt;span class="nd"&gt;@delayed&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;agg&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;y&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;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;y&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;merge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;seq&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;seq&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;seq&lt;/span&gt;
    &lt;span class="n"&gt;middle&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;seq&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="n"&gt;left&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;merge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;seq&lt;/span&gt;&lt;span class="p"&gt;[:&lt;/span&gt;&lt;span class="n"&gt;middle&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
    &lt;span class="n"&gt;right&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;merge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;seq&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;middle&lt;/span&gt;&lt;span class="p"&gt;:])&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="ow"&gt;not&lt;/span&gt; &lt;span class="n"&gt;right&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;left&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;agg&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;left&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;right&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;hist_yrs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;incomes&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;inc_hist&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;increment&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n&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;n&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;hist_yrs&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;halved_income&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;halve&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n&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;n&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;incomes&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;estimated_defaults&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;default&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;hist&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;income&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;hist&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;income&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="nb"&gt;zip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inc_hist&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;halved_income&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;
&lt;span class="n"&gt;default_sum&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;merge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;estimated_defaults&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;avg_default&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;default_sum&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mi"&gt;10&lt;/span&gt;
&lt;span class="n"&gt;avg_default&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;avg_default&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;visualize&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# requires graphviz and python-graphviz to be installed&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/2018/02/09/credit-models-with-dask.md&lt;/span&gt;, line 261)&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;Special thanks to Matt Rocklin, Michael Grant, Gus Cavanagh, and Rory Merritt for their feedback when writing this article.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/02/09/credit-models-with-dask/"/>
    <summary>This post explores a real-world use case calculating complex credit models in Python using Dask.
It is an example of a complex parallel system that is well outside of the traditional “big data” workloads.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <published>2018-02-09T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/01/22/pangeo-2/</id>
    <title>Pangeo: JupyterHub, Dask, and XArray on the Cloud</title>
    <updated>2018-01-22T00: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://anaconda.com"&gt;Anaconda Inc&lt;/a&gt;, the NSF
EarthCube program, and UC Berkeley BIDS&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;A few weeks ago a few of us stood up &lt;a class="reference external" href="http://pangeo.pydata.org"&gt;pangeo.pydata.org&lt;/a&gt;,
an experimental deployment of JupyterHub, Dask, and XArray on Google Container Engine (GKE)
to support atmospheric and oceanographic data analysis on large datasets.
This follows on &lt;a class="reference internal" href="../2017/09/18/pangeo-1/"&gt;&lt;span class="doc std std-doc"&gt;recent work&lt;/span&gt;&lt;/a&gt; to deploy Dask and XArray for the same workloads on super computers.
This system is a proof of concept that has taught us a great deal about how to move forward.
This blogpost briefly describes the problem,
the system,
then describes the collaboration,
and finally discusses a number of challenges that we’ll be working on in coming months.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/01/22/pangeo-2.md&lt;/span&gt;, line 21)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="the-problem"&gt;

&lt;p&gt;Atmospheric and oceanographic sciences collect (with satellites) and generate (with simulations) large datasets
that they would like to analyze with distributed systems.
Libraries like Dask and XArray already solve this problem computationally if scientists have their own clusters,
but we seek to expand access by deploying on cloud-based systems.
We build a system to which people can log in, get Jupyter Notebooks, and launch Dask clusters without much hassle.
We hope that this increases access, and connects more scientists with more cloud-based datasets.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/01/22/pangeo-2.md&lt;/span&gt;, line 30)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="the-system"&gt;
&lt;h1&gt;The System&lt;/h1&gt;
&lt;p&gt;We integrate several pre-existing technologies to build a system where people can log in,
get access to a Jupyter notebook,
launch distributed compute clusters using Dask,
and analyze large datasets stored in the cloud.
They have a full user environment available to them through a website,
can leverage thousands of cores for computation,
and use existing APIs and workflows that look familiar to how they work on their laptop.&lt;/p&gt;
&lt;p&gt;A video walk-through follows below:&lt;/p&gt;
&lt;iframe width="560"
        height="315"
        src="https://www.youtube.com/embed/rSOJKbfNBNk"
        frameborder="0"
        allow="autoplay; encrypted-media"
        allowfullscreen&gt;&lt;/iframe&gt;
&lt;p&gt;We assembled this system from a number of pieces and technologies:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://jupyterhub.readthedocs.io/en/latest/"&gt;JupyterHub&lt;/a&gt;: Provides both the ability to launch single-user notebook servers
and handles user management for us.
In particular we use the KubeSpawner and the excellent documentation at &lt;a class="reference external" href="https://zero-to-jupyterhub.readthedocs.io/en/latest"&gt;Zero to JupyterHub&lt;/a&gt;,
which we recommend to anyone interested in this area.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/jupyterhub/kubespawner"&gt;KubeSpawner&lt;/a&gt;: A JupyterHub spawner that makes it easy to launch single-user notebook servers on Kubernetes systems&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://jupyterlab-tutorial.readthedocs.io/en/latest/"&gt;JupyterLab&lt;/a&gt;: The newer version of the classic notebook,
which we use to provide a richer remote user interface,
complete with terminals, file management, and more.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://xarray.pydata.org"&gt;XArray&lt;/a&gt;: Provides computation on NetCDF-style data.
XArray extends NumPy and Pandas to enable scientists to express complex computations on complex datasets
in ways that they find intuitive.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask.pydata.org"&gt;Dask&lt;/a&gt;: Provides the parallel computation behind XArray&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/daskernetes"&gt;Daskernetes&lt;/a&gt;: Makes it easy to launch Dask clusters on Kubernetes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://kubernetes.io/"&gt;Kubernetes&lt;/a&gt;: In case it’s not already clear, all of this is based on Kubernetes,
which manages launching programs (like Jupyter notebook servers or Dask workers) on different machines,
while handling load balancing, permissions, and so on&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://cloud.google.com/kubernetes-engine/"&gt;Google Container Engine&lt;/a&gt;: Google’s managed Kubernetes service.
Every major cloud provider now has such a system,
which makes us happy about not relying too heavily on one system&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://gcsfs.readthedocs.io/en/latest/"&gt;GCSFS&lt;/a&gt;: A Python library providing intuitive access to Google Cloud Storage,
either through Python file interfaces or through a &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Filesystem_in_Userspace"&gt;FUSE&lt;/a&gt; file system&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://zarr.readthedocs.io/en/stable/"&gt;Zarr&lt;/a&gt;: A chunked array storage format that is suitable for the cloud&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/2018/01/22/pangeo-2.md&lt;/span&gt;, line 74)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="collaboration"&gt;
&lt;h1&gt;Collaboration&lt;/h1&gt;
&lt;p&gt;We were able to build, deploy, and use this system to answer real science questions in a couple weeks.
We feel that this result is significant in its own right,
and is largely because we collaborated widely.
This project required the expertise of several individuals across several projects, institutions, and funding sources.
Here are a few examples of who did what from which organization.
We list institutions and positions mostly to show the roles involved.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Alistair Miles, Professor, Oxford:
Helped to optimize Zarr for XArray on GCS&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jacob Tomlinson, Staff, UK Met Informatics Lab:
Developed original JADE deployment and early Dask-Kubernetes work.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joe Hamman, Postdoc, National Center for Atmospheric Research:
Provided scientific use case, data, and work flow.
Tuned XArray and Zarr for efficient data storing and saving.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant, Software developer, Anaconda Inc.:
Tuned GCSFS for many-access workloads. Also provided FUSE system for NetCDF support&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matt Pryor, Staff, Centre for Envronmental Data Analysis:
Extended original JADE deployment and early Dask-Kubernetes work.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin, Software Developer, Anaconda Inc.
Integration. Also performance testing.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ryan Abernathey, Assistant Professor, Columbia University:
XArray + Zarr support, scientific use cases, coordination&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stephan Hoyer, Software engineer, Google:
XArray support&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Yuvi Panda, Staff, UC Berkeley BIDS and Data Science Education Program:
Provided assistance configuring JupyterHub with KubeSpawner.
Also prototyped the Daskernetes Dask + Kubernetes tool.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Notice the mix of academic and for-profit institutions.
Also notice the mix of scientists, staff, and professional software developers.
We believe that this mixture helps ensure the efficient construction of useful solutions.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/01/22/pangeo-2.md&lt;/span&gt;, line 108)&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="lessons"&gt;
&lt;h1&gt;Lessons&lt;/h1&gt;
&lt;p&gt;This experiment has taught us a few things that we hope to explore further:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;Users can launch Kubernetes deployments from Kubernetes pods,
such as launching Dask clusters from their JupyterHub single-user notebooks.&lt;/p&gt;
&lt;p&gt;To do this well we need to start defining user roles more explicitly within JupyterHub.
We need to give users a safe an isolated space on the cluster to use without affecting their neighbors.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;HDF5 and NetCDF on cloud storage is an open question&lt;/p&gt;
&lt;p&gt;The file formats used for this sort of data are pervasive,
but not particulary convenient or efficent on cloud storage.
In particular the libraries used to read them make many small reads,
each of which is costly when operating on cloud object storage&lt;/p&gt;
&lt;p&gt;I see a few options:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Use FUSE file systems,
but tune them with tricks like read-ahead and caching
in order to compensate for HDF’s access patterns&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Use the HDF group’s proposed HSDS service,
which promises to resolve these issues&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Adopt new file formats that are more cloud friendly.
Zarr is one such example that has so far performed admirably,
but certainly doesn’t have the long history of trust that HDF and NetCDF have earned.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Environment customization is important and tricky, especially when adding distributed computing.&lt;/p&gt;
&lt;p&gt;Immediately after showing this to science groups they want to try it out with their own software environments.
They can do this easily in their notebook session with tools like pip or conda,
but to apply those same changes to their dask workers is a bit more challenging,
especially when those workers come and go dynamically.&lt;/p&gt;
&lt;p&gt;We have solutions for this.
They can bulid and publish docker images.
They can add environment variables to specify extra pip or conda packages.
They can deploy their own pangeo deployment for their own group.&lt;/p&gt;
&lt;p&gt;However these have all taken some work to do well so far.
We hope that some combination of Binder-like publishing and small modification tricks like environment variables resolve this problem.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Our docker images are very large.
This means that users sometimes need to wait a minute or more for their session or their dask workers to start up
(less after things have warmed up a bit).&lt;/p&gt;
&lt;p&gt;It is surprising how much of this comes from conda and node packages.
We hope to resolve this both by improving our Docker hygeine
and by engaging packaging communities to audit package size.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Explore other clouds&lt;/p&gt;
&lt;p&gt;We started with Google just because their Kubernetes support has been around the longest,
but all major cloud providers (Google, AWS, Azure) now provide some level of managed Kubernetes support.
Everything we’ve done has been cloud-vendor agnostic, and various groups with data already on other clouds have reached out and are starting deployment on those systems.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Combine efforts with other groups&lt;/p&gt;
&lt;p&gt;We’re actually not the first group to do this.
The UK Met Informatics Lab quietly built a similar prototype, JADE (Jupyter and Dask Environment) many months ago.
We’re now collaborating to merge efforts.&lt;/p&gt;
&lt;p&gt;It’s also worth mentioning that they prototyped the first iteration of Daskernetes.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Reach out to other communities&lt;/p&gt;
&lt;p&gt;While we started our collaboration with atmospheric and oceanographic scientists,
these same solutions apply to many other disciplines.
We should investigate other fields and start collaborations with those communities.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Improve Dask + XArray algorithms&lt;/p&gt;
&lt;p&gt;When we try new problems in new environments we often uncover new opportunities to improve Dask’s internal scheduling algorithms.
This case is no different :)&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Much of this upcoming work is happening in the upstream projects
so this experimentation is both of concrete use to ongoing scientific research
as well as more broad use to the open source communities that these projects serve.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/01/22/pangeo-2.md&lt;/span&gt;, line 188)&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="community-uptake"&gt;
&lt;h1&gt;Community uptake&lt;/h1&gt;
&lt;p&gt;We presented this at a couple conferences over the past week.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;American Meteorological Society, Python Symposium, Keynote. Slides: &lt;a class="reference external" href="http://matthewrocklin.com/slides/ams-2018.html#/"&gt;http://matthewrocklin.com/slides/ams-2018.html#/&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Earth Science Information Partners Winter Meeting. Video: &lt;a class="reference external" href="https://www.youtube.com/watch?v=mDrjGxaXQT4"&gt;https://www.youtube.com/watch?v=mDrjGxaXQT4&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We found that this project aligns well with current efforts from many government agencies to publish large datasets on cloud stores (mostly S3).
Many of these data publication endeavors seek a computational system to enable access for the scientific public.
Our project seems to complement these needs without significant coordination.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2018/01/22/pangeo-2.md&lt;/span&gt;, line 199)&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="disclaimers"&gt;
&lt;h1&gt;Disclaimers&lt;/h1&gt;
&lt;p&gt;While we encourage people to try out &lt;a class="reference external" href="http://pangeo.pydata.org"&gt;pangeo.pydata.org&lt;/a&gt; we also warn you that this system is immature.
In particular it has the following issues:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;it is insecure, please do not host sensitive data&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;it is unstable, and may be taken down at any time&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;it is small, we only have a handful of cores deployed at any time, mostly for experimentation purposes&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;However it is also &lt;em&gt;open&lt;/em&gt;, and instructions to deploy your own &lt;a class="reference external" href="https://github.com/pangeo-data/pangeo/tree/master/gce"&gt;live 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/2018/01/22/pangeo-2.md&lt;/span&gt;, line 210)&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="come-help"&gt;
&lt;h1&gt;Come help&lt;/h1&gt;
&lt;p&gt;We are a growing group comprised of many institutions including technologists, scientists, and open source projects.
There is plenty to do and plenty to discuss.
Please engage with us at &lt;a class="reference external" href="https://github.com/pangeo-data/pangeo/issues/new"&gt;github.com/pangeo-data/pangeo/issues/new&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/01/22/pangeo-2/"/>
    <summary>This work is supported by Anaconda Inc, the NSF
EarthCube program, and UC Berkeley BIDS</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2018-01-22T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/12/06/dask-dev-9/</id>
    <title>Dask Development Log</title>
    <updated>2017-12-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://anaconda.com"&gt;Anaconda Inc&lt;/a&gt; and the
Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;To increase transparency I’m trying to blog more often about the current work
going on around Dask and related projects. Nothing here is ready for
production. This blogpost is written in haste, so refined polish should not be
expected.&lt;/p&gt;
&lt;p&gt;Current development in Dask and Dask-related projects includes the following
efforts:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;A possible change to our community communication model&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A rewrite of the distributed scheduler&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kubernetes and Helm Charts for Dask&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Adaptive deployment fixes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Continued support for NumPy and Pandas’ growth&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Spectral Clustering in Dask-ML&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/2017/12/06/dask-dev-9.md&lt;/span&gt;, line 27)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="community-communication"&gt;

&lt;p&gt;Dask community communication generally happens in Github issues for bug and
feature tracking, the Stack Overflow #dask tag for user questions, and an
infrequently used Gitter chat.&lt;/p&gt;
&lt;p&gt;Separately, Dask developers who work for Anaconda Inc (there are about five of
us part-time) use an internal company chat and a closed weekly video meeting.
We’re now trying to migrate away from closed systems when possible.&lt;/p&gt;
&lt;p&gt;Details about future directions are in &lt;a class="reference external" href="https://github.com/dask/dask/issues/2945"&gt;dask/dask
#2945&lt;/a&gt;. Thoughts and comments on
that issue would be welcome.&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/2017/12/06/dask-dev-9.md&lt;/span&gt;, line 41)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="scheduler-rewrite"&gt;
&lt;h1&gt;Scheduler Rewrite&lt;/h1&gt;
&lt;p&gt;When you start building clusters with 1000 workers the distributed scheduler
can become a bottleneck on some workloads. After working with PyPy and Cython
development teams we’ve decided to rewrite parts of the scheduler to make it
more amenable to acceleration by those technologies. Note that no actual
acceleration has occurred yet, just a refactor of internal state.&lt;/p&gt;
&lt;p&gt;Previously the distributed scheduler was focused around a large set of Python
dictionaries, sets, and lists that indexed into each other heavily. This was
done both for low-tech code technology reasons and for performance reasons
(Python core data structures are fast). However, compiler technologies like
PyPy and Cython can optimize Python object access down to C speeds, so we’re
experimenting with switching away from Python data structures to Python objects
to see how much this is able to help.&lt;/p&gt;
&lt;p&gt;This change will be invisible operationally (the full test suite remains
virtually unchanged), but will be a significant change to the scheduler’s
internal state. We’re keeping around a compatibility layer, but people who
were building their own diagnostics around the internal state should check out
with the new changes.&lt;/p&gt;
&lt;p&gt;Ongoing work by Antoine Pitrou in &lt;a class="reference external" href="https://github.com/dask/distributed/pull/1594"&gt;dask/distributed #1594&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/2017/12/06/dask-dev-9.md&lt;/span&gt;, line 65)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="kubernetes-and-helm-charts-for-dask"&gt;
&lt;h1&gt;Kubernetes and Helm Charts for Dask&lt;/h1&gt;
&lt;p&gt;In service of the &lt;a class="reference external" href="https://pangeo-data.github.io"&gt;Pangeo&lt;/a&gt; project to enable
scalable data analysis of atmospheric and oceanographic data we’ve been
improving the tooling around launching Dask on Cloud infrastructure,
particularly leveraging Kubernetes.&lt;/p&gt;
&lt;p&gt;To that end we’re making some flexible Docker containers and Helm Charts for
Dask, and hope to combine them with JupyterHub in the coming weeks.&lt;/p&gt;
&lt;p&gt;Work done by myself in the following repositories. Feedback would be very
welcome. I am learning on the job with Helm here.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/helm-chart"&gt;https://github.com/dask/dask-docker&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/dask/helm-chart"&gt;dask/helm-chart&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;If you use Helm on Kubernetes then you might want to try the following:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;helm&lt;/span&gt; &lt;span class="n"&gt;repo&lt;/span&gt; &lt;span class="n"&gt;add&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt; &lt;span class="n"&gt;https&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;github&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;helm&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;chart&lt;/span&gt;
&lt;span class="n"&gt;helm&lt;/span&gt; &lt;span class="n"&gt;update&lt;/span&gt;
&lt;span class="n"&gt;helm&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This installs a full Dask cluster and a Jupyter server. The Docker containers
contain entry points that allow their environments to be updated with custom
packages easily.&lt;/p&gt;
&lt;p&gt;This work extends prior work on the previous package,
&lt;a class="reference external" href="https://github.com/dask/dask-kubernetes"&gt;dask-kubernetes&lt;/a&gt;, but is slightly
more modular for use alongside other systems.&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/2017/12/06/dask-dev-9.md&lt;/span&gt;, line 97)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="adaptive-deployment-fixes"&gt;
&lt;h1&gt;Adaptive deployment fixes&lt;/h1&gt;
&lt;p&gt;Adaptive deployments, where a cluster manager scales a Dask cluster up or down
based on current workloads recently got a makeover, including a number of bug
fixes around odd or infrequent behavior.&lt;/p&gt;
&lt;p&gt;Work done by &lt;a class="reference external" href="https://github.com/rbubley"&gt;Russ Bubley&lt;/a&gt; here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/1607"&gt;dask/distributed #1607&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/1608"&gt;dask/distributed #1608&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/1609"&gt;dask/distributed #1609&lt;/a&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/2017/12/06/dask-dev-9.md&lt;/span&gt;, line 109)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="keeping-up-with-numpy-and-pandas"&gt;
&lt;h1&gt;Keeping up with NumPy and Pandas&lt;/h1&gt;
&lt;p&gt;NumPy 1.14 is due to release soon. Dask.array had to update how it handled
structured dtypes in &lt;a class="reference external" href="https://github.com/dask/dask/pull/2964"&gt;dask/dask #2694&lt;/a&gt;
(Work by Tom Augspurger).&lt;/p&gt;
&lt;p&gt;Dask.dataframe is gaining the ability to merge/join simultaneously on columns
and indices, following a similar feature released in Pandas 0.22. Work done by
Jon Mease in &lt;a class="reference external" href="https://github.com/dask/dask/pull/2960"&gt;dask/dask #2960&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/2017/12/06/dask-dev-9.md&lt;/span&gt;, line 119)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="spectral-clustering-in-dask-ml"&gt;
&lt;h1&gt;Spectral Clustering in Dask-ML&lt;/h1&gt;
&lt;p&gt;Dask-ML recently added an approximate and scalable Spectral Clustering
algorithm in &lt;a class="reference external" href="https://github.com/dask/dask-ml/pull/91"&gt;dask/dask-ml #91&lt;/a&gt;
(&lt;a class="reference external" href="http://dask-ml.readthedocs.io/en/latest/auto_examples/plot_spectral_clustering.html"&gt;gallery example&lt;/a&gt;).&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/12/06/dask-dev-9/"/>
    <summary>This work is supported by Anaconda Inc and the
Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2017-12-06T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/11/21/dask-0.16.0/</id>
    <title>Dask Release 0.16.0</title>
    <updated>2017-11-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://anaconda.com"&gt;Anaconda Inc.&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;I’m pleased to announce the release of Dask version 0.16.0. This is a major
release with new features, breaking changes, and stability improvements. This
blogpost outlines notable changes since the 0.15.3 release on September 24th.&lt;/p&gt;
&lt;p&gt;You can conda install Dask:&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 pip install from PyPI:&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[complete] --upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Conda packages are available on both conda-forge and default channels.&lt;/p&gt;
&lt;p&gt;Full changelogs are available here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/blob/master/docs/source/changelog.rst"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/docs/source/changelog.rst"&gt;dask/distributed&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Some notable changes follow.&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/2017/11/21/dask-0.16.0.md&lt;/span&gt;, line 33)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="breaking-changes"&gt;

&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.async&lt;/span&gt;&lt;/code&gt; module was moved to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.local&lt;/span&gt;&lt;/code&gt; for Python 3.7
compatibility. This was previously deprecated and is now fully removed.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The distributed scheduler’s diagnostic JSON pages have been removed and
replaced by more informative templated HTML.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The use of commonly-used private methods &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;_keys&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;_optimize&lt;/span&gt;&lt;/code&gt; have been
replaced with the Dask collection interface (see below).&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/2017/11/21/dask-0.16.0.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="dask-collection-interface"&gt;
&lt;h1&gt;Dask collection interface&lt;/h1&gt;
&lt;p&gt;It is now easier to implement custom collections using the Dask collection
interface.&lt;/p&gt;
&lt;p&gt;Dask collections (arrays, dataframes, bags, delayed) interact with Dask
schedulers (single-machine, distributed) with a few internal methods. We
formalized this interface into protocols like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.__dask_graph__()&lt;/span&gt;&lt;/code&gt; and
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.__dask_keys__()&lt;/span&gt;&lt;/code&gt; and have
&lt;a class="reference external" href="http://dask.pydata.org/en/latest/custom-collections.html"&gt;published that interface&lt;/a&gt;.
Any object that implements the methods described in that document will interact
with all Dask scheduler features as a first-class Dask object.&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;class&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nc"&gt;MyDaskCollection&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;object&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;__dask_graph__&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="o"&gt;...&lt;/span&gt;

    &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;__dask_keys__&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="o"&gt;...&lt;/span&gt;

    &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;__dask_optimize__&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="o"&gt;...&lt;/span&gt;

    &lt;span class="o"&gt;...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This interface has already been implemented within the XArray project for
labeled and indexed arrays. Now all XArray classes (DataSet, DataArray,
Variable) are fully understood by all Dask schedulers. They are as first-class
as dask.arrays or dask.dataframes.&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;xarray&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;xa&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;

&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;ds&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xa&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_mfdataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;*.nc&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;ds&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ds&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# XArray object integrate seamlessly with Dask schedulers&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Documentation:
&lt;a class="reference external" href="http://dask.pydata.org/en/latest/custom-collections.html"&gt;http://dask.pydata.org/en/latest/custom-collections.html&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;em&gt;Work on Dask’s collection interfaces was primarily done by Jim Crist.&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/2017/11/21/dask-0.16.0.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="bandwidth-and-tornado-5-compatibility"&gt;
&lt;h1&gt;Bandwidth and Tornado 5 compatibility&lt;/h1&gt;
&lt;p&gt;Dask is built on the Tornado library for concurrent network programming. In an
effort to improve inter-worker bandwidth on exotic hardware (Infiniband), Dask
developers are proposing changes to Tornado’s network infrastructure.&lt;/p&gt;
&lt;p&gt;However, in order to use these changes Dask itself needs to run on the next
version of Tornado in development, Tornado 5.0.0, which breaks a number of
interfaces on which Dask has relied. Dask developers have been resolving these
and we encourage other PyData developers to do the same. For example, neither
Bokeh nor Jupyter work on Tornado 5.0.0-dev.&lt;/p&gt;
&lt;p&gt;Dask inter-worker bandwidth is peaking at around 1.5-2GB/s on a network
theoretically capable of 3GB/s. &lt;a class="reference external" href="https://github.com/pangeo-data/pangeo/issues/6"&gt;GitHub issue: pangeo #6&lt;/a&gt;&lt;/p&gt;
&lt;a href= "/images/bandwidth-plot.png"&gt;
  &lt;img src="/images/bandwidth-plot.png"
       alt="Dask worker bandwidth"
       width="100%"&gt;&lt;/a&gt;
&lt;p&gt;&lt;em&gt;Network performance and Tornado compatibility are primarily being handled by
Antoine Pitrou.&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/2017/11/21/dask-0.16.0.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="parquet-compatibility"&gt;
&lt;h1&gt;Parquet Compatibility&lt;/h1&gt;
&lt;p&gt;Dask.dataframe can use either of the two common Parquet libraries in Python,
Apache Arrow and Fastparquet. Each has its own strengths and its own base of
users who prefer it. We’ve significantly extended Dask’s parquet test suite to
cover each library, extending roundtrip compatibility. Notably, you can now
both read and write with PyArrow.&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_parquet&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="n"&gt;engine&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;fastparquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&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="n"&gt;engine&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;pyarrow&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;There is still work to be done here. The variety of parquet reader/writers and
conventions out there makes completely solving this problem difficult. It’s
nice seeing the various projects slowly converge on common functionality.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;This work was jointly done by Uwe Korn, Jim Crist, and Martin Durant.&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/2017/11/21/dask-0.16.0.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="retrying-tasks"&gt;
&lt;h1&gt;Retrying Tasks&lt;/h1&gt;
&lt;p&gt;One of the most requested features for the Dask.distributed scheduler is the
ability to retry failed tasks. This is particularly useful to people using
Dask as a task queue, rather than as a big dataframe or 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;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&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="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;args&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;retries&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;5&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;Task retries were primarily built by Antoine Pitrou.&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/2017/11/21/dask-0.16.0.md&lt;/span&gt;, line 144)&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="transactional-work-stealing"&gt;
&lt;h1&gt;Transactional Work Stealing&lt;/h1&gt;
&lt;p&gt;The Dask.distributed task scheduler performs load balancing through work
stealing. Previously this would sometimes result in the same task running
simultaneously in two locations. Now stealing is transactional, meaning that
it will avoid accidentally running the same task twice. This behavior is
especially important for people using Dask tasks for side effects.&lt;/p&gt;
&lt;p&gt;It is still possible for the same task to run twice, but now this only happens
in more extreme situations, such as when a worker dies or a TCP connection is
severed, neither of which are common on standard hardware.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Transactional work stealing was primarily implemented by Matthew Rocklin.&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/2017/11/21/dask-0.16.0.md&lt;/span&gt;, line 158)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="new-diagnostic-pages"&gt;
&lt;h1&gt;New Diagnostic Pages&lt;/h1&gt;
&lt;p&gt;There is a new set of diagnostic web pages available in the &lt;em&gt;Info&lt;/em&gt; tab of the
dashboard. These pages provide more in-depth information about each worker and
task, but are not dynamic in any way. They use Tornado templates rather than
Bokeh plots, which means that they are less responsive but are much easier to
build. This is an easy and cheap way to expose more scheduler state.&lt;/p&gt;
&lt;a href= "/images/scheduler-info-task.png"&gt;
  &lt;img src="/images/scheduler-info-task.png"
       alt="Task page of Dask's scheduler info dashboard"
       width="100%"&gt;&lt;/a&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/tree/master/distributed/bokeh/templates"&gt;Existing templates&lt;/a&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/2017/11/21/dask-0.16.0.md&lt;/span&gt;, line 173)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="nested-compute-calls"&gt;
&lt;h1&gt;Nested compute calls&lt;/h1&gt;
&lt;p&gt;Calling &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.compute()&lt;/span&gt;&lt;/code&gt; &lt;em&gt;within&lt;/em&gt; a task now invokes the same distributed
scheduler. This enables writing more complex workloads with less thought to
starting worker clients.&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;dask&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# only works for the newer scheduler&lt;/span&gt;

&lt;span class="nd"&gt;@dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;
&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;f&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="o"&gt;...&lt;/span&gt;
    &lt;span class="k"&gt;return&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;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# can call dask.compute within delayed task&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;compute&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="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="o"&gt;...&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;Nested compute calls were primarily developed by Matthew Rocklin and Olivier
Grisel.&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/2017/11/21/dask-0.16.0.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="more-aggressive-garbage-collection"&gt;
&lt;h1&gt;More aggressive Garbage Collection&lt;/h1&gt;
&lt;p&gt;The workers now explicitly call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;gc.collect()&lt;/span&gt;&lt;/code&gt; at various times when under
memory pressure and when releasing data. This helps to avoid some memory
leaks, especially when using Pandas dataframes. Doing this carefully proved
to require a surprising degree of detail.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Improved garbage collection was primarily implemented and tested by Fabian
Keller and Olivier Grisel, with recommendations by Antoine Pitrou.&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/2017/11/21/dask-0.16.0.md&lt;/span&gt;, line 205)&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-projects"&gt;
&lt;h1&gt;Related projects&lt;/h1&gt;
&lt;section id="dask-ml"&gt;
&lt;h2&gt;Dask-ML&lt;/h2&gt;
&lt;p&gt;A variety of Dask Machine Learning projects are now being assembled under one
unified repository, &lt;a class="reference external" href="http://dask-ml.readthedocs.io/en/latest/"&gt;dask-ml&lt;/a&gt;. We
encourage users and researchers alike to read through that project. We believe
there are many useful and interesting approaches contained within.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Docs: &lt;a class="reference external" href="http://dask-ml.readthedocs.io/en/latest/"&gt;dask-ml.readthedocs.io&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Github: &lt;a class="reference external" href="https://github.com/dask/dask-ml"&gt;github.com/dask/dask-ml&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;em&gt;The work to assemble and curate these algorithms is primarily being handled by
Tom Augspurger.&lt;/em&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="xarray"&gt;
&lt;h2&gt;XArray&lt;/h2&gt;
&lt;p&gt;The &lt;a class="reference external" href="http://xarray.pydata.org/en/stable/"&gt;XArray&lt;/a&gt; project for indexed and
labeled arrays is also releasing their major 0.10.0 release this week, which
includes many performance improvements, particularly for using Dask on larger
datasets.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Docs: &lt;a class="reference external" href="http://xarray.pydata.org/en/stable/"&gt;xarray.pydata.org&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Release notes: &lt;a class="reference external" href="http://xarray.pydata.org/en/latest/whats-new.html"&gt;xarray.pydata.org/en/latest/whats-new.html&lt;/a&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/2017/11/21/dask-0.16.0.md&lt;/span&gt;, line 230)&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="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;The following people contributed to the dask/dask repository since the 0.15.3
release on September 24th:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Ced4&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Christopher Prohm&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;fjetter&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Hai Nguyen Mau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ian Hopkinson&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Bourbeau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Munroe&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jesse Vogt&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Keisuke Fujii&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthias Bussonnier&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;mayl&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Olivier Grisel&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;severo&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Simon Perkins&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stephan Hoyer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Thomas A Caswell&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Uwe L. Korn&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Wei Ji&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;xwang777&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The following people contributed to the dask/distributed repository since the
1.19.1 release on September 24nd:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Alvaro Ulloa&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;chkoar&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Fabian Keller&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ian Hopkinson&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kelvin Yang&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Krisztián Szűcs&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Mike DePalatis&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Olivier Grisel&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;rbubley&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The following people contributed to the dask/dask-ml repository&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Evan Welch&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;severo&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Trey Causey&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;In addition, we are proud to announce that Olivier Grisel has accepted commit
rights to the Dask projects. Olivier has been particularly active on the
distributed scheduler, and on related projects like Joblib, SKLearn, and
Cloudpickle.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/11/21/dask-0.16.0/"/>
    <summary>This work is supported by Anaconda Inc.
and the Data Driven Discovery Initiative from the Moore
Foundation.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2017-11-21T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/11/03/data-structure-benchmark/</id>
    <title>Optimizing Data Structure Access in Python</title>
    <updated>2017-11-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://anaconda.com"&gt;Anaconda Inc&lt;/a&gt; and the Data
Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Last week at &lt;a class="reference external" href="https://de.pycon.org/"&gt;PyCon DE&lt;/a&gt; I had the good fortune to meet
&lt;a class="reference external" href="http://www.behnel.de/"&gt;Stefan Behnel&lt;/a&gt;, one of the core developers of Cython.
Together we worked to optimize a small benchmark that is representative of
Dask’s central task scheduler, a pure-Python application that is primarily data
structure bound.&lt;/p&gt;
&lt;p&gt;Our benchmark is a toy problem that creates three data structures that index
each other with dictionaries, lists, and sets, and then does some simple
arithmetic. (You don’t need to understand this benchmark deeply to read this
article.)&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;random&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;time&lt;/span&gt;

&lt;span class="n"&gt;nA&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;100&lt;/span&gt;
&lt;span class="n"&gt;nB&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;100&lt;/span&gt;
&lt;span class="n"&gt;nC&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;100&lt;/span&gt;

&lt;span class="n"&gt;A&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;A-&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="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;B-&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;nB&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="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="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;5&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="n"&gt;nA&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="p"&gt;{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;B-&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="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;C-&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;nC&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="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="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;3&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="n"&gt;nB&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="p"&gt;{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;C-&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="p"&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="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nC&lt;/span&gt;&lt;span class="p"&gt;)}&lt;/span&gt;

&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;A-&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="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nA&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;f&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="n"&gt;C&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="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;a_key&lt;/span&gt; &lt;span class="ow"&gt;in&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_keys&lt;/span&gt; &lt;span class="o"&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;a_key&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;b_key&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;b_keys&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;c_key&lt;/span&gt; &lt;span class="ow"&gt;in&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;b_key&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt;
                &lt;span class="n"&gt;C&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;c_key&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="n"&gt;start&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;time&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;10000&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="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="n"&gt;C&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="n"&gt;end&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Duration: &lt;/span&gt;&lt;span class="si"&gt;%0.3f&lt;/span&gt;&lt;span class="s2"&gt; seconds&amp;quot;&lt;/span&gt; &lt;span class="o"&gt;%&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;end&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;start&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ python benchmark.py
Duration: 1.12 seconds
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is an atypical problem Python optimization because it is primarily bound
by data structure access (dicts, lists, sets), rather than numerical operations
commonly optimized by Cython (nested for loops over floating point arithmetic).
Python is already decently fast here, typically within a factor of 2-5x of
compiled languages like Java or C++, but still we’d like to improve this when
possible.&lt;/p&gt;
&lt;p&gt;In this post we combine two different methods to optimize data-structure bound
workloads:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Compiling Python code with Cython with no other annotations&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Interning strings for more rapid dict lookups&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Finally at the end of the post we also run the benchmark under PyPy to compare
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/2017/11/03/data-structure-benchmark.md&lt;/span&gt;, line 82)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="cython"&gt;

&lt;p&gt;First we compile our Python code with Cython. Normally when using Cython we
annotate our variables with types, giving the compiler enough information
to avoid using Python altogether. However in our case we don’t have many
numeric operations and we’re going to be using Python data structures
regardless, so this won’t help much. We compile our original Python code
without alteration.&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;cythonize -i benchmark.py
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And run&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ python -c &amp;quot;import benchmark&amp;quot;
Duration: 0.73 seconds
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This gives us a decent speedup from 1.1 seconds to 0.73 seconds. This isn’t
huge relative to typical Cython speedups (which are often 10-100x) but would be
a &lt;em&gt;very welcome&lt;/em&gt; change for our scheduler, where we’ve been chasing 5%
optimizations for a while now.&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/2017/11/03/data-structure-benchmark.md&lt;/span&gt;, line 103)&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="interning-strings"&gt;
&lt;h1&gt;Interning Strings&lt;/h1&gt;
&lt;p&gt;Our second trick is to intern strings. This means that we try to always have
only one copy of every string. This improves performance when doing dictionary
lookups because of the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Python computes the hash of the string only once (strings cache their hash
value once computed)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Python checks for object identity (fast) before moving on to value equality
(slow)&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Or, anecdotally, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;text&lt;/span&gt; &lt;span class="pre"&gt;is&lt;/span&gt; &lt;span class="pre"&gt;text&lt;/span&gt;&lt;/code&gt; is faster in Python than &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;text&lt;/span&gt; &lt;span class="pre"&gt;==&lt;/span&gt; &lt;span class="pre"&gt;text&lt;/span&gt;&lt;/code&gt;. If
you ensure that there is only one copy of every string then you only need to do
identity comparisons like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;text&lt;/span&gt; &lt;span class="pre"&gt;is&lt;/span&gt; &lt;span class="pre"&gt;text&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;So, if any time we see a string &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;&amp;quot;abc&amp;quot;&lt;/span&gt;&lt;/code&gt; it is exactly the same object as all
other &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;&amp;quot;abc&amp;quot;&lt;/span&gt;&lt;/code&gt; strings in our program, then string-dict lookups will only
require a pointer/integer equality check, rather than having to do a full
string comparison.&lt;/p&gt;
&lt;p&gt;Adding string interning to our benchmark 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="n"&gt;inter&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{}&lt;/span&gt;

&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;intern&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;try&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;inter&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;except&lt;/span&gt; &lt;span class="ne"&gt;KeyError&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;inter&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;
        &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;

&lt;span class="n"&gt;A&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="n"&gt;intern&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;A-&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="p"&gt;):&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;intern&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;B-&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;nB&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="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="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;5&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="n"&gt;nA&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="p"&gt;{&lt;/span&gt;&lt;span class="n"&gt;intern&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;B-&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="p"&gt;):&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="n"&gt;intern&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;C-&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;nC&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="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="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;3&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="n"&gt;nB&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="p"&gt;{&lt;/span&gt;&lt;span class="n"&gt;intern&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;C-&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="p"&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="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nC&lt;/span&gt;&lt;span class="p"&gt;)}&lt;/span&gt;

&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;intern&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;A-&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="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="n"&gt;nA&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;

&lt;span class="c1"&gt;# The rest of the benchmark is as before&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This brings our duration from 1.1s down to 0.75s. Note that this is without
the separate Cython improvements described just above.&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/2017/11/03/data-structure-benchmark.md&lt;/span&gt;, line 153)&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="cython-interning"&gt;
&lt;h1&gt;Cython + Interning&lt;/h1&gt;
&lt;p&gt;We can combine both optimizations. This brings us to around 0.45s, a 2-3x
improvement over our original time.&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;cythonize -i benchmark2.py

$ python -c &amp;quot;import benchmark2&amp;quot;
Duration: 0.46 seconds
&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/2017/11/03/data-structure-benchmark.md&lt;/span&gt;, line 163)&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="pypy"&gt;
&lt;h1&gt;PyPy&lt;/h1&gt;
&lt;p&gt;Alternatively, we can just run everything in PyPy.&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ pypy3 benchmark1.py  # original
Duration: 0.25 seconds

$ pypy3 benchmark2.py  # includes interning
Duraiton: 0.20 seconds
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So PyPy can be quite a bit faster than Cython on this sort of code (which is
not a big surprise). Interning helps a bit, but not quite as much.&lt;/p&gt;
&lt;p&gt;This is fairly encouraging. The Dask scheduler can run under PyPy even while
Dask clients and workers run under normal CPython (for use with the full PyData
stack).&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/2017/11/03/data-structure-benchmark.md&lt;/span&gt;, line 180)&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="preliminary-results-on-dask-benchmark"&gt;
&lt;h1&gt;Preliminary Results on Dask Benchmark&lt;/h1&gt;
&lt;p&gt;We started this experiment with the assumption that our toy benchmark somehow
represented the Dask’s scheduler in terms of performance characteristics. This
assumption, of course, is false. The Dask scheduler is significantly more
complex and it is difficult to build a single toy example to represent its
performance.&lt;/p&gt;
&lt;p&gt;When we try these tricks on a &lt;a class="reference external" href="https://gist.github.com/88b3c29e645ba2eae2d079a1de25d266"&gt;slightly more complex
benchmark&lt;/a&gt; that
actually uses the Dask scheduler we find the following results:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Cython&lt;/strong&gt;: almost no effect&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;String Interning&lt;/strong&gt;: almost no effect&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;PyPy&lt;/strong&gt;: almost no effect&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;However I have only spent a brief amount of time on this (twenty minutes?) and
so I hope that the lack of a performance gain here is due to lack of effort.&lt;/p&gt;
&lt;p&gt;If anyone is interested in this I hope that this blogpost contains enough
information to get anyone started if they want to investigate further.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/11/03/data-structure-benchmark/"/>
    <summary>This work is supported by Anaconda Inc and the Data
Driven Discovery Initiative from the Moore Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2017-11-03T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/10/16/streaming-dataframes-1/</id>
    <title>Streaming Dataframes</title>
    <updated>2017-10-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://anaconda.com"&gt;Anaconda Inc&lt;/a&gt; and the Data
Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;This post is about experimental software. This is not ready for public use.
All code examples and API in this post are subject to change without warning.&lt;/em&gt;&lt;/p&gt;
&lt;a href="/images/streaming-dataframes-plot.gif"&gt;
  &lt;img src="/images/streaming-dataframes-plot.gif"
     align="right"
     width="70%"&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/2017/10/16/streaming-dataframes-1.md&lt;/span&gt;, line 19)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;This post describes a prototype project to handle continuous data sources of
tabular data using Pandas and Streamz.&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/2017/10/16/streaming-dataframes-1.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="introduction"&gt;
&lt;h1&gt;Introduction&lt;/h1&gt;
&lt;p&gt;Some data never stops. It arrives continuously in a constant, never-ending
stream. This happens in financial time series, web server logs, scientific
instruments, IoT telemetry, and more. Algorithms to handle this data are
slightly different from what you find in libraries like NumPy and Pandas, which
assume that they know all of the data up-front. It’s still possible to use
NumPy and Pandas, but you need to combine them with some cleverness and keep
enough intermediate data around to compute marginal updates when new data comes
in.&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/2017/10/16/streaming-dataframes-1.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-streaming-mean"&gt;
&lt;h1&gt;Example: Streaming Mean&lt;/h1&gt;
&lt;p&gt;For example, imagine that we have a continuous stream of CSV files arriving
and we want to print out the mean of our data over time. Whenever a new CSV
file arrives we need to recompute the mean of the entire dataset. If we’re
clever we keep around enough state so that we can compute this mean without
looking back over the rest of our historical data. We can accomplish this by keeping
running totals and running counts 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="n"&gt;total&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;
&lt;span class="n"&gt;count&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;

&lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;filename&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="c1"&gt;# filenames is an infinite iterator&lt;/span&gt;
    &lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&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="n"&gt;total&lt;/span&gt; &lt;span class="o"&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;df&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;count&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;count&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;count&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;mean&lt;/span&gt; &lt;span class="o"&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;count&lt;/span&gt;
    &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now as we add new files to our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;filenames&lt;/span&gt;&lt;/code&gt; iterator our code prints out new
means that are updated over time. We don’t have a single mean result, we have
continuous stream of mean results that are each valid for the data up to that
point. Our output data is an infinite stream, just like our input data.&lt;/p&gt;
&lt;p&gt;When our computations are linear and straightforward like this a for loop
suffices. However when our computations have several streams branching out or
converging, possibly with rate limiting or buffering between them, this
for-loop approach can grow complex and difficult to manage.&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/2017/10/16/streaming-dataframes-1.md&lt;/span&gt;, line 66)&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="streamz"&gt;
&lt;h1&gt;Streamz&lt;/h1&gt;
&lt;p&gt;A few months ago I pushed a small library called
&lt;a class="reference external" href="http://streamz.readthedocs.io/en/latest/"&gt;streamz&lt;/a&gt;, which handled control flow
for pipelines, including linear map operations, operations that accumulated
state, branching, joining, as well as back pressure, flow control, feedback,
and so on. Streamz was designed to handle all of the movement of data and
signaling of computation at the right time. This library was quietly used by a
couple of groups and now feels fairly clean and useful.&lt;/p&gt;
&lt;p&gt;Streamz was designed to handle the &lt;em&gt;control flow&lt;/em&gt; of such a system, but did
nothing to help you with streaming algorithms. Over the past week I’ve been
building a dataframe module on top of streamz to help with common streaming
tabular data situations. This module uses Pandas and implements a subset of
the Pandas API, so hopefully it will be easy to use for programmers with
existing Python knowledge.&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/2017/10/16/streaming-dataframes-1.md&lt;/span&gt;, line 83)&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: Streaming Mean&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/2017/10/16/streaming-dataframes-1.md&lt;/span&gt;, line 83); &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: streaming mean”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Our example above could be written as follows with streamz&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;source&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Stream&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;filenames&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;path/to/dir/*.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# stream of filenames&lt;/span&gt;
&lt;span class="n"&gt;sdf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;source&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;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="c1"&gt;# stream of Pandas dataframes&lt;/span&gt;
             &lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_dataframe&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;example&lt;/span&gt;&lt;span class="o"&gt;=...&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;        &lt;span class="c1"&gt;# logical streaming dataframe&lt;/span&gt;

&lt;span class="n"&gt;sdf&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;stream&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sink&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;                   &lt;span class="c1"&gt;# printed stream of mean values&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This example is no more clear than the for-loop version. On its own this is
probably a &lt;em&gt;worse&lt;/em&gt; solution than what we had before, just because it involves
new technology. However it starts to become useful in two situations:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;You want to do more complex streaming algorithms&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;sdf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sdf&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;sdf&lt;/span&gt;&lt;span class="o"&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;Alice&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;sdf&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;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sdf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;y&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sink&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# or&lt;/span&gt;

&lt;span class="n"&gt;sdf&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;rolling&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;300ms&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;mean&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;It would require more cleverness to build these algorithms with a for loop
as above.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You want to do multiple operations, deal with flow control, etc..&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;sdf&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;sink&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;sdf&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;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;rate_limit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;0.500&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sink&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;write_to_database&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="o"&gt;...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Consistently branching off computations, routing data correctly, and
handling time can all be challenging to accomplish consistently.&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/2017/10/16/streaming-dataframes-1.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="jupyter-integration-and-streaming-outputs"&gt;
&lt;h1&gt;Jupyter Integration and Streaming Outputs&lt;/h1&gt;
&lt;p&gt;During development we’ve found it very useful to have live updating outputs in
Jupyter.&lt;/p&gt;
&lt;p&gt;Usually when we evaluate code in Jupyter we have static inputs and static
outputs:&lt;/p&gt;
&lt;img src="/images/jupyter-output-static.png" width="40%"&gt;
&lt;p&gt;However now both our inputs and our outputs are live:&lt;/p&gt;
&lt;img src="/images/jupyter-output-streaming.gif" width="70%"&gt;
&lt;p&gt;We accomplish this using a combination of
&lt;a class="reference external" href="https://ipywidgets.readthedocs.io/en/stable/"&gt;ipywidgets&lt;/a&gt; and &lt;a class="reference external" href="https://bokeh.pydata.org/en/latest/"&gt;Bokeh
plots&lt;/a&gt; both of which provide nice hooks to
change previous Jupyter outputs and work well with the Tornado IOLoop (streamz,
Bokeh, Jupyter, and Dask all use Tornado for concurrency). We’re able to build
nicely responsive feedback whenever things change.&lt;/p&gt;
&lt;p&gt;In the following example we build our CSV to dataframe pipeline that updates
whenever new files appear in a directory. Whenever we drag files to the data
directory on the left we see that all of our outputs update on the right.&lt;/p&gt;
&lt;a href="/images/streaming-dataframes-files.gif"&gt;
  &lt;img src="/images/streaming-dataframes-files.gif"
     width="100%"&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/2017/10/16/streaming-dataframes-1.md&lt;/span&gt;, line 153)&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-is-supported"&gt;
&lt;h1&gt;What is supported?&lt;/h1&gt;
&lt;p&gt;This project is very young and could use some help. There are plenty of holes
in the API. That being said, the following works well:&lt;/p&gt;
&lt;p&gt;Elementwise operations:&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;sdf&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="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sdf&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;sdf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;
&lt;span class="n"&gt;sdf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sdf&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;sdf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;z&lt;/span&gt; &lt;span class="o"&gt;&amp;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;Simple reductions:&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;sdf&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;sdf&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Groupby reductions:&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;sdf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sdf&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;y&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Rolling reductions by number of rows or time window&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;sdf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rolling&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="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;sdf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rolling&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;100ms&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;quantile&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;0.9&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Real time plotting with &lt;a class="reference external" href="https://bokeh.pydata.org"&gt;Bokeh&lt;/a&gt; (one of my favorite features)&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;sdf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;plot&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/streaming-dataframes-plot.gif"&gt;
  &lt;img src="/images/streaming-dataframes-plot.gif"
     width="100%"&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/2017/10/16/streaming-dataframes-1.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="what-s-missing"&gt;
&lt;h1&gt;What’s missing?&lt;/h1&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Parallel computing:&lt;/strong&gt; The core streamz library has an optional
&lt;a class="reference internal" href="#https;//dask.pydata.org/"&gt;&lt;span class="xref myst"&gt;Dask&lt;/span&gt;&lt;/a&gt; backend for parallel computing. I haven’t
yet made any attempt to attach this to the dataframe implementation.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Data ingestion&lt;/strong&gt; from common streaming sources like Kafka. We’re in the
process now of building asynchronous-aware wrappers around Kafka Python
client libraries, so this is likely to come soon.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Out-of-order data access:&lt;/strong&gt; soon after parallel data ingestion (like
reading from multiple Kafka partitions at once) we’ll need to figure out
how to handle out-of-order data access. This is doable, but will take some
effort. This is where more mature libraries like
&lt;a class="reference external" href="https://flink.apache.org/"&gt;Flink&lt;/a&gt; are quite strong.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Performance:&lt;/strong&gt; Some of the operations above (particularly rolling
operations) do involve non-trivial copying, especially with larger windows.
We’re relying heavily on the Pandas library which wasn’t designed with
rapidly changing data in mind. Hopefully future iterations of Pandas
(Arrow/libpandas/Pandas 2.0?) will make this more efficient.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Filled out API:&lt;/strong&gt; Many common operations (like variance) haven’t yet
been implemented. Some of this is due to laziness and some is due to
wanting to find the right algorithm.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Robust plotting:&lt;/strong&gt; Currently this works well for numeric data with a
timeseries index but not so well for other data.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;But most importantly this needs &lt;strong&gt;use&lt;/strong&gt; by people with real problems to help us
understand what here is valuable and what is unpleasant.&lt;/p&gt;
&lt;p&gt;Help would be welcome with any of this.&lt;/p&gt;
&lt;p&gt;You can install this from github&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 git+https://github.com/mrocklin/streamz.git
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Documentation and code are here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://streamz.readthedocs.io/en/latest/"&gt;streamz.readthedocs.io&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/mrocklin/streamz"&gt;github.com/mrocklin/streamz&lt;/a&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/2017/10/16/streaming-dataframes-1.md&lt;/span&gt;, line 233)&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;Current and upcoming work is focused on data ingestion from Kafka and
parallelizing with Dask.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/10/16/streaming-dataframes-1/"/>
    <summary>This work is supported by Anaconda Inc and the Data
Driven Discovery Initiative from the Moore Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2017-10-16T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/10/10/kafka-python/</id>
    <title>Notes on Kafka in Python</title>
    <updated>2017-10-10T00:00:00+00:00</updated>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2017/10/10/kafka-python.md&lt;/span&gt;, line 8)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;&lt;img src="https://kafka.apache.org/images/logo.png"
     align="right"
     width="40%"&gt;&lt;/p&gt;
&lt;p&gt;I recently investigated the state of Python libraries for Kafka. This blogpost
contains my findings.&lt;/p&gt;
&lt;p&gt;Both &lt;a class="reference external" href="http://pykafka.readthedocs.io/en/latest/"&gt;PyKafka&lt;/a&gt; and
&lt;a class="reference external" href="https://github.com/confluentinc/confluent-kafka-python"&gt;confluent-kafka&lt;/a&gt; have
mature implementations and are maintained by invested companies.
Confluent-kafka is generally faster while PyKafka is arguably better designed
and documented for Python usability.&lt;/p&gt;
&lt;p&gt;Conda packages are now available for both. I hope to extend one or both to
support asynchronous workloads with Tornado.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Disclaimer: I am not an expert in this space. I have no strong affiliation
with any of these projects. This is a report based on my experience of the
past few weeks. I don’t encourage anyone to draw conclusions from this work.
I encourage people to investigate on their own.&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/2017/10/10/kafka-python.md&lt;/span&gt;, line 31)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="introduction"&gt;
&lt;h1&gt;Introduction&lt;/h1&gt;
&lt;p&gt;&lt;a class="reference external" href="https://kafka.apache.org/"&gt;Apache Kafka&lt;/a&gt; is a common data system for streaming
architectures. It manages rolling buffers of byte messages and provides a
scalable mechanism to publish or subscribe to those buffers in real time.
While Kafka was originally designed within the JVM space the fact that it only
manages bytes makes it easy to access from native code systems like C/C++ and
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/2017/10/10/kafka-python.md&lt;/span&gt;, line 40)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="python-options"&gt;
&lt;h1&gt;Python Options&lt;/h1&gt;
&lt;p&gt;Today there are three independent Kafka implementations in Python, two of which
are optionally backed by a C implementation,
&lt;a class="reference external" href="https://github.com/edenhill/librdkafka"&gt;librdkafka&lt;/a&gt;, for speed:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://kafka-python.readthedocs.io/en/master/"&gt;kafka-python&lt;/a&gt;: The first on
the scene, a Pure Python Kafka client with robust documentation and an API
that is fairly faithful to the original Java API. This implementation has
the most stars on GitHub, the most active development team (by number of
committers) but also lacks a connection to the fast C library. I’ll admit
that I didn’t spend enough time on this project to judge it well because of
this.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://pykafka.readthedocs.io/en/latest/"&gt;PyKafka&lt;/a&gt;: The second
implementation chronologically. This library is maintained by
&lt;a class="reference external" href="https://www.parse.ly/"&gt;Parse.ly&lt;/a&gt; a web analytics company that heavily uses
both streaming systems and Python. PyKafka’s API is more creative and
designed to follow common Python idioms rather than the Java API. PyKafka
has both a pure Python implementation and connections to the low-level
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;librdkafka&lt;/span&gt;&lt;/code&gt; C library for increased performance.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/confluentinc/confluent-kafka-python"&gt;Confluent-kafka&lt;/a&gt;:
Is the final implementation chronologically. It is maintained by
&lt;a class="reference external" href="https://www.confluent.io/home"&gt;Confluent&lt;/a&gt;, the primary for-profit company
that supports and maintains Kafka. This library is the fastest, but also
the least accessible from a Python perspective. This implementation is
written in CPython extensions, and the documentation is minimal. However,
if you are coming from the Java API then this is entirely consistent with
that experience, so that documentation probably suffices.&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/2017/10/10/kafka-python.md&lt;/span&gt;, line 71)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="performance"&gt;
&lt;h1&gt;Performance&lt;/h1&gt;
&lt;p&gt;Confluent-kafka message-consumption bandwidths are around 50% higher and
message-production bandwidths are around 3x higher than PyKafka, both of which
are significantly higher than kafka-python. I’m taking these numbers from
&lt;a class="reference external" href="http://activisiongamescience.github.io/2016/06/15/Kafka-Client-Benchmarking/"&gt;this
blogpost&lt;/a&gt;
which gives benchmarks comparing the three libraries. The primary numeric
results follow below:&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: It’s worth noting that this blogpost was moving smallish 100 byte messages
around. I would hope that Kafka would perform better (closer to network
bandwidths) when messages are of a decent size.&lt;/em&gt;&lt;/p&gt;
&lt;section id="producer-throughput"&gt;
&lt;h2&gt;Producer Throughput&lt;/h2&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
    &lt;tr style="text-align: right;"&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;time_in_seconds&lt;/th&gt;
      &lt;th&gt;MBs/s&lt;/th&gt;
      &lt;th&gt;Msgs/s&lt;/th&gt;
    &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt;confluent_kafka_producer&lt;/th&gt;
      &lt;td&gt;5.4&lt;/td&gt;
      &lt;td&gt;17&lt;/td&gt;
      &lt;td&gt;183000&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;pykafka_producer_rdkafka&lt;/th&gt;
      &lt;td&gt;16&lt;/td&gt;
      &lt;td&gt;6.1&lt;/td&gt;
      &lt;td&gt;64000&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;pykafka_producer&lt;/th&gt;
      &lt;td&gt;57&lt;/td&gt;
      &lt;td&gt;1.7&lt;/td&gt;
      &lt;td&gt;17000&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;python_kafka_producer&lt;/th&gt;
      &lt;td&gt;68&lt;/td&gt;
      &lt;td&gt;1.4&lt;/td&gt;
      &lt;td&gt;15000&lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;/section&gt;
&lt;section id="consumer-throughput"&gt;
&lt;h2&gt;Consumer Throughput&lt;/h2&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
    &lt;tr style="text-align: right;"&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;time_in_seconds&lt;/th&gt;
      &lt;th&gt;MBs/s&lt;/th&gt;
      &lt;th&gt;Msgs/s&lt;/th&gt;
    &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt;confluent_kafka_consumer&lt;/th&gt;
      &lt;td&gt;3.8&lt;/td&gt;
      &lt;td&gt;25&lt;/td&gt;
      &lt;td&gt;261000&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;pykafka_consumer_rdkafka&lt;/th&gt;
      &lt;td&gt;6.1&lt;/td&gt;
      &lt;td&gt;17&lt;/td&gt;
      &lt;td&gt;164000&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;pykafka_consumer&lt;/th&gt;
      &lt;td&gt;29&lt;/td&gt;
      &lt;td&gt;3.2&lt;/td&gt;
      &lt;td&gt;34000&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;python_kafka_consumer&lt;/th&gt;
      &lt;td&gt;26&lt;/td&gt;
      &lt;td&gt;3.6&lt;/td&gt;
      &lt;td&gt;38000&lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;&lt;em&gt;Note: I discovered this article on &lt;a class="reference external" href="https://github.com/Parsely/pykafka/issues/559"&gt;parsely/pykafka #559&lt;/a&gt;, which has good conversation about the three libraries.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;I profiled PyKafka in these cases and it doesn’t appear that these code paths
have yet been optimized. I expect that modest effort could close that gap
considerably. This difference seems to be more from lack of interest than any
hard design constraint.&lt;/p&gt;
&lt;p&gt;It’s not clear how critical these speeds are. According to the PyKafka
maintainers at Parse.ly they haven’t actually turned on the librdkafka
optimizations in their internal pipelines, and are instead using the slow
Pure Python implementation, which is apparently more than fast enough for
common use. Getting messages out of Kafka just isn’t their bottleneck. It may
be that these 250,000 messages/sec limits are not significant in most
applications. I suspect that this matters more in bulk analysis workloads than
in online 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/2017/10/10/kafka-python.md&lt;/span&gt;, line 179)&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="pythonic-vs-java-apis"&gt;
&lt;h1&gt;Pythonic vs Java APIs&lt;/h1&gt;
&lt;p&gt;It took me a few times to get confluent-kafka to work. It wasn’t clear what
information I needed to pass to the constructor to connect to Kafka and when I
gave the wrong information I received no message that I had done anything
incorrectly. Docstrings and documentation were both minimal. In contrast,
PyKafka’s API and error messages quickly led me to correct behavior and I was
up and running within a minute.&lt;/p&gt;
&lt;p&gt;However, I persisted with confluent-kafka, found the right &lt;a class="reference external" href="https://kafka.apache.org/documentation/#api"&gt;Java
documentation&lt;/a&gt;, and eventually did
get things up and running. Once this happened everything fell into place and I
was able to easily build applications with Confluent-kafka that were both
simple and fast.&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/2017/10/10/kafka-python.md&lt;/span&gt;, line 194)&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="development-experience"&gt;
&lt;h1&gt;Development experience&lt;/h1&gt;
&lt;p&gt;I would like to add asynchronous support to one or both of these libraries so
that they can read or write data in a non-blocking fashion and play nicely with
other asynchronous systems like Tornado or Asyncio. I started investigating
this with both libraries on GitHub.&lt;/p&gt;
&lt;section id="developers"&gt;
&lt;h2&gt;Developers&lt;/h2&gt;
&lt;p&gt;Both libraries have a maintainer who is somewhat responsive and whose time is
funded by the parent company. Both maintainers seem active on a day-to-day
basis and handle contributions from external developers.&lt;/p&gt;
&lt;p&gt;Both libraries are fully active with a common pattern of a single main dev
merging work from a number of less active developers. Distributions of commits
over the last six months look similar:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;confluent-kafka-python$ git shortlog -ns --since &amp;quot;six months ago&amp;quot;
38  Magnus Edenhill
5  Christos Trochalakis
4  Ewen Cheslack-Postava
1  Simon Wahlgren

pykafka$ git shortlog -ns --since &amp;quot;six months ago&amp;quot;
52  Emmett Butler
23  Emmett J. Butler
20  Marc-Antoine Parent
18  Tanay Soni
5  messense
1  Erik Stephens
1  Jeff Widman
1  Prateek Shrivastava
1  aleatha
1  zpcui
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="codebase"&gt;
&lt;h2&gt;Codebase&lt;/h2&gt;
&lt;p&gt;In regards to the codebases I found that PyKafka was easier to hack on for a
few reasons:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Most of PyKafka is written in Python rather than C extensions, and so it is
more accessible to a broader development base. I find that Python C
extensions are not pleasant to work with, even if you are comfortable with
C.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;PyKafka appears to be much more extensively tested. PyKafka actually spins
up a local Kafka instance to do comprehensive integration tests while
Confluent-kafka seems to only test API without actually running against a
real Kakfa instance.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For what it’s worth, PyKafka maintainers &lt;a class="reference external" href="https://github.com/Parsely/pykafka/issues/731"&gt;responded
quickly&lt;/a&gt; to an issue on
Tornado. Confluent-kafka maintainers still have not responded to a
&lt;a class="reference external" href="https://github.com/confluentinc/confluent-kafka-python/issues/100#issuecomment-334152182"&gt;comment on an existing Tornado
issue&lt;/a&gt;,
even though that comment had signfiicnatly more content (a working
prototype).&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;em&gt;To be clear, no maintainer has any responsibility to answer my questions on
github. They are likely busy with other things that are of more relevance to
their particular mandate.&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/2017/10/10/kafka-python.md&lt;/span&gt;, line 256)&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="conda-packages"&gt;
&lt;h1&gt;Conda packages&lt;/h1&gt;
&lt;p&gt;I’ve pushed/updated recipes for both packages on conda-forge. You can install
them as follows:&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 -c conda-forge pykafka                 # Linux, Mac, Windows
conda install -c conda-forge python-confluent-kafka  # Linux, Mac
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In both cases this these are built against the fast &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;librdkafka&lt;/span&gt;&lt;/code&gt; C library
(except on Windows) and install that library as well.&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/2017/10/10/kafka-python.md&lt;/span&gt;, line 267)&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-plans"&gt;
&lt;h1&gt;Future plans&lt;/h1&gt;
&lt;p&gt;I’ve recently started work on streaming systems and pipelines for
&lt;a class="reference external" href="http://dask.pydata.org/en/latest/"&gt;Dask&lt;/a&gt;, so I’ll probably continue to
investigate this space. I’m still torn between the two implementations. There
are strong reasons to use either of them.&lt;/p&gt;
&lt;p&gt;Culturally I am drawn to Parse.ly’s PyKafka library. They’re clearly Python
developers writing for Python users. However the costs of using a non-Pythonic
system here just aren’t that large (Kafka’s API is small), and Confluent’s
interests are more aligned with investing in Kafka long term than are
Parse.ly’s.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/10/10/kafka-python/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2017-10-10T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/09/24/dask-0.15.3/</id>
    <title>Dask Release 0.15.3</title>
    <updated>2017-09-24T00: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://anaconda.com"&gt;Anaconda Inc.&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;I’m pleased to announce the release of Dask version 0.15.3. This release
contains stability enhancements and bug fixes. This blogpost outlines
notable changes since the 0.15.2 release on August 30th.&lt;/p&gt;
&lt;p&gt;You can conda install Dask:&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 -c conda-forge dask
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or pip install from PyPI&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[complete] --upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Conda packages are available both on conda-forge channels. They will be on
defaults in a few days.&lt;/p&gt;
&lt;p&gt;Full changelogs are available here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/blob/master/docs/source/changelog.rst"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/docs/source/changelog.rst"&gt;dask/distributed&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Some notable changes follow.&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/2017/09/24/dask-0.15.3.md&lt;/span&gt;, line 34)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="masked-arrays"&gt;

&lt;p&gt;Dask.array now supports &lt;a class="reference external" href="https://docs.scipy.org/doc/numpy-1.13.0/reference/maskedarray.html"&gt;masked arrays similar to NumPy&lt;/a&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;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="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;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;10&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="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;mask&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="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="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;m&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;ma&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;masked_array&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;mask&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;m&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;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;masked_array&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;10&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="n"&gt;int64&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="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;&amp;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;m&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;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="n"&gt;masked_array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="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="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="mi"&gt;5&lt;/span&gt; &lt;span class="o"&gt;--&lt;/span&gt; &lt;span class="mi"&gt;7&lt;/span&gt; &lt;span class="o"&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;mask&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt; &lt;span class="kc"&gt;True&lt;/span&gt; &lt;span class="kc"&gt;False&lt;/span&gt;  &lt;span class="kc"&gt;True&lt;/span&gt; &lt;span class="kc"&gt;False&lt;/span&gt;  &lt;span class="kc"&gt;True&lt;/span&gt; &lt;span class="kc"&gt;False&lt;/span&gt;  &lt;span class="kc"&gt;True&lt;/span&gt; &lt;span class="kc"&gt;False&lt;/span&gt;  &lt;span class="kc"&gt;True&lt;/span&gt; &lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
       &lt;span class="n"&gt;fill_value&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;999999&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This work was primarily done by &lt;a class="reference external" href="https://jcristharif.com/"&gt;Jim Crist&lt;/a&gt; and partially funded by the UK
Met office in support of the &lt;a class="reference external" href="http://scitools.org.uk/iris/"&gt;Iris project&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/2017/09/24/dask-0.15.3.md&lt;/span&gt;, line 60)&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="constants-in-atop"&gt;
&lt;h1&gt;Constants in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;atop&lt;/span&gt;&lt;/code&gt;&lt;/h1&gt;
&lt;p&gt;Dask.array experts will be familiar with the &lt;a class="reference external" href="http://dask.pydata.org/en/latest/array-api.html#dask.array.core.atop"&gt;atop
function&lt;/a&gt;, which powers a non-trivial amount of dask.array and is commonly used by people building custom algorithms. This function now supports constants when the index given is &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;None&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;atop&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="s1"&gt;&amp;#39;ijk&amp;#39;&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="s1"&gt;&amp;#39;ik&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;kj&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;CONSTANT&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;None&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/2017/09/24/dask-0.15.3.md&lt;/span&gt;, line 69)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="memory-management-for-workers"&gt;
&lt;h1&gt;Memory management for workers&lt;/h1&gt;
&lt;p&gt;Dask workers spill excess data to disk when they reach 60% of their alloted
memory limit. Previously we only measured memory use by adding up the memory
use of every piece of data produce by the worker. This could fail under a few
situations&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Our per-data estiamtes were faulty&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;User code consumed a large amount of memory without our tracking it&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;To compensate we now also periodically check the memory use of the worker using
system utilities with the psutil module. We dump data to disk if the process
rises about 70% use, stop running new tasks if it rises above 80%, and restart
the worker if it rises above 95% (assuming that the worker has a nanny
process).&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Breaking Change:&lt;/strong&gt; Previously the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;--memory-limit&lt;/span&gt;&lt;/code&gt; keyword to the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-worker&lt;/span&gt;&lt;/code&gt; process specified the 60% “start pushing to disk” limit. So if
you had 100GB of RAM then you previously might have started a dask-worker as
follows:&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;dask-worker ... --memory-limit 60e9  # before specify 60% target
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And the worker would start pushing to disk once it had 60GB of data in memory.
However, now we are changing this meaning to be the full amount of memory given
to the process.&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;dask-worker ... --memory-limit 100e9A  # now specify 100% target
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Of course, you don’t have to sepcify this limit (many don’t). It will be
chosen for you automatically. If you’ve never cared about this then you
shouldn’t start caring now.&lt;/p&gt;
&lt;p&gt;More about memory management here: &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/worker.html?highlight=memory-limit#memory-management"&gt;http://distributed.readthedocs.io/en/latest/worker.html?highlight=memory-limit#memory-management&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/2017/09/24/dask-0.15.3.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="statistical-profiling"&gt;
&lt;h1&gt;Statistical Profiling&lt;/h1&gt;
&lt;p&gt;Workers now poll their worker threads every 10ms and keep a running count of
which functions are being used. This information is available on the
diagnostic dashboard as a new “Profile” page. It provides information that is
orthogonal, and generally more detailed than the typical task-stream plot.&lt;/p&gt;
&lt;img src="/images/daskboard-profile.gif" width="70%"&gt;
&lt;p&gt;These plots are available on each worker, and an aggregated view is available
on the scheduler. The timeseries on the bottom allows you to select time
windows of your computation to restrict the parallel profile.&lt;/p&gt;
&lt;p&gt;More information about diagnosing performance available here:
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/diagnosing-performance.html"&gt;http://distributed.readthedocs.io/en/latest/diagnosing-performance.html&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/2017/09/24/dask-0.15.3.md&lt;/span&gt;, line 120)&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;The following people contributed to the dask/dask repository since the 0.15.2
release on August 30th&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Adonis&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Christopher Prohm&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Danilo Horta&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;jakirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jon Mease&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;jschendel&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Keisuke Fujii&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Will Warner&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The following people contributed to the dask/distributed repository since the
1.18.3 release on September 2nd:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Casey Law&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Edrian Irizarry&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;rbubley&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;ywangd&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/09/24/dask-0.15.3/"/>
    <summary>This work is supported by Anaconda Inc.
and the Data Driven Discovery Initiative from the Moore
Foundation.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2017-09-24T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/09/21/accelerating-geopandas-1/</id>
    <title>Fast GeoSpatial Analysis in Python</title>
    <updated>2017-09-21T00:00:00+00:00</updated>
    <content type="html">&lt;link href="https://cdn.pydata.org/bokeh/release/bokeh-0.12.9.min.css"
      rel="stylesheet" type="text/css"&gt;
&lt;link href="https://cdn.pydata.org/bokeh/release/bokeh-widgets-0.12.9.min.css"
      rel="stylesheet" type="text/css"&gt;
&lt;script src="https://cdn.pydata.org/bokeh/release/bokeh-0.12.9.min.js"&gt;&lt;/script&gt;
&lt;script src="https://cdn.pydata.org/bokeh/release/bokeh-widgets-0.12.9.min.js"&gt;&lt;/script&gt;
&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://anaconda.com"&gt;Anaconda Inc.&lt;/a&gt;, the Data
Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;, and &lt;a class="reference external" href="https://sbir.nasa.gov/SBIR/abstracts/16/sbir/phase2/SBIR-16-2-S5.03-7927.html"&gt;NASA SBIR
NNX16CG43P&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;This work is a collaboration with &lt;a class="reference external" href="https://github.com/jorisvandenbossche/"&gt;Joris Van den Bossche&lt;/a&gt;. This blogpost builds on &lt;a class="reference external" href="https://www.youtube.com/watch?v=bWsA2R707BM"&gt;Joris’s EuroSciPy talk&lt;/a&gt; (&lt;a class="reference external" href="https://jorisvandenbossche.github.io/talks/2017_EuroScipy_geopandas/#1"&gt;slides&lt;/a&gt;) on the same topic. You can also see Joris’ &lt;a class="reference external" href="https://jorisvandenbossche.github.io/blog/2017/09/19/geopandas-cython/"&gt;blogpost on this same topic&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2017/09/21/accelerating-geopandas-1.md&lt;/span&gt;, line 23)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="tl-dr"&gt;

&lt;p&gt;Python’s Geospatial stack is slow. We accelerate the GeoPandas library with
Cython and Dask. Cython provides 10-100x speedups. Dask gives an additional
3-4x on a multi-core laptop. Everything is still rough, please come help.&lt;/p&gt;
&lt;p&gt;We start by reproducing a
&lt;a class="reference external" href="https://medium.com/towards-data-science/geospatial-operations-at-scale-with-dask-and-geopandas-4d92d00eb7e8"&gt;blogpost&lt;/a&gt;
published last June, but with 30x speedups. Then we talk about how we achieved
the speedup with Cython and Dask.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;All code in this post is experimental. It should not be relied upon.&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/2017/09/21/accelerating-geopandas-1.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="experiment"&gt;
&lt;h1&gt;Experiment&lt;/h1&gt;
&lt;p&gt;In June &lt;a class="reference external" href="http://people.earth.yale.edu/profile/ravi-shekhar/about"&gt;Ravi Shekhar&lt;/a&gt;
published a blogpost &lt;a class="reference external" href="https://medium.com/towards-data-science/geospatial-operations-at-scale-with-dask-and-geopandas-4d92d00eb7e8"&gt;Geospatial Operations at Scale with Dask and GeoPandas&lt;/a&gt;
in which he counted the number of rides originating from each of the official
taxi zones of New York City. He read, processed, and plotted 120 million
rides, performing an expensive point-in-polygon test for each ride, and produced a
figure much like the following:&lt;/p&gt;
&lt;div class="bk-root"&gt;&lt;div class="bk-plotdiv" id="deeab81b-7229-4831-8db6-ac0dd99e8ed5"&gt;&lt;/div&gt;&lt;/div&gt;
&lt;p&gt;This took about three hours on his laptop. He used Dask and a bit of custom
code to parallelize Geopandas across all of his cores. Using this combination he
got close to the speed of PostGIS, but from Python.&lt;/p&gt;
&lt;p&gt;Today, using an accelerated GeoPandas and a new dask-geopandas library, we can do
the above computation in around eight minutes (half of which is reading CSV
files) and so can produce a number of other interesting images with faster
interaction times.&lt;/p&gt;
&lt;p&gt;A full notebook producing these plots is available below:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://nbviewer.jupyter.org/urls/gist.githubusercontent.com/mrocklin/ba6d3e2376e478c344af7e874e6fcbb1/raw/e0db89644f78f4371ee30fbdd517ce9bd6032a5e/nyc-taxi-geospatial.ipynb"&gt;NYC Taxi GeoSpatial Analysis Notebook&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;div class="bk-root"&gt;&lt;div class="bk-plotdiv" id="95089bfe-ef2e-4c17-89f1-9681a14d8107"&gt;&lt;/div&gt;&lt;/div&gt;
&lt;div class="bk-root"&gt;&lt;div class="bk-plotdiv" id="ee11cb3f-5262-44a5-8438-9ecbad37cf86"&gt;&lt;/div&gt;&lt;/div&gt;
&lt;p&gt;The rest of this article talks about GeoPandas, Cython, and speeding up
geospatial 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/2017/09/21/accelerating-geopandas-1.md&lt;/span&gt;, line 67)&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="background-in-geospatial-data"&gt;
&lt;h1&gt;Background in Geospatial Data&lt;/h1&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://shapely.readthedocs.io/en/stable/manual.html"&gt;Shapely User Manual&lt;/a&gt; begins
with the following passage on the utility of geospatial analysis to our society.&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Deterministic spatial analysis is an important component of computational
approaches to problems in agriculture, ecology, epidemiology, sociology, and
many other fields. What is the surveyed perimeter/area ratio of these patches
of animal habitat? Which properties in this town intersect with the 50-year
flood contour from this new flooding model? What are the extents of findspots
for ancient ceramic wares with maker’s marks “A” and “B”, and where do the
extents overlap? What’s the path from home to office that best skirts
identified zones of location based spam? These are just a few of the possible
questions addressable using non-statistical spatial analysis, and more
specifically, computational geometry.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;Shapely is part of Python’s GeoSpatial stack which is currently composed of the
following libraries:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://toblerity.org/shapely/manual.html"&gt;Shapely&lt;/a&gt;:
Manages shapes like points, linestrings, and polygons.
Wraps the GEOS C++ library&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://toblerity.org/fiona/manual.html"&gt;Fiona&lt;/a&gt;:
Handles data ingestion. Wraps the GDAL library&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://mapbox.github.io/rasterio/"&gt;Rasterio&lt;/a&gt;:
Handles raster data like satelite imagery&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://geopandas.org/"&gt;GeoPandas&lt;/a&gt;:
Extends Pandas with a column of shapely geometries to
intuitively query tables of geospatially annotated data.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;These libraries provide intuitive Python wrappers around the OSGeo C/C++
libraries (GEOS, GDAL, …) which power virtually every open source geospatial
library, like PostGIS, QGIS, etc.. They provide the same functionality, but
are typically much slower due to how they use Python. This is acceptable for
small datasets, but becomes an issue as we transition to larger and larger
datasets.&lt;/p&gt;
&lt;p&gt;In this post we focus on GeoPandas, a geospatial extension of Pandas which
manages tabular data that is annotated with geometry information like points,
paths, and polygons.&lt;/p&gt;
&lt;section id="geopandas-example"&gt;
&lt;h2&gt;GeoPandas Example&lt;/h2&gt;
&lt;p&gt;GeoPandas makes it easy to load, manipulate, and plot geospatial data. For
example, we can download the &lt;a class="reference external" href="https://s3.amazonaws.com/nyc-tlc/misc/taxi_zones.zip"&gt;NYC taxi
zones&lt;/a&gt;, load and plot
them in a single line of code.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;geopandas&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_file&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;taxi_zones.shp&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;to_crs&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;init&amp;#39;&lt;/span&gt; &lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;epsg:4326&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;plot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;column&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;borough&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;categorical&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/nyc-taxi-zones.svg" width="50%"&gt;
&lt;p&gt;Cities are now doing a wonderful job publishing data into the open. This
provides transparency and an opportunity for civic involvement to help analyze,
understand, and improve our communities. Here are a few fun geospatially-aware
datasets to make you interested:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://data.cityofchicago.org/Public-Safety/Crimes-2001-to-present/ijzp-q8t2"&gt;Chicago Crimes from 2001 to present (one week ago)&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://opendata.paris.fr/explore/dataset/stations-velib-disponibilites-en-temps-reel/export/"&gt;Paris Velib (bikeshare) in real time&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://portal-nolagis.opendata.arcgis.com/datasets/bike-lanes"&gt;Bike lanes in New Orleans&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://data.nola.gov/Public-Safety-and-Preparedness/NOPD-Use-of-Force-Incidents/9mnw-mbde"&gt;New Orleans Police Department incidents involving the use of force&lt;/a&gt;&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/2017/09/21/accelerating-geopandas-1.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&gt;
&lt;section id="performance"&gt;
&lt;h1&gt;Performance&lt;/h1&gt;
&lt;p&gt;Unfortunately GeoPandas is slow. This limits interactive exploration on larger
datasets. For example the Chicago crimes data (the first dataset above) has
seven million entries and is several gigabytes in memory. Analyzing a dataset
of this size interactively with GeoPandas is not feasible today.&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/geopandas-shapely-1.svg"
     width="50%"
     align="right"&gt;&lt;/p&gt;
&lt;p&gt;This slowdown is because GeoPandas wraps each geometry (like a point, line, or
polygon) with a Shapely object and stores all of those objects in an
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;object&lt;/span&gt;&lt;/code&gt;-dtype column. When we compute a GeoPandas operation on all of our
shapes we just iterate over these shapes in Python. As an example, here is how
one might implement a distance method in GeoPandas today.&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;distance&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;other&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="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;geom&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;distance&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;other&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;geom&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;geometry&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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Series&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Unfortunately this just iterates over elements in the series, each of which is
an individual Shapely object. This is inefficient for two reasons:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Iterating through Python objects is slow relative to iterating through those same objects in C.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Shapely Python objects consume more memory than the GEOS Geometry objects that they wrap.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This results in slow 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/2017/09/21/accelerating-geopandas-1.md&lt;/span&gt;, line 165)&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="cythonizing-geopandas"&gt;
&lt;h1&gt;Cythonizing GeoPandas&lt;/h1&gt;
&lt;p&gt;Fortunately, we’ve rewritten GeoPandas with Cython to directly loop over the
underlying GEOS pointers. This provides a 10-100x speedup depending on the
operation.
So instead of using a Pandas &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;object&lt;/span&gt;&lt;/code&gt;-dtype column that &lt;em&gt;holds shapely objects&lt;/em&gt;
we instead store a NumPy array of &lt;em&gt;direct pointers to the GEOS objects&lt;/em&gt;.&lt;/p&gt;
&lt;section id="before"&gt;
&lt;h2&gt;Before&lt;/h2&gt;
&lt;p&gt;&lt;img src="/images/geopandas-shapely-1.svg"
     width="49%"&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="after"&gt;
&lt;h2&gt;After&lt;/h2&gt;
&lt;p&gt;&lt;img src="/images/geopandas-shapely-2.svg"
     width="49%"&gt;&lt;/p&gt;
&lt;p&gt;As an example, our function for distance now looks like the following Cython
implementation (some liberties taken for brevity):&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;cpdef&lt;/span&gt; &lt;span class="n"&gt;distance&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;other&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;cdef&lt;/span&gt; &lt;span class="nb"&gt;int&lt;/span&gt; &lt;span class="n"&gt;n&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;size&lt;/span&gt;
    &lt;span class="n"&gt;cdef&lt;/span&gt; &lt;span class="n"&gt;GEOSGeometry&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;left_geom&lt;/span&gt;
    &lt;span class="n"&gt;cdef&lt;/span&gt; &lt;span class="n"&gt;GEOSGeometry&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;right_geom&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;other&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;__geom__&lt;/span&gt;  &lt;span class="c1"&gt;# a geometry pointer&lt;/span&gt;
    &lt;span class="n"&gt;geometries&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_geometry_array&lt;/span&gt;

    &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;nogil&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;idx&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;xrange&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
            &lt;span class="n"&gt;left_geom&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;GEOSGeometry&lt;/span&gt; &lt;span class="o"&gt;*&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;geometries&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;idx&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;left_geom&lt;/span&gt; &lt;span class="o"&gt;!=&lt;/span&gt; &lt;span class="n"&gt;NULL&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                &lt;span class="n"&gt;distance&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;GEOSDistance_r&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;left_geom&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;some_point&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;__geom&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="n"&gt;distance&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;NaN&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;For fast operations we see speedups of 100x. For slower operations we’re
closer to 10x. Now these operations run at full C speed.&lt;/p&gt;
&lt;p&gt;In his &lt;a class="reference external" href="https://www.youtube.com/watch?v=bWsA2R707BM"&gt;EuroSciPy
talk&lt;/a&gt; Joris compares the
performance of GeoPandas (both before and after Cython) with &lt;a class="reference external" href="http://postgis.net/"&gt;PostGIS&lt;/a&gt;, the standard geospatial plugin for the popular
PostgreSQL database (&lt;a class="reference external" href="https://github.com/jorisvandenbossche/talks/blob/master/2017_EuroScipy_geopandas/geopandas_postgis_comparison.ipynb"&gt;original
notebook&lt;/a&gt;
with the comparison). I’m stealing some plots from his talk below:&lt;/p&gt;
&lt;p&gt;&lt;a href="/images/timings_sjoin_all.png"&gt;&lt;img src="/images/timings_sjoin_all.png" width="33%"&gt;&lt;/a&gt;
&lt;a href="/images/geopandas-timings_distance2_all.png"&gt;&lt;img src="/images/geopandas-timings_distance2_all.png" width="33%"&gt;&lt;/a&gt;
&lt;a href="/images/geopandas-timings_within_all.png"&gt;&lt;img src="/images/geopandas-timings_within_all.png" width="31%"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Cythonized GeoPandas and PostGIS run at almost exactly the same speed. This is
because they use the same underlying C library, GEOS. These algorithms are not
particularly complex, so it is not surprising that everyone implements them
in exactly the same way.&lt;/p&gt;
&lt;p&gt;This is great. The Python GIS stack now has a full-speed library that operates
as fast as any other open GIS system is likely to manage.&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/2017/09/21/accelerating-geopandas-1.md&lt;/span&gt;, line 224)&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="problems"&gt;
&lt;h1&gt;Problems&lt;/h1&gt;
&lt;p&gt;However, this is still a work in progress, and there is still plenty of work
to do.&lt;/p&gt;
&lt;p&gt;First, we need for Pandas to track our arrays of GEOS pointers differently from
how it tracks a normal integer array. This is both for usability reasons, like
we want to render them differently and don’t want users to be able to perform
numeric operations like sum and mean on these arrays, and also for stability
reasons, because we need to track these pointers and release their allocated
GEOSGeometry objects from memory at the appropriate times. Currently, this
goal is pursued by creating a new block type, the GeometryBlock (‘blocks’ are
the internal building blocks of pandas that hold the data of the different columns).
This will require some changes to Pandas itself to enable custom block types
(see &lt;a class="reference external" href="https://github.com/pandas-dev/pandas/issues/17144"&gt;this issue&lt;/a&gt; on the pandas
issue tracker).&lt;/p&gt;
&lt;p&gt;Second, data ingestion is still quite slow. This relies not on GEOS, but on
GDAL/OGR, which is handled in Python today by Fiona. Fiona is more optimized
for consistency and usability rather than raw speed. Previously when GeoPandas
was slow this made sense because no one was operating on particularly large
datasets. However now we observe that data loading is often several times more
expensive than all of our manipulations so this will probably need some effort
in the future.&lt;/p&gt;
&lt;p&gt;Third, there are some algorithms within GeoPandas that we haven’t yet
Cythonized. This includes both particular features like overlay and dissolve
operations as well as small components like GeoJSON output.&lt;/p&gt;
&lt;p&gt;Finally as with any rewrite on a codebase that is not exhaustively tested
(we’re trying to improve testing as we do this) there are probably several bugs
that we won’t detect until some patient and forgiving user runs into them
first.&lt;/p&gt;
&lt;p&gt;Still though, all linear geospatial operations work well and are thoroughly
tested. Also spatial joins (a backbone of many geospatial operations) are up
and running at full speed. If you work in a non-production environment then
Cythonized GeoPandas may be worth your time to investigate.&lt;/p&gt;
&lt;p&gt;You can track future progress on this effort at
&lt;a class="reference external" href="https://github.com/geopandas/geopandas/issues/473"&gt;geopandas/geopandas #473&lt;/a&gt;
which includes installation instructions.&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/2017/09/21/accelerating-geopandas-1.md&lt;/span&gt;, line 267)&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="parallelize-with-dask"&gt;
&lt;h1&gt;Parallelize with Dask&lt;/h1&gt;
&lt;p&gt;Cythonizing gives us speedups in the 10x-100x range. We use a single core as
effectively as is possible with the GEOS library. Now we move on to using
multiple cores in parallel. This gives us an extra 3-4x on a standard 4 core
laptop. We can also scale to clusters, though I’ll leave that for a future
blogpost.&lt;/p&gt;
&lt;p&gt;To parallelize we need to split apart our dataset into multiple chunks. We can
do this naively by placing the first million rows in one chunk, the second
million rows in another chunk, etc. or we can partition our data spatially,
for example by placing all of the data for one region of our dataset in one
chunk and all of the data for another region in another chunk, and so on.
Both approaches are implemented in a rudimentary
&lt;a class="reference external" href="https://github.com/mrocklin/dask-geopandas"&gt;dask-geopandas&lt;/a&gt; library
available on GitHub.&lt;/p&gt;
&lt;p&gt;So just as dask-array organizes many NumPy arrays along a grid
and dask-dataframe organizes many Pandas dataframes along a linear index&lt;/p&gt;
&lt;img src="/images/dask-array-black-text.svg" width="60%"&gt;
&lt;img src="/images/dask-dataframe.svg" width="30%"&gt;
&lt;p&gt;the dask-geopandas library organizes many GeoPandas dataframes into spatial
regions. In the example below we might partition data in the city of New York
into its different boroughs. Data for each borough would be handled
separately by a different thread or, in a distributed situation, might live on
a different machine.&lt;/p&gt;
&lt;img src="/images/nyc-boroughs.svg" width="50%" align="right"&gt;
&lt;p&gt;This gives us two advantages:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Even without geospatial partitioning, we can use many cores (or many
machines) to accelerate simple operations.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For spatially aware operations, like spatial joins or subselections we can
engage only those parts of the parallel dataframe that we know are relevant
for various parts of the computation.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;However this is also expensive and not always necessary. In our initial
exercise with the NYC Taxi data we didn’t do this, and will still got
significant speedups just from normal multicore operation.&lt;/p&gt;
&lt;section id="exercise"&gt;
&lt;h2&gt;Exercise&lt;/h2&gt;
&lt;p&gt;And so to produce the images we did at the top of this post we used a
combination of dask.dataframe to load in CSV files, dask-geopandas to perform
the spatial join, and then dask.dataframe and normal pandas to perform the
actual computations. Our code looked something 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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_geopandas&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;dg&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;dd&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;yellow_tripdata_2015-*.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;gf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dg&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_geometry&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;geometry&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;pickup_longitude&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;pickup_latitude&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]],&lt;/span&gt;
                     &lt;span class="n"&gt;crs&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;init&amp;#39;&lt;/span&gt; &lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;epsg:4326&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;})&lt;/span&gt;
&lt;span class="n"&gt;gf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dg&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sjoin&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;gf&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;zones&lt;/span&gt;&lt;span class="p"&gt;[[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;zone&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;borough&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;geometry&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]])&lt;/span&gt;
&lt;span class="n"&gt;full&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;gf&lt;/span&gt;&lt;span class="p"&gt;[[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;zone&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;payment_type&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;tip_amount&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;fare_amount&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]]&lt;/span&gt;

&lt;span class="n"&gt;full&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;nyc-zones.parquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# compute and cache result on disk&lt;/span&gt;
&lt;span class="n"&gt;full&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;nyc-zones.parquet&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;And then we can do typical groupbys and joins on the more typical pandas-like
data now properly annotated with zones.&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;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;full&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;passenger_count&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;full&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;zone&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;compute&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;name&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;count&amp;#39;&lt;/span&gt;
&lt;span class="n"&gt;joined&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;merge&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;to_frame&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;zones&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                  &lt;span class="n"&gt;left_index&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;right_on&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;zone&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;joined&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;geopandas&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;GeoDataFrame&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;joined&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# convert back for plotting&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We’ve replaced most of Ravi’s custom analysis with a few lines of new standard
code. This maxes our or CPU when doing spatial joins. Everything here
releases the GIL well and the entire computation operates in under a couple
gigabytes of RAM.&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/2017/09/21/accelerating-geopandas-1.md&lt;/span&gt;, line 347)&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="id1"&gt;
&lt;h1&gt;Problems&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/2017/09/21/accelerating-geopandas-1.md&lt;/span&gt;, line 347); &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: “problems”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://github.com/mrocklin/dask-geopandas"&gt;dask-geopandas&lt;/a&gt; project is
currently a prototype. It will easily break for non-trivial applications (and
indeed many trivial ones). It was designed to see how hard it would be to
implement some of the trickier operations like spatial joins, repartitioning,
and overlays. This is why, for example, it supports a fully distributed
spatial join, but lacks simple operations like indexing. There are
other longer-term issues as well.&lt;/p&gt;
&lt;p&gt;Serialization costs are manageable, but decently high. We currently use the
standard “well known binary” WKB format common in other geospatial applications
but have found it to be fairly slow, which bogs down inter-process parallelism.&lt;/p&gt;
&lt;p&gt;Similarly distributed and spatially partitioned data stores don’t seem to be
common (or at least I haven’t run across them yet).&lt;/p&gt;
&lt;p&gt;It’s not clear how dask-geopandas dataframes and normal dask dataframes should
interact. It would be very convenient to reuse all of the algorithms in
dask.dataframe, but the index structures of the two libraries is very
different. This may require some clever software engineering on the part of
the Dask developers.&lt;/p&gt;
&lt;p&gt;Still though, these seem surmountable and generally this process has been easy
so far. I suspect that we can build an intuitive and performant parallel GIS
analytics system with modest effort.&lt;/p&gt;
&lt;p&gt;The notebook for the example at the start of the blogpost shows using
dask-geopandas with good results.&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/2017/09/21/accelerating-geopandas-1.md&lt;/span&gt;, line 377)&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;With established technologies in the PyData space like Cython and Dask we’ve
been able to accelerate and scale GeoPandas operations above and beyond
industry standards. However this work is still experimental and not ready for
production use. This work is a bit of a side project for both Joris and
Matthew and they would welcome effort from other experienced open source
developers. We believe that this project can have a large social impact and
are enthusiastic about pursuing it in the future. We hope that you share our
enthusiasm.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://nbviewer.jupyter.org/urls/gist.githubusercontent.com/mrocklin/ba6d3e2376e478c344af7e874e6fcbb1/raw/e0db89644f78f4371ee30fbdd517ce9bd6032a5e/nyc-taxi-geospatial.ipynb"&gt;NYC Taxi GeoSpatial Analysis Notebook&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://www.youtube.com/watch?v=bWsA2R707BM"&gt;Joris’s EuroSciPy talk&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://jorisvandenbossche.github.io/blog/2017/09/19/geopandas-cython/"&gt;Joris’s blogpost&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ravi’s original post &lt;a class="reference external" href="https://medium.com/towards-data-science/geospatial-operations-at-scale-with-dask-and-geopandas-4d92d00eb7e8"&gt;Geospatial Operations at Scale with Dask and GeoPandas&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;script type="text/javascript"&gt;
    (function() {
  var fn = function() {
    Bokeh.safely(function() {
      (function(root) {
        function embed_document(root) {
          var docs_json = {"3eecbd1c-0b14-4777-8f07-6537dfa40e4c":{"roots":{"references":[{"attributes":{"plot":null,"text":"Average Tip Fraction"},"id":"27996c4c-da44-4dda-b78a-1aeaac693387","type":"Title"},{"attributes":{},"id":"a0586146-b37e-490f-a532-dc621364d57f","type":"SaveTool"},{"attributes":{},"id":"dddfd9bf-e505-4455-a357-7daffd14f49d","type":"BasicTickFormatter"},{"attributes":{},"id":"085f3735-0ce5-463d-99a2-c12f409f4b0e","type":"HelpTool"},{"attributes":{"source":{"id":"79833421-4b37-4874-b257-e91f6932982f","type":"GeoJSONDataSource"}},"id":"eb918a72-48fd-4453-8b79-b2032735ae5a","type":"CDSView"},{"attributes":{},"id":"6f5fe3f1-e966-4fa5-aaac-66841af5b809","type":"BasicTicker"},{"attributes":{},"id":"6960645d-737b-442b-a2f0-cdb7e93d4e91","type":"PanTool"},{"attributes":{},"id":"c421e4d7-01a7-43de-a95e-fdff91e61d6f","type":"BasicTicker"},{"attributes":{"plot":{"id":"9a6525f3-7059-45ab-90ee-2d7e83c0a07a","subtype":"Figure","type":"Plot"},"ticker":{"id":"c421e4d7-01a7-43de-a95e-fdff91e61d6f","type":"BasicTicker"},"visible":false},"id":"e8cb9eb4-de5c-457e-ab86-dc3772cc3f40","type":"Grid"},{"attributes":{},"id":"d301bef6-61e1-4d2e-85d6-2a8e369373b7","type":"SaveTool"},{"attributes":{"overlay":{"id":"b63fb838-d1f0-4417-b06a-2253fff02424","type":"BoxAnnotation"}},"id":"ee3c44cb-3c2f-4de1-82d6-8a8ccf913021","type":"BoxZoomTool"},{"attributes":{"color_mapper":{"id":"b661dcd9-576f-4c79-90d3-65f41d14b544","type":"LogColorMapper"},"formatter":{"id":"25d38d1a-e1d0-4ee9-9a51-324a7ddfa9d8","type":"BasicTickFormatter"},"label_standoff":12,"location":[0,0],"plot":{"id":"3058396c-651f-4f74-b4de-a7e737c8665a","subtype":"Figure","type":"Plot"},"ticker":{"id":"fde2ffef-37b2-487c-8cd3-049402dc2fab","type":"BasicTicker"}},"id":"3f99e069-bc47-41e6-9ed0-a9e0856d105c","type":"ColorBar"},{"attributes":{},"id":"5150062a-68b9-4228-a3ca-5a2795f8ae24","type":"WheelZoomTool"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"xs":{"field":"xs"},"ys":{"field":"ys"}},"id":"4529a4e7-b57a-4b05-8697-4ffefc419c71","type":"Patches"},{"attributes":{},"id":"25d38d1a-e1d0-4ee9-9a51-324a7ddfa9d8","type":"BasicTickFormatter"},{"attributes":{},"id":"7f3fe531-7586-4d07-b55d-917e998d17dc","type":"ResetTool"},{"attributes":{},"id":"4863a02c-a14f-44ad-b059-4fa0a7811065","type":"BasicTickFormatter"},{"attributes":{"callback":null,"point_policy":"follow_mouse","tooltips":"&lt;div&gt;&lt;b&gt;Borough&lt;/b&gt;: @borough&lt;/div&gt;&lt;div&gt;&lt;b&gt;Zone&lt;/b&gt;: @zone&lt;/div&gt;&lt;div&gt;&lt;b&gt;Rides&lt;/b&gt;: @count&lt;/div&gt;"},"id":"bf29975f-fcdb-444f-ad6b-270f10ab2b06","type":"HoverTool"},{"attributes":{"dimension":1,"plot":{"id":"3058396c-651f-4f74-b4de-a7e737c8665a","subtype":"Figure","type":"Plot"},"ticker":{"id":"2a7eb668-a58d-46ed-9e0b-0f25d7fff4c5","type":"BasicTicker"},"visible":false},"id":"f396af5d-43b1-43d6-ad17-82a9e1272208","type":"Grid"},{"attributes":{"data_source":{"id":"79833421-4b37-4874-b257-e91f6932982f","type":"GeoJSONDataSource"},"glyph":{"id":"f0c3dc48-49ec-4cee-9b0a-73efd89e128a","type":"Patches"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"2fc94ead-d7e7-4b3b-99d4-3a1cd74e873f","type":"Patches"},"selection_glyph":null,"view":{"id":"eb918a72-48fd-4453-8b79-b2032735ae5a","type":"CDSView"}},"id":"3b1fa4e4-5bf6-4bf4-85d2-091d7ff789d5","type":"GlyphRenderer"},{"attributes":{"callback":null},"id":"f0c21fdb-3905-4866-bbf5-d191494f134c","type":"DataRange1d"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"b63fb838-d1f0-4417-b06a-2253fff02424","type":"BoxAnnotation"},{"attributes":{"data_source":{"id":"18193dc2-af18-430d-8541-849ff6cf74a5","type":"GeoJSONDataSource"},"glyph":{"id":"ae4388bc-abe2-4174-ba6b-6c017c8007fb","type":"Patches"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"faeedd3b-119f-4af2-9847-ca2652d1dbb7","type":"Patches"},"selection_glyph":null,"view":{"id":"4bcfec7d-c05c-448a-94b4-1d2866409a23","type":"CDSView"}},"id":"698fbc56-a4d3-478b-bd22-f23784c8fbf5","type":"GlyphRenderer"},{"attributes":{"overlay":{"id":"be3cc134-22fc-4afb-bd0a-a527d820071d","type":"BoxAnnotation"}},"id":"4c82bbf0-af1f-4b96-b5c7-19931b678383","type":"BoxZoomTool"},{"attributes":{},"id":"e384a837-7ce6-4ba0-8f0e-e4a8ffc64518","type":"BasicTickFormatter"},{"attributes":{},"id":"d9c616aa-dc6d-4459-af43-73444362b3b2","type":"HelpTool"},{"attributes":{"fill_alpha":{"value":0.9},"fill_color":{"field":"payment_type","transform":{"id":"fd673fd5-348f-4441-b09b-846332ca8ea1","type":"LinearColorMapper"}},"line_alpha":{"value":0.5},"xs":{"field":"xs"},"ys":{"field":"ys"}},"id":"950c7878-9415-4d3a-976f-fff4dd96ee66","type":"Patches"},{"attributes":{"palette":["#440154","#440255","#440357","#450558","#45065A","#45085B","#46095C","#460B5E","#460C5F","#460E61","#470F62","#471163","#471265","#471466","#471567","#471669","#47186A","#48196B","#481A6C","#481C6E","#481D6F","#481E70","#482071","#482172","#482273","#482374","#472575","#472676","#472777","#472878","#472A79","#472B7A","#472C7B","#462D7C","#462F7C","#46307D","#46317E","#45327F","#45347F","#453580","#453681","#443781","#443982","#433A83","#433B83","#433C84","#423D84","#423E85","#424085","#414186","#414286","#404387","#404487","#3F4587","#3F4788","#3E4888","#3E4989","#3D4A89","#3D4B89","#3D4C89","#3C4D8A","#3C4E8A","#3B508A","#3B518A","#3A528B","#3A538B","#39548B","#39558B","#38568B","#38578C","#37588C","#37598C","#365A8C","#365B8C","#355C8C","#355D8C","#345E8D","#345F8D","#33608D","#33618D","#32628D","#32638D","#31648D","#31658D","#31668D","#30678D","#30688D","#2F698D","#2F6A8D","#2E6B8E","#2E6C8E","#2E6D8E","#2D6E8E","#2D6F8E","#2C708E","#2C718E","#2C728E","#2B738E","#2B748E","#2A758E","#2A768E","#2A778E","#29788E","#29798E","#287A8E","#287A8E","#287B8E","#277C8E","#277D8E","#277E8E","#267F8E","#26808E","#26818E","#25828E","#25838D","#24848D","#24858D","#24868D","#23878D","#23888D","#23898D","#22898D","#228A8D","#228B8D","#218C8D","#218D8C","#218E8C","#208F8C","#20908C","#20918C","#1F928C","#1F938B","#1F948B","#1F958B","#1F968B","#1E978A","#1E988A","#1E998A","#1E998A","#1E9A89","#1E9B89","#1E9C89","#1E9D88","#1E9E88","#1E9F88","#1EA087","#1FA187","#1FA286","#1FA386","#20A485","#20A585","#21A685","#21A784","#22A784","#23A883","#23A982","#24AA82","#25AB81","#26AC81","#27AD80","#28AE7F","#29AF7F","#2AB07E","#2BB17D","#2CB17D","#2EB27C","#2FB37B","#30B47A","#32B57A","#33B679","#35B778","#36B877","#38B976","#39B976","#3BBA75","#3DBB74","#3EBC73","#40BD72","#42BE71","#44BE70","#45BF6F","#47C06E","#49C16D","#4BC26C","#4DC26B","#4FC369","#51C468","#53C567","#55C666","#57C665","#59C764","#5BC862","#5EC961","#60C960","#62CA5F","#64CB5D","#67CC5C","#69CC5B","#6BCD59","#6DCE58","#70CE56","#72CF55","#74D054","#77D052","#79D151","#7CD24F","#7ED24E","#81D34C","#83D34B","#86D449","#88D547","#8BD546","#8DD644","#90D643","#92D741","#95D73F","#97D83E","#9AD83C","#9DD93A","#9FD938","#A2DA37","#A5DA35","#A7DB33","#AADB32","#ADDC30","#AFDC2E","#B2DD2C","#B5DD2B","#B7DD29","#BADE27","#BDDE26","#BFDF24","#C2DF22","#C5DF21","#C7E01F","#CAE01E","#CDE01D","#CFE11C","#D2E11B","#D4E11A","#D7E219","#DAE218","#DCE218","#DFE318","#E1E318","#E4E318","#E7E419","#E9E419","#ECE41A","#EEE51B","#F1E51C","#F3E51E","#F6E61F","#F8E621","#FAE622","#FDE724"]},"id":"fe7eac4c-a6ac-449b-a2da-89f027e5010b","type":"LogColorMapper"},{"attributes":{"color_mapper":{"id":"fd673fd5-348f-4441-b09b-846332ca8ea1","type":"LinearColorMapper"},"formatter":{"id":"a5f4691a-c99e-4675-9f31-5445af64b13e","type":"BasicTickFormatter"},"label_standoff":12,"location":[0,0],"plot":{"id":"9a6525f3-7059-45ab-90ee-2d7e83c0a07a","subtype":"Figure","type":"Plot"},"ticker":{"id":"4592c80c-76be-46be-829c-55d8de2919cc","type":"BasicTicker"}},"id":"10980cdc-edaa-4fec-9488-26914877b608","type":"ColorBar"},{"attributes":{},"id":"f7b30ba4-fd79-4d13-a3c7-49803d251345","type":"ResetTool"},{"attributes":{},"id":"f6f9dda6-1d51-4e39-80d2-22bb8a3e4a7d","type":"WheelZoomTool"},{"attributes":{"formatter":{"id":"dddfd9bf-e505-4455-a357-7daffd14f49d","type":"BasicTickFormatter"},"plot":{"id":"9a6525f3-7059-45ab-90ee-2d7e83c0a07a","subtype":"Figure","type":"Plot"},"ticker":{"id":"6f5fe3f1-e966-4fa5-aaac-66841af5b809","type":"BasicTicker"},"visible":false},"id":"a5d2472e-792f-4f8c-ae2a-7820e4e4cabb","type":"LinearAxis"},{"attributes":{},"id":"4a794f89-97c4-4344-b538-0ef0498b3219","type":"SaveTool"},{"attributes":{},"id":"0d27ecfc-706b-4857-9f4b-080d014b15e4","type":"ResetTool"},{"attributes":{"callback":null,"point_policy":"follow_mouse","tooltips":"&lt;p&gt;&lt;b&gt;Borough&lt;/b&gt;: @borough&lt;/p&gt;&lt;p&gt;&lt;b&gt;Zone&lt;/b&gt;: @zone&lt;/p&gt;"},"id":"a84bfab3-9687-4112-8322-4e604f5954f2","type":"HoverTool"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"08b5d6f0-3cea-4e7d-a11b-4a4cd2eab30a","type":"BoxAnnotation"},{"attributes":{},"id":"2a7eb668-a58d-46ed-9e0b-0f25d7fff4c5","type":"BasicTicker"},{"attributes":{},"id":"4592c80c-76be-46be-829c-55d8de2919cc","type":"BasicTicker"},{"attributes":{},"id":"6351fc86-4cb0-48f1-906d-40c4646bf983","type":"PanTool"},{"attributes":{"data_source":{"id":"10ea5377-ee43-4b43-9633-3c452e6ae727","type":"GeoJSONDataSource"},"glyph":{"id":"950c7878-9415-4d3a-976f-fff4dd96ee66","type":"Patches"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"4529a4e7-b57a-4b05-8697-4ffefc419c71","type":"Patches"},"selection_glyph":null,"view":{"id":"8c9a62d6-f494-487d-8914-6e73f40e3b72","type":"CDSView"}},"id":"6492638d-6d80-49f2-b7a9-94a19d3ef2d9","type":"GlyphRenderer"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"be3cc134-22fc-4afb-bd0a-a527d820071d","type":"BoxAnnotation"},{"attributes":{"fill_alpha":{"value":0.9},"fill_color":{"field":"borough","transform":{"id":"6f4f819e-7190-454c-a7a3-ba5d768d5c31","type":"CategoricalColorMapper"}},"line_alpha":{"value":0.9},"line_color":{"field":"borough","transform":{"id":"6f4f819e-7190-454c-a7a3-ba5d768d5c31","type":"CategoricalColorMapper"}},"xs":{"field":"xs"},"ys":{"field":"ys"}},"id":"ae4388bc-abe2-4174-ba6b-6c017c8007fb","type":"Patches"},{"attributes":{},"id":"ed93bc07-ab38-4459-adf7-0bce0b3fe3b5","type":"PanTool"},{"attributes":{},"id":"a5f4691a-c99e-4675-9f31-5445af64b13e","type":"BasicTickFormatter"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"xs":{"field":"xs"},"ys":{"field":"ys"}},"id":"2fc94ead-d7e7-4b3b-99d4-3a1cd74e873f","type":"Patches"},{"attributes":{"source":{"id":"10ea5377-ee43-4b43-9633-3c452e6ae727","type":"GeoJSONDataSource"}},"id":"8c9a62d6-f494-487d-8914-6e73f40e3b72","type":"CDSView"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"xs":{"field":"xs"},"ys":{"field":"ys"}},"id":"faeedd3b-119f-4af2-9847-ca2652d1dbb7","type":"Patches"},{"attributes":{},"id":"fde2ffef-37b2-487c-8cd3-049402dc2fab","type":"BasicTicker"},{"attributes":{"dimension":1,"plot":{"id":"9a6525f3-7059-45ab-90ee-2d7e83c0a07a","subtype":"Figure","type":"Plot"},"ticker":{"id":"6f5fe3f1-e966-4fa5-aaac-66841af5b809","type":"BasicTicker"},"visible":false},"id":"da23e3ce-8181-4da5-8600-c4ddd6806ef2","type":"Grid"},{"attributes":{"fill_alpha":{"value":0.9},"fill_color":{"field":"tip_fraction","transform":{"id":"b661dcd9-576f-4c79-90d3-65f41d14b544","type":"LogColorMapper"}},"line_alpha":{"value":0.5},"xs":{"field":"xs"},"ys":{"field":"ys"}},"id":"f0c3dc48-49ec-4cee-9b0a-73efd89e128a","type":"Patches"},{"attributes":{"below":[{"id":"9139b6c7-121e-4d5b-9b54-caeceab315d1","type":"LinearAxis"}],"left":[{"id":"a5d2472e-792f-4f8c-ae2a-7820e4e4cabb","type":"LinearAxis"}],"renderers":[{"id":"9139b6c7-121e-4d5b-9b54-caeceab315d1","type":"LinearAxis"},{"id":"e8cb9eb4-de5c-457e-ab86-dc3772cc3f40","type":"Grid"},{"id":"a5d2472e-792f-4f8c-ae2a-7820e4e4cabb","type":"LinearAxis"},{"id":"da23e3ce-8181-4da5-8600-c4ddd6806ef2","type":"Grid"},{"id":"08b5d6f0-3cea-4e7d-a11b-4a4cd2eab30a","type":"BoxAnnotation"},{"id":"6492638d-6d80-49f2-b7a9-94a19d3ef2d9","type":"GlyphRenderer"},{"id":"10980cdc-edaa-4fec-9488-26914877b608","type":"ColorBar"}],"right":[{"id":"10980cdc-edaa-4fec-9488-26914877b608","type":"ColorBar"}],"title":{"id":"94dcf2d1-8432-4868-9512-d05dfca60e65","type":"Title"},"toolbar":{"id":"61f82afe-2e85-4d6b-8872-1435a20bc13a","type":"Toolbar"},"x_range":{"id":"169e471d-7586-4e4b-82ae-deea3e05ec36","type":"DataRange1d"},"x_scale":{"id":"62e9be12-1576-4008-82b6-0a61f5cf77ea","type":"LinearScale"},"y_range":{"id":"dc69c649-db35-4baa-b925-35cf40cf46ef","type":"DataRange1d"},"y_scale":{"id":"7077852c-6855-4480-8785-509b9fbeb00e","type":"LinearScale"}},"id":"9a6525f3-7059-45ab-90ee-2d7e83c0a07a","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"2693a195-e9cc-48e0-b8c7-94ab6f8d0cf7","type":"HelpTool"},{"attributes":{"palette":["#440154","#440255","#440357","#450558","#45065A","#45085B","#46095C","#460B5E","#460C5F","#460E61","#470F62","#471163","#471265","#471466","#471567","#471669","#47186A","#48196B","#481A6C","#481C6E","#481D6F","#481E70","#482071","#482172","#482273","#482374","#472575","#472676","#472777","#472878","#472A79","#472B7A","#472C7B","#462D7C","#462F7C","#46307D","#46317E","#45327F","#45347F","#453580","#453681","#443781","#443982","#433A83","#433B83","#433C84","#423D84","#423E85","#424085","#414186","#414286","#404387","#404487","#3F4587","#3F4788","#3E4888","#3E4989","#3D4A89","#3D4B89","#3D4C89","#3C4D8A","#3C4E8A","#3B508A","#3B518A","#3A528B","#3A538B","#39548B","#39558B","#38568B","#38578C","#37588C","#37598C","#365A8C","#365B8C","#355C8C","#355D8C","#345E8D","#345F8D","#33608D","#33618D","#32628D","#32638D","#31648D","#31658D","#31668D","#30678D","#30688D","#2F698D","#2F6A8D","#2E6B8E","#2E6C8E","#2E6D8E","#2D6E8E","#2D6F8E","#2C708E","#2C718E","#2C728E","#2B738E","#2B748E","#2A758E","#2A768E","#2A778E","#29788E","#29798E","#287A8E","#287A8E","#287B8E","#277C8E","#277D8E","#277E8E","#267F8E","#26808E","#26818E","#25828E","#25838D","#24848D","#24858D","#24868D","#23878D","#23888D","#23898D","#22898D","#228A8D","#228B8D","#218C8D","#218D8C","#218E8C","#208F8C","#20908C","#20918C","#1F928C","#1F938B","#1F948B","#1F958B","#1F968B","#1E978A","#1E988A","#1E998A","#1E998A","#1E9A89","#1E9B89","#1E9C89","#1E9D88","#1E9E88","#1E9F88","#1EA087","#1FA187","#1FA286","#1FA386","#20A485","#20A585","#21A685","#21A784","#22A784","#23A883","#23A982","#24AA82","#25AB81","#26AC81","#27AD80","#28AE7F","#29AF7F","#2AB07E","#2BB17D","#2CB17D","#2EB27C","#2FB37B","#30B47A","#32B57A","#33B679","#35B778","#36B877","#38B976","#39B976","#3BBA75","#3DBB74","#3EBC73","#40BD72","#42BE71","#44BE70","#45BF6F","#47C06E","#49C16D","#4BC26C","#4DC26B","#4FC369","#51C468","#53C567","#55C666","#57C665","#59C764","#5BC862","#5EC961","#60C960","#62CA5F","#64CB5D","#67CC5C","#69CC5B","#6BCD59","#6DCE58","#70CE56","#72CF55","#74D054","#77D052","#79D151","#7CD24F","#7ED24E","#81D34C","#83D34B","#86D449","#88D547","#8BD546","#8DD644","#90D643","#92D741","#95D73F","#97D83E","#9AD83C","#9DD93A","#9FD938","#A2DA37","#A5DA35","#A7DB33","#AADB32","#ADDC30","#AFDC2E","#B2DD2C","#B5DD2B","#B7DD29","#BADE27","#BDDE26","#BFDF24","#C2DF22","#C5DF21","#C7E01F","#CAE01E","#CDE01D","#CFE11C","#D2E11B","#D4E11A","#D7E219","#DAE218","#DCE218","#DFE318","#E1E318","#E4E318","#E7E419","#E9E419","#ECE41A","#EEE51B","#F1E51C","#F3E51E","#F6E61F","#F8E621","#FAE622","#FDE724"]},"id":"b661dcd9-576f-4c79-90d3-65f41d14b544","type":"LogColorMapper"},{"attributes":{},"id":"b7d58927-6ebb-4002-8dd1-f651f3cd7cc3","type":"BasicTickFormatter"},{"attributes":{"overlay":{"id":"08b5d6f0-3cea-4e7d-a11b-4a4cd2eab30a","type":"BoxAnnotation"}},"id":"9bdf7e1b-f00b-4377-87b0-82d52037d7df","type":"BoxZoomTool"},{"attributes":{"callback":null,"geojson":"{\"type\": \"FeatureCollection\", \"features\": [{\"id\": \"2\", \"type\": \"Feature\", \"properties\": {\"count\": 654, \"OBJECTID\": 3, \"Shape_Leng\": 0.0843411059012, \"Shape_Area\": 0.000314414156821, \"zone\": \"Allerton/Pelham Gardens\", \"LocationID\": 3, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84792614099985, 40.87134223399991], [-73.82950892699994, 40.86184121299988], [-73.82834347399992, 40.86089202599988], [-73.8368191849999, 40.85727680899989], [-73.8566377469999, 40.8581191719999], [-73.85780378799993, 40.86303621699989], [-73.85961281199982, 40.865512717999856], [-73.86156274099987, 40.86554946399989], [-73.86137924099984, 40.8713365119999], [-73.85661382499993, 40.87125016699991], [-73.85651292399994, 40.874217792999936], [-73.84792614099985, 40.87134223399991]]]}}, {\"id\": \"3\", \"type\": \"Feature\", \"properties\": {\"count\": 471596, \"OBJECTID\": 4, \"Shape_Leng\": 0.0435665270921, \"Shape_Area\": 0.000111871946192, \"zone\": \"Alphabet City\", \"LocationID\": 4, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97177410965318, 40.72582128133705], [-73.97347975574512, 40.71886142911285], [-73.98382387299982, 40.72147287199987], [-73.97802697999985, 40.729433059999984], [-73.97177410965318, 40.72582128133705]]]}}, {\"id\": \"4\", \"type\": \"Feature\", \"properties\": {\"count\": 39, \"OBJECTID\": 5, \"Shape_Leng\": 0.0921464898574, \"Shape_Area\": 0.000497957489363, \"zone\": \"Arden Heights\", \"LocationID\": 5, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.17421738099989, 40.56256808599987], [-74.17185166499996, 40.56152092299991], [-74.18018524699995, 40.54837960899992], [-74.18903145199995, 40.53815934199989], [-74.19377169299999, 40.53583384699993], [-74.20314825999996, 40.55606178499986], [-74.20046730799992, 40.55623049099987], [-74.19317388399995, 40.56260434399992], [-74.18340311699991, 40.56540496199994], [-74.17421738099989, 40.56256808599987]]]}}, {\"id\": \"5\", \"type\": \"Feature\", \"properties\": {\"count\": 262, \"OBJECTID\": 6, \"Shape_Leng\": 0.150490542523, \"Shape_Area\": 0.000606460984581, \"zone\": \"Arrochar/Fort Wadsworth\", \"LocationID\": 6, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.06367318899999, 40.60219816599994], [-74.0651709129999, 40.604213507999894], [-74.05685812390232, 40.608056357772554], [-74.05381069805436, 40.605912718852984], [-74.05222826954189, 40.599776102303636], [-74.06503332378131, 40.58899099843518], [-74.07043976099992, 40.59673474699988], [-74.08173342999991, 40.59215628599987], [-74.08376862299998, 40.595520956999934], [-74.08587511899998, 40.59589012099985], [-74.0814942209999, 40.599081990999935], [-74.08184498199992, 40.601535196999926], [-74.08947630599992, 40.60090432199991], [-74.09172051499995, 40.59922085699985], [-74.09271908099993, 40.60636222199992], [-74.09045495899996, 40.60909452599991], [-74.06367318899999, 40.60219816599994]]]}}, {\"id\": \"6\", \"type\": \"Feature\", \"properties\": {\"count\": 301322, \"OBJECTID\": 7, \"Shape_Leng\": 0.107417171123, \"Shape_Area\": 0.000389787989274, \"zone\": \"Astoria\", \"LocationID\": 7, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90413637799996, 40.76752031699986], [-73.90202516599992, 40.767416783999884], [-73.90779405499987, 40.760517799999896], [-73.90631505099988, 40.75984387599992], [-73.91023124699991, 40.75297652299985], [-73.91763192900005, 40.75385784899993], [-73.92562743799996, 40.75213724499991], [-73.92415699999985, 40.753961141999845], [-73.93679091699993, 40.75988288099991], [-73.93398092100001, 40.76282598199987], [-73.93858976899998, 40.766817358999916], [-73.93493511099992, 40.76701302399987], [-73.93344468499993, 40.768817789999915], [-73.92425288799978, 40.76472456399995], [-73.92218946199995, 40.767267356999895], [-73.91977764799998, 40.7661521839999], [-73.91849336999996, 40.76773465699995], [-73.91995596899991, 40.768440064999865], [-73.91811686499992, 40.77002848599989], [-73.90413637799996, 40.76752031699986]]]}}, {\"id\": \"7\", \"type\": \"Feature\", \"properties\": {\"count\": 2190, \"OBJECTID\": 8, \"Shape_Leng\": 0.0275906911574, \"Shape_Area\": 2.6587716279e-05, \"zone\": \"Astoria Park\", \"LocationID\": 8, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92334041500001, 40.77512891199993], [-73.92827730816353, 40.77689897516217], [-73.91996458796496, 40.78263506876824], [-73.91796581399998, 40.78125629099988], [-73.92375761299998, 40.77654701199991], [-73.92334041500001, 40.77512891199993]]]}}, {\"id\": \"8\", \"type\": \"Feature\", \"properties\": {\"count\": 422, \"OBJECTID\": 9, \"Shape_Leng\": 0.0997840924705, \"Shape_Area\": 0.000338443803197, \"zone\": \"Auburndale\", \"LocationID\": 9, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.78502434699996, 40.761036515999855], [-73.77573836999991, 40.74332564699993], [-73.79672497900003, 40.738416642999894], [-73.79723084999992, 40.74169550699993], [-73.7944636839999, 40.74497002199991], [-73.79472264899997, 40.74759932999989], [-73.79729524699995, 40.74943505299987], [-73.79471901099991, 40.75064761699991], [-73.79493246199993, 40.75779802999994], [-73.79031517, 40.75782345099988], [-73.79076004599987, 40.76098576899991], [-73.79343991199988, 40.760764127999934], [-73.79317944599984, 40.77040317799988], [-73.78502434699996, 40.761036515999855]]]}}, {\"id\": \"9\", \"type\": \"Feature\", \"properties\": {\"count\": 22388, \"OBJECTID\": 10, \"Shape_Leng\": 0.0998394794152, \"Shape_Area\": 0.000435823818081, \"zone\": \"Baisley Park\", \"LocationID\": 10, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7832662499999, 40.68999429299992], [-73.77966850599998, 40.685377122999945], [-73.77699767599997, 40.68628268899992], [-73.77220275299989, 40.681610338999896], [-73.78025859499995, 40.6778409239999], [-73.78573730899993, 40.67238503499988], [-73.78808647399985, 40.6730096269999], [-73.78940999399993, 40.66684115799993], [-73.80155565300004, 40.6667533639999], [-73.80142836799988, 40.673666833999874], [-73.80576840199991, 40.682932390999916], [-73.79898209099989, 40.68485873699989], [-73.79995042799996, 40.68682771899991], [-73.79561486799987, 40.688656890999916], [-73.793908417, 40.68630469099985], [-73.7832662499999, 40.68999429299992]]]}}, {\"id\": \"10\", \"type\": \"Feature\", \"properties\": {\"count\": 928, \"OBJECTID\": 11, \"Shape_Leng\": 0.0792110389596, \"Shape_Area\": 0.00026452053504, \"zone\": \"Bath Beach\", \"LocationID\": 11, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00109809499993, 40.60303462599992], [-73.99559320799995, 40.600232221999875], [-74.00200394754667, 40.594132371137796], [-74.01041238886523, 40.60047518174621], [-74.01942760861903, 40.602871578425265], [-74.01549854999998, 40.606841523999925], [-74.01701910199995, 40.60765441699994], [-74.01050403599997, 40.613896610999866], [-74.00094408399997, 40.60863970999988], [-74.00493448899994, 40.60507440599987], [-74.00109809499993, 40.60303462599992]]]}}, {\"id\": \"11\", \"type\": \"Feature\", \"properties\": {\"count\": 56712, \"OBJECTID\": 12, \"Shape_Leng\": 0.0366613013579, \"Shape_Area\": 4.15116236727e-05, \"zone\": \"Battery Park\", \"LocationID\": 12, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01565756599994, 40.70483308799993], [-74.01425861399996, 40.70448944299991], [-74.01376867799992, 40.7011432629999], [-74.014220631955, 40.70011149186829], [-74.0176479750685, 40.70349459987955], [-74.01565756599994, 40.70483308799993]]]}}, {\"id\": \"12\", \"type\": \"Feature\", \"properties\": {\"count\": 1319172, \"OBJECTID\": 13, \"Shape_Leng\": 0.0502813228631, \"Shape_Area\": 0.000149358592917, \"zone\": \"Battery Park City\", \"LocationID\": 13, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01244109299991, 40.7190576729999], [-74.01668714699993, 40.704743941999865], [-74.01844770243774, 40.70416216823156], [-74.0174808615258, 40.704667487696156], [-74.01934254624483, 40.706093673029706], [-74.01777014254499, 40.71283457478916], [-74.01662424425285, 40.71215731899529], [-74.01632006627581, 40.71340798247826], [-74.01772496219284, 40.71307018162274], [-74.01671018605829, 40.718624176057965], [-74.01244109299991, 40.7190576729999]]]}}, {\"id\": \"13\", \"type\": \"Feature\", \"properties\": {\"count\": 8052, \"OBJECTID\": 14, \"Shape_Leng\": 0.175213698053, \"Shape_Area\": 0.00138177826442, \"zone\": \"Bay Ridge\", \"LocationID\": 14, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.03407329297129, 40.64431393298185], [-74.02475437599998, 40.6381214179999], [-74.02108051399998, 40.64149093699994], [-74.02141038399992, 40.638584902999845], [-74.01514630699995, 40.63322193799993], [-74.01970491599991, 40.6221849649999], [-74.02754297299998, 40.61293718299988], [-74.02637609099996, 40.61245060299993], [-74.02710660599995, 40.61024395599991], [-74.02150602899994, 40.60782209899992], [-74.02247769499994, 40.60667421599987], [-74.02078541, 40.605673685999875], [-74.02139467299999, 40.60449399799991], [-74.01869135799996, 40.60360792399991], [-74.01942760861903, 40.602871578425265], [-74.0321315955131, 40.605901836264245], [-74.04038132338673, 40.61534178232843], [-74.04189123741789, 40.62406041492981], [-74.04108656611493, 40.63009852389527], [-74.03680662048858, 40.63898422154557], [-74.03876297920752, 40.639588258980524], [-74.03672484374877, 40.63914119020668], [-74.03571276150724, 40.64064015404647], [-74.03675840317513, 40.641611762713936], [-74.03407329297129, 40.64431393298185]]]}}, {\"id\": \"14\", \"type\": \"Feature\", \"properties\": {\"count\": 286, \"OBJECTID\": 15, \"Shape_Leng\": 0.14433622262, \"Shape_Area\": 0.000925219395547, \"zone\": \"Bay Terrace/Fort Totten\", \"LocationID\": 15, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7774039129087, 40.79659824126783], [-73.77063124537418, 40.78846290603716], [-73.77426836058959, 40.78675665865473], [-73.77633728818715, 40.78848722575376], [-73.76820055460033, 40.77944417919097], [-73.76702392346498, 40.78008623415827], [-73.76677570373936, 40.77981245869834], [-73.76782207615058, 40.77881037615661], [-73.77553732199992, 40.777025597999916], [-73.77583958299994, 40.778612371999884], [-73.79565702099983, 40.773036085999905], [-73.79853009399987, 40.775190728999924], [-73.8037902289999, 40.77561011199992], [-73.80242791699989, 40.78606827399991], [-73.79491977600001, 40.78633880799987], [-73.79058298999988, 40.78819290999989], [-73.79179280401884, 40.789351616549254], [-73.79028202353197, 40.79031696339975], [-73.78174708763098, 40.79113340664446], [-73.78079182072646, 40.79403591986439], [-73.78325271025878, 40.79492032976596], [-73.7774039129087, 40.79659824126783]]]}}, {\"id\": \"15\", \"type\": \"Feature\", \"properties\": {\"count\": 752, \"OBJECTID\": 16, \"Shape_Leng\": 0.141291873771, \"Shape_Area\": 0.000871889446182, \"zone\": \"Bayside\", \"LocationID\": 16, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7685730499999, 40.77910542899991], [-73.75864264358073, 40.767357170035446], [-73.75377036482266, 40.76530463124142], [-73.75921689799992, 40.76432485599988], [-73.75800131799996, 40.76037986699988], [-73.76239510499991, 40.75949699799992], [-73.76184343999986, 40.75553726899989], [-73.75684127399991, 40.74932207699987], [-73.77573836999991, 40.74332564699993], [-73.78546383099984, 40.76259192499988], [-73.79565702099983, 40.773036085999905], [-73.77678363299991, 40.77840958299987], [-73.77553732199992, 40.777025597999916], [-73.7685730499999, 40.77910542899991]]]}}, {\"id\": \"16\", \"type\": \"Feature\", \"properties\": {\"count\": 66103, \"OBJECTID\": 17, \"Shape_Leng\": 0.093522632948, \"Shape_Area\": 0.000322957654799, \"zone\": \"Bedford\", \"LocationID\": 17, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94306406899986, 40.701424434999886], [-73.94193078899983, 40.70072523399989], [-73.94438788599984, 40.70042452299993], [-73.94032794, 40.679889974999874], [-73.9533701749999, 40.680640507999904], [-73.95468418899998, 40.68724485399985], [-73.95956770199984, 40.686682554999955], [-73.96029281699998, 40.69034624999995], [-73.95541057999996, 40.69090829199994], [-73.95614239299985, 40.694579018999896], [-73.96015854699988, 40.69411730899989], [-73.96105100699992, 40.69832607899989], [-73.95701993199991, 40.69897391399995], [-73.95745736399992, 40.70082260299993], [-73.95381196900003, 40.703180979999914], [-73.95128819399989, 40.700922364999855], [-73.94705205299991, 40.70366394899985], [-73.94306406899986, 40.701424434999886]]]}}, {\"id\": \"17\", \"type\": \"Feature\", \"properties\": {\"count\": 1787, \"OBJECTID\": 18, \"Shape_Leng\": 0.0697995498569, \"Shape_Area\": 0.000148850163948, \"zone\": \"Bedford Park\", \"LocationID\": 18, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88513907699999, 40.86638287399992], [-73.88889566099989, 40.86475415299988], [-73.89573924699978, 40.85813820699986], [-73.89883814599983, 40.859307307999906], [-73.897143798, 40.86244506499991], [-73.90107966999994, 40.86275638299992], [-73.8974032329998, 40.86747418999989], [-73.89438130299995, 40.866068850999966], [-73.88785196299986, 40.87211417599992], [-73.88937163799984, 40.87338229799993], [-73.88527683199987, 40.87921766899991], [-73.88146889699985, 40.868573645999916], [-73.88513907699999, 40.86638287399992]]]}}, {\"id\": \"18\", \"type\": \"Feature\", \"properties\": {\"count\": 1246, \"OBJECTID\": 19, \"Shape_Leng\": 0.101824875452, \"Shape_Area\": 0.000546661094782, \"zone\": \"Bellerose\", \"LocationID\": 19, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.72339596299987, 40.750389075999855], [-73.72258571899978, 40.74885045099991], [-73.72010148299994, 40.74977568599991], [-73.71147804899985, 40.73250140099988], [-73.71049995381028, 40.72722708720663], [-73.72990512499977, 40.723272494999954], [-73.7314721509999, 40.727411794999924], [-73.74640108899996, 40.7312928389999], [-73.74672140400006, 40.73247715099991], [-73.73933151399987, 40.739474474999916], [-73.72829094199989, 40.74745895699992], [-73.727664654, 40.749654641999946], [-73.72339596299987, 40.750389075999855]]]}}, {\"id\": \"19\", \"type\": \"Feature\", \"properties\": {\"count\": 1115, \"OBJECTID\": 20, \"Shape_Leng\": 0.0514401924362, \"Shape_Area\": 0.000134512633032, \"zone\": \"Belmont\", \"LocationID\": 20, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88386792099986, 40.8642908889999], [-73.88060896899991, 40.860357092999884], [-73.88311982600001, 40.848221091999925], [-73.88987871399985, 40.85207022099993], [-73.89109710400002, 40.856779813999886], [-73.89027555599993, 40.86110907299996], [-73.89177188799992, 40.86187171599992], [-73.88362518100003, 40.86725758799991], [-73.88267624699999, 40.86608914099984], [-73.88386792099986, 40.8642908889999]]]}}, {\"id\": \"20\", \"type\": \"Feature\", \"properties\": {\"count\": 1226, \"OBJECTID\": 21, \"Shape_Leng\": 0.115973569062, \"Shape_Area\": 0.000380251345507, \"zone\": \"Bensonhurst East\", \"LocationID\": 21, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97418385499991, 40.609463501999905], [-73.97299433999983, 40.6088141419999], [-73.97291116900001, 40.599315803999964], [-73.97148350899987, 40.59717158499996], [-73.98035785499995, 40.59619154099992], [-73.97995443499985, 40.594073811999884], [-73.98601936799993, 40.594446048999906], [-73.98962150099992, 40.59098199299996], [-73.99923690396953, 40.588935328269024], [-73.99710730365224, 40.591390755943046], [-74.00004314944917, 40.59120428665461], [-73.9994830488618, 40.59296919994875], [-73.99530829599992, 40.59442758699989], [-73.99674706899995, 40.59564312599992], [-73.9868476259999, 40.60509723599994], [-73.98913191499994, 40.6064561369999], [-73.97963821799982, 40.61556454699991], [-73.97477658, 40.6126384749999], [-73.97418385499991, 40.609463501999905]]]}}, {\"id\": \"21\", \"type\": \"Feature\", \"properties\": {\"count\": 2497, \"OBJECTID\": 22, \"Shape_Leng\": 0.126170229196, \"Shape_Area\": 0.000472062684608, \"zone\": \"Bensonhurst West\", \"LocationID\": 22, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99254973599997, 40.62427426799996], [-73.97963821799982, 40.61556454699991], [-73.98913191499994, 40.6064561369999], [-73.9868476259999, 40.60509723599994], [-73.99674706899995, 40.59564312599992], [-73.99530829599992, 40.59442758699989], [-74.00105748445709, 40.5925370050623], [-74.00200394804675, 40.594132371137796], [-73.99559320799995, 40.600232221999875], [-74.00493448899994, 40.60507440599987], [-74.00094408399997, 40.60863970999988], [-74.01050403599997, 40.613896610999866], [-73.99932090299998, 40.62465524699996], [-73.99771358999998, 40.62368026999996], [-73.99678847599995, 40.626826050999895], [-73.99254973599997, 40.62427426799996]]]}}, {\"id\": \"22\", \"type\": \"Feature\", \"properties\": {\"count\": 246, \"OBJECTID\": 23, \"Shape_Leng\": 0.290556028962, \"Shape_Area\": 0.00219556576201, \"zone\": \"Bloomfield/Emerson Hill\", \"LocationID\": 23, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.19568609223377, 40.63501686464005], [-74.17459706299995, 40.622364010999874], [-74.16163010299996, 40.61793493599989], [-74.15018666499996, 40.60949781099991], [-74.14391816399994, 40.60798983699987], [-74.11742466500002, 40.609289008999916], [-74.11487058499988, 40.602136659999864], [-74.12059546299996, 40.601809757999874], [-74.12506115499997, 40.59818005599987], [-74.13091909399999, 40.598855047999905], [-74.13864458599996, 40.58906080499989], [-74.14790576499993, 40.58855634599993], [-74.16025756099997, 40.59534393799988], [-74.16189940599996, 40.59529877099993], [-74.16369888199995, 40.59225833899993], [-74.16360268199993, 40.593374198999854], [-74.16886362399997, 40.59472808699995], [-74.16719634899995, 40.602075311999926], [-74.17260229799992, 40.602971490999934], [-74.17822457299994, 40.5998280609999], [-74.17932277499995, 40.59676057899984], [-74.18730248699987, 40.588488221999896], [-74.185371048, 40.58745101499987], [-74.19423261399992, 40.58514680599991], [-74.19964981299998, 40.58226262599991], [-74.20048848899994, 40.5799497779999], [-74.20583829199992, 40.57984126099989], [-74.204646089432, 40.589285745465865], [-74.19751357718701, 40.59679898603677], [-74.20281628374593, 40.608270827967345], [-74.20244374449518, 40.61328469393197], [-74.2003834842897, 40.616428098243766], [-74.20163201104123, 40.623121565457346], [-74.2007872998309, 40.63034627446781], [-74.19568609223377, 40.63501686464005]]]}}, {\"id\": \"23\", \"type\": \"Feature\", \"properties\": {\"count\": 428620, \"OBJECTID\": 24, \"Shape_Leng\": 0.0469999619287, \"Shape_Area\": 6.07235737749e-05, \"zone\": \"Bloomingdale\", \"LocationID\": 24, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95953658899998, 40.798718525999895], [-73.96004456499999, 40.79804123499991], [-73.97287179090726, 40.8033561875739], [-73.97110765876137, 40.80579013958964], [-73.95817297099987, 40.800582540999876], [-73.95953658899998, 40.798718525999895]]]}}, {\"id\": \"24\", \"type\": \"Feature\", \"properties\": {\"count\": 168900, \"OBJECTID\": 25, \"Shape_Leng\": 0.0471458199319, \"Shape_Area\": 0.000124168267356, \"zone\": \"Boerum Hill\", \"LocationID\": 25, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98155298299992, 40.689146163999936], [-73.97804289599988, 40.68485661499987], [-73.98175642299985, 40.67931462399987], [-73.99490113599977, 40.6844302379999], [-73.99236366999985, 40.689690123999924], [-73.99054474599986, 40.689168104999936], [-73.98902944799994, 40.69212386099992], [-73.98155298299992, 40.689146163999936]]]}}, {\"id\": \"25\", \"type\": \"Feature\", \"properties\": {\"count\": 3005, \"OBJECTID\": 26, \"Shape_Leng\": 0.12354780707, \"Shape_Area\": 0.000534039927626, \"zone\": \"Borough Park\", \"LocationID\": 26, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98331628499983, 40.6414786819999], [-73.98042057899988, 40.64238522699991], [-73.97911188699993, 40.63544038599991], [-73.9768886119999, 40.6356748619999], [-73.97705352899999, 40.62153252199987], [-73.97539380499998, 40.62076998699995], [-73.97785009299986, 40.617287605999906], [-73.97537335500002, 40.6157547219999], [-73.97335879399992, 40.616541714999904], [-73.97290326899996, 40.61415296799988], [-73.97517176999993, 40.61472418599991], [-73.97477658, 40.6126384749999], [-74.00702302499997, 40.63210999499996], [-74.00556490399993, 40.63308317599992], [-74.00735636699996, 40.6341636269999], [-73.99917564799993, 40.642025441999856], [-73.9955103369999, 40.63980966699993], [-73.98834986699991, 40.64456224399992], [-73.98331628499983, 40.6414786819999]]]}}, {\"id\": \"26\", \"type\": \"Feature\", \"properties\": {\"count\": 48, \"OBJECTID\": 27, \"Shape_Leng\": 0.202508808518, \"Shape_Area\": 0.00134088762746, \"zone\": \"Breezy Point/Fort Tilden/Riis Beach\", \"LocationID\": 27, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86522555399998, 40.57045847199989], [-73.86268391251235, 40.56650565604353], [-73.8949181726631, 40.5570787962983], [-73.90658574923901, 40.555710625377294], [-73.94073681682697, 40.54182008707755], [-73.93985107460422, 40.55456005374592], [-73.9261572327542, 40.56155260972308], [-73.91766081900093, 40.56282614968639], [-73.91197024752503, 40.56586941753247], [-73.90683894638656, 40.562856359344565], [-73.90121507272005, 40.563008460632346], [-73.89261013445609, 40.56858724728667], [-73.87780808995733, 40.56880063006316], [-73.86288301945429, 40.576059761127304], [-73.86361132999998, 40.573823056999835], [-73.86662812000002, 40.573057648999914], [-73.86522555399998, 40.57045847199989]]]}}, {\"id\": \"27\", \"type\": \"Feature\", \"properties\": {\"count\": 11566, \"OBJECTID\": 28, \"Shape_Leng\": 0.097960782214, \"Shape_Area\": 0.000291203927662, \"zone\": \"Briarwood/Jamaica Hills\", \"LocationID\": 28, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79240413399991, 40.716193040999926], [-73.79303800099989, 40.71071107499988], [-73.79776810199988, 40.7091431879999], [-73.79870150699992, 40.71066337699988], [-73.80130625, 40.709742148999965], [-73.80302414799988, 40.70813965699997], [-73.80165725599993, 40.705761474999846], [-73.8056477309999, 40.70467172799994], [-73.80694856299993, 40.70736709999989], [-73.81201248599983, 40.70594494499993], [-73.81152856899989, 40.70227287499991], [-73.81637824499992, 40.70245196399987], [-73.82591945199992, 40.71598987599992], [-73.80494148499987, 40.715996230999956], [-73.79073248899982, 40.71920875699987], [-73.79240413399991, 40.716193040999926]]]}}, {\"id\": \"28\", \"type\": \"Feature\", \"properties\": {\"count\": 1132, \"OBJECTID\": 29, \"Shape_Leng\": 0.0714083127733, \"Shape_Area\": 0.000201673837402, \"zone\": \"Brighton Beach\", \"LocationID\": 29, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96004798699995, 40.58326987199995], [-73.95414287899986, 40.58310613999989], [-73.95235846771321, 40.574274685845246], [-73.9589871897255, 40.57343065573189], [-73.95941778299985, 40.57517798299994], [-73.9688899589999, 40.57526123899986], [-73.96514385199995, 40.5911019159999], [-73.9606798409999, 40.591597582999945], [-73.96004798699995, 40.58326987199995]]]}}, {\"id\": \"29\", \"type\": \"Feature\", \"properties\": {\"count\": 46, \"OBJECTID\": 30, \"Shape_Leng\": 0.0945097669793, \"Shape_Area\": 0.000145862107626, \"zone\": \"Broad Channel\", \"LocationID\": 30, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82075892499992, 40.61523267899991], [-73.81652681189118, 40.61494701007998], [-73.81475287105245, 40.6086327967523], [-73.81739192103247, 40.60515812414338], [-73.82071206789055, 40.59517051496212], [-73.83465599507487, 40.59516845882606], [-73.83239364476839, 40.59780335319121], [-73.82806159687702, 40.59768040282441], [-73.8259798598142, 40.599472067181495], [-73.82465420648664, 40.5981492468515], [-73.82376612290614, 40.60004007091038], [-73.82146929866913, 40.59999372263383], [-73.82445218054455, 40.60079696284418], [-73.81961795381072, 40.610479041032136], [-73.82087841795061, 40.61225734590997], [-73.82161460094558, 40.60848031911391], [-73.8238246859766, 40.61149866495507], [-73.82075892499992, 40.61523267899991]]]}}, {\"id\": \"30\", \"type\": \"Feature\", \"properties\": {\"count\": 543, \"OBJECTID\": 31, \"Shape_Leng\": 0.0964245666516, \"Shape_Area\": 0.000333975927329, \"zone\": \"Bronx Park\", \"LocationID\": 31, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87094462199981, 40.85726608099992], [-73.86841539699994, 40.85778035799994], [-73.86990379599995, 40.85587302199991], [-73.86823466099995, 40.85091373699986], [-73.86889243299996, 40.848112321999906], [-73.87459175399982, 40.841520647999914], [-73.878171756, 40.84262443699987], [-73.8773292409999, 40.84398312499995], [-73.88311982600001, 40.848221091999925], [-73.88060896899991, 40.860357092999884], [-73.88394303099996, 40.86494818699985], [-73.87619224900003, 40.87047671099988], [-73.87094367500002, 40.87851076699998], [-73.86943471300005, 40.87812919899994], [-73.87094462199981, 40.85726608099992]]]}}, {\"id\": \"31\", \"type\": \"Feature\", \"properties\": {\"count\": 605, \"OBJECTID\": 32, \"Shape_Leng\": 0.05426721601, \"Shape_Area\": 0.000150879171971, \"zone\": \"Bronxdale\", \"LocationID\": 32, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85882515999995, 40.85806135699987], [-73.87104944999982, 40.857304836999965], [-73.87056192499989, 40.87151077799995], [-73.86137924099984, 40.8713365119999], [-73.86156274099987, 40.86554946399989], [-73.85961281199982, 40.865512717999856], [-73.85780378799993, 40.86303621699989], [-73.8566377469999, 40.8581191719999], [-73.85882515999995, 40.85806135699987]]]}}, {\"id\": \"32\", \"type\": \"Feature\", \"properties\": {\"count\": 174931, \"OBJECTID\": 33, \"Shape_Leng\": 0.0532702931967, \"Shape_Area\": 0.000147416802448, \"zone\": \"Brooklyn Heights\", \"LocationID\": 33, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99619227264343, 40.70337714093203], [-73.99087453800003, 40.70067311699993], [-73.99097187, 40.692548799999926], [-73.98902944799994, 40.69212386099992], [-73.99054474599986, 40.689168104999936], [-74.00110519399988, 40.692056594999954], [-74.0009586845849, 40.694069083791064], [-74.00301393814541, 40.69477784423956], [-74.00026872377784, 40.69496580257801], [-74.00043147340423, 40.69705246617853], [-73.99876458027717, 40.697120733093975], [-74.0010490305752, 40.697908236697415], [-73.9983776068576, 40.698063296146074], [-74.00001840433616, 40.699466048073575], [-73.99716054243916, 40.69979281264407], [-73.99813879899439, 40.701518788248414], [-73.99619227264343, 40.70337714093203]]]}}, {\"id\": \"33\", \"type\": \"Feature\", \"properties\": {\"count\": 5137, \"OBJECTID\": 34, \"Shape_Leng\": 0.0657059323545, \"Shape_Area\": 0.000173946146651, \"zone\": \"Brooklyn Navy Yard\", \"LocationID\": 34, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.97232032119902, 40.70908288315216], [-73.9702621853228, 40.706887504284005], [-73.97018075572805, 40.70492927056683], [-73.96751516691087, 40.703437356878126], [-73.96929693837701, 40.705088331229625], [-73.96838933699995, 40.70682918699989], [-73.96217978199998, 40.70022070999989], [-73.963971858, 40.69637865499991], [-73.98054419199988, 40.69824651199988], [-73.97885303360711, 40.7060655352846], [-73.97713562625376, 40.703005581402515], [-73.97596096344841, 40.704715506440465], [-73.9769274918834, 40.70279782040038], [-73.97455023388345, 40.70159034430477], [-73.97573541084542, 40.69957044770558], [-73.97411513926478, 40.70130527141927], [-73.97239201397464, 40.700166322752274], [-73.97318676670886, 40.70166980367368], [-73.97078200587988, 40.69997866301123], [-73.97283512788735, 40.70272874074404], [-73.96948655218084, 40.70051906125818], [-73.97283889974949, 40.70334642860182], [-73.96898517428966, 40.70174708878653], [-73.97465569608048, 40.70607426299424], [-73.97254880057814, 40.70620449000379], [-73.9742361696854, 40.70802438612241], [-73.97109655489108, 40.705850056558255], [-73.97232032119902, 40.70908288315216]]], [[[-73.97892920137538, 40.705931589879214], [-73.97893110285521, 40.70592490623034], [-73.97906084872884, 40.705946028470365], [-73.97905767006142, 40.705952889050316], [-73.97892920137538, 40.705931589879214]]], [[[-73.98237340600002, 40.70554334999991], [-73.98242287179853, 40.70582205595453], [-73.98102390618274, 40.705898913893535], [-73.98230205183869, 40.705736979399674], [-73.98237340600002, 40.70554334999991]]]]}}, {\"id\": \"34\", \"type\": \"Feature\", \"properties\": {\"count\": 2243, \"OBJECTID\": 35, \"Shape_Leng\": 0.085787918592, \"Shape_Area\": 0.000323825195932, \"zone\": \"Brownsville\", \"LocationID\": 35, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90105329800004, 40.66212482099994], [-73.89880698499995, 40.65740666199988], [-73.90784346399994, 40.65459152999988], [-73.90855790499995, 40.65209593799989], [-73.91012255699995, 40.655851770999945], [-73.91236458899995, 40.65551356499985], [-73.9195947979998, 40.66199567899985], [-73.92008167099986, 40.659857632999866], [-73.92637970499992, 40.6655148919999], [-73.90347422300006, 40.67550668299985], [-73.90105329800004, 40.66212482099994]]]}}, {\"id\": \"35\", \"type\": \"Feature\", \"properties\": {\"count\": 43748, \"OBJECTID\": 36, \"Shape_Leng\": 0.0870507867063, \"Shape_Area\": 0.000247666139269, \"zone\": \"Bushwick North\", \"LocationID\": 36, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91254913799993, 40.70387341999989], [-73.912904041, 40.7023618909999], [-73.91067882699998, 40.701045968999914], [-73.91180820099989, 40.69993800299986], [-73.9042601839999, 40.69570037099989], [-73.90579597099993, 40.69412715499987], [-73.90123290699994, 40.69144227899997], [-73.904055772, 40.68854627799986], [-73.93115533899987, 40.70377233499987], [-73.93269784899994, 40.70317039099989], [-73.93391870799988, 40.70748702599992], [-73.92189184700005, 40.709396096999896], [-73.91254913799993, 40.70387341999989]]]}}, {\"id\": \"36\", \"type\": \"Feature\", \"properties\": {\"count\": 87183, \"OBJECTID\": 37, \"Shape_Leng\": 0.142810040466, \"Shape_Area\": 0.000452062644782, \"zone\": \"Bushwick South\", \"LocationID\": 37, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93312548700004, 40.70470086199992], [-73.93269784899994, 40.70317039099989], [-73.93115533899987, 40.70377233499987], [-73.90482499699988, 40.688808569999956], [-73.90180467199995, 40.6907662979999], [-73.90116154999988, 40.68787793499991], [-73.89646625099995, 40.68233642199988], [-73.89704472199993, 40.68056181599996], [-73.90047001100004, 40.679535243999936], [-73.90262262299991, 40.6806645029999], [-73.9040463979999, 40.67922059799985], [-73.93856854799981, 40.698848135999874], [-73.94394947299996, 40.698221278999924], [-73.94438788599984, 40.70042452299993], [-73.94042057899993, 40.701076597999865], [-73.94103009699987, 40.7046361769999], [-73.94273448399997, 40.70447330599988], [-73.94378799599994, 40.710900378999945], [-73.93480281900003, 40.713372495999884], [-73.93229708600003, 40.70774944699988], [-73.93391870799988, 40.70748702599992], [-73.93312548700004, 40.70470086199992]]]}}, {\"id\": \"37\", \"type\": \"Feature\", \"properties\": {\"count\": 747, \"OBJECTID\": 38, \"Shape_Leng\": 0.0832175685234, \"Shape_Area\": 0.000327392684821, \"zone\": \"Cambria Heights\", \"LocationID\": 38, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.73624941199995, 40.702536977999884], [-73.72677759516957, 40.7030592911994], [-73.72586264191175, 40.683241080973865], [-73.7475517539999, 40.68934733599984], [-73.7435928609999, 40.69621532499986], [-73.74618157999997, 40.696121925999904], [-73.74773012199991, 40.69955313699991], [-73.74500119100004, 40.70026221299988], [-73.74590871000004, 40.70228727399988], [-73.74109293999989, 40.7034810629999], [-73.73624941199995, 40.702536977999884]]]}}, {\"id\": \"38\", \"type\": \"Feature\", \"properties\": {\"count\": 2174, \"OBJECTID\": 39, \"Shape_Leng\": 0.132369620757, \"Shape_Area\": 0.00090028293862, \"zone\": \"Canarsie\", \"LocationID\": 39, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.89121853499988, 40.64998769899987], [-73.88805587182881, 40.64511978725729], [-73.87714434884276, 40.63597649800373], [-73.88364451728226, 40.630943312138584], [-73.88263248433404, 40.62827427312689], [-73.88369231571497, 40.62755937391403], [-73.88752673706196, 40.628331824482444], [-73.89577374598754, 40.62251907138477], [-73.91860649599992, 40.632090414999894], [-73.91995062999999, 40.64470762999989], [-73.91830152299998, 40.645812262999925], [-73.90944846399994, 40.65152159899992], [-73.90398200599991, 40.64662856799988], [-73.89586154900002, 40.65188052699987], [-73.89761013299987, 40.65345381599992], [-73.89354677099988, 40.65520271399988], [-73.89000589199988, 40.651148696999925], [-73.89121853499988, 40.64998769899987]]], [[[-73.88833929256269, 40.64671022891759], [-73.88849720193863, 40.64675678600008], [-73.88853291212868, 40.646861882230674], [-73.88846787961647, 40.646899734025276], [-73.88833929256269, 40.64671022891759]]]]}}, {\"id\": \"39\", \"type\": \"Feature\", \"properties\": {\"count\": 101535, \"OBJECTID\": 40, \"Shape_Leng\": 0.0518506648762, \"Shape_Area\": 0.000108949211644, \"zone\": \"Carroll Gardens\", \"LocationID\": 40, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.998192111, 40.68228732999989], [-73.99620846299992, 40.681723424999916], [-73.99490113599977, 40.6844302379999], [-73.98722413899985, 40.68144730199988], [-73.99069861899994, 40.67580741899994], [-73.99506451099984, 40.67584047499984], [-73.9986188799999, 40.67183017199989], [-74.00329231299999, 40.67964328799992], [-74.00064737999996, 40.685253946999914], [-73.99908066099994, 40.68481599199985], [-74.00004865199998, 40.682803355999965], [-73.998192111, 40.68228732999989]]]}}, {\"id\": \"40\", \"type\": \"Feature\", \"properties\": {\"count\": 443934, \"OBJECTID\": 41, \"Shape_Leng\": 0.052793109453, \"Shape_Area\": 0.000143093037737, \"zone\": \"Central Harlem\", \"LocationID\": 41, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94773985499985, 40.809599720999884], [-73.94177140199992, 40.807088999999955], [-73.94523965199991, 40.80513601399989], [-73.94613131, 40.80387735799992], [-73.94459751299998, 40.803228144999906], [-73.94922045699984, 40.796909858999896], [-73.95964685399987, 40.80115642299993], [-73.954966572, 40.81006455499988], [-73.9535757779999, 40.8094766979999], [-73.95210125199992, 40.811442853999935], [-73.94773985499985, 40.809599720999884]]]}}, {\"id\": \"41\", \"type\": \"Feature\", \"properties\": {\"count\": 202779, \"OBJECTID\": 42, \"Shape_Leng\": 0.0927092274616, \"Shape_Area\": 0.000263896952605, \"zone\": \"Central Harlem North\", \"LocationID\": 42, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93436121591056, 40.83620061961779], [-73.93383089287109, 40.819521063761556], [-73.93901753299998, 40.81085655699991], [-73.93580780199986, 40.80949763799987], [-73.93854407899988, 40.80572965299991], [-73.95210125199992, 40.811442853999935], [-73.94607828700005, 40.821263215999906], [-73.93924885599988, 40.82829635099989], [-73.93864026699997, 40.82973887399988], [-73.9401708989999, 40.83038439099996], [-73.93868316299985, 40.832800788999904], [-73.93436121591056, 40.83620061961779]]]}}, {\"id\": \"42\", \"type\": \"Feature\", \"properties\": {\"count\": 1882534, \"OBJECTID\": 43, \"Shape_Leng\": 0.0997386183576, \"Shape_Area\": 0.000379662912054, \"zone\": \"Central Park\", \"LocationID\": 43, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97255352299985, 40.76489773199991], [-73.98164804599995, 40.76843632199994], [-73.95817297099987, 40.800582540999876], [-73.94922045699984, 40.796909858999896], [-73.97255352299985, 40.76489773199991]]]}}, {\"id\": \"43\", \"type\": \"Feature\", \"properties\": {\"count\": 37, \"OBJECTID\": 44, \"Shape_Leng\": 0.235688967594, \"Shape_Area\": 0.00194465649192, \"zone\": \"Charleston/Tottenville\", \"LocationID\": 44, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.21220034099994, 40.556443429999874], [-74.21784397899994, 40.554432717999894], [-74.22131562699998, 40.55025307399992], [-74.21776941599994, 40.54112662999992], [-74.21800676499996, 40.53229858999995], [-74.21354593399995, 40.521235715999886], [-74.22379595899992, 40.518196345999876], [-74.22690993299994, 40.5191713979999], [-74.22159300278173, 40.50250043822093], [-74.23114494958575, 40.50185121588201], [-74.23988017293566, 40.49757362656005], [-74.24917156868965, 40.49656729705339], [-74.25335538022304, 40.50041319788582], [-74.25554269163935, 40.50783774249614], [-74.25056007272197, 40.516106741535], [-74.24921528676096, 40.51508486129925], [-74.24994158808036, 40.516137488190985], [-74.24854589291671, 40.51616991714839], [-74.24940770789283, 40.516980831423645], [-74.24851449056031, 40.517475626265046], [-74.24644039651318, 40.515965028508525], [-74.2456347316994, 40.518075580745766], [-74.23991937852388, 40.52004699652993], [-74.24293235036517, 40.52122716113276], [-74.24396500758859, 40.52490536633781], [-74.24150733322897, 40.531041342795135], [-74.24204570286014, 40.53434637705037], [-74.24533651176053, 40.53690599443462], [-74.24560205698324, 40.54094959457726], [-74.24803463735688, 40.54309324044144], [-74.24336346552776, 40.54786513523582], [-74.24039182810051, 40.547663310173775], [-74.23641623448965, 40.55050862684339], [-74.23641559733107, 40.55232806634515], [-74.23334188482498, 40.55249971061369], [-74.2287242997478, 40.5562970171392], [-74.22062175949124, 40.55589882100512], [-74.21974356930723, 40.55461267207239], [-74.21921716370927, 40.55579696987855], [-74.21859325782798, 40.55467421024778], [-74.21870767560165, 40.55604101620115], [-74.21775342391726, 40.55500528477618], [-74.2056960519999, 40.55851600599987], [-74.21220034099994, 40.556443429999874]]]}}, {\"id\": \"44\", \"type\": \"Feature\", \"properties\": {\"count\": 340354, \"OBJECTID\": 45, \"Shape_Leng\": 0.0459068626235, \"Shape_Area\": 9.13594688556e-05, \"zone\": \"Chinatown\", \"LocationID\": 45, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99750445299988, 40.71406913199995], [-73.99256242199989, 40.71438807699996], [-73.99190824271031, 40.70953292664334], [-73.99919451154429, 40.70794737667571], [-74.0058917509999, 40.71200715999989], [-73.99995620900002, 40.71801709499987], [-73.99605872699998, 40.7162316399999], [-73.99750445299988, 40.71406913199995]]]}}, {\"id\": \"45\", \"type\": \"Feature\", \"properties\": {\"count\": 120, \"OBJECTID\": 46, \"Shape_Leng\": 0.134475429879, \"Shape_Area\": 0.000926391677672, \"zone\": \"City Island\", \"LocationID\": 46, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.78452431937963, 40.86047706314786], [-73.78519664387113, 40.85882702363741], [-73.78743186194176, 40.858935245614425], [-73.78685886773299, 40.86004505783956], [-73.78452431937963, 40.86047706314786]]], [[[-73.76783205637173, 40.854442205742075], [-73.76938529139694, 40.85253613399675], [-73.76959674311465, 40.84759451909819], [-73.76789492672636, 40.84528943351961], [-73.76901389181107, 40.84509734272193], [-73.77157218833237, 40.8475000585803], [-73.7708892022117, 40.84975177133807], [-73.77296356611284, 40.8521592539586], [-73.77173060411904, 40.852476831319095], [-73.77296285334867, 40.853800062246464], [-73.77219223055407, 40.85998663859697], [-73.76947305423998, 40.85933013666858], [-73.76533243995276, 40.85504359512487], [-73.76783205637173, 40.854442205742075]]], [[[-73.79018745194999, 40.8586099173861], [-73.788043429758, 40.85755991768708], [-73.7884773265335, 40.8544005205322], [-73.78690734702697, 40.854063972299144], [-73.78796264288079, 40.8534747209355], [-73.78338158610377, 40.85158673391434], [-73.78420047134377, 40.8500606442406], [-73.78154449088599, 40.84940043464333], [-73.78267467078706, 40.84813967294664], [-73.78097820938946, 40.8484583479135], [-73.78263740397888, 40.848072649245836], [-73.78148431950568, 40.8478821938486], [-73.78216393435586, 40.84663633778076], [-73.780403269586, 40.846931134166056], [-73.7833653613534, 40.84444472298169], [-73.78348428748797, 40.84367854599843], [-73.78290650949755, 40.84393612584013], [-73.78052736048552, 40.84452514033816], [-73.7799270374445, 40.843883081355834], [-73.78347481249244, 40.843654738109784], [-73.78288778301486, 40.84249482884962], [-73.78022773872677, 40.84377818129771], [-73.78286499304913, 40.84247810188268], [-73.78059735822595, 40.84278659994098], [-73.78216624418964, 40.84168914411004], [-73.7804257718152, 40.842494394206874], [-73.78019599955293, 40.84204621097616], [-73.78185048197766, 40.841653946394324], [-73.7807975292347, 40.84125614604844], [-73.78282260139727, 40.836338155887844], [-73.78485009222643, 40.83747048917844], [-73.78563997845448, 40.83706443458537], [-73.78583067964128, 40.83720998000451], [-73.7857350929868, 40.83923423177581], [-73.78815921166512, 40.84049077856704], [-73.79179343625451, 40.846731672329284], [-73.78951371516173, 40.85130077171307], [-73.79311837183832, 40.851578295240834], [-73.79042053780161, 40.85302297821258], [-73.79281511966936, 40.852494819308916], [-73.79080758729923, 40.85319906006435], [-73.79287610048584, 40.8529441813284], [-73.79105721254825, 40.85391841242664], [-73.79245816844812, 40.85360070875487], [-73.79107002765942, 40.85394495666465], [-73.79253714604873, 40.85384932781285], [-73.79129282160633, 40.854584025976855], [-73.79201712228527, 40.85617694860626], [-73.79270523985137, 40.85619232890572], [-73.79268768222656, 40.856335218720375], [-73.79315700296156, 40.85635091107476], [-73.7931468608596, 40.856467828471146], [-73.79018745194999, 40.8586099173861]]], [[[-73.78283291447852, 40.85587030844573], [-73.78302371522504, 40.85509276666495], [-73.78394699722007, 40.85563043752662], [-73.78343487501385, 40.856362865958886], [-73.78283291447852, 40.85587030844573]]], [[[-73.78061730829718, 40.855735175810025], [-73.78090476851291, 40.8549681331623], [-73.7815638969994, 40.855001286433904], [-73.78122782332017, 40.856087789162764], [-73.78061730829718, 40.855735175810025]]], [[[-73.78833349834521, 40.834667129759346], [-73.78931223606624, 40.83446488655343], [-73.78951019872322, 40.835364042525754], [-73.78845700015209, 40.835309914315395], [-73.78833349834521, 40.834667129759346]]]]}}, {\"id\": \"46\", \"type\": \"Feature\", \"properties\": {\"count\": 1623, \"OBJECTID\": 47, \"Shape_Leng\": 0.0898275563294, \"Shape_Area\": 0.000163198117339, \"zone\": \"Claremont/Bathgate\", \"LocationID\": 47, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89090203399996, 40.854116885999865], [-73.89431774099977, 40.851891310999854], [-73.89617712999998, 40.8466583209999], [-73.89290102299996, 40.84626832999992], [-73.89537836199995, 40.842938778999965], [-73.89666161699988, 40.84352460699992], [-73.90553116799988, 40.82910413299989], [-73.91049010699992, 40.830737372999884], [-73.90538357999992, 40.839768982999885], [-73.90483121999989, 40.84480034199987], [-73.901344046, 40.84442875099994], [-73.89106280699994, 40.8615294119999], [-73.89090203399996, 40.854116885999865]]]}}, {\"id\": \"47\", \"type\": \"Feature\", \"properties\": {\"count\": 4595762, \"OBJECTID\": 48, \"Shape_Leng\": 0.0437467441431, \"Shape_Area\": 9.42538425377e-05, \"zone\": \"Clinton East\", \"LocationID\": 48, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99117738199989, 40.75532984899992], [-73.99730671499998, 40.757110506999894], [-73.98806289599996, 40.769790799999925], [-73.98236545099999, 40.7673921529999], [-73.99117738199989, 40.75532984899992]]]}}, {\"id\": \"48\", \"type\": \"Feature\", \"properties\": {\"count\": 81694, \"OBJECTID\": 49, \"Shape_Leng\": 0.0752900183427, \"Shape_Area\": 0.000203185525471, \"zone\": \"Clinton Hill\", \"LocationID\": 49, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96062056499981, 40.69632013999993], [-73.96015854699988, 40.69411730899989], [-73.95614239299985, 40.694579018999896], [-73.95541057999996, 40.69090829199994], [-73.96029281699998, 40.69034624999995], [-73.95956770199984, 40.686682554999955], [-73.95468418899998, 40.68724485399985], [-73.95328187399996, 40.68020003099985], [-73.95529361599996, 40.679203150999825], [-73.96868961400001, 40.68200525799988], [-73.97013154199992, 40.68950630799986], [-73.96910362499993, 40.68946752199986], [-73.9693945009999, 40.696080388999846], [-73.9645507049999, 40.696107591999855], [-73.96105100699992, 40.69832607899989], [-73.96062056499981, 40.69632013999993]]]}}, {\"id\": \"49\", \"type\": \"Feature\", \"properties\": {\"count\": 1574829, \"OBJECTID\": 50, \"Shape_Leng\": 0.0557479867536, \"Shape_Area\": 0.000173249283326, \"zone\": \"Clinton West\", \"LocationID\": 50, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99667705784736, 40.77351005394221], [-73.98806289599996, 40.769790799999925], [-73.99547450199992, 40.759626856999894], [-74.00375735018837, 40.763582241820224], [-74.0009730802201, 40.763420271725614], [-74.00311291631381, 40.76474559615368], [-73.99915098087908, 40.76420525958265], [-74.0020245433034, 40.76589326657037], [-73.99857638530527, 40.76499860727083], [-74.00157335385273, 40.766977192210646], [-73.9980667590535, 40.76587188113572], [-73.99737808447954, 40.76682338637958], [-74.00064950530414, 40.76824523610559], [-73.99715588949206, 40.76712869617278], [-73.9964571540504, 40.76809881579563], [-73.99973590247859, 40.76950617733152], [-73.99707176039315, 40.768732643576804], [-73.9987620691431, 40.7708250567092], [-73.99631327022087, 40.76979883633644], [-73.99493501658995, 40.77146814657798], [-73.99667705784736, 40.77351005394221]]]}}, {\"id\": \"50\", \"type\": \"Feature\", \"properties\": {\"count\": 749, \"OBJECTID\": 51, \"Shape_Leng\": 0.0953613442277, \"Shape_Area\": 0.000395756553505, \"zone\": \"Co-Op City\", \"LocationID\": 51, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82091617199997, 40.86891693399989], [-73.81578764099999, 40.8657580609999], [-73.82037552999996, 40.86103300499993], [-73.82699653199998, 40.85908778599986], [-73.83315960399985, 40.86831975599994], [-73.83565766499987, 40.86751275599992], [-73.83829221499984, 40.87295133899995], [-73.83924788999988, 40.876618737999905], [-73.8381484219999, 40.876953298999915], [-73.84324113799987, 40.879108912999904], [-73.82387140599998, 40.88778362799996], [-73.82077408699998, 40.886710602999884], [-73.81992586499996, 40.88211933299987], [-73.82329599399986, 40.87309741799988], [-73.82091617199997, 40.86891693399989]]]}}, {\"id\": \"51\", \"type\": \"Feature\", \"properties\": {\"count\": 95260, \"OBJECTID\": 52, \"Shape_Leng\": 0.0289076710093, \"Shape_Area\": 4.52027509603e-05, \"zone\": \"Cobble Hill\", \"LocationID\": 52, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99588110299992, 40.68239495399995], [-74.00004865199998, 40.682803355999965], [-73.99908066099994, 40.68481599199985], [-74.00064737999996, 40.685253946999914], [-73.99776924399993, 40.691194697999954], [-73.99236366999985, 40.689690123999924], [-73.99588110299992, 40.68239495399995]]]}}, {\"id\": \"52\", \"type\": \"Feature\", \"properties\": {\"count\": 997, \"OBJECTID\": 53, \"Shape_Leng\": 0.161500913385, \"Shape_Area\": 0.000947530980821, \"zone\": \"College Point\", \"LocationID\": 53, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.83839013354552, 40.798304872075995], [-73.83919004727183, 40.7977772664477], [-73.84025860809128, 40.79758885843212], [-73.83840083088981, 40.79675525865344], [-73.83663288911686, 40.79251637576927], [-73.84067828300006, 40.79390789199993], [-73.84081532599994, 40.7915048849999], [-73.83895676099984, 40.79143843099996], [-73.84008157499997, 40.78785153099992], [-73.83824429099994, 40.78778591599995], [-73.83869921299997, 40.781676956999874], [-73.82513740299993, 40.78169970599994], [-73.83000011099983, 40.77373240699996], [-73.83913189281712, 40.765991105784146], [-73.83975387982034, 40.767120445440874], [-73.84491332846012, 40.76552838817853], [-73.84780575104425, 40.76677294817163], [-73.84977407793218, 40.769936012942956], [-73.8486715286884, 40.770761572902536], [-73.85130108374476, 40.77180636079771], [-73.84897619867189, 40.77356166795698], [-73.84895559325399, 40.77796167373135], [-73.84963631633588, 40.77948192035157], [-73.85094705724252, 40.77906915543026], [-73.85148755480326, 40.77899297861757], [-73.84981720066014, 40.77963034805418], [-73.85232086242219, 40.77916928470371], [-73.85238832778414, 40.7795289023869], [-73.84982102259761, 40.779651339303875], [-73.84955315550077, 40.779990415384006], [-73.85169028085423, 40.779867508478176], [-73.85167099162219, 40.780035228878106], [-73.84949381208321, 40.780065828114246], [-73.85093646396534, 40.781119575156005], [-73.8492525818285, 40.782264775014035], [-73.85528242786589, 40.78197538434999], [-73.8595236950597, 40.785561954252664], [-73.85284230133867, 40.78821137173992], [-73.85480194733185, 40.788619154725914], [-73.85265830693045, 40.79110642061034], [-73.85377854910695, 40.79242127400702], [-73.8526309825624, 40.79494485424474], [-73.84906160835088, 40.79336261029852], [-73.84855988738983, 40.79549506021838], [-73.84354668938991, 40.79491394401049], [-73.84225959368331, 40.79558496893736], [-73.8428667525936, 40.79692222907191], [-73.84158721174394, 40.795401069221235], [-73.84041517975831, 40.7976784372587], [-73.83839013354552, 40.798304872075995]]], [[[-73.8678840747076, 40.78736103669457], [-73.86813032000018, 40.78723277784419], [-73.8692349173292, 40.78596991495815], [-73.8680991225789, 40.78745866181839], [-73.8678840747076, 40.78736103669457]]]]}}, {\"id\": \"53\", \"type\": \"Feature\", \"properties\": {\"count\": 10701, \"OBJECTID\": 54, \"Shape_Leng\": 0.0762192448554, \"Shape_Area\": 0.000132452130536, \"zone\": \"Columbia Street\", \"LocationID\": 54, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-74.00174362072502, 40.69240674985266], [-74.00400795971088, 40.693208199632345], [-74.00127014951309, 40.69329606183691], [-74.00174362072502, 40.69240674985266]]], [[[-74.00174362072502, 40.69240674985266], [-73.99776924399993, 40.691194697999954], [-74.00329231299999, 40.67964328799992], [-74.00596041099988, 40.683362870999936], [-74.00485241699997, 40.68566813099993], [-74.00783293766679, 40.68738505516274], [-74.00459270521766, 40.688215222986614], [-74.00636461225261, 40.68966966863697], [-74.00382038919723, 40.688929644699236], [-74.0053420962491, 40.6910916029778], [-74.0009615685494, 40.69012876835003], [-74.00479415294552, 40.69176162037443], [-74.00174362072502, 40.69240674985266]]], [[[-74.01092841300002, 40.684491472999824], [-74.01217596614636, 40.68409518562848], [-74.00816320571415, 40.68617364485845], [-74.0086007239999, 40.68590956499989], [-74.01092841300002, 40.684491472999824]]]]}}, {\"id\": \"54\", \"type\": \"Feature\", \"properties\": {\"count\": 994, \"OBJECTID\": 55, \"Shape_Leng\": 0.149956807524, \"Shape_Area\": 0.000736274713586, \"zone\": \"Coney Island\", \"LocationID\": 55, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98671582299981, 40.58456849999991], [-73.97742525299988, 40.58290659299994], [-73.96661531499983, 40.58457103699992], [-73.9688899589999, 40.57526123899986], [-73.95941778299985, 40.57517798299994], [-73.95939182223685, 40.57391210937612], [-73.98367270153267, 40.57140705768737], [-73.98336058039273, 40.56952999448666], [-73.98375790448641, 40.571396227250865], [-74.00208547639728, 40.569585983985014], [-74.00303186164808, 40.57218559475904], [-74.01115668729254, 40.57416676185885], [-74.01302222952468, 40.577804459633505], [-74.0111737229432, 40.58028397882522], [-74.00604734235807, 40.58198365303581], [-73.98814208290418, 40.57886596371379], [-73.98604509916208, 40.58172068515157], [-73.98800973376738, 40.579670687155776], [-73.98995504960881, 40.58066166976562], [-73.98671582299981, 40.58456849999991]]]}}, {\"id\": \"55\", \"type\": \"Feature\", \"properties\": {\"count\": 5740, \"OBJECTID\": 56, \"Shape_Leng\": 0.0568478126677, \"Shape_Area\": 0.000180907844436, \"zone\": \"Corona\", \"LocationID\": 56, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85360568399982, 40.74749682799991], [-73.85011556599989, 40.74072356299995], [-73.84750820299992, 40.73900780699995], [-73.86462193299997, 40.73428275699993], [-73.86790963899986, 40.74493371599989], [-73.85442575299999, 40.748851725999884], [-73.85360568399982, 40.74749682799991]]]}}, {\"id\": \"56\", \"type\": \"Feature\", \"properties\": {\"count\": 5740, \"OBJECTID\": 57, \"Shape_Leng\": 0.0192705048557, \"Shape_Area\": 1.80259807917e-05, \"zone\": \"Corona\", \"LocationID\": 56, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8513110499998, 40.74984337599988], [-73.85442575299999, 40.748851725999884], [-73.85579909099987, 40.75440952799994], [-73.85312635499996, 40.75510934299989], [-73.8513110499998, 40.74984337599988]]]}}, {\"id\": \"57\", \"type\": \"Feature\", \"properties\": {\"count\": 85, \"OBJECTID\": 58, \"Shape_Leng\": 0.0598554094851, \"Shape_Area\": 0.000204980931361, \"zone\": \"Country Club\", \"LocationID\": 58, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.81561382067721, 40.84818955758243], [-73.81602685114903, 40.846400458770034], [-73.81464594697542, 40.84721961352901], [-73.81570353189448, 40.84629134042384], [-73.81368634738199, 40.84727432550905], [-73.81356372861671, 40.847151411278524], [-73.8150262760428, 40.845499995058404], [-73.81284939958525, 40.84631879949822], [-73.81324326350918, 40.84523590835256], [-73.81310050626846, 40.84517241453328], [-73.8128770359758, 40.8451346679459], [-73.81315597044244, 40.84521210037631], [-73.81285536957633, 40.845186285461466], [-73.81241445328128, 40.846559380748744], [-73.81227790900483, 40.846543458279285], [-73.8128577433794, 40.845129544589135], [-73.81254457815619, 40.84497959800826], [-73.81169971823665, 40.84668170081364], [-73.81266141349049, 40.84420764950083], [-73.81254909120709, 40.84495227514607], [-73.8136558066588, 40.8450591224657], [-73.81414462049185, 40.84481817326274], [-73.81437458252545, 40.8430889259994], [-73.81569792948075, 40.8443864281129], [-73.81823722746202, 40.844355041712625], [-73.81473345817534, 40.8413757283108], [-73.81606068217324, 40.83603019622843], [-73.82601252600003, 40.83402077799985], [-73.82624132999996, 40.84299359399991], [-73.82517999, 40.842700058999874], [-73.82495697699986, 40.846105444999864], [-73.82048405599983, 40.84850102599995], [-73.81507165904237, 40.84913361979451], [-73.81561382067721, 40.84818955758243]]]}}, {\"id\": \"58\", \"type\": \"Feature\", \"properties\": {\"count\": 76, \"OBJECTID\": 59, \"Shape_Leng\": 0.0377948070893, \"Shape_Area\": 6.28765230648e-05, \"zone\": \"Crotona Park\", \"LocationID\": 59, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88941724899998, 40.839422732999836], [-73.88659802299989, 40.839279049999895], [-73.89697220299995, 40.8343452539999], [-73.9012157839999, 40.8354101409999], [-73.89666161699988, 40.84352460699992], [-73.88941724899998, 40.839422732999836]]]}}, {\"id\": \"59\", \"type\": \"Feature\", \"properties\": {\"count\": 1062, \"OBJECTID\": 60, \"Shape_Leng\": 0.0786482878226, \"Shape_Area\": 0.000161833284912, \"zone\": \"Crotona Park East\", \"LocationID\": 60, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87823856799989, 40.839985917999854], [-73.87277204699981, 40.83975128199995], [-73.87379590199993, 40.837445132999896], [-73.88005188499987, 40.83470205899989], [-73.88390094800003, 40.82980281599986], [-73.8843985489999, 40.822965575999866], [-73.88753429599987, 40.82250933899995], [-73.88601262699991, 40.828238530999926], [-73.88761307899992, 40.8293374189999], [-73.89717220999982, 40.830368832999966], [-73.89697220299995, 40.8343452539999], [-73.89408527499982, 40.8362973469999], [-73.88730373200002, 40.83812719099994], [-73.88618432099992, 40.840105043999884], [-73.88200384100001, 40.837443781999895], [-73.880061996, 40.840162234999845], [-73.87823856799989, 40.839985917999854]]]}}, {\"id\": \"60\", \"type\": \"Feature\", \"properties\": {\"count\": 57441, \"OBJECTID\": 61, \"Shape_Leng\": 0.117310808097, \"Shape_Area\": 0.000510772092295, \"zone\": \"Crown Heights North\", \"LocationID\": 61, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92916130699993, 40.68343632399992], [-73.92281374299995, 40.68341442999991], [-73.92164666399992, 40.67886992299994], [-73.92275257199998, 40.667097332999894], [-73.92872204599985, 40.66449556899988], [-73.93698362399992, 40.665535507999955], [-73.93674946999998, 40.66812737099991], [-73.96131877999991, 40.67140667599988], [-73.96007428300001, 40.67487807099994], [-73.96318238899983, 40.675529503999876], [-73.96134655899995, 40.6804635239999], [-73.95529361599996, 40.679203150999825], [-73.9533701749999, 40.680640507999904], [-73.93435066499993, 40.67956340799991], [-73.93500821699992, 40.68276376799985], [-73.92916130699993, 40.68343632399992]]]}}, {\"id\": \"61\", \"type\": \"Feature\", \"properties\": {\"count\": 10543, \"OBJECTID\": 62, \"Shape_Leng\": 0.0822593359838, \"Shape_Area\": 0.000158238169004, \"zone\": \"Crown Heights South\", \"LocationID\": 62, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92916752199997, 40.664299531999895], [-73.94282892199992, 40.6628474859999], [-73.94269628999989, 40.664055128999934], [-73.94546293699983, 40.66422610399989], [-73.96095595499996, 40.66328505499992], [-73.96258784899996, 40.671711596999884], [-73.93674946999998, 40.66812737099991], [-73.93698362399992, 40.665535507999955], [-73.92916752199997, 40.664299531999895]]]}}, {\"id\": \"62\", \"type\": \"Feature\", \"properties\": {\"count\": 1432, \"OBJECTID\": 63, \"Shape_Leng\": 0.119100721883, \"Shape_Area\": 0.000353190910142, \"zone\": \"Cypress Hills\", \"LocationID\": 63, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86650407899994, 40.684474051999885], [-73.86602666799992, 40.681918051999936], [-73.86410096700003, 40.68237284999988], [-73.86234580499983, 40.67916478599994], [-73.86106294999989, 40.67434293499984], [-73.87231018000001, 40.67775574699987], [-73.87531627599998, 40.68011098399988], [-73.88061377799994, 40.679110031999876], [-73.88116730300005, 40.68135366299988], [-73.9000462609999, 40.676893089999936], [-73.9040463979999, 40.67922059799985], [-73.90262262299991, 40.6806645029999], [-73.89786299899986, 40.67987501099987], [-73.89417463299992, 40.6852832479999], [-73.89252316799995, 40.683424532999865], [-73.88962787599998, 40.684236453999894], [-73.87402053199992, 40.69419129499995], [-73.86891704399994, 40.69515042299987], [-73.86650407899994, 40.684474051999885]]]}}, {\"id\": \"65\", \"type\": \"Feature\", \"properties\": {\"count\": 66839, \"OBJECTID\": 66, \"Shape_Leng\": 0.0546334593634, \"Shape_Area\": 0.000108378855948, \"zone\": \"DUMBO/Vinegar Hill\", \"LocationID\": 66, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97906084869959, 40.7059460290057], [-73.98050926199988, 40.69744627499991], [-73.98696892299995, 40.698503613999954], [-73.98679382799996, 40.700633222999954], [-73.99093201799984, 40.7007662159999], [-73.99544564312069, 40.70327530547179], [-73.99350881403605, 40.70462350527397], [-73.97906084869959, 40.7059460290057]]]}}, {\"id\": \"63\", \"type\": \"Feature\", \"properties\": {\"count\": 385, \"OBJECTID\": 64, \"Shape_Leng\": 0.18445188474, \"Shape_Area\": 0.00105790284614, \"zone\": \"Douglaston\", \"LocationID\": 64, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.74412314584049, 40.77863865777706], [-73.71351828186042, 40.75983773091619], [-73.72031595299998, 40.75789725799992], [-73.72771356199995, 40.750626894999904], [-73.7310646189999, 40.744888375999885], [-73.74016951799993, 40.73848189799986], [-73.74306820799991, 40.7384408179999], [-73.74396193599988, 40.740789125999925], [-73.75098230100001, 40.740021462999884], [-73.75347299999999, 40.74240484399992], [-73.74477587099986, 40.74328109699994], [-73.74625984399985, 40.74671147499993], [-73.74384322099985, 40.74769009599988], [-73.75004764499985, 40.75840465799996], [-73.75230259399993, 40.759455198999916], [-73.75583713299987, 40.757742061999934], [-73.75921689799992, 40.76432485599988], [-73.75377036482259, 40.76530463174143], [-73.7510743225255, 40.76176367133622], [-73.74455864030779, 40.756557885832535], [-73.75507210849031, 40.76753310678742], [-73.7555733264932, 40.77152912592269], [-73.75336691999294, 40.77320784776336], [-73.75551737581792, 40.77770404836418], [-73.75080593606877, 40.782893378833954], [-73.74412314584049, 40.77863865777706]]]}}, {\"id\": \"64\", \"type\": \"Feature\", \"properties\": {\"count\": 202755, \"OBJECTID\": 65, \"Shape_Leng\": 0.0446070683658, \"Shape_Area\": 8.1803882541e-05, \"zone\": \"Downtown Brooklyn/MetroTech\", \"LocationID\": 65, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98712491499988, 40.70063447999997], [-73.98696892299995, 40.698503613999954], [-73.98050926199988, 40.69744627499991], [-73.97916718399999, 40.69348832299992], [-73.98251122999989, 40.69361433399992], [-73.98147568199991, 40.689930943999876], [-73.99097186899995, 40.69254879999995], [-73.99093201799984, 40.7007662159999], [-73.98712491499988, 40.70063447999997]]]}}, {\"id\": \"66\", \"type\": \"Feature\", \"properties\": {\"count\": 1356, \"OBJECTID\": 67, \"Shape_Leng\": 0.0997470781551, \"Shape_Area\": 0.000394282272487, \"zone\": \"Dyker Heights\", \"LocationID\": 67, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.0109702639999, 40.63068186899988], [-74.00702302499997, 40.63210999499996], [-73.99724067099987, 40.62528868699994], [-73.99771358999998, 40.62368026999996], [-73.99932090299998, 40.62465524699996], [-74.01701910199995, 40.60765441699994], [-74.01549854999998, 40.606841523999925], [-74.0186913569999, 40.60360792399995], [-74.02129949199998, 40.60442832299987], [-74.02078540899996, 40.60567368599989], [-74.02247769499994, 40.60667421599987], [-74.02150602899994, 40.60782209899992], [-74.02710660599995, 40.61024395599991], [-74.02637609099996, 40.61245060299993], [-74.02754297299998, 40.61293718299988], [-74.01970491599991, 40.6221849649999], [-74.01514630699995, 40.63322193799993], [-74.0109702639999, 40.63068186899988]]]}}, {\"id\": \"67\", \"type\": \"Feature\", \"properties\": {\"count\": 3718064, \"OBJECTID\": 68, \"Shape_Leng\": 0.0493373795569, \"Shape_Area\": 0.000111073378655, \"zone\": \"East Chelsea\", \"LocationID\": 68, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00201937799989, 40.74044408899989], [-74.00820401899988, 40.74214751799986], [-73.99730671499998, 40.757110506999894], [-73.99163271299996, 40.75470680499996], [-74.00201937799989, 40.74044408899989]]]}}, {\"id\": \"68\", \"type\": \"Feature\", \"properties\": {\"count\": 6247, \"OBJECTID\": 69, \"Shape_Leng\": 0.0789896450483, \"Shape_Area\": 0.00019905160381, \"zone\": \"East Concourse/Concourse Village\", \"LocationID\": 69, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91179361999997, 40.8264281559999], [-73.91130974499993, 40.824848557999886], [-73.91541108199998, 40.82577390899993], [-73.92518486499998, 40.818012668999884], [-73.92731653099999, 40.81855963399991], [-73.91193751499992, 40.843202387999916], [-73.90514893999989, 40.84254719499995], [-73.91049010699992, 40.830737372999884], [-73.908959903, 40.83022623299989], [-73.91179361999997, 40.8264281559999]]]}}, {\"id\": \"69\", \"type\": \"Feature\", \"properties\": {\"count\": 23915, \"OBJECTID\": 70, \"Shape_Leng\": 0.0638403183367, \"Shape_Area\": 0.000195458476728, \"zone\": \"East Elmhurst\", \"LocationID\": 70, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85912375899987, 40.76127549899994], [-73.85449593299987, 40.75827863899985], [-73.87513874300004, 40.756633815999905], [-73.87608949699984, 40.77153690299987], [-73.868098735, 40.77017200899994], [-73.85912375899987, 40.76127549899994]]]}}, {\"id\": \"70\", \"type\": \"Feature\", \"properties\": {\"count\": 2060, \"OBJECTID\": 71, \"Shape_Leng\": 0.0931694551828, \"Shape_Area\": 0.000382572587364, \"zone\": \"East Flatbush/Farragut\", \"LocationID\": 71, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93177153199984, 40.65527364199989], [-73.92910623099985, 40.63979510599992], [-73.92574389399985, 40.64001446599985], [-73.92652490899994, 40.63649272299989], [-73.92744822299987, 40.635258409999864], [-73.9372670019998, 40.634650624999885], [-73.93701746200003, 40.63231097899989], [-73.94462885499995, 40.63183773499993], [-73.94537272499998, 40.638786954999865], [-73.94826499699982, 40.63860718999994], [-73.94841343400003, 40.639987463999944], [-73.94778994699976, 40.643305735999924], [-73.94586879199989, 40.64342374399988], [-73.94715896299984, 40.65559378699993], [-73.931907438, 40.656537652999916], [-73.93177153199984, 40.65527364199989]]]}}, {\"id\": \"71\", \"type\": \"Feature\", \"properties\": {\"count\": 2326, \"OBJECTID\": 72, \"Shape_Leng\": 0.0952020695166, \"Shape_Area\": 0.000323532461947, \"zone\": \"East Flatbush/Remsen Village\", \"LocationID\": 72, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92021313099984, 40.65997544099987], [-73.9195947979998, 40.66199567899985], [-73.91236458899995, 40.65551356499985], [-73.91012255699995, 40.655851770999945], [-73.90855790499995, 40.65209593799989], [-73.91995062999999, 40.64470762999989], [-73.91973736999988, 40.64277814799987], [-73.92332654599986, 40.63990021599991], [-73.92356525899983, 40.642499297999905], [-73.92538233099987, 40.64238453199987], [-73.92574389499987, 40.64001446599983], [-73.92910623199991, 40.6397951059999], [-73.93190743899989, 40.656537652999944], [-73.92799465199988, 40.65680210099988], [-73.92834017599988, 40.660060035999884], [-73.92646424299983, 40.659814361999956], [-73.93068465499985, 40.66362047699992], [-73.92637970499992, 40.6655148919999], [-73.92021313099984, 40.65997544099987]]]}}, {\"id\": \"72\", \"type\": \"Feature\", \"properties\": {\"count\": 446, \"OBJECTID\": 73, \"Shape_Leng\": 0.0853020209129, \"Shape_Area\": 0.000291799754395, \"zone\": \"East Flushing\", \"LocationID\": 73, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79566545000003, 40.75023619999991], [-73.79729524799988, 40.74943505299992], [-73.7946756879998, 40.74721192799994], [-73.8033521419999, 40.74643196499987], [-73.80287666499987, 40.749765148999856], [-73.80750599299991, 40.750136586999936], [-73.81416172099998, 40.74613857999986], [-73.820849288, 40.75224154499988], [-73.8184078289999, 40.75383042199998], [-73.82030067699999, 40.75551497999992], [-73.81789245699991, 40.75708633799991], [-73.82063593299985, 40.75887226799989], [-73.81104963199985, 40.761417020999964], [-73.79940088899988, 40.759397218999936], [-73.79493246199993, 40.75779802999994], [-73.79566545000003, 40.75023619999991]]]}}, {\"id\": \"73\", \"type\": \"Feature\", \"properties\": {\"count\": 465525, \"OBJECTID\": 74, \"Shape_Leng\": 0.11029093625, \"Shape_Area\": 0.000295038418204, \"zone\": \"East Harlem North\", \"LocationID\": 74, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.93380589881859, 40.81651249324708], [-73.93412073484292, 40.80865645197743], [-73.92903490240305, 40.80108090385195], [-73.9290364017962, 40.79676259410132], [-73.93505300002766, 40.79168720295447], [-73.93773612099982, 40.792815803999936], [-73.93864883299987, 40.79155775499987], [-73.94327251199996, 40.79348810299993], [-73.94187604699994, 40.79540586899984], [-73.94831125500004, 40.79812921799992], [-73.94459751299998, 40.803228144999906], [-73.94613131, 40.80387735799992], [-73.94430194599991, 40.806390824999916], [-73.93854407899988, 40.80572965299991], [-73.93580780199986, 40.80949763799987], [-73.93901753299998, 40.81085655699991], [-73.93407463999996, 40.81782651399992], [-73.93440039999986, 40.81966778299989], [-73.93380589881859, 40.81651249324708]]], [[[-73.92688606022001, 40.800840648368855], [-73.92650954883642, 40.80065703126224], [-73.92704640297914, 40.800651743031324], [-73.92688606022001, 40.800840648368855]]], [[[-73.92666565490181, 40.80018271401784], [-73.92669572614771, 40.80014697707624], [-73.9272477078927, 40.800392957368906], [-73.92711464230626, 40.80056375622957], [-73.92666565490181, 40.80018271401784]]], [[[-73.92672091168967, 40.800414210943586], [-73.92653846445016, 40.80033389815394], [-73.92674263892977, 40.800403498972756], [-73.92672091168967, 40.800414210943586]]], [[[-73.92718514789182, 40.797349896890104], [-73.92767122620857, 40.79721701426623], [-73.92770694127064, 40.7972986638821], [-73.92738263624496, 40.797377063580456], [-73.92718514789182, 40.797349896890104]]]]}}, {\"id\": \"74\", \"type\": \"Feature\", \"properties\": {\"count\": 864875, \"OBJECTID\": 75, \"Shape_Leng\": 0.0876638997685, \"Shape_Area\": 0.00024056392715, \"zone\": \"East Harlem South\", \"LocationID\": 75, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94103496899987, 40.79254666899993], [-73.93505300002766, 40.79168720295447], [-73.93708104382307, 40.78936639571], [-73.93621859869943, 40.78880049757431], [-73.94354420706223, 40.78288052416259], [-73.9557773589999, 40.78791392399995], [-73.94831125500004, 40.79812921799992], [-73.94187604699994, 40.79540586899984], [-73.94327251199996, 40.79348810299993], [-73.94103496899987, 40.79254666899993]]]}}, {\"id\": \"75\", \"type\": \"Feature\", \"properties\": {\"count\": 4552, \"OBJECTID\": 76, \"Shape_Leng\": 0.241203016269, \"Shape_Area\": 0.00126660143241, \"zone\": \"East New York\", \"LocationID\": 76, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88063900599983, 40.679176984999856], [-73.87507136399991, 40.68003514599991], [-73.87231018000001, 40.67775574699987], [-73.86106294999989, 40.67434293499984], [-73.86038937900003, 40.67126877499985], [-73.85763323199998, 40.67165619399985], [-73.85568461199988, 40.66386749199993], [-73.85842950899992, 40.66345335999987], [-73.85761052799985, 40.660108355999924], [-73.86317083299996, 40.65827651199994], [-73.86327623068586, 40.656941215328104], [-73.85695870295893, 40.65058737348713], [-73.85840315341014, 40.64700390123735], [-73.85716978298207, 40.64367855888597], [-73.86289463224384, 40.64255067968245], [-73.86670411973783, 40.6400217634955], [-73.86563247391335, 40.63878676362291], [-73.86867282895084, 40.640925079203356], [-73.87061492461066, 40.64485674188673], [-73.87971979034293, 40.65437592977636], [-73.86989583386247, 40.63893452891954], [-73.86799709668125, 40.63811526520958], [-73.86960972136559, 40.63708553460871], [-73.87347939154502, 40.63628554579879], [-73.87841589406925, 40.638912189353306], [-73.8745808109999, 40.64561010599992], [-73.88221364899994, 40.65451863799996], [-73.89000589199988, 40.651148696999925], [-73.89354677099988, 40.65520271399988], [-73.89761013299987, 40.65345381599992], [-73.89586154900002, 40.65188052699987], [-73.90398200599991, 40.64662856799988], [-73.90944846399994, 40.65152159899992], [-73.90740035699999, 40.6528439899999], [-73.90784346399994, 40.65459152999988], [-73.89612587299993, 40.657814663999936], [-73.88185557299985, 40.66414998799994], [-73.88330883399975, 40.66611200499991], [-73.889906696, 40.66377828999986], [-73.89087805100002, 40.671318086999875], [-73.89271090700004, 40.67104582799992], [-73.89367036399987, 40.67482177099988], [-73.89930675399987, 40.67398204199991], [-73.90004626199993, 40.67689308999995], [-73.88116730300005, 40.68135366299988], [-73.88063900599983, 40.679176984999856]]]}}, {\"id\": \"76\", \"type\": \"Feature\", \"properties\": {\"count\": 1309, \"OBJECTID\": 77, \"Shape_Leng\": 0.075461081444, \"Shape_Area\": 0.000191917981315, \"zone\": \"East New York/Pennsylvania Avenue\", \"LocationID\": 77, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8997089949999, 40.67549425099991], [-73.89930675399987, 40.67398204199991], [-73.89367036399987, 40.67482177099988], [-73.89271090699994, 40.671045828999915], [-73.89087805100002, 40.671318086999875], [-73.889906696, 40.66377828999986], [-73.88330883399975, 40.66611200499991], [-73.88185557299985, 40.66414998799994], [-73.89880698499995, 40.65740666199988], [-73.90066118699986, 40.66058615799992], [-73.90347422300006, 40.67550668299985], [-73.90004626199993, 40.67689308999995], [-73.8997089949999, 40.67549425099991]]]}}, {\"id\": \"77\", \"type\": \"Feature\", \"properties\": {\"count\": 1378, \"OBJECTID\": 78, \"Shape_Leng\": 0.0935944495806, \"Shape_Area\": 0.000191114419551, \"zone\": \"East Tremont\", \"LocationID\": 78, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88963328299991, 40.85317381799993], [-73.88987871399985, 40.85207022099993], [-73.8773292409999, 40.84398312499995], [-73.878171756, 40.84262443699987], [-73.87459175399982, 40.841520647999914], [-73.87109207899995, 40.84432478099992], [-73.86920944599997, 40.84923837199993], [-73.86785992999985, 40.84886493099987], [-73.86804200399997, 40.8448138389999], [-73.87359713599987, 40.83979855499992], [-73.880061996, 40.840162234999845], [-73.88200384100001, 40.837443781999895], [-73.88618432099993, 40.8401050449999], [-73.88941724899988, 40.83942273399985], [-73.89537836099996, 40.842938778999944], [-73.89290102299996, 40.84626832999992], [-73.89617712999998, 40.8466583209999], [-73.89307140900002, 40.85324998699989], [-73.89038953999983, 40.85468905799996], [-73.88963328299991, 40.85317381799993]]]}}, {\"id\": \"78\", \"type\": \"Feature\", \"properties\": {\"count\": 4822927, \"OBJECTID\": 79, \"Shape_Leng\": 0.0426249113144, \"Shape_Area\": 0.000107893068218, \"zone\": \"East Village\", \"LocationID\": 79, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98377990999991, 40.72153595399992], [-73.99260322199994, 40.72413644999987], [-73.98990295999991, 40.7344347899999], [-73.97802697999985, 40.729433059999984], [-73.98377990999991, 40.72153595399992]]]}}, {\"id\": \"79\", \"type\": \"Feature\", \"properties\": {\"count\": 151054, \"OBJECTID\": 80, \"Shape_Leng\": 0.117212621448, \"Shape_Area\": 0.00040732245622, \"zone\": \"East Williamsburg\", \"LocationID\": 80, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93174841322003, 40.728053727696675], [-73.92849418540158, 40.72661460330786], [-73.92453854985803, 40.719342580204575], [-73.93220607307306, 40.71496398092936], [-73.93124067059496, 40.71372481088467], [-73.93328349320237, 40.71097870745788], [-73.9319010192085, 40.71133847142828], [-73.9303609456579, 40.7087177385352], [-73.93184868877432, 40.71256477418327], [-73.93042294313065, 40.71317255867104], [-73.93105231901723, 40.71461022392334], [-73.92460339476072, 40.717953306073376], [-73.92305500014334, 40.71634266479336], [-73.92488553507611, 40.71526718765259], [-73.92433589757124, 40.71412259346647], [-73.92074519699985, 40.71052968599989], [-73.93229708600003, 40.70774944699988], [-73.93480281900003, 40.713372495999884], [-73.94378799599994, 40.710900378999945], [-73.94273448399997, 40.70447330599988], [-73.9408928719998, 40.70391415699989], [-73.94042057899993, 40.701076597999865], [-73.94193078899983, 40.70072523399989], [-73.95023693799996, 40.705473245999876], [-73.95087492699996, 40.70951311499991], [-73.94865533499987, 40.70972498299988], [-73.94937231899985, 40.71406687799986], [-73.95348472199983, 40.71407020799988], [-73.93174841322003, 40.728053727696675]]]}}, {\"id\": \"80\", \"type\": \"Feature\", \"properties\": {\"count\": 427, \"OBJECTID\": 81, \"Shape_Leng\": 0.197592771731, \"Shape_Area\": 0.00039956655019, \"zone\": \"Eastchester\", \"LocationID\": 81, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8236684811033, 40.889903740147645], [-73.82285335053221, 40.8912068290476], [-73.79300652624502, 40.88335873744268], [-73.81209659199986, 40.88671500999993], [-73.82027432999985, 40.885550699999925], [-73.82387140599998, 40.88778362799996], [-73.84324113799987, 40.879108912999904], [-73.8381484219999, 40.876953298999915], [-73.83924788999988, 40.876618737999905], [-73.83829221499984, 40.87295133899995], [-73.83565766499987, 40.86751275599992], [-73.83315960399985, 40.86831975599994], [-73.82834347399992, 40.86089202599988], [-73.85363638799987, 40.87330059899996], [-73.8506895569999, 40.87548667799993], [-73.85254854099988, 40.877091586999924], [-73.85091896099982, 40.877893440999856], [-73.85248305800003, 40.87946370499989], [-73.84704957999995, 40.8876718159999], [-73.83554853999995, 40.88913350399993], [-73.83737956161106, 40.893924741304346], [-73.8236684811033, 40.889903740147645]]]}}, {\"id\": \"81\", \"type\": \"Feature\", \"properties\": {\"count\": 51345, \"OBJECTID\": 82, \"Shape_Leng\": 0.119875649697, \"Shape_Area\": 0.000323601079994, \"zone\": \"Elmhurst\", \"LocationID\": 82, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86768504899995, 40.744071717999915], [-73.86447086199989, 40.73400727399998], [-73.88720524999994, 40.727792996999874], [-73.8885228599999, 40.734531159999904], [-73.87752195799992, 40.731593168999915], [-73.87708276299998, 40.73290931699991], [-73.88066188999987, 40.73570787399994], [-73.88033059399996, 40.73733367699989], [-73.88230252700001, 40.737707700999934], [-73.87743058400002, 40.73860051599991], [-73.88100387999987, 40.74184066199995], [-73.88551225099985, 40.74016501899988], [-73.887511321, 40.74108252699992], [-73.88798650799986, 40.743548191999935], [-73.89071647499993, 40.74312778399992], [-73.89175048799991, 40.7468117269999], [-73.86942458599987, 40.74915685199991], [-73.86768504899995, 40.744071717999915]]]}}, {\"id\": \"82\", \"type\": \"Feature\", \"properties\": {\"count\": 24587, \"OBJECTID\": 83, \"Shape_Leng\": 0.105984933269, \"Shape_Area\": 0.000217463718718, \"zone\": \"Elmhurst/Maspeth\", \"LocationID\": 83, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89111026199994, 40.745202717999874], [-73.89071647499993, 40.74312778399992], [-73.88798650799986, 40.743548191999935], [-73.887511321, 40.74108252699992], [-73.88551225099985, 40.74016501899988], [-73.88100387999987, 40.74184066199995], [-73.87791831799989, 40.739394956999874], [-73.88230252700001, 40.737707700999934], [-73.88033059399996, 40.73733367699989], [-73.88066188999987, 40.73570787399994], [-73.87725122499988, 40.7317941529999], [-73.88910318999984, 40.73459239599987], [-73.89854772799987, 40.735191909999855], [-73.90107970900003, 40.73391463399987], [-73.90516948899983, 40.73585964299986], [-73.90628903499987, 40.732400751999876], [-73.91068876399997, 40.733471081999916], [-73.90926002999988, 40.736403179999854], [-73.90057825399985, 40.73949289599988], [-73.89524819599988, 40.74381972999992], [-73.89624245599988, 40.74871954499987], [-73.89139145499985, 40.74664100399984], [-73.89111026199994, 40.745202717999874]]]}}, {\"id\": \"83\", \"type\": \"Feature\", \"properties\": {\"count\": 29, \"OBJECTID\": 84, \"Shape_Leng\": 0.233623987032, \"Shape_Area\": 0.00207375572052, \"zone\": \"Eltingville/Annadale/Prince's Bay\", \"LocationID\": 84, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.16005732699996, 40.527635041999886], [-74.1720632017547, 40.52332056936472], [-74.17756306042942, 40.519217618159864], [-74.18104081208887, 40.520705036287154], [-74.1848311647976, 40.519341358600826], [-74.19521339999068, 40.5099776529556], [-74.19967205357709, 40.511426850497486], [-74.19936127407836, 40.51301019443217], [-74.20821982874101, 40.51161821258057], [-74.21754755095681, 40.50336004949062], [-74.22153919955196, 40.50250347421612], [-74.22690993399992, 40.51917139799985], [-74.22379595899992, 40.518196345999876], [-74.21354593499994, 40.521235714999925], [-74.21409806799993, 40.52409719499991], [-74.20152661599995, 40.526265205999934], [-74.19174368799999, 40.53191669699986], [-74.19377169299999, 40.53583384699993], [-74.18903145199995, 40.53815934199989], [-74.18128532999992, 40.54695757299989], [-74.17185166499996, 40.56152092299991], [-74.16982582400001, 40.56109042099991], [-74.16540080799992, 40.54443272299989], [-74.15397419799994, 40.53426814799993], [-74.16134961699994, 40.5293734239999], [-74.16005732699996, 40.527635041999886]]]}}, {\"id\": \"84\", \"type\": \"Feature\", \"properties\": {\"count\": 4670, \"OBJECTID\": 85, \"Shape_Leng\": 0.0574751473562, \"Shape_Area\": 0.000143637804933, \"zone\": \"Erasmus\", \"LocationID\": 85, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94715896299984, 40.65559378699993], [-73.94586879199989, 40.64342374399988], [-73.94778994699976, 40.643305735999924], [-73.94826499600005, 40.63860718999994], [-73.9536000469999, 40.63842234699991], [-73.95763644599988, 40.64268949899989], [-73.9587078989999, 40.65038727299989], [-73.94957113100006, 40.65080789999991], [-73.9500666489999, 40.65542299699989], [-73.94715896299984, 40.65559378699993]]]}}, {\"id\": \"85\", \"type\": \"Feature\", \"properties\": {\"count\": 286, \"OBJECTID\": 86, \"Shape_Leng\": 0.134245282582, \"Shape_Area\": 0.000623278815249, \"zone\": \"Far Rockaway\", \"LocationID\": 86, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76669071688973, 40.61422662157113], [-73.76505961679173, 40.612035230610466], [-73.76023635756965, 40.610820864950284], [-73.76178242960594, 40.60915167005445], [-73.76155051905889, 40.60903136580077], [-73.75998446974822, 40.61087937784668], [-73.7556083701559, 40.61007791963748], [-73.7457615018933, 40.611991654374336], [-73.73815143112287, 40.60271044008338], [-73.73808925304178, 40.59768829704553], [-73.74183083531219, 40.59669555145679], [-73.73812220887619, 40.597386102069386], [-73.73763679389783, 40.594415399837956], [-73.74664920374843, 40.594280895008744], [-73.75352988157324, 40.59094648500059], [-73.76376223423699, 40.591439709016804], [-73.76440183099987, 40.59457168299989], [-73.76277845599988, 40.59475654399989], [-73.7633344969999, 40.59782287399992], [-73.76128753399995, 40.59954927099993], [-73.76654204999997, 40.599434441999904], [-73.76771811799993, 40.596261309999925], [-73.76795043655434, 40.5981585114185], [-73.77098168371995, 40.59889409104954], [-73.76895714023856, 40.60927991805636], [-73.77417180232338, 40.61182871527586], [-73.77339803484179, 40.61368296748421], [-73.76669071688973, 40.61422662157113]]]}}, {\"id\": \"86\", \"type\": \"Feature\", \"properties\": {\"count\": 1153199, \"OBJECTID\": 87, \"Shape_Leng\": 0.0369015496117, \"Shape_Area\": 6.72050210692e-05, \"zone\": \"Financial District North\", \"LocationID\": 87, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00372234899996, 40.70693234399989], [-74.00197014837809, 40.704731159202304], [-74.00332307270776, 40.70562859522001], [-74.00459003883662, 40.70478775294495], [-74.00349632042315, 40.70379676926777], [-74.0053502722475, 40.70431007996148], [-74.004272587521, 40.70301566641148], [-74.00659591913875, 40.70368590245183], [-74.00530412596079, 40.70255986563306], [-74.00550166067457, 40.70243086779703], [-74.01250773999998, 40.706767065999934], [-74.00906264799988, 40.71088670399992], [-74.00372234899996, 40.70693234399989]]]}}, {\"id\": \"87\", \"type\": \"Feature\", \"properties\": {\"count\": 542234, \"OBJECTID\": 88, \"Shape_Leng\": 0.0352046035327, \"Shape_Area\": 5.72999455414e-05, \"zone\": \"Financial District South\", \"LocationID\": 88, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00974639299997, 40.704775247999905], [-74.00760498189766, 40.70299314132918], [-74.00930907655483, 40.70195460302249], [-74.00759136204225, 40.70132808104815], [-74.01367479307396, 40.70012582122537], [-74.01428922899998, 40.70454907499989], [-74.01250773999998, 40.706767065999934], [-74.00974639299997, 40.704775247999905]]]}}, {\"id\": \"88\", \"type\": \"Feature\", \"properties\": {\"count\": 18697, \"OBJECTID\": 89, \"Shape_Leng\": 0.122794569553, \"Shape_Area\": 0.000447548142373, \"zone\": \"Flatbush/Ditmas Park\", \"LocationID\": 89, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95374853799996, 40.63858633799993], [-73.94537272499998, 40.638786954999865], [-73.94462885499995, 40.63183773499993], [-73.94647961799998, 40.631708161999896], [-73.94415535299993, 40.629508284999886], [-73.95995565099992, 40.627778671999884], [-73.96040734299989, 40.6301625309999], [-73.96232599399991, 40.6299687169999], [-73.97136622199992, 40.62892916199996], [-73.97092126300004, 40.62656998499993], [-73.97623453700001, 40.625976349999895], [-73.97716511999984, 40.63074665399988], [-73.96986869299992, 40.63419467499989], [-73.97029036199991, 40.63640634099988], [-73.96803868499985, 40.636650984999925], [-73.97139631700004, 40.64825778599986], [-73.96465438699984, 40.650887727999894], [-73.96621321999996, 40.653189404999885], [-73.95986103199981, 40.65563478899989], [-73.95763644599988, 40.64268949899989], [-73.95374853799996, 40.63858633799993]]]}}, {\"id\": \"89\", \"type\": \"Feature\", \"properties\": {\"count\": 2871189, \"OBJECTID\": 90, \"Shape_Leng\": 0.0307591620819, \"Shape_Area\": 5.53132476305e-05, \"zone\": \"Flatiron\", \"LocationID\": 90, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9963368949999, 40.7380437469999], [-74.00252381499989, 40.73975264399995], [-73.99709902899994, 40.74720510199991], [-73.99141858599994, 40.74480268199988], [-73.9963368949999, 40.7380437469999]]]}}, {\"id\": \"90\", \"type\": \"Feature\", \"properties\": {\"count\": 2596, \"OBJECTID\": 91, \"Shape_Leng\": 0.124996624585, \"Shape_Area\": 0.000537330013243, \"zone\": \"Flatlands\", \"LocationID\": 91, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91945472999996, 40.6401731369999], [-73.91690302699992, 40.61627422899993], [-73.9205269409999, 40.61394269499987], [-73.92640044599996, 40.61447853499988], [-73.92629604599998, 40.61311679899991], [-73.93686699599992, 40.620253280999904], [-73.94573691999982, 40.61536340699989], [-73.94651373199997, 40.61949401899995], [-73.94451833400001, 40.61971364699989], [-73.94732672199994, 40.629166566999835], [-73.94415535299993, 40.629508284999886], [-73.94647961799998, 40.631708161999896], [-73.93701746200003, 40.63231097899989], [-73.9372670019998, 40.634650624999885], [-73.92744822299987, 40.635258409999864], [-73.92538233099987, 40.64238453199987], [-73.92356525899983, 40.642499297999905], [-73.92332654599986, 40.63990021599991], [-73.91973736999988, 40.64277814799987], [-73.91945472999996, 40.6401731369999]]]}}, {\"id\": \"91\", \"type\": \"Feature\", \"properties\": {\"count\": 9309, \"OBJECTID\": 92, \"Shape_Leng\": 0.117830066799, \"Shape_Area\": 0.000374946617289, \"zone\": \"Flushing\", \"LocationID\": 92, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82656578899986, 40.77064147599989], [-73.82466908999987, 40.77040760799985], [-73.82080684899987, 40.7589417829999], [-73.81789245699991, 40.75708633799991], [-73.82030067699999, 40.75551497999992], [-73.8184078289999, 40.75383042199998], [-73.820849288, 40.75224154499988], [-73.81575069499985, 40.7469794799999], [-73.82108427599998, 40.748558445999954], [-73.82426493900002, 40.75183288899994], [-73.82584440999999, 40.74934839099993], [-73.83300302099987, 40.74783363099988], [-73.8322953569999, 40.751941790999844], [-73.83560230899995, 40.75139485299991], [-73.83805974199996, 40.75466660699988], [-73.83689970199995, 40.7626981179999], [-73.8394581342306, 40.76544576822929], [-73.83000011099983, 40.77373240699996], [-73.82513740299993, 40.78169970599994], [-73.82224127699999, 40.781693878999924], [-73.82583089999997, 40.77591527099986], [-73.82656578899986, 40.77064147599989]]]}}, {\"id\": \"92\", \"type\": \"Feature\", \"properties\": {\"count\": 14054, \"OBJECTID\": 93, \"Shape_Leng\": 0.170998027597, \"Shape_Area\": 0.000594278183978, \"zone\": \"Flushing Meadows-Corona Park\", \"LocationID\": 93, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8569635593979, 40.76407442786308], [-73.8566748236235, 40.763930188326135], [-73.85758285470693, 40.76307290062661], [-73.8565867538878, 40.762722840114236], [-73.85817631070509, 40.7623962942238], [-73.85661195813331, 40.761649542759294], [-73.8585852628444, 40.76191700859695], [-73.85747662435531, 40.76130891668733], [-73.85158552901191, 40.75935542065696], [-73.84930110814332, 40.76024287534997], [-73.85207200177854, 40.7603928846721], [-73.84670933714987, 40.76134526931977], [-73.84324381399976, 40.75533680399995], [-73.83867321899987, 40.75701997899995], [-73.83560230899995, 40.75139485299991], [-73.8322953569999, 40.751941790999844], [-73.83300302099987, 40.74783363099988], [-73.83699037399995, 40.74209330099992], [-73.8361277859999, 40.734241311999924], [-73.83073214599987, 40.72654932799991], [-73.8273090039999, 40.72464607699988], [-73.82531644999993, 40.71727569999994], [-73.81729568999994, 40.71757964499995], [-73.83145866999996, 40.714554448999934], [-73.8297961659998, 40.71556571899986], [-73.83488359399998, 40.719303271999934], [-73.85436928099995, 40.748738929999874], [-73.85086439699992, 40.74999564199986], [-73.85379813799992, 40.757177611999914], [-73.86109724399991, 40.76366447799989], [-73.85861915281858, 40.7619355969989], [-73.8569635593979, 40.76407442786308]]]}}, {\"id\": \"93\", \"type\": \"Feature\", \"properties\": {\"count\": 957, \"OBJECTID\": 94, \"Shape_Leng\": 0.0498326083147, \"Shape_Area\": 6.25819407393e-05, \"zone\": \"Fordham South\", \"LocationID\": 94, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89964261700001, 40.862210436999874], [-73.897143798, 40.86244506499991], [-73.89883814599983, 40.859307307999906], [-73.89573924699978, 40.85813820699986], [-73.89106059999996, 40.861471814999945], [-73.89560883299981, 40.85457036999986], [-73.89863543699985, 40.85390697499993], [-73.89816577499982, 40.85514637599989], [-73.90077922699986, 40.85604710999993], [-73.90185996099993, 40.85458614899994], [-73.90557007299988, 40.855787321999884], [-73.90351390599986, 40.86091750999989], [-73.89964261700001, 40.862210436999874]]]}}, {\"id\": \"94\", \"type\": \"Feature\", \"properties\": {\"count\": 27893, \"OBJECTID\": 95, \"Shape_Leng\": 0.113629605476, \"Shape_Area\": 0.00057277033803, \"zone\": \"Forest Hills\", \"LocationID\": 95, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84732494199989, 40.738771457999896], [-73.83488359399998, 40.719303271999934], [-73.8297961659998, 40.71556571899986], [-73.8377089419999, 40.71103648399989], [-73.85593991699984, 40.7063980339999], [-73.8586809759999, 40.72261243999988], [-73.86203833799982, 40.72595678499994], [-73.85805416899993, 40.72444540599986], [-73.85727108100001, 40.72568129799988], [-73.8594198799999, 40.726464837999885], [-73.85741240699991, 40.72832247999986], [-73.85851097899999, 40.73028680199993], [-73.85625665099991, 40.73095912299993], [-73.85872465799989, 40.73570864399988], [-73.84732494199989, 40.738771457999896]]]}}, {\"id\": \"95\", \"type\": \"Feature\", \"properties\": {\"count\": 565, \"OBJECTID\": 96, \"Shape_Leng\": 0.185180248095, \"Shape_Area\": 0.000547566253402, \"zone\": \"Forest Park/Highland Park\", \"LocationID\": 96, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83804164799996, 40.70832329499994], [-73.83696237799982, 40.70421245199992], [-73.84502472899999, 40.6993593769999], [-73.84913837299978, 40.69828274099995], [-73.85070842300001, 40.70020312499988], [-73.85443455699989, 40.70060295199994], [-73.85277238699999, 40.69755711199987], [-73.874020533, 40.69419129499994], [-73.88962787599998, 40.684236453999894], [-73.89252316899987, 40.683424532999865], [-73.89417463299992, 40.6852832479999], [-73.89646625099995, 40.68233642199988], [-73.90116155099986, 40.687877934999946], [-73.90180467199995, 40.6907662979999], [-73.9003866789998, 40.69313995999992], [-73.89736265799978, 40.6941108469999], [-73.89695653499987, 40.6919492069999], [-73.89326209899984, 40.69220094699994], [-73.8950769439999, 40.694591389999886], [-73.88642048199995, 40.696458148999945], [-73.88771297999986, 40.699678075999906], [-73.88280948499987, 40.7006834549999], [-73.88182690599986, 40.69952929899987], [-73.87639785199997, 40.70229404199989], [-73.86285374199994, 40.7034144919999], [-73.85819890699996, 40.70291107399986], [-73.85893224399979, 40.70132034499989], [-73.85725351500002, 40.70121217699989], [-73.85252019099998, 40.70229172399997], [-73.85473240499985, 40.70676414299989], [-73.84314625299982, 40.70922345599992], [-73.83804164799996, 40.70832329499994]]]}}, {\"id\": \"96\", \"type\": \"Feature\", \"properties\": {\"count\": 155921, \"OBJECTID\": 97, \"Shape_Leng\": 0.0624760147423, \"Shape_Area\": 0.000163303970435, \"zone\": \"Fort Greene\", \"LocationID\": 97, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9693409769999, 40.69576842999993], [-73.96910362499993, 40.68946752199986], [-73.97013154199992, 40.68950630799986], [-73.96868961400001, 40.68200525799988], [-73.97745722099988, 40.68405228699988], [-73.98092822899991, 40.68890247699988], [-73.98372011099993, 40.69000671899992], [-73.98147568199991, 40.689930943999876], [-73.98251122999989, 40.69361433399992], [-73.97916718399999, 40.69348832299992], [-73.98054419199988, 40.69824651199988], [-73.9697278989999, 40.69780401399986], [-73.9693409769999, 40.69576842999993]]]}}, {\"id\": \"97\", \"type\": \"Feature\", \"properties\": {\"count\": 684, \"OBJECTID\": 98, \"Shape_Leng\": 0.121661018278, \"Shape_Area\": 0.000485773020954, \"zone\": \"Fresh Meadows\", \"LocationID\": 98, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76360860199983, 40.73917476499987], [-73.76273303499985, 40.73526216299995], [-73.76510920699994, 40.73455658499992], [-73.7636997919999, 40.73286411699993], [-73.76564374500005, 40.732533126999954], [-73.75669771299987, 40.7262287109999], [-73.76924074899993, 40.72590270099988], [-73.77727808000003, 40.72234633599988], [-73.77803765999981, 40.723284268999855], [-73.7752338719998, 40.725237532999884], [-73.77757506899988, 40.73019327299987], [-73.79174472699992, 40.725788849999944], [-73.79651029099988, 40.7379086369999], [-73.76772991399994, 40.74577622599993], [-73.76360860199983, 40.73917476499987]]]}}, {\"id\": \"98\", \"type\": \"Feature\", \"properties\": {\"count\": 16, \"OBJECTID\": 99, \"Shape_Leng\": 0.1833714893, \"Shape_Area\": 0.00121016463877, \"zone\": \"Freshkills Park\", \"LocationID\": 99, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.16842916199994, 40.58649792299988], [-74.16982582400001, 40.56109042099991], [-74.18295815999996, 40.56548019999992], [-74.193410946, 40.56248503899994], [-74.20091501299991, 40.55600525999987], [-74.20699885299997, 40.55756748899989], [-74.2056960519999, 40.55851600599987], [-74.21324031454323, 40.55667262386906], [-74.20596154297344, 40.580046993882775], [-74.20048848799996, 40.57994977799989], [-74.19964981299998, 40.58226262599991], [-74.19423261299991, 40.58514680599988], [-74.185371048, 40.58745101499987], [-74.18730248699987, 40.588488221999896], [-74.17488959399996, 40.602216572999914], [-74.16719634899995, 40.602075311999926], [-74.16886362299992, 40.59472808699995], [-74.16350785499993, 40.59327217999993], [-74.16842916199994, 40.58649792299988]]]}}, {\"id\": \"99\", \"type\": \"Feature\", \"properties\": {\"count\": 2647238, \"OBJECTID\": 100, \"Shape_Leng\": 0.0248131090342, \"Shape_Area\": 3.74700210291e-05, \"zone\": \"Garment District\", \"LocationID\": 100, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98729377099981, 40.75045160899988], [-73.99346417699986, 40.75219005499987], [-73.98979085399996, 40.7572331319999], [-73.98411754799999, 40.75484205299995], [-73.98729377099981, 40.75045160899988]]]}}, {\"id\": \"100\", \"type\": \"Feature\", \"properties\": {\"count\": 636, \"OBJECTID\": 101, \"Shape_Leng\": 0.101709836277, \"Shape_Area\": 0.000452342528877, \"zone\": \"Glen Oaks\", \"LocationID\": 101, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.70134715908382, 40.750780580935846], [-73.70002020501624, 40.73923654186594], [-73.70766217302989, 40.727830934754685], [-73.71049995381028, 40.72722708720663], [-73.71147804899985, 40.73250140099988], [-73.72010148299994, 40.74977568599991], [-73.727664654, 40.749654641999946], [-73.728962516, 40.74763288799986], [-73.72737889099987, 40.751029566999854], [-73.7206331939999, 40.75761683499996], [-73.71612528199985, 40.75966071399991], [-73.71244940715339, 40.75924170410932], [-73.70163345846068, 40.75249332984642], [-73.70134715908382, 40.750780580935846]]]}}, {\"id\": \"101\", \"type\": \"Feature\", \"properties\": {\"count\": 1532, \"OBJECTID\": 102, \"Shape_Leng\": 0.136900484646, \"Shape_Area\": 0.000296595466345, \"zone\": \"Glendale\", \"LocationID\": 102, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85595536199988, 40.70642977899992], [-73.85473240499985, 40.70676414299989], [-73.85252019099998, 40.70229172399997], [-73.85827405799986, 40.70122365899992], [-73.85819890699996, 40.70291107399986], [-73.86285374199994, 40.7034144919999], [-73.87639785199997, 40.70229404199989], [-73.88182690599986, 40.69952929899987], [-73.88280948499988, 40.70068345599986], [-73.88771297999986, 40.699678075999906], [-73.88642048199995, 40.696458148999945], [-73.8950769439999, 40.694591389999886], [-73.89326209899984, 40.69220094699994], [-73.89695653499987, 40.6919492069999], [-73.89736265799978, 40.6941108469999], [-73.900030392, 40.693437450999895], [-73.88911917799989, 40.705598876999915], [-73.86911007199988, 40.707089336999914], [-73.87016462499984, 40.70920774299987], [-73.85965416799995, 40.713379960999866], [-73.85975932900001, 40.711845058999856], [-73.85729287299985, 40.711467676999874], [-73.85595536199988, 40.70642977899992]]]}}, {\"id\": \"102\", \"type\": \"Feature\", \"properties\": {\"count\": 57, \"OBJECTID\": 103, \"Shape_Leng\": 0.0143055167343, \"Shape_Area\": 6.33056361314e-06, \"zone\": \"Governor's Island/Ellis Island/Liberty Island\", \"LocationID\": 103, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.04388559600675, 40.690184824096946], [-74.04350596056258, 40.689687359818144], [-74.04270428413375, 40.69015520482211], [-74.04255372018648, 40.68996275914475], [-74.04438521705613, 40.68851617820147], [-74.04772962698515, 40.689915318233105], [-74.04614707204328, 40.69112264601653], [-74.04388559600675, 40.690184824096946]]]}}, {\"id\": \"103\", \"type\": \"Feature\", \"properties\": {\"count\": 57, \"OBJECTID\": 104, \"Shape_Leng\": 0.0212208330928, \"Shape_Area\": 1.19205339715e-05, \"zone\": \"Governor's Island/Ellis Island/Liberty Island\", \"LocationID\": 103, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.03995040794246, 40.70089063032128], [-74.03771124798313, 40.69934404017382], [-74.03934037685066, 40.698115514418], [-74.04124261816106, 40.69953674121738], [-74.0399124888631, 40.69770204019913], [-74.0416605190741, 40.696452971319864], [-74.04367371215382, 40.69802040416498], [-74.03995040794246, 40.70089063032128]]]}}, {\"id\": \"104\", \"type\": \"Feature\", \"properties\": {\"count\": 57, \"OBJECTID\": 105, \"Shape_Leng\": 0.0774253398314, \"Shape_Area\": 0.000368636392805, \"zone\": \"Governor's Island/Ellis Island/Liberty Island\", \"LocationID\": 103, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01674756096064, 40.693343368217555], [-74.0118118564723, 40.69247933506435], [-74.01333264149613, 40.69201997010238], [-74.01182575354963, 40.69106481621614], [-74.01316926402752, 40.6881547712032], [-74.02259359581036, 40.68435969594992], [-74.02213250186877, 40.683767672385194], [-74.02305574749599, 40.6829169454452], [-74.0227589902996, 40.68428444833225], [-74.026334042424, 40.6848016977242], [-74.01971454942077, 40.69313300682718], [-74.01674756096064, 40.693343368217555]]]}}, {\"id\": \"105\", \"type\": \"Feature\", \"properties\": {\"count\": 34542, \"OBJECTID\": 106, \"Shape_Leng\": 0.0492110119798, \"Shape_Area\": 0.000113616826567, \"zone\": \"Gowanus\", \"LocationID\": 106, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98610388199985, 40.68100852199994], [-73.98175642299985, 40.67931462399987], [-73.99277080699981, 40.665507280999876], [-73.9986188799999, 40.67183017199989], [-73.99506451099984, 40.67584047499984], [-73.99069861899994, 40.67580741899994], [-73.9876680399999, 40.68081780999993], [-73.98610388199985, 40.68100852199994]]]}}, {\"id\": \"106\", \"type\": \"Feature\", \"properties\": {\"count\": 3810928, \"OBJECTID\": 107, \"Shape_Leng\": 0.0380413645908, \"Shape_Area\": 7.46112192675e-05, \"zone\": \"Gramercy\", \"LocationID\": 107, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98238628799993, 40.73141124499993], [-73.98990295999991, 40.7344347899999], [-73.98407485299995, 40.74332471799989], [-73.98085965399994, 40.74196976799989], [-73.98266497499989, 40.73949733299987], [-73.97805226499996, 40.7375539539999], [-73.98238628799993, 40.73141124499993]]]}}, {\"id\": \"107\", \"type\": \"Feature\", \"properties\": {\"count\": 597, \"OBJECTID\": 108, \"Shape_Leng\": 0.11117141227, \"Shape_Area\": 0.000352866009465, \"zone\": \"Gravesend\", \"LocationID\": 108, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97148350899987, 40.59717158499996], [-73.97115101799994, 40.59044244399989], [-73.96514385199995, 40.5911019159999], [-73.96661531499983, 40.58457103699992], [-73.97742525299988, 40.58290659299994], [-73.98710120299997, 40.58464459199988], [-73.98654457099988, 40.583658590999946], [-73.98829948299982, 40.58346674699986], [-73.98995504960881, 40.58066166976562], [-73.99185535656628, 40.5816239785461], [-73.99026244714351, 40.5846191482053], [-73.99547188560318, 40.58221005511802], [-74.0003088081067, 40.58317703751469], [-73.99808523776004, 40.585364721581485], [-73.99956511570304, 40.5862439200957], [-73.9940790462178, 40.58869344199059], [-74.00038207107852, 40.58713305717468], [-73.98962150099992, 40.59098199299996], [-73.98601936799993, 40.594446048999906], [-73.97995443499985, 40.594073811999884], [-73.98035785499995, 40.59619154099992], [-73.97148350899987, 40.59717158499996]]]}}, {\"id\": \"108\", \"type\": \"Feature\", \"properties\": {\"count\": 40, \"OBJECTID\": 109, \"Shape_Leng\": 0.178267819599, \"Shape_Area\": 0.00116960076185, \"zone\": \"Great Kills\", \"LocationID\": 109, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.14706472699994, 40.5705086149999], [-74.13170009, 40.56453574899991], [-74.13885092699991, 40.5569511789999], [-74.13316825999992, 40.552356178999936], [-74.13861798499994, 40.548492384999875], [-74.13585712583111, 40.546451696357344], [-74.1371798712987, 40.5457928646606], [-74.13611540724054, 40.54501984567978], [-74.13794966050746, 40.54600033758068], [-74.13886951340419, 40.545130197607286], [-74.13762430745145, 40.54425356641828], [-74.13875534703455, 40.544933958426874], [-74.13786798524545, 40.54405254413092], [-74.13903307178627, 40.54470491029999], [-74.13953937915736, 40.54429514087214], [-74.13834151288823, 40.543551828814046], [-74.14038573073617, 40.54403318500786], [-74.13921993485874, 40.54316608513788], [-74.14141073560397, 40.543022415080344], [-74.14026290400692, 40.54196151152258], [-74.14134901379589, 40.54257752313157], [-74.14179719957052, 40.54220041538235], [-74.14063984172124, 40.54169454203723], [-74.14095816832888, 40.54143207796496], [-74.14229632246158, 40.54225784915546], [-74.14104157916547, 40.53734688546733], [-74.14374951249268, 40.53719106134627], [-74.15970134520207, 40.52729599283152], [-74.16134961699994, 40.5293734239999], [-74.15397419799994, 40.53426814799993], [-74.16540080799992, 40.54443272299989], [-74.16982582400001, 40.56109042099991], [-74.1649414939999, 40.56011179699991], [-74.15725974199995, 40.562553318999896], [-74.14706472699994, 40.5705086149999]]]}}, {\"id\": \"109\", \"type\": \"Feature\", \"properties\": {\"count\": 2, \"OBJECTID\": 110, \"Shape_Leng\": 0.103946292913, \"Shape_Area\": 0.000525745098785, \"zone\": \"Great Kills Park\", \"LocationID\": 110, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.11813477599993, 40.550592692999906], [-74.11338340608127, 40.54807337507532], [-74.12245306050386, 40.544770545629824], [-74.1335644349803, 40.5315111748056], [-74.1369079907634, 40.529281918882695], [-74.13883030715276, 40.529889356352996], [-74.14032475393472, 40.5352845797219], [-74.13381005153853, 40.53564293988989], [-74.12738789634466, 40.54229542996431], [-74.13011404715266, 40.54597572663567], [-74.13585712583111, 40.546451696357344], [-74.13861798499991, 40.548492383999886], [-74.12730057799996, 40.55729517199993], [-74.11813477599993, 40.550592692999906]]]}}, {\"id\": \"110\", \"type\": \"Feature\", \"properties\": {\"count\": 404, \"OBJECTID\": 111, \"Shape_Leng\": 0.0599308800658, \"Shape_Area\": 0.000208683276503, \"zone\": \"Green-Wood Cemetery\", \"LocationID\": 111, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99516890099986, 40.6596695219999], [-73.990188415, 40.657600817999906], [-73.98843135299977, 40.65925121199996], [-73.98179568699996, 40.65524609699991], [-73.98029115499993, 40.647297160999855], [-73.98905872499985, 40.64411924099995], [-74.00214474599989, 40.65295610699996], [-73.99516890099986, 40.6596695219999]]]}}, {\"id\": \"111\", \"type\": \"Feature\", \"properties\": {\"count\": 131998, \"OBJECTID\": 112, \"Shape_Leng\": 0.107267875765, \"Shape_Area\": 0.000462068671429, \"zone\": \"Greenpoint\", \"LocationID\": 112, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.95439555417066, 40.73911477252253], [-73.94652352854791, 40.73692685395814], [-73.94719983367108, 40.73535517811412], [-73.94706532915383, 40.734401992819265], [-73.94557048457217, 40.736612303910405], [-73.94239443542429, 40.73542574994758], [-73.9381033496066, 40.729728519521366], [-73.93226852175046, 40.72817465801565], [-73.94841638599995, 40.71845553299995], [-73.94907816799986, 40.72254028699993], [-73.95477662900004, 40.72245950499994], [-73.95842611843923, 40.725400535832165], [-73.9616151883593, 40.725865563073995], [-73.96143187677845, 40.730965171647846], [-73.96226340423281, 40.732915551623485], [-73.96406507049178, 40.732779040653625], [-73.95837507902779, 40.73809694936863], [-73.95439555417066, 40.73911477252253]]], [[[-73.96236596889429, 40.72420906161418], [-73.96200744849497, 40.7239919013038], [-73.96207271921921, 40.72388030020147], [-73.96246790011047, 40.72413157960129], [-73.96236596889429, 40.72420906161418]]]]}}, {\"id\": \"112\", \"type\": \"Feature\", \"properties\": {\"count\": 2576597, \"OBJECTID\": 113, \"Shape_Leng\": 0.032745350566, \"Shape_Area\": 5.79634566998e-05, \"zone\": \"Greenwich Village North\", \"LocationID\": 113, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99136388099988, 40.72755978299991], [-73.99657214499979, 40.729552233999904], [-73.99556245499996, 40.73071521599994], [-74.000066577, 40.7329292359999], [-73.99683993899998, 40.73736088899995], [-73.98990295999991, 40.7344347899999], [-73.99136388099988, 40.72755978299991]]]}}, {\"id\": \"113\", \"type\": \"Feature\", \"properties\": {\"count\": 2156520, \"OBJECTID\": 114, \"Shape_Leng\": 0.0317270347711, \"Shape_Area\": 4.70207704945e-05, \"zone\": \"Greenwich Village South\", \"LocationID\": 114, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99257987399992, 40.724201554999915], [-74.00281726599992, 40.72836312899993], [-74.000066577, 40.7329292359999], [-73.99154832699986, 40.72708947099994], [-73.99257987399992, 40.724201554999915]]]}}, {\"id\": \"114\", \"type\": \"Feature\", \"properties\": {\"count\": 136, \"OBJECTID\": 115, \"Shape_Leng\": 0.116169413964, \"Shape_Area\": 0.000373168991958, \"zone\": \"Grymes Hill/Clifton\", \"LocationID\": 115, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08069514799999, 40.62753689699988], [-74.08326107499997, 40.627194742999954], [-74.08496211899991, 40.624625232999925], [-74.08785147699994, 40.615243072999924], [-74.08720982799997, 40.61330697399994], [-74.07253551699989, 40.622464021999896], [-74.071134623, 40.62035994799992], [-74.0815547009999, 40.60629029699987], [-74.0977696879999, 40.61062359299988], [-74.10350925299994, 40.61641607499991], [-74.0922397139999, 40.62574291499989], [-74.08574928999992, 40.633955556999865], [-74.07972492199995, 40.62895068799986], [-74.08069514799999, 40.62753689699988]]]}}, {\"id\": \"115\", \"type\": \"Feature\", \"properties\": {\"count\": 154287, \"OBJECTID\": 116, \"Shape_Leng\": 0.0681164844265, \"Shape_Area\": 0.000260415337217, \"zone\": \"Hamilton Heights\", \"LocationID\": 116, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93924885699987, 40.82829635099989], [-73.94515456999994, 40.82087626499993], [-73.94808289799983, 40.82210802199994], [-73.95082880000002, 40.820827936999876], [-73.95954472552476, 40.82364494182928], [-73.95015521048293, 40.83439675940791], [-73.93864026699997, 40.82973887399988], [-73.93924885699987, 40.82829635099989]]]}}, {\"id\": \"116\", \"type\": \"Feature\", \"properties\": {\"count\": 503, \"OBJECTID\": 117, \"Shape_Leng\": 0.169885857987, \"Shape_Area\": 0.00090429427072, \"zone\": \"Hammels/Arverne\", \"LocationID\": 117, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7763584369479, 40.609655838242304], [-73.77412897394224, 40.60874659586809], [-73.77460294599074, 40.607417648882524], [-73.77484886546486, 40.60802483315967], [-73.77523851986173, 40.60855094300021], [-73.77600772152827, 40.60919075424813], [-73.77469363854065, 40.60430696481727], [-73.7818572569845, 40.59837293018506], [-73.78005493695551, 40.596861344908895], [-73.7749751364952, 40.60174817941184], [-73.77363244061033, 40.598153123466695], [-73.76809925802587, 40.59784597005487], [-73.76771811799993, 40.596261309999925], [-73.76654204999997, 40.599434441999904], [-73.76165797299986, 40.599973069999876], [-73.7633344969999, 40.59782287399992], [-73.76277845599988, 40.59475654399989], [-73.76440183099987, 40.59457168299989], [-73.76376223423699, 40.591439709016804], [-73.81730795916116, 40.58233475897], [-73.81544517899997, 40.58337349199993], [-73.81909407399996, 40.58783196099988], [-73.8176380045051, 40.58928579948519], [-73.81170344836342, 40.59159374663789], [-73.81149423585859, 40.592901326696776], [-73.81132554567534, 40.59174649614576], [-73.81081791874632, 40.59209026869603], [-73.81138593152033, 40.592442318454566], [-73.81140751227157, 40.592871257238706], [-73.81107287021051, 40.593022330572076], [-73.80991130015991, 40.59225371768247], [-73.80959170493088, 40.593580688618054], [-73.80577902839048, 40.59075949966858], [-73.8058134261234, 40.59186886150206], [-73.80430333508215, 40.59131887880457], [-73.8079775709933, 40.59415786866439], [-73.80604149807385, 40.595554555213546], [-73.8024384629031, 40.59264006383509], [-73.80234513845657, 40.59449597525689], [-73.80344017041448, 40.59443420917679], [-73.80321494251235, 40.59514564845063], [-73.80389507006726, 40.594990875367245], [-73.80331453729063, 40.59517956193947], [-73.80368468695728, 40.59519779018381], [-73.80486462646938, 40.5967336283515], [-73.80513406252756, 40.59660758897049], [-73.80524832538248, 40.596895032613546], [-73.80297943325748, 40.598844304553005], [-73.79233256449508, 40.59995439142583], [-73.78635332811808, 40.60319465903842], [-73.79034006550985, 40.59919167636005], [-73.79102536652763, 40.595527157215926], [-73.78973901889819, 40.59474366855456], [-73.7890682912148, 40.598019927382616], [-73.78350560905308, 40.60256343874268], [-73.78366696292244, 40.60532443693426], [-73.77927015444938, 40.60930736180752], [-73.7763584369479, 40.609655838242304]]]}}, {\"id\": \"117\", \"type\": \"Feature\", \"properties\": {\"count\": 127, \"OBJECTID\": 118, \"Shape_Leng\": 0.243966217692, \"Shape_Area\": 0.00182693922711, \"zone\": \"Heartland Village/Todt Hill\", \"LocationID\": 118, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.10834417899996, 40.610234766999945], [-74.09555829799993, 40.61062477799989], [-74.09045495899996, 40.60909452599991], [-74.09271908099993, 40.60636222199992], [-74.09192827899989, 40.598504050999914], [-74.10041498999992, 40.59334309299988], [-74.10149482299994, 40.58934426099992], [-74.10790851599995, 40.58368563799992], [-74.1212571389999, 40.575254893999904], [-74.12708814699995, 40.57636910599994], [-74.13261022999991, 40.573530162999894], [-74.14418588199999, 40.57203740999988], [-74.14438744999991, 40.56970610099991], [-74.14789716899996, 40.57088728599995], [-74.15725974199995, 40.562553318999896], [-74.1649414939999, 40.56011179699991], [-74.16982582400001, 40.56109042099991], [-74.16842916199994, 40.58649792299988], [-74.16189940599996, 40.59529877099993], [-74.1481980769999, 40.588599680999934], [-74.13864458599996, 40.58906080499989], [-74.13091909399999, 40.598855047999905], [-74.12506115499997, 40.59818005599987], [-74.12059546299996, 40.601809757999874], [-74.11487058399989, 40.602136659999935], [-74.11742466500002, 40.609289008999916], [-74.12135179599994, 40.609765267999926], [-74.11508745399988, 40.61120943099993], [-74.10834417899996, 40.610234766999945]]]}}, {\"id\": \"118\", \"type\": \"Feature\", \"properties\": {\"count\": 4599, \"OBJECTID\": 119, \"Shape_Leng\": 0.0835070355744, \"Shape_Area\": 0.000185779447534, \"zone\": \"Highbridge\", \"LocationID\": 119, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9247831879999, 40.84475266499989], [-73.91729224999992, 40.84510993999991], [-73.91794965199988, 40.842229570999976], [-73.91618344399991, 40.84233489999991], [-73.92802724899998, 40.82971953899987], [-73.93307251084072, 40.828156152518105], [-73.93314306709256, 40.83519412761651], [-73.92861574822331, 40.844676865029925], [-73.9247831879999, 40.84475266499989]]]}}, {\"id\": \"119\", \"type\": \"Feature\", \"properties\": {\"count\": 973, \"OBJECTID\": 120, \"Shape_Leng\": 0.0832612702158, \"Shape_Area\": 9.31462821313e-05, \"zone\": \"Highbridge Park\", \"LocationID\": 120, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92295449499989, 40.85885076199988], [-73.92227873064479, 40.85565498884376], [-73.93436121591056, 40.83620061961779], [-73.94034643299983, 40.8304578419999], [-73.93505508799984, 40.8393332359999], [-73.93556697199989, 40.841712527999924], [-73.92707982300001, 40.851432342999885], [-73.92466977999999, 40.856982610999886], [-73.92709951599986, 40.85827552099992], [-73.92499818399988, 40.86146373799991], [-73.92295449499989, 40.85885076199988]]]}}, {\"id\": \"120\", \"type\": \"Feature\", \"properties\": {\"count\": 1606, \"OBJECTID\": 121, \"Shape_Leng\": 0.0969153373445, \"Shape_Area\": 0.000384563286473, \"zone\": \"Hillcrest/Pomonok\", \"LocationID\": 121, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79911195699995, 40.738206139999875], [-73.79584803299987, 40.7380746799999], [-73.79037661699995, 40.720103723999934], [-73.80780927699995, 40.715853895999906], [-73.80904032699983, 40.71991160499987], [-73.80452680199981, 40.721462545999884], [-73.80487958299983, 40.725049592999895], [-73.80652142399995, 40.7248730639999], [-73.80717042299993, 40.72826913299987], [-73.8111179729999, 40.72781641599994], [-73.81074426599986, 40.72626511299991], [-73.81510024099985, 40.72854342299991], [-73.81504998799986, 40.73894313399992], [-73.79911195699995, 40.738206139999875]]]}}, {\"id\": \"121\", \"type\": \"Feature\", \"properties\": {\"count\": 957, \"OBJECTID\": 122, \"Shape_Leng\": 0.0665318439446, \"Shape_Area\": 0.000226597104976, \"zone\": \"Hollis\", \"LocationID\": 122, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.75725671499985, 40.71813860199992], [-73.75047058199985, 40.70563993099994], [-73.76612489399999, 40.702334485999884], [-73.76825969099993, 40.709526367999956], [-73.771724447, 40.71031681899991], [-73.76943987099992, 40.711976892999864], [-73.77192122100001, 40.71568504499992], [-73.76664095199988, 40.717646613999875], [-73.765714472, 40.71639337899988], [-73.75832728699997, 40.71958208899985], [-73.75725671499985, 40.71813860199992]]]}}, {\"id\": \"122\", \"type\": \"Feature\", \"properties\": {\"count\": 1881, \"OBJECTID\": 123, \"Shape_Leng\": 0.0834211389387, \"Shape_Area\": 0.000296445173366, \"zone\": \"Homecrest\", \"LocationID\": 123, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96285973999984, 40.61254948599989], [-73.96237947799978, 40.609986373999966], [-73.95859278499987, 40.61040303099988], [-73.95619313699986, 40.58776226899994], [-73.9603495359999, 40.58730628599994], [-73.9606798409999, 40.59159758199995], [-73.97115101799994, 40.59044244399989], [-73.97299433999983, 40.6088141419999], [-73.96429649899991, 40.60977642599992], [-73.96478112799986, 40.61233092199988], [-73.96285973999984, 40.61254948599989]]]}}, {\"id\": \"123\", \"type\": \"Feature\", \"properties\": {\"count\": 1197, \"OBJECTID\": 124, \"Shape_Leng\": 0.127724410062, \"Shape_Area\": 0.000754837821479, \"zone\": \"Howard Beach\", \"LocationID\": 124, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.85096453700001, 40.671719510999885], [-73.83334819299995, 40.66677318699994], [-73.82859942399985, 40.657196972999884], [-73.82639242872236, 40.64832434880783], [-73.83133151403268, 40.65274755744263], [-73.82901417557953, 40.65610788445241], [-73.8317081441153, 40.65393117429689], [-73.83221698518575, 40.654460386292925], [-73.83149330103005, 40.65497982053967], [-73.83116100809627, 40.65591923373715], [-73.82937603410723, 40.65650633521598], [-73.82896778740296, 40.65720524792007], [-73.8291791973457, 40.65779977854427], [-73.83013028740389, 40.65840264372116], [-73.82970706838907, 40.658502513864484], [-73.82959986903019, 40.65865680691539], [-73.83014950104759, 40.6598086849573], [-73.83024427039376, 40.658349969348166], [-73.82909434825203, 40.65733421866331], [-73.83127112568108, 40.656003271494676], [-73.8318523847929, 40.65488503826691], [-73.83290309524602, 40.657483771623006], [-73.83194394578256, 40.65617142725338], [-73.832600363706, 40.65782659162732], [-73.83096907996668, 40.65823029417234], [-73.83338439472544, 40.65805185920798], [-73.83109184464118, 40.64848480675928], [-73.83561034795801, 40.64850568470053], [-73.8387360855594, 40.662454060021076], [-73.83978752612934, 40.660485998048], [-73.83592320073285, 40.645499790674954], [-73.84950699084165, 40.64413376567876], [-73.85183280737658, 40.64586005671262], [-73.85230349886227, 40.647771028550196], [-73.84939766406322, 40.65143487895031], [-73.85591932944749, 40.65130926166608], [-73.86136381950391, 40.65641475416531], [-73.85983363521494, 40.65697706322045], [-73.8596355866937, 40.65845777205655], [-73.86124943398261, 40.65872091007865], [-73.85976839588899, 40.65828193634574], [-73.86087597607579, 40.65690816207006], [-73.86317083299996, 40.65827651199994], [-73.85761052799985, 40.660108355999924], [-73.85842950899992, 40.66345335999987], [-73.85568461199988, 40.66386749199993], [-73.85763323099987, 40.67165619399989], [-73.86038937900003, 40.67126877499985], [-73.86131863599985, 40.6750212689999], [-73.85096453700001, 40.671719510999885]]], [[[-73.83032725337058, 40.65513280580328], [-73.83026601851493, 40.65542864678473], [-73.82988919856028, 40.655522822584075], [-73.82994556435834, 40.655424494353], [-73.83032725337058, 40.65513280580328]]], [[[-73.85975533686633, 40.65410417609493], [-73.86048308199986, 40.65465892799988], [-73.86098917376947, 40.65520976705256], [-73.85979242750741, 40.65447615737696], [-73.85975533686633, 40.65410417609493]]], [[[-73.85898002989568, 40.65267545791771], [-73.85975533686633, 40.65410417609493], [-73.85941022942576, 40.65386473792502], [-73.85898002989568, 40.65267545791771]]], [[[-73.85898002989568, 40.65267545791771], [-73.85731941515513, 40.65123709900783], [-73.85695870295893, 40.65058737348713], [-73.85898002989568, 40.65267545791771]]]]}}, {\"id\": \"124\", \"type\": \"Feature\", \"properties\": {\"count\": 893114, \"OBJECTID\": 125, \"Shape_Leng\": 0.0497322121757, \"Shape_Area\": 7.64293209892e-05, \"zone\": \"Hudson Sq\", \"LocationID\": 125, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00269315099992, 40.72831405499989], [-74.0053098609999, 40.721824191999865], [-74.0108123829999, 40.72578980299994], [-74.0119643202455, 40.72432986461987], [-74.01165326569942, 40.725871761851344], [-74.01520426531133, 40.726361949379644], [-74.011542018976, 40.72644734692706], [-74.01067264999996, 40.72911266599987], [-74.00269315099992, 40.72831405499989]]]}}, {\"id\": \"125\", \"type\": \"Feature\", \"properties\": {\"count\": 1579, \"OBJECTID\": 126, \"Shape_Leng\": 0.12994872606, \"Shape_Area\": 0.0007032738273, \"zone\": \"Hunts Point\", \"LocationID\": 126, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88615037899993, 40.82730567799994], [-73.88753429599987, 40.82250933899995], [-73.8843985489999, 40.822965575999866], [-73.88244779599998, 40.81926832299992], [-73.87056806927927, 40.81208731762241], [-73.86811219140058, 40.806751832831786], [-73.87234684998835, 40.800174763010446], [-73.87814218819929, 40.801319290306495], [-73.87843295513449, 40.80270273469493], [-73.8850517233671, 40.80213090679797], [-73.89197929534278, 40.80638470897619], [-73.89128402413917, 40.80497628189916], [-73.89509836487399, 40.80686363917353], [-73.8952796287401, 40.805766184067906], [-73.90222284749719, 40.80494811309474], [-73.90307457199998, 40.80988061199988], [-73.89542858799986, 40.81584467199988], [-73.8970446669999, 40.8168018599999], [-73.89283653100001, 40.820974054999965], [-73.8958813589999, 40.82099257499987], [-73.89586019599982, 40.82263104599987], [-73.89320004999998, 40.82400818299992], [-73.89487088500002, 40.82393727199992], [-73.89483522899992, 40.82624684099993], [-73.88615037899993, 40.82730567799994]]]}}, {\"id\": \"126\", \"type\": \"Feature\", \"properties\": {\"count\": 15780, \"OBJECTID\": 127, \"Shape_Leng\": 0.0600188855815, \"Shape_Area\": 0.000163195647796, \"zone\": \"Inwood\", \"LocationID\": 127, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.911224294065, 40.87302771883775], [-73.91043824727869, 40.871377478551786], [-73.91254775130287, 40.866514943004354], [-73.91589911645785, 40.86430803808506], [-73.91538673051149, 40.863084384200725], [-73.91950213660932, 40.85879113379047], [-73.92159014848224, 40.86007280002688], [-73.92035142875443, 40.858195220619145], [-73.92282845999992, 40.8568514089999], [-73.92872203199994, 40.86674266299988], [-73.92135975499993, 40.869551853999894], [-73.91845930899993, 40.873022052999936], [-73.9140098549999, 40.871160593999946], [-73.911224294065, 40.87302771883775]]], [[[-73.92151894059094, 40.85660117062315], [-73.92229611007241, 40.855663835114505], [-73.92234459387058, 40.85568233590737], [-73.92169841904696, 40.85667230185494], [-73.92151894059094, 40.85660117062315]]]]}}, {\"id\": \"127\", \"type\": \"Feature\", \"properties\": {\"count\": 731, \"OBJECTID\": 128, \"Shape_Leng\": 0.0702090290681, \"Shape_Area\": 0.000207405982869, \"zone\": \"Inwood Hill Park\", \"LocationID\": 128, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.92359742020389, 40.878898712992644], [-73.92363695289211, 40.87881535593209], [-73.92367919311681, 40.878826074498626], [-73.92362973975199, 40.87896394312155], [-73.92359742020389, 40.878898712992644]]], [[[-73.9264055692112, 40.87762147653741], [-73.92258387170895, 40.87692253928837], [-73.921169296781, 40.87306939228805], [-73.91978312944534, 40.87360369743557], [-73.92090325515422, 40.87546616638258], [-73.91855365026461, 40.873180588115396], [-73.9175772988473, 40.87454867854727], [-73.91154281116535, 40.873253706161705], [-73.9140098549999, 40.871160593999946], [-73.91845930899993, 40.873022052999936], [-73.92135975499993, 40.869551853999894], [-73.92872203199994, 40.86674266299988], [-73.93220600531866, 40.870096445104515], [-73.9264055692112, 40.87762147653741]]]]}}, {\"id\": \"131\", \"type\": \"Feature\", \"properties\": {\"count\": 3104157, \"OBJECTID\": 132, \"Shape_Leng\": 0.24547851707, \"Shape_Area\": 0.00203830095472, \"zone\": \"JFK Airport\", \"LocationID\": 132, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.8250346749999, 40.663580133999844], [-73.81496922499986, 40.66247266799991], [-73.80072889200001, 40.66526937699988], [-73.780755286, 40.66240660199992], [-73.7541108669999, 40.648697664999915], [-73.75522386899995, 40.64744446899989], [-73.7474246205621, 40.6413764680137], [-73.74849297657127, 40.63567172844184], [-73.7650077554797, 40.62941398784963], [-73.77077016777163, 40.62003118215498], [-73.77082069703974, 40.62328299797239], [-73.77704080084247, 40.62766646976843], [-73.77999420781757, 40.62668929803533], [-73.78419483298926, 40.6208926699551], [-73.78949653224412, 40.62274664610792], [-73.78264543417806, 40.630247968544396], [-73.79076956011139, 40.63386374895698], [-73.79222318018806, 40.63589008813999], [-73.81830607819735, 40.64638698303703], [-73.82182513624882, 40.649899795874894], [-73.82348381110747, 40.65536991271862], [-73.82227996040434, 40.659425007931176], [-73.8186953155868, 40.661105051055245], [-73.81143921922407, 40.66074562606795], [-73.81867189288894, 40.66213298725428], [-73.82338324351308, 40.6602057470252], [-73.82498070378553, 40.65543197118367], [-73.82290717700891, 40.64809344600474], [-73.82614483059669, 40.65008184094253], [-73.82639242822242, 40.64832434880782], [-73.82859942399985, 40.657196972999884], [-73.832705339, 40.665405772999854], [-73.8250346749999, 40.663580133999844]]], [[[-73.74661040472857, 40.638724973885545], [-73.74653979797041, 40.63871136266965], [-73.74654132522075, 40.63869678799164], [-73.74664302817305, 40.638662249807076], [-73.74661040472857, 40.638724973885545]]], [[[-73.74693721618013, 40.63755610186007], [-73.74694320619099, 40.637529202088494], [-73.74704913170525, 40.63754513439954], [-73.74704316342154, 40.637566426459216], [-73.74693721618013, 40.63755610186007]]], [[[-73.74712154262465, 40.63716227570799], [-73.74706485671244, 40.637160999268374], [-73.74699391228548, 40.63715064484292], [-73.74715781614569, 40.637060524028364], [-73.74712154262465, 40.63716227570799]]]]}}, {\"id\": \"128\", \"type\": \"Feature\", \"properties\": {\"count\": 95995, \"OBJECTID\": 129, \"Shape_Leng\": 0.0932727273403, \"Shape_Area\": 0.00047398833179, \"zone\": \"Jackson Heights\", \"LocationID\": 129, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87586503899996, 40.76605165399989], [-73.87513874300004, 40.756633815999905], [-73.87365721799983, 40.756803485999946], [-73.86942458599987, 40.74915685199991], [-73.89175048799984, 40.74681172799989], [-73.89624245599988, 40.74871954499987], [-73.89984592699983, 40.75756215299985], [-73.89437624799997, 40.7655265849999], [-73.87608949699984, 40.77153690299987], [-73.87586503899996, 40.76605165399989]]]}}, {\"id\": \"129\", \"type\": \"Feature\", \"properties\": {\"count\": 13144, \"OBJECTID\": 130, \"Shape_Leng\": 0.142028320665, \"Shape_Area\": 0.000468322514327, \"zone\": \"Jamaica\", \"LocationID\": 130, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76962552399995, 40.71178631999993], [-73.771724447, 40.71031681899991], [-73.76844546499984, 40.709637026999886], [-73.77555451099995, 40.70607083499987], [-73.77230768699997, 40.70057150699989], [-73.77395659499993, 40.69980578099988], [-73.78034491200003, 40.70541241699986], [-73.78256582599995, 40.70542486399989], [-73.79239170299988, 40.7021665299999], [-73.79260506799987, 40.70043209999988], [-73.80642479399984, 40.690873857999904], [-73.80890216900004, 40.69256434899993], [-73.81063298299996, 40.69194738299988], [-73.81637824499992, 40.70245196399987], [-73.81152856899989, 40.70227287499991], [-73.81201248599983, 40.70594494499993], [-73.80694856299993, 40.70736709999989], [-73.8056477309999, 40.70467172799994], [-73.80165725599993, 40.705761474999846], [-73.80302414799988, 40.70813965699997], [-73.79870150699992, 40.71066337699988], [-73.79776810199988, 40.7091431879999], [-73.77192121999992, 40.715685043999905], [-73.76962552399995, 40.71178631999993]]]}}, {\"id\": \"1\", \"type\": \"Feature\", \"properties\": {\"count\": 132, \"OBJECTID\": 2, \"Shape_Leng\": 0.43346966679, \"Shape_Area\": 0.00486634037837, \"zone\": \"Jamaica Bay\", \"LocationID\": 2, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.82337597260663, 40.63898704717671], [-73.82107505533321, 40.62973377456623], [-73.81611483429545, 40.624604655173044], [-73.81891708039825, 40.62309026716333], [-73.81571468044605, 40.62148955151693], [-73.81777429834523, 40.61750112169565], [-73.81652681189122, 40.61494700958006], [-73.82141601599992, 40.615054762999925], [-73.8238246859766, 40.61149866495507], [-73.82575322105271, 40.614192081540075], [-73.83776702143372, 40.61552223141874], [-73.83496222645587, 40.61778030881177], [-73.83514355859735, 40.62548708279667], [-73.83342230809954, 40.626781983462095], [-73.83464377224975, 40.62742707942435], [-73.83115393179625, 40.63256052752952], [-73.83340607979225, 40.63850135589897], [-73.82591657926903, 40.63574741296666], [-73.82443032497055, 40.6362165210408], [-73.82419327293158, 40.63995327448778], [-73.82337597260663, 40.63898704717671]]], [[[-73.84721834832125, 40.63196568670605], [-73.8485899664393, 40.63013198680654], [-73.8585583634798, 40.63045828819559], [-73.8561374187591, 40.63508154601468], [-73.84667914840234, 40.63873171175714], [-73.84352007697278, 40.636503685327924], [-73.84721834832125, 40.63196568670605]]], [[[-73.79577948747226, 40.63215847659964], [-73.79811872226199, 40.6310375092611], [-73.79827615415297, 40.63125217936581], [-73.7907928429953, 40.63428204310243], [-73.79577948747226, 40.63215847659964]]], [[[-73.83623731597828, 40.631110861231726], [-73.84110205447426, 40.62455643651514], [-73.84445711462239, 40.62984779443998], [-73.83765174362401, 40.63283973295598], [-73.83623731597828, 40.631110861231726]]], [[[-73.81307233892618, 40.62926109964263], [-73.81352502122876, 40.62730617268596], [-73.81258556038132, 40.628371916183724], [-73.81129432207656, 40.624703779000406], [-73.81089990481607, 40.61621847134095], [-73.81679073502552, 40.62634891812118], [-73.81382899493154, 40.62613507764411], [-73.81627026043981, 40.62714580591711], [-73.8173057692762, 40.63062197379782], [-73.81506788441455, 40.63116405857793], [-73.81307233892618, 40.62926109964263]]], [[[-73.84734350666976, 40.62909473971626], [-73.84984630167418, 40.62622449074592], [-73.8472783772798, 40.626621669791945], [-73.84358010755503, 40.623178164184395], [-73.86364988794934, 40.617878773642374], [-73.86404106276488, 40.61920759671402], [-73.86151825485233, 40.619049218399226], [-73.86102883383386, 40.624138986994076], [-73.86347880947007, 40.6233886093158], [-73.86334955670225, 40.62592490877777], [-73.86041082270874, 40.62516691297755], [-73.8565055097758, 40.629097353726586], [-73.8539654419115, 40.62761094389788], [-73.85436931062769, 40.62881767654389], [-73.84734350666976, 40.62909473971626]]], [[[-73.87084603194215, 40.61696742645758], [-73.8782477932424, 40.615444608331536], [-73.87481386865049, 40.62358718918363], [-73.86684384142526, 40.62782924286026], [-73.86498193495926, 40.625660706041025], [-73.86626127665735, 40.618456348023955], [-73.87084603194215, 40.61696742645758]]], [[[-73.79783533398513, 40.627408716571516], [-73.79566256726108, 40.626114004327796], [-73.79921493427759, 40.6240847129885], [-73.7936292748135, 40.62515647619494], [-73.79809093107174, 40.62306294824432], [-73.79992057632684, 40.61857524016598], [-73.80172498640165, 40.61996700873167], [-73.80128049504093, 40.615879839747535], [-73.80270343867807, 40.61358065232171], [-73.80508906012292, 40.61343256041943], [-73.80476592616557, 40.61580099137086], [-73.80764116896117, 40.62119965469225], [-73.80698502381061, 40.623871998829976], [-73.8001117714566, 40.626927441987085], [-73.80092841565032, 40.627524104895656], [-73.79783533398513, 40.627408716571516]]], [[[-73.77348616522504, 40.625085118163526], [-73.77381024409397, 40.624925596730954], [-73.77383258726536, 40.624944848253456], [-73.77348560409436, 40.62511314810922], [-73.77348616522504, 40.625085118163526]]], [[[-73.77172760292548, 40.62358502801516], [-73.7720313143135, 40.62299121290296], [-73.77218824028017, 40.622995793795134], [-73.77511891254433, 40.619233847487365], [-73.77518045935567, 40.61926471788484], [-73.77172760292548, 40.62358502801516]]], [[[-73.79162058913477, 40.62231334348308], [-73.78419483298926, 40.6208926699551], [-73.78446087067887, 40.617027835129704], [-73.78070887195283, 40.614026809005864], [-73.79139721011913, 40.60761085524994], [-73.79480633875043, 40.61037836048121], [-73.80080125919478, 40.611933904481596], [-73.7946062540462, 40.62257170055433], [-73.79162058913477, 40.62231334348308]]], [[[-73.76670827781236, 40.61491086618553], [-73.76825288003378, 40.614877725169315], [-73.77397692018825, 40.61600357567147], [-73.76873454755717, 40.62090086822023], [-73.76745926975256, 40.620511322031255], [-73.76670827781236, 40.61491086618553]]], [[[-73.84304421296773, 40.617737696390435], [-73.84454193473684, 40.614867921675135], [-73.8409994253963, 40.613029126216134], [-73.83825667290532, 40.613847216216406], [-73.8407944240835, 40.60564326638265], [-73.8471480968766, 40.60361424219198], [-73.85179402215192, 40.60689716858368], [-73.84779013405834, 40.60923538526278], [-73.84614804513843, 40.61772842036632], [-73.84304421296773, 40.617737696390435]]], [[[-73.85523067631884, 40.61613145335797], [-73.85530325465119, 40.6131454135728], [-73.84973986306498, 40.61515605814331], [-73.85010857555366, 40.61021219334599], [-73.8551559228834, 40.606120513001486], [-73.85708638173246, 40.60646824293355], [-73.85924855452187, 40.61125483087423], [-73.85648360749232, 40.61319069486381], [-73.85858056782772, 40.61691708722184], [-73.85523067631884, 40.61613145335797]]], [[[-73.814090115668, 40.614466920099765], [-73.81503638494628, 40.6129762137494], [-73.81615626666235, 40.61294211575494], [-73.8164315038978, 40.614282334964926], [-73.814090115668, 40.614466920099765]]], [[[-73.81496830662114, 40.61042065495561], [-73.81520720849484, 40.60981673379666], [-73.81559858789666, 40.61067251616471], [-73.81450261566708, 40.61105847350542], [-73.81496830662114, 40.61042065495561]]], [[[-73.86966442958492, 40.606540123412906], [-73.87109330768071, 40.604694173994694], [-73.87300122724774, 40.61055545845987], [-73.8710387011171, 40.60972358252655], [-73.86966442958492, 40.606540123412906]]], [[[-73.79420172624032, 40.607730675613354], [-73.79352152715475, 40.60632759139594], [-73.79509002319158, 40.607271060105596], [-73.79424799033224, 40.60590038003164], [-73.79696327795033, 40.60411784876779], [-73.80428530678597, 40.604519279530564], [-73.80520676583323, 40.60673953810017], [-73.80159206744115, 40.61023096250127], [-73.79420172624032, 40.607730675613354]]], [[[-73.83244207358176, 40.605929449536745], [-73.83587601623029, 40.605636907545566], [-73.83610156628845, 40.60736279382329], [-73.83252192275347, 40.610384993603816], [-73.83032947905777, 40.60848787991649], [-73.83045815392722, 40.606669201403214], [-73.83244207358176, 40.605929449536745]]], [[[-73.82607472604538, 40.60843779954612], [-73.82771104235977, 40.608398668130704], [-73.82805077260065, 40.61004629686527], [-73.82688326414805, 40.6095966780694], [-73.82607472604538, 40.60843779954612]]], [[[-73.86702399475149, 40.60806817876942], [-73.86874317559169, 40.60703517927931], [-73.86868468826752, 40.608241158183915], [-73.86702399475149, 40.60806817876942]]], [[[-73.82718282107048, 40.607919778091194], [-73.82590092720767, 40.60646593064943], [-73.82861671457653, 40.60583848413761], [-73.82815085191197, 40.60795028983291], [-73.82718282107048, 40.607919778091194]]], [[[-73.81569926225602, 40.6064570673649], [-73.81546066483351, 40.60642813700742], [-73.81552816722274, 40.606163069945154], [-73.81552536723665, 40.60640738028192], [-73.81569926225602, 40.6064570673649]]], [[[-73.81339665223344, 40.60436407224534], [-73.80986333857152, 40.60189579000784], [-73.81157550345523, 40.59814119925956], [-73.81501471568424, 40.60205629156354], [-73.81339665223344, 40.60436407224534]]], [[[-73.86702905185578, 40.603743388853935], [-73.8676819415421, 40.603070766677334], [-73.8683080846461, 40.603139161146146], [-73.86793505372694, 40.60396113545295], [-73.86702905185578, 40.603743388853935]]], [[[-73.85243335807012, 40.59733339779527], [-73.86044420534076, 40.596471905596275], [-73.86335999147606, 40.600322202771196], [-73.86112814224546, 40.60255033107426], [-73.85482595782968, 40.601867195353634], [-73.85076748001559, 40.598839443837406], [-73.85243335807012, 40.59733339779527]]], [[[-73.87019491489939, 40.598996405901886], [-73.87108141811476, 40.598901147470094], [-73.87049411914771, 40.602604446204595], [-73.87026280564918, 40.60168122225861], [-73.87019491489939, 40.598996405901886]]], [[[-73.8231174097477, 40.60007949252535], [-73.82315293179705, 40.599976662824794], [-73.82317356168161, 40.59997800515576], [-73.82318015964296, 40.60008384902813], [-73.8231174097477, 40.60007949252535]]], [[[-73.82292757811203, 40.60006142816309], [-73.822978576463, 40.600043707224835], [-73.82303195183424, 40.59993051694214], [-73.82304007849375, 40.60006807478574], [-73.82292757811203, 40.60006142816309]]], [[[-73.82445629476595, 40.59911583868227], [-73.82395871079126, 40.59887689683851], [-73.82477088992478, 40.5983918438742], [-73.82453084258555, 40.59911949829181], [-73.82445629476595, 40.59911583868227]]], [[[-73.86952707548932, 40.598713750354406], [-73.86967070711478, 40.59668856362211], [-73.87020899836229, 40.59695423278937], [-73.86952707548932, 40.598713750354406]]], [[[-73.83668274106707, 40.5949466970158], [-73.8325749810638, 40.592118209113956], [-73.83422777067327, 40.59137381382269], [-73.83253569485043, 40.59021893579835], [-73.83495422767699, 40.589728155618005], [-73.83347866645357, 40.589368748710896], [-73.83719541135855, 40.58969911736292], [-73.8398907019092, 40.59280021149386], [-73.84385894083933, 40.59342531196059], [-73.83902964248091, 40.59625348474866], [-73.83668274106707, 40.5949466970158]]], [[[-73.86706149472118, 40.5820879767934], [-73.87000313642912, 40.582636614815954], [-73.87115922675414, 40.5868738879673], [-73.86635052164085, 40.591892676564676], [-73.86327273492819, 40.59090807569255], [-73.86125299377363, 40.58798808592123], [-73.86327471071942, 40.58387684853184], [-73.86706149472118, 40.5820879767934]]]]}}, {\"id\": \"130\", \"type\": \"Feature\", \"properties\": {\"count\": 1339, \"OBJECTID\": 131, \"Shape_Leng\": 0.116547120922, \"Shape_Area\": 0.000423696152789, \"zone\": \"Jamaica Estates\", \"LocationID\": 131, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.77732562899998, 40.722392700999926], [-73.76924074899993, 40.72590270099988], [-73.75669771299987, 40.7262287109999], [-73.7523343839998, 40.72026246099993], [-73.79303800099989, 40.71071107499988], [-73.79358570299998, 40.714128677999874], [-73.790302674, 40.7208914649999], [-73.79174472699992, 40.725788849999944], [-73.77757506899988, 40.73019327299987], [-73.7752338719998, 40.725237532999884], [-73.77803765999981, 40.723284268999855], [-73.77732562899998, 40.722392700999926]]]}}, {\"id\": \"132\", \"type\": \"Feature\", \"properties\": {\"count\": 9968, \"OBJECTID\": 133, \"Shape_Leng\": 0.065134863162, \"Shape_Area\": 0.000157188987507, \"zone\": \"Kensington\", \"LocationID\": 133, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97819074399992, 40.64752599199992], [-73.97777380600002, 40.645341876999964], [-73.974642245, 40.644813218999914], [-73.97084113799984, 40.64637857099989], [-73.96803868499985, 40.636650984999925], [-73.97029036199991, 40.63640634099988], [-73.96986869299992, 40.63419467499989], [-73.97604935699995, 40.63127590599986], [-73.9768886119999, 40.6356748619999], [-73.97911188699993, 40.63544038599991], [-73.98042057899988, 40.64238522699991], [-73.98299419199994, 40.64132499699987], [-73.98834986699991, 40.64456224399992], [-73.97819074399992, 40.64752599199992]]]}}, {\"id\": \"133\", \"type\": \"Feature\", \"properties\": {\"count\": 8060, \"OBJECTID\": 134, \"Shape_Leng\": 0.0695878223284, \"Shape_Area\": 0.000202375182143, \"zone\": \"Kew Gardens\", \"LocationID\": 134, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82288529599995, 40.71192805799993], [-73.81637824499992, 40.70245196399987], [-73.82245890399994, 40.70365553099989], [-73.83152109599999, 40.70138962899994], [-73.83460244999996, 40.70392746699988], [-73.83661882199989, 40.70342001799988], [-73.83809584699989, 40.70616458099993], [-73.83714702999997, 40.70795704599989], [-73.84314625299982, 40.70922345599992], [-73.82591945199992, 40.71598987599992], [-73.82288529599995, 40.71192805799993]]]}}, {\"id\": \"134\", \"type\": \"Feature\", \"properties\": {\"count\": 3424, \"OBJECTID\": 135, \"Shape_Leng\": 0.121793045659, \"Shape_Area\": 0.000467912240542, \"zone\": \"Kew Gardens Hills\", \"LocationID\": 135, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83126834299986, 40.74026227599991], [-73.8294675199999, 40.73702448299991], [-73.8243501109999, 40.73975360599992], [-73.81497208299996, 40.738560715999874], [-73.81510024099985, 40.72854342299991], [-73.81074426599986, 40.72626511299991], [-73.8111179729999, 40.72781641599994], [-73.80717042299993, 40.72826913299987], [-73.80652142399995, 40.7248730639999], [-73.80487958299983, 40.725049592999895], [-73.80452680199981, 40.721462545999884], [-73.80904032699983, 40.71991160499987], [-73.80780927699995, 40.715853895999906], [-73.82018883899984, 40.71673522399984], [-73.81732944799997, 40.717660456999916], [-73.82531644999993, 40.71727569999994], [-73.8273090039999, 40.72464607699988], [-73.83073214599987, 40.72654932799991], [-73.83635625099998, 40.73500859199989], [-73.83564585599999, 40.738491010999894], [-73.83699037399995, 40.74209330099992], [-73.83586131899989, 40.743227984999926], [-73.83113122799989, 40.74233799999994], [-73.83126834299986, 40.74026227599991]]]}}, {\"id\": \"135\", \"type\": \"Feature\", \"properties\": {\"count\": 2037, \"OBJECTID\": 136, \"Shape_Leng\": 0.0906287154122, \"Shape_Area\": 0.000148524994129, \"zone\": \"Kingsbridge Heights\", \"LocationID\": 136, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89996501599988, 40.86801852399986], [-73.89891854299988, 40.86965857399992], [-73.89656060599998, 40.868565199999935], [-73.90486847199993, 40.8586932249999], [-73.90777320599983, 40.85960753799984], [-73.90976165899988, 40.857182611999896], [-73.91328437100006, 40.859618600999916], [-73.90908822499988, 40.8623996959999], [-73.91422823999919, 40.862490790359615], [-73.91044282747865, 40.866527868551835], [-73.90746489699994, 40.87354735399985], [-73.90452070499992, 40.87249521099988], [-73.90318667499999, 40.86914899099988], [-73.89996501599988, 40.86801852399986]]]}}, {\"id\": \"136\", \"type\": \"Feature\", \"properties\": {\"count\": 1860394, \"OBJECTID\": 137, \"Shape_Leng\": 0.0461076242707, \"Shape_Area\": 0.000116455098947, \"zone\": \"Kips Bay\", \"LocationID\": 137, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97219641647804, 40.74040257875022], [-73.97248994174741, 40.73580328010761], [-73.97446647047282, 40.736278456829886], [-73.9735447289197, 40.73494432655072], [-73.98266497499989, 40.73949733299987], [-73.97812378199983, 40.745727172999885], [-73.97167921200406, 40.74305446402989], [-73.97219641647804, 40.74040257875022]]]}}, {\"id\": \"137\", \"type\": \"Feature\", \"properties\": {\"count\": 3515777, \"OBJECTID\": 138, \"Shape_Leng\": 0.107466933508, \"Shape_Area\": 0.000536797294367, \"zone\": \"LaGuardia Airport\", \"LocationID\": 138, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.8712485624251, 40.78603815125499], [-73.86918113468879, 40.78594348978492], [-73.86847981674143, 40.78369082919537], [-73.87255732673727, 40.78082112154256], [-73.85505106271819, 40.772195389989086], [-73.8582051105021, 40.77032345647716], [-73.85624565746174, 40.768885714198944], [-73.86268864176631, 40.76687591050486], [-73.86043337809801, 40.76386888243776], [-73.86728193799998, 40.76965000299989], [-73.87223577699984, 40.77169565899993], [-73.87743339799992, 40.77120025499991], [-73.88770661799987, 40.76683859799995], [-73.88945486901865, 40.77353295106357], [-73.88391627963797, 40.774132606591536], [-73.88493466381185, 40.77993691280691], [-73.87870562518874, 40.78058590995002], [-73.878858374856, 40.78239817868118], [-73.87974777853563, 40.782836564207], [-73.8797016367226, 40.78288281807235], [-73.8751575489337, 40.781509449653285], [-73.8712485624251, 40.78603815125499]]], [[[-73.8728719590193, 40.785975027902346], [-73.87282839783694, 40.78595446173833], [-73.8727740594054, 40.78594192145881], [-73.87280920016116, 40.78590310909999], [-73.8728719590193, 40.785975027902346]]], [[[-73.89008921683674, 40.77362643576237], [-73.89034195999996, 40.77323760499998], [-73.89183244902246, 40.77488019994431], [-73.89017369508576, 40.77785081277782], [-73.89008921683674, 40.77362643576237]]]]}}, {\"id\": \"138\", \"type\": \"Feature\", \"properties\": {\"count\": 323, \"OBJECTID\": 139, \"Shape_Leng\": 0.0932997884483, \"Shape_Area\": 0.000446669880154, \"zone\": \"Laurelton\", \"LocationID\": 139, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.74125253099989, 40.68745783899989], [-73.72718272499995, 40.68359732999987], [-73.74117817299991, 40.666402953999885], [-73.75763129599993, 40.66645115699992], [-73.75636239300002, 40.67242923699999], [-73.75857198899986, 40.67263733099987], [-73.76031463999993, 40.67510997099984], [-73.75576169199995, 40.67588590899988], [-73.747551754, 40.689347336999944], [-73.74125253099989, 40.68745783899989]]]}}, {\"id\": \"139\", \"type\": \"Feature\", \"properties\": {\"count\": 2490848, \"OBJECTID\": 140, \"Shape_Leng\": 0.0475842911325, \"Shape_Area\": 0.000114203907779, \"zone\": \"Lenox Hill East\", \"LocationID\": 140, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95854329689507, 40.758495622967885], [-73.96196573699997, 40.759619460999865], [-73.95268514999995, 40.772339404999926], [-73.9474897547817, 40.77011515400953], [-73.95854329689507, 40.758495622967885]]]}}, {\"id\": \"140\", \"type\": \"Feature\", \"properties\": {\"count\": 3383595, \"OBJECTID\": 141, \"Shape_Leng\": 0.0415144638712, \"Shape_Area\": 7.66545579019e-05, \"zone\": \"Lenox Hill West\", \"LocationID\": 141, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96177668399997, 40.75987971599993], [-73.9665833839999, 40.76155093499988], [-73.95729409999986, 40.7742835549999], [-73.95268514999995, 40.772339404999926], [-73.96177668399997, 40.75987971599993]]]}}, {\"id\": \"141\", \"type\": \"Feature\", \"properties\": {\"count\": 4095043, \"OBJECTID\": 142, \"Shape_Leng\": 0.0381758942321, \"Shape_Area\": 7.56537920738e-05, \"zone\": \"Lincoln Square East\", \"LocationID\": 142, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98147898799989, 40.7677250589999], [-73.98806289599996, 40.769790799999925], [-73.980673654, 40.77992518199991], [-73.97499744000007, 40.77753254599988], [-73.98147898799989, 40.7677250589999]]]}}, {\"id\": \"142\", \"type\": \"Feature\", \"properties\": {\"count\": 1480265, \"OBJECTID\": 143, \"Shape_Leng\": 0.0541798538849, \"Shape_Area\": 0.00015109426901, \"zone\": \"Lincoln Square West\", \"LocationID\": 143, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.981627874, 40.780328934999936], [-73.980673654, 40.77992518199991], [-73.98806289599996, 40.769790799999925], [-73.99624065194922, 40.77378979119886], [-73.99393587681126, 40.773179512586104], [-73.98886861739992, 40.779692922911416], [-73.9915510592274, 40.779574821437386], [-73.98812746184332, 40.78140179672363], [-73.981627874, 40.780328934999936]]]}}, {\"id\": \"143\", \"type\": \"Feature\", \"properties\": {\"count\": 1793176, \"OBJECTID\": 144, \"Shape_Leng\": 0.0276201668505, \"Shape_Area\": 4.74789670369e-05, \"zone\": \"Little Italy/NoLiTa\", \"LocationID\": 144, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99580912300002, 40.71664465899988], [-74.00188406299992, 40.71939757099991], [-73.9967717579999, 40.7254319439999], [-73.99260322299989, 40.724136449999925], [-73.99580912300002, 40.71664465899988]]]}}, {\"id\": \"144\", \"type\": \"Feature\", \"properties\": {\"count\": 216874, \"OBJECTID\": 145, \"Shape_Leng\": 0.114313844981, \"Shape_Area\": 0.000346641733797, \"zone\": \"Long Island City/Hunters Point\", \"LocationID\": 145, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94345411699997, 40.75205360799992], [-73.93663681899982, 40.748204093999895], [-73.94419561599983, 40.742882145999914], [-73.93726110699994, 40.745176490999896], [-73.9386966449998, 40.73814081699989], [-73.94140886345744, 40.73929957085316], [-73.93866991753185, 40.74253408202486], [-73.9400192297566, 40.74320827221074], [-73.94159470030033, 40.73986412616895], [-73.94548236590111, 40.738207031293705], [-73.95380883920839, 40.73982029297784], [-73.96256114655104, 40.73868995735329], [-73.95978468552276, 40.74349771658391], [-73.96064863700711, 40.74413031761407], [-73.9587753700448, 40.74455559986341], [-73.95965229755977, 40.745216588173435], [-73.95849389303115, 40.74498786491623], [-73.9567945329049, 40.748839516955584], [-73.95318382492853, 40.74773481196097], [-73.9564856879466, 40.74911169015581], [-73.95080763809156, 40.75526368005708], [-73.94345411699997, 40.75205360799992]]]}}, {\"id\": \"145\", \"type\": \"Feature\", \"properties\": {\"count\": 155778, \"OBJECTID\": 146, \"Shape_Leng\": 0.0471405215717, \"Shape_Area\": 0.000104638923425, \"zone\": \"Long Island City/Queens Plaza\", \"LocationID\": 146, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93675697199996, 40.74914697199991], [-73.944052652, 40.75231325999992], [-73.93679091699993, 40.75988288099991], [-73.92415699999985, 40.753961141999845], [-73.92562743799989, 40.752137243999904], [-73.93383269399993, 40.7516047019999], [-73.93675697199996, 40.74914697199991]]]}}, {\"id\": \"146\", \"type\": \"Feature\", \"properties\": {\"count\": 1131, \"OBJECTID\": 147, \"Shape_Leng\": 0.0587654949268, \"Shape_Area\": 0.000106418172757, \"zone\": \"Longwood\", \"LocationID\": 147, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89515277899991, 40.82783428999991], [-73.89487088500002, 40.82393727199992], [-73.89320004999998, 40.82400818299992], [-73.89586019599982, 40.82263104599987], [-73.8958813589999, 40.82099257499987], [-73.89290156199988, 40.82081122299989], [-73.8970446669999, 40.8168018599999], [-73.89542858799986, 40.81584467199988], [-73.90307457199998, 40.80988061199988], [-73.9044655149999, 40.81228195999994], [-73.9006114109999, 40.822108807999875], [-73.90198758599993, 40.82244042299992], [-73.90093510699997, 40.82792810199992], [-73.89515277899991, 40.82783428999991]]]}}, {\"id\": \"147\", \"type\": \"Feature\", \"properties\": {\"count\": 2276043, \"OBJECTID\": 148, \"Shape_Leng\": 0.0391305015633, \"Shape_Area\": 6.97489921327e-05, \"zone\": \"Lower East Side\", \"LocationID\": 148, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98447731699999, 40.72023423899987], [-73.98675136699994, 40.71959284599997], [-73.99022012699989, 40.71440493999988], [-73.99750445299988, 40.71406913199995], [-73.99260322199994, 40.72413644999987], [-73.98382387299982, 40.72147287199987], [-73.98447731699999, 40.72023423899987]]]}}, {\"id\": \"148\", \"type\": \"Feature\", \"properties\": {\"count\": 1035, \"OBJECTID\": 149, \"Shape_Leng\": 0.0836805821837, \"Shape_Area\": 0.000270639618955, \"zone\": \"Madison\", \"LocationID\": 149, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94405560899996, 40.61199284199993], [-73.94231175899985, 40.60826819699985], [-73.93934703399997, 40.610187140999884], [-73.93753749399987, 40.60855738999986], [-73.93972613899992, 40.60713291499995], [-73.93447158899997, 40.60123151199989], [-73.94288954099984, 40.600318649999934], [-73.94248473299992, 40.598200364999855], [-73.956927197, 40.596612075999865], [-73.95859278499987, 40.61040303099988], [-73.94978380499997, 40.611371749999854], [-73.95026514899993, 40.61392435199983], [-73.9448277099998, 40.61604620899986], [-73.94405560899996, 40.61199284199993]]]}}, {\"id\": \"149\", \"type\": \"Feature\", \"properties\": {\"count\": 663, \"OBJECTID\": 150, \"Shape_Leng\": 0.089331491183, \"Shape_Area\": 0.000332568036088, \"zone\": \"Manhattan Beach\", \"LocationID\": 150, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9301907789999, 40.58508559299986], [-73.9319281118675, 40.58286976893519], [-73.95349322048675, 40.58298819826098], [-73.94274062592493, 40.58099558955925], [-73.93258006568031, 40.58126474120341], [-73.93106028107997, 40.57626592009687], [-73.9523584672133, 40.574274685845204], [-73.95414287899986, 40.58310613999989], [-73.95900739999992, 40.58283812799991], [-73.96025825899996, 40.58502930799994], [-73.9344471859999, 40.586652074999925], [-73.9301907789999, 40.58508559299986]]]}}, {\"id\": \"150\", \"type\": \"Feature\", \"properties\": {\"count\": 1182328, \"OBJECTID\": 151, \"Shape_Leng\": 0.0548899410999, \"Shape_Area\": 0.000128848901918, \"zone\": \"Manhattan Valley\", \"LocationID\": 151, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96371096499993, 40.7930129339999], [-73.97639951915279, 40.79827321034625], [-73.97287179090726, 40.8033561875739], [-73.96004456499999, 40.79804123499991], [-73.96371096499993, 40.7930129339999]]]}}, {\"id\": \"151\", \"type\": \"Feature\", \"properties\": {\"count\": 137419, \"OBJECTID\": 152, \"Shape_Leng\": 0.0579094768782, \"Shape_Area\": 0.000146637644064, \"zone\": \"Manhattanville\", \"LocationID\": 152, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95357262899982, 40.82197589199988], [-73.94607828700005, 40.821263215999906], [-73.95210125199992, 40.811442853999935], [-73.96203105797969, 40.81808633074085], [-73.95881892233997, 40.82151852484672], [-73.9596033246027, 40.82299468678336], [-73.95793904909117, 40.82277435354731], [-73.9595089796959, 40.82359578895142], [-73.95357262899982, 40.82197589199988]]]}}, {\"id\": \"152\", \"type\": \"Feature\", \"properties\": {\"count\": 1485, \"OBJECTID\": 153, \"Shape_Leng\": 0.024737239022, \"Shape_Area\": 3.24684964817e-05, \"zone\": \"Marble Hill\", \"LocationID\": 153, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90666530999995, 40.8757156089999], [-73.90893235220783, 40.87215734798524], [-73.91578609320084, 40.87571718248032], [-73.91033193599998, 40.879038046999895], [-73.90666530999995, 40.8757156089999]]]}}, {\"id\": \"153\", \"type\": \"Feature\", \"properties\": {\"count\": 379, \"OBJECTID\": 154, \"Shape_Leng\": 0.22733157076, \"Shape_Area\": 0.00178968692433, \"zone\": \"Marine Park/Floyd Bennett Field\", \"LocationID\": 154, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.93531145199985, 40.60886003999987], [-73.92986880199989, 40.60507134099995], [-73.92624885099994, 40.60741352199989], [-73.92444144399987, 40.60579356899993], [-73.91970928699986, 40.607302162999886], [-73.91331444593793, 40.60225844819831], [-73.90903689530528, 40.60267734322018], [-73.91073123013267, 40.60202137602758], [-73.90894225138031, 40.60160704069869], [-73.8839804295303, 40.60556651972071], [-73.87671672482064, 40.584914430200875], [-73.88174676983894, 40.5791588072124], [-73.89552381007165, 40.576769818436354], [-73.89903425420563, 40.587676171745805], [-73.90426874447125, 40.58691226298856], [-73.90651874663754, 40.588025473892536], [-73.91148171990072, 40.586078718712315], [-73.91204234015626, 40.594506954288846], [-73.91513297459494, 40.59878111593029], [-73.92151689981944, 40.60192365414108], [-73.92457042932716, 40.599723602732396], [-73.92950671202776, 40.60270923350431], [-73.92927458471804, 40.60399357480484], [-73.93117476161734, 40.603850406434695], [-73.93108919528449, 40.601722258264104], [-73.91454059121104, 40.58994422917544], [-73.91819087799986, 40.58660685499996], [-73.92956858299992, 40.5968202649999], [-73.92813139699999, 40.59777792799987], [-73.93972613899992, 40.60713291499995], [-73.93531145199985, 40.60886003999987]]], [[[-73.91990064335972, 40.59960052259278], [-73.91626393108346, 40.59816113731273], [-73.91295122631982, 40.59205432996962], [-73.91524478304683, 40.59158750817847], [-73.92182277731894, 40.59733293287253], [-73.92213049353394, 40.598662596177306], [-73.91990064335972, 40.59960052259278]]], [[[-73.91513331918175, 40.58626938540072], [-73.91282142952436, 40.58589683033704], [-73.91242103069457, 40.58339278402727], [-73.91729861410717, 40.58319080174952], [-73.91191058860355, 40.58296337725262], [-73.9114996013066, 40.58181878146464], [-73.92443604094977, 40.58355820344426], [-73.93184655018902, 40.58294105529734], [-73.9320424499999, 40.5842339599999], [-73.9300961629999, 40.584502075999886], [-73.93139947599984, 40.58607989499989], [-73.92101255523748, 40.58457501880612], [-73.91513331918175, 40.58626938540072]]]]}}, {\"id\": \"154\", \"type\": \"Feature\", \"properties\": {\"count\": 971, \"OBJECTID\": 155, \"Shape_Leng\": 0.169808473032, \"Shape_Area\": 0.000868261018498, \"zone\": \"Marine Park/Mill Basin\", \"LocationID\": 155, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90464209220775, 40.62513319188153], [-73.90546206137252, 40.62577481968345], [-73.9022870793228, 40.624409853930224], [-73.90071952540437, 40.62163492049855], [-73.89644123346895, 40.621394756190135], [-73.89586112894177, 40.614518137494784], [-73.89095414564672, 40.61283404731705], [-73.89394013317667, 40.61118142665126], [-73.88971513183803, 40.611598996904085], [-73.89334621420008, 40.606743446234645], [-73.89986880363854, 40.605524640544004], [-73.90133303458649, 40.611717882905985], [-73.90249392985292, 40.611814999032276], [-73.90119253465456, 40.61056381754662], [-73.90132962659396, 40.61048104310328], [-73.90879088017641, 40.617436396721416], [-73.90797306390525, 40.615736844061175], [-73.90959740910328, 40.616392669799325], [-73.90225859966276, 40.61042277259991], [-73.90177669809985, 40.605674686555815], [-73.90524300992108, 40.60491009032442], [-73.90863959535422, 40.606658722047854], [-73.90803306097814, 40.60535860211721], [-73.9098515484547, 40.605999722177685], [-73.9083253683517, 40.60407337391859], [-73.91365119759595, 40.60398251383695], [-73.91708838503304, 40.608820854898525], [-73.91449542205973, 40.61165322366488], [-73.91428072663338, 40.61493881616294], [-73.91662804371141, 40.6134435892654], [-73.91573181927129, 40.61183084451381], [-73.91811036679019, 40.61020839405809], [-73.91813814889971, 40.607645386907706], [-73.91935398484114, 40.60804292228766], [-73.91825269092949, 40.6072293506405], [-73.91950358918922, 40.60797490397521], [-73.91794980550603, 40.60620925217101], [-73.92158429099997, 40.607639084999875], [-73.92444144399987, 40.60579356899993], [-73.92624885099994, 40.60741352199989], [-73.92986880199989, 40.60507134099995], [-73.936046277, 40.609514868999966], [-73.93753749399987, 40.60855738999986], [-73.93934703399997, 40.610187140999884], [-73.94231175899985, 40.60826819699985], [-73.94354698000001, 40.6093465549999], [-73.9448277099998, 40.61604620899986], [-73.93686699599992, 40.620253280999904], [-73.92629604599998, 40.61311679899991], [-73.92640044599996, 40.61447853499988], [-73.9205269409999, 40.61394269499987], [-73.91690302699992, 40.61627422899993], [-73.91860649599992, 40.632090414999894], [-73.90464209220775, 40.62513319188153]]]}}, {\"id\": \"155\", \"type\": \"Feature\", \"properties\": {\"count\": 181, \"OBJECTID\": 156, \"Shape_Leng\": 0.144476890476, \"Shape_Area\": 0.0010521217443, \"zone\": \"Mariners Harbor\", \"LocationID\": 156, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-74.17989568340026, 40.64526879647802], [-74.1795871372141, 40.64191768346404], [-74.17918728067563, 40.64298753999647], [-74.1763578661754, 40.64232722056289], [-74.17436572943608, 40.64513625864542], [-74.17133587281808, 40.642855765052], [-74.17298798800311, 40.640489105066244], [-74.17105017239898, 40.642593819129914], [-74.1660998989604, 40.642406587001155], [-74.1658011075589, 40.64122451144941], [-74.16494139041178, 40.64220899988932], [-74.16519041758224, 40.64064230577938], [-74.16167534694229, 40.640610784766764], [-74.1622846525209, 40.63876565277233], [-74.16034819993155, 40.63844372645346], [-74.15986410465037, 40.639927823692716], [-74.1592271162294, 40.637796119756985], [-74.15900957887435, 40.63882257790974], [-74.15718782782525, 40.63795080522895], [-74.15710126690709, 40.63927771860018], [-74.15707077305574, 40.63794713564092], [-74.15482932461856, 40.63772385541876], [-74.15469263864514, 40.63919396811424], [-74.15459670163321, 40.63734334406899], [-74.15279703511403, 40.637132754852544], [-74.15124827966567, 40.63940934288591], [-74.14855815092196, 40.63743783752723], [-74.14877792005885, 40.6388762511277], [-74.14312802471521, 40.63968037306379], [-74.14752618999992, 40.62938928799988], [-74.14554071999994, 40.62783252199995], [-74.14650354399993, 40.62420010099991], [-74.15231170399994, 40.61716275599988], [-74.15379116699992, 40.612255611999906], [-74.1623545779999, 40.618335637999955], [-74.17459706299995, 40.622364010999874], [-74.1955597527974, 40.635023986316575], [-74.19462310145185, 40.637391183563665], [-74.18652496897599, 40.64338866217145], [-74.18483594580836, 40.644128266338726], [-74.18365067064063, 40.642239641050644], [-74.1836674587201, 40.64461048505006], [-74.17989568340026, 40.64526879647802]]], [[[-74.15945602438187, 40.641448333324036], [-74.16111242522163, 40.64183545373729], [-74.16146036002637, 40.644294969882374], [-74.15743349200979, 40.643302857779], [-74.15945602438187, 40.641448333324036]]]]}}, {\"id\": \"156\", \"type\": \"Feature\", \"properties\": {\"count\": 7409, \"OBJECTID\": 157, \"Shape_Leng\": 0.131300217777, \"Shape_Area\": 0.000354370128323, \"zone\": \"Maspeth\", \"LocationID\": 157, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8978323689999, 40.73516035399992], [-73.8885228599999, 40.734531159999904], [-73.8870771209998, 40.72463770199994], [-73.90010558699989, 40.72329624099992], [-73.899816102, 40.721236899999894], [-73.89739704899986, 40.7215512099999], [-73.89736598399989, 40.72046438299989], [-73.90062644399997, 40.719764053999874], [-73.90033904299986, 40.718019947999885], [-73.89834760099983, 40.719197677999894], [-73.897267579, 40.71814715699985], [-73.89901821099994, 40.71738212699993], [-73.89750226399987, 40.716383867999916], [-73.90477244599997, 40.71286579999984], [-73.90639010099999, 40.71736369199989], [-73.90965980099996, 40.71779315499991], [-73.91283037699982, 40.71536090699993], [-73.91077807399988, 40.71318759999986], [-73.92404011299993, 40.714008312999916], [-73.92352547251075, 40.71561660836783], [-73.92064258940593, 40.715773192028834], [-73.92232773978448, 40.71659264406108], [-73.91132095099978, 40.71962422999992], [-73.91384828299984, 40.72289841199989], [-73.905180579, 40.72895657699995], [-73.90041661699985, 40.73465058799992], [-73.8978323689999, 40.73516035399992]]]}}, {\"id\": \"157\", \"type\": \"Feature\", \"properties\": {\"count\": 2021710, \"OBJECTID\": 158, \"Shape_Leng\": 0.0548099905515, \"Shape_Area\": 0.000185568253002, \"zone\": \"Meatpacking/West Village West\", \"LocationID\": 158, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.0077911609999, 40.74197084499982], [-74.00515808199998, 40.74085808099991], [-74.0070902079999, 40.72877225299993], [-74.014390699162, 40.728463047760016], [-74.01408693559922, 40.730663173040284], [-74.01116195696464, 40.730450676349065], [-74.01099350879154, 40.732320417940606], [-74.01399957240955, 40.733332277517704], [-74.01072514326883, 40.733459523511115], [-74.01201792010478, 40.734063744456314], [-74.01081359479103, 40.734283443647534], [-74.0103930313456, 40.739174296059204], [-74.01265928208485, 40.74074763849748], [-74.0077911609999, 40.74197084499982]]]}}, {\"id\": \"158\", \"type\": \"Feature\", \"properties\": {\"count\": 7860, \"OBJECTID\": 159, \"Shape_Leng\": 0.064363233206, \"Shape_Area\": 0.00017124910387, \"zone\": \"Melrose South\", \"LocationID\": 159, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91189154999992, 40.822141481999914], [-73.9012927759998, 40.820475442999914], [-73.9044655149999, 40.81228195999994], [-73.91768626999985, 40.8160812429999], [-73.92048117399989, 40.8145636869999], [-73.92309299899993, 40.81669772799995], [-73.92399951299991, 40.81543891199991], [-73.92662718199988, 40.816118958999915], [-73.91677400599986, 40.82438708699987], [-73.91189154999992, 40.822141481999914]]]}}, {\"id\": \"159\", \"type\": \"Feature\", \"properties\": {\"count\": 2917, \"OBJECTID\": 160, \"Shape_Leng\": 0.14151914297, \"Shape_Area\": 0.000571234977994, \"zone\": \"Middle Village\", \"LocationID\": 160, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.869707654, 40.72514643399992], [-73.85962920999985, 40.713975294999926], [-73.87016462499984, 40.70920774299987], [-73.86911007199988, 40.707089336999914], [-73.88088332699985, 40.7064539769999], [-73.87761134199985, 40.70716770499993], [-73.88006267699978, 40.71265257399993], [-73.90477244599997, 40.71286579999984], [-73.89750226399987, 40.716383867999916], [-73.89901821099994, 40.71738212699993], [-73.897267579, 40.71814715699985], [-73.89834760099983, 40.719197677999894], [-73.90033904299986, 40.718019947999885], [-73.90062644399997, 40.719764053999874], [-73.89736598399989, 40.72046438299989], [-73.89739704899986, 40.7215512099999], [-73.899816102, 40.721236899999894], [-73.89903191799989, 40.723856959999864], [-73.89426494199992, 40.723143188999856], [-73.8870771209998, 40.72463770199994], [-73.88720524999994, 40.727792996999874], [-73.87509154199992, 40.73067136299992], [-73.87146074199987, 40.729326373999854], [-73.869707654, 40.72514643399992]]]}}, {\"id\": \"160\", \"type\": \"Feature\", \"properties\": {\"count\": 5054217, \"OBJECTID\": 161, \"Shape_Leng\": 0.0358039100611, \"Shape_Area\": 7.19130682992e-05, \"zone\": \"Midtown Center\", \"LocationID\": 161, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97534377999993, 40.7552996949999], [-73.9783696929999, 40.752416226999905], [-73.98411754799999, 40.75484205299995], [-73.977686006, 40.76364440299992], [-73.97124277300004, 40.76093641799984], [-73.97534377999993, 40.7552996949999]]]}}, {\"id\": \"161\", \"type\": \"Feature\", \"properties\": {\"count\": 4758657, \"OBJECTID\": 162, \"Shape_Leng\": 0.0352698146219, \"Shape_Area\": 4.78936962112e-05, \"zone\": \"Midtown East\", \"LocationID\": 162, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9712170449999, 40.75519332399994], [-73.9744494109998, 40.75076644499991], [-73.9783696929999, 40.752416226999905], [-73.96980028899992, 40.76291351199995], [-73.9665833839999, 40.76155093499988], [-73.9712170449999, 40.75519332399994]]]}}, {\"id\": \"162\", \"type\": \"Feature\", \"properties\": {\"count\": 3885186, \"OBJECTID\": 163, \"Shape_Leng\": 0.0341768669752, \"Shape_Area\": 4.08567804196e-05, \"zone\": \"Midtown North\", \"LocationID\": 163, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98136213699995, 40.767836738999904], [-73.9736358469999, 40.7645699129999], [-73.96980028899992, 40.76291351199995], [-73.97124277300004, 40.76093641799984], [-73.98428230099987, 40.76479181499994], [-73.98136213699995, 40.767836738999904]]]}}, {\"id\": \"163\", \"type\": \"Feature\", \"properties\": {\"count\": 3586399, \"OBJECTID\": 164, \"Shape_Leng\": 0.0357716119908, \"Shape_Area\": 5.56318237441e-05, \"zone\": \"Midtown South\", \"LocationID\": 164, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98362519999995, 40.7439431149999], [-73.99051761299985, 40.746038637999895], [-73.98411754799999, 40.75484205299995], [-73.98088709899987, 40.753480988999875], [-73.98407636100002, 40.74910212499993], [-73.98085054499998, 40.74775036999995], [-73.98362519999995, 40.7439431149999]]]}}, {\"id\": \"164\", \"type\": \"Feature\", \"properties\": {\"count\": 3181, \"OBJECTID\": 165, \"Shape_Leng\": 0.088608036213, \"Shape_Area\": 0.000353959317019, \"zone\": \"Midwood\", \"LocationID\": 165, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96014773499996, 40.628915184999876], [-73.95995565099992, 40.627778671999884], [-73.94732672199994, 40.629166566999835], [-73.94451833400001, 40.61971364699989], [-73.94651373199997, 40.61949401899995], [-73.94573691999982, 40.61536340699989], [-73.95026514899993, 40.61392435199983], [-73.94978380499997, 40.611371749999854], [-73.96237947799978, 40.609986373999966], [-73.96471983399981, 40.62237296099993], [-73.96864510599995, 40.6219416949999], [-73.97136622199992, 40.62892916199996], [-73.96040734299989, 40.6301625309999], [-73.96014773499996, 40.628915184999876]]]}}, {\"id\": \"165\", \"type\": \"Feature\", \"properties\": {\"count\": 702932, \"OBJECTID\": 166, \"Shape_Leng\": 0.0688237542871, \"Shape_Area\": 0.000255276270405, \"zone\": \"Morningside Heights\", \"LocationID\": 166, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95707738499995, 40.8142980939999], [-73.95210125199992, 40.811442853999935], [-73.9535757779999, 40.8094766979999], [-73.95496657099991, 40.81006455499994], [-73.95964685399987, 40.80115642299993], [-73.97110765876137, 40.80579013958964], [-73.96203105797969, 40.81808633074085], [-73.95707738499995, 40.8142980939999]]]}}, {\"id\": \"166\", \"type\": \"Feature\", \"properties\": {\"count\": 3088, \"OBJECTID\": 167, \"Shape_Leng\": 0.0908159737292, \"Shape_Area\": 0.000167529391193, \"zone\": \"Morrisania/Melrose\", \"LocationID\": 167, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89690017899987, 40.8310333359999], [-73.8899110649999, 40.830124191999865], [-73.88609624199987, 40.82765577699989], [-73.89483522899992, 40.82624684099993], [-73.89548522799996, 40.82824854099993], [-73.90093510699997, 40.82792810199992], [-73.9012927759998, 40.820475442999914], [-73.91181615799985, 40.82203502199991], [-73.91677400599986, 40.82438708699987], [-73.91541108199998, 40.82577390899993], [-73.91130974499993, 40.824848557999886], [-73.91169430299985, 40.82775871599993], [-73.908959903, 40.83022623299989], [-73.90553116799988, 40.82910413299989], [-73.90122534999992, 40.83549991099991], [-73.89697220299995, 40.8343452539999], [-73.89690017899987, 40.8310333359999]]]}}, {\"id\": \"167\", \"type\": \"Feature\", \"properties\": {\"count\": 30875, \"OBJECTID\": 168, \"Shape_Leng\": 0.114517434155, \"Shape_Area\": 0.000547058100444, \"zone\": \"Mott Haven/Port Morris\", \"LocationID\": 168, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.92480953899991, 40.81565901999993], [-73.92309299899993, 40.81669772799995], [-73.92048117399989, 40.8145636869999], [-73.91768626999985, 40.8160812429999], [-73.9044655149999, 40.81228195999994], [-73.90229231992774, 40.80456477614167], [-73.91168831241185, 40.79662376988872], [-73.91904978748826, 40.798993935843235], [-73.9229271008289, 40.80237329482137], [-73.92762788658843, 40.80269566548149], [-73.93252708785968, 40.80882328177534], [-73.93242632000944, 40.81411751897961], [-73.93027812699984, 40.813248024999965], [-73.92731653099999, 40.81855963399991], [-73.92518486499998, 40.818012668999884], [-73.92662718199988, 40.816118958999915], [-73.92480953899991, 40.81565901999993]]], [[[-73.89833036270552, 40.80241282093997], [-73.89646668834577, 40.80079047089134], [-73.90021004993142, 40.79926415589601], [-73.90003735815736, 40.800908742050225], [-73.89833036270552, 40.80241282093997]]], [[[-73.89680883223778, 40.795808445159786], [-73.89796839783742, 40.79564483916198], [-73.89919434249981, 40.796502456018175], [-73.8978825324018, 40.79711653214704], [-73.89680883223778, 40.795808445159786]]]]}}, {\"id\": \"168\", \"type\": \"Feature\", \"properties\": {\"count\": 3090, \"OBJECTID\": 169, \"Shape_Leng\": 0.0601046580332, \"Shape_Area\": 0.000146027537733, \"zone\": \"Mount Hope\", \"LocationID\": 169, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89816577499982, 40.85514637599989], [-73.89863543699985, 40.85390697499993], [-73.89560883299981, 40.85457036999986], [-73.901344046, 40.84442875099994], [-73.90483121999989, 40.84480034199987], [-73.9050151499998, 40.842404697999875], [-73.91689747599989, 40.84511773399991], [-73.91416184199993, 40.845179187999925], [-73.90557007299988, 40.855787321999884], [-73.89816577499982, 40.85514637599989]]]}}, {\"id\": \"169\", \"type\": \"Feature\", \"properties\": {\"count\": 4731080, \"OBJECTID\": 170, \"Shape_Leng\": 0.0457690091051, \"Shape_Area\": 7.43150675972e-05, \"zone\": \"Murray Hill\", \"LocationID\": 170, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97203331099985, 40.745804967999945], [-73.97350835000003, 40.7437820339999], [-73.97812378199983, 40.745727172999885], [-73.98085965399994, 40.74196976799989], [-73.98407485299995, 40.74332471799989], [-73.98085054399985, 40.74775036999991], [-73.98407636100002, 40.74910212499993], [-73.98088709899987, 40.753480988999875], [-73.9744494109998, 40.75076644499991], [-73.9758346669999, 40.74886314599996], [-73.97121971999988, 40.74692154099996], [-73.97203331099985, 40.745804967999945]]]}}, {\"id\": \"170\", \"type\": \"Feature\", \"properties\": {\"count\": 980, \"OBJECTID\": 171, \"Shape_Leng\": 0.107353876244, \"Shape_Area\": 0.000520099538161, \"zone\": \"Murray Hill-Queens\", \"LocationID\": 171, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79317944599988, 40.77040317699989], [-73.79343991199988, 40.760764127999934], [-73.79076004599987, 40.76098576899991], [-73.79031517, 40.75782345099988], [-73.81104963199985, 40.761417020999964], [-73.82063593299985, 40.75887226799989], [-73.82657481899986, 40.772345397999885], [-73.82331286899986, 40.77957865999986], [-73.81452997799975, 40.77888817499992], [-73.81485850699985, 40.776485922999875], [-73.79853009399987, 40.775190728999924], [-73.79317944599988, 40.77040317699989]]]}}, {\"id\": \"171\", \"type\": \"Feature\", \"properties\": {\"count\": 95, \"OBJECTID\": 172, \"Shape_Leng\": 0.118476116148, \"Shape_Area\": 0.000658402501406, \"zone\": \"New Dorp/Midland Beach\", \"LocationID\": 172, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08654858694766, 40.56963231913558], [-74.08621632224266, 40.56853269814713], [-74.09054960106333, 40.567083597211294], [-74.09938031432728, 40.5589641933612], [-74.11376161199998, 40.56649718899993], [-74.11636336399992, 40.56391563799986], [-74.12489904399992, 40.568949141999916], [-74.12150213699994, 40.571721558999904], [-74.1215372899999, 40.57515325299986], [-74.10525775499997, 40.58592143599986], [-74.09844000899997, 40.581241705999915], [-74.10039083499996, 40.579688665999925], [-74.08597647499992, 40.57223697099986], [-74.08495715819193, 40.57125984002107], [-74.08654858694766, 40.56963231913558]]]}}, {\"id\": \"0\", \"type\": \"Feature\", \"properties\": {\"count\": 9046, \"OBJECTID\": 1, \"Shape_Leng\": 0.116357453189, \"Shape_Area\": 0.0007823067885, \"zone\": \"Newark Airport\", \"LocationID\": 1, \"borough\": \"EWR\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.18445299999996, 40.6949959999999], [-74.18284199999994, 40.70346499999988], [-74.17565999999994, 40.707379999999866], [-74.16081899999995, 40.70764099999991], [-74.15306999999996, 40.7052829999999], [-74.17283399999997, 40.67656399999988], [-74.17862999999994, 40.671037999999896], [-74.18503799999996, 40.67335999999989], [-74.18991599999998, 40.68154099999988], [-74.18922299999997, 40.688623999999905], [-74.18563199999994, 40.69164799999987], [-74.18445299999996, 40.6949959999999]]]}}, {\"id\": \"172\", \"type\": \"Feature\", \"properties\": {\"count\": 6118, \"OBJECTID\": 173, \"Shape_Leng\": 0.0600639661414, \"Shape_Area\": 0.000178329851357, \"zone\": \"North Corona\", \"LocationID\": 173, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85540366099997, 40.75834030899989], [-73.85312635499996, 40.75510934299989], [-73.85579909099987, 40.75440952799994], [-73.85442575299999, 40.748851725999884], [-73.86790963899986, 40.74493371599989], [-73.87365721799983, 40.756803485999946], [-73.85540366099997, 40.75834030899989]]]}}, {\"id\": \"173\", \"type\": \"Feature\", \"properties\": {\"count\": 1312, \"OBJECTID\": 174, \"Shape_Leng\": 0.0678177559602, \"Shape_Area\": 0.000155928664422, \"zone\": \"Norwood\", \"LocationID\": 174, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87772817699984, 40.88345419499995], [-73.87812594399992, 40.88215382599989], [-73.87467601499999, 40.88156285199993], [-73.87494589799991, 40.87994766899988], [-73.87024100099985, 40.88022935299995], [-73.87619224999979, 40.87047671099987], [-73.8830900319999, 40.86659150899993], [-73.88146889699985, 40.868573645999916], [-73.88705142599984, 40.884349575999906], [-73.88124422499992, 40.88270963299992], [-73.87803562399996, 40.887118321999885], [-73.87772817699984, 40.88345419499995]]]}}, {\"id\": \"174\", \"type\": \"Feature\", \"properties\": {\"count\": 256, \"OBJECTID\": 175, \"Shape_Leng\": 0.134897665102, \"Shape_Area\": 0.000504920726134, \"zone\": \"Oakland Gardens\", \"LocationID\": 175, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7562560639999, 40.7584304689999], [-73.75230259399993, 40.759455198999916], [-73.75004764499985, 40.75840465799996], [-73.74384322099985, 40.74769009599988], [-73.74625984399985, 40.74671147499993], [-73.74477587099986, 40.74328109699994], [-73.75347299999999, 40.74240484399992], [-73.75098230099987, 40.74002146199996], [-73.7439619359999, 40.74078912699986], [-73.74306820799991, 40.7384408179999], [-73.74016951799993, 40.73848189799986], [-73.75048295699996, 40.72922129399996], [-73.75669771299987, 40.7262287109999], [-73.76564374500005, 40.732533126999954], [-73.7636997919999, 40.73286411699993], [-73.76510920699994, 40.73455658499992], [-73.76248609299982, 40.736730981999884], [-73.76772991399994, 40.74577622599993], [-73.75684127399992, 40.74932207799991], [-73.76184343999986, 40.75553726899989], [-73.76239510599996, 40.759496997999854], [-73.75800131799996, 40.76037986699988], [-73.7562560639999, 40.7584304689999]]]}}, {\"id\": \"175\", \"type\": \"Feature\", \"properties\": {\"count\": 38, \"OBJECTID\": 176, \"Shape_Leng\": 0.151995190703, \"Shape_Area\": 0.000657782126267, \"zone\": \"Oakwood\", \"LocationID\": 176, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.12187198599999, 40.57510757499989], [-74.12150213699994, 40.571721558999904], [-74.12489904399992, 40.568949141999916], [-74.11636336299996, 40.56391563799985], [-74.11376161199998, 40.56649718899993], [-74.09992914200652, 40.55907068582615], [-74.10140516910823, 40.555538526164405], [-74.10309839111532, 40.555904641451136], [-74.10484447636338, 40.55302581455626], [-74.10768623464054, 40.553976671477514], [-74.11301046102231, 40.547780065214994], [-74.12730057799996, 40.55729517199993], [-74.13321002499998, 40.552383875999936], [-74.13885092699991, 40.5569511789999], [-74.13170009, 40.56453574899991], [-74.1443874499999, 40.56970609999989], [-74.14418588199999, 40.57203740999988], [-74.13261022999991, 40.573530162999894], [-74.12708814799998, 40.57636910599995], [-74.12187198599999, 40.57510757499989]]]}}, {\"id\": \"176\", \"type\": \"Feature\", \"properties\": {\"count\": 4622, \"OBJECTID\": 177, \"Shape_Leng\": 0.0715805407056, \"Shape_Area\": 0.000198894762248, \"zone\": \"Ocean Hill\", \"LocationID\": 177, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90287079599995, 40.6786314149999], [-73.90004626199993, 40.67689308999995], [-73.92275257199998, 40.667097332999894], [-73.92164666399992, 40.67886992299994], [-73.9163011989999, 40.67857711199991], [-73.91804607, 40.68721324799989], [-73.90287079599995, 40.6786314149999]]]}}, {\"id\": \"177\", \"type\": \"Feature\", \"properties\": {\"count\": 2902, \"OBJECTID\": 178, \"Shape_Leng\": 0.0682528770739, \"Shape_Area\": 0.000175805409143, \"zone\": \"Ocean Parkway South\", \"LocationID\": 178, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97080408199997, 40.62593266699988], [-73.96864510599995, 40.6219416949999], [-73.96471983399981, 40.62237296099993], [-73.96285973999984, 40.61254948599989], [-73.96478112799986, 40.61233092199988], [-73.96429649899991, 40.60977642599992], [-73.97403097799986, 40.608702309999906], [-73.97517176999993, 40.61472418599991], [-73.97290326899996, 40.61415296799988], [-73.97335879399992, 40.616541714999904], [-73.97537335500002, 40.6157547219999], [-73.97785009299986, 40.617287605999906], [-73.97539380499998, 40.62076998699995], [-73.97705352899999, 40.62153252199987], [-73.9772614999999, 40.625852772999906], [-73.97080408199997, 40.62593266699988]]]}}, {\"id\": \"178\", \"type\": \"Feature\", \"properties\": {\"count\": 106845, \"OBJECTID\": 179, \"Shape_Leng\": 0.0713808644377, \"Shape_Area\": 0.000183893478476, \"zone\": \"Old Astoria\", \"LocationID\": 179, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93290347544752, 40.77794449857707], [-73.92782147400004, 40.776697526999946], [-73.91770217299991, 40.7703931119999], [-73.91995596899991, 40.768440064999865], [-73.91849336999996, 40.76773465699995], [-73.91977764799998, 40.7661521839999], [-73.92218946199995, 40.767267356999895], [-73.92425288799978, 40.76472456399995], [-73.93344468499993, 40.768817789999915], [-73.93493511099992, 40.76701302399987], [-73.93618379699991, 40.767691800999934], [-73.93424609600005, 40.771029275999894], [-73.93746638537067, 40.77253149512518], [-73.93765483090819, 40.775085685383594], [-73.93508232492144, 40.77794423372776], [-73.93290347544752, 40.77794449857707]]]}}, {\"id\": \"179\", \"type\": \"Feature\", \"properties\": {\"count\": 1537, \"OBJECTID\": 180, \"Shape_Leng\": 0.0805277671523, \"Shape_Area\": 0.000247682193353, \"zone\": \"Ozone Park\", \"LocationID\": 180, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84119605699982, 40.68335136599989], [-73.83334819299995, 40.66677318699994], [-73.86131863599985, 40.6750212689999], [-73.86334943599995, 40.67935163999989], [-73.85878576899987, 40.68063046299989], [-73.85819836099995, 40.67943084899987], [-73.85392389199987, 40.679686155999875], [-73.85099089299987, 40.68056836699987], [-73.85197864599989, 40.682564210999864], [-73.84786817599985, 40.681486831999905], [-73.84119605699982, 40.68335136599989]]]}}, {\"id\": \"180\", \"type\": \"Feature\", \"properties\": {\"count\": 223234, \"OBJECTID\": 181, \"Shape_Leng\": 0.0895372399547, \"Shape_Area\": 0.000306890292807, \"zone\": \"Park Slope\", \"LocationID\": 181, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97583132299987, 40.66581507999991], [-73.97965353499985, 40.661243232999915], [-73.98223034199995, 40.66214149299991], [-73.985086794, 40.65901328799989], [-73.988716511, 40.66240493799995], [-73.9906354539999, 40.66057908799997], [-73.99692689899985, 40.664374914999925], [-73.99490350199993, 40.66678543499988], [-73.99277080699981, 40.665507280999876], [-73.97804289599988, 40.68485661499987], [-73.97375598499991, 40.68303401999988], [-73.97618051399984, 40.67806432099995], [-73.96949738899991, 40.67590637999996], [-73.96871822199984, 40.67392423399988], [-73.97583132299987, 40.66581507999991]]]}}, {\"id\": \"181\", \"type\": \"Feature\", \"properties\": {\"count\": 1624, \"OBJECTID\": 182, \"Shape_Leng\": 0.0393606369479, \"Shape_Area\": 9.10644156732e-05, \"zone\": \"Parkchester\", \"LocationID\": 182, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85465135700002, 40.83388802699987], [-73.86271114399993, 40.83299150899997], [-73.86448011799995, 40.84108779699994], [-73.85333524999986, 40.84247015499987], [-73.85168459699997, 40.8342010519999], [-73.85465135700002, 40.83388802699987]]]}}, {\"id\": \"182\", \"type\": \"Feature\", \"properties\": {\"count\": 316, \"OBJECTID\": 183, \"Shape_Leng\": 0.0398262960348, \"Shape_Area\": 9.51929451066e-05, \"zone\": \"Pelham Bay\", \"LocationID\": 183, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83676643699985, 40.84338596999994], [-73.83463970299987, 40.853817993999904], [-73.82860197199996, 40.85571784799988], [-73.82721751200002, 40.853250512999935], [-73.82799676999998, 40.84862431399989], [-73.82611300699996, 40.84533869399991], [-73.83676643699985, 40.84338596999994]]]}}, {\"id\": \"183\", \"type\": \"Feature\", \"properties\": {\"count\": 186, \"OBJECTID\": 184, \"Shape_Leng\": 0.260815683043, \"Shape_Area\": 0.00198883363103, \"zone\": \"Pelham Bay Park\", \"LocationID\": 184, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.80728647099995, 40.885931813999875], [-73.79322870891143, 40.88282259495222], [-73.80087270458245, 40.87159011989393], [-73.80252284756597, 40.87050664469679], [-73.80339117523091, 40.87250503763108], [-73.80431849468228, 40.87115974106797], [-73.80302203685277, 40.86915511979648], [-73.80588410012888, 40.869068502184916], [-73.80841596818787, 40.87118969336752], [-73.80464604502912, 40.86664711643685], [-73.80300388108948, 40.86704183656889], [-73.80225816157765, 40.8656082064109], [-73.79780377219292, 40.87225613227397], [-73.794790446816, 40.873200484033966], [-73.79384702788171, 40.87859445979086], [-73.7895587294431, 40.881261284915404], [-73.789523093688, 40.87899812706048], [-73.7843816810755, 40.878420531670514], [-73.78719087560506, 40.872730553729134], [-73.78508139658601, 40.87358996322568], [-73.78526152075709, 40.87174532233236], [-73.78358738486025, 40.87378095468029], [-73.78320175426292, 40.87067036085376], [-73.78560245305444, 40.86834165222368], [-73.78795446145405, 40.86984541906357], [-73.79139032586131, 40.868141307466146], [-73.79190454927502, 40.86082956271207], [-73.79454562318031, 40.85888825916121], [-73.79435005934273, 40.85652702380392], [-73.7987225560008, 40.85491440117259], [-73.79780522990592, 40.85194959859714], [-73.80040387554423, 40.84810822100142], [-73.80417557932397, 40.853340120189074], [-73.80478410547944, 40.86154633053462], [-73.80592716427573, 40.86194119699028], [-73.80612390783546, 40.85981220661901], [-73.80783343870087, 40.860318133305384], [-73.80768048107562, 40.858538063289515], [-73.81411414337536, 40.86373145821164], [-73.81641392102723, 40.86118062770394], [-73.81273146495035, 40.85887754194771], [-73.81230602708418, 40.854162026473176], [-73.81684183599076, 40.85362521315728], [-73.81712409042197, 40.85126957344483], [-73.81507165904237, 40.84913361979451], [-73.82495697699986, 40.846105444999864], [-73.82517999, 40.842700058999874], [-73.82624132899997, 40.84299359399996], [-73.82799676999987, 40.8486243129999], [-73.82723026100001, 40.853350756999895], [-73.82860197199996, 40.85571784799988], [-73.83283502299986, 40.8560358449999], [-73.82834347399992, 40.86089202599988], [-73.82699653199998, 40.85908778599986], [-73.82037552999996, 40.86103300499993], [-73.81578764099999, 40.8657580609999], [-73.821266938, 40.869071605999885], [-73.82329599399986, 40.87309741799988], [-73.81992586499996, 40.88211933299987], [-73.82027432999985, 40.885550699999925], [-73.81442762200005, 40.88677196499989], [-73.80728647099995, 40.885931813999875]]], [[[-73.78650554049733, 40.880940134479225], [-73.78578002611279, 40.88036396266133], [-73.78742039384403, 40.87977089028999], [-73.787127834652, 40.88092346317125], [-73.78650554049733, 40.880940134479225]]], [[[-73.78103351104939, 40.87648400204775], [-73.78120649649391, 40.87628502546013], [-73.78132704118002, 40.87636132654635], [-73.78112767478156, 40.87665268050809], [-73.78103351104939, 40.87648400204775]]], [[[-73.78241811865315, 40.87492327042177], [-73.78265785360905, 40.87479265669398], [-73.78280207510406, 40.87494886620542], [-73.78247604125525, 40.87516813337884], [-73.78241811865315, 40.87492327042177]]], [[[-73.77435244645233, 40.874169955874805], [-73.77453640510954, 40.87471001543833], [-73.77409538292126, 40.87489618448013], [-73.7740318153443, 40.874394774956215], [-73.77435244645233, 40.874169955874805]]], [[[-73.78605394964902, 40.873782646192794], [-73.78618924358646, 40.87359972769974], [-73.78633325996694, 40.87367772488745], [-73.7861946371647, 40.87388531941464], [-73.78605394964902, 40.873782646192794]]], [[[-73.78648510546586, 40.87320925495041], [-73.7867268909535, 40.87307453926568], [-73.78626202153426, 40.87339368663393], [-73.78632869488987, 40.873271144554096], [-73.78648510546586, 40.87320925495041]]], [[[-73.77080975398195, 40.871549946847985], [-73.76990710668544, 40.87047694447026], [-73.77288042665229, 40.871245263430254], [-73.77217786492288, 40.87198111068936], [-73.77080975398195, 40.871549946847985]]], [[[-73.76668965841925, 40.86709778162448], [-73.76709907439783, 40.8667045727482], [-73.76779449382023, 40.86678117376544], [-73.76760072096499, 40.86749638654143], [-73.76668965841925, 40.86709778162448]]], [[[-73.76964988627088, 40.86548557242442], [-73.76979148260493, 40.86512823356445], [-73.77022931630273, 40.865138939396005], [-73.770068972998, 40.86552587618341], [-73.76964988627088, 40.86548557242442]]], [[[-73.78401249138903, 40.863131994074394], [-73.78376770348979, 40.86261060846434], [-73.78469165850701, 40.86254278990822], [-73.7842865050008, 40.86320508878048], [-73.78401249138903, 40.863131994074394]]], [[[-73.78312589594455, 40.862856167300684], [-73.7828819783119, 40.862406433091266], [-73.7831450428813, 40.86206867890555], [-73.78328053505923, 40.86278070671258], [-73.78312589594455, 40.862856167300684]]], [[[-73.77460156350928, 40.86206904745972], [-73.7749432115575, 40.861391499889486], [-73.77529297955127, 40.86123200821588], [-73.77485681624742, 40.86198363433426], [-73.77460156350928, 40.86206904745972]]], [[[-73.77290231992438, 40.86120858327854], [-73.77323150244695, 40.86074716979338], [-73.77351264165688, 40.861241184087085], [-73.77310632921937, 40.861614575032775], [-73.77290231992438, 40.86120858327854]]], [[[-73.8022229535527, 40.84163481314409], [-73.80263811156135, 40.841081153267076], [-73.80694608641589, 40.84146244718637], [-73.80680801372397, 40.84248913998752], [-73.8022229535527, 40.84163481314409]]]]}}, {\"id\": \"184\", \"type\": \"Feature\", \"properties\": {\"count\": 1295, \"OBJECTID\": 185, \"Shape_Leng\": 0.0861619681636, \"Shape_Area\": 0.000228537100218, \"zone\": \"Pelham Parkway\", \"LocationID\": 185, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84004614299985, 40.857806268999866], [-73.83499552599994, 40.85657120099989], [-73.84049767399992, 40.8541841049999], [-73.84494053799982, 40.8482182209999], [-73.84430846999992, 40.85119046399989], [-73.85304526899994, 40.849547331999865], [-73.85457388299992, 40.85150834599995], [-73.86250923799993, 40.84836228099987], [-73.86303938099996, 40.850611306999944], [-73.86860515599989, 40.85206864599991], [-73.86990421600002, 40.85577329299987], [-73.8681883059999, 40.858063833999964], [-73.84004614299985, 40.857806268999866]]]}}, {\"id\": \"185\", \"type\": \"Feature\", \"properties\": {\"count\": 4678544, \"OBJECTID\": 186, \"Shape_Leng\": 0.0246963902234, \"Shape_Area\": 3.70729416953e-05, \"zone\": \"Penn Station/Madison Sq West\", \"LocationID\": 186, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99096832799995, 40.74542088999985], [-73.99709902899994, 40.74720510199991], [-73.99346417699986, 40.75219005499987], [-73.98776905899994, 40.749787028999926], [-73.99096832799995, 40.74542088999985]]]}}, {\"id\": \"186\", \"type\": \"Feature\", \"properties\": {\"count\": 49, \"OBJECTID\": 187, \"Shape_Leng\": 0.126868431324, \"Shape_Area\": 0.000421195755741, \"zone\": \"Port Richmond\", \"LocationID\": 187, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-74.13435269573621, 40.64188679740515], [-74.12890729033458, 40.64119482758617], [-74.1268887269999, 40.637210087999925], [-74.12867047299994, 40.63734826499993], [-74.13047428999985, 40.6308855089999], [-74.13806393799992, 40.627568470999904], [-74.13581008299997, 40.62506433999991], [-74.13826358199992, 40.624243769999936], [-74.13982184799993, 40.61629982499988], [-74.14603562099998, 40.61701277699996], [-74.14696864199988, 40.610151355999896], [-74.15018666499996, 40.60949781099991], [-74.15379116699992, 40.612255611999906], [-74.15231170399994, 40.61716275599988], [-74.14622812499992, 40.6249253619999], [-74.14554071999994, 40.62783252199995], [-74.14752618999992, 40.62938928799988], [-74.14478228999994, 40.637953779999904], [-74.14229766980965, 40.64030580709358], [-74.13435269573621, 40.64188679740515]]], [[[-74.12117246275159, 40.64166914557247], [-74.12128074797168, 40.64143249599354], [-74.12131194814832, 40.64145253116248], [-74.12117246275159, 40.64166914557247]]], [[[-74.1215534360538, 40.64161162832474], [-74.12176052602824, 40.64157431506241], [-74.12175764698422, 40.64142342422718], [-74.12177911197513, 40.641592547117035], [-74.1215534360538, 40.64161162832474]]], [[[-74.12107727405213, 40.64160742058869], [-74.12118039503834, 40.64138344518971], [-74.12121470277087, 40.64139683999368], [-74.12110124737936, 40.64161328229901], [-74.12107727405213, 40.64160742058869]]], [[[-74.12130529083001, 40.64160070313003], [-74.12137332919554, 40.64147647512165], [-74.12132617111645, 40.64160656922885], [-74.12130529083001, 40.64160070313003]]], [[[-74.12185329597821, 40.64158069227959], [-74.12180661091023, 40.64141215903768], [-74.1218414704778, 40.641405894387034], [-74.12199083807539, 40.64155463949942], [-74.12185329597821, 40.64158069227959]]], [[[-74.12270319595869, 40.64137664893253], [-74.12272114424832, 40.64134762377306], [-74.12276203509289, 40.641426635297144], [-74.12271103075855, 40.641433755252706], [-74.12270319595869, 40.64137664893253]]], [[[-74.12613276551227, 40.6412643880282], [-74.1259954246122, 40.640231242814394], [-74.12636808688976, 40.64133238048696], [-74.12564026870598, 40.64141849208226], [-74.12613276551227, 40.6412643880282]]]]}}, {\"id\": \"188\", \"type\": \"Feature\", \"properties\": {\"count\": 60405, \"OBJECTID\": 189, \"Shape_Leng\": 0.0486743306502, \"Shape_Area\": 0.000101427594033, \"zone\": \"Prospect Heights\", \"LocationID\": 189, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96070974499996, 40.673121393999985], [-73.96131877999991, 40.67140667599988], [-73.96892332199981, 40.67293755299988], [-73.96949738899991, 40.67590637999996], [-73.97618051399984, 40.67806432099995], [-73.97375598499991, 40.68303401999988], [-73.96134655899995, 40.6804635239999], [-73.96318238899983, 40.675529503999876], [-73.96007428300001, 40.67487807099994], [-73.96070974499996, 40.673121393999985]]]}}, {\"id\": \"189\", \"type\": \"Feature\", \"properties\": {\"count\": 9627, \"OBJECTID\": 190, \"Shape_Leng\": 0.0743294660829, \"Shape_Area\": 0.000270873666522, \"zone\": \"Prospect Park\", \"LocationID\": 190, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96307724199983, 40.66213019499987], [-73.96190028499997, 40.65487096199992], [-73.96621321999996, 40.653189404999885], [-73.96465438699984, 40.650887727999894], [-73.97139631700004, 40.64825778599986], [-73.97436701299992, 40.65818790399992], [-73.98022902599985, 40.660985325999896], [-73.96865541400003, 40.673023838999924], [-73.96258784899996, 40.671711596999884], [-73.96095595499996, 40.66328505499992], [-73.96307724199983, 40.66213019499987]]]}}, {\"id\": \"187\", \"type\": \"Feature\", \"properties\": {\"count\": 24954, \"OBJECTID\": 188, \"Shape_Leng\": 0.0977983968367, \"Shape_Area\": 0.000313003032771, \"zone\": \"Prospect-Lefferts Gardens\", \"LocationID\": 188, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95729172699996, 40.6643026319999], [-73.94269628999989, 40.664055128999934], [-73.94282892199992, 40.6628474859999], [-73.93068465499985, 40.66362047699992], [-73.92646424299983, 40.659814361999956], [-73.92834017599988, 40.660060035999884], [-73.92799465199988, 40.65680210099988], [-73.9500666489999, 40.65542299699989], [-73.94957113100006, 40.65080789999991], [-73.9587078989999, 40.65038727299989], [-73.95986103199981, 40.65563478899989], [-73.96190028499997, 40.65487096199992], [-73.96307724199983, 40.66213019499987], [-73.95729172699996, 40.6643026319999]]]}}, {\"id\": \"190\", \"type\": \"Feature\", \"properties\": {\"count\": 1208, \"OBJECTID\": 191, \"Shape_Leng\": 0.130932637664, \"Shape_Area\": 0.000688460603185, \"zone\": \"Queens Village\", \"LocationID\": 191, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.73016587199996, 40.72395859599988], [-73.72816233910697, 40.7230841398913], [-73.73032628984046, 40.722157296892334], [-73.7269714679152, 40.710714504016735], [-73.72677759516957, 40.7030592911994], [-73.74590871000004, 40.70228727399988], [-73.74500119100004, 40.70026221299988], [-73.74773012199991, 40.69955313699991], [-73.74618157999997, 40.696121925999904], [-73.74863325999986, 40.6954447609999], [-73.75205066499986, 40.70522802799995], [-73.75047058199985, 40.70563993099994], [-73.75725671499985, 40.71813860199992], [-73.7523343839998, 40.72026246099993], [-73.75669771299987, 40.7262287109999], [-73.74672140400006, 40.73247715099991], [-73.74555667199989, 40.73063689799994], [-73.7314721509999, 40.727411794999924], [-73.73016587199996, 40.72395859599988]]]}}, {\"id\": \"191\", \"type\": \"Feature\", \"properties\": {\"count\": 1886, \"OBJECTID\": 192, \"Shape_Leng\": 0.11196461971, \"Shape_Area\": 0.000373808019026, \"zone\": \"Queensboro Hill\", \"LocationID\": 192, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82239598999983, 40.74986422299986], [-73.81416171999987, 40.74613857999992], [-73.80750599299991, 40.750136586999936], [-73.80287666499983, 40.749765149999895], [-73.8033521419999, 40.74643196499987], [-73.7946756879998, 40.74721192799994], [-73.79453507400001, 40.74450763799989], [-73.79723084999992, 40.74169550699993], [-73.79640218700004, 40.73828235799987], [-73.8243501109999, 40.73975360599992], [-73.82946751999994, 40.737024481999896], [-73.83143025500006, 40.73913573299994], [-73.83113122799989, 40.74233799999994], [-73.83586131899989, 40.743227984999926], [-73.83392218299996, 40.74753207499989], [-73.82584440999999, 40.74934839099993], [-73.82426493799994, 40.75183288799992], [-73.82239598999983, 40.74986422299986]]]}}, {\"id\": \"192\", \"type\": \"Feature\", \"properties\": {\"count\": 55067, \"OBJECTID\": 193, \"Shape_Leng\": 0.0655299584801, \"Shape_Area\": 0.000145591657944, \"zone\": \"Queensbridge/Ravenswood\", \"LocationID\": 193, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93522718099996, 40.76952572899989], [-73.93858976899998, 40.766817358999916], [-73.93398092199995, 40.76282598199987], [-73.944052652, 40.75231325999992], [-73.95080763809156, 40.75526368005708], [-73.9413117494144, 40.76691800476961], [-73.93424609599985, 40.771029274999925], [-73.93522718099996, 40.76952572899989]]]}}, {\"id\": \"193\", \"type\": \"Feature\", \"properties\": {\"count\": 9652, \"OBJECTID\": 194, \"Shape_Leng\": 0.0817938353532, \"Shape_Area\": 0.000273481702719, \"zone\": \"Randalls Island\", \"LocationID\": 194, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91639852914044, 40.79768484203896], [-73.91378385370595, 40.793836285482065], [-73.92412113996787, 40.78252307401082], [-73.92810118993664, 40.78092032002105], [-73.93598226003293, 40.7839423680532], [-73.93084178719639, 40.79099143000079], [-73.9261041024332, 40.79073160730996], [-73.92560814781015, 40.79189695828146], [-73.9278688922052, 40.79095483831721], [-73.9283194359912, 40.79309126608271], [-73.9254929354651, 40.80195616882528], [-73.92263378594883, 40.80186873814113], [-73.91639852914044, 40.79768484203896]]]}}, {\"id\": \"194\", \"type\": \"Feature\", \"properties\": {\"count\": 14397, \"OBJECTID\": 195, \"Shape_Leng\": 0.131308244067, \"Shape_Area\": 0.000801651953985, \"zone\": \"Red Hook\", \"LocationID\": 195, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.0071710969999, 40.68635700999991], [-74.00485241599992, 40.685668130999886], [-74.00596041099988, 40.683362870999936], [-73.9986188799999, 40.67183017199989], [-74.00289483876944, 40.66734846594933], [-74.0058636618068, 40.66799376258568], [-74.00683236078251, 40.666049645526776], [-74.00696707834247, 40.666052182356665], [-74.00543926529353, 40.6709214682384], [-74.00754821618384, 40.66705174573358], [-74.00708724221813, 40.668591358535764], [-74.00985411855632, 40.668523628335336], [-74.01154017543327, 40.6650859197566], [-74.01574872882654, 40.6645683362916], [-74.01690267361069, 40.66484660233888], [-74.01933946871777, 40.671624948422625], [-74.01751037086694, 40.671034530984144], [-74.01648665554487, 40.664930508953816], [-74.01227967872792, 40.66573241415882], [-74.01033845947926, 40.669078195448], [-74.01162527521356, 40.670581028275876], [-74.01424926931875, 40.66972860473708], [-74.01175350509699, 40.670657757578326], [-74.0151998057007, 40.6707540276644], [-74.01423289452782, 40.67213000051297], [-74.01568625497515, 40.671038834928844], [-74.0129758779219, 40.67332791872093], [-74.01632099922307, 40.6729082695637], [-74.01496392184576, 40.67467763029424], [-74.0188009477692, 40.6722507648487], [-74.01727138470127, 40.67360864580548], [-74.0188577695892, 40.6751022924089], [-74.01791198226178, 40.676510462801936], [-74.01995197864673, 40.67710301044993], [-74.01825267004557, 40.67855288672587], [-74.01928128101008, 40.67964814029763], [-74.01282000316418, 40.68362241624865], [-74.01410549647976, 40.68171251937541], [-74.013018354221, 40.68041237401427], [-74.00967108558879, 40.683268727715], [-74.01193259955522, 40.683887749077535], [-74.0071710969999, 40.68635700999991]]]}}, {\"id\": \"195\", \"type\": \"Feature\", \"properties\": {\"count\": 16548, \"OBJECTID\": 196, \"Shape_Leng\": 0.0832421106332, \"Shape_Area\": 0.0001974713336, \"zone\": \"Rego Park\", \"LocationID\": 196, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85762923599995, 40.728606921999884], [-73.8594198799999, 40.726464837999885], [-73.85727108100001, 40.72568129799988], [-73.85805416899993, 40.72444540599986], [-73.86203833799982, 40.72595678499994], [-73.8586809759999, 40.72261243999988], [-73.85729287299985, 40.711467676999874], [-73.85975932900001, 40.711845058999856], [-73.86003789899996, 40.71493223599987], [-73.86929270499994, 40.72443592899991], [-73.87146074199987, 40.729326373999854], [-73.87509154199992, 40.73067136299992], [-73.85886555799988, 40.73593913899987], [-73.85625665099991, 40.73095912299993], [-73.85851097899999, 40.73028680199993], [-73.85762923599995, 40.728606921999884]]]}}, {\"id\": \"196\", \"type\": \"Feature\", \"properties\": {\"count\": 5603, \"OBJECTID\": 197, \"Shape_Leng\": 0.108568532229, \"Shape_Area\": 0.000504689160432, \"zone\": \"Richmond Hill\", \"LocationID\": 197, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83696237799982, 40.70421245199992], [-73.83152109599999, 40.70138962899994], [-73.82245890399994, 40.70365553099989], [-73.81637824499992, 40.70245196399987], [-73.81063298299996, 40.69194738299988], [-73.82501128899997, 40.68795231799991], [-73.82599347399987, 40.689953661999894], [-73.82851136799994, 40.689239055999934], [-73.82753021399995, 40.687239482999914], [-73.83807769099997, 40.68423890999988], [-73.83697486299991, 40.6819880179999], [-73.84012856099982, 40.6811466509999], [-73.84947763499991, 40.69819580699987], [-73.83696237799982, 40.70421245199992]]]}}, {\"id\": \"197\", \"type\": \"Feature\", \"properties\": {\"count\": 7437, \"OBJECTID\": 198, \"Shape_Leng\": 0.134094242763, \"Shape_Area\": 0.000499809882564, \"zone\": \"Ridgewood\", \"LocationID\": 198, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90768387299997, 40.71737969499987], [-73.90639010099999, 40.71736369199989], [-73.90537582199983, 40.71287658099986], [-73.88006267699978, 40.71265257399993], [-73.87761134199985, 40.70716770499993], [-73.88911917799989, 40.705598876999915], [-73.90123290699994, 40.69144227899997], [-73.90579597099993, 40.69412715499987], [-73.9042601839999, 40.69570037099989], [-73.91180820099989, 40.69993800299986], [-73.91067882699998, 40.701045968999914], [-73.91290404099983, 40.70236189199989], [-73.91180710099992, 40.703434952999864], [-73.92189184699987, 40.709396095999836], [-73.92074519699985, 40.71052968599989], [-73.92404011299993, 40.714008312999916], [-73.91077807399988, 40.71318759999986], [-73.91283037699992, 40.71536090799986], [-73.90965980099996, 40.71779315499991], [-73.90768387299997, 40.71737969499987]]]}}, {\"id\": \"198\", \"type\": \"Feature\", \"properties\": {\"count\": 88, \"OBJECTID\": 199, \"Shape_Leng\": 0.0778085005169, \"Shape_Area\": 0.000288747549303, \"zone\": \"Rikers Island\", \"LocationID\": 199, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88885148496314, 40.79870632895875], [-73.88665108524579, 40.79803819669988], [-73.88375741590156, 40.79570856541983], [-73.87288457445655, 40.79145210820714], [-73.87080888583708, 40.78789669184152], [-73.87308761694455, 40.78585495239161], [-73.8783068007881, 40.785356620754236], [-73.88905212421602, 40.78737256012899], [-73.89282283610926, 40.79281708195147], [-73.89179289686568, 40.79677524575864], [-73.88885148496314, 40.79870632895875]]]}}, {\"id\": \"199\", \"type\": \"Feature\", \"properties\": {\"count\": 1357, \"OBJECTID\": 200, \"Shape_Leng\": 0.112661735435, \"Shape_Area\": 0.000744643168558, \"zone\": \"Riverdale/North Riverdale/Fieldston\", \"LocationID\": 200, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.896411335, 40.90450452099991], [-73.89643170799987, 40.89365527899989], [-73.89787887499983, 40.89027610099993], [-73.9028594909999, 40.88916474399989], [-73.90413642399992, 40.887080708999896], [-73.90975930299997, 40.889250027999886], [-73.91489276999991, 40.8846247709999], [-73.92058641888602, 40.88718267830985], [-73.9103325682445, 40.91553277700258], [-73.89663333881637, 40.911417374923055], [-73.896411335, 40.90450452099991]]]}}, {\"id\": \"200\", \"type\": \"Feature\", \"properties\": {\"count\": 152, \"OBJECTID\": 201, \"Shape_Leng\": 0.130404117686, \"Shape_Area\": 0.000619350747819, \"zone\": \"Rockaway Park\", \"LocationID\": 201, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.81849901800001, 40.5873623529999], [-73.81544517899997, 40.58337349199993], [-73.81730795916116, 40.58233475897], [-73.86255286029154, 40.56681001230082], [-73.86662812000002, 40.573057648999914], [-73.8504438514919, 40.582130243581396], [-73.83929597982016, 40.581863545478186], [-73.82480116623435, 40.587154615369926], [-73.81849901800001, 40.5873623529999]]]}}, {\"id\": \"201\", \"type\": \"Feature\", \"properties\": {\"count\": 10604, \"OBJECTID\": 202, \"Shape_Leng\": 0.0699215712618, \"Shape_Area\": 0.000106175396322, \"zone\": \"Roosevelt Island\", \"LocationID\": 202, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94180032729437, 40.76904692662474], [-73.95254079336881, 40.757095314537246], [-73.96158304969282, 40.74941430105318], [-73.94472478498723, 40.76978627176733], [-73.94007665725826, 40.77292618617896], [-73.94180032729437, 40.76904692662474]]]}}, {\"id\": \"202\", \"type\": \"Feature\", \"properties\": {\"count\": 784, \"OBJECTID\": 203, \"Shape_Leng\": 0.189937815868, \"Shape_Area\": 0.000615129861659, \"zone\": \"Rosedale\", \"LocationID\": 203, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7275350030123, 40.673930284146124], [-73.72833071887973, 40.66304394386548], [-73.72522608103498, 40.652001000073724], [-73.74143706021165, 40.646889178160706], [-73.74227387212949, 40.64012338122567], [-73.739471216134, 40.63570643191466], [-73.74246642615259, 40.635119228252094], [-73.74096487686366, 40.63730477503007], [-73.74397128496219, 40.63788001261484], [-73.7437342150463, 40.6390221176639], [-73.7464954765674, 40.63651030383521], [-73.74516219973704, 40.63811218797305], [-73.74704226024105, 40.636932365240085], [-73.74631467883957, 40.641394313090785], [-73.7487767478731, 40.64511819098248], [-73.75522386899995, 40.64744446899989], [-73.7541108669999, 40.648697664999915], [-73.76406290099993, 40.65435626599989], [-73.76165732899996, 40.658238085999855], [-73.74867194699979, 40.65267808799992], [-73.74332546799988, 40.652537803999905], [-73.74328714099991, 40.65613473199991], [-73.74853177199994, 40.65757569699993], [-73.74281884699987, 40.66530554899986], [-73.73891022900001, 40.66595449199986], [-73.74117817299991, 40.666402953999885], [-73.72718272499995, 40.68359732999987], [-73.72586264191175, 40.683241080973865], [-73.72563005109944, 40.67958795089065], [-73.7275350030123, 40.673930284146124]]]}}, {\"id\": \"203\", \"type\": \"Feature\", \"properties\": {\"count\": 23, \"OBJECTID\": 204, \"Shape_Leng\": 0.101912418311, \"Shape_Area\": 0.000641384265972, \"zone\": \"Rossville/Woodrow\", \"LocationID\": 204, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.2032648999999, 40.55562847699989], [-74.19174368799999, 40.53191669699986], [-74.20152661599995, 40.526265205999934], [-74.2156723039999, 40.52395762799989], [-74.21800676499996, 40.53229858999995], [-74.21776941599994, 40.54112662999992], [-74.22131562699998, 40.55025307399992], [-74.21784397799998, 40.55443271799988], [-74.21050789400002, 40.557064181999934], [-74.205854485, 40.55750245699993], [-74.2032648999999, 40.55562847699989]]]}}, {\"id\": \"204\", \"type\": \"Feature\", \"properties\": {\"count\": 610, \"OBJECTID\": 205, \"Shape_Leng\": 0.141953388327, \"Shape_Area\": 0.000766216513505, \"zone\": \"Saint Albans\", \"LocationID\": 205, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.766369252, 40.70256339699992], [-73.75205066499986, 40.70522802799995], [-73.74863325999986, 40.6954447609999], [-73.7435928609999, 40.69621532499986], [-73.75576169199995, 40.67588590899988], [-73.76380484799996, 40.67376993299995], [-73.77469730199986, 40.680486045999906], [-73.77220275299989, 40.681610338999896], [-73.7732870399999, 40.683454731999866], [-73.77699767599997, 40.68628268899992], [-73.77966850599998, 40.685377122999945], [-73.7832662499999, 40.68999429299992], [-73.78078155499992, 40.69102994299994], [-73.77977295099997, 40.689281169999916], [-73.77789337399993, 40.68991768999992], [-73.7802372379999, 40.692972334999894], [-73.77230768699997, 40.70057150699989], [-73.77559218799982, 40.70532930899986], [-73.7749009329999, 40.707066661999875], [-73.76844546499984, 40.709637026999886], [-73.766369252, 40.70256339699992]]]}}, {\"id\": \"205\", \"type\": \"Feature\", \"properties\": {\"count\": 184, \"OBJECTID\": 206, \"Shape_Leng\": 0.212756793386, \"Shape_Area\": 0.000944392507762, \"zone\": \"Saint George/New Brighton\", \"LocationID\": 206, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08221272914942, 40.64828016229008], [-74.07165829759784, 40.64503374643502], [-74.07093784471476, 40.64334596384062], [-74.07210952444638, 40.64246321679278], [-74.07032923715386, 40.64252612813239], [-74.07314400586021, 40.641852974480116], [-74.07019399559213, 40.6420075028053], [-74.07289005609624, 40.64092267287602], [-74.06998354653952, 40.64106894311676], [-74.07285041194942, 40.64080368844508], [-74.07258133886755, 40.63794187429597], [-74.07865269699992, 40.63830402499987], [-74.08373397299994, 40.63573714399991], [-74.08870582299987, 40.63684520599986], [-74.09242513499994, 40.63558181299992], [-74.09409312599998, 40.639266913999975], [-74.09767290799995, 40.63896067899989], [-74.09787969199995, 40.64035805499988], [-74.10974833299994, 40.63786583299993], [-74.11765722499996, 40.638062466999905], [-74.11614321199995, 40.62870226399988], [-74.1132624959999, 40.62305442499995], [-74.12125677899994, 40.62782274999989], [-74.1313098529999, 40.62632834299998], [-74.12867047299994, 40.63734826499993], [-74.1268887269999, 40.637210087999925], [-74.12705362202625, 40.64024173781949], [-74.11861635236262, 40.641530365028935], [-74.11883624616948, 40.64254186406039], [-74.1171709328468, 40.64162427104175], [-74.1172247008255, 40.64302735996259], [-74.11535082182925, 40.64247054015812], [-74.1097393256449, 40.64546372818479], [-74.09880066885808, 40.64504703431087], [-74.08570754378212, 40.648880553370454], [-74.08221272914942, 40.64828016229008]]]}}, {\"id\": \"206\", \"type\": \"Feature\", \"properties\": {\"count\": 12964, \"OBJECTID\": 207, \"Shape_Leng\": 0.0283756831492, \"Shape_Area\": 4.47230848441e-05, \"zone\": \"Saint Michaels Cemetery/Woodside\", \"LocationID\": 207, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90137640299986, 40.75964203799989], [-73.90356915599997, 40.765781788999924], [-73.90217951699996, 40.76731584199997], [-73.89349605899986, 40.765950929999924], [-73.89866137899992, 40.75992101699997], [-73.90137640299986, 40.75964203799989]]]}}, {\"id\": \"207\", \"type\": \"Feature\", \"properties\": {\"count\": 1037, \"OBJECTID\": 208, \"Shape_Leng\": 0.214721009581, \"Shape_Area\": 0.00202032598969, \"zone\": \"Schuylerville/Edgewater Park\", \"LocationID\": 208, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.825986869, 40.84067575199988], [-73.82601252600003, 40.83402077799985], [-73.81552500922504, 40.83597630692601], [-73.81476387691956, 40.830816799993485], [-73.81275469994516, 40.82824823625135], [-73.80992515914596, 40.82817971199259], [-73.81384974851248, 40.8268035287504], [-73.81379523517661, 40.82529808600147], [-73.81211350556435, 40.82555263465937], [-73.81388934712506, 40.82408736742192], [-73.80766604765235, 40.825896992059], [-73.80451800414906, 40.81892768751854], [-73.79759032777322, 40.816507532070055], [-73.80140380968845, 40.8123633086586], [-73.80361390158114, 40.812521106621354], [-73.80453081609834, 40.81425461992442], [-73.80351667337368, 40.81395196364859], [-73.80704600153508, 40.816533693875236], [-73.80505213646254, 40.81289496243412], [-73.80224245334347, 40.81019739366033], [-73.79086475738406, 40.80734114044052], [-73.79028410874172, 40.80481710368888], [-73.7933898228672, 40.804204330036214], [-73.80185524738366, 40.808967826140574], [-73.80392673417204, 40.80852797081171], [-73.8099202908178, 40.81292915226176], [-73.81620056896291, 40.81384747647932], [-73.83017915330238, 40.81083454948945], [-73.8321661704888, 40.80850227767013], [-73.831615744777, 40.80493825599657], [-73.83743612112924, 40.80620264259155], [-73.84043199520461, 40.81254112811847], [-73.83974355921865, 40.816406071751366], [-73.83696070299992, 40.81634054099991], [-73.83614239499985, 40.81919890299987], [-73.83661819699992, 40.82711414899993], [-73.83709061399992, 40.82896714699996], [-73.84233077477266, 40.8290739624537], [-73.83885232384739, 40.83369066784824], [-73.83956700299987, 40.84054722799986], [-73.83771374899992, 40.840406309999885], [-73.83688925199996, 40.84337478799992], [-73.82611300699996, 40.84533869399991], [-73.825986869, 40.84067575199988]]], [[[-73.83960098545435, 40.83567071869858], [-73.83961672561374, 40.835667752627316], [-73.83962599488439, 40.83569971218477], [-73.83960850400152, 40.83570805084221], [-73.83960098545435, 40.83567071869858]]], [[[-73.83979488581072, 40.835619207246324], [-73.8396148723857, 40.83561056836579], [-73.83948374192397, 40.83457420079436], [-73.83959266810578, 40.83456047582325], [-73.83979488581072, 40.835619207246324]]]]}}, {\"id\": \"208\", \"type\": \"Feature\", \"properties\": {\"count\": 264208, \"OBJECTID\": 209, \"Shape_Leng\": 0.0306953113473, \"Shape_Area\": 3.88703351209e-05, \"zone\": \"Seaport\", \"LocationID\": 209, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00499805099996, 40.71201189699987], [-73.99919451154408, 40.707947376175724], [-74.00143661179398, 40.70487217770521], [-74.00906264699988, 40.71088670399993], [-74.00499805099996, 40.71201189699987]]]}}, {\"id\": \"209\", \"type\": \"Feature\", \"properties\": {\"count\": 1132, \"OBJECTID\": 210, \"Shape_Leng\": 0.114851983372, \"Shape_Area\": 0.000453155131253, \"zone\": \"Sheepshead Bay\", \"LocationID\": 210, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92759936700004, 40.5950607789999], [-73.91805571865743, 40.58643685311621], [-73.92520361514241, 40.58580627115536], [-73.92802185276429, 40.588068756350886], [-73.9239529485009, 40.59113985052502], [-73.92863061415638, 40.58857263642073], [-73.92844090493661, 40.589030428141996], [-73.9291344038264, 40.588879241681944], [-73.92875703499415, 40.589152966925], [-73.92976615773313, 40.589292069447644], [-73.93003427450557, 40.58942599650075], [-73.93057705718242, 40.589890641203176], [-73.93010325988745, 40.58960610507142], [-73.93001650952081, 40.5940945360621], [-73.93260420556844, 40.59492954452495], [-73.93121294858955, 40.58953130811921], [-73.92972846380192, 40.58861391582969], [-73.93125191267133, 40.58684982521422], [-73.92857393936458, 40.58706150237135], [-73.92552860874584, 40.58497837657265], [-73.9344471859999, 40.586652074999925], [-73.96025825899996, 40.58502930799994], [-73.9603495359999, 40.58730628599994], [-73.95619313699986, 40.58776226899994], [-73.956927197, 40.596612075999865], [-73.94248473299992, 40.598200364999855], [-73.94288954099984, 40.600318649999934], [-73.93447158899997, 40.60123151199989], [-73.9335511719998, 40.60268728099991], [-73.92813139699999, 40.59777792799987], [-73.92956858299992, 40.5968202649999], [-73.92759936700004, 40.5950607789999]]]}}, {\"id\": \"210\", \"type\": \"Feature\", \"properties\": {\"count\": 1332225, \"OBJECTID\": 211, \"Shape_Leng\": 0.0252345082132, \"Shape_Area\": 3.97291966087e-05, \"zone\": \"SoHo\", \"LocationID\": 211, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.001523638, 40.719820797999944], [-74.00539186699989, 40.722037365999896], [-74.00214891499996, 40.728035366999904], [-73.9967717579999, 40.7254319439999], [-74.001523638, 40.719820797999944]]]}}, {\"id\": \"211\", \"type\": \"Feature\", \"properties\": {\"count\": 1677, \"OBJECTID\": 212, \"Shape_Leng\": 0.0671290653429, \"Shape_Area\": 0.000161276191858, \"zone\": \"Soundview/Bruckner\", \"LocationID\": 212, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88421976599987, 40.82299645799987], [-73.88378414099992, 40.828274291999946], [-73.87729930499995, 40.8287427739999], [-73.86085019699983, 40.833613537999874], [-73.857391213, 40.83229737199986], [-73.85615112900003, 40.82677004599987], [-73.88421976599987, 40.82299645799987]]]}}, {\"id\": \"212\", \"type\": \"Feature\", \"properties\": {\"count\": 1725, \"OBJECTID\": 213, \"Shape_Leng\": 0.159958783822, \"Shape_Area\": 0.000904077906339, \"zone\": \"Soundview/Castle Hill\", \"LocationID\": 213, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.83667856399991, 40.81758876299995], [-73.83696070299992, 40.81634054099991], [-73.84003308726955, 40.81682214096213], [-73.83888231646733, 40.822140489856025], [-73.84228428387955, 40.82864660449135], [-73.83709061399992, 40.82896714699996], [-73.83667856399991, 40.81758876299995]]], [[[-73.84231712480313, 40.82718468468027], [-73.83999860407422, 40.81993494681306], [-73.84198684834051, 40.818774373908084], [-73.84617302872044, 40.81067666443039], [-73.8511447226069, 40.814396548876], [-73.85538030243693, 40.81428888208278], [-73.84903294180673, 40.81211891007852], [-73.84996696802818, 40.808578018843576], [-73.847488226419, 40.805448909149185], [-73.85019839036403, 40.80451002497395], [-73.856614854502, 40.80479692395033], [-73.85866549456955, 40.806833088596065], [-73.85895141221772, 40.810271417586556], [-73.8601269523559, 40.80960549989125], [-73.85935625734362, 40.80898803428646], [-73.85938875426787, 40.80816614120708], [-73.86065222514291, 40.80963668169183], [-73.86765646284456, 40.81058376193711], [-73.87077804885932, 40.814486895487775], [-73.8780628329999, 40.81618263199993], [-73.88244779599998, 40.81926832299992], [-73.8843985489999, 40.822965575999866], [-73.85635677499994, 40.82675208499993], [-73.85513639699997, 40.822436188999866], [-73.84289858800003, 40.82408305899986], [-73.84390624999999, 40.828407828999936], [-73.84231712480313, 40.82718468468027]]]]}}, {\"id\": \"213\", \"type\": \"Feature\", \"properties\": {\"count\": 144, \"OBJECTID\": 214, \"Shape_Leng\": 0.135300314246, \"Shape_Area\": 0.000811976523012, \"zone\": \"South Beach/Dongan Hills\", \"LocationID\": 214, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08491019899996, 40.59570472799985], [-74.08173342999991, 40.59215628599987], [-74.07043976099992, 40.59673474699988], [-74.06436003721248, 40.58826789117477], [-74.07499413073505, 40.57923655741631], [-74.07336118472098, 40.578275488289144], [-74.07508288192615, 40.579160158755506], [-74.08495715819193, 40.57125984002107], [-74.10039083499996, 40.579688665999925], [-74.09844000899997, 40.581241705999915], [-74.10525775499997, 40.58592143599986], [-74.10088251699995, 40.59017921599992], [-74.09979588999995, 40.59387136599988], [-74.09375453799998, 40.59674488999988], [-74.08983655899989, 40.60077409499996], [-74.08218729299988, 40.601941007999876], [-74.08130343799999, 40.59938765599991], [-74.08532650099991, 40.59700832199995], [-74.08491019899996, 40.59570472799985]]]}}, {\"id\": \"214\", \"type\": \"Feature\", \"properties\": {\"count\": 9516, \"OBJECTID\": 215, \"Shape_Leng\": 0.103556453625, \"Shape_Area\": 0.000395647022725, \"zone\": \"South Jamaica\", \"LocationID\": 215, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7769103769999, 40.70252488499994], [-73.77395659499986, 40.699805781999906], [-73.7802372379999, 40.692972334999894], [-73.77789337399993, 40.68991768999992], [-73.77977295099997, 40.689281169999916], [-73.78078155499992, 40.69102994299994], [-73.793908417, 40.68630469099985], [-73.79561486799987, 40.688656890999916], [-73.79995042799996, 40.68682771899991], [-73.79898209099994, 40.68485873799988], [-73.80576840199991, 40.682932390999916], [-73.81063298299996, 40.69194738299988], [-73.80890216900004, 40.69256434899993], [-73.80642479399984, 40.690873857999904], [-73.79260506799987, 40.70043209999988], [-73.79239170299988, 40.7021665299999], [-73.78256582599995, 40.70542486399989], [-73.78034491200003, 40.70541241699986], [-73.7769103769999, 40.70252488499994]]]}}, {\"id\": \"215\", \"type\": \"Feature\", \"properties\": {\"count\": 8084, \"OBJECTID\": 216, \"Shape_Leng\": 0.116797554681, \"Shape_Area\": 0.000816076647781, \"zone\": \"South Ozone Park\", \"LocationID\": 216, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.80857655199986, 40.688330889999925], [-73.80142836799988, 40.673666833999874], [-73.80203207699995, 40.66524802499994], [-73.81496922499986, 40.66247266799991], [-73.832705339, 40.665405772999854], [-73.84012856099982, 40.6811466509999], [-73.83697486299991, 40.6819880179999], [-73.83807769099997, 40.68423890999988], [-73.82753021399995, 40.687239482999914], [-73.82851136799994, 40.689239055999934], [-73.82599347399987, 40.689953661999894], [-73.82501128899997, 40.68795231799991], [-73.81063298299996, 40.69194738299988], [-73.80857655199986, 40.688330889999925]]]}}, {\"id\": \"216\", \"type\": \"Feature\", \"properties\": {\"count\": 18314, \"OBJECTID\": 217, \"Shape_Leng\": 0.0553905251285, \"Shape_Area\": 0.000114733270758, \"zone\": \"South Williamsburg\", \"LocationID\": 217, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95554165799997, 40.70759344799989], [-73.94705205299991, 40.70366394899985], [-73.95128819399989, 40.700922364999855], [-73.95381196900003, 40.703180979999914], [-73.95745736399992, 40.70082260299993], [-73.95701993199991, 40.69897391399995], [-73.96283963699993, 40.698038667999946], [-73.96217978199998, 40.70022070999989], [-73.96838933699995, 40.70682918699989], [-73.95554165799997, 40.70759344799989]]]}}, {\"id\": \"217\", \"type\": \"Feature\", \"properties\": {\"count\": 860, \"OBJECTID\": 218, \"Shape_Leng\": 0.0837008281049, \"Shape_Area\": 0.000281293736407, \"zone\": \"Springfield Gardens North\", \"LocationID\": 218, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76561573099998, 40.6747790199999], [-73.76380484799996, 40.67376993299995], [-73.76031463999993, 40.67510997099984], [-73.75857198899986, 40.67263733099987], [-73.75636239300002, 40.67242923699999], [-73.75763129599993, 40.66645115699992], [-73.77342462799993, 40.66802761899988], [-73.78940999399987, 40.66684115899995], [-73.78808647399985, 40.6730096269999], [-73.78573730899993, 40.67238503499988], [-73.77722923600001, 40.679652720999904], [-73.77469730199986, 40.680486045999906], [-73.76561573099998, 40.6747790199999]]]}}, {\"id\": \"218\", \"type\": \"Feature\", \"properties\": {\"count\": 7350, \"OBJECTID\": 219, \"Shape_Leng\": 0.155126669675, \"Shape_Area\": 0.000428176336771, \"zone\": \"Springfield Gardens South\", \"LocationID\": 219, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.761822313, 40.666877759999856], [-73.73856469500002, 40.66639689799991], [-73.74450922599979, 40.663821277999865], [-73.74853177199994, 40.65757569699993], [-73.74328714099991, 40.65613473199991], [-73.74332546899993, 40.65253780399992], [-73.74867194699979, 40.65267808799992], [-73.76165732899996, 40.658238085999855], [-73.76406290099993, 40.65435626599989], [-73.78321833499997, 40.663166283999864], [-73.80203207699995, 40.66524802499994], [-73.80155565300004, 40.6667533639999], [-73.77342462799993, 40.66802761899988], [-73.761822313, 40.666877759999856]]]}}, {\"id\": \"219\", \"type\": \"Feature\", \"properties\": {\"count\": 3984, \"OBJECTID\": 220, \"Shape_Leng\": 0.106015943267, \"Shape_Area\": 0.000313035744467, \"zone\": \"Spuyten Duyvil/Kingsbridge\", \"LocationID\": 220, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89827216899992, 40.8894579429999], [-73.89972588599989, 40.88651488699985], [-73.89661546599986, 40.885955255999946], [-73.90369454799992, 40.878501617999895], [-73.90608130599988, 40.873239628999904], [-73.90746489699994, 40.87354735399985], [-73.9068187309999, 40.87661958799993], [-73.91033193599998, 40.879038046999895], [-73.91775669364831, 40.8756636279301], [-73.92490327486523, 40.87888836792653], [-73.92058641888602, 40.88718267830985], [-73.91489276999991, 40.8846247709999], [-73.90900536, 40.88950014099986], [-73.90413642399992, 40.887080708999896], [-73.9028594909999, 40.88916474399989], [-73.89827216899992, 40.8894579429999]]]}}, {\"id\": \"220\", \"type\": \"Feature\", \"properties\": {\"count\": 160, \"OBJECTID\": 221, \"Shape_Leng\": 0.166218220846, \"Shape_Area\": 0.000890111567051, \"zone\": \"Stapleton\", \"LocationID\": 221, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.07530550099992, 40.637729958999955], [-74.07053959705199, 40.63747140462426], [-74.07341562632824, 40.63689395096554], [-74.07209483807503, 40.63664727762461], [-74.07335822045584, 40.63653609416517], [-74.0729847461233, 40.630308978981404], [-74.06802173543946, 40.62878758518541], [-74.07291177936548, 40.62999637940653], [-74.07225500955778, 40.62460875019394], [-74.06975543088323, 40.6212081398724], [-74.06544586069478, 40.61920262828925], [-74.06668207982919, 40.61806749593851], [-74.06425721751587, 40.61817168758555], [-74.06509908407213, 40.61751095612543], [-74.05685812390232, 40.608056357772554], [-74.0651709129999, 40.604213507999894], [-74.06350565699996, 40.60215005499991], [-74.07399965299999, 40.60572556699987], [-74.0815547009999, 40.60629029699987], [-74.0714060829999, 40.61943137899991], [-74.07253551699989, 40.622464021999896], [-74.08720982799996, 40.61330697299989], [-74.0878234189999, 40.615087934999885], [-74.08415419899998, 40.626267114999905], [-74.07972492199995, 40.62895068799986], [-74.08574928999992, 40.633955556999865], [-74.07878831099995, 40.63826855899991], [-74.07530550099992, 40.637729958999955]]]}}, {\"id\": \"221\", \"type\": \"Feature\", \"properties\": {\"count\": 403, \"OBJECTID\": 222, \"Shape_Leng\": 0.0483893527976, \"Shape_Area\": 0.000130268343918, \"zone\": \"Starrett City\", \"LocationID\": 222, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.87908517399995, 40.65020211699988], [-73.8745808109999, 40.64561010599992], [-73.87841589406925, 40.638912189353306], [-73.88006714132425, 40.64200344662675], [-73.88341999203011, 40.64452381564944], [-73.88557762124442, 40.644040950146135], [-73.88764072166725, 40.64780239000726], [-73.89125147199995, 40.65002559699993], [-73.88221364899994, 40.65451863799996], [-73.87908517399995, 40.65020211699988]]], [[[-73.8882853135604, 40.64672241327608], [-73.88864137620679, 40.647079553158], [-73.88851637568528, 40.64736069152191], [-73.88820050726648, 40.647140869529046], [-73.8882853135604, 40.64672241327608]]]]}}, {\"id\": \"222\", \"type\": \"Feature\", \"properties\": {\"count\": 114788, \"OBJECTID\": 223, \"Shape_Leng\": 0.166021925275, \"Shape_Area\": 0.00076436070058, \"zone\": \"Steinway\", \"LocationID\": 223, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90649492923363, 40.7901809903447], [-73.90032629743816, 40.789007662825696], [-73.8959296297551, 40.785648944313756], [-73.89874897638398, 40.78278115576602], [-73.90114577552107, 40.78271110609054], [-73.90257929672345, 40.78028912083556], [-73.89437777934127, 40.78424320348422], [-73.89606377738421, 40.783157201805224], [-73.895284679113, 40.78154124925482], [-73.89274917876409, 40.78296570668481], [-73.89165459123714, 40.78219478153064], [-73.89300986180193, 40.78179802356213], [-73.89119589704391, 40.77856328788702], [-73.89251064542336, 40.77739286927271], [-73.89183244902246, 40.77488019994431], [-73.88945486901865, 40.77353295106357], [-73.88770661799987, 40.76683859799995], [-73.89349605899986, 40.765950929999924], [-73.91759120499997, 40.769900870999926], [-73.92398142900002, 40.77461511299991], [-73.92375761299998, 40.77654701199991], [-73.91796581399993, 40.78125629199991], [-73.91996458796496, 40.78263506876824], [-73.912599859115, 40.78937632963394], [-73.90985862925774, 40.790945493781884], [-73.90649492923363, 40.7901809903447]]]}}, {\"id\": \"223\", \"type\": \"Feature\", \"properties\": {\"count\": 495970, \"OBJECTID\": 224, \"Shape_Leng\": 0.0440204462769, \"Shape_Area\": 9.84786989135e-05, \"zone\": \"Stuy Town/Peter Cooper Village\", \"LocationID\": 224, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97511055399991, 40.735390254999864], [-73.97151882628022, 40.72935136174009], [-73.97162697285862, 40.72662845299842], [-73.98255629299986, 40.731350133999875], [-73.97849845599995, 40.73679095699989], [-73.97511055399991, 40.735390254999864]]]}}, {\"id\": \"224\", \"type\": \"Feature\", \"properties\": {\"count\": 28660, \"OBJECTID\": 225, \"Shape_Leng\": 0.0887992073281, \"Shape_Area\": 0.000310835395162, \"zone\": \"Stuyvesant Heights\", \"LocationID\": 225, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91741294299999, 40.68403405199993], [-73.9163011989999, 40.67857711199991], [-73.92183500099998, 40.67894555899991], [-73.92281374299995, 40.68341442999991], [-73.93500821699992, 40.68276376799985], [-73.93435066499993, 40.67956340799991], [-73.94032794, 40.679889974999874], [-73.94394947299996, 40.698221278999924], [-73.93817718599986, 40.69864022299987], [-73.91804607, 40.68721324799989], [-73.91741294299999, 40.68403405199993]]]}}, {\"id\": \"225\", \"type\": \"Feature\", \"properties\": {\"count\": 269571, \"OBJECTID\": 226, \"Shape_Leng\": 0.168957762041, \"Shape_Area\": 0.00076194414504, \"zone\": \"Sunnyside\", \"LocationID\": 226, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90989961499994, 40.749087274999944], [-73.91187504099992, 40.74822079599992], [-73.9119412979999, 40.742382948999925], [-73.91567469599988, 40.740784929999904], [-73.91903825199986, 40.73200596599987], [-73.917342506, 40.731382825999916], [-73.91796696699996, 40.7285007709999], [-73.91205021299994, 40.72850636299993], [-73.91174734199988, 40.72991203299991], [-73.90712253099991, 40.727424727999924], [-73.91384828299984, 40.72289841199989], [-73.91132095099978, 40.71962422999992], [-73.92232773978448, 40.71659264406108], [-73.92515207271921, 40.722335339820816], [-73.9244229875485, 40.723589975986165], [-73.92037064235664, 40.723681113821044], [-73.92490316165177, 40.72450657567146], [-73.92923485876162, 40.72826016576104], [-73.93804478507161, 40.730509967846054], [-73.94162472636452, 40.73584144233816], [-73.94594410333028, 40.73751251570048], [-73.94140886345744, 40.73929957085316], [-73.93869664599998, 40.738140816999866], [-73.93726110699994, 40.745176490999896], [-73.94401030399996, 40.7436394539999], [-73.93383269399993, 40.7516047019999], [-73.91547326299991, 40.75379181799991], [-73.91035492999984, 40.75274990899987], [-73.90989961499994, 40.749087274999944]]]}}, {\"id\": \"226\", \"type\": \"Feature\", \"properties\": {\"count\": 2393, \"OBJECTID\": 227, \"Shape_Leng\": 0.0828301570789, \"Shape_Area\": 0.000268316529534, \"zone\": \"Sunset Park East\", \"LocationID\": 227, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.991928124, 40.6422049039999], [-73.9955103369999, 40.63980966699993], [-73.99917564699989, 40.642025441999934], [-74.00735636699996, 40.6341636269999], [-74.00556490399993, 40.63308317599992], [-74.01000340799995, 40.630113088999906], [-74.01602010100001, 40.63331711399997], [-74.016835401, 40.63535542899995], [-74.0139608859999, 40.63815326699995], [-74.01616711499993, 40.63948244099991], [-74.00214474699993, 40.65295610699995], [-73.98905872499985, 40.64411924099995], [-73.991928124, 40.6422049039999]]]}}, {\"id\": \"227\", \"type\": \"Feature\", \"properties\": {\"count\": 20354, \"OBJECTID\": 228, \"Shape_Leng\": 0.177685177706, \"Shape_Area\": 0.000993414783816, \"zone\": \"Sunset Park West\", \"LocationID\": 228, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9969179149999, 40.6691534839999], [-73.99490350099988, 40.6667854349999], [-73.99692689899985, 40.664374914999925], [-73.99063545399997, 40.660579088999896], [-73.988716511, 40.66240493799995], [-73.98725068899996, 40.66037895599992], [-73.990188415, 40.657600817999906], [-73.99516890199999, 40.659669521999945], [-74.01616711499993, 40.63948244099991], [-74.0139608859999, 40.63815326699995], [-74.016835401, 40.63535542899995], [-74.02141038399992, 40.638584902999845], [-74.02108051399998, 40.64149093699994], [-74.02475437599998, 40.6381214179999], [-74.03266558998958, 40.643720344890056], [-74.03051830863413, 40.645274425208974], [-74.02820569545007, 40.644015982890345], [-74.02604948400315, 40.64626135032611], [-74.0293899048662, 40.64880405511391], [-74.02571758804807, 40.64661549229093], [-74.02461487103106, 40.64715112511878], [-74.02622102475574, 40.64824842780924], [-74.02414566050564, 40.647615101259845], [-74.02605133196793, 40.6509943007431], [-74.02285141795983, 40.651169195161685], [-74.02522662972656, 40.652829856959634], [-74.02154138319214, 40.650671622911226], [-74.02380703903121, 40.65304472851146], [-74.02118670027885, 40.652839836895325], [-74.02320464727141, 40.654096116817925], [-74.0186223830857, 40.653491349022964], [-74.02005182454292, 40.655335948874416], [-74.01715319581021, 40.65477510436213], [-74.01970596410222, 40.656581163751035], [-74.01639437533773, 40.65562064285071], [-74.01910843803118, 40.657368884303494], [-74.01585370056823, 40.65621324055534], [-74.01786573109669, 40.65830844729852], [-74.01442130484548, 40.657608021951376], [-74.01745803087898, 40.6594501973867], [-74.01554165388997, 40.66075835547614], [-74.01183862056543, 40.65891283727472], [-74.0131555493881, 40.66198159772593], [-74.00859956968284, 40.65952092309827], [-74.00745858163884, 40.66055847620986], [-74.0103720129723, 40.66239888023248], [-74.00955074142026, 40.6632676941304], [-74.0034880110065, 40.66224055612849], [-74.00599748682693, 40.66325861082859], [-74.00772133506703, 40.66471863301977], [-74.0071253264673, 40.66491521847423], [-74.00362862508561, 40.66273484753458], [-74.00486193792649, 40.66506482193817], [-74.00083401108444, 40.66300644269171], [-73.99953152609916, 40.6643974859369], [-74.00292463516384, 40.66646231216332], [-73.99998149339123, 40.66745027490869], [-73.99902600998384, 40.66844297212693], [-73.9986188799999, 40.67183017199989], [-73.9969179149999, 40.6691534839999]]]}}, {\"id\": \"228\", \"type\": \"Feature\", \"properties\": {\"count\": 2808372, \"OBJECTID\": 229, \"Shape_Leng\": 0.0425252900116, \"Shape_Area\": 9.47058230931e-05, \"zone\": \"Sutton Place/Turtle Bay North\", \"LocationID\": 229, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96162927534907, 40.75515238425915], [-73.96482214210387, 40.75159632579112], [-73.9712170449999, 40.75519332399994], [-73.9665833839999, 40.76155093499988], [-73.95877790811335, 40.75827092092435], [-73.96162927534907, 40.75515238425915]]]}}, {\"id\": \"229\", \"type\": \"Feature\", \"properties\": {\"count\": 4704265, \"OBJECTID\": 230, \"Shape_Leng\": 0.0310283096779, \"Shape_Area\": 5.60914463266e-05, \"zone\": \"Times Sq/Theatre District\", \"LocationID\": 230, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98362433299988, 40.75551634599986], [-73.98979085399996, 40.7572331319999], [-73.98428230099987, 40.76479181499994], [-73.97860169399989, 40.762396071999945], [-73.98362433299988, 40.75551634599986]]]}}, {\"id\": \"230\", \"type\": \"Feature\", \"properties\": {\"count\": 2850971, \"OBJECTID\": 231, \"Shape_Leng\": 0.0634201165587, \"Shape_Area\": 0.000166929752992, \"zone\": \"TriBeCa/Civic Center\", \"LocationID\": 231, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00931760399995, 40.72470155099989], [-73.99995620900002, 40.71801709499987], [-74.00485768099996, 40.7124525409999], [-74.00834760999993, 40.71135854599987], [-74.01375388499997, 40.71369055499987], [-74.01244109199989, 40.719057672999874], [-74.01457231438083, 40.72027413774219], [-74.01296558526825, 40.720328675587126], [-74.01102457999994, 40.72579386199991], [-74.00931760399995, 40.72470155099989]]]}}, {\"id\": \"231\", \"type\": \"Feature\", \"properties\": {\"count\": 318177, \"OBJECTID\": 232, \"Shape_Leng\": 0.0614709085331, \"Shape_Area\": 0.000216049973456, \"zone\": \"Two Bridges/Seward Park\", \"LocationID\": 232, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97635525199995, 40.71913288399983], [-73.97347975524508, 40.7188614291129], [-73.97782091638247, 40.71064544490706], [-73.9919434659999, 40.70958905299992], [-73.99256242199989, 40.71438807699996], [-73.99022012699989, 40.71440493999988], [-73.98382387299982, 40.72147287199987], [-73.97635525199995, 40.71913288399983]]]}}, {\"id\": \"232\", \"type\": \"Feature\", \"properties\": {\"count\": 2024461, \"OBJECTID\": 233, \"Shape_Leng\": 0.048035918294, \"Shape_Area\": 0.000116188094682, \"zone\": \"UN/Turtle Bay South\", \"LocationID\": 233, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.96547092299994, 40.75193959699991], [-73.97123301520192, 40.743414663960074], [-73.97350835000003, 40.7437820339999], [-73.97121971999988, 40.74692154099996], [-73.9758346669999, 40.74886314599996], [-73.9712170449999, 40.75519332399994], [-73.96547092299994, 40.75193959699991]]], [[[-73.96421230395673, 40.74660431847665], [-73.96444522752618, 40.74641042576318], [-73.96458318426942, 40.746454400673315], [-73.96415980288437, 40.74686554735954], [-73.96421230395673, 40.74660431847665]]]]}}, {\"id\": \"233\", \"type\": \"Feature\", \"properties\": {\"count\": 4755323, \"OBJECTID\": 234, \"Shape_Leng\": 0.0360721994984, \"Shape_Area\": 7.31054382894e-05, \"zone\": \"Union Sq\", \"LocationID\": 234, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98996936399989, 40.73490456699994], [-73.99683993899998, 40.73736088899995], [-73.99051761299985, 40.746038637999895], [-73.98407485299995, 40.74332471799989], [-73.98996936399989, 40.73490456699994]]]}}, {\"id\": \"234\", \"type\": \"Feature\", \"properties\": {\"count\": 3476, \"OBJECTID\": 235, \"Shape_Leng\": 0.0761668426519, \"Shape_Area\": 0.000212844547337, \"zone\": \"University Heights/Morris Heights\", \"LocationID\": 235, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90947862999988, 40.86180812899987], [-73.91328437000003, 40.859618600999944], [-73.90976165899988, 40.857182611999896], [-73.90679734899983, 40.85937358899988], [-73.90468371399987, 40.85706030199991], [-73.91416184199993, 40.845179187999925], [-73.92808209386837, 40.84539159691672], [-73.91422824049913, 40.862490790359615], [-73.90947862999988, 40.86180812899987]]]}}, {\"id\": \"235\", \"type\": \"Feature\", \"properties\": {\"count\": 4835549, \"OBJECTID\": 236, \"Shape_Leng\": 0.0442519223099, \"Shape_Area\": 0.000102864345211, \"zone\": \"Upper East Side North\", \"LocationID\": 236, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95779380499984, 40.773599896999926], [-73.96468735799992, 40.77567708199994], [-73.9557773589999, 40.78791392399995], [-73.94933170599992, 40.78519312699994], [-73.95779380499984, 40.773599896999926]]]}}, {\"id\": \"236\", \"type\": \"Feature\", \"properties\": {\"count\": 5325938, \"OBJECTID\": 237, \"Shape_Leng\": 0.0422126003388, \"Shape_Area\": 9.59873206621e-05, \"zone\": \"Upper East Side South\", \"LocationID\": 237, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9661274729999, 40.762179299999914], [-73.97301487199995, 40.76427887899991], [-73.96468735799992, 40.77567708199994], [-73.95825534899998, 40.772965336999896], [-73.9661274729999, 40.762179299999914]]]}}, {\"id\": \"237\", \"type\": \"Feature\", \"properties\": {\"count\": 2555828, \"OBJECTID\": 238, \"Shape_Leng\": 0.0601093114033, \"Shape_Area\": 0.000184763693765, \"zone\": \"Upper West Side North\", \"LocationID\": 238, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96884378999985, 40.78596738899994], [-73.98188645923244, 40.7905165804675], [-73.97639951965265, 40.79827321084623], [-73.96417598599989, 40.79236204499989], [-73.96884378999985, 40.78596738899994]]]}}, {\"id\": \"238\", \"type\": \"Feature\", \"properties\": {\"count\": 3486261, \"OBJECTID\": 239, \"Shape_Leng\": 0.0636261152958, \"Shape_Area\": 0.000204715440774, \"zone\": \"Upper West Side South\", \"LocationID\": 239, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97501417199996, 40.78768560599987], [-73.96933573299991, 40.78529327999992], [-73.97499744000007, 40.77753254599988], [-73.98507184299989, 40.781779680999946], [-73.98567936099997, 40.780312632999944], [-73.9881274613434, 40.78140179672366], [-73.98546581197031, 40.78536070057545], [-73.98711901394246, 40.7852103190041], [-73.98542932126932, 40.78541394218458], [-73.98465507883023, 40.78653474180792], [-73.98594956155647, 40.78648711396651], [-73.98188645923244, 40.7905165804675], [-73.97501417199996, 40.78768560599987]]]}}, {\"id\": \"239\", \"type\": \"Feature\", \"properties\": {\"count\": 474, \"OBJECTID\": 240, \"Shape_Leng\": 0.146069764379, \"Shape_Area\": 0.000722130920707, \"zone\": \"Van Cortlandt Park\", \"LocationID\": 240, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87643743099994, 40.89687059299995], [-73.8771197409999, 40.895468663999964], [-73.87048672599987, 40.896640210999905], [-73.86135927899984, 40.89466231899991], [-73.86943471300005, 40.87812919899994], [-73.87094367500002, 40.87851076699998], [-73.87024100099985, 40.88022935299995], [-73.87494589799991, 40.87994766899988], [-73.87467601499999, 40.88156285199993], [-73.87812594399992, 40.88215382599989], [-73.87803562399996, 40.887118321999885], [-73.88124422499992, 40.88270963299992], [-73.89972588499982, 40.886514886999926], [-73.89643170799987, 40.89365527899989], [-73.89663333831646, 40.91141737492314], [-73.86789043706806, 40.90298695407611], [-73.87643743099994, 40.89687059299995]]]}}, {\"id\": \"240\", \"type\": \"Feature\", \"properties\": {\"count\": 1630, \"OBJECTID\": 241, \"Shape_Leng\": 0.0687645777649, \"Shape_Area\": 0.00025467697592, \"zone\": \"Van Cortlandt Village\", \"LocationID\": 241, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88840157099996, 40.88441707499993], [-73.88705142599984, 40.884349575999906], [-73.88567517999988, 40.87887251099986], [-73.88937163799984, 40.87338229799993], [-73.8879350369998, 40.87203023199988], [-73.89438130299995, 40.866068850999966], [-73.89891854299988, 40.86965857399992], [-73.89942436199988, 40.86788083799993], [-73.90374975899987, 40.86959562699992], [-73.90608130599988, 40.873239628999904], [-73.89855059899999, 40.88437472699996], [-73.89661546599986, 40.885955255999946], [-73.88840157099996, 40.88441707499993]]]}}, {\"id\": \"241\", \"type\": \"Feature\", \"properties\": {\"count\": 1682, \"OBJECTID\": 242, \"Shape_Leng\": 0.138136446433, \"Shape_Area\": 0.000360072450014, \"zone\": \"Van Nest/Morris Park\", \"LocationID\": 242, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83593362199987, 40.84840635599993], [-73.83771374799997, 40.84040630999987], [-73.84204144699997, 40.84064589699992], [-73.84458129199986, 40.838535507999914], [-73.85052278099987, 40.84278948899985], [-73.87359713599987, 40.83979855499992], [-73.86804200399997, 40.8448138389999], [-73.86860515599989, 40.85206864599991], [-73.86303938099996, 40.850611306999944], [-73.86250923799993, 40.84836228099987], [-73.85457388299992, 40.85150834599995], [-73.85304526899994, 40.849547331999865], [-73.84430846999992, 40.85119046399989], [-73.84494053799982, 40.8482182209999], [-73.84049767399992, 40.8541841049999], [-73.83499552599994, 40.85657120099989], [-73.8368191849999, 40.85727680899989], [-73.82847346699995, 40.860731352999906], [-73.83283502299986, 40.8560358449999], [-73.83129157699989, 40.85543410499991], [-73.83463970199993, 40.853817993999904], [-73.83593362199987, 40.84840635599993]]]}}, {\"id\": \"242\", \"type\": \"Feature\", \"properties\": {\"count\": 32518, \"OBJECTID\": 243, \"Shape_Leng\": 0.0943314999364, \"Shape_Area\": 0.000438377341503, \"zone\": \"Washington Heights North\", \"LocationID\": 243, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93156536999994, 40.86958215799991], [-73.92474153399992, 40.86159363799992], [-73.92709951599986, 40.85827552099992], [-73.92467094699998, 40.8568108889999], [-73.92747680599985, 40.85069436999992], [-73.93139355399987, 40.84742823099985], [-73.94203871799984, 40.851931028999914], [-73.94317806099995, 40.84979994299991], [-73.94692626452422, 40.850528055834154], [-73.94186996426673, 40.853867739442755], [-73.93156536999994, 40.86958215799991]]]}}, {\"id\": \"243\", \"type\": \"Feature\", \"properties\": {\"count\": 134680, \"OBJECTID\": 244, \"Shape_Leng\": 0.0805686044599, \"Shape_Area\": 0.000359702747951, \"zone\": \"Washington Heights South\", \"LocationID\": 244, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94068822000003, 40.85131543299985], [-73.93139355399987, 40.84742823099985], [-73.93556697199989, 40.841712527999924], [-73.93505508799984, 40.8393332359999], [-73.94034643299983, 40.8304578419999], [-73.95015521048293, 40.83439675940791], [-73.94612416421772, 40.84389249655712], [-73.9469644175959, 40.85046552581819], [-73.94317806099984, 40.84979994399989], [-73.94203871799984, 40.851931028999914], [-73.94068822000003, 40.85131543299985]]]}}, {\"id\": \"244\", \"type\": \"Feature\", \"properties\": {\"count\": 84, \"OBJECTID\": 245, \"Shape_Leng\": 0.0959830596604, \"Shape_Area\": 0.000466175414294, \"zone\": \"West Brighton\", \"LocationID\": 245, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.09787969199995, 40.64035805499988], [-74.09767290799995, 40.63896067899989], [-74.09409312599998, 40.639266913999975], [-74.09242513499994, 40.63558181299992], [-74.08870582299987, 40.63684520599986], [-74.08373397299994, 40.63573714399991], [-74.09223971399994, 40.62574291599987], [-74.10350925299994, 40.61641607499991], [-74.1132624959999, 40.62305442499995], [-74.11614321199995, 40.62870226399988], [-74.11765722499997, 40.63806246599996], [-74.10974833299994, 40.63786583299993], [-74.09787969199995, 40.64035805499988]]]}}, {\"id\": \"245\", \"type\": \"Feature\", \"properties\": {\"count\": 2220769, \"OBJECTID\": 246, \"Shape_Leng\": 0.0694671937437, \"Shape_Area\": 0.000281298327076, \"zone\": \"West Chelsea/Hudson Yards\", \"LocationID\": 246, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00439976203513, 40.76267135909888], [-73.99547450199992, 40.759626856999894], [-74.00820401899988, 40.74214751799986], [-74.01213726936172, 40.74353192705926], [-74.0087278005888, 40.74485823290996], [-74.01183642872407, 40.74512596367872], [-74.00946389855962, 40.74575014365948], [-74.01150411235528, 40.74699080250515], [-74.0091373625992, 40.74735976280298], [-74.01133174441699, 40.748000575341926], [-74.0090584790135, 40.74778564533286], [-74.00894606502312, 40.74841189316866], [-74.01121562130406, 40.74869682117583], [-74.00842636447226, 40.75215785023921], [-74.0099905581974, 40.75281618714907], [-74.00997058797012, 40.75293356415314], [-74.00835352838844, 40.752350739342255], [-74.00480392560178, 40.75780984316466], [-74.00700358904015, 40.759231870341075], [-74.00390742032765, 40.75937569218354], [-74.00233528130057, 40.76154372700156], [-74.00439976203513, 40.76267135909888]]]}}, {\"id\": \"246\", \"type\": \"Feature\", \"properties\": {\"count\": 19472, \"OBJECTID\": 247, \"Shape_Leng\": 0.0929678582271, \"Shape_Area\": 0.000205617157843, \"zone\": \"West Concourse\", \"LocationID\": 247, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91222180499983, 40.84235659099988], [-73.91986089399987, 40.83194722699992], [-73.92885642099992, 40.81487986599991], [-73.93027812699984, 40.813248024999965], [-73.93242632000944, 40.81411751897961], [-73.9330725108407, 40.82815615201811], [-73.92850073899989, 40.829352349999965], [-73.92399049499993, 40.83505229799987], [-73.91901668400004, 40.83838109299989], [-73.91618344399991, 40.84233489999991], [-73.91794965199988, 40.842229570999976], [-73.91729224999992, 40.84510993999991], [-73.91222180499983, 40.84235659099988]]]}}, {\"id\": \"247\", \"type\": \"Feature\", \"properties\": {\"count\": 1126, \"OBJECTID\": 248, \"Shape_Leng\": 0.0569191167671, \"Shape_Area\": 0.000149592769799, \"zone\": \"West Farms/Bronx River\", \"LocationID\": 248, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86393748099981, 40.84004456599994], [-73.86271114399993, 40.83299150899997], [-73.88378414099992, 40.828274291999946], [-73.88005188499987, 40.83470205899989], [-73.87379590199993, 40.837445132999896], [-73.87277204699981, 40.83975128199995], [-73.86448011799995, 40.84108779699994], [-73.86393748099981, 40.84004456599994]]]}}, {\"id\": \"248\", \"type\": \"Feature\", \"properties\": {\"count\": 3193371, \"OBJECTID\": 249, \"Shape_Leng\": 0.03638434365, \"Shape_Area\": 7.22155957729e-05, \"zone\": \"West Village\", \"LocationID\": 249, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00250642399995, 40.729016384999966], [-74.0070902079999, 40.72877225299993], [-74.00515808199998, 40.74085808099991], [-73.99683993899998, 40.73736088899995], [-74.00250642399995, 40.729016384999966]]]}}, {\"id\": \"249\", \"type\": \"Feature\", \"properties\": {\"count\": 1728, \"OBJECTID\": 250, \"Shape_Leng\": 0.0796263465454, \"Shape_Area\": 0.000240975845956, \"zone\": \"Westchester Village/Unionport\", \"LocationID\": 250, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8455308949999, 40.83917330699989], [-73.83936452799996, 40.84087706699995], [-73.83959108476137, 40.834041688045254], [-73.84390624999997, 40.82840782999993], [-73.84289858800003, 40.82408305899986], [-73.85513639799981, 40.822436188999866], [-73.857391213, 40.83229737199986], [-73.86033845899995, 40.833253165999935], [-73.85168459699997, 40.8342010519999], [-73.85333524999986, 40.84247015499987], [-73.85052278099987, 40.84278948899985], [-73.8455308949999, 40.83917330699989]]]}}, {\"id\": \"250\", \"type\": \"Feature\", \"properties\": {\"count\": 134, \"OBJECTID\": 251, \"Shape_Leng\": 0.1377111611, \"Shape_Area\": 0.000625754983157, \"zone\": \"Westerleigh\", \"LocationID\": 251, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.13107460299996, 40.631147728999956], [-74.1313098529999, 40.62632834299998], [-74.12125677899994, 40.62782274999989], [-74.10552598799994, 40.618592329999906], [-74.0977696879999, 40.61062359299988], [-74.1156683379999, 40.611175384999896], [-74.13210459699991, 40.608152923999924], [-74.14471100599997, 40.60803429699994], [-74.14954343599996, 40.60962717299988], [-74.14696864199988, 40.610151355999896], [-74.14603562099998, 40.61701277699996], [-74.13982184799993, 40.61629982499988], [-74.13826358199992, 40.624243769999936], [-74.13581008299997, 40.62506433999991], [-74.13806393799992, 40.627568470999904], [-74.13107460299996, 40.631147728999956]]]}}, {\"id\": \"251\", \"type\": \"Feature\", \"properties\": {\"count\": 740, \"OBJECTID\": 252, \"Shape_Leng\": 0.158004373564, \"Shape_Area\": 0.00102526406057, \"zone\": \"Whitestone\", \"LocationID\": 252, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82049919995306, 40.80101146781907], [-73.81369963725258, 40.79707941566509], [-73.81225365567188, 40.79812694680175], [-73.79640953426332, 40.795728824199614], [-73.79442844782817, 40.794806810498535], [-73.79375126484103, 40.78906193333896], [-73.79058298999988, 40.78819290999989], [-73.79582463099997, 40.7861305879999], [-73.80242791599977, 40.786068273999945], [-73.8037902289999, 40.77561011199992], [-73.81485850699985, 40.776485922999875], [-73.81452997799975, 40.77888817499992], [-73.82331286899986, 40.77957865999986], [-73.82224127699999, 40.781693878999924], [-73.83869921299997, 40.781676956999874], [-73.83824429099994, 40.78778591599995], [-73.84008157499997, 40.78785153099992], [-73.83895676099984, 40.79143843099996], [-73.84081532599994, 40.7915048849999], [-73.84067828300006, 40.79390789199993], [-73.83663288911686, 40.79251637576927], [-73.83728024446076, 40.78900952077265], [-73.83251063648018, 40.788624476626765], [-73.83166080927866, 40.79110259115967], [-73.8278487557192, 40.79308767321416], [-73.82945991534359, 40.79682336759143], [-73.82506496699443, 40.79737577656827], [-73.82049919995306, 40.80101146781907]]]}}, {\"id\": \"252\", \"type\": \"Feature\", \"properties\": {\"count\": 433, \"OBJECTID\": 253, \"Shape_Leng\": 0.0360514998192, \"Shape_Area\": 7.83395761191e-05, \"zone\": \"Willets Point\", \"LocationID\": 253, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83908354399988, 40.76525691299991], [-73.83679107299986, 40.76116667099992], [-73.83867321899987, 40.75701997899995], [-73.84324381399976, 40.75533680399995], [-73.84670933714987, 40.76134526931977], [-73.84272819396486, 40.76462854893166], [-73.83908354399988, 40.76525691299991]]]}}, {\"id\": \"253\", \"type\": \"Feature\", \"properties\": {\"count\": 943, \"OBJECTID\": 254, \"Shape_Leng\": 0.0858863754861, \"Shape_Area\": 0.000360040216032, \"zone\": \"Williamsbridge/Olinville\", \"LocationID\": 254, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85186563799999, 40.87905886499989], [-73.85091896099982, 40.877893440999856], [-73.85254854099988, 40.877091586999924], [-73.8506895569999, 40.87548667799993], [-73.85363638799987, 40.87330059899996], [-73.85651292399994, 40.874217792999936], [-73.85661382499993, 40.87125016699991], [-73.87056192499989, 40.87151077799995], [-73.87019944899986, 40.87631135499987], [-73.860710982, 40.89537054399996], [-73.84590568599998, 40.889741368999864], [-73.85186563799999, 40.87905886499989]]]}}, {\"id\": \"254\", \"type\": \"Feature\", \"properties\": {\"count\": 331479, \"OBJECTID\": 255, \"Shape_Leng\": 0.0623841997664, \"Shape_Area\": 0.000172309184842, \"zone\": \"Williamsburg (North Side)\", \"LocationID\": 255, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96176070375392, 40.72522879205536], [-73.95477662900004, 40.72245950499994], [-73.94907816799986, 40.72254028699993], [-73.94841638599995, 40.71845553299995], [-73.94721341799986, 40.718560985999865], [-73.95547596099992, 40.7122654259999], [-73.96736391666703, 40.71648367952745], [-73.96176070375392, 40.72522879205536]]]}}, {\"id\": \"255\", \"type\": \"Feature\", \"properties\": {\"count\": 272435, \"OBJECTID\": 256, \"Shape_Leng\": 0.0679149669603, \"Shape_Area\": 0.000168611097013, \"zone\": \"Williamsburg (South Side)\", \"LocationID\": 256, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95834207500002, 40.71330630099992], [-73.95547596099992, 40.7122654259999], [-73.95433904200002, 40.714071701999906], [-73.94937231899985, 40.71406687799986], [-73.94865533499987, 40.70972498299988], [-73.95087492699996, 40.70951311499991], [-73.95023693799996, 40.70547324699994], [-73.9579043079999, 40.70842717299994], [-73.958455768, 40.707251846999895], [-73.96838933699995, 40.70682918699989], [-73.96984864823531, 40.708003041687675], [-73.96736391666705, 40.71648367902752], [-73.95834207500002, 40.71330630099992]]]}}, {\"id\": \"256\", \"type\": \"Feature\", \"properties\": {\"count\": 7913, \"OBJECTID\": 257, \"Shape_Leng\": 0.0586690259793, \"Shape_Area\": 0.00013890947321, \"zone\": \"Windsor Terrace\", \"LocationID\": 257, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97984261899994, 40.660727440999956], [-73.97436701299992, 40.65818790399992], [-73.97084113699994, 40.64637857099993], [-73.97777380600002, 40.645341876999964], [-73.97819074399992, 40.64752599199992], [-73.98029115499993, 40.647297160999855], [-73.98179568699996, 40.65524609699991], [-73.98843135299977, 40.65925121199996], [-73.9868846449999, 40.6603663889999], [-73.985086794, 40.65901328799989], [-73.982230342, 40.66214149199994], [-73.97984261899994, 40.660727440999956]]]}}, {\"id\": \"257\", \"type\": \"Feature\", \"properties\": {\"count\": 2879, \"OBJECTID\": 258, \"Shape_Leng\": 0.0890133787693, \"Shape_Area\": 0.000366209617143, \"zone\": \"Woodhaven\", \"LocationID\": 258, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8450419489999, 40.68931894699996], [-73.84119605699982, 40.68335136599989], [-73.84786817599985, 40.681486831999905], [-73.85197864599989, 40.682564210999864], [-73.85099089299987, 40.68056836699987], [-73.85392389199987, 40.679686155999875], [-73.85819836099995, 40.67943084899987], [-73.85878576899987, 40.68063046299989], [-73.86334943599995, 40.67935163999989], [-73.86410096700003, 40.68237284999988], [-73.86602666899996, 40.681918051999915], [-73.86868454399996, 40.694034691999924], [-73.85277238699999, 40.69755711199987], [-73.85443455699989, 40.70060295199994], [-73.85070842300001, 40.70020312499988], [-73.8450419489999, 40.68931894699996]]]}}, {\"id\": \"258\", \"type\": \"Feature\", \"properties\": {\"count\": 873, \"OBJECTID\": 259, \"Shape_Leng\": 0.126750305191, \"Shape_Area\": 0.000394552487366, \"zone\": \"Woodlawn/Wakefield\", \"LocationID\": 259, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.851071161919, 40.91037152011096], [-73.85347522366716, 40.90753029351422], [-73.84138689269206, 40.90417274176649], [-73.83554853999995, 40.88913350399993], [-73.84704957999995, 40.8876718159999], [-73.84590568599998, 40.889741368999864], [-73.860710982, 40.89537054399996], [-73.87048672599987, 40.896640210999905], [-73.8771197409999, 40.895468663999964], [-73.86789043706806, 40.90298695407611], [-73.85946778750369, 40.90051720913338], [-73.85957882974115, 40.9024408428857], [-73.85601023344621, 40.9053059838135], [-73.85681013619275, 40.90615582816297], [-73.85374103615364, 40.90778791636777], [-73.85458940544117, 40.9089394991491], [-73.851071161919, 40.91037152011096]]]}}, {\"id\": \"259\", \"type\": \"Feature\", \"properties\": {\"count\": 94392, \"OBJECTID\": 260, \"Shape_Leng\": 0.133514154636, \"Shape_Area\": 0.000422345326907, \"zone\": \"Woodside\", \"LocationID\": 260, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9017537339999, 40.760775474999946], [-73.90147053099994, 40.759633049999884], [-73.89866137799999, 40.759921016999904], [-73.89976862099998, 40.75666423699991], [-73.8949120899999, 40.74428123099996], [-73.90057825399985, 40.73949289599988], [-73.90926002999988, 40.736403179999854], [-73.91068876399997, 40.73347108099995], [-73.90628903499987, 40.732400751999876], [-73.90516948899983, 40.73585964299986], [-73.90107970900003, 40.73391463399987], [-73.90712253099991, 40.727424727999924], [-73.91174734199988, 40.72991203299991], [-73.91205021299994, 40.72850636299993], [-73.91796696699996, 40.7285007709999], [-73.917342506, 40.731382825999916], [-73.91903825199986, 40.73200596599987], [-73.91651190599994, 40.738944757999874], [-73.9137938869998, 40.742514491999934], [-73.9119412979999, 40.742382948999925], [-73.91190979099993, 40.748105394999875], [-73.90945182799983, 40.75065550699994], [-73.91067474899997, 40.752995076999895], [-73.90631505099988, 40.75984387599992], [-73.90779405499987, 40.760517799999896], [-73.90356915599997, 40.765781788999924], [-73.9017537339999, 40.760775474999946]]]}}, {\"id\": \"260\", \"type\": \"Feature\", \"properties\": {\"count\": 731080, \"OBJECTID\": 261, \"Shape_Leng\": 0.0271204563616, \"Shape_Area\": 3.43423231652e-05, \"zone\": \"World Trade Center\", \"LocationID\": 261, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01332610899989, 40.7050307879999], [-74.01668613099999, 40.70479933799992], [-74.01375388499997, 40.71369055499987], [-74.00860085399992, 40.711392239999945], [-74.01332610899989, 40.7050307879999]]]}}, {\"id\": \"261\", \"type\": \"Feature\", \"properties\": {\"count\": 1702036, \"OBJECTID\": 262, \"Shape_Leng\": 0.0490636231541, \"Shape_Area\": 0.000122330270966, \"zone\": \"Yorkville East\", \"LocationID\": 262, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.94383256699986, 40.78285908899991], [-73.94207418803879, 40.77691784697707], [-73.94293043781381, 40.774676268035776], [-73.9474897547817, 40.77011515400953], [-73.95268514999995, 40.772339404999926], [-73.94383256699986, 40.78285908899991]]], [[[-73.93804640603422, 40.78082954427547], [-73.93759894622617, 40.78046784086141], [-73.93958378972465, 40.779576474007136], [-73.9387442308927, 40.78104387604228], [-73.93804640603422, 40.78082954427547]]]]}}, {\"id\": \"262\", \"type\": \"Feature\", \"properties\": {\"count\": 2514462, \"OBJECTID\": 263, \"Shape_Leng\": 0.0370166252994, \"Shape_Area\": 6.57697664169e-05, \"zone\": \"Yorkville West\", \"LocationID\": 263, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95218621999996, 40.7730198449999], [-73.95729409999986, 40.7742835549999], [-73.94933170599992, 40.78519312699994], [-73.94472264499986, 40.78324757199987], [-73.95218621999996, 40.7730198449999]]]}}]}"},"id":"76c418d0-c7d9-48f9-8111-ae726987d2b2","type":"GeoJSONDataSource"},{"attributes":{},"id":"022fd809-ec00-4851-b661-c49991bfae3a","type":"WheelZoomTool"},{"attributes":{"fill_alpha":{"value":0.9},"fill_color":{"field":"count","transform":{"id":"fe7eac4c-a6ac-449b-a2da-89f027e5010b","type":"LogColorMapper"}},"line_alpha":{"value":0.5},"xs":{"field":"xs"},"ys":{"field":"ys"}},"id":"f1f65e0f-b376-4763-ba65-310d09a85747","type":"Patches"},{"attributes":{"dimension":1,"plot":{"id":"62ba6d47-3ae9-4aa5-be3a-40808ec4ef1a","subtype":"Figure","type":"Plot"},"ticker":{"id":"7d621023-4d97-44a1-8ccf-0992cf3ef267","type":"BasicTicker"},"visible":false},"id":"e47b8c6a-ee73-4698-8e94-0349d21c6c88","type":"Grid"},{"attributes":{},"id":"d41fbb68-5ad0-439f-87ea-f1e11384576c","type":"BasicTickFormatter"},{"attributes":{},"id":"6a2d253b-ce19-4ab5-8e57-e3afe7f8a209","type":"WheelZoomTool"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"3872cd96-a3c4-4d3c-a8d3-d671c5190f24","type":"BoxAnnotation"},{"attributes":{},"id":"26eacfd5-0f25-4429-acdb-7da3f6282337","type":"BasicTicker"},{"attributes":{"callback":null,"point_policy":"follow_mouse","tooltips":"&lt;div&gt;&lt;b&gt;Borough&lt;/b&gt;: @borough&lt;/div&gt;&lt;div&gt;&lt;b&gt;Zone&lt;/b&gt;: @zone&lt;/div&gt;&lt;div&gt;&lt;b&gt;Cash Fraction&lt;/b&gt;: @payment_type&lt;/div&gt;"},"id":"05d6bea6-6455-4d7f-a047-890c32987b9a","type":"HoverTool"},{"attributes":{"callback":null},"id":"169e471d-7586-4e4b-82ae-deea3e05ec36","type":"DataRange1d"},{"attributes":{"source":{"id":"76c418d0-c7d9-48f9-8111-ae726987d2b2","type":"GeoJSONDataSource"}},"id":"a00c5370-a7c6-4a7c-8839-0d6510dfa166","type":"CDSView"},{"attributes":{},"id":"2e37b525-1a12-4a01-9264-a099c6ca242b","type":"LinearScale"},{"attributes":{},"id":"f3b86a6f-66d5-48ed-adbf-ebdc3e9fd079","type":"BasicTickFormatter"},{"attributes":{},"id":"8b6fba3c-2dc4-43a3-9e6b-3b0aa55f67a7","type":"SaveTool"},{"attributes":{"formatter":{"id":"e384a837-7ce6-4ba0-8f0e-e4a8ffc64518","type":"BasicTickFormatter"},"plot":{"id":"9a6525f3-7059-45ab-90ee-2d7e83c0a07a","subtype":"Figure","type":"Plot"},"ticker":{"id":"c421e4d7-01a7-43de-a95e-fdff91e61d6f","type":"BasicTicker"},"visible":false},"id":"9139b6c7-121e-4d5b-9b54-caeceab315d1","type":"LinearAxis"},{"attributes":{},"id":"3b6a7eac-00a3-410a-8bc1-41a46ac7b38a","type":"ResetTool"},{"attributes":{"overlay":{"id":"3872cd96-a3c4-4d3c-a8d3-d671c5190f24","type":"BoxAnnotation"}},"id":"5e99a9d3-a981-45fc-8281-f0e7c0c6021c","type":"BoxZoomTool"},{"attributes":{},"id":"c936782d-eea5-4fb4-bd79-1d11686a2dfa","type":"LinearScale"},{"attributes":{"callback":null},"id":"dc69c649-db35-4baa-b925-35cf40cf46ef","type":"DataRange1d"},{"attributes":{"callback":null,"geojson":"{\"type\": \"FeatureCollection\", \"features\": [{\"id\": \"2\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5611620795107034, \"OBJECTID\": 3, \"Shape_Leng\": 0.0843411059012, \"Shape_Area\": 0.000314414156821, \"zone\": \"Allerton/Pelham Gardens\", \"LocationID\": 3, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84792614099985, 40.87134223399991], [-73.82950892699994, 40.86184121299988], [-73.82834347399992, 40.86089202599988], [-73.8368191849999, 40.85727680899989], [-73.8566377469999, 40.8581191719999], [-73.85780378799993, 40.86303621699989], [-73.85961281199982, 40.865512717999856], [-73.86156274099987, 40.86554946399989], [-73.86137924099984, 40.8713365119999], [-73.85661382499993, 40.87125016699991], [-73.85651292399994, 40.874217792999936], [-73.84792614099985, 40.87134223399991]]]}}, {\"id\": \"3\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3347314226583771, \"OBJECTID\": 4, \"Shape_Leng\": 0.0435665270921, \"Shape_Area\": 0.000111871946192, \"zone\": \"Alphabet City\", \"LocationID\": 4, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97177410965318, 40.72582128133705], [-73.97347975574512, 40.71886142911285], [-73.98382387299982, 40.72147287199987], [-73.97802697999985, 40.729433059999984], [-73.97177410965318, 40.72582128133705]]]}}, {\"id\": \"4\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4358974358974359, \"OBJECTID\": 5, \"Shape_Leng\": 0.0921464898574, \"Shape_Area\": 0.000497957489363, \"zone\": \"Arden Heights\", \"LocationID\": 5, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.17421738099989, 40.56256808599987], [-74.17185166499996, 40.56152092299991], [-74.18018524699995, 40.54837960899992], [-74.18903145199995, 40.53815934199989], [-74.19377169299999, 40.53583384699993], [-74.20314825999996, 40.55606178499986], [-74.20046730799992, 40.55623049099987], [-74.19317388399995, 40.56260434399992], [-74.18340311699991, 40.56540496199994], [-74.17421738099989, 40.56256808599987]]]}}, {\"id\": \"5\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5, \"OBJECTID\": 6, \"Shape_Leng\": 0.150490542523, \"Shape_Area\": 0.000606460984581, \"zone\": \"Arrochar/Fort Wadsworth\", \"LocationID\": 6, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.06367318899999, 40.60219816599994], [-74.0651709129999, 40.604213507999894], [-74.05685812390232, 40.608056357772554], [-74.05381069805436, 40.605912718852984], [-74.05222826954189, 40.599776102303636], [-74.06503332378131, 40.58899099843518], [-74.07043976099992, 40.59673474699988], [-74.08173342999991, 40.59215628599987], [-74.08376862299998, 40.595520956999934], [-74.08587511899998, 40.59589012099985], [-74.0814942209999, 40.599081990999935], [-74.08184498199992, 40.601535196999926], [-74.08947630599992, 40.60090432199991], [-74.09172051499995, 40.59922085699985], [-74.09271908099993, 40.60636222199992], [-74.09045495899996, 40.60909452599991], [-74.06367318899999, 40.60219816599994]]]}}, {\"id\": \"6\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5391674023138039, \"OBJECTID\": 7, \"Shape_Leng\": 0.107417171123, \"Shape_Area\": 0.000389787989274, \"zone\": \"Astoria\", \"LocationID\": 7, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90413637799996, 40.76752031699986], [-73.90202516599992, 40.767416783999884], [-73.90779405499987, 40.760517799999896], [-73.90631505099988, 40.75984387599992], [-73.91023124699991, 40.75297652299985], [-73.91763192900005, 40.75385784899993], [-73.92562743799996, 40.75213724499991], [-73.92415699999985, 40.753961141999845], [-73.93679091699993, 40.75988288099991], [-73.93398092100001, 40.76282598199987], [-73.93858976899998, 40.766817358999916], [-73.93493511099992, 40.76701302399987], [-73.93344468499993, 40.768817789999915], [-73.92425288799978, 40.76472456399995], [-73.92218946199995, 40.767267356999895], [-73.91977764799998, 40.7661521839999], [-73.91849336999996, 40.76773465699995], [-73.91995596899991, 40.768440064999865], [-73.91811686499992, 40.77002848599989], [-73.90413637799996, 40.76752031699986]]]}}, {\"id\": \"7\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4, \"OBJECTID\": 8, \"Shape_Leng\": 0.0275906911574, \"Shape_Area\": 2.6587716279e-05, \"zone\": \"Astoria Park\", \"LocationID\": 8, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92334041500001, 40.77512891199993], [-73.92827730816353, 40.77689897516217], [-73.91996458796496, 40.78263506876824], [-73.91796581399998, 40.78125629099988], [-73.92375761299998, 40.77654701199991], [-73.92334041500001, 40.77512891199993]]]}}, {\"id\": \"8\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5355450236966824, \"OBJECTID\": 9, \"Shape_Leng\": 0.0997840924705, \"Shape_Area\": 0.000338443803197, \"zone\": \"Auburndale\", \"LocationID\": 9, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.78502434699996, 40.761036515999855], [-73.77573836999991, 40.74332564699993], [-73.79672497900003, 40.738416642999894], [-73.79723084999992, 40.74169550699993], [-73.7944636839999, 40.74497002199991], [-73.79472264899997, 40.74759932999989], [-73.79729524699995, 40.74943505299987], [-73.79471901099991, 40.75064761699991], [-73.79493246199993, 40.75779802999994], [-73.79031517, 40.75782345099988], [-73.79076004599987, 40.76098576899991], [-73.79343991199988, 40.760764127999934], [-73.79317944599984, 40.77040317799988], [-73.78502434699996, 40.761036515999855]]]}}, {\"id\": \"9\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.40316240843308915, \"OBJECTID\": 10, \"Shape_Leng\": 0.0998394794152, \"Shape_Area\": 0.000435823818081, \"zone\": \"Baisley Park\", \"LocationID\": 10, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7832662499999, 40.68999429299992], [-73.77966850599998, 40.685377122999945], [-73.77699767599997, 40.68628268899992], [-73.77220275299989, 40.681610338999896], [-73.78025859499995, 40.6778409239999], [-73.78573730899993, 40.67238503499988], [-73.78808647399985, 40.6730096269999], [-73.78940999399993, 40.66684115799993], [-73.80155565300004, 40.6667533639999], [-73.80142836799988, 40.673666833999874], [-73.80576840199991, 40.682932390999916], [-73.79898209099989, 40.68485873699989], [-73.79995042799996, 40.68682771899991], [-73.79561486799987, 40.688656890999916], [-73.793908417, 40.68630469099985], [-73.7832662499999, 40.68999429299992]]]}}, {\"id\": \"10\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5905172413793104, \"OBJECTID\": 11, \"Shape_Leng\": 0.0792110389596, \"Shape_Area\": 0.00026452053504, \"zone\": \"Bath Beach\", \"LocationID\": 11, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00109809499993, 40.60303462599992], [-73.99559320799995, 40.600232221999875], [-74.00200394754667, 40.594132371137796], [-74.01041238886523, 40.60047518174621], [-74.01942760861903, 40.602871578425265], [-74.01549854999998, 40.606841523999925], [-74.01701910199995, 40.60765441699994], [-74.01050403599997, 40.613896610999866], [-74.00094408399997, 40.60863970999988], [-74.00493448899994, 40.60507440599987], [-74.00109809499993, 40.60303462599992]]]}}, {\"id\": \"11\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5267139229792637, \"OBJECTID\": 12, \"Shape_Leng\": 0.0366613013579, \"Shape_Area\": 4.15116236727e-05, \"zone\": \"Battery Park\", \"LocationID\": 12, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01565756599994, 40.70483308799993], [-74.01425861399996, 40.70448944299991], [-74.01376867799992, 40.7011432629999], [-74.014220631955, 40.70011149186829], [-74.0176479750685, 40.70349459987955], [-74.01565756599994, 40.70483308799993]]]}}, {\"id\": \"12\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.2875136828252874, \"OBJECTID\": 13, \"Shape_Leng\": 0.0502813228631, \"Shape_Area\": 0.000149358592917, \"zone\": \"Battery Park City\", \"LocationID\": 13, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01244109299991, 40.7190576729999], [-74.01668714699993, 40.704743941999865], [-74.01844770243774, 40.70416216823156], [-74.0174808615258, 40.704667487696156], [-74.01934254624483, 40.706093673029706], [-74.01777014254499, 40.71283457478916], [-74.01662424425285, 40.71215731899529], [-74.01632006627581, 40.71340798247826], [-74.01772496219284, 40.71307018162274], [-74.01671018605829, 40.718624176057965], [-74.01244109299991, 40.7190576729999]]]}}, {\"id\": \"13\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5293094883258818, \"OBJECTID\": 14, \"Shape_Leng\": 0.175213698053, \"Shape_Area\": 0.00138177826442, \"zone\": \"Bay Ridge\", \"LocationID\": 14, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.03407329297129, 40.64431393298185], [-74.02475437599998, 40.6381214179999], [-74.02108051399998, 40.64149093699994], [-74.02141038399992, 40.638584902999845], [-74.01514630699995, 40.63322193799993], [-74.01970491599991, 40.6221849649999], [-74.02754297299998, 40.61293718299988], [-74.02637609099996, 40.61245060299993], [-74.02710660599995, 40.61024395599991], [-74.02150602899994, 40.60782209899992], [-74.02247769499994, 40.60667421599987], [-74.02078541, 40.605673685999875], [-74.02139467299999, 40.60449399799991], [-74.01869135799996, 40.60360792399991], [-74.01942760861903, 40.602871578425265], [-74.0321315955131, 40.605901836264245], [-74.04038132338673, 40.61534178232843], [-74.04189123741789, 40.62406041492981], [-74.04108656611493, 40.63009852389527], [-74.03680662048858, 40.63898422154557], [-74.03876297920752, 40.639588258980524], [-74.03672484374877, 40.63914119020668], [-74.03571276150724, 40.64064015404647], [-74.03675840317513, 40.641611762713936], [-74.03407329297129, 40.64431393298185]]]}}, {\"id\": \"14\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.46503496503496505, \"OBJECTID\": 15, \"Shape_Leng\": 0.14433622262, \"Shape_Area\": 0.000925219395547, \"zone\": \"Bay Terrace/Fort Totten\", \"LocationID\": 15, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7774039129087, 40.79659824126783], [-73.77063124537418, 40.78846290603716], [-73.77426836058959, 40.78675665865473], [-73.77633728818715, 40.78848722575376], [-73.76820055460033, 40.77944417919097], [-73.76702392346498, 40.78008623415827], [-73.76677570373936, 40.77981245869834], [-73.76782207615058, 40.77881037615661], [-73.77553732199992, 40.777025597999916], [-73.77583958299994, 40.778612371999884], [-73.79565702099983, 40.773036085999905], [-73.79853009399987, 40.775190728999924], [-73.8037902289999, 40.77561011199992], [-73.80242791699989, 40.78606827399991], [-73.79491977600001, 40.78633880799987], [-73.79058298999988, 40.78819290999989], [-73.79179280401884, 40.789351616549254], [-73.79028202353197, 40.79031696339975], [-73.78174708763098, 40.79113340664446], [-73.78079182072646, 40.79403591986439], [-73.78325271025878, 40.79492032976596], [-73.7774039129087, 40.79659824126783]]]}}, {\"id\": \"15\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5345744680851063, \"OBJECTID\": 16, \"Shape_Leng\": 0.141291873771, \"Shape_Area\": 0.000871889446182, \"zone\": \"Bayside\", \"LocationID\": 16, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7685730499999, 40.77910542899991], [-73.75864264358073, 40.767357170035446], [-73.75377036482266, 40.76530463124142], [-73.75921689799992, 40.76432485599988], [-73.75800131799996, 40.76037986699988], [-73.76239510499991, 40.75949699799992], [-73.76184343999986, 40.75553726899989], [-73.75684127399991, 40.74932207699987], [-73.77573836999991, 40.74332564699993], [-73.78546383099984, 40.76259192499988], [-73.79565702099983, 40.773036085999905], [-73.77678363299991, 40.77840958299987], [-73.77553732199992, 40.777025597999916], [-73.7685730499999, 40.77910542899991]]]}}, {\"id\": \"16\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4699786696519069, \"OBJECTID\": 17, \"Shape_Leng\": 0.093522632948, \"Shape_Area\": 0.000322957654799, \"zone\": \"Bedford\", \"LocationID\": 17, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94306406899986, 40.701424434999886], [-73.94193078899983, 40.70072523399989], [-73.94438788599984, 40.70042452299993], [-73.94032794, 40.679889974999874], [-73.9533701749999, 40.680640507999904], [-73.95468418899998, 40.68724485399985], [-73.95956770199984, 40.686682554999955], [-73.96029281699998, 40.69034624999995], [-73.95541057999996, 40.69090829199994], [-73.95614239299985, 40.694579018999896], [-73.96015854699988, 40.69411730899989], [-73.96105100699992, 40.69832607899989], [-73.95701993199991, 40.69897391399995], [-73.95745736399992, 40.70082260299993], [-73.95381196900003, 40.703180979999914], [-73.95128819399989, 40.700922364999855], [-73.94705205299991, 40.70366394899985], [-73.94306406899986, 40.701424434999886]]]}}, {\"id\": \"17\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.646894236149972, \"OBJECTID\": 18, \"Shape_Leng\": 0.0697995498569, \"Shape_Area\": 0.000148850163948, \"zone\": \"Bedford Park\", \"LocationID\": 18, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88513907699999, 40.86638287399992], [-73.88889566099989, 40.86475415299988], [-73.89573924699978, 40.85813820699986], [-73.89883814599983, 40.859307307999906], [-73.897143798, 40.86244506499991], [-73.90107966999994, 40.86275638299992], [-73.8974032329998, 40.86747418999989], [-73.89438130299995, 40.866068850999966], [-73.88785196299986, 40.87211417599992], [-73.88937163799984, 40.87338229799993], [-73.88527683199987, 40.87921766899991], [-73.88146889699985, 40.868573645999916], [-73.88513907699999, 40.86638287399992]]]}}, {\"id\": \"18\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6982343499197432, \"OBJECTID\": 19, \"Shape_Leng\": 0.101824875452, \"Shape_Area\": 0.000546661094782, \"zone\": \"Bellerose\", \"LocationID\": 19, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.72339596299987, 40.750389075999855], [-73.72258571899978, 40.74885045099991], [-73.72010148299994, 40.74977568599991], [-73.71147804899985, 40.73250140099988], [-73.71049995381028, 40.72722708720663], [-73.72990512499977, 40.723272494999954], [-73.7314721509999, 40.727411794999924], [-73.74640108899996, 40.7312928389999], [-73.74672140400006, 40.73247715099991], [-73.73933151399987, 40.739474474999916], [-73.72829094199989, 40.74745895699992], [-73.727664654, 40.749654641999946], [-73.72339596299987, 40.750389075999855]]]}}, {\"id\": \"19\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5946188340807175, \"OBJECTID\": 20, \"Shape_Leng\": 0.0514401924362, \"Shape_Area\": 0.000134512633032, \"zone\": \"Belmont\", \"LocationID\": 20, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88386792099986, 40.8642908889999], [-73.88060896899991, 40.860357092999884], [-73.88311982600001, 40.848221091999925], [-73.88987871399985, 40.85207022099993], [-73.89109710400002, 40.856779813999886], [-73.89027555599993, 40.86110907299996], [-73.89177188799992, 40.86187171599992], [-73.88362518100003, 40.86725758799991], [-73.88267624699999, 40.86608914099984], [-73.88386792099986, 40.8642908889999]]]}}, {\"id\": \"20\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5815660685154975, \"OBJECTID\": 21, \"Shape_Leng\": 0.115973569062, \"Shape_Area\": 0.000380251345507, \"zone\": \"Bensonhurst East\", \"LocationID\": 21, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97418385499991, 40.609463501999905], [-73.97299433999983, 40.6088141419999], [-73.97291116900001, 40.599315803999964], [-73.97148350899987, 40.59717158499996], [-73.98035785499995, 40.59619154099992], [-73.97995443499985, 40.594073811999884], [-73.98601936799993, 40.594446048999906], [-73.98962150099992, 40.59098199299996], [-73.99923690396953, 40.588935328269024], [-73.99710730365224, 40.591390755943046], [-74.00004314944917, 40.59120428665461], [-73.9994830488618, 40.59296919994875], [-73.99530829599992, 40.59442758699989], [-73.99674706899995, 40.59564312599992], [-73.9868476259999, 40.60509723599994], [-73.98913191499994, 40.6064561369999], [-73.97963821799982, 40.61556454699991], [-73.97477658, 40.6126384749999], [-73.97418385499991, 40.609463501999905]]]}}, {\"id\": \"21\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5322386864237084, \"OBJECTID\": 22, \"Shape_Leng\": 0.126170229196, \"Shape_Area\": 0.000472062684608, \"zone\": \"Bensonhurst West\", \"LocationID\": 22, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99254973599997, 40.62427426799996], [-73.97963821799982, 40.61556454699991], [-73.98913191499994, 40.6064561369999], [-73.9868476259999, 40.60509723599994], [-73.99674706899995, 40.59564312599992], [-73.99530829599992, 40.59442758699989], [-74.00105748445709, 40.5925370050623], [-74.00200394804675, 40.594132371137796], [-73.99559320799995, 40.600232221999875], [-74.00493448899994, 40.60507440599987], [-74.00094408399997, 40.60863970999988], [-74.01050403599997, 40.613896610999866], [-73.99932090299998, 40.62465524699996], [-73.99771358999998, 40.62368026999996], [-73.99678847599995, 40.626826050999895], [-73.99254973599997, 40.62427426799996]]]}}, {\"id\": \"22\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.37398373983739835, \"OBJECTID\": 23, \"Shape_Leng\": 0.290556028962, \"Shape_Area\": 0.00219556576201, \"zone\": \"Bloomfield/Emerson Hill\", \"LocationID\": 23, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.19568609223377, 40.63501686464005], [-74.17459706299995, 40.622364010999874], [-74.16163010299996, 40.61793493599989], [-74.15018666499996, 40.60949781099991], [-74.14391816399994, 40.60798983699987], [-74.11742466500002, 40.609289008999916], [-74.11487058499988, 40.602136659999864], [-74.12059546299996, 40.601809757999874], [-74.12506115499997, 40.59818005599987], [-74.13091909399999, 40.598855047999905], [-74.13864458599996, 40.58906080499989], [-74.14790576499993, 40.58855634599993], [-74.16025756099997, 40.59534393799988], [-74.16189940599996, 40.59529877099993], [-74.16369888199995, 40.59225833899993], [-74.16360268199993, 40.593374198999854], [-74.16886362399997, 40.59472808699995], [-74.16719634899995, 40.602075311999926], [-74.17260229799992, 40.602971490999934], [-74.17822457299994, 40.5998280609999], [-74.17932277499995, 40.59676057899984], [-74.18730248699987, 40.588488221999896], [-74.185371048, 40.58745101499987], [-74.19423261399992, 40.58514680599991], [-74.19964981299998, 40.58226262599991], [-74.20048848899994, 40.5799497779999], [-74.20583829199992, 40.57984126099989], [-74.204646089432, 40.589285745465865], [-74.19751357718701, 40.59679898603677], [-74.20281628374593, 40.608270827967345], [-74.20244374449518, 40.61328469393197], [-74.2003834842897, 40.616428098243766], [-74.20163201104123, 40.623121565457346], [-74.2007872998309, 40.63034627446781], [-74.19568609223377, 40.63501686464005]]]}}, {\"id\": \"23\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.40467547011338717, \"OBJECTID\": 24, \"Shape_Leng\": 0.0469999619287, \"Shape_Area\": 6.07235737749e-05, \"zone\": \"Bloomingdale\", \"LocationID\": 24, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95953658899998, 40.798718525999895], [-73.96004456499999, 40.79804123499991], [-73.97287179090726, 40.8033561875739], [-73.97110765876137, 40.80579013958964], [-73.95817297099987, 40.800582540999876], [-73.95953658899998, 40.798718525999895]]]}}, {\"id\": \"24\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.38183540556542334, \"OBJECTID\": 25, \"Shape_Leng\": 0.0471458199319, \"Shape_Area\": 0.000124168267356, \"zone\": \"Boerum Hill\", \"LocationID\": 25, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98155298299992, 40.689146163999936], [-73.97804289599988, 40.68485661499987], [-73.98175642299985, 40.67931462399987], [-73.99490113599977, 40.6844302379999], [-73.99236366999985, 40.689690123999924], [-73.99054474599986, 40.689168104999936], [-73.98902944799994, 40.69212386099992], [-73.98155298299992, 40.689146163999936]]]}}, {\"id\": \"25\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.621963394342762, \"OBJECTID\": 26, \"Shape_Leng\": 0.12354780707, \"Shape_Area\": 0.000534039927626, \"zone\": \"Borough Park\", \"LocationID\": 26, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98331628499983, 40.6414786819999], [-73.98042057899988, 40.64238522699991], [-73.97911188699993, 40.63544038599991], [-73.9768886119999, 40.6356748619999], [-73.97705352899999, 40.62153252199987], [-73.97539380499998, 40.62076998699995], [-73.97785009299986, 40.617287605999906], [-73.97537335500002, 40.6157547219999], [-73.97335879399992, 40.616541714999904], [-73.97290326899996, 40.61415296799988], [-73.97517176999993, 40.61472418599991], [-73.97477658, 40.6126384749999], [-74.00702302499997, 40.63210999499996], [-74.00556490399993, 40.63308317599992], [-74.00735636699996, 40.6341636269999], [-73.99917564799993, 40.642025441999856], [-73.9955103369999, 40.63980966699993], [-73.98834986699991, 40.64456224399992], [-73.98331628499983, 40.6414786819999]]]}}, {\"id\": \"26\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5208333333333334, \"OBJECTID\": 27, \"Shape_Leng\": 0.202508808518, \"Shape_Area\": 0.00134088762746, \"zone\": \"Breezy Point/Fort Tilden/Riis Beach\", \"LocationID\": 27, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86522555399998, 40.57045847199989], [-73.86268391251235, 40.56650565604353], [-73.8949181726631, 40.5570787962983], [-73.90658574923901, 40.555710625377294], [-73.94073681682697, 40.54182008707755], [-73.93985107460422, 40.55456005374592], [-73.9261572327542, 40.56155260972308], [-73.91766081900093, 40.56282614968639], [-73.91197024752503, 40.56586941753247], [-73.90683894638656, 40.562856359344565], [-73.90121507272005, 40.563008460632346], [-73.89261013445609, 40.56858724728667], [-73.87780808995733, 40.56880063006316], [-73.86288301945429, 40.576059761127304], [-73.86361132999998, 40.573823056999835], [-73.86662812000002, 40.573057648999914], [-73.86522555399998, 40.57045847199989]]]}}, {\"id\": \"27\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.534238284627356, \"OBJECTID\": 28, \"Shape_Leng\": 0.097960782214, \"Shape_Area\": 0.000291203927662, \"zone\": \"Briarwood/Jamaica Hills\", \"LocationID\": 28, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79240413399991, 40.716193040999926], [-73.79303800099989, 40.71071107499988], [-73.79776810199988, 40.7091431879999], [-73.79870150699992, 40.71066337699988], [-73.80130625, 40.709742148999965], [-73.80302414799988, 40.70813965699997], [-73.80165725599993, 40.705761474999846], [-73.8056477309999, 40.70467172799994], [-73.80694856299993, 40.70736709999989], [-73.81201248599983, 40.70594494499993], [-73.81152856899989, 40.70227287499991], [-73.81637824499992, 40.70245196399987], [-73.82591945199992, 40.71598987599992], [-73.80494148499987, 40.715996230999956], [-73.79073248899982, 40.71920875699987], [-73.79240413399991, 40.716193040999926]]]}}, {\"id\": \"28\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6466431095406361, \"OBJECTID\": 29, \"Shape_Leng\": 0.0714083127733, \"Shape_Area\": 0.000201673837402, \"zone\": \"Brighton Beach\", \"LocationID\": 29, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96004798699995, 40.58326987199995], [-73.95414287899986, 40.58310613999989], [-73.95235846771321, 40.574274685845246], [-73.9589871897255, 40.57343065573189], [-73.95941778299985, 40.57517798299994], [-73.9688899589999, 40.57526123899986], [-73.96514385199995, 40.5911019159999], [-73.9606798409999, 40.591597582999945], [-73.96004798699995, 40.58326987199995]]]}}, {\"id\": \"29\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6956521739130435, \"OBJECTID\": 30, \"Shape_Leng\": 0.0945097669793, \"Shape_Area\": 0.000145862107626, \"zone\": \"Broad Channel\", \"LocationID\": 30, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82075892499992, 40.61523267899991], [-73.81652681189118, 40.61494701007998], [-73.81475287105245, 40.6086327967523], [-73.81739192103247, 40.60515812414338], [-73.82071206789055, 40.59517051496212], [-73.83465599507487, 40.59516845882606], [-73.83239364476839, 40.59780335319121], [-73.82806159687702, 40.59768040282441], [-73.8259798598142, 40.599472067181495], [-73.82465420648664, 40.5981492468515], [-73.82376612290614, 40.60004007091038], [-73.82146929866913, 40.59999372263383], [-73.82445218054455, 40.60079696284418], [-73.81961795381072, 40.610479041032136], [-73.82087841795061, 40.61225734590997], [-73.82161460094558, 40.60848031911391], [-73.8238246859766, 40.61149866495507], [-73.82075892499992, 40.61523267899991]]]}}, {\"id\": \"30\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.56353591160221, \"OBJECTID\": 31, \"Shape_Leng\": 0.0964245666516, \"Shape_Area\": 0.000333975927329, \"zone\": \"Bronx Park\", \"LocationID\": 31, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87094462199981, 40.85726608099992], [-73.86841539699994, 40.85778035799994], [-73.86990379599995, 40.85587302199991], [-73.86823466099995, 40.85091373699986], [-73.86889243299996, 40.848112321999906], [-73.87459175399982, 40.841520647999914], [-73.878171756, 40.84262443699987], [-73.8773292409999, 40.84398312499995], [-73.88311982600001, 40.848221091999925], [-73.88060896899991, 40.860357092999884], [-73.88394303099996, 40.86494818699985], [-73.87619224900003, 40.87047671099988], [-73.87094367500002, 40.87851076699998], [-73.86943471300005, 40.87812919899994], [-73.87094462199981, 40.85726608099992]]]}}, {\"id\": \"31\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6628099173553719, \"OBJECTID\": 32, \"Shape_Leng\": 0.05426721601, \"Shape_Area\": 0.000150879171971, \"zone\": \"Bronxdale\", \"LocationID\": 32, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85882515999995, 40.85806135699987], [-73.87104944999982, 40.857304836999965], [-73.87056192499989, 40.87151077799995], [-73.86137924099984, 40.8713365119999], [-73.86156274099987, 40.86554946399989], [-73.85961281199982, 40.865512717999856], [-73.85780378799993, 40.86303621699989], [-73.8566377469999, 40.8581191719999], [-73.85882515999995, 40.85806135699987]]]}}, {\"id\": \"32\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.33940239294350344, \"OBJECTID\": 33, \"Shape_Leng\": 0.0532702931967, \"Shape_Area\": 0.000147416802448, \"zone\": \"Brooklyn Heights\", \"LocationID\": 33, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99619227264343, 40.70337714093203], [-73.99087453800003, 40.70067311699993], [-73.99097187, 40.692548799999926], [-73.98902944799994, 40.69212386099992], [-73.99054474599986, 40.689168104999936], [-74.00110519399988, 40.692056594999954], [-74.0009586845849, 40.694069083791064], [-74.00301393814541, 40.69477784423956], [-74.00026872377784, 40.69496580257801], [-74.00043147340423, 40.69705246617853], [-73.99876458027717, 40.697120733093975], [-74.0010490305752, 40.697908236697415], [-73.9983776068576, 40.698063296146074], [-74.00001840433616, 40.699466048073575], [-73.99716054243916, 40.69979281264407], [-73.99813879899439, 40.701518788248414], [-73.99619227264343, 40.70337714093203]]]}}, {\"id\": \"33\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.343001751995328, \"OBJECTID\": 34, \"Shape_Leng\": 0.0657059323545, \"Shape_Area\": 0.000173946146651, \"zone\": \"Brooklyn Navy Yard\", \"LocationID\": 34, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.97232032119902, 40.70908288315216], [-73.9702621853228, 40.706887504284005], [-73.97018075572805, 40.70492927056683], [-73.96751516691087, 40.703437356878126], [-73.96929693837701, 40.705088331229625], [-73.96838933699995, 40.70682918699989], [-73.96217978199998, 40.70022070999989], [-73.963971858, 40.69637865499991], [-73.98054419199988, 40.69824651199988], [-73.97885303360711, 40.7060655352846], [-73.97713562625376, 40.703005581402515], [-73.97596096344841, 40.704715506440465], [-73.9769274918834, 40.70279782040038], [-73.97455023388345, 40.70159034430477], [-73.97573541084542, 40.69957044770558], [-73.97411513926478, 40.70130527141927], [-73.97239201397464, 40.700166322752274], [-73.97318676670886, 40.70166980367368], [-73.97078200587988, 40.69997866301123], [-73.97283512788735, 40.70272874074404], [-73.96948655218084, 40.70051906125818], [-73.97283889974949, 40.70334642860182], [-73.96898517428966, 40.70174708878653], [-73.97465569608048, 40.70607426299424], [-73.97254880057814, 40.70620449000379], [-73.9742361696854, 40.70802438612241], [-73.97109655489108, 40.705850056558255], [-73.97232032119902, 40.70908288315216]]], [[[-73.97892920137538, 40.705931589879214], [-73.97893110285521, 40.70592490623034], [-73.97906084872884, 40.705946028470365], [-73.97905767006142, 40.705952889050316], [-73.97892920137538, 40.705931589879214]]], [[[-73.98237340600002, 40.70554334999991], [-73.98242287179853, 40.70582205595453], [-73.98102390618274, 40.705898913893535], [-73.98230205183869, 40.705736979399674], [-73.98237340600002, 40.70554334999991]]]]}}, {\"id\": \"34\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5185020062416407, \"OBJECTID\": 35, \"Shape_Leng\": 0.085787918592, \"Shape_Area\": 0.000323825195932, \"zone\": \"Brownsville\", \"LocationID\": 35, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90105329800004, 40.66212482099994], [-73.89880698499995, 40.65740666199988], [-73.90784346399994, 40.65459152999988], [-73.90855790499995, 40.65209593799989], [-73.91012255699995, 40.655851770999945], [-73.91236458899995, 40.65551356499985], [-73.9195947979998, 40.66199567899985], [-73.92008167099986, 40.659857632999866], [-73.92637970499992, 40.6655148919999], [-73.90347422300006, 40.67550668299985], [-73.90105329800004, 40.66212482099994]]]}}, {\"id\": \"35\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.38390326414921827, \"OBJECTID\": 36, \"Shape_Leng\": 0.0870507867063, \"Shape_Area\": 0.000247666139269, \"zone\": \"Bushwick North\", \"LocationID\": 36, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91254913799993, 40.70387341999989], [-73.912904041, 40.7023618909999], [-73.91067882699998, 40.701045968999914], [-73.91180820099989, 40.69993800299986], [-73.9042601839999, 40.69570037099989], [-73.90579597099993, 40.69412715499987], [-73.90123290699994, 40.69144227899997], [-73.904055772, 40.68854627799986], [-73.93115533899987, 40.70377233499987], [-73.93269784899994, 40.70317039099989], [-73.93391870799988, 40.70748702599992], [-73.92189184700005, 40.709396096999896], [-73.91254913799993, 40.70387341999989]]]}}, {\"id\": \"36\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.40281935698473326, \"OBJECTID\": 37, \"Shape_Leng\": 0.142810040466, \"Shape_Area\": 0.000452062644782, \"zone\": \"Bushwick South\", \"LocationID\": 37, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93312548700004, 40.70470086199992], [-73.93269784899994, 40.70317039099989], [-73.93115533899987, 40.70377233499987], [-73.90482499699988, 40.688808569999956], [-73.90180467199995, 40.6907662979999], [-73.90116154999988, 40.68787793499991], [-73.89646625099995, 40.68233642199988], [-73.89704472199993, 40.68056181599996], [-73.90047001100004, 40.679535243999936], [-73.90262262299991, 40.6806645029999], [-73.9040463979999, 40.67922059799985], [-73.93856854799981, 40.698848135999874], [-73.94394947299996, 40.698221278999924], [-73.94438788599984, 40.70042452299993], [-73.94042057899993, 40.701076597999865], [-73.94103009699987, 40.7046361769999], [-73.94273448399997, 40.70447330599988], [-73.94378799599994, 40.710900378999945], [-73.93480281900003, 40.713372495999884], [-73.93229708600003, 40.70774944699988], [-73.93391870799988, 40.70748702599992], [-73.93312548700004, 40.70470086199992]]]}}, {\"id\": \"37\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.500669344042838, \"OBJECTID\": 38, \"Shape_Leng\": 0.0832175685234, \"Shape_Area\": 0.000327392684821, \"zone\": \"Cambria Heights\", \"LocationID\": 38, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.73624941199995, 40.702536977999884], [-73.72677759516957, 40.7030592911994], [-73.72586264191175, 40.683241080973865], [-73.7475517539999, 40.68934733599984], [-73.7435928609999, 40.69621532499986], [-73.74618157999997, 40.696121925999904], [-73.74773012199991, 40.69955313699991], [-73.74500119100004, 40.70026221299988], [-73.74590871000004, 40.70228727399988], [-73.74109293999989, 40.7034810629999], [-73.73624941199995, 40.702536977999884]]]}}, {\"id\": \"38\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.49310027598896045, \"OBJECTID\": 39, \"Shape_Leng\": 0.132369620757, \"Shape_Area\": 0.00090028293862, \"zone\": \"Canarsie\", \"LocationID\": 39, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.89121853499988, 40.64998769899987], [-73.88805587182881, 40.64511978725729], [-73.87714434884276, 40.63597649800373], [-73.88364451728226, 40.630943312138584], [-73.88263248433404, 40.62827427312689], [-73.88369231571497, 40.62755937391403], [-73.88752673706196, 40.628331824482444], [-73.89577374598754, 40.62251907138477], [-73.91860649599992, 40.632090414999894], [-73.91995062999999, 40.64470762999989], [-73.91830152299998, 40.645812262999925], [-73.90944846399994, 40.65152159899992], [-73.90398200599991, 40.64662856799988], [-73.89586154900002, 40.65188052699987], [-73.89761013299987, 40.65345381599992], [-73.89354677099988, 40.65520271399988], [-73.89000589199988, 40.651148696999925], [-73.89121853499988, 40.64998769899987]]], [[[-73.88833929256269, 40.64671022891759], [-73.88849720193863, 40.64675678600008], [-73.88853291212868, 40.646861882230674], [-73.88846787961647, 40.646899734025276], [-73.88833929256269, 40.64671022891759]]]]}}, {\"id\": \"39\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.351327128576353, \"OBJECTID\": 40, \"Shape_Leng\": 0.0518506648762, \"Shape_Area\": 0.000108949211644, \"zone\": \"Carroll Gardens\", \"LocationID\": 40, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.998192111, 40.68228732999989], [-73.99620846299992, 40.681723424999916], [-73.99490113599977, 40.6844302379999], [-73.98722413899985, 40.68144730199988], [-73.99069861899994, 40.67580741899994], [-73.99506451099984, 40.67584047499984], [-73.9986188799999, 40.67183017199989], [-74.00329231299999, 40.67964328799992], [-74.00064737999996, 40.685253946999914], [-73.99908066099994, 40.68481599199985], [-74.00004865199998, 40.682803355999965], [-73.998192111, 40.68228732999989]]]}}, {\"id\": \"40\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4468749859213306, \"OBJECTID\": 41, \"Shape_Leng\": 0.052793109453, \"Shape_Area\": 0.000143093037737, \"zone\": \"Central Harlem\", \"LocationID\": 41, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94773985499985, 40.809599720999884], [-73.94177140199992, 40.807088999999955], [-73.94523965199991, 40.80513601399989], [-73.94613131, 40.80387735799992], [-73.94459751299998, 40.803228144999906], [-73.94922045699984, 40.796909858999896], [-73.95964685399987, 40.80115642299993], [-73.954966572, 40.81006455499988], [-73.9535757779999, 40.8094766979999], [-73.95210125199992, 40.811442853999935], [-73.94773985499985, 40.809599720999884]]]}}, {\"id\": \"41\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.528057639104641, \"OBJECTID\": 42, \"Shape_Leng\": 0.0927092274616, \"Shape_Area\": 0.000263896952605, \"zone\": \"Central Harlem North\", \"LocationID\": 42, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93436121591056, 40.83620061961779], [-73.93383089287109, 40.819521063761556], [-73.93901753299998, 40.81085655699991], [-73.93580780199986, 40.80949763799987], [-73.93854407899988, 40.80572965299991], [-73.95210125199992, 40.811442853999935], [-73.94607828700005, 40.821263215999906], [-73.93924885599988, 40.82829635099989], [-73.93864026699997, 40.82973887399988], [-73.9401708989999, 40.83038439099996], [-73.93868316299985, 40.832800788999904], [-73.93436121591056, 40.83620061961779]]]}}, {\"id\": \"42\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.45217191296412174, \"OBJECTID\": 43, \"Shape_Leng\": 0.0997386183576, \"Shape_Area\": 0.000379662912054, \"zone\": \"Central Park\", \"LocationID\": 43, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97255352299985, 40.76489773199991], [-73.98164804599995, 40.76843632199994], [-73.95817297099987, 40.800582540999876], [-73.94922045699984, 40.796909858999896], [-73.97255352299985, 40.76489773199991]]]}}, {\"id\": \"43\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.43243243243243246, \"OBJECTID\": 44, \"Shape_Leng\": 0.235688967594, \"Shape_Area\": 0.00194465649192, \"zone\": \"Charleston/Tottenville\", \"LocationID\": 44, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.21220034099994, 40.556443429999874], [-74.21784397899994, 40.554432717999894], [-74.22131562699998, 40.55025307399992], [-74.21776941599994, 40.54112662999992], [-74.21800676499996, 40.53229858999995], [-74.21354593399995, 40.521235715999886], [-74.22379595899992, 40.518196345999876], [-74.22690993299994, 40.5191713979999], [-74.22159300278173, 40.50250043822093], [-74.23114494958575, 40.50185121588201], [-74.23988017293566, 40.49757362656005], [-74.24917156868965, 40.49656729705339], [-74.25335538022304, 40.50041319788582], [-74.25554269163935, 40.50783774249614], [-74.25056007272197, 40.516106741535], [-74.24921528676096, 40.51508486129925], [-74.24994158808036, 40.516137488190985], [-74.24854589291671, 40.51616991714839], [-74.24940770789283, 40.516980831423645], [-74.24851449056031, 40.517475626265046], [-74.24644039651318, 40.515965028508525], [-74.2456347316994, 40.518075580745766], [-74.23991937852388, 40.52004699652993], [-74.24293235036517, 40.52122716113276], [-74.24396500758859, 40.52490536633781], [-74.24150733322897, 40.531041342795135], [-74.24204570286014, 40.53434637705037], [-74.24533651176053, 40.53690599443462], [-74.24560205698324, 40.54094959457726], [-74.24803463735688, 40.54309324044144], [-74.24336346552776, 40.54786513523582], [-74.24039182810051, 40.547663310173775], [-74.23641623448965, 40.55050862684339], [-74.23641559733107, 40.55232806634515], [-74.23334188482498, 40.55249971061369], [-74.2287242997478, 40.5562970171392], [-74.22062175949124, 40.55589882100512], [-74.21974356930723, 40.55461267207239], [-74.21921716370927, 40.55579696987855], [-74.21859325782798, 40.55467421024778], [-74.21870767560165, 40.55604101620115], [-74.21775342391726, 40.55500528477618], [-74.2056960519999, 40.55851600599987], [-74.21220034099994, 40.556443429999874]]]}}, {\"id\": \"44\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4297731185765409, \"OBJECTID\": 45, \"Shape_Leng\": 0.0459068626235, \"Shape_Area\": 9.13594688556e-05, \"zone\": \"Chinatown\", \"LocationID\": 45, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99750445299988, 40.71406913199995], [-73.99256242199989, 40.71438807699996], [-73.99190824271031, 40.70953292664334], [-73.99919451154429, 40.70794737667571], [-74.0058917509999, 40.71200715999989], [-73.99995620900002, 40.71801709499987], [-73.99605872699998, 40.7162316399999], [-73.99750445299988, 40.71406913199995]]]}}, {\"id\": \"45\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6166666666666667, \"OBJECTID\": 46, \"Shape_Leng\": 0.134475429879, \"Shape_Area\": 0.000926391677672, \"zone\": \"City Island\", \"LocationID\": 46, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.78452431937963, 40.86047706314786], [-73.78519664387113, 40.85882702363741], [-73.78743186194176, 40.858935245614425], [-73.78685886773299, 40.86004505783956], [-73.78452431937963, 40.86047706314786]]], [[[-73.76783205637173, 40.854442205742075], [-73.76938529139694, 40.85253613399675], [-73.76959674311465, 40.84759451909819], [-73.76789492672636, 40.84528943351961], [-73.76901389181107, 40.84509734272193], [-73.77157218833237, 40.8475000585803], [-73.7708892022117, 40.84975177133807], [-73.77296356611284, 40.8521592539586], [-73.77173060411904, 40.852476831319095], [-73.77296285334867, 40.853800062246464], [-73.77219223055407, 40.85998663859697], [-73.76947305423998, 40.85933013666858], [-73.76533243995276, 40.85504359512487], [-73.76783205637173, 40.854442205742075]]], [[[-73.79018745194999, 40.8586099173861], [-73.788043429758, 40.85755991768708], [-73.7884773265335, 40.8544005205322], [-73.78690734702697, 40.854063972299144], [-73.78796264288079, 40.8534747209355], [-73.78338158610377, 40.85158673391434], [-73.78420047134377, 40.8500606442406], [-73.78154449088599, 40.84940043464333], [-73.78267467078706, 40.84813967294664], [-73.78097820938946, 40.8484583479135], [-73.78263740397888, 40.848072649245836], [-73.78148431950568, 40.8478821938486], [-73.78216393435586, 40.84663633778076], [-73.780403269586, 40.846931134166056], [-73.7833653613534, 40.84444472298169], [-73.78348428748797, 40.84367854599843], [-73.78290650949755, 40.84393612584013], [-73.78052736048552, 40.84452514033816], [-73.7799270374445, 40.843883081355834], [-73.78347481249244, 40.843654738109784], [-73.78288778301486, 40.84249482884962], [-73.78022773872677, 40.84377818129771], [-73.78286499304913, 40.84247810188268], [-73.78059735822595, 40.84278659994098], [-73.78216624418964, 40.84168914411004], [-73.7804257718152, 40.842494394206874], [-73.78019599955293, 40.84204621097616], [-73.78185048197766, 40.841653946394324], [-73.7807975292347, 40.84125614604844], [-73.78282260139727, 40.836338155887844], [-73.78485009222643, 40.83747048917844], [-73.78563997845448, 40.83706443458537], [-73.78583067964128, 40.83720998000451], [-73.7857350929868, 40.83923423177581], [-73.78815921166512, 40.84049077856704], [-73.79179343625451, 40.846731672329284], [-73.78951371516173, 40.85130077171307], [-73.79311837183832, 40.851578295240834], [-73.79042053780161, 40.85302297821258], [-73.79281511966936, 40.852494819308916], [-73.79080758729923, 40.85319906006435], [-73.79287610048584, 40.8529441813284], [-73.79105721254825, 40.85391841242664], [-73.79245816844812, 40.85360070875487], [-73.79107002765942, 40.85394495666465], [-73.79253714604873, 40.85384932781285], [-73.79129282160633, 40.854584025976855], [-73.79201712228527, 40.85617694860626], [-73.79270523985137, 40.85619232890572], [-73.79268768222656, 40.856335218720375], [-73.79315700296156, 40.85635091107476], [-73.7931468608596, 40.856467828471146], [-73.79018745194999, 40.8586099173861]]], [[[-73.78283291447852, 40.85587030844573], [-73.78302371522504, 40.85509276666495], [-73.78394699722007, 40.85563043752662], [-73.78343487501385, 40.856362865958886], [-73.78283291447852, 40.85587030844573]]], [[[-73.78061730829718, 40.855735175810025], [-73.78090476851291, 40.8549681331623], [-73.7815638969994, 40.855001286433904], [-73.78122782332017, 40.856087789162764], [-73.78061730829718, 40.855735175810025]]], [[[-73.78833349834521, 40.834667129759346], [-73.78931223606624, 40.83446488655343], [-73.78951019872322, 40.835364042525754], [-73.78845700015209, 40.835309914315395], [-73.78833349834521, 40.834667129759346]]]]}}, {\"id\": \"46\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6623536660505237, \"OBJECTID\": 47, \"Shape_Leng\": 0.0898275563294, \"Shape_Area\": 0.000163198117339, \"zone\": \"Claremont/Bathgate\", \"LocationID\": 47, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89090203399996, 40.854116885999865], [-73.89431774099977, 40.851891310999854], [-73.89617712999998, 40.8466583209999], [-73.89290102299996, 40.84626832999992], [-73.89537836199995, 40.842938778999965], [-73.89666161699988, 40.84352460699992], [-73.90553116799988, 40.82910413299989], [-73.91049010699992, 40.830737372999884], [-73.90538357999992, 40.839768982999885], [-73.90483121999989, 40.84480034199987], [-73.901344046, 40.84442875099994], [-73.89106280699994, 40.8615294119999], [-73.89090203399996, 40.854116885999865]]]}}, {\"id\": \"47\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.40902683820441527, \"OBJECTID\": 48, \"Shape_Leng\": 0.0437467441431, \"Shape_Area\": 9.42538425377e-05, \"zone\": \"Clinton East\", \"LocationID\": 48, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99117738199989, 40.75532984899992], [-73.99730671499998, 40.757110506999894], [-73.98806289599996, 40.769790799999925], [-73.98236545099999, 40.7673921529999], [-73.99117738199989, 40.75532984899992]]]}}, {\"id\": \"48\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.40843880823561096, \"OBJECTID\": 49, \"Shape_Leng\": 0.0752900183427, \"Shape_Area\": 0.000203185525471, \"zone\": \"Clinton Hill\", \"LocationID\": 49, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96062056499981, 40.69632013999993], [-73.96015854699988, 40.69411730899989], [-73.95614239299985, 40.694579018999896], [-73.95541057999996, 40.69090829199994], [-73.96029281699998, 40.69034624999995], [-73.95956770199984, 40.686682554999955], [-73.95468418899998, 40.68724485399985], [-73.95328187399996, 40.68020003099985], [-73.95529361599996, 40.679203150999825], [-73.96868961400001, 40.68200525799988], [-73.97013154199992, 40.68950630799986], [-73.96910362499993, 40.68946752199986], [-73.9693945009999, 40.696080388999846], [-73.9645507049999, 40.696107591999855], [-73.96105100699992, 40.69832607899989], [-73.96062056499981, 40.69632013999993]]]}}, {\"id\": \"49\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.37405203993576447, \"OBJECTID\": 50, \"Shape_Leng\": 0.0557479867536, \"Shape_Area\": 0.000173249283326, \"zone\": \"Clinton West\", \"LocationID\": 50, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99667705784736, 40.77351005394221], [-73.98806289599996, 40.769790799999925], [-73.99547450199992, 40.759626856999894], [-74.00375735018837, 40.763582241820224], [-74.0009730802201, 40.763420271725614], [-74.00311291631381, 40.76474559615368], [-73.99915098087908, 40.76420525958265], [-74.0020245433034, 40.76589326657037], [-73.99857638530527, 40.76499860727083], [-74.00157335385273, 40.766977192210646], [-73.9980667590535, 40.76587188113572], [-73.99737808447954, 40.76682338637958], [-74.00064950530414, 40.76824523610559], [-73.99715588949206, 40.76712869617278], [-73.9964571540504, 40.76809881579563], [-73.99973590247859, 40.76950617733152], [-73.99707176039315, 40.768732643576804], [-73.9987620691431, 40.7708250567092], [-73.99631327022087, 40.76979883633644], [-73.99493501658995, 40.77146814657798], [-73.99667705784736, 40.77351005394221]]]}}, {\"id\": \"50\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5246995994659546, \"OBJECTID\": 51, \"Shape_Leng\": 0.0953613442277, \"Shape_Area\": 0.000395756553505, \"zone\": \"Co-Op City\", \"LocationID\": 51, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82091617199997, 40.86891693399989], [-73.81578764099999, 40.8657580609999], [-73.82037552999996, 40.86103300499993], [-73.82699653199998, 40.85908778599986], [-73.83315960399985, 40.86831975599994], [-73.83565766499987, 40.86751275599992], [-73.83829221499984, 40.87295133899995], [-73.83924788999988, 40.876618737999905], [-73.8381484219999, 40.876953298999915], [-73.84324113799987, 40.879108912999904], [-73.82387140599998, 40.88778362799996], [-73.82077408699998, 40.886710602999884], [-73.81992586499996, 40.88211933299987], [-73.82329599399986, 40.87309741799988], [-73.82091617199997, 40.86891693399989]]]}}, {\"id\": \"51\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.33141927356707956, \"OBJECTID\": 52, \"Shape_Leng\": 0.0289076710093, \"Shape_Area\": 4.52027509603e-05, \"zone\": \"Cobble Hill\", \"LocationID\": 52, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99588110299992, 40.68239495399995], [-74.00004865199998, 40.682803355999965], [-73.99908066099994, 40.68481599199985], [-74.00064737999996, 40.685253946999914], [-73.99776924399993, 40.691194697999954], [-73.99236366999985, 40.689690123999924], [-73.99588110299992, 40.68239495399995]]]}}, {\"id\": \"52\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6158475426278837, \"OBJECTID\": 53, \"Shape_Leng\": 0.161500913385, \"Shape_Area\": 0.000947530980821, \"zone\": \"College Point\", \"LocationID\": 53, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.83839013354552, 40.798304872075995], [-73.83919004727183, 40.7977772664477], [-73.84025860809128, 40.79758885843212], [-73.83840083088981, 40.79675525865344], [-73.83663288911686, 40.79251637576927], [-73.84067828300006, 40.79390789199993], [-73.84081532599994, 40.7915048849999], [-73.83895676099984, 40.79143843099996], [-73.84008157499997, 40.78785153099992], [-73.83824429099994, 40.78778591599995], [-73.83869921299997, 40.781676956999874], [-73.82513740299993, 40.78169970599994], [-73.83000011099983, 40.77373240699996], [-73.83913189281712, 40.765991105784146], [-73.83975387982034, 40.767120445440874], [-73.84491332846012, 40.76552838817853], [-73.84780575104425, 40.76677294817163], [-73.84977407793218, 40.769936012942956], [-73.8486715286884, 40.770761572902536], [-73.85130108374476, 40.77180636079771], [-73.84897619867189, 40.77356166795698], [-73.84895559325399, 40.77796167373135], [-73.84963631633588, 40.77948192035157], [-73.85094705724252, 40.77906915543026], [-73.85148755480326, 40.77899297861757], [-73.84981720066014, 40.77963034805418], [-73.85232086242219, 40.77916928470371], [-73.85238832778414, 40.7795289023869], [-73.84982102259761, 40.779651339303875], [-73.84955315550077, 40.779990415384006], [-73.85169028085423, 40.779867508478176], [-73.85167099162219, 40.780035228878106], [-73.84949381208321, 40.780065828114246], [-73.85093646396534, 40.781119575156005], [-73.8492525818285, 40.782264775014035], [-73.85528242786589, 40.78197538434999], [-73.8595236950597, 40.785561954252664], [-73.85284230133867, 40.78821137173992], [-73.85480194733185, 40.788619154725914], [-73.85265830693045, 40.79110642061034], [-73.85377854910695, 40.79242127400702], [-73.8526309825624, 40.79494485424474], [-73.84906160835088, 40.79336261029852], [-73.84855988738983, 40.79549506021838], [-73.84354668938991, 40.79491394401049], [-73.84225959368331, 40.79558496893736], [-73.8428667525936, 40.79692222907191], [-73.84158721174394, 40.795401069221235], [-73.84041517975831, 40.7976784372587], [-73.83839013354552, 40.798304872075995]]], [[[-73.8678840747076, 40.78736103669457], [-73.86813032000018, 40.78723277784419], [-73.8692349173292, 40.78596991495815], [-73.8680991225789, 40.78745866181839], [-73.8678840747076, 40.78736103669457]]]]}}, {\"id\": \"53\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.33837959069245865, \"OBJECTID\": 54, \"Shape_Leng\": 0.0762192448554, \"Shape_Area\": 0.000132452130536, \"zone\": \"Columbia Street\", \"LocationID\": 54, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-74.00174362072502, 40.69240674985266], [-74.00400795971088, 40.693208199632345], [-74.00127014951309, 40.69329606183691], [-74.00174362072502, 40.69240674985266]]], [[[-74.00174362072502, 40.69240674985266], [-73.99776924399993, 40.691194697999954], [-74.00329231299999, 40.67964328799992], [-74.00596041099988, 40.683362870999936], [-74.00485241699997, 40.68566813099993], [-74.00783293766679, 40.68738505516274], [-74.00459270521766, 40.688215222986614], [-74.00636461225261, 40.68966966863697], [-74.00382038919723, 40.688929644699236], [-74.0053420962491, 40.6910916029778], [-74.0009615685494, 40.69012876835003], [-74.00479415294552, 40.69176162037443], [-74.00174362072502, 40.69240674985266]]], [[[-74.01092841300002, 40.684491472999824], [-74.01217596614636, 40.68409518562848], [-74.00816320571415, 40.68617364485845], [-74.0086007239999, 40.68590956499989], [-74.01092841300002, 40.684491472999824]]]]}}, {\"id\": \"54\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5935613682092555, \"OBJECTID\": 55, \"Shape_Leng\": 0.149956807524, \"Shape_Area\": 0.000736274713586, \"zone\": \"Coney Island\", \"LocationID\": 55, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98671582299981, 40.58456849999991], [-73.97742525299988, 40.58290659299994], [-73.96661531499983, 40.58457103699992], [-73.9688899589999, 40.57526123899986], [-73.95941778299985, 40.57517798299994], [-73.95939182223685, 40.57391210937612], [-73.98367270153267, 40.57140705768737], [-73.98336058039273, 40.56952999448666], [-73.98375790448641, 40.571396227250865], [-74.00208547639728, 40.569585983985014], [-74.00303186164808, 40.57218559475904], [-74.01115668729254, 40.57416676185885], [-74.01302222952468, 40.577804459633505], [-74.0111737229432, 40.58028397882522], [-74.00604734235807, 40.58198365303581], [-73.98814208290418, 40.57886596371379], [-73.98604509916208, 40.58172068515157], [-73.98800973376738, 40.579670687155776], [-73.98995504960881, 40.58066166976562], [-73.98671582299981, 40.58456849999991]]]}}, {\"id\": \"55\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5883275261324041, \"OBJECTID\": 56, \"Shape_Leng\": 0.0568478126677, \"Shape_Area\": 0.000180907844436, \"zone\": \"Corona\", \"LocationID\": 56, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85360568399982, 40.74749682799991], [-73.85011556599989, 40.74072356299995], [-73.84750820299992, 40.73900780699995], [-73.86462193299997, 40.73428275699993], [-73.86790963899986, 40.74493371599989], [-73.85442575299999, 40.748851725999884], [-73.85360568399982, 40.74749682799991]]]}}, {\"id\": \"56\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5883275261324041, \"OBJECTID\": 57, \"Shape_Leng\": 0.0192705048557, \"Shape_Area\": 1.80259807917e-05, \"zone\": \"Corona\", \"LocationID\": 56, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8513110499998, 40.74984337599988], [-73.85442575299999, 40.748851725999884], [-73.85579909099987, 40.75440952799994], [-73.85312635499996, 40.75510934299989], [-73.8513110499998, 40.74984337599988]]]}}, {\"id\": \"57\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.32941176470588235, \"OBJECTID\": 58, \"Shape_Leng\": 0.0598554094851, \"Shape_Area\": 0.000204980931361, \"zone\": \"Country Club\", \"LocationID\": 58, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.81561382067721, 40.84818955758243], [-73.81602685114903, 40.846400458770034], [-73.81464594697542, 40.84721961352901], [-73.81570353189448, 40.84629134042384], [-73.81368634738199, 40.84727432550905], [-73.81356372861671, 40.847151411278524], [-73.8150262760428, 40.845499995058404], [-73.81284939958525, 40.84631879949822], [-73.81324326350918, 40.84523590835256], [-73.81310050626846, 40.84517241453328], [-73.8128770359758, 40.8451346679459], [-73.81315597044244, 40.84521210037631], [-73.81285536957633, 40.845186285461466], [-73.81241445328128, 40.846559380748744], [-73.81227790900483, 40.846543458279285], [-73.8128577433794, 40.845129544589135], [-73.81254457815619, 40.84497959800826], [-73.81169971823665, 40.84668170081364], [-73.81266141349049, 40.84420764950083], [-73.81254909120709, 40.84495227514607], [-73.8136558066588, 40.8450591224657], [-73.81414462049185, 40.84481817326274], [-73.81437458252545, 40.8430889259994], [-73.81569792948075, 40.8443864281129], [-73.81823722746202, 40.844355041712625], [-73.81473345817534, 40.8413757283108], [-73.81606068217324, 40.83603019622843], [-73.82601252600003, 40.83402077799985], [-73.82624132999996, 40.84299359399991], [-73.82517999, 40.842700058999874], [-73.82495697699986, 40.846105444999864], [-73.82048405599983, 40.84850102599995], [-73.81507165904237, 40.84913361979451], [-73.81561382067721, 40.84818955758243]]]}}, {\"id\": \"58\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6052631578947368, \"OBJECTID\": 59, \"Shape_Leng\": 0.0377948070893, \"Shape_Area\": 6.28765230648e-05, \"zone\": \"Crotona Park\", \"LocationID\": 59, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88941724899998, 40.839422732999836], [-73.88659802299989, 40.839279049999895], [-73.89697220299995, 40.8343452539999], [-73.9012157839999, 40.8354101409999], [-73.89666161699988, 40.84352460699992], [-73.88941724899998, 40.839422732999836]]]}}, {\"id\": \"59\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6139359698681732, \"OBJECTID\": 60, \"Shape_Leng\": 0.0786482878226, \"Shape_Area\": 0.000161833284912, \"zone\": \"Crotona Park East\", \"LocationID\": 60, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87823856799989, 40.839985917999854], [-73.87277204699981, 40.83975128199995], [-73.87379590199993, 40.837445132999896], [-73.88005188499987, 40.83470205899989], [-73.88390094800003, 40.82980281599986], [-73.8843985489999, 40.822965575999866], [-73.88753429599987, 40.82250933899995], [-73.88601262699991, 40.828238530999926], [-73.88761307899992, 40.8293374189999], [-73.89717220999982, 40.830368832999966], [-73.89697220299995, 40.8343452539999], [-73.89408527499982, 40.8362973469999], [-73.88730373200002, 40.83812719099994], [-73.88618432099992, 40.840105043999884], [-73.88200384100001, 40.837443781999895], [-73.880061996, 40.840162234999845], [-73.87823856799989, 40.839985917999854]]]}}, {\"id\": \"60\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.43763165683048694, \"OBJECTID\": 61, \"Shape_Leng\": 0.117310808097, \"Shape_Area\": 0.000510772092295, \"zone\": \"Crown Heights North\", \"LocationID\": 61, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92916130699993, 40.68343632399992], [-73.92281374299995, 40.68341442999991], [-73.92164666399992, 40.67886992299994], [-73.92275257199998, 40.667097332999894], [-73.92872204599985, 40.66449556899988], [-73.93698362399992, 40.665535507999955], [-73.93674946999998, 40.66812737099991], [-73.96131877999991, 40.67140667599988], [-73.96007428300001, 40.67487807099994], [-73.96318238899983, 40.675529503999876], [-73.96134655899995, 40.6804635239999], [-73.95529361599996, 40.679203150999825], [-73.9533701749999, 40.680640507999904], [-73.93435066499993, 40.67956340799991], [-73.93500821699992, 40.68276376799985], [-73.92916130699993, 40.68343632399992]]]}}, {\"id\": \"61\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4873375699516267, \"OBJECTID\": 62, \"Shape_Leng\": 0.0822593359838, \"Shape_Area\": 0.000158238169004, \"zone\": \"Crown Heights South\", \"LocationID\": 62, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92916752199997, 40.664299531999895], [-73.94282892199992, 40.6628474859999], [-73.94269628999989, 40.664055128999934], [-73.94546293699983, 40.66422610399989], [-73.96095595499996, 40.66328505499992], [-73.96258784899996, 40.671711596999884], [-73.93674946999998, 40.66812737099991], [-73.93698362399992, 40.665535507999955], [-73.92916752199997, 40.664299531999895]]]}}, {\"id\": \"62\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4685754189944134, \"OBJECTID\": 63, \"Shape_Leng\": 0.119100721883, \"Shape_Area\": 0.000353190910142, \"zone\": \"Cypress Hills\", \"LocationID\": 63, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86650407899994, 40.684474051999885], [-73.86602666799992, 40.681918051999936], [-73.86410096700003, 40.68237284999988], [-73.86234580499983, 40.67916478599994], [-73.86106294999989, 40.67434293499984], [-73.87231018000001, 40.67775574699987], [-73.87531627599998, 40.68011098399988], [-73.88061377799994, 40.679110031999876], [-73.88116730300005, 40.68135366299988], [-73.9000462609999, 40.676893089999936], [-73.9040463979999, 40.67922059799985], [-73.90262262299991, 40.6806645029999], [-73.89786299899986, 40.67987501099987], [-73.89417463299992, 40.6852832479999], [-73.89252316799995, 40.683424532999865], [-73.88962787599998, 40.684236453999894], [-73.87402053199992, 40.69419129499995], [-73.86891704399994, 40.69515042299987], [-73.86650407899994, 40.684474051999885]]]}}, {\"id\": \"65\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3268301440775595, \"OBJECTID\": 66, \"Shape_Leng\": 0.0546334593634, \"Shape_Area\": 0.000108378855948, \"zone\": \"DUMBO/Vinegar Hill\", \"LocationID\": 66, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97906084869959, 40.7059460290057], [-73.98050926199988, 40.69744627499991], [-73.98696892299995, 40.698503613999954], [-73.98679382799996, 40.700633222999954], [-73.99093201799984, 40.7007662159999], [-73.99544564312069, 40.70327530547179], [-73.99350881403605, 40.70462350527397], [-73.97906084869959, 40.7059460290057]]]}}, {\"id\": \"63\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.43376623376623374, \"OBJECTID\": 64, \"Shape_Leng\": 0.18445188474, \"Shape_Area\": 0.00105790284614, \"zone\": \"Douglaston\", \"LocationID\": 64, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.74412314584049, 40.77863865777706], [-73.71351828186042, 40.75983773091619], [-73.72031595299998, 40.75789725799992], [-73.72771356199995, 40.750626894999904], [-73.7310646189999, 40.744888375999885], [-73.74016951799993, 40.73848189799986], [-73.74306820799991, 40.7384408179999], [-73.74396193599988, 40.740789125999925], [-73.75098230100001, 40.740021462999884], [-73.75347299999999, 40.74240484399992], [-73.74477587099986, 40.74328109699994], [-73.74625984399985, 40.74671147499993], [-73.74384322099985, 40.74769009599988], [-73.75004764499985, 40.75840465799996], [-73.75230259399993, 40.759455198999916], [-73.75583713299987, 40.757742061999934], [-73.75921689799992, 40.76432485599988], [-73.75377036482259, 40.76530463174143], [-73.7510743225255, 40.76176367133622], [-73.74455864030779, 40.756557885832535], [-73.75507210849031, 40.76753310678742], [-73.7555733264932, 40.77152912592269], [-73.75336691999294, 40.77320784776336], [-73.75551737581792, 40.77770404836418], [-73.75080593606877, 40.782893378833954], [-73.74412314584049, 40.77863865777706]]]}}, {\"id\": \"64\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3741264087198836, \"OBJECTID\": 65, \"Shape_Leng\": 0.0446070683658, \"Shape_Area\": 8.1803882541e-05, \"zone\": \"Downtown Brooklyn/MetroTech\", \"LocationID\": 65, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98712491499988, 40.70063447999997], [-73.98696892299995, 40.698503613999954], [-73.98050926199988, 40.69744627499991], [-73.97916718399999, 40.69348832299992], [-73.98251122999989, 40.69361433399992], [-73.98147568199991, 40.689930943999876], [-73.99097186899995, 40.69254879999995], [-73.99093201799984, 40.7007662159999], [-73.98712491499988, 40.70063447999997]]]}}, {\"id\": \"66\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5390855457227138, \"OBJECTID\": 67, \"Shape_Leng\": 0.0997470781551, \"Shape_Area\": 0.000394282272487, \"zone\": \"Dyker Heights\", \"LocationID\": 67, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.0109702639999, 40.63068186899988], [-74.00702302499997, 40.63210999499996], [-73.99724067099987, 40.62528868699994], [-73.99771358999998, 40.62368026999996], [-73.99932090299998, 40.62465524699996], [-74.01701910199995, 40.60765441699994], [-74.01549854999998, 40.606841523999925], [-74.0186913569999, 40.60360792399995], [-74.02129949199998, 40.60442832299987], [-74.02078540899996, 40.60567368599989], [-74.02247769499994, 40.60667421599987], [-74.02150602899994, 40.60782209899992], [-74.02710660599995, 40.61024395599991], [-74.02637609099996, 40.61245060299993], [-74.02754297299998, 40.61293718299988], [-74.01970491599991, 40.6221849649999], [-74.01514630699995, 40.63322193799993], [-74.0109702639999, 40.63068186899988]]]}}, {\"id\": \"67\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.34986003468471766, \"OBJECTID\": 68, \"Shape_Leng\": 0.0493373795569, \"Shape_Area\": 0.000111073378655, \"zone\": \"East Chelsea\", \"LocationID\": 68, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00201937799989, 40.74044408899989], [-74.00820401899988, 40.74214751799986], [-73.99730671499998, 40.757110506999894], [-73.99163271299996, 40.75470680499996], [-74.00201937799989, 40.74044408899989]]]}}, {\"id\": \"68\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6387065791579959, \"OBJECTID\": 69, \"Shape_Leng\": 0.0789896450483, \"Shape_Area\": 0.00019905160381, \"zone\": \"East Concourse/Concourse Village\", \"LocationID\": 69, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91179361999997, 40.8264281559999], [-73.91130974499993, 40.824848557999886], [-73.91541108199998, 40.82577390899993], [-73.92518486499998, 40.818012668999884], [-73.92731653099999, 40.81855963399991], [-73.91193751499992, 40.843202387999916], [-73.90514893999989, 40.84254719499995], [-73.91049010699992, 40.830737372999884], [-73.908959903, 40.83022623299989], [-73.91179361999997, 40.8264281559999]]]}}, {\"id\": \"69\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5087183775872883, \"OBJECTID\": 70, \"Shape_Leng\": 0.0638403183367, \"Shape_Area\": 0.000195458476728, \"zone\": \"East Elmhurst\", \"LocationID\": 70, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85912375899987, 40.76127549899994], [-73.85449593299987, 40.75827863899985], [-73.87513874300004, 40.756633815999905], [-73.87608949699984, 40.77153690299987], [-73.868098735, 40.77017200899994], [-73.85912375899987, 40.76127549899994]]]}}, {\"id\": \"70\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4368932038834951, \"OBJECTID\": 71, \"Shape_Leng\": 0.0931694551828, \"Shape_Area\": 0.000382572587364, \"zone\": \"East Flatbush/Farragut\", \"LocationID\": 71, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93177153199984, 40.65527364199989], [-73.92910623099985, 40.63979510599992], [-73.92574389399985, 40.64001446599985], [-73.92652490899994, 40.63649272299989], [-73.92744822299987, 40.635258409999864], [-73.9372670019998, 40.634650624999885], [-73.93701746200003, 40.63231097899989], [-73.94462885499995, 40.63183773499993], [-73.94537272499998, 40.638786954999865], [-73.94826499699982, 40.63860718999994], [-73.94841343400003, 40.639987463999944], [-73.94778994699976, 40.643305735999924], [-73.94586879199989, 40.64342374399988], [-73.94715896299984, 40.65559378699993], [-73.931907438, 40.656537652999916], [-73.93177153199984, 40.65527364199989]]]}}, {\"id\": \"71\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4871023215821152, \"OBJECTID\": 72, \"Shape_Leng\": 0.0952020695166, \"Shape_Area\": 0.000323532461947, \"zone\": \"East Flatbush/Remsen Village\", \"LocationID\": 72, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92021313099984, 40.65997544099987], [-73.9195947979998, 40.66199567899985], [-73.91236458899995, 40.65551356499985], [-73.91012255699995, 40.655851770999945], [-73.90855790499995, 40.65209593799989], [-73.91995062999999, 40.64470762999989], [-73.91973736999988, 40.64277814799987], [-73.92332654599986, 40.63990021599991], [-73.92356525899983, 40.642499297999905], [-73.92538233099987, 40.64238453199987], [-73.92574389499987, 40.64001446599983], [-73.92910623199991, 40.6397951059999], [-73.93190743899989, 40.656537652999944], [-73.92799465199988, 40.65680210099988], [-73.92834017599988, 40.660060035999884], [-73.92646424299983, 40.659814361999956], [-73.93068465499985, 40.66362047699992], [-73.92637970499992, 40.6655148919999], [-73.92021313099984, 40.65997544099987]]]}}, {\"id\": \"72\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5246636771300448, \"OBJECTID\": 73, \"Shape_Leng\": 0.0853020209129, \"Shape_Area\": 0.000291799754395, \"zone\": \"East Flushing\", \"LocationID\": 73, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79566545000003, 40.75023619999991], [-73.79729524799988, 40.74943505299992], [-73.7946756879998, 40.74721192799994], [-73.8033521419999, 40.74643196499987], [-73.80287666499987, 40.749765148999856], [-73.80750599299991, 40.750136586999936], [-73.81416172099998, 40.74613857999986], [-73.820849288, 40.75224154499988], [-73.8184078289999, 40.75383042199998], [-73.82030067699999, 40.75551497999992], [-73.81789245699991, 40.75708633799991], [-73.82063593299985, 40.75887226799989], [-73.81104963199985, 40.761417020999964], [-73.79940088899988, 40.759397218999936], [-73.79493246199993, 40.75779802999994], [-73.79566545000003, 40.75023619999991]]]}}, {\"id\": \"73\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5461532678159068, \"OBJECTID\": 74, \"Shape_Leng\": 0.11029093625, \"Shape_Area\": 0.000295038418204, \"zone\": \"East Harlem North\", \"LocationID\": 74, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.93380589881859, 40.81651249324708], [-73.93412073484292, 40.80865645197743], [-73.92903490240305, 40.80108090385195], [-73.9290364017962, 40.79676259410132], [-73.93505300002766, 40.79168720295447], [-73.93773612099982, 40.792815803999936], [-73.93864883299987, 40.79155775499987], [-73.94327251199996, 40.79348810299993], [-73.94187604699994, 40.79540586899984], [-73.94831125500004, 40.79812921799992], [-73.94459751299998, 40.803228144999906], [-73.94613131, 40.80387735799992], [-73.94430194599991, 40.806390824999916], [-73.93854407899988, 40.80572965299991], [-73.93580780199986, 40.80949763799987], [-73.93901753299998, 40.81085655699991], [-73.93407463999996, 40.81782651399992], [-73.93440039999986, 40.81966778299989], [-73.93380589881859, 40.81651249324708]]], [[[-73.92688606022001, 40.800840648368855], [-73.92650954883642, 40.80065703126224], [-73.92704640297914, 40.800651743031324], [-73.92688606022001, 40.800840648368855]]], [[[-73.92666565490181, 40.80018271401784], [-73.92669572614771, 40.80014697707624], [-73.9272477078927, 40.800392957368906], [-73.92711464230626, 40.80056375622957], [-73.92666565490181, 40.80018271401784]]], [[[-73.92672091168967, 40.800414210943586], [-73.92653846445016, 40.80033389815394], [-73.92674263892977, 40.800403498972756], [-73.92672091168967, 40.800414210943586]]], [[[-73.92718514789182, 40.797349896890104], [-73.92767122620857, 40.79721701426623], [-73.92770694127064, 40.7972986638821], [-73.92738263624496, 40.797377063580456], [-73.92718514789182, 40.797349896890104]]]]}}, {\"id\": \"74\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4492545165486342, \"OBJECTID\": 75, \"Shape_Leng\": 0.0876638997685, \"Shape_Area\": 0.00024056392715, \"zone\": \"East Harlem South\", \"LocationID\": 75, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94103496899987, 40.79254666899993], [-73.93505300002766, 40.79168720295447], [-73.93708104382307, 40.78936639571], [-73.93621859869943, 40.78880049757431], [-73.94354420706223, 40.78288052416259], [-73.9557773589999, 40.78791392399995], [-73.94831125500004, 40.79812921799992], [-73.94187604699994, 40.79540586899984], [-73.94327251199996, 40.79348810299993], [-73.94103496899987, 40.79254666899993]]]}}, {\"id\": \"75\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5230667838312829, \"OBJECTID\": 76, \"Shape_Leng\": 0.241203016269, \"Shape_Area\": 0.00126660143241, \"zone\": \"East New York\", \"LocationID\": 76, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88063900599983, 40.679176984999856], [-73.87507136399991, 40.68003514599991], [-73.87231018000001, 40.67775574699987], [-73.86106294999989, 40.67434293499984], [-73.86038937900003, 40.67126877499985], [-73.85763323199998, 40.67165619399985], [-73.85568461199988, 40.66386749199993], [-73.85842950899992, 40.66345335999987], [-73.85761052799985, 40.660108355999924], [-73.86317083299996, 40.65827651199994], [-73.86327623068586, 40.656941215328104], [-73.85695870295893, 40.65058737348713], [-73.85840315341014, 40.64700390123735], [-73.85716978298207, 40.64367855888597], [-73.86289463224384, 40.64255067968245], [-73.86670411973783, 40.6400217634955], [-73.86563247391335, 40.63878676362291], [-73.86867282895084, 40.640925079203356], [-73.87061492461066, 40.64485674188673], [-73.87971979034293, 40.65437592977636], [-73.86989583386247, 40.63893452891954], [-73.86799709668125, 40.63811526520958], [-73.86960972136559, 40.63708553460871], [-73.87347939154502, 40.63628554579879], [-73.87841589406925, 40.638912189353306], [-73.8745808109999, 40.64561010599992], [-73.88221364899994, 40.65451863799996], [-73.89000589199988, 40.651148696999925], [-73.89354677099988, 40.65520271399988], [-73.89761013299987, 40.65345381599992], [-73.89586154900002, 40.65188052699987], [-73.90398200599991, 40.64662856799988], [-73.90944846399994, 40.65152159899992], [-73.90740035699999, 40.6528439899999], [-73.90784346399994, 40.65459152999988], [-73.89612587299993, 40.657814663999936], [-73.88185557299985, 40.66414998799994], [-73.88330883399975, 40.66611200499991], [-73.889906696, 40.66377828999986], [-73.89087805100002, 40.671318086999875], [-73.89271090700004, 40.67104582799992], [-73.89367036399987, 40.67482177099988], [-73.89930675399987, 40.67398204199991], [-73.90004626199993, 40.67689308999995], [-73.88116730300005, 40.68135366299988], [-73.88063900599983, 40.679176984999856]]]}}, {\"id\": \"76\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4744079449961803, \"OBJECTID\": 77, \"Shape_Leng\": 0.075461081444, \"Shape_Area\": 0.000191917981315, \"zone\": \"East New York/Pennsylvania Avenue\", \"LocationID\": 77, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8997089949999, 40.67549425099991], [-73.89930675399987, 40.67398204199991], [-73.89367036399987, 40.67482177099988], [-73.89271090699994, 40.671045828999915], [-73.89087805100002, 40.671318086999875], [-73.889906696, 40.66377828999986], [-73.88330883399975, 40.66611200499991], [-73.88185557299985, 40.66414998799994], [-73.89880698499995, 40.65740666199988], [-73.90066118699986, 40.66058615799992], [-73.90347422300006, 40.67550668299985], [-73.90004626199993, 40.67689308999995], [-73.8997089949999, 40.67549425099991]]]}}, {\"id\": \"77\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6226415094339622, \"OBJECTID\": 78, \"Shape_Leng\": 0.0935944495806, \"Shape_Area\": 0.000191114419551, \"zone\": \"East Tremont\", \"LocationID\": 78, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88963328299991, 40.85317381799993], [-73.88987871399985, 40.85207022099993], [-73.8773292409999, 40.84398312499995], [-73.878171756, 40.84262443699987], [-73.87459175399982, 40.841520647999914], [-73.87109207899995, 40.84432478099992], [-73.86920944599997, 40.84923837199993], [-73.86785992999985, 40.84886493099987], [-73.86804200399997, 40.8448138389999], [-73.87359713599987, 40.83979855499992], [-73.880061996, 40.840162234999845], [-73.88200384100001, 40.837443781999895], [-73.88618432099993, 40.8401050449999], [-73.88941724899988, 40.83942273399985], [-73.89537836099996, 40.842938778999944], [-73.89290102299996, 40.84626832999992], [-73.89617712999998, 40.8466583209999], [-73.89307140900002, 40.85324998699989], [-73.89038953999983, 40.85468905799996], [-73.88963328299991, 40.85317381799993]]]}}, {\"id\": \"78\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.30423765485150406, \"OBJECTID\": 79, \"Shape_Leng\": 0.0426249113144, \"Shape_Area\": 0.000107893068218, \"zone\": \"East Village\", \"LocationID\": 79, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98377990999991, 40.72153595399992], [-73.99260322199994, 40.72413644999987], [-73.98990295999991, 40.7344347899999], [-73.97802697999985, 40.729433059999984], [-73.98377990999991, 40.72153595399992]]]}}, {\"id\": \"79\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3536616044593324, \"OBJECTID\": 80, \"Shape_Leng\": 0.117212621448, \"Shape_Area\": 0.00040732245622, \"zone\": \"East Williamsburg\", \"LocationID\": 80, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93174841322003, 40.728053727696675], [-73.92849418540158, 40.72661460330786], [-73.92453854985803, 40.719342580204575], [-73.93220607307306, 40.71496398092936], [-73.93124067059496, 40.71372481088467], [-73.93328349320237, 40.71097870745788], [-73.9319010192085, 40.71133847142828], [-73.9303609456579, 40.7087177385352], [-73.93184868877432, 40.71256477418327], [-73.93042294313065, 40.71317255867104], [-73.93105231901723, 40.71461022392334], [-73.92460339476072, 40.717953306073376], [-73.92305500014334, 40.71634266479336], [-73.92488553507611, 40.71526718765259], [-73.92433589757124, 40.71412259346647], [-73.92074519699985, 40.71052968599989], [-73.93229708600003, 40.70774944699988], [-73.93480281900003, 40.713372495999884], [-73.94378799599994, 40.710900378999945], [-73.94273448399997, 40.70447330599988], [-73.9408928719998, 40.70391415699989], [-73.94042057899993, 40.701076597999865], [-73.94193078899983, 40.70072523399989], [-73.95023693799996, 40.705473245999876], [-73.95087492699996, 40.70951311499991], [-73.94865533499987, 40.70972498299988], [-73.94937231899985, 40.71406687799986], [-73.95348472199983, 40.71407020799988], [-73.93174841322003, 40.728053727696675]]]}}, {\"id\": \"80\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5878220140515222, \"OBJECTID\": 81, \"Shape_Leng\": 0.197592771731, \"Shape_Area\": 0.00039956655019, \"zone\": \"Eastchester\", \"LocationID\": 81, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8236684811033, 40.889903740147645], [-73.82285335053221, 40.8912068290476], [-73.79300652624502, 40.88335873744268], [-73.81209659199986, 40.88671500999993], [-73.82027432999985, 40.885550699999925], [-73.82387140599998, 40.88778362799996], [-73.84324113799987, 40.879108912999904], [-73.8381484219999, 40.876953298999915], [-73.83924788999988, 40.876618737999905], [-73.83829221499984, 40.87295133899995], [-73.83565766499987, 40.86751275599992], [-73.83315960399985, 40.86831975599994], [-73.82834347399992, 40.86089202599988], [-73.85363638799987, 40.87330059899996], [-73.8506895569999, 40.87548667799993], [-73.85254854099988, 40.877091586999924], [-73.85091896099982, 40.877893440999856], [-73.85248305800003, 40.87946370499989], [-73.84704957999995, 40.8876718159999], [-73.83554853999995, 40.88913350399993], [-73.83737956161106, 40.893924741304346], [-73.8236684811033, 40.889903740147645]]]}}, {\"id\": \"81\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.686123283669296, \"OBJECTID\": 82, \"Shape_Leng\": 0.119875649697, \"Shape_Area\": 0.000323601079994, \"zone\": \"Elmhurst\", \"LocationID\": 82, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86768504899995, 40.744071717999915], [-73.86447086199989, 40.73400727399998], [-73.88720524999994, 40.727792996999874], [-73.8885228599999, 40.734531159999904], [-73.87752195799992, 40.731593168999915], [-73.87708276299998, 40.73290931699991], [-73.88066188999987, 40.73570787399994], [-73.88033059399996, 40.73733367699989], [-73.88230252700001, 40.737707700999934], [-73.87743058400002, 40.73860051599991], [-73.88100387999987, 40.74184066199995], [-73.88551225099985, 40.74016501899988], [-73.887511321, 40.74108252699992], [-73.88798650799986, 40.743548191999935], [-73.89071647499993, 40.74312778399992], [-73.89175048799991, 40.7468117269999], [-73.86942458599987, 40.74915685199991], [-73.86768504899995, 40.744071717999915]]]}}, {\"id\": \"82\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6719811282385, \"OBJECTID\": 83, \"Shape_Leng\": 0.105984933269, \"Shape_Area\": 0.000217463718718, \"zone\": \"Elmhurst/Maspeth\", \"LocationID\": 83, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89111026199994, 40.745202717999874], [-73.89071647499993, 40.74312778399992], [-73.88798650799986, 40.743548191999935], [-73.887511321, 40.74108252699992], [-73.88551225099985, 40.74016501899988], [-73.88100387999987, 40.74184066199995], [-73.87791831799989, 40.739394956999874], [-73.88230252700001, 40.737707700999934], [-73.88033059399996, 40.73733367699989], [-73.88066188999987, 40.73570787399994], [-73.87725122499988, 40.7317941529999], [-73.88910318999984, 40.73459239599987], [-73.89854772799987, 40.735191909999855], [-73.90107970900003, 40.73391463399987], [-73.90516948899983, 40.73585964299986], [-73.90628903499987, 40.732400751999876], [-73.91068876399997, 40.733471081999916], [-73.90926002999988, 40.736403179999854], [-73.90057825399985, 40.73949289599988], [-73.89524819599988, 40.74381972999992], [-73.89624245599988, 40.74871954499987], [-73.89139145499985, 40.74664100399984], [-73.89111026199994, 40.745202717999874]]]}}, {\"id\": \"83\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3448275862068966, \"OBJECTID\": 84, \"Shape_Leng\": 0.233623987032, \"Shape_Area\": 0.00207375572052, \"zone\": \"Eltingville/Annadale/Prince's Bay\", \"LocationID\": 84, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.16005732699996, 40.527635041999886], [-74.1720632017547, 40.52332056936472], [-74.17756306042942, 40.519217618159864], [-74.18104081208887, 40.520705036287154], [-74.1848311647976, 40.519341358600826], [-74.19521339999068, 40.5099776529556], [-74.19967205357709, 40.511426850497486], [-74.19936127407836, 40.51301019443217], [-74.20821982874101, 40.51161821258057], [-74.21754755095681, 40.50336004949062], [-74.22153919955196, 40.50250347421612], [-74.22690993399992, 40.51917139799985], [-74.22379595899992, 40.518196345999876], [-74.21354593499994, 40.521235714999925], [-74.21409806799993, 40.52409719499991], [-74.20152661599995, 40.526265205999934], [-74.19174368799999, 40.53191669699986], [-74.19377169299999, 40.53583384699993], [-74.18903145199995, 40.53815934199989], [-74.18128532999992, 40.54695757299989], [-74.17185166499996, 40.56152092299991], [-74.16982582400001, 40.56109042099991], [-74.16540080799992, 40.54443272299989], [-74.15397419799994, 40.53426814799993], [-74.16134961699994, 40.5293734239999], [-74.16005732699996, 40.527635041999886]]]}}, {\"id\": \"84\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5650963597430407, \"OBJECTID\": 85, \"Shape_Leng\": 0.0574751473562, \"Shape_Area\": 0.000143637804933, \"zone\": \"Erasmus\", \"LocationID\": 85, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94715896299984, 40.65559378699993], [-73.94586879199989, 40.64342374399988], [-73.94778994699976, 40.643305735999924], [-73.94826499600005, 40.63860718999994], [-73.9536000469999, 40.63842234699991], [-73.95763644599988, 40.64268949899989], [-73.9587078989999, 40.65038727299989], [-73.94957113100006, 40.65080789999991], [-73.9500666489999, 40.65542299699989], [-73.94715896299984, 40.65559378699993]]]}}, {\"id\": \"85\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.36713286713286714, \"OBJECTID\": 86, \"Shape_Leng\": 0.134245282582, \"Shape_Area\": 0.000623278815249, \"zone\": \"Far Rockaway\", \"LocationID\": 86, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76669071688973, 40.61422662157113], [-73.76505961679173, 40.612035230610466], [-73.76023635756965, 40.610820864950284], [-73.76178242960594, 40.60915167005445], [-73.76155051905889, 40.60903136580077], [-73.75998446974822, 40.61087937784668], [-73.7556083701559, 40.61007791963748], [-73.7457615018933, 40.611991654374336], [-73.73815143112287, 40.60271044008338], [-73.73808925304178, 40.59768829704553], [-73.74183083531219, 40.59669555145679], [-73.73812220887619, 40.597386102069386], [-73.73763679389783, 40.594415399837956], [-73.74664920374843, 40.594280895008744], [-73.75352988157324, 40.59094648500059], [-73.76376223423699, 40.591439709016804], [-73.76440183099987, 40.59457168299989], [-73.76277845599988, 40.59475654399989], [-73.7633344969999, 40.59782287399992], [-73.76128753399995, 40.59954927099993], [-73.76654204999997, 40.599434441999904], [-73.76771811799993, 40.596261309999925], [-73.76795043655434, 40.5981585114185], [-73.77098168371995, 40.59889409104954], [-73.76895714023856, 40.60927991805636], [-73.77417180232338, 40.61182871527586], [-73.77339803484179, 40.61368296748421], [-73.76669071688973, 40.61422662157113]]]}}, {\"id\": \"86\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.26709180288918044, \"OBJECTID\": 87, \"Shape_Leng\": 0.0369015496117, \"Shape_Area\": 6.72050210692e-05, \"zone\": \"Financial District North\", \"LocationID\": 87, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00372234899996, 40.70693234399989], [-74.00197014837809, 40.704731159202304], [-74.00332307270776, 40.70562859522001], [-74.00459003883662, 40.70478775294495], [-74.00349632042315, 40.70379676926777], [-74.0053502722475, 40.70431007996148], [-74.004272587521, 40.70301566641148], [-74.00659591913875, 40.70368590245183], [-74.00530412596079, 40.70255986563306], [-74.00550166067457, 40.70243086779703], [-74.01250773999998, 40.706767065999934], [-74.00906264799988, 40.71088670399992], [-74.00372234899996, 40.70693234399989]]]}}, {\"id\": \"87\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.34389027615383766, \"OBJECTID\": 88, \"Shape_Leng\": 0.0352046035327, \"Shape_Area\": 5.72999455414e-05, \"zone\": \"Financial District South\", \"LocationID\": 88, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00974639299997, 40.704775247999905], [-74.00760498189766, 40.70299314132918], [-74.00930907655483, 40.70195460302249], [-74.00759136204225, 40.70132808104815], [-74.01367479307396, 40.70012582122537], [-74.01428922899998, 40.70454907499989], [-74.01250773999998, 40.706767065999934], [-74.00974639299997, 40.704775247999905]]]}}, {\"id\": \"88\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4829651815799326, \"OBJECTID\": 89, \"Shape_Leng\": 0.122794569553, \"Shape_Area\": 0.000447548142373, \"zone\": \"Flatbush/Ditmas Park\", \"LocationID\": 89, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95374853799996, 40.63858633799993], [-73.94537272499998, 40.638786954999865], [-73.94462885499995, 40.63183773499993], [-73.94647961799998, 40.631708161999896], [-73.94415535299993, 40.629508284999886], [-73.95995565099992, 40.627778671999884], [-73.96040734299989, 40.6301625309999], [-73.96232599399991, 40.6299687169999], [-73.97136622199992, 40.62892916199996], [-73.97092126300004, 40.62656998499993], [-73.97623453700001, 40.625976349999895], [-73.97716511999984, 40.63074665399988], [-73.96986869299992, 40.63419467499989], [-73.97029036199991, 40.63640634099988], [-73.96803868499985, 40.636650984999925], [-73.97139631700004, 40.64825778599986], [-73.96465438699984, 40.650887727999894], [-73.96621321999996, 40.653189404999885], [-73.95986103199981, 40.65563478899989], [-73.95763644599988, 40.64268949899989], [-73.95374853799996, 40.63858633799993]]]}}, {\"id\": \"89\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.31804628674740676, \"OBJECTID\": 90, \"Shape_Leng\": 0.0307591620819, \"Shape_Area\": 5.53132476305e-05, \"zone\": \"Flatiron\", \"LocationID\": 90, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9963368949999, 40.7380437469999], [-74.00252381499989, 40.73975264399995], [-73.99709902899994, 40.74720510199991], [-73.99141858599994, 40.74480268199988], [-73.9963368949999, 40.7380437469999]]]}}, {\"id\": \"90\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5238828967642527, \"OBJECTID\": 91, \"Shape_Leng\": 0.124996624585, \"Shape_Area\": 0.000537330013243, \"zone\": \"Flatlands\", \"LocationID\": 91, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91945472999996, 40.6401731369999], [-73.91690302699992, 40.61627422899993], [-73.9205269409999, 40.61394269499987], [-73.92640044599996, 40.61447853499988], [-73.92629604599998, 40.61311679899991], [-73.93686699599992, 40.620253280999904], [-73.94573691999982, 40.61536340699989], [-73.94651373199997, 40.61949401899995], [-73.94451833400001, 40.61971364699989], [-73.94732672199994, 40.629166566999835], [-73.94415535299993, 40.629508284999886], [-73.94647961799998, 40.631708161999896], [-73.93701746200003, 40.63231097899989], [-73.9372670019998, 40.634650624999885], [-73.92744822299987, 40.635258409999864], [-73.92538233099987, 40.64238453199987], [-73.92356525899983, 40.642499297999905], [-73.92332654599986, 40.63990021599991], [-73.91973736999988, 40.64277814799987], [-73.91945472999996, 40.6401731369999]]]}}, {\"id\": \"91\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6834246428187776, \"OBJECTID\": 92, \"Shape_Leng\": 0.117830066799, \"Shape_Area\": 0.000374946617289, \"zone\": \"Flushing\", \"LocationID\": 92, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82656578899986, 40.77064147599989], [-73.82466908999987, 40.77040760799985], [-73.82080684899987, 40.7589417829999], [-73.81789245699991, 40.75708633799991], [-73.82030067699999, 40.75551497999992], [-73.8184078289999, 40.75383042199998], [-73.820849288, 40.75224154499988], [-73.81575069499985, 40.7469794799999], [-73.82108427599998, 40.748558445999954], [-73.82426493900002, 40.75183288899994], [-73.82584440999999, 40.74934839099993], [-73.83300302099987, 40.74783363099988], [-73.8322953569999, 40.751941790999844], [-73.83560230899995, 40.75139485299991], [-73.83805974199996, 40.75466660699988], [-73.83689970199995, 40.7626981179999], [-73.8394581342306, 40.76544576822929], [-73.83000011099983, 40.77373240699996], [-73.82513740299993, 40.78169970599994], [-73.82224127699999, 40.781693878999924], [-73.82583089999997, 40.77591527099986], [-73.82656578899986, 40.77064147599989]]]}}, {\"id\": \"92\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3998149992884588, \"OBJECTID\": 93, \"Shape_Leng\": 0.170998027597, \"Shape_Area\": 0.000594278183978, \"zone\": \"Flushing Meadows-Corona Park\", \"LocationID\": 93, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8569635593979, 40.76407442786308], [-73.8566748236235, 40.763930188326135], [-73.85758285470693, 40.76307290062661], [-73.8565867538878, 40.762722840114236], [-73.85817631070509, 40.7623962942238], [-73.85661195813331, 40.761649542759294], [-73.8585852628444, 40.76191700859695], [-73.85747662435531, 40.76130891668733], [-73.85158552901191, 40.75935542065696], [-73.84930110814332, 40.76024287534997], [-73.85207200177854, 40.7603928846721], [-73.84670933714987, 40.76134526931977], [-73.84324381399976, 40.75533680399995], [-73.83867321899987, 40.75701997899995], [-73.83560230899995, 40.75139485299991], [-73.8322953569999, 40.751941790999844], [-73.83300302099987, 40.74783363099988], [-73.83699037399995, 40.74209330099992], [-73.8361277859999, 40.734241311999924], [-73.83073214599987, 40.72654932799991], [-73.8273090039999, 40.72464607699988], [-73.82531644999993, 40.71727569999994], [-73.81729568999994, 40.71757964499995], [-73.83145866999996, 40.714554448999934], [-73.8297961659998, 40.71556571899986], [-73.83488359399998, 40.719303271999934], [-73.85436928099995, 40.748738929999874], [-73.85086439699992, 40.74999564199986], [-73.85379813799992, 40.757177611999914], [-73.86109724399991, 40.76366447799989], [-73.85861915281858, 40.7619355969989], [-73.8569635593979, 40.76407442786308]]]}}, {\"id\": \"93\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6886102403343782, \"OBJECTID\": 94, \"Shape_Leng\": 0.0498326083147, \"Shape_Area\": 6.25819407393e-05, \"zone\": \"Fordham South\", \"LocationID\": 94, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89964261700001, 40.862210436999874], [-73.897143798, 40.86244506499991], [-73.89883814599983, 40.859307307999906], [-73.89573924699978, 40.85813820699986], [-73.89106059999996, 40.861471814999945], [-73.89560883299981, 40.85457036999986], [-73.89863543699985, 40.85390697499993], [-73.89816577499982, 40.85514637599989], [-73.90077922699986, 40.85604710999993], [-73.90185996099993, 40.85458614899994], [-73.90557007299988, 40.855787321999884], [-73.90351390599986, 40.86091750999989], [-73.89964261700001, 40.862210436999874]]]}}, {\"id\": \"94\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5287348080163482, \"OBJECTID\": 95, \"Shape_Leng\": 0.113629605476, \"Shape_Area\": 0.00057277033803, \"zone\": \"Forest Hills\", \"LocationID\": 95, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84732494199989, 40.738771457999896], [-73.83488359399998, 40.719303271999934], [-73.8297961659998, 40.71556571899986], [-73.8377089419999, 40.71103648399989], [-73.85593991699984, 40.7063980339999], [-73.8586809759999, 40.72261243999988], [-73.86203833799982, 40.72595678499994], [-73.85805416899993, 40.72444540599986], [-73.85727108100001, 40.72568129799988], [-73.8594198799999, 40.726464837999885], [-73.85741240699991, 40.72832247999986], [-73.85851097899999, 40.73028680199993], [-73.85625665099991, 40.73095912299993], [-73.85872465799989, 40.73570864399988], [-73.84732494199989, 40.738771457999896]]]}}, {\"id\": \"95\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5415929203539823, \"OBJECTID\": 96, \"Shape_Leng\": 0.185180248095, \"Shape_Area\": 0.000547566253402, \"zone\": \"Forest Park/Highland Park\", \"LocationID\": 96, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83804164799996, 40.70832329499994], [-73.83696237799982, 40.70421245199992], [-73.84502472899999, 40.6993593769999], [-73.84913837299978, 40.69828274099995], [-73.85070842300001, 40.70020312499988], [-73.85443455699989, 40.70060295199994], [-73.85277238699999, 40.69755711199987], [-73.874020533, 40.69419129499994], [-73.88962787599998, 40.684236453999894], [-73.89252316899987, 40.683424532999865], [-73.89417463299992, 40.6852832479999], [-73.89646625099995, 40.68233642199988], [-73.90116155099986, 40.687877934999946], [-73.90180467199995, 40.6907662979999], [-73.9003866789998, 40.69313995999992], [-73.89736265799978, 40.6941108469999], [-73.89695653499987, 40.6919492069999], [-73.89326209899984, 40.69220094699994], [-73.8950769439999, 40.694591389999886], [-73.88642048199995, 40.696458148999945], [-73.88771297999986, 40.699678075999906], [-73.88280948499987, 40.7006834549999], [-73.88182690599986, 40.69952929899987], [-73.87639785199997, 40.70229404199989], [-73.86285374199994, 40.7034144919999], [-73.85819890699996, 40.70291107399986], [-73.85893224399979, 40.70132034499989], [-73.85725351500002, 40.70121217699989], [-73.85252019099998, 40.70229172399997], [-73.85473240499985, 40.70676414299989], [-73.84314625299982, 40.70922345599992], [-73.83804164799996, 40.70832329499994]]]}}, {\"id\": \"96\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.42130950930278793, \"OBJECTID\": 97, \"Shape_Leng\": 0.0624760147423, \"Shape_Area\": 0.000163303970435, \"zone\": \"Fort Greene\", \"LocationID\": 97, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9693409769999, 40.69576842999993], [-73.96910362499993, 40.68946752199986], [-73.97013154199992, 40.68950630799986], [-73.96868961400001, 40.68200525799988], [-73.97745722099988, 40.68405228699988], [-73.98092822899991, 40.68890247699988], [-73.98372011099993, 40.69000671899992], [-73.98147568199991, 40.689930943999876], [-73.98251122999989, 40.69361433399992], [-73.97916718399999, 40.69348832299992], [-73.98054419199988, 40.69824651199988], [-73.9697278989999, 40.69780401399986], [-73.9693409769999, 40.69576842999993]]]}}, {\"id\": \"97\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5862573099415205, \"OBJECTID\": 98, \"Shape_Leng\": 0.121661018278, \"Shape_Area\": 0.000485773020954, \"zone\": \"Fresh Meadows\", \"LocationID\": 98, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76360860199983, 40.73917476499987], [-73.76273303499985, 40.73526216299995], [-73.76510920699994, 40.73455658499992], [-73.7636997919999, 40.73286411699993], [-73.76564374500005, 40.732533126999954], [-73.75669771299987, 40.7262287109999], [-73.76924074899993, 40.72590270099988], [-73.77727808000003, 40.72234633599988], [-73.77803765999981, 40.723284268999855], [-73.7752338719998, 40.725237532999884], [-73.77757506899988, 40.73019327299987], [-73.79174472699992, 40.725788849999944], [-73.79651029099988, 40.7379086369999], [-73.76772991399994, 40.74577622599993], [-73.76360860199983, 40.73917476499987]]]}}, {\"id\": \"98\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4375, \"OBJECTID\": 99, \"Shape_Leng\": 0.1833714893, \"Shape_Area\": 0.00121016463877, \"zone\": \"Freshkills Park\", \"LocationID\": 99, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.16842916199994, 40.58649792299988], [-74.16982582400001, 40.56109042099991], [-74.18295815999996, 40.56548019999992], [-74.193410946, 40.56248503899994], [-74.20091501299991, 40.55600525999987], [-74.20699885299997, 40.55756748899989], [-74.2056960519999, 40.55851600599987], [-74.21324031454323, 40.55667262386906], [-74.20596154297344, 40.580046993882775], [-74.20048848799996, 40.57994977799989], [-74.19964981299998, 40.58226262599991], [-74.19423261299991, 40.58514680599988], [-74.185371048, 40.58745101499987], [-74.18730248699987, 40.588488221999896], [-74.17488959399996, 40.602216572999914], [-74.16719634899995, 40.602075311999926], [-74.16886362299992, 40.59472808699995], [-74.16350785499993, 40.59327217999993], [-74.16842916199994, 40.58649792299988]]]}}, {\"id\": \"99\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4536520705731785, \"OBJECTID\": 100, \"Shape_Leng\": 0.0248131090342, \"Shape_Area\": 3.74700210291e-05, \"zone\": \"Garment District\", \"LocationID\": 100, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98729377099981, 40.75045160899988], [-73.99346417699986, 40.75219005499987], [-73.98979085399996, 40.7572331319999], [-73.98411754799999, 40.75484205299995], [-73.98729377099981, 40.75045160899988]]]}}, {\"id\": \"100\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5330188679245284, \"OBJECTID\": 101, \"Shape_Leng\": 0.101709836277, \"Shape_Area\": 0.000452342528877, \"zone\": \"Glen Oaks\", \"LocationID\": 101, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.70134715908382, 40.750780580935846], [-73.70002020501624, 40.73923654186594], [-73.70766217302989, 40.727830934754685], [-73.71049995381028, 40.72722708720663], [-73.71147804899985, 40.73250140099988], [-73.72010148299994, 40.74977568599991], [-73.727664654, 40.749654641999946], [-73.728962516, 40.74763288799986], [-73.72737889099987, 40.751029566999854], [-73.7206331939999, 40.75761683499996], [-73.71612528199985, 40.75966071399991], [-73.71244940715339, 40.75924170410932], [-73.70163345846068, 40.75249332984642], [-73.70134715908382, 40.750780580935846]]]}}, {\"id\": \"101\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.589425587467363, \"OBJECTID\": 102, \"Shape_Leng\": 0.136900484646, \"Shape_Area\": 0.000296595466345, \"zone\": \"Glendale\", \"LocationID\": 102, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85595536199988, 40.70642977899992], [-73.85473240499985, 40.70676414299989], [-73.85252019099998, 40.70229172399997], [-73.85827405799986, 40.70122365899992], [-73.85819890699996, 40.70291107399986], [-73.86285374199994, 40.7034144919999], [-73.87639785199997, 40.70229404199989], [-73.88182690599986, 40.69952929899987], [-73.88280948499988, 40.70068345599986], [-73.88771297999986, 40.699678075999906], [-73.88642048199995, 40.696458148999945], [-73.8950769439999, 40.694591389999886], [-73.89326209899984, 40.69220094699994], [-73.89695653499987, 40.6919492069999], [-73.89736265799978, 40.6941108469999], [-73.900030392, 40.693437450999895], [-73.88911917799989, 40.705598876999915], [-73.86911007199988, 40.707089336999914], [-73.87016462499984, 40.70920774299987], [-73.85965416799995, 40.713379960999866], [-73.85975932900001, 40.711845058999856], [-73.85729287299985, 40.711467676999874], [-73.85595536199988, 40.70642977899992]]]}}, {\"id\": \"102\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.40350877192982454, \"OBJECTID\": 103, \"Shape_Leng\": 0.0143055167343, \"Shape_Area\": 6.33056361314e-06, \"zone\": \"Governor's Island/Ellis Island/Liberty Island\", \"LocationID\": 103, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.04388559600675, 40.690184824096946], [-74.04350596056258, 40.689687359818144], [-74.04270428413375, 40.69015520482211], [-74.04255372018648, 40.68996275914475], [-74.04438521705613, 40.68851617820147], [-74.04772962698515, 40.689915318233105], [-74.04614707204328, 40.69112264601653], [-74.04388559600675, 40.690184824096946]]]}}, {\"id\": \"103\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.40350877192982454, \"OBJECTID\": 104, \"Shape_Leng\": 0.0212208330928, \"Shape_Area\": 1.19205339715e-05, \"zone\": \"Governor's Island/Ellis Island/Liberty Island\", \"LocationID\": 103, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.03995040794246, 40.70089063032128], [-74.03771124798313, 40.69934404017382], [-74.03934037685066, 40.698115514418], [-74.04124261816106, 40.69953674121738], [-74.0399124888631, 40.69770204019913], [-74.0416605190741, 40.696452971319864], [-74.04367371215382, 40.69802040416498], [-74.03995040794246, 40.70089063032128]]]}}, {\"id\": \"104\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.40350877192982454, \"OBJECTID\": 105, \"Shape_Leng\": 0.0774253398314, \"Shape_Area\": 0.000368636392805, \"zone\": \"Governor's Island/Ellis Island/Liberty Island\", \"LocationID\": 103, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01674756096064, 40.693343368217555], [-74.0118118564723, 40.69247933506435], [-74.01333264149613, 40.69201997010238], [-74.01182575354963, 40.69106481621614], [-74.01316926402752, 40.6881547712032], [-74.02259359581036, 40.68435969594992], [-74.02213250186877, 40.683767672385194], [-74.02305574749599, 40.6829169454452], [-74.0227589902996, 40.68428444833225], [-74.026334042424, 40.6848016977242], [-74.01971454942077, 40.69313300682718], [-74.01674756096064, 40.693343368217555]]]}}, {\"id\": \"105\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.37945110300503737, \"OBJECTID\": 106, \"Shape_Leng\": 0.0492110119798, \"Shape_Area\": 0.000113616826567, \"zone\": \"Gowanus\", \"LocationID\": 106, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98610388199985, 40.68100852199994], [-73.98175642299985, 40.67931462399987], [-73.99277080699981, 40.665507280999876], [-73.9986188799999, 40.67183017199989], [-73.99506451099984, 40.67584047499984], [-73.99069861899994, 40.67580741899994], [-73.9876680399999, 40.68081780999993], [-73.98610388199985, 40.68100852199994]]]}}, {\"id\": \"106\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.30370004366390546, \"OBJECTID\": 107, \"Shape_Leng\": 0.0380413645908, \"Shape_Area\": 7.46112192675e-05, \"zone\": \"Gramercy\", \"LocationID\": 107, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98238628799993, 40.73141124499993], [-73.98990295999991, 40.7344347899999], [-73.98407485299995, 40.74332471799989], [-73.98085965399994, 40.74196976799989], [-73.98266497499989, 40.73949733299987], [-73.97805226499996, 40.7375539539999], [-73.98238628799993, 40.73141124499993]]]}}, {\"id\": \"107\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5326633165829145, \"OBJECTID\": 108, \"Shape_Leng\": 0.11117141227, \"Shape_Area\": 0.000352866009465, \"zone\": \"Gravesend\", \"LocationID\": 108, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97148350899987, 40.59717158499996], [-73.97115101799994, 40.59044244399989], [-73.96514385199995, 40.5911019159999], [-73.96661531499983, 40.58457103699992], [-73.97742525299988, 40.58290659299994], [-73.98710120299997, 40.58464459199988], [-73.98654457099988, 40.583658590999946], [-73.98829948299982, 40.58346674699986], [-73.98995504960881, 40.58066166976562], [-73.99185535656628, 40.5816239785461], [-73.99026244714351, 40.5846191482053], [-73.99547188560318, 40.58221005511802], [-74.0003088081067, 40.58317703751469], [-73.99808523776004, 40.585364721581485], [-73.99956511570304, 40.5862439200957], [-73.9940790462178, 40.58869344199059], [-74.00038207107852, 40.58713305717468], [-73.98962150099992, 40.59098199299996], [-73.98601936799993, 40.594446048999906], [-73.97995443499985, 40.594073811999884], [-73.98035785499995, 40.59619154099992], [-73.97148350899987, 40.59717158499996]]]}}, {\"id\": \"108\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.45, \"OBJECTID\": 109, \"Shape_Leng\": 0.178267819599, \"Shape_Area\": 0.00116960076185, \"zone\": \"Great Kills\", \"LocationID\": 109, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.14706472699994, 40.5705086149999], [-74.13170009, 40.56453574899991], [-74.13885092699991, 40.5569511789999], [-74.13316825999992, 40.552356178999936], [-74.13861798499994, 40.548492384999875], [-74.13585712583111, 40.546451696357344], [-74.1371798712987, 40.5457928646606], [-74.13611540724054, 40.54501984567978], [-74.13794966050746, 40.54600033758068], [-74.13886951340419, 40.545130197607286], [-74.13762430745145, 40.54425356641828], [-74.13875534703455, 40.544933958426874], [-74.13786798524545, 40.54405254413092], [-74.13903307178627, 40.54470491029999], [-74.13953937915736, 40.54429514087214], [-74.13834151288823, 40.543551828814046], [-74.14038573073617, 40.54403318500786], [-74.13921993485874, 40.54316608513788], [-74.14141073560397, 40.543022415080344], [-74.14026290400692, 40.54196151152258], [-74.14134901379589, 40.54257752313157], [-74.14179719957052, 40.54220041538235], [-74.14063984172124, 40.54169454203723], [-74.14095816832888, 40.54143207796496], [-74.14229632246158, 40.54225784915546], [-74.14104157916547, 40.53734688546733], [-74.14374951249268, 40.53719106134627], [-74.15970134520207, 40.52729599283152], [-74.16134961699994, 40.5293734239999], [-74.15397419799994, 40.53426814799993], [-74.16540080799992, 40.54443272299989], [-74.16982582400001, 40.56109042099991], [-74.1649414939999, 40.56011179699991], [-74.15725974199995, 40.562553318999896], [-74.14706472699994, 40.5705086149999]]]}}, {\"id\": \"109\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5, \"OBJECTID\": 110, \"Shape_Leng\": 0.103946292913, \"Shape_Area\": 0.000525745098785, \"zone\": \"Great Kills Park\", \"LocationID\": 110, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.11813477599993, 40.550592692999906], [-74.11338340608127, 40.54807337507532], [-74.12245306050386, 40.544770545629824], [-74.1335644349803, 40.5315111748056], [-74.1369079907634, 40.529281918882695], [-74.13883030715276, 40.529889356352996], [-74.14032475393472, 40.5352845797219], [-74.13381005153853, 40.53564293988989], [-74.12738789634466, 40.54229542996431], [-74.13011404715266, 40.54597572663567], [-74.13585712583111, 40.546451696357344], [-74.13861798499991, 40.548492383999886], [-74.12730057799996, 40.55729517199993], [-74.11813477599993, 40.550592692999906]]]}}, {\"id\": \"110\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5247524752475248, \"OBJECTID\": 111, \"Shape_Leng\": 0.0599308800658, \"Shape_Area\": 0.000208683276503, \"zone\": \"Green-Wood Cemetery\", \"LocationID\": 111, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99516890099986, 40.6596695219999], [-73.990188415, 40.657600817999906], [-73.98843135299977, 40.65925121199996], [-73.98179568699996, 40.65524609699991], [-73.98029115499993, 40.647297160999855], [-73.98905872499985, 40.64411924099995], [-74.00214474599989, 40.65295610699996], [-73.99516890099986, 40.6596695219999]]]}}, {\"id\": \"111\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3601569720753345, \"OBJECTID\": 112, \"Shape_Leng\": 0.107267875765, \"Shape_Area\": 0.000462068671429, \"zone\": \"Greenpoint\", \"LocationID\": 112, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.95439555417066, 40.73911477252253], [-73.94652352854791, 40.73692685395814], [-73.94719983367108, 40.73535517811412], [-73.94706532915383, 40.734401992819265], [-73.94557048457217, 40.736612303910405], [-73.94239443542429, 40.73542574994758], [-73.9381033496066, 40.729728519521366], [-73.93226852175046, 40.72817465801565], [-73.94841638599995, 40.71845553299995], [-73.94907816799986, 40.72254028699993], [-73.95477662900004, 40.72245950499994], [-73.95842611843923, 40.725400535832165], [-73.9616151883593, 40.725865563073995], [-73.96143187677845, 40.730965171647846], [-73.96226340423281, 40.732915551623485], [-73.96406507049178, 40.732779040653625], [-73.95837507902779, 40.73809694936863], [-73.95439555417066, 40.73911477252253]]], [[[-73.96236596889429, 40.72420906161418], [-73.96200744849497, 40.7239919013038], [-73.96207271921921, 40.72388030020147], [-73.96246790011047, 40.72413157960129], [-73.96236596889429, 40.72420906161418]]]]}}, {\"id\": \"112\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.32631335051620414, \"OBJECTID\": 113, \"Shape_Leng\": 0.032745350566, \"Shape_Area\": 5.79634566998e-05, \"zone\": \"Greenwich Village North\", \"LocationID\": 113, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99136388099988, 40.72755978299991], [-73.99657214499979, 40.729552233999904], [-73.99556245499996, 40.73071521599994], [-74.000066577, 40.7329292359999], [-73.99683993899998, 40.73736088899995], [-73.98990295999991, 40.7344347899999], [-73.99136388099988, 40.72755978299991]]]}}, {\"id\": \"113\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.32410967670135216, \"OBJECTID\": 114, \"Shape_Leng\": 0.0317270347711, \"Shape_Area\": 4.70207704945e-05, \"zone\": \"Greenwich Village South\", \"LocationID\": 114, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99257987399992, 40.724201554999915], [-74.00281726599992, 40.72836312899993], [-74.000066577, 40.7329292359999], [-73.99154832699986, 40.72708947099994], [-73.99257987399992, 40.724201554999915]]]}}, {\"id\": \"114\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.45588235294117646, \"OBJECTID\": 115, \"Shape_Leng\": 0.116169413964, \"Shape_Area\": 0.000373168991958, \"zone\": \"Grymes Hill/Clifton\", \"LocationID\": 115, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08069514799999, 40.62753689699988], [-74.08326107499997, 40.627194742999954], [-74.08496211899991, 40.624625232999925], [-74.08785147699994, 40.615243072999924], [-74.08720982799997, 40.61330697399994], [-74.07253551699989, 40.622464021999896], [-74.071134623, 40.62035994799992], [-74.0815547009999, 40.60629029699987], [-74.0977696879999, 40.61062359299988], [-74.10350925299994, 40.61641607499991], [-74.0922397139999, 40.62574291499989], [-74.08574928999992, 40.633955556999865], [-74.07972492199995, 40.62895068799986], [-74.08069514799999, 40.62753689699988]]]}}, {\"id\": \"115\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.45451658273218093, \"OBJECTID\": 116, \"Shape_Leng\": 0.0681164844265, \"Shape_Area\": 0.000260415337217, \"zone\": \"Hamilton Heights\", \"LocationID\": 116, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93924885699987, 40.82829635099989], [-73.94515456999994, 40.82087626499993], [-73.94808289799983, 40.82210802199994], [-73.95082880000002, 40.820827936999876], [-73.95954472552476, 40.82364494182928], [-73.95015521048293, 40.83439675940791], [-73.93864026699997, 40.82973887399988], [-73.93924885699987, 40.82829635099989]]]}}, {\"id\": \"116\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3061630218687873, \"OBJECTID\": 117, \"Shape_Leng\": 0.169885857987, \"Shape_Area\": 0.00090429427072, \"zone\": \"Hammels/Arverne\", \"LocationID\": 117, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7763584369479, 40.609655838242304], [-73.77412897394224, 40.60874659586809], [-73.77460294599074, 40.607417648882524], [-73.77484886546486, 40.60802483315967], [-73.77523851986173, 40.60855094300021], [-73.77600772152827, 40.60919075424813], [-73.77469363854065, 40.60430696481727], [-73.7818572569845, 40.59837293018506], [-73.78005493695551, 40.596861344908895], [-73.7749751364952, 40.60174817941184], [-73.77363244061033, 40.598153123466695], [-73.76809925802587, 40.59784597005487], [-73.76771811799993, 40.596261309999925], [-73.76654204999997, 40.599434441999904], [-73.76165797299986, 40.599973069999876], [-73.7633344969999, 40.59782287399992], [-73.76277845599988, 40.59475654399989], [-73.76440183099987, 40.59457168299989], [-73.76376223423699, 40.591439709016804], [-73.81730795916116, 40.58233475897], [-73.81544517899997, 40.58337349199993], [-73.81909407399996, 40.58783196099988], [-73.8176380045051, 40.58928579948519], [-73.81170344836342, 40.59159374663789], [-73.81149423585859, 40.592901326696776], [-73.81132554567534, 40.59174649614576], [-73.81081791874632, 40.59209026869603], [-73.81138593152033, 40.592442318454566], [-73.81140751227157, 40.592871257238706], [-73.81107287021051, 40.593022330572076], [-73.80991130015991, 40.59225371768247], [-73.80959170493088, 40.593580688618054], [-73.80577902839048, 40.59075949966858], [-73.8058134261234, 40.59186886150206], [-73.80430333508215, 40.59131887880457], [-73.8079775709933, 40.59415786866439], [-73.80604149807385, 40.595554555213546], [-73.8024384629031, 40.59264006383509], [-73.80234513845657, 40.59449597525689], [-73.80344017041448, 40.59443420917679], [-73.80321494251235, 40.59514564845063], [-73.80389507006726, 40.594990875367245], [-73.80331453729063, 40.59517956193947], [-73.80368468695728, 40.59519779018381], [-73.80486462646938, 40.5967336283515], [-73.80513406252756, 40.59660758897049], [-73.80524832538248, 40.596895032613546], [-73.80297943325748, 40.598844304553005], [-73.79233256449508, 40.59995439142583], [-73.78635332811808, 40.60319465903842], [-73.79034006550985, 40.59919167636005], [-73.79102536652763, 40.595527157215926], [-73.78973901889819, 40.59474366855456], [-73.7890682912148, 40.598019927382616], [-73.78350560905308, 40.60256343874268], [-73.78366696292244, 40.60532443693426], [-73.77927015444938, 40.60930736180752], [-73.7763584369479, 40.609655838242304]]]}}, {\"id\": \"117\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.47244094488188976, \"OBJECTID\": 118, \"Shape_Leng\": 0.243966217692, \"Shape_Area\": 0.00182693922711, \"zone\": \"Heartland Village/Todt Hill\", \"LocationID\": 118, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.10834417899996, 40.610234766999945], [-74.09555829799993, 40.61062477799989], [-74.09045495899996, 40.60909452599991], [-74.09271908099993, 40.60636222199992], [-74.09192827899989, 40.598504050999914], [-74.10041498999992, 40.59334309299988], [-74.10149482299994, 40.58934426099992], [-74.10790851599995, 40.58368563799992], [-74.1212571389999, 40.575254893999904], [-74.12708814699995, 40.57636910599994], [-74.13261022999991, 40.573530162999894], [-74.14418588199999, 40.57203740999988], [-74.14438744999991, 40.56970610099991], [-74.14789716899996, 40.57088728599995], [-74.15725974199995, 40.562553318999896], [-74.1649414939999, 40.56011179699991], [-74.16982582400001, 40.56109042099991], [-74.16842916199994, 40.58649792299988], [-74.16189940599996, 40.59529877099993], [-74.1481980769999, 40.588599680999934], [-74.13864458599996, 40.58906080499989], [-74.13091909399999, 40.598855047999905], [-74.12506115499997, 40.59818005599987], [-74.12059546299996, 40.601809757999874], [-74.11487058399989, 40.602136659999935], [-74.11742466500002, 40.609289008999916], [-74.12135179599994, 40.609765267999926], [-74.11508745399988, 40.61120943099993], [-74.10834417899996, 40.610234766999945]]]}}, {\"id\": \"118\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6160034790171777, \"OBJECTID\": 119, \"Shape_Leng\": 0.0835070355744, \"Shape_Area\": 0.000185779447534, \"zone\": \"Highbridge\", \"LocationID\": 119, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9247831879999, 40.84475266499989], [-73.91729224999992, 40.84510993999991], [-73.91794965199988, 40.842229570999976], [-73.91618344399991, 40.84233489999991], [-73.92802724899998, 40.82971953899987], [-73.93307251084072, 40.828156152518105], [-73.93314306709256, 40.83519412761651], [-73.92861574822331, 40.844676865029925], [-73.9247831879999, 40.84475266499989]]]}}, {\"id\": \"119\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5056526207605344, \"OBJECTID\": 120, \"Shape_Leng\": 0.0832612702158, \"Shape_Area\": 9.31462821313e-05, \"zone\": \"Highbridge Park\", \"LocationID\": 120, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92295449499989, 40.85885076199988], [-73.92227873064479, 40.85565498884376], [-73.93436121591056, 40.83620061961779], [-73.94034643299983, 40.8304578419999], [-73.93505508799984, 40.8393332359999], [-73.93556697199989, 40.841712527999924], [-73.92707982300001, 40.851432342999885], [-73.92466977999999, 40.856982610999886], [-73.92709951599986, 40.85827552099992], [-73.92499818399988, 40.86146373799991], [-73.92295449499989, 40.85885076199988]]]}}, {\"id\": \"120\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5180572851805728, \"OBJECTID\": 121, \"Shape_Leng\": 0.0969153373445, \"Shape_Area\": 0.000384563286473, \"zone\": \"Hillcrest/Pomonok\", \"LocationID\": 121, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79911195699995, 40.738206139999875], [-73.79584803299987, 40.7380746799999], [-73.79037661699995, 40.720103723999934], [-73.80780927699995, 40.715853895999906], [-73.80904032699983, 40.71991160499987], [-73.80452680199981, 40.721462545999884], [-73.80487958299983, 40.725049592999895], [-73.80652142399995, 40.7248730639999], [-73.80717042299993, 40.72826913299987], [-73.8111179729999, 40.72781641599994], [-73.81074426599986, 40.72626511299991], [-73.81510024099985, 40.72854342299991], [-73.81504998799986, 40.73894313399992], [-73.79911195699995, 40.738206139999875]]]}}, {\"id\": \"121\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4117032392894462, \"OBJECTID\": 122, \"Shape_Leng\": 0.0665318439446, \"Shape_Area\": 0.000226597104976, \"zone\": \"Hollis\", \"LocationID\": 122, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.75725671499985, 40.71813860199992], [-73.75047058199985, 40.70563993099994], [-73.76612489399999, 40.702334485999884], [-73.76825969099993, 40.709526367999956], [-73.771724447, 40.71031681899991], [-73.76943987099992, 40.711976892999864], [-73.77192122100001, 40.71568504499992], [-73.76664095199988, 40.717646613999875], [-73.765714472, 40.71639337899988], [-73.75832728699997, 40.71958208899985], [-73.75725671499985, 40.71813860199992]]]}}, {\"id\": \"122\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5869218500797448, \"OBJECTID\": 123, \"Shape_Leng\": 0.0834211389387, \"Shape_Area\": 0.000296445173366, \"zone\": \"Homecrest\", \"LocationID\": 123, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96285973999984, 40.61254948599989], [-73.96237947799978, 40.609986373999966], [-73.95859278499987, 40.61040303099988], [-73.95619313699986, 40.58776226899994], [-73.9603495359999, 40.58730628599994], [-73.9606798409999, 40.59159758199995], [-73.97115101799994, 40.59044244399989], [-73.97299433999983, 40.6088141419999], [-73.96429649899991, 40.60977642599992], [-73.96478112799986, 40.61233092199988], [-73.96285973999984, 40.61254948599989]]]}}, {\"id\": \"123\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4218880534670008, \"OBJECTID\": 124, \"Shape_Leng\": 0.127724410062, \"Shape_Area\": 0.000754837821479, \"zone\": \"Howard Beach\", \"LocationID\": 124, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.85096453700001, 40.671719510999885], [-73.83334819299995, 40.66677318699994], [-73.82859942399985, 40.657196972999884], [-73.82639242872236, 40.64832434880783], [-73.83133151403268, 40.65274755744263], [-73.82901417557953, 40.65610788445241], [-73.8317081441153, 40.65393117429689], [-73.83221698518575, 40.654460386292925], [-73.83149330103005, 40.65497982053967], [-73.83116100809627, 40.65591923373715], [-73.82937603410723, 40.65650633521598], [-73.82896778740296, 40.65720524792007], [-73.8291791973457, 40.65779977854427], [-73.83013028740389, 40.65840264372116], [-73.82970706838907, 40.658502513864484], [-73.82959986903019, 40.65865680691539], [-73.83014950104759, 40.6598086849573], [-73.83024427039376, 40.658349969348166], [-73.82909434825203, 40.65733421866331], [-73.83127112568108, 40.656003271494676], [-73.8318523847929, 40.65488503826691], [-73.83290309524602, 40.657483771623006], [-73.83194394578256, 40.65617142725338], [-73.832600363706, 40.65782659162732], [-73.83096907996668, 40.65823029417234], [-73.83338439472544, 40.65805185920798], [-73.83109184464118, 40.64848480675928], [-73.83561034795801, 40.64850568470053], [-73.8387360855594, 40.662454060021076], [-73.83978752612934, 40.660485998048], [-73.83592320073285, 40.645499790674954], [-73.84950699084165, 40.64413376567876], [-73.85183280737658, 40.64586005671262], [-73.85230349886227, 40.647771028550196], [-73.84939766406322, 40.65143487895031], [-73.85591932944749, 40.65130926166608], [-73.86136381950391, 40.65641475416531], [-73.85983363521494, 40.65697706322045], [-73.8596355866937, 40.65845777205655], [-73.86124943398261, 40.65872091007865], [-73.85976839588899, 40.65828193634574], [-73.86087597607579, 40.65690816207006], [-73.86317083299996, 40.65827651199994], [-73.85761052799985, 40.660108355999924], [-73.85842950899992, 40.66345335999987], [-73.85568461199988, 40.66386749199993], [-73.85763323099987, 40.67165619399989], [-73.86038937900003, 40.67126877499985], [-73.86131863599985, 40.6750212689999], [-73.85096453700001, 40.671719510999885]]], [[[-73.83032725337058, 40.65513280580328], [-73.83026601851493, 40.65542864678473], [-73.82988919856028, 40.655522822584075], [-73.82994556435834, 40.655424494353], [-73.83032725337058, 40.65513280580328]]], [[[-73.85975533686633, 40.65410417609493], [-73.86048308199986, 40.65465892799988], [-73.86098917376947, 40.65520976705256], [-73.85979242750741, 40.65447615737696], [-73.85975533686633, 40.65410417609493]]], [[[-73.85898002989568, 40.65267545791771], [-73.85975533686633, 40.65410417609493], [-73.85941022942576, 40.65386473792502], [-73.85898002989568, 40.65267545791771]]], [[[-73.85898002989568, 40.65267545791771], [-73.85731941515513, 40.65123709900783], [-73.85695870295893, 40.65058737348713], [-73.85898002989568, 40.65267545791771]]]]}}, {\"id\": \"124\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.29250241290585527, \"OBJECTID\": 125, \"Shape_Leng\": 0.0497322121757, \"Shape_Area\": 7.64293209892e-05, \"zone\": \"Hudson Sq\", \"LocationID\": 125, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00269315099992, 40.72831405499989], [-74.0053098609999, 40.721824191999865], [-74.0108123829999, 40.72578980299994], [-74.0119643202455, 40.72432986461987], [-74.01165326569942, 40.725871761851344], [-74.01520426531133, 40.726361949379644], [-74.011542018976, 40.72644734692706], [-74.01067264999996, 40.72911266599987], [-74.00269315099992, 40.72831405499989]]]}}, {\"id\": \"125\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5991133628879037, \"OBJECTID\": 126, \"Shape_Leng\": 0.12994872606, \"Shape_Area\": 0.0007032738273, \"zone\": \"Hunts Point\", \"LocationID\": 126, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88615037899993, 40.82730567799994], [-73.88753429599987, 40.82250933899995], [-73.8843985489999, 40.822965575999866], [-73.88244779599998, 40.81926832299992], [-73.87056806927927, 40.81208731762241], [-73.86811219140058, 40.806751832831786], [-73.87234684998835, 40.800174763010446], [-73.87814218819929, 40.801319290306495], [-73.87843295513449, 40.80270273469493], [-73.8850517233671, 40.80213090679797], [-73.89197929534278, 40.80638470897619], [-73.89128402413917, 40.80497628189916], [-73.89509836487399, 40.80686363917353], [-73.8952796287401, 40.805766184067906], [-73.90222284749719, 40.80494811309474], [-73.90307457199998, 40.80988061199988], [-73.89542858799986, 40.81584467199988], [-73.8970446669999, 40.8168018599999], [-73.89283653100001, 40.820974054999965], [-73.8958813589999, 40.82099257499987], [-73.89586019599982, 40.82263104599987], [-73.89320004999998, 40.82400818299992], [-73.89487088500002, 40.82393727199992], [-73.89483522899992, 40.82624684099993], [-73.88615037899993, 40.82730567799994]]]}}, {\"id\": \"126\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5110266159695818, \"OBJECTID\": 127, \"Shape_Leng\": 0.0600188855815, \"Shape_Area\": 0.000163195647796, \"zone\": \"Inwood\", \"LocationID\": 127, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.911224294065, 40.87302771883775], [-73.91043824727869, 40.871377478551786], [-73.91254775130287, 40.866514943004354], [-73.91589911645785, 40.86430803808506], [-73.91538673051149, 40.863084384200725], [-73.91950213660932, 40.85879113379047], [-73.92159014848224, 40.86007280002688], [-73.92035142875443, 40.858195220619145], [-73.92282845999992, 40.8568514089999], [-73.92872203199994, 40.86674266299988], [-73.92135975499993, 40.869551853999894], [-73.91845930899993, 40.873022052999936], [-73.9140098549999, 40.871160593999946], [-73.911224294065, 40.87302771883775]]], [[[-73.92151894059094, 40.85660117062315], [-73.92229611007241, 40.855663835114505], [-73.92234459387058, 40.85568233590737], [-73.92169841904696, 40.85667230185494], [-73.92151894059094, 40.85660117062315]]]]}}, {\"id\": \"127\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.47332421340629277, \"OBJECTID\": 128, \"Shape_Leng\": 0.0702090290681, \"Shape_Area\": 0.000207405982869, \"zone\": \"Inwood Hill Park\", \"LocationID\": 128, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.92359742020389, 40.878898712992644], [-73.92363695289211, 40.87881535593209], [-73.92367919311681, 40.878826074498626], [-73.92362973975199, 40.87896394312155], [-73.92359742020389, 40.878898712992644]]], [[[-73.9264055692112, 40.87762147653741], [-73.92258387170895, 40.87692253928837], [-73.921169296781, 40.87306939228805], [-73.91978312944534, 40.87360369743557], [-73.92090325515422, 40.87546616638258], [-73.91855365026461, 40.873180588115396], [-73.9175772988473, 40.87454867854727], [-73.91154281116535, 40.873253706161705], [-73.9140098549999, 40.871160593999946], [-73.91845930899993, 40.873022052999936], [-73.92135975499993, 40.869551853999894], [-73.92872203199994, 40.86674266299988], [-73.93220600531866, 40.870096445104515], [-73.9264055692112, 40.87762147653741]]]]}}, {\"id\": \"131\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.42499042413125365, \"OBJECTID\": 132, \"Shape_Leng\": 0.24547851707, \"Shape_Area\": 0.00203830095472, \"zone\": \"JFK Airport\", \"LocationID\": 132, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.8250346749999, 40.663580133999844], [-73.81496922499986, 40.66247266799991], [-73.80072889200001, 40.66526937699988], [-73.780755286, 40.66240660199992], [-73.7541108669999, 40.648697664999915], [-73.75522386899995, 40.64744446899989], [-73.7474246205621, 40.6413764680137], [-73.74849297657127, 40.63567172844184], [-73.7650077554797, 40.62941398784963], [-73.77077016777163, 40.62003118215498], [-73.77082069703974, 40.62328299797239], [-73.77704080084247, 40.62766646976843], [-73.77999420781757, 40.62668929803533], [-73.78419483298926, 40.6208926699551], [-73.78949653224412, 40.62274664610792], [-73.78264543417806, 40.630247968544396], [-73.79076956011139, 40.63386374895698], [-73.79222318018806, 40.63589008813999], [-73.81830607819735, 40.64638698303703], [-73.82182513624882, 40.649899795874894], [-73.82348381110747, 40.65536991271862], [-73.82227996040434, 40.659425007931176], [-73.8186953155868, 40.661105051055245], [-73.81143921922407, 40.66074562606795], [-73.81867189288894, 40.66213298725428], [-73.82338324351308, 40.6602057470252], [-73.82498070378553, 40.65543197118367], [-73.82290717700891, 40.64809344600474], [-73.82614483059669, 40.65008184094253], [-73.82639242822242, 40.64832434880782], [-73.82859942399985, 40.657196972999884], [-73.832705339, 40.665405772999854], [-73.8250346749999, 40.663580133999844]]], [[[-73.74661040472857, 40.638724973885545], [-73.74653979797041, 40.63871136266965], [-73.74654132522075, 40.63869678799164], [-73.74664302817305, 40.638662249807076], [-73.74661040472857, 40.638724973885545]]], [[[-73.74693721618013, 40.63755610186007], [-73.74694320619099, 40.637529202088494], [-73.74704913170525, 40.63754513439954], [-73.74704316342154, 40.637566426459216], [-73.74693721618013, 40.63755610186007]]], [[[-73.74712154262465, 40.63716227570799], [-73.74706485671244, 40.637160999268374], [-73.74699391228548, 40.63715064484292], [-73.74715781614569, 40.637060524028364], [-73.74712154262465, 40.63716227570799]]]]}}, {\"id\": \"128\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6921506328454607, \"OBJECTID\": 129, \"Shape_Leng\": 0.0932727273403, \"Shape_Area\": 0.00047398833179, \"zone\": \"Jackson Heights\", \"LocationID\": 129, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87586503899996, 40.76605165399989], [-73.87513874300004, 40.756633815999905], [-73.87365721799983, 40.756803485999946], [-73.86942458599987, 40.74915685199991], [-73.89175048799984, 40.74681172799989], [-73.89624245599988, 40.74871954499987], [-73.89984592699983, 40.75756215299985], [-73.89437624799997, 40.7655265849999], [-73.87608949699984, 40.77153690299987], [-73.87586503899996, 40.76605165399989]]]}}, {\"id\": \"129\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.48143639683505784, \"OBJECTID\": 130, \"Shape_Leng\": 0.142028320665, \"Shape_Area\": 0.000468322514327, \"zone\": \"Jamaica\", \"LocationID\": 130, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76962552399995, 40.71178631999993], [-73.771724447, 40.71031681899991], [-73.76844546499984, 40.709637026999886], [-73.77555451099995, 40.70607083499987], [-73.77230768699997, 40.70057150699989], [-73.77395659499993, 40.69980578099988], [-73.78034491200003, 40.70541241699986], [-73.78256582599995, 40.70542486399989], [-73.79239170299988, 40.7021665299999], [-73.79260506799987, 40.70043209999988], [-73.80642479399984, 40.690873857999904], [-73.80890216900004, 40.69256434899993], [-73.81063298299996, 40.69194738299988], [-73.81637824499992, 40.70245196399987], [-73.81152856899989, 40.70227287499991], [-73.81201248599983, 40.70594494499993], [-73.80694856299993, 40.70736709999989], [-73.8056477309999, 40.70467172799994], [-73.80165725599993, 40.705761474999846], [-73.80302414799988, 40.70813965699997], [-73.79870150699992, 40.71066337699988], [-73.79776810199988, 40.7091431879999], [-73.77192121999992, 40.715685043999905], [-73.76962552399995, 40.71178631999993]]]}}, {\"id\": \"1\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4090909090909091, \"OBJECTID\": 2, \"Shape_Leng\": 0.43346966679, \"Shape_Area\": 0.00486634037837, \"zone\": \"Jamaica Bay\", \"LocationID\": 2, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.82337597260663, 40.63898704717671], [-73.82107505533321, 40.62973377456623], [-73.81611483429545, 40.624604655173044], [-73.81891708039825, 40.62309026716333], [-73.81571468044605, 40.62148955151693], [-73.81777429834523, 40.61750112169565], [-73.81652681189122, 40.61494700958006], [-73.82141601599992, 40.615054762999925], [-73.8238246859766, 40.61149866495507], [-73.82575322105271, 40.614192081540075], [-73.83776702143372, 40.61552223141874], [-73.83496222645587, 40.61778030881177], [-73.83514355859735, 40.62548708279667], [-73.83342230809954, 40.626781983462095], [-73.83464377224975, 40.62742707942435], [-73.83115393179625, 40.63256052752952], [-73.83340607979225, 40.63850135589897], [-73.82591657926903, 40.63574741296666], [-73.82443032497055, 40.6362165210408], [-73.82419327293158, 40.63995327448778], [-73.82337597260663, 40.63898704717671]]], [[[-73.84721834832125, 40.63196568670605], [-73.8485899664393, 40.63013198680654], [-73.8585583634798, 40.63045828819559], [-73.8561374187591, 40.63508154601468], [-73.84667914840234, 40.63873171175714], [-73.84352007697278, 40.636503685327924], [-73.84721834832125, 40.63196568670605]]], [[[-73.79577948747226, 40.63215847659964], [-73.79811872226199, 40.6310375092611], [-73.79827615415297, 40.63125217936581], [-73.7907928429953, 40.63428204310243], [-73.79577948747226, 40.63215847659964]]], [[[-73.83623731597828, 40.631110861231726], [-73.84110205447426, 40.62455643651514], [-73.84445711462239, 40.62984779443998], [-73.83765174362401, 40.63283973295598], [-73.83623731597828, 40.631110861231726]]], [[[-73.81307233892618, 40.62926109964263], [-73.81352502122876, 40.62730617268596], [-73.81258556038132, 40.628371916183724], [-73.81129432207656, 40.624703779000406], [-73.81089990481607, 40.61621847134095], [-73.81679073502552, 40.62634891812118], [-73.81382899493154, 40.62613507764411], [-73.81627026043981, 40.62714580591711], [-73.8173057692762, 40.63062197379782], [-73.81506788441455, 40.63116405857793], [-73.81307233892618, 40.62926109964263]]], [[[-73.84734350666976, 40.62909473971626], [-73.84984630167418, 40.62622449074592], [-73.8472783772798, 40.626621669791945], [-73.84358010755503, 40.623178164184395], [-73.86364988794934, 40.617878773642374], [-73.86404106276488, 40.61920759671402], [-73.86151825485233, 40.619049218399226], [-73.86102883383386, 40.624138986994076], [-73.86347880947007, 40.6233886093158], [-73.86334955670225, 40.62592490877777], [-73.86041082270874, 40.62516691297755], [-73.8565055097758, 40.629097353726586], [-73.8539654419115, 40.62761094389788], [-73.85436931062769, 40.62881767654389], [-73.84734350666976, 40.62909473971626]]], [[[-73.87084603194215, 40.61696742645758], [-73.8782477932424, 40.615444608331536], [-73.87481386865049, 40.62358718918363], [-73.86684384142526, 40.62782924286026], [-73.86498193495926, 40.625660706041025], [-73.86626127665735, 40.618456348023955], [-73.87084603194215, 40.61696742645758]]], [[[-73.79783533398513, 40.627408716571516], [-73.79566256726108, 40.626114004327796], [-73.79921493427759, 40.6240847129885], [-73.7936292748135, 40.62515647619494], [-73.79809093107174, 40.62306294824432], [-73.79992057632684, 40.61857524016598], [-73.80172498640165, 40.61996700873167], [-73.80128049504093, 40.615879839747535], [-73.80270343867807, 40.61358065232171], [-73.80508906012292, 40.61343256041943], [-73.80476592616557, 40.61580099137086], [-73.80764116896117, 40.62119965469225], [-73.80698502381061, 40.623871998829976], [-73.8001117714566, 40.626927441987085], [-73.80092841565032, 40.627524104895656], [-73.79783533398513, 40.627408716571516]]], [[[-73.77348616522504, 40.625085118163526], [-73.77381024409397, 40.624925596730954], [-73.77383258726536, 40.624944848253456], [-73.77348560409436, 40.62511314810922], [-73.77348616522504, 40.625085118163526]]], [[[-73.77172760292548, 40.62358502801516], [-73.7720313143135, 40.62299121290296], [-73.77218824028017, 40.622995793795134], [-73.77511891254433, 40.619233847487365], [-73.77518045935567, 40.61926471788484], [-73.77172760292548, 40.62358502801516]]], [[[-73.79162058913477, 40.62231334348308], [-73.78419483298926, 40.6208926699551], [-73.78446087067887, 40.617027835129704], [-73.78070887195283, 40.614026809005864], [-73.79139721011913, 40.60761085524994], [-73.79480633875043, 40.61037836048121], [-73.80080125919478, 40.611933904481596], [-73.7946062540462, 40.62257170055433], [-73.79162058913477, 40.62231334348308]]], [[[-73.76670827781236, 40.61491086618553], [-73.76825288003378, 40.614877725169315], [-73.77397692018825, 40.61600357567147], [-73.76873454755717, 40.62090086822023], [-73.76745926975256, 40.620511322031255], [-73.76670827781236, 40.61491086618553]]], [[[-73.84304421296773, 40.617737696390435], [-73.84454193473684, 40.614867921675135], [-73.8409994253963, 40.613029126216134], [-73.83825667290532, 40.613847216216406], [-73.8407944240835, 40.60564326638265], [-73.8471480968766, 40.60361424219198], [-73.85179402215192, 40.60689716858368], [-73.84779013405834, 40.60923538526278], [-73.84614804513843, 40.61772842036632], [-73.84304421296773, 40.617737696390435]]], [[[-73.85523067631884, 40.61613145335797], [-73.85530325465119, 40.6131454135728], [-73.84973986306498, 40.61515605814331], [-73.85010857555366, 40.61021219334599], [-73.8551559228834, 40.606120513001486], [-73.85708638173246, 40.60646824293355], [-73.85924855452187, 40.61125483087423], [-73.85648360749232, 40.61319069486381], [-73.85858056782772, 40.61691708722184], [-73.85523067631884, 40.61613145335797]]], [[[-73.814090115668, 40.614466920099765], [-73.81503638494628, 40.6129762137494], [-73.81615626666235, 40.61294211575494], [-73.8164315038978, 40.614282334964926], [-73.814090115668, 40.614466920099765]]], [[[-73.81496830662114, 40.61042065495561], [-73.81520720849484, 40.60981673379666], [-73.81559858789666, 40.61067251616471], [-73.81450261566708, 40.61105847350542], [-73.81496830662114, 40.61042065495561]]], [[[-73.86966442958492, 40.606540123412906], [-73.87109330768071, 40.604694173994694], [-73.87300122724774, 40.61055545845987], [-73.8710387011171, 40.60972358252655], [-73.86966442958492, 40.606540123412906]]], [[[-73.79420172624032, 40.607730675613354], [-73.79352152715475, 40.60632759139594], [-73.79509002319158, 40.607271060105596], [-73.79424799033224, 40.60590038003164], [-73.79696327795033, 40.60411784876779], [-73.80428530678597, 40.604519279530564], [-73.80520676583323, 40.60673953810017], [-73.80159206744115, 40.61023096250127], [-73.79420172624032, 40.607730675613354]]], [[[-73.83244207358176, 40.605929449536745], [-73.83587601623029, 40.605636907545566], [-73.83610156628845, 40.60736279382329], [-73.83252192275347, 40.610384993603816], [-73.83032947905777, 40.60848787991649], [-73.83045815392722, 40.606669201403214], [-73.83244207358176, 40.605929449536745]]], [[[-73.82607472604538, 40.60843779954612], [-73.82771104235977, 40.608398668130704], [-73.82805077260065, 40.61004629686527], [-73.82688326414805, 40.6095966780694], [-73.82607472604538, 40.60843779954612]]], [[[-73.86702399475149, 40.60806817876942], [-73.86874317559169, 40.60703517927931], [-73.86868468826752, 40.608241158183915], [-73.86702399475149, 40.60806817876942]]], [[[-73.82718282107048, 40.607919778091194], [-73.82590092720767, 40.60646593064943], [-73.82861671457653, 40.60583848413761], [-73.82815085191197, 40.60795028983291], [-73.82718282107048, 40.607919778091194]]], [[[-73.81569926225602, 40.6064570673649], [-73.81546066483351, 40.60642813700742], [-73.81552816722274, 40.606163069945154], [-73.81552536723665, 40.60640738028192], [-73.81569926225602, 40.6064570673649]]], [[[-73.81339665223344, 40.60436407224534], [-73.80986333857152, 40.60189579000784], [-73.81157550345523, 40.59814119925956], [-73.81501471568424, 40.60205629156354], [-73.81339665223344, 40.60436407224534]]], [[[-73.86702905185578, 40.603743388853935], [-73.8676819415421, 40.603070766677334], [-73.8683080846461, 40.603139161146146], [-73.86793505372694, 40.60396113545295], [-73.86702905185578, 40.603743388853935]]], [[[-73.85243335807012, 40.59733339779527], [-73.86044420534076, 40.596471905596275], [-73.86335999147606, 40.600322202771196], [-73.86112814224546, 40.60255033107426], [-73.85482595782968, 40.601867195353634], [-73.85076748001559, 40.598839443837406], [-73.85243335807012, 40.59733339779527]]], [[[-73.87019491489939, 40.598996405901886], [-73.87108141811476, 40.598901147470094], [-73.87049411914771, 40.602604446204595], [-73.87026280564918, 40.60168122225861], [-73.87019491489939, 40.598996405901886]]], [[[-73.8231174097477, 40.60007949252535], [-73.82315293179705, 40.599976662824794], [-73.82317356168161, 40.59997800515576], [-73.82318015964296, 40.60008384902813], [-73.8231174097477, 40.60007949252535]]], [[[-73.82292757811203, 40.60006142816309], [-73.822978576463, 40.600043707224835], [-73.82303195183424, 40.59993051694214], [-73.82304007849375, 40.60006807478574], [-73.82292757811203, 40.60006142816309]]], [[[-73.82445629476595, 40.59911583868227], [-73.82395871079126, 40.59887689683851], [-73.82477088992478, 40.5983918438742], [-73.82453084258555, 40.59911949829181], [-73.82445629476595, 40.59911583868227]]], [[[-73.86952707548932, 40.598713750354406], [-73.86967070711478, 40.59668856362211], [-73.87020899836229, 40.59695423278937], [-73.86952707548932, 40.598713750354406]]], [[[-73.83668274106707, 40.5949466970158], [-73.8325749810638, 40.592118209113956], [-73.83422777067327, 40.59137381382269], [-73.83253569485043, 40.59021893579835], [-73.83495422767699, 40.589728155618005], [-73.83347866645357, 40.589368748710896], [-73.83719541135855, 40.58969911736292], [-73.8398907019092, 40.59280021149386], [-73.84385894083933, 40.59342531196059], [-73.83902964248091, 40.59625348474866], [-73.83668274106707, 40.5949466970158]]], [[[-73.86706149472118, 40.5820879767934], [-73.87000313642912, 40.582636614815954], [-73.87115922675414, 40.5868738879673], [-73.86635052164085, 40.591892676564676], [-73.86327273492819, 40.59090807569255], [-73.86125299377363, 40.58798808592123], [-73.86327471071942, 40.58387684853184], [-73.86706149472118, 40.5820879767934]]]]}}, {\"id\": \"130\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.572068707991038, \"OBJECTID\": 131, \"Shape_Leng\": 0.116547120922, \"Shape_Area\": 0.000423696152789, \"zone\": \"Jamaica Estates\", \"LocationID\": 131, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.77732562899998, 40.722392700999926], [-73.76924074899993, 40.72590270099988], [-73.75669771299987, 40.7262287109999], [-73.7523343839998, 40.72026246099993], [-73.79303800099989, 40.71071107499988], [-73.79358570299998, 40.714128677999874], [-73.790302674, 40.7208914649999], [-73.79174472699992, 40.725788849999944], [-73.77757506899988, 40.73019327299987], [-73.7752338719998, 40.725237532999884], [-73.77803765999981, 40.723284268999855], [-73.77732562899998, 40.722392700999926]]]}}, {\"id\": \"132\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.47491974317817015, \"OBJECTID\": 133, \"Shape_Leng\": 0.065134863162, \"Shape_Area\": 0.000157188987507, \"zone\": \"Kensington\", \"LocationID\": 133, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97819074399992, 40.64752599199992], [-73.97777380600002, 40.645341876999964], [-73.974642245, 40.644813218999914], [-73.97084113799984, 40.64637857099989], [-73.96803868499985, 40.636650984999925], [-73.97029036199991, 40.63640634099988], [-73.96986869299992, 40.63419467499989], [-73.97604935699995, 40.63127590599986], [-73.9768886119999, 40.6356748619999], [-73.97911188699993, 40.63544038599991], [-73.98042057899988, 40.64238522699991], [-73.98299419199994, 40.64132499699987], [-73.98834986699991, 40.64456224399992], [-73.97819074399992, 40.64752599199992]]]}}, {\"id\": \"133\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5255583126550869, \"OBJECTID\": 134, \"Shape_Leng\": 0.0695878223284, \"Shape_Area\": 0.000202375182143, \"zone\": \"Kew Gardens\", \"LocationID\": 134, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82288529599995, 40.71192805799993], [-73.81637824499992, 40.70245196399987], [-73.82245890399994, 40.70365553099989], [-73.83152109599999, 40.70138962899994], [-73.83460244999996, 40.70392746699988], [-73.83661882199989, 40.70342001799988], [-73.83809584699989, 40.70616458099993], [-73.83714702999997, 40.70795704599989], [-73.84314625299982, 40.70922345599992], [-73.82591945199992, 40.71598987599992], [-73.82288529599995, 40.71192805799993]]]}}, {\"id\": \"134\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5858644859813084, \"OBJECTID\": 135, \"Shape_Leng\": 0.121793045659, \"Shape_Area\": 0.000467912240542, \"zone\": \"Kew Gardens Hills\", \"LocationID\": 135, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83126834299986, 40.74026227599991], [-73.8294675199999, 40.73702448299991], [-73.8243501109999, 40.73975360599992], [-73.81497208299996, 40.738560715999874], [-73.81510024099985, 40.72854342299991], [-73.81074426599986, 40.72626511299991], [-73.8111179729999, 40.72781641599994], [-73.80717042299993, 40.72826913299987], [-73.80652142399995, 40.7248730639999], [-73.80487958299983, 40.725049592999895], [-73.80452680199981, 40.721462545999884], [-73.80904032699983, 40.71991160499987], [-73.80780927699995, 40.715853895999906], [-73.82018883899984, 40.71673522399984], [-73.81732944799997, 40.717660456999916], [-73.82531644999993, 40.71727569999994], [-73.8273090039999, 40.72464607699988], [-73.83073214599987, 40.72654932799991], [-73.83635625099998, 40.73500859199989], [-73.83564585599999, 40.738491010999894], [-73.83699037399995, 40.74209330099992], [-73.83586131899989, 40.743227984999926], [-73.83113122799989, 40.74233799999994], [-73.83126834299986, 40.74026227599991]]]}}, {\"id\": \"135\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6671575846833578, \"OBJECTID\": 136, \"Shape_Leng\": 0.0906287154122, \"Shape_Area\": 0.000148524994129, \"zone\": \"Kingsbridge Heights\", \"LocationID\": 136, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89996501599988, 40.86801852399986], [-73.89891854299988, 40.86965857399992], [-73.89656060599998, 40.868565199999935], [-73.90486847199993, 40.8586932249999], [-73.90777320599983, 40.85960753799984], [-73.90976165899988, 40.857182611999896], [-73.91328437100006, 40.859618600999916], [-73.90908822499988, 40.8623996959999], [-73.91422823999919, 40.862490790359615], [-73.91044282747865, 40.866527868551835], [-73.90746489699994, 40.87354735399985], [-73.90452070499992, 40.87249521099988], [-73.90318667499999, 40.86914899099988], [-73.89996501599988, 40.86801852399986]]]}}, {\"id\": \"136\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.321364184145939, \"OBJECTID\": 137, \"Shape_Leng\": 0.0461076242707, \"Shape_Area\": 0.000116455098947, \"zone\": \"Kips Bay\", \"LocationID\": 137, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97219641647804, 40.74040257875022], [-73.97248994174741, 40.73580328010761], [-73.97446647047282, 40.736278456829886], [-73.9735447289197, 40.73494432655072], [-73.98266497499989, 40.73949733299987], [-73.97812378199983, 40.745727172999885], [-73.97167921200406, 40.74305446402989], [-73.97219641647804, 40.74040257875022]]]}}, {\"id\": \"137\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.2660254049104935, \"OBJECTID\": 138, \"Shape_Leng\": 0.107466933508, \"Shape_Area\": 0.000536797294367, \"zone\": \"LaGuardia Airport\", \"LocationID\": 138, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.8712485624251, 40.78603815125499], [-73.86918113468879, 40.78594348978492], [-73.86847981674143, 40.78369082919537], [-73.87255732673727, 40.78082112154256], [-73.85505106271819, 40.772195389989086], [-73.8582051105021, 40.77032345647716], [-73.85624565746174, 40.768885714198944], [-73.86268864176631, 40.76687591050486], [-73.86043337809801, 40.76386888243776], [-73.86728193799998, 40.76965000299989], [-73.87223577699984, 40.77169565899993], [-73.87743339799992, 40.77120025499991], [-73.88770661799987, 40.76683859799995], [-73.88945486901865, 40.77353295106357], [-73.88391627963797, 40.774132606591536], [-73.88493466381185, 40.77993691280691], [-73.87870562518874, 40.78058590995002], [-73.878858374856, 40.78239817868118], [-73.87974777853563, 40.782836564207], [-73.8797016367226, 40.78288281807235], [-73.8751575489337, 40.781509449653285], [-73.8712485624251, 40.78603815125499]]], [[[-73.8728719590193, 40.785975027902346], [-73.87282839783694, 40.78595446173833], [-73.8727740594054, 40.78594192145881], [-73.87280920016116, 40.78590310909999], [-73.8728719590193, 40.785975027902346]]], [[[-73.89008921683674, 40.77362643576237], [-73.89034195999996, 40.77323760499998], [-73.89183244902246, 40.77488019994431], [-73.89017369508576, 40.77785081277782], [-73.89008921683674, 40.77362643576237]]]]}}, {\"id\": \"138\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.541795665634675, \"OBJECTID\": 139, \"Shape_Leng\": 0.0932997884483, \"Shape_Area\": 0.000446669880154, \"zone\": \"Laurelton\", \"LocationID\": 139, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.74125253099989, 40.68745783899989], [-73.72718272499995, 40.68359732999987], [-73.74117817299991, 40.666402953999885], [-73.75763129599993, 40.66645115699992], [-73.75636239300002, 40.67242923699999], [-73.75857198899986, 40.67263733099987], [-73.76031463999993, 40.67510997099984], [-73.75576169199995, 40.67588590899988], [-73.747551754, 40.689347336999944], [-73.74125253099989, 40.68745783899989]]]}}, {\"id\": \"139\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3668911149937692, \"OBJECTID\": 140, \"Shape_Leng\": 0.0475842911325, \"Shape_Area\": 0.000114203907779, \"zone\": \"Lenox Hill East\", \"LocationID\": 140, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95854329689507, 40.758495622967885], [-73.96196573699997, 40.759619460999865], [-73.95268514999995, 40.772339404999926], [-73.9474897547817, 40.77011515400953], [-73.95854329689507, 40.758495622967885]]]}}, {\"id\": \"140\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3792365812102217, \"OBJECTID\": 141, \"Shape_Leng\": 0.0415144638712, \"Shape_Area\": 7.66545579019e-05, \"zone\": \"Lenox Hill West\", \"LocationID\": 141, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96177668399997, 40.75987971599993], [-73.9665833839999, 40.76155093499988], [-73.95729409999986, 40.7742835549999], [-73.95268514999995, 40.772339404999926], [-73.96177668399997, 40.75987971599993]]]}}, {\"id\": \"141\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.38368754613809913, \"OBJECTID\": 142, \"Shape_Leng\": 0.0381758942321, \"Shape_Area\": 7.56537920738e-05, \"zone\": \"Lincoln Square East\", \"LocationID\": 142, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98147898799989, 40.7677250589999], [-73.98806289599996, 40.769790799999925], [-73.980673654, 40.77992518199991], [-73.97499744000007, 40.77753254599988], [-73.98147898799989, 40.7677250589999]]]}}, {\"id\": \"142\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.35632808990282144, \"OBJECTID\": 143, \"Shape_Leng\": 0.0541798538849, \"Shape_Area\": 0.00015109426901, \"zone\": \"Lincoln Square West\", \"LocationID\": 143, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.981627874, 40.780328934999936], [-73.980673654, 40.77992518199991], [-73.98806289599996, 40.769790799999925], [-73.99624065194922, 40.77378979119886], [-73.99393587681126, 40.773179512586104], [-73.98886861739992, 40.779692922911416], [-73.9915510592274, 40.779574821437386], [-73.98812746184332, 40.78140179672363], [-73.981627874, 40.780328934999936]]]}}, {\"id\": \"143\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.34628725791556436, \"OBJECTID\": 144, \"Shape_Leng\": 0.0276201668505, \"Shape_Area\": 4.74789670369e-05, \"zone\": \"Little Italy/NoLiTa\", \"LocationID\": 144, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99580912300002, 40.71664465899988], [-74.00188406299992, 40.71939757099991], [-73.9967717579999, 40.7254319439999], [-73.99260322299989, 40.724136449999925], [-73.99580912300002, 40.71664465899988]]]}}, {\"id\": \"144\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.46636295729317484, \"OBJECTID\": 145, \"Shape_Leng\": 0.114313844981, \"Shape_Area\": 0.000346641733797, \"zone\": \"Long Island City/Hunters Point\", \"LocationID\": 145, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94345411699997, 40.75205360799992], [-73.93663681899982, 40.748204093999895], [-73.94419561599983, 40.742882145999914], [-73.93726110699994, 40.745176490999896], [-73.9386966449998, 40.73814081699989], [-73.94140886345744, 40.73929957085316], [-73.93866991753185, 40.74253408202486], [-73.9400192297566, 40.74320827221074], [-73.94159470030033, 40.73986412616895], [-73.94548236590111, 40.738207031293705], [-73.95380883920839, 40.73982029297784], [-73.96256114655104, 40.73868995735329], [-73.95978468552276, 40.74349771658391], [-73.96064863700711, 40.74413031761407], [-73.9587753700448, 40.74455559986341], [-73.95965229755977, 40.745216588173435], [-73.95849389303115, 40.74498786491623], [-73.9567945329049, 40.748839516955584], [-73.95318382492853, 40.74773481196097], [-73.9564856879466, 40.74911169015581], [-73.95080763809156, 40.75526368005708], [-73.94345411699997, 40.75205360799992]]]}}, {\"id\": \"145\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.48634595385741247, \"OBJECTID\": 146, \"Shape_Leng\": 0.0471405215717, \"Shape_Area\": 0.000104638923425, \"zone\": \"Long Island City/Queens Plaza\", \"LocationID\": 146, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93675697199996, 40.74914697199991], [-73.944052652, 40.75231325999992], [-73.93679091699993, 40.75988288099991], [-73.92415699999985, 40.753961141999845], [-73.92562743799989, 40.752137243999904], [-73.93383269399993, 40.7516047019999], [-73.93675697199996, 40.74914697199991]]]}}, {\"id\": \"146\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.7029177718832891, \"OBJECTID\": 147, \"Shape_Leng\": 0.0587654949268, \"Shape_Area\": 0.000106418172757, \"zone\": \"Longwood\", \"LocationID\": 147, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89515277899991, 40.82783428999991], [-73.89487088500002, 40.82393727199992], [-73.89320004999998, 40.82400818299992], [-73.89586019599982, 40.82263104599987], [-73.8958813589999, 40.82099257499987], [-73.89290156199988, 40.82081122299989], [-73.8970446669999, 40.8168018599999], [-73.89542858799986, 40.81584467199988], [-73.90307457199998, 40.80988061199988], [-73.9044655149999, 40.81228195999994], [-73.9006114109999, 40.822108807999875], [-73.90198758599993, 40.82244042299992], [-73.90093510699997, 40.82792810199992], [-73.89515277899991, 40.82783428999991]]]}}, {\"id\": \"147\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.31510740350687577, \"OBJECTID\": 148, \"Shape_Leng\": 0.0391305015633, \"Shape_Area\": 6.97489921327e-05, \"zone\": \"Lower East Side\", \"LocationID\": 148, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98447731699999, 40.72023423899987], [-73.98675136699994, 40.71959284599997], [-73.99022012699989, 40.71440493999988], [-73.99750445299988, 40.71406913199995], [-73.99260322199994, 40.72413644999987], [-73.98382387299982, 40.72147287199987], [-73.98447731699999, 40.72023423899987]]]}}, {\"id\": \"148\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.48792270531400966, \"OBJECTID\": 149, \"Shape_Leng\": 0.0836805821837, \"Shape_Area\": 0.000270639618955, \"zone\": \"Madison\", \"LocationID\": 149, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94405560899996, 40.61199284199993], [-73.94231175899985, 40.60826819699985], [-73.93934703399997, 40.610187140999884], [-73.93753749399987, 40.60855738999986], [-73.93972613899992, 40.60713291499995], [-73.93447158899997, 40.60123151199989], [-73.94288954099984, 40.600318649999934], [-73.94248473299992, 40.598200364999855], [-73.956927197, 40.596612075999865], [-73.95859278499987, 40.61040303099988], [-73.94978380499997, 40.611371749999854], [-73.95026514899993, 40.61392435199983], [-73.9448277099998, 40.61604620899986], [-73.94405560899996, 40.61199284199993]]]}}, {\"id\": \"149\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6063348416289592, \"OBJECTID\": 150, \"Shape_Leng\": 0.089331491183, \"Shape_Area\": 0.000332568036088, \"zone\": \"Manhattan Beach\", \"LocationID\": 150, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9301907789999, 40.58508559299986], [-73.9319281118675, 40.58286976893519], [-73.95349322048675, 40.58298819826098], [-73.94274062592493, 40.58099558955925], [-73.93258006568031, 40.58126474120341], [-73.93106028107997, 40.57626592009687], [-73.9523584672133, 40.574274685845204], [-73.95414287899986, 40.58310613999989], [-73.95900739999992, 40.58283812799991], [-73.96025825899996, 40.58502930799994], [-73.9344471859999, 40.586652074999925], [-73.9301907789999, 40.58508559299986]]]}}, {\"id\": \"150\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.398524774850972, \"OBJECTID\": 151, \"Shape_Leng\": 0.0548899410999, \"Shape_Area\": 0.000128848901918, \"zone\": \"Manhattan Valley\", \"LocationID\": 151, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96371096499993, 40.7930129339999], [-73.97639951915279, 40.79827321034625], [-73.97287179090726, 40.8033561875739], [-73.96004456499999, 40.79804123499991], [-73.96371096499993, 40.7930129339999]]]}}, {\"id\": \"151\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.47001506341917787, \"OBJECTID\": 152, \"Shape_Leng\": 0.0579094768782, \"Shape_Area\": 0.000146637644064, \"zone\": \"Manhattanville\", \"LocationID\": 152, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95357262899982, 40.82197589199988], [-73.94607828700005, 40.821263215999906], [-73.95210125199992, 40.811442853999935], [-73.96203105797969, 40.81808633074085], [-73.95881892233997, 40.82151852484672], [-73.9596033246027, 40.82299468678336], [-73.95793904909117, 40.82277435354731], [-73.9595089796959, 40.82359578895142], [-73.95357262899982, 40.82197589199988]]]}}, {\"id\": \"152\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6289562289562289, \"OBJECTID\": 153, \"Shape_Leng\": 0.024737239022, \"Shape_Area\": 3.24684964817e-05, \"zone\": \"Marble Hill\", \"LocationID\": 153, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90666530999995, 40.8757156089999], [-73.90893235220783, 40.87215734798524], [-73.91578609320084, 40.87571718248032], [-73.91033193599998, 40.879038046999895], [-73.90666530999995, 40.8757156089999]]]}}, {\"id\": \"153\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.40897097625329815, \"OBJECTID\": 154, \"Shape_Leng\": 0.22733157076, \"Shape_Area\": 0.00178968692433, \"zone\": \"Marine Park/Floyd Bennett Field\", \"LocationID\": 154, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.93531145199985, 40.60886003999987], [-73.92986880199989, 40.60507134099995], [-73.92624885099994, 40.60741352199989], [-73.92444144399987, 40.60579356899993], [-73.91970928699986, 40.607302162999886], [-73.91331444593793, 40.60225844819831], [-73.90903689530528, 40.60267734322018], [-73.91073123013267, 40.60202137602758], [-73.90894225138031, 40.60160704069869], [-73.8839804295303, 40.60556651972071], [-73.87671672482064, 40.584914430200875], [-73.88174676983894, 40.5791588072124], [-73.89552381007165, 40.576769818436354], [-73.89903425420563, 40.587676171745805], [-73.90426874447125, 40.58691226298856], [-73.90651874663754, 40.588025473892536], [-73.91148171990072, 40.586078718712315], [-73.91204234015626, 40.594506954288846], [-73.91513297459494, 40.59878111593029], [-73.92151689981944, 40.60192365414108], [-73.92457042932716, 40.599723602732396], [-73.92950671202776, 40.60270923350431], [-73.92927458471804, 40.60399357480484], [-73.93117476161734, 40.603850406434695], [-73.93108919528449, 40.601722258264104], [-73.91454059121104, 40.58994422917544], [-73.91819087799986, 40.58660685499996], [-73.92956858299992, 40.5968202649999], [-73.92813139699999, 40.59777792799987], [-73.93972613899992, 40.60713291499995], [-73.93531145199985, 40.60886003999987]]], [[[-73.91990064335972, 40.59960052259278], [-73.91626393108346, 40.59816113731273], [-73.91295122631982, 40.59205432996962], [-73.91524478304683, 40.59158750817847], [-73.92182277731894, 40.59733293287253], [-73.92213049353394, 40.598662596177306], [-73.91990064335972, 40.59960052259278]]], [[[-73.91513331918175, 40.58626938540072], [-73.91282142952436, 40.58589683033704], [-73.91242103069457, 40.58339278402727], [-73.91729861410717, 40.58319080174952], [-73.91191058860355, 40.58296337725262], [-73.9114996013066, 40.58181878146464], [-73.92443604094977, 40.58355820344426], [-73.93184655018902, 40.58294105529734], [-73.9320424499999, 40.5842339599999], [-73.9300961629999, 40.584502075999886], [-73.93139947599984, 40.58607989499989], [-73.92101255523748, 40.58457501880612], [-73.91513331918175, 40.58626938540072]]]]}}, {\"id\": \"154\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5386199794026777, \"OBJECTID\": 155, \"Shape_Leng\": 0.169808473032, \"Shape_Area\": 0.000868261018498, \"zone\": \"Marine Park/Mill Basin\", \"LocationID\": 155, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90464209220775, 40.62513319188153], [-73.90546206137252, 40.62577481968345], [-73.9022870793228, 40.624409853930224], [-73.90071952540437, 40.62163492049855], [-73.89644123346895, 40.621394756190135], [-73.89586112894177, 40.614518137494784], [-73.89095414564672, 40.61283404731705], [-73.89394013317667, 40.61118142665126], [-73.88971513183803, 40.611598996904085], [-73.89334621420008, 40.606743446234645], [-73.89986880363854, 40.605524640544004], [-73.90133303458649, 40.611717882905985], [-73.90249392985292, 40.611814999032276], [-73.90119253465456, 40.61056381754662], [-73.90132962659396, 40.61048104310328], [-73.90879088017641, 40.617436396721416], [-73.90797306390525, 40.615736844061175], [-73.90959740910328, 40.616392669799325], [-73.90225859966276, 40.61042277259991], [-73.90177669809985, 40.605674686555815], [-73.90524300992108, 40.60491009032442], [-73.90863959535422, 40.606658722047854], [-73.90803306097814, 40.60535860211721], [-73.9098515484547, 40.605999722177685], [-73.9083253683517, 40.60407337391859], [-73.91365119759595, 40.60398251383695], [-73.91708838503304, 40.608820854898525], [-73.91449542205973, 40.61165322366488], [-73.91428072663338, 40.61493881616294], [-73.91662804371141, 40.6134435892654], [-73.91573181927129, 40.61183084451381], [-73.91811036679019, 40.61020839405809], [-73.91813814889971, 40.607645386907706], [-73.91935398484114, 40.60804292228766], [-73.91825269092949, 40.6072293506405], [-73.91950358918922, 40.60797490397521], [-73.91794980550603, 40.60620925217101], [-73.92158429099997, 40.607639084999875], [-73.92444144399987, 40.60579356899993], [-73.92624885099994, 40.60741352199989], [-73.92986880199989, 40.60507134099995], [-73.936046277, 40.609514868999966], [-73.93753749399987, 40.60855738999986], [-73.93934703399997, 40.610187140999884], [-73.94231175899985, 40.60826819699985], [-73.94354698000001, 40.6093465549999], [-73.9448277099998, 40.61604620899986], [-73.93686699599992, 40.620253280999904], [-73.92629604599998, 40.61311679899991], [-73.92640044599996, 40.61447853499988], [-73.9205269409999, 40.61394269499987], [-73.91690302699992, 40.61627422899993], [-73.91860649599992, 40.632090414999894], [-73.90464209220775, 40.62513319188153]]]}}, {\"id\": \"155\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.425414364640884, \"OBJECTID\": 156, \"Shape_Leng\": 0.144476890476, \"Shape_Area\": 0.0010521217443, \"zone\": \"Mariners Harbor\", \"LocationID\": 156, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-74.17989568340026, 40.64526879647802], [-74.1795871372141, 40.64191768346404], [-74.17918728067563, 40.64298753999647], [-74.1763578661754, 40.64232722056289], [-74.17436572943608, 40.64513625864542], [-74.17133587281808, 40.642855765052], [-74.17298798800311, 40.640489105066244], [-74.17105017239898, 40.642593819129914], [-74.1660998989604, 40.642406587001155], [-74.1658011075589, 40.64122451144941], [-74.16494139041178, 40.64220899988932], [-74.16519041758224, 40.64064230577938], [-74.16167534694229, 40.640610784766764], [-74.1622846525209, 40.63876565277233], [-74.16034819993155, 40.63844372645346], [-74.15986410465037, 40.639927823692716], [-74.1592271162294, 40.637796119756985], [-74.15900957887435, 40.63882257790974], [-74.15718782782525, 40.63795080522895], [-74.15710126690709, 40.63927771860018], [-74.15707077305574, 40.63794713564092], [-74.15482932461856, 40.63772385541876], [-74.15469263864514, 40.63919396811424], [-74.15459670163321, 40.63734334406899], [-74.15279703511403, 40.637132754852544], [-74.15124827966567, 40.63940934288591], [-74.14855815092196, 40.63743783752723], [-74.14877792005885, 40.6388762511277], [-74.14312802471521, 40.63968037306379], [-74.14752618999992, 40.62938928799988], [-74.14554071999994, 40.62783252199995], [-74.14650354399993, 40.62420010099991], [-74.15231170399994, 40.61716275599988], [-74.15379116699992, 40.612255611999906], [-74.1623545779999, 40.618335637999955], [-74.17459706299995, 40.622364010999874], [-74.1955597527974, 40.635023986316575], [-74.19462310145185, 40.637391183563665], [-74.18652496897599, 40.64338866217145], [-74.18483594580836, 40.644128266338726], [-74.18365067064063, 40.642239641050644], [-74.1836674587201, 40.64461048505006], [-74.17989568340026, 40.64526879647802]]], [[[-74.15945602438187, 40.641448333324036], [-74.16111242522163, 40.64183545373729], [-74.16146036002637, 40.644294969882374], [-74.15743349200979, 40.643302857779], [-74.15945602438187, 40.641448333324036]]]]}}, {\"id\": \"156\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5531110811175597, \"OBJECTID\": 157, \"Shape_Leng\": 0.131300217777, \"Shape_Area\": 0.000354370128323, \"zone\": \"Maspeth\", \"LocationID\": 157, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8978323689999, 40.73516035399992], [-73.8885228599999, 40.734531159999904], [-73.8870771209998, 40.72463770199994], [-73.90010558699989, 40.72329624099992], [-73.899816102, 40.721236899999894], [-73.89739704899986, 40.7215512099999], [-73.89736598399989, 40.72046438299989], [-73.90062644399997, 40.719764053999874], [-73.90033904299986, 40.718019947999885], [-73.89834760099983, 40.719197677999894], [-73.897267579, 40.71814715699985], [-73.89901821099994, 40.71738212699993], [-73.89750226399987, 40.716383867999916], [-73.90477244599997, 40.71286579999984], [-73.90639010099999, 40.71736369199989], [-73.90965980099996, 40.71779315499991], [-73.91283037699982, 40.71536090699993], [-73.91077807399988, 40.71318759999986], [-73.92404011299993, 40.714008312999916], [-73.92352547251075, 40.71561660836783], [-73.92064258940593, 40.715773192028834], [-73.92232773978448, 40.71659264406108], [-73.91132095099978, 40.71962422999992], [-73.91384828299984, 40.72289841199989], [-73.905180579, 40.72895657699995], [-73.90041661699985, 40.73465058799992], [-73.8978323689999, 40.73516035399992]]]}}, {\"id\": \"157\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3155719663057511, \"OBJECTID\": 158, \"Shape_Leng\": 0.0548099905515, \"Shape_Area\": 0.000185568253002, \"zone\": \"Meatpacking/West Village West\", \"LocationID\": 158, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.0077911609999, 40.74197084499982], [-74.00515808199998, 40.74085808099991], [-74.0070902079999, 40.72877225299993], [-74.014390699162, 40.728463047760016], [-74.01408693559922, 40.730663173040284], [-74.01116195696464, 40.730450676349065], [-74.01099350879154, 40.732320417940606], [-74.01399957240955, 40.733332277517704], [-74.01072514326883, 40.733459523511115], [-74.01201792010478, 40.734063744456314], [-74.01081359479103, 40.734283443647534], [-74.0103930313456, 40.739174296059204], [-74.01265928208485, 40.74074763849748], [-74.0077911609999, 40.74197084499982]]]}}, {\"id\": \"158\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6711195928753181, \"OBJECTID\": 159, \"Shape_Leng\": 0.064363233206, \"Shape_Area\": 0.00017124910387, \"zone\": \"Melrose South\", \"LocationID\": 159, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91189154999992, 40.822141481999914], [-73.9012927759998, 40.820475442999914], [-73.9044655149999, 40.81228195999994], [-73.91768626999985, 40.8160812429999], [-73.92048117399989, 40.8145636869999], [-73.92309299899993, 40.81669772799995], [-73.92399951299991, 40.81543891199991], [-73.92662718199988, 40.816118958999915], [-73.91677400599986, 40.82438708699987], [-73.91189154999992, 40.822141481999914]]]}}, {\"id\": \"159\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5745629070963318, \"OBJECTID\": 160, \"Shape_Leng\": 0.14151914297, \"Shape_Area\": 0.000571234977994, \"zone\": \"Middle Village\", \"LocationID\": 160, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.869707654, 40.72514643399992], [-73.85962920999985, 40.713975294999926], [-73.87016462499984, 40.70920774299987], [-73.86911007199988, 40.707089336999914], [-73.88088332699985, 40.7064539769999], [-73.87761134199985, 40.70716770499993], [-73.88006267699978, 40.71265257399993], [-73.90477244599997, 40.71286579999984], [-73.89750226399987, 40.716383867999916], [-73.89901821099994, 40.71738212699993], [-73.897267579, 40.71814715699985], [-73.89834760099983, 40.719197677999894], [-73.90033904299986, 40.718019947999885], [-73.90062644399997, 40.719764053999874], [-73.89736598399989, 40.72046438299989], [-73.89739704899986, 40.7215512099999], [-73.899816102, 40.721236899999894], [-73.89903191799989, 40.723856959999864], [-73.89426494199992, 40.723143188999856], [-73.8870771209998, 40.72463770199994], [-73.88720524999994, 40.727792996999874], [-73.87509154199992, 40.73067136299992], [-73.87146074199987, 40.729326373999854], [-73.869707654, 40.72514643399992]]]}}, {\"id\": \"160\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4022771479736624, \"OBJECTID\": 161, \"Shape_Leng\": 0.0358039100611, \"Shape_Area\": 7.19130682992e-05, \"zone\": \"Midtown Center\", \"LocationID\": 161, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97534377999993, 40.7552996949999], [-73.9783696929999, 40.752416226999905], [-73.98411754799999, 40.75484205299995], [-73.977686006, 40.76364440299992], [-73.97124277300004, 40.76093641799984], [-73.97534377999993, 40.7552996949999]]]}}, {\"id\": \"161\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3596184805923184, \"OBJECTID\": 162, \"Shape_Leng\": 0.0352698146219, \"Shape_Area\": 4.78936962112e-05, \"zone\": \"Midtown East\", \"LocationID\": 162, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9712170449999, 40.75519332399994], [-73.9744494109998, 40.75076644499991], [-73.9783696929999, 40.752416226999905], [-73.96980028899992, 40.76291351199995], [-73.9665833839999, 40.76155093499988], [-73.9712170449999, 40.75519332399994]]]}}, {\"id\": \"162\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.41203381253818994, \"OBJECTID\": 163, \"Shape_Leng\": 0.0341768669752, \"Shape_Area\": 4.08567804196e-05, \"zone\": \"Midtown North\", \"LocationID\": 163, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98136213699995, 40.767836738999904], [-73.9736358469999, 40.7645699129999], [-73.96980028899992, 40.76291351199995], [-73.97124277300004, 40.76093641799984], [-73.98428230099987, 40.76479181499994], [-73.98136213699995, 40.767836738999904]]]}}, {\"id\": \"163\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3815021697251198, \"OBJECTID\": 164, \"Shape_Leng\": 0.0357716119908, \"Shape_Area\": 5.56318237441e-05, \"zone\": \"Midtown South\", \"LocationID\": 164, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98362519999995, 40.7439431149999], [-73.99051761299985, 40.746038637999895], [-73.98411754799999, 40.75484205299995], [-73.98088709899987, 40.753480988999875], [-73.98407636100002, 40.74910212499993], [-73.98085054499998, 40.74775036999995], [-73.98362519999995, 40.7439431149999]]]}}, {\"id\": \"164\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5545425966677145, \"OBJECTID\": 165, \"Shape_Leng\": 0.088608036213, \"Shape_Area\": 0.000353959317019, \"zone\": \"Midwood\", \"LocationID\": 165, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96014773499996, 40.628915184999876], [-73.95995565099992, 40.627778671999884], [-73.94732672199994, 40.629166566999835], [-73.94451833400001, 40.61971364699989], [-73.94651373199997, 40.61949401899995], [-73.94573691999982, 40.61536340699989], [-73.95026514899993, 40.61392435199983], [-73.94978380499997, 40.611371749999854], [-73.96237947799978, 40.609986373999966], [-73.96471983399981, 40.62237296099993], [-73.96864510599995, 40.6219416949999], [-73.97136622199992, 40.62892916199996], [-73.96040734299989, 40.6301625309999], [-73.96014773499996, 40.628915184999876]]]}}, {\"id\": \"165\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.36056688271411746, \"OBJECTID\": 166, \"Shape_Leng\": 0.0688237542871, \"Shape_Area\": 0.000255276270405, \"zone\": \"Morningside Heights\", \"LocationID\": 166, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95707738499995, 40.8142980939999], [-73.95210125199992, 40.811442853999935], [-73.9535757779999, 40.8094766979999], [-73.95496657099991, 40.81006455499994], [-73.95964685399987, 40.80115642299993], [-73.97110765876137, 40.80579013958964], [-73.96203105797969, 40.81808633074085], [-73.95707738499995, 40.8142980939999]]]}}, {\"id\": \"166\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6557642487046632, \"OBJECTID\": 167, \"Shape_Leng\": 0.0908159737292, \"Shape_Area\": 0.000167529391193, \"zone\": \"Morrisania/Melrose\", \"LocationID\": 167, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89690017899987, 40.8310333359999], [-73.8899110649999, 40.830124191999865], [-73.88609624199987, 40.82765577699989], [-73.89483522899992, 40.82624684099993], [-73.89548522799996, 40.82824854099993], [-73.90093510699997, 40.82792810199992], [-73.9012927759998, 40.820475442999914], [-73.91181615799985, 40.82203502199991], [-73.91677400599986, 40.82438708699987], [-73.91541108199998, 40.82577390899993], [-73.91130974499993, 40.824848557999886], [-73.91169430299985, 40.82775871599993], [-73.908959903, 40.83022623299989], [-73.90553116799988, 40.82910413299989], [-73.90122534999992, 40.83549991099991], [-73.89697220299995, 40.8343452539999], [-73.89690017899987, 40.8310333359999]]]}}, {\"id\": \"167\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6374089068825911, \"OBJECTID\": 168, \"Shape_Leng\": 0.114517434155, \"Shape_Area\": 0.000547058100444, \"zone\": \"Mott Haven/Port Morris\", \"LocationID\": 168, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.92480953899991, 40.81565901999993], [-73.92309299899993, 40.81669772799995], [-73.92048117399989, 40.8145636869999], [-73.91768626999985, 40.8160812429999], [-73.9044655149999, 40.81228195999994], [-73.90229231992774, 40.80456477614167], [-73.91168831241185, 40.79662376988872], [-73.91904978748826, 40.798993935843235], [-73.9229271008289, 40.80237329482137], [-73.92762788658843, 40.80269566548149], [-73.93252708785968, 40.80882328177534], [-73.93242632000944, 40.81411751897961], [-73.93027812699984, 40.813248024999965], [-73.92731653099999, 40.81855963399991], [-73.92518486499998, 40.818012668999884], [-73.92662718199988, 40.816118958999915], [-73.92480953899991, 40.81565901999993]]], [[[-73.89833036270552, 40.80241282093997], [-73.89646668834577, 40.80079047089134], [-73.90021004993142, 40.79926415589601], [-73.90003735815736, 40.800908742050225], [-73.89833036270552, 40.80241282093997]]], [[[-73.89680883223778, 40.795808445159786], [-73.89796839783742, 40.79564483916198], [-73.89919434249981, 40.796502456018175], [-73.8978825324018, 40.79711653214704], [-73.89680883223778, 40.795808445159786]]]]}}, {\"id\": \"168\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6177993527508091, \"OBJECTID\": 169, \"Shape_Leng\": 0.0601046580332, \"Shape_Area\": 0.000146027537733, \"zone\": \"Mount Hope\", \"LocationID\": 169, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89816577499982, 40.85514637599989], [-73.89863543699985, 40.85390697499993], [-73.89560883299981, 40.85457036999986], [-73.901344046, 40.84442875099994], [-73.90483121999989, 40.84480034199987], [-73.9050151499998, 40.842404697999875], [-73.91689747599989, 40.84511773399991], [-73.91416184199993, 40.845179187999925], [-73.90557007299988, 40.855787321999884], [-73.89816577499982, 40.85514637599989]]]}}, {\"id\": \"169\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.33307616865493717, \"OBJECTID\": 170, \"Shape_Leng\": 0.0457690091051, \"Shape_Area\": 7.43150675972e-05, \"zone\": \"Murray Hill\", \"LocationID\": 170, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97203331099985, 40.745804967999945], [-73.97350835000003, 40.7437820339999], [-73.97812378199983, 40.745727172999885], [-73.98085965399994, 40.74196976799989], [-73.98407485299995, 40.74332471799989], [-73.98085054399985, 40.74775036999991], [-73.98407636100002, 40.74910212499993], [-73.98088709899987, 40.753480988999875], [-73.9744494109998, 40.75076644499991], [-73.9758346669999, 40.74886314599996], [-73.97121971999988, 40.74692154099996], [-73.97203331099985, 40.745804967999945]]]}}, {\"id\": \"170\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6336734693877552, \"OBJECTID\": 171, \"Shape_Leng\": 0.107353876244, \"Shape_Area\": 0.000520099538161, \"zone\": \"Murray Hill-Queens\", \"LocationID\": 171, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79317944599988, 40.77040317699989], [-73.79343991199988, 40.760764127999934], [-73.79076004599987, 40.76098576899991], [-73.79031517, 40.75782345099988], [-73.81104963199985, 40.761417020999964], [-73.82063593299985, 40.75887226799989], [-73.82657481899986, 40.772345397999885], [-73.82331286899986, 40.77957865999986], [-73.81452997799975, 40.77888817499992], [-73.81485850699985, 40.776485922999875], [-73.79853009399987, 40.775190728999924], [-73.79317944599988, 40.77040317699989]]]}}, {\"id\": \"171\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5789473684210527, \"OBJECTID\": 172, \"Shape_Leng\": 0.118476116148, \"Shape_Area\": 0.000658402501406, \"zone\": \"New Dorp/Midland Beach\", \"LocationID\": 172, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08654858694766, 40.56963231913558], [-74.08621632224266, 40.56853269814713], [-74.09054960106333, 40.567083597211294], [-74.09938031432728, 40.5589641933612], [-74.11376161199998, 40.56649718899993], [-74.11636336399992, 40.56391563799986], [-74.12489904399992, 40.568949141999916], [-74.12150213699994, 40.571721558999904], [-74.1215372899999, 40.57515325299986], [-74.10525775499997, 40.58592143599986], [-74.09844000899997, 40.581241705999915], [-74.10039083499996, 40.579688665999925], [-74.08597647499992, 40.57223697099986], [-74.08495715819193, 40.57125984002107], [-74.08654858694766, 40.56963231913558]]]}}, {\"id\": \"0\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.16460313950917532, \"OBJECTID\": 1, \"Shape_Leng\": 0.116357453189, \"Shape_Area\": 0.0007823067885, \"zone\": \"Newark Airport\", \"LocationID\": 1, \"borough\": \"EWR\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.18445299999996, 40.6949959999999], [-74.18284199999994, 40.70346499999988], [-74.17565999999994, 40.707379999999866], [-74.16081899999995, 40.70764099999991], [-74.15306999999996, 40.7052829999999], [-74.17283399999997, 40.67656399999988], [-74.17862999999994, 40.671037999999896], [-74.18503799999996, 40.67335999999989], [-74.18991599999998, 40.68154099999988], [-74.18922299999997, 40.688623999999905], [-74.18563199999994, 40.69164799999987], [-74.18445299999996, 40.6949959999999]]]}}, {\"id\": \"172\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.7711670480549199, \"OBJECTID\": 173, \"Shape_Leng\": 0.0600639661414, \"Shape_Area\": 0.000178329851357, \"zone\": \"North Corona\", \"LocationID\": 173, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85540366099997, 40.75834030899989], [-73.85312635499996, 40.75510934299989], [-73.85579909099987, 40.75440952799994], [-73.85442575299999, 40.748851725999884], [-73.86790963899986, 40.74493371599989], [-73.87365721799983, 40.756803485999946], [-73.85540366099997, 40.75834030899989]]]}}, {\"id\": \"173\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6394817073170732, \"OBJECTID\": 174, \"Shape_Leng\": 0.0678177559602, \"Shape_Area\": 0.000155928664422, \"zone\": \"Norwood\", \"LocationID\": 174, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87772817699984, 40.88345419499995], [-73.87812594399992, 40.88215382599989], [-73.87467601499999, 40.88156285199993], [-73.87494589799991, 40.87994766899988], [-73.87024100099985, 40.88022935299995], [-73.87619224999979, 40.87047671099987], [-73.8830900319999, 40.86659150899993], [-73.88146889699985, 40.868573645999916], [-73.88705142599984, 40.884349575999906], [-73.88124422499992, 40.88270963299992], [-73.87803562399996, 40.887118321999885], [-73.87772817699984, 40.88345419499995]]]}}, {\"id\": \"174\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4921875, \"OBJECTID\": 175, \"Shape_Leng\": 0.134897665102, \"Shape_Area\": 0.000504920726134, \"zone\": \"Oakland Gardens\", \"LocationID\": 175, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7562560639999, 40.7584304689999], [-73.75230259399993, 40.759455198999916], [-73.75004764499985, 40.75840465799996], [-73.74384322099985, 40.74769009599988], [-73.74625984399985, 40.74671147499993], [-73.74477587099986, 40.74328109699994], [-73.75347299999999, 40.74240484399992], [-73.75098230099987, 40.74002146199996], [-73.7439619359999, 40.74078912699986], [-73.74306820799991, 40.7384408179999], [-73.74016951799993, 40.73848189799986], [-73.75048295699996, 40.72922129399996], [-73.75669771299987, 40.7262287109999], [-73.76564374500005, 40.732533126999954], [-73.7636997919999, 40.73286411699993], [-73.76510920699994, 40.73455658499992], [-73.76248609299982, 40.736730981999884], [-73.76772991399994, 40.74577622599993], [-73.75684127399992, 40.74932207799991], [-73.76184343999986, 40.75553726899989], [-73.76239510599996, 40.759496997999854], [-73.75800131799996, 40.76037986699988], [-73.7562560639999, 40.7584304689999]]]}}, {\"id\": \"175\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3157894736842105, \"OBJECTID\": 176, \"Shape_Leng\": 0.151995190703, \"Shape_Area\": 0.000657782126267, \"zone\": \"Oakwood\", \"LocationID\": 176, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.12187198599999, 40.57510757499989], [-74.12150213699994, 40.571721558999904], [-74.12489904399992, 40.568949141999916], [-74.11636336299996, 40.56391563799985], [-74.11376161199998, 40.56649718899993], [-74.09992914200652, 40.55907068582615], [-74.10140516910823, 40.555538526164405], [-74.10309839111532, 40.555904641451136], [-74.10484447636338, 40.55302581455626], [-74.10768623464054, 40.553976671477514], [-74.11301046102231, 40.547780065214994], [-74.12730057799996, 40.55729517199993], [-74.13321002499998, 40.552383875999936], [-74.13885092699991, 40.5569511789999], [-74.13170009, 40.56453574899991], [-74.1443874499999, 40.56970609999989], [-74.14418588199999, 40.57203740999988], [-74.13261022999991, 40.573530162999894], [-74.12708814799998, 40.57636910599995], [-74.12187198599999, 40.57510757499989]]]}}, {\"id\": \"176\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5387278234530506, \"OBJECTID\": 177, \"Shape_Leng\": 0.0715805407056, \"Shape_Area\": 0.000198894762248, \"zone\": \"Ocean Hill\", \"LocationID\": 177, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90287079599995, 40.6786314149999], [-73.90004626199993, 40.67689308999995], [-73.92275257199998, 40.667097332999894], [-73.92164666399992, 40.67886992299994], [-73.9163011989999, 40.67857711199991], [-73.91804607, 40.68721324799989], [-73.90287079599995, 40.6786314149999]]]}}, {\"id\": \"177\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5568573397656789, \"OBJECTID\": 178, \"Shape_Leng\": 0.0682528770739, \"Shape_Area\": 0.000175805409143, \"zone\": \"Ocean Parkway South\", \"LocationID\": 178, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97080408199997, 40.62593266699988], [-73.96864510599995, 40.6219416949999], [-73.96471983399981, 40.62237296099993], [-73.96285973999984, 40.61254948599989], [-73.96478112799986, 40.61233092199988], [-73.96429649899991, 40.60977642599992], [-73.97403097799986, 40.608702309999906], [-73.97517176999993, 40.61472418599991], [-73.97290326899996, 40.61415296799988], [-73.97335879399992, 40.616541714999904], [-73.97537335500002, 40.6157547219999], [-73.97785009299986, 40.617287605999906], [-73.97539380499998, 40.62076998699995], [-73.97705352899999, 40.62153252199987], [-73.9772614999999, 40.625852772999906], [-73.97080408199997, 40.62593266699988]]]}}, {\"id\": \"178\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4905891712293509, \"OBJECTID\": 179, \"Shape_Leng\": 0.0713808644377, \"Shape_Area\": 0.000183893478476, \"zone\": \"Old Astoria\", \"LocationID\": 179, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93290347544752, 40.77794449857707], [-73.92782147400004, 40.776697526999946], [-73.91770217299991, 40.7703931119999], [-73.91995596899991, 40.768440064999865], [-73.91849336999996, 40.76773465699995], [-73.91977764799998, 40.7661521839999], [-73.92218946199995, 40.767267356999895], [-73.92425288799978, 40.76472456399995], [-73.93344468499993, 40.768817789999915], [-73.93493511099992, 40.76701302399987], [-73.93618379699991, 40.767691800999934], [-73.93424609600005, 40.771029275999894], [-73.93746638537067, 40.77253149512518], [-73.93765483090819, 40.775085685383594], [-73.93508232492144, 40.77794423372776], [-73.93290347544752, 40.77794449857707]]]}}, {\"id\": \"179\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5601821730644112, \"OBJECTID\": 180, \"Shape_Leng\": 0.0805277671523, \"Shape_Area\": 0.000247682193353, \"zone\": \"Ozone Park\", \"LocationID\": 180, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84119605699982, 40.68335136599989], [-73.83334819299995, 40.66677318699994], [-73.86131863599985, 40.6750212689999], [-73.86334943599995, 40.67935163999989], [-73.85878576899987, 40.68063046299989], [-73.85819836099995, 40.67943084899987], [-73.85392389199987, 40.679686155999875], [-73.85099089299987, 40.68056836699987], [-73.85197864599989, 40.682564210999864], [-73.84786817599985, 40.681486831999905], [-73.84119605699982, 40.68335136599989]]]}}, {\"id\": \"180\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.376107582178342, \"OBJECTID\": 181, \"Shape_Leng\": 0.0895372399547, \"Shape_Area\": 0.000306890292807, \"zone\": \"Park Slope\", \"LocationID\": 181, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97583132299987, 40.66581507999991], [-73.97965353499985, 40.661243232999915], [-73.98223034199995, 40.66214149299991], [-73.985086794, 40.65901328799989], [-73.988716511, 40.66240493799995], [-73.9906354539999, 40.66057908799997], [-73.99692689899985, 40.664374914999925], [-73.99490350199993, 40.66678543499988], [-73.99277080699981, 40.665507280999876], [-73.97804289599988, 40.68485661499987], [-73.97375598499991, 40.68303401999988], [-73.97618051399984, 40.67806432099995], [-73.96949738899991, 40.67590637999996], [-73.96871822199984, 40.67392423399988], [-73.97583132299987, 40.66581507999991]]]}}, {\"id\": \"181\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6496305418719212, \"OBJECTID\": 182, \"Shape_Leng\": 0.0393606369479, \"Shape_Area\": 9.10644156732e-05, \"zone\": \"Parkchester\", \"LocationID\": 182, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85465135700002, 40.83388802699987], [-73.86271114399993, 40.83299150899997], [-73.86448011799995, 40.84108779699994], [-73.85333524999986, 40.84247015499987], [-73.85168459699997, 40.8342010519999], [-73.85465135700002, 40.83388802699987]]]}}, {\"id\": \"182\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.620253164556962, \"OBJECTID\": 183, \"Shape_Leng\": 0.0398262960348, \"Shape_Area\": 9.51929451066e-05, \"zone\": \"Pelham Bay\", \"LocationID\": 183, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83676643699985, 40.84338596999994], [-73.83463970299987, 40.853817993999904], [-73.82860197199996, 40.85571784799988], [-73.82721751200002, 40.853250512999935], [-73.82799676999998, 40.84862431399989], [-73.82611300699996, 40.84533869399991], [-73.83676643699985, 40.84338596999994]]]}}, {\"id\": \"183\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.489247311827957, \"OBJECTID\": 184, \"Shape_Leng\": 0.260815683043, \"Shape_Area\": 0.00198883363103, \"zone\": \"Pelham Bay Park\", \"LocationID\": 184, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.80728647099995, 40.885931813999875], [-73.79322870891143, 40.88282259495222], [-73.80087270458245, 40.87159011989393], [-73.80252284756597, 40.87050664469679], [-73.80339117523091, 40.87250503763108], [-73.80431849468228, 40.87115974106797], [-73.80302203685277, 40.86915511979648], [-73.80588410012888, 40.869068502184916], [-73.80841596818787, 40.87118969336752], [-73.80464604502912, 40.86664711643685], [-73.80300388108948, 40.86704183656889], [-73.80225816157765, 40.8656082064109], [-73.79780377219292, 40.87225613227397], [-73.794790446816, 40.873200484033966], [-73.79384702788171, 40.87859445979086], [-73.7895587294431, 40.881261284915404], [-73.789523093688, 40.87899812706048], [-73.7843816810755, 40.878420531670514], [-73.78719087560506, 40.872730553729134], [-73.78508139658601, 40.87358996322568], [-73.78526152075709, 40.87174532233236], [-73.78358738486025, 40.87378095468029], [-73.78320175426292, 40.87067036085376], [-73.78560245305444, 40.86834165222368], [-73.78795446145405, 40.86984541906357], [-73.79139032586131, 40.868141307466146], [-73.79190454927502, 40.86082956271207], [-73.79454562318031, 40.85888825916121], [-73.79435005934273, 40.85652702380392], [-73.7987225560008, 40.85491440117259], [-73.79780522990592, 40.85194959859714], [-73.80040387554423, 40.84810822100142], [-73.80417557932397, 40.853340120189074], [-73.80478410547944, 40.86154633053462], [-73.80592716427573, 40.86194119699028], [-73.80612390783546, 40.85981220661901], [-73.80783343870087, 40.860318133305384], [-73.80768048107562, 40.858538063289515], [-73.81411414337536, 40.86373145821164], [-73.81641392102723, 40.86118062770394], [-73.81273146495035, 40.85887754194771], [-73.81230602708418, 40.854162026473176], [-73.81684183599076, 40.85362521315728], [-73.81712409042197, 40.85126957344483], [-73.81507165904237, 40.84913361979451], [-73.82495697699986, 40.846105444999864], [-73.82517999, 40.842700058999874], [-73.82624132899997, 40.84299359399996], [-73.82799676999987, 40.8486243129999], [-73.82723026100001, 40.853350756999895], [-73.82860197199996, 40.85571784799988], [-73.83283502299986, 40.8560358449999], [-73.82834347399992, 40.86089202599988], [-73.82699653199998, 40.85908778599986], [-73.82037552999996, 40.86103300499993], [-73.81578764099999, 40.8657580609999], [-73.821266938, 40.869071605999885], [-73.82329599399986, 40.87309741799988], [-73.81992586499996, 40.88211933299987], [-73.82027432999985, 40.885550699999925], [-73.81442762200005, 40.88677196499989], [-73.80728647099995, 40.885931813999875]]], [[[-73.78650554049733, 40.880940134479225], [-73.78578002611279, 40.88036396266133], [-73.78742039384403, 40.87977089028999], [-73.787127834652, 40.88092346317125], [-73.78650554049733, 40.880940134479225]]], [[[-73.78103351104939, 40.87648400204775], [-73.78120649649391, 40.87628502546013], [-73.78132704118002, 40.87636132654635], [-73.78112767478156, 40.87665268050809], [-73.78103351104939, 40.87648400204775]]], [[[-73.78241811865315, 40.87492327042177], [-73.78265785360905, 40.87479265669398], [-73.78280207510406, 40.87494886620542], [-73.78247604125525, 40.87516813337884], [-73.78241811865315, 40.87492327042177]]], [[[-73.77435244645233, 40.874169955874805], [-73.77453640510954, 40.87471001543833], [-73.77409538292126, 40.87489618448013], [-73.7740318153443, 40.874394774956215], [-73.77435244645233, 40.874169955874805]]], [[[-73.78605394964902, 40.873782646192794], [-73.78618924358646, 40.87359972769974], [-73.78633325996694, 40.87367772488745], [-73.7861946371647, 40.87388531941464], [-73.78605394964902, 40.873782646192794]]], [[[-73.78648510546586, 40.87320925495041], [-73.7867268909535, 40.87307453926568], [-73.78626202153426, 40.87339368663393], [-73.78632869488987, 40.873271144554096], [-73.78648510546586, 40.87320925495041]]], [[[-73.77080975398195, 40.871549946847985], [-73.76990710668544, 40.87047694447026], [-73.77288042665229, 40.871245263430254], [-73.77217786492288, 40.87198111068936], [-73.77080975398195, 40.871549946847985]]], [[[-73.76668965841925, 40.86709778162448], [-73.76709907439783, 40.8667045727482], [-73.76779449382023, 40.86678117376544], [-73.76760072096499, 40.86749638654143], [-73.76668965841925, 40.86709778162448]]], [[[-73.76964988627088, 40.86548557242442], [-73.76979148260493, 40.86512823356445], [-73.77022931630273, 40.865138939396005], [-73.770068972998, 40.86552587618341], [-73.76964988627088, 40.86548557242442]]], [[[-73.78401249138903, 40.863131994074394], [-73.78376770348979, 40.86261060846434], [-73.78469165850701, 40.86254278990822], [-73.7842865050008, 40.86320508878048], [-73.78401249138903, 40.863131994074394]]], [[[-73.78312589594455, 40.862856167300684], [-73.7828819783119, 40.862406433091266], [-73.7831450428813, 40.86206867890555], [-73.78328053505923, 40.86278070671258], [-73.78312589594455, 40.862856167300684]]], [[[-73.77460156350928, 40.86206904745972], [-73.7749432115575, 40.861391499889486], [-73.77529297955127, 40.86123200821588], [-73.77485681624742, 40.86198363433426], [-73.77460156350928, 40.86206904745972]]], [[[-73.77290231992438, 40.86120858327854], [-73.77323150244695, 40.86074716979338], [-73.77351264165688, 40.861241184087085], [-73.77310632921937, 40.861614575032775], [-73.77290231992438, 40.86120858327854]]], [[[-73.8022229535527, 40.84163481314409], [-73.80263811156135, 40.841081153267076], [-73.80694608641589, 40.84146244718637], [-73.80680801372397, 40.84248913998752], [-73.8022229535527, 40.84163481314409]]]]}}, {\"id\": \"184\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6146718146718146, \"OBJECTID\": 185, \"Shape_Leng\": 0.0861619681636, \"Shape_Area\": 0.000228537100218, \"zone\": \"Pelham Parkway\", \"LocationID\": 185, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84004614299985, 40.857806268999866], [-73.83499552599994, 40.85657120099989], [-73.84049767399992, 40.8541841049999], [-73.84494053799982, 40.8482182209999], [-73.84430846999992, 40.85119046399989], [-73.85304526899994, 40.849547331999865], [-73.85457388299992, 40.85150834599995], [-73.86250923799993, 40.84836228099987], [-73.86303938099996, 40.850611306999944], [-73.86860515599989, 40.85206864599991], [-73.86990421600002, 40.85577329299987], [-73.8681883059999, 40.858063833999964], [-73.84004614299985, 40.857806268999866]]]}}, {\"id\": \"185\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4211778279738312, \"OBJECTID\": 186, \"Shape_Leng\": 0.0246963902234, \"Shape_Area\": 3.70729416953e-05, \"zone\": \"Penn Station/Madison Sq West\", \"LocationID\": 186, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99096832799995, 40.74542088999985], [-73.99709902899994, 40.74720510199991], [-73.99346417699986, 40.75219005499987], [-73.98776905899994, 40.749787028999926], [-73.99096832799995, 40.74542088999985]]]}}, {\"id\": \"186\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4489795918367347, \"OBJECTID\": 187, \"Shape_Leng\": 0.126868431324, \"Shape_Area\": 0.000421195755741, \"zone\": \"Port Richmond\", \"LocationID\": 187, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-74.13435269573621, 40.64188679740515], [-74.12890729033458, 40.64119482758617], [-74.1268887269999, 40.637210087999925], [-74.12867047299994, 40.63734826499993], [-74.13047428999985, 40.6308855089999], [-74.13806393799992, 40.627568470999904], [-74.13581008299997, 40.62506433999991], [-74.13826358199992, 40.624243769999936], [-74.13982184799993, 40.61629982499988], [-74.14603562099998, 40.61701277699996], [-74.14696864199988, 40.610151355999896], [-74.15018666499996, 40.60949781099991], [-74.15379116699992, 40.612255611999906], [-74.15231170399994, 40.61716275599988], [-74.14622812499992, 40.6249253619999], [-74.14554071999994, 40.62783252199995], [-74.14752618999992, 40.62938928799988], [-74.14478228999994, 40.637953779999904], [-74.14229766980965, 40.64030580709358], [-74.13435269573621, 40.64188679740515]]], [[[-74.12117246275159, 40.64166914557247], [-74.12128074797168, 40.64143249599354], [-74.12131194814832, 40.64145253116248], [-74.12117246275159, 40.64166914557247]]], [[[-74.1215534360538, 40.64161162832474], [-74.12176052602824, 40.64157431506241], [-74.12175764698422, 40.64142342422718], [-74.12177911197513, 40.641592547117035], [-74.1215534360538, 40.64161162832474]]], [[[-74.12107727405213, 40.64160742058869], [-74.12118039503834, 40.64138344518971], [-74.12121470277087, 40.64139683999368], [-74.12110124737936, 40.64161328229901], [-74.12107727405213, 40.64160742058869]]], [[[-74.12130529083001, 40.64160070313003], [-74.12137332919554, 40.64147647512165], [-74.12132617111645, 40.64160656922885], [-74.12130529083001, 40.64160070313003]]], [[[-74.12185329597821, 40.64158069227959], [-74.12180661091023, 40.64141215903768], [-74.1218414704778, 40.641405894387034], [-74.12199083807539, 40.64155463949942], [-74.12185329597821, 40.64158069227959]]], [[[-74.12270319595869, 40.64137664893253], [-74.12272114424832, 40.64134762377306], [-74.12276203509289, 40.641426635297144], [-74.12271103075855, 40.641433755252706], [-74.12270319595869, 40.64137664893253]]], [[[-74.12613276551227, 40.6412643880282], [-74.1259954246122, 40.640231242814394], [-74.12636808688976, 40.64133238048696], [-74.12564026870598, 40.64141849208226], [-74.12613276551227, 40.6412643880282]]]]}}, {\"id\": \"188\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.38907375217283335, \"OBJECTID\": 189, \"Shape_Leng\": 0.0486743306502, \"Shape_Area\": 0.000101427594033, \"zone\": \"Prospect Heights\", \"LocationID\": 189, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96070974499996, 40.673121393999985], [-73.96131877999991, 40.67140667599988], [-73.96892332199981, 40.67293755299988], [-73.96949738899991, 40.67590637999996], [-73.97618051399984, 40.67806432099995], [-73.97375598499991, 40.68303401999988], [-73.96134655899995, 40.6804635239999], [-73.96318238899983, 40.675529503999876], [-73.96007428300001, 40.67487807099994], [-73.96070974499996, 40.673121393999985]]]}}, {\"id\": \"189\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.43336449568920743, \"OBJECTID\": 190, \"Shape_Leng\": 0.0743294660829, \"Shape_Area\": 0.000270873666522, \"zone\": \"Prospect Park\", \"LocationID\": 190, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96307724199983, 40.66213019499987], [-73.96190028499997, 40.65487096199992], [-73.96621321999996, 40.653189404999885], [-73.96465438699984, 40.650887727999894], [-73.97139631700004, 40.64825778599986], [-73.97436701299992, 40.65818790399992], [-73.98022902599985, 40.660985325999896], [-73.96865541400003, 40.673023838999924], [-73.96258784899996, 40.671711596999884], [-73.96095595499996, 40.66328505499992], [-73.96307724199983, 40.66213019499987]]]}}, {\"id\": \"187\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5083754107557906, \"OBJECTID\": 188, \"Shape_Leng\": 0.0977983968367, \"Shape_Area\": 0.000313003032771, \"zone\": \"Prospect-Lefferts Gardens\", \"LocationID\": 188, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95729172699996, 40.6643026319999], [-73.94269628999989, 40.664055128999934], [-73.94282892199992, 40.6628474859999], [-73.93068465499985, 40.66362047699992], [-73.92646424299983, 40.659814361999956], [-73.92834017599988, 40.660060035999884], [-73.92799465199988, 40.65680210099988], [-73.9500666489999, 40.65542299699989], [-73.94957113100006, 40.65080789999991], [-73.9587078989999, 40.65038727299989], [-73.95986103199981, 40.65563478899989], [-73.96190028499997, 40.65487096199992], [-73.96307724199983, 40.66213019499987], [-73.95729172699996, 40.6643026319999]]]}}, {\"id\": \"190\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6291390728476821, \"OBJECTID\": 191, \"Shape_Leng\": 0.130932637664, \"Shape_Area\": 0.000688460603185, \"zone\": \"Queens Village\", \"LocationID\": 191, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.73016587199996, 40.72395859599988], [-73.72816233910697, 40.7230841398913], [-73.73032628984046, 40.722157296892334], [-73.7269714679152, 40.710714504016735], [-73.72677759516957, 40.7030592911994], [-73.74590871000004, 40.70228727399988], [-73.74500119100004, 40.70026221299988], [-73.74773012199991, 40.69955313699991], [-73.74618157999997, 40.696121925999904], [-73.74863325999986, 40.6954447609999], [-73.75205066499986, 40.70522802799995], [-73.75047058199985, 40.70563993099994], [-73.75725671499985, 40.71813860199992], [-73.7523343839998, 40.72026246099993], [-73.75669771299987, 40.7262287109999], [-73.74672140400006, 40.73247715099991], [-73.74555667199989, 40.73063689799994], [-73.7314721509999, 40.727411794999924], [-73.73016587199996, 40.72395859599988]]]}}, {\"id\": \"191\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6389183457051962, \"OBJECTID\": 192, \"Shape_Leng\": 0.11196461971, \"Shape_Area\": 0.000373808019026, \"zone\": \"Queensboro Hill\", \"LocationID\": 192, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82239598999983, 40.74986422299986], [-73.81416171999987, 40.74613857999992], [-73.80750599299991, 40.750136586999936], [-73.80287666499983, 40.749765149999895], [-73.8033521419999, 40.74643196499987], [-73.7946756879998, 40.74721192799994], [-73.79453507400001, 40.74450763799989], [-73.79723084999992, 40.74169550699993], [-73.79640218700004, 40.73828235799987], [-73.8243501109999, 40.73975360599992], [-73.82946751999994, 40.737024481999896], [-73.83143025500006, 40.73913573299994], [-73.83113122799989, 40.74233799999994], [-73.83586131899989, 40.743227984999926], [-73.83392218299996, 40.74753207499989], [-73.82584440999999, 40.74934839099993], [-73.82426493799994, 40.75183288799992], [-73.82239598999983, 40.74986422299986]]]}}, {\"id\": \"192\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5230718942379283, \"OBJECTID\": 193, \"Shape_Leng\": 0.0655299584801, \"Shape_Area\": 0.000145591657944, \"zone\": \"Queensbridge/Ravenswood\", \"LocationID\": 193, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93522718099996, 40.76952572899989], [-73.93858976899998, 40.766817358999916], [-73.93398092199995, 40.76282598199987], [-73.944052652, 40.75231325999992], [-73.95080763809156, 40.75526368005708], [-73.9413117494144, 40.76691800476961], [-73.93424609599985, 40.771029274999925], [-73.93522718099996, 40.76952572899989]]]}}, {\"id\": \"193\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.35847492747617077, \"OBJECTID\": 194, \"Shape_Leng\": 0.0817938353532, \"Shape_Area\": 0.000273481702719, \"zone\": \"Randalls Island\", \"LocationID\": 194, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91639852914044, 40.79768484203896], [-73.91378385370595, 40.793836285482065], [-73.92412113996787, 40.78252307401082], [-73.92810118993664, 40.78092032002105], [-73.93598226003293, 40.7839423680532], [-73.93084178719639, 40.79099143000079], [-73.9261041024332, 40.79073160730996], [-73.92560814781015, 40.79189695828146], [-73.9278688922052, 40.79095483831721], [-73.9283194359912, 40.79309126608271], [-73.9254929354651, 40.80195616882528], [-73.92263378594883, 40.80186873814113], [-73.91639852914044, 40.79768484203896]]]}}, {\"id\": \"194\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4375217059109537, \"OBJECTID\": 195, \"Shape_Leng\": 0.131308244067, \"Shape_Area\": 0.000801651953985, \"zone\": \"Red Hook\", \"LocationID\": 195, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.0071710969999, 40.68635700999991], [-74.00485241599992, 40.685668130999886], [-74.00596041099988, 40.683362870999936], [-73.9986188799999, 40.67183017199989], [-74.00289483876944, 40.66734846594933], [-74.0058636618068, 40.66799376258568], [-74.00683236078251, 40.666049645526776], [-74.00696707834247, 40.666052182356665], [-74.00543926529353, 40.6709214682384], [-74.00754821618384, 40.66705174573358], [-74.00708724221813, 40.668591358535764], [-74.00985411855632, 40.668523628335336], [-74.01154017543327, 40.6650859197566], [-74.01574872882654, 40.6645683362916], [-74.01690267361069, 40.66484660233888], [-74.01933946871777, 40.671624948422625], [-74.01751037086694, 40.671034530984144], [-74.01648665554487, 40.664930508953816], [-74.01227967872792, 40.66573241415882], [-74.01033845947926, 40.669078195448], [-74.01162527521356, 40.670581028275876], [-74.01424926931875, 40.66972860473708], [-74.01175350509699, 40.670657757578326], [-74.0151998057007, 40.6707540276644], [-74.01423289452782, 40.67213000051297], [-74.01568625497515, 40.671038834928844], [-74.0129758779219, 40.67332791872093], [-74.01632099922307, 40.6729082695637], [-74.01496392184576, 40.67467763029424], [-74.0188009477692, 40.6722507648487], [-74.01727138470127, 40.67360864580548], [-74.0188577695892, 40.6751022924089], [-74.01791198226178, 40.676510462801936], [-74.01995197864673, 40.67710301044993], [-74.01825267004557, 40.67855288672587], [-74.01928128101008, 40.67964814029763], [-74.01282000316418, 40.68362241624865], [-74.01410549647976, 40.68171251937541], [-74.013018354221, 40.68041237401427], [-74.00967108558879, 40.683268727715], [-74.01193259955522, 40.683887749077535], [-74.0071710969999, 40.68635700999991]]]}}, {\"id\": \"195\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.576746434614455, \"OBJECTID\": 196, \"Shape_Leng\": 0.0832421106332, \"Shape_Area\": 0.0001974713336, \"zone\": \"Rego Park\", \"LocationID\": 196, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85762923599995, 40.728606921999884], [-73.8594198799999, 40.726464837999885], [-73.85727108100001, 40.72568129799988], [-73.85805416899993, 40.72444540599986], [-73.86203833799982, 40.72595678499994], [-73.8586809759999, 40.72261243999988], [-73.85729287299985, 40.711467676999874], [-73.85975932900001, 40.711845058999856], [-73.86003789899996, 40.71493223599987], [-73.86929270499994, 40.72443592899991], [-73.87146074199987, 40.729326373999854], [-73.87509154199992, 40.73067136299992], [-73.85886555799988, 40.73593913899987], [-73.85625665099991, 40.73095912299993], [-73.85851097899999, 40.73028680199993], [-73.85762923599995, 40.728606921999884]]]}}, {\"id\": \"196\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5873639121898983, \"OBJECTID\": 197, \"Shape_Leng\": 0.108568532229, \"Shape_Area\": 0.000504689160432, \"zone\": \"Richmond Hill\", \"LocationID\": 197, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83696237799982, 40.70421245199992], [-73.83152109599999, 40.70138962899994], [-73.82245890399994, 40.70365553099989], [-73.81637824499992, 40.70245196399987], [-73.81063298299996, 40.69194738299988], [-73.82501128899997, 40.68795231799991], [-73.82599347399987, 40.689953661999894], [-73.82851136799994, 40.689239055999934], [-73.82753021399995, 40.687239482999914], [-73.83807769099997, 40.68423890999988], [-73.83697486299991, 40.6819880179999], [-73.84012856099982, 40.6811466509999], [-73.84947763499991, 40.69819580699987], [-73.83696237799982, 40.70421245199992]]]}}, {\"id\": \"197\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5301869033212316, \"OBJECTID\": 198, \"Shape_Leng\": 0.134094242763, \"Shape_Area\": 0.000499809882564, \"zone\": \"Ridgewood\", \"LocationID\": 198, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90768387299997, 40.71737969499987], [-73.90639010099999, 40.71736369199989], [-73.90537582199983, 40.71287658099986], [-73.88006267699978, 40.71265257399993], [-73.87761134199985, 40.70716770499993], [-73.88911917799989, 40.705598876999915], [-73.90123290699994, 40.69144227899997], [-73.90579597099993, 40.69412715499987], [-73.9042601839999, 40.69570037099989], [-73.91180820099989, 40.69993800299986], [-73.91067882699998, 40.701045968999914], [-73.91290404099983, 40.70236189199989], [-73.91180710099992, 40.703434952999864], [-73.92189184699987, 40.709396095999836], [-73.92074519699985, 40.71052968599989], [-73.92404011299993, 40.714008312999916], [-73.91077807399988, 40.71318759999986], [-73.91283037699992, 40.71536090799986], [-73.90965980099996, 40.71779315499991], [-73.90768387299997, 40.71737969499987]]]}}, {\"id\": \"198\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.36363636363636365, \"OBJECTID\": 199, \"Shape_Leng\": 0.0778085005169, \"Shape_Area\": 0.000288747549303, \"zone\": \"Rikers Island\", \"LocationID\": 199, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88885148496314, 40.79870632895875], [-73.88665108524579, 40.79803819669988], [-73.88375741590156, 40.79570856541983], [-73.87288457445655, 40.79145210820714], [-73.87080888583708, 40.78789669184152], [-73.87308761694455, 40.78585495239161], [-73.8783068007881, 40.785356620754236], [-73.88905212421602, 40.78737256012899], [-73.89282283610926, 40.79281708195147], [-73.89179289686568, 40.79677524575864], [-73.88885148496314, 40.79870632895875]]]}}, {\"id\": \"199\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5158437730287398, \"OBJECTID\": 200, \"Shape_Leng\": 0.112661735435, \"Shape_Area\": 0.000744643168558, \"zone\": \"Riverdale/North Riverdale/Fieldston\", \"LocationID\": 200, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.896411335, 40.90450452099991], [-73.89643170799987, 40.89365527899989], [-73.89787887499983, 40.89027610099993], [-73.9028594909999, 40.88916474399989], [-73.90413642399992, 40.887080708999896], [-73.90975930299997, 40.889250027999886], [-73.91489276999991, 40.8846247709999], [-73.92058641888602, 40.88718267830985], [-73.9103325682445, 40.91553277700258], [-73.89663333881637, 40.911417374923055], [-73.896411335, 40.90450452099991]]]}}, {\"id\": \"200\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.40789473684210525, \"OBJECTID\": 201, \"Shape_Leng\": 0.130404117686, \"Shape_Area\": 0.000619350747819, \"zone\": \"Rockaway Park\", \"LocationID\": 201, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.81849901800001, 40.5873623529999], [-73.81544517899997, 40.58337349199993], [-73.81730795916116, 40.58233475897], [-73.86255286029154, 40.56681001230082], [-73.86662812000002, 40.573057648999914], [-73.8504438514919, 40.582130243581396], [-73.83929597982016, 40.581863545478186], [-73.82480116623435, 40.587154615369926], [-73.81849901800001, 40.5873623529999]]]}}, {\"id\": \"201\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4365333836288193, \"OBJECTID\": 202, \"Shape_Leng\": 0.0699215712618, \"Shape_Area\": 0.000106175396322, \"zone\": \"Roosevelt Island\", \"LocationID\": 202, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94180032729437, 40.76904692662474], [-73.95254079336881, 40.757095314537246], [-73.96158304969282, 40.74941430105318], [-73.94472478498723, 40.76978627176733], [-73.94007665725826, 40.77292618617896], [-73.94180032729437, 40.76904692662474]]]}}, {\"id\": \"202\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4336734693877551, \"OBJECTID\": 203, \"Shape_Leng\": 0.189937815868, \"Shape_Area\": 0.000615129861659, \"zone\": \"Rosedale\", \"LocationID\": 203, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7275350030123, 40.673930284146124], [-73.72833071887973, 40.66304394386548], [-73.72522608103498, 40.652001000073724], [-73.74143706021165, 40.646889178160706], [-73.74227387212949, 40.64012338122567], [-73.739471216134, 40.63570643191466], [-73.74246642615259, 40.635119228252094], [-73.74096487686366, 40.63730477503007], [-73.74397128496219, 40.63788001261484], [-73.7437342150463, 40.6390221176639], [-73.7464954765674, 40.63651030383521], [-73.74516219973704, 40.63811218797305], [-73.74704226024105, 40.636932365240085], [-73.74631467883957, 40.641394313090785], [-73.7487767478731, 40.64511819098248], [-73.75522386899995, 40.64744446899989], [-73.7541108669999, 40.648697664999915], [-73.76406290099993, 40.65435626599989], [-73.76165732899996, 40.658238085999855], [-73.74867194699979, 40.65267808799992], [-73.74332546799988, 40.652537803999905], [-73.74328714099991, 40.65613473199991], [-73.74853177199994, 40.65757569699993], [-73.74281884699987, 40.66530554899986], [-73.73891022900001, 40.66595449199986], [-73.74117817299991, 40.666402953999885], [-73.72718272499995, 40.68359732999987], [-73.72586264191175, 40.683241080973865], [-73.72563005109944, 40.67958795089065], [-73.7275350030123, 40.673930284146124]]]}}, {\"id\": \"203\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.2608695652173913, \"OBJECTID\": 204, \"Shape_Leng\": 0.101912418311, \"Shape_Area\": 0.000641384265972, \"zone\": \"Rossville/Woodrow\", \"LocationID\": 204, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.2032648999999, 40.55562847699989], [-74.19174368799999, 40.53191669699986], [-74.20152661599995, 40.526265205999934], [-74.2156723039999, 40.52395762799989], [-74.21800676499996, 40.53229858999995], [-74.21776941599994, 40.54112662999992], [-74.22131562699998, 40.55025307399992], [-74.21784397799998, 40.55443271799988], [-74.21050789400002, 40.557064181999934], [-74.205854485, 40.55750245699993], [-74.2032648999999, 40.55562847699989]]]}}, {\"id\": \"204\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5065573770491804, \"OBJECTID\": 205, \"Shape_Leng\": 0.141953388327, \"Shape_Area\": 0.000766216513505, \"zone\": \"Saint Albans\", \"LocationID\": 205, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.766369252, 40.70256339699992], [-73.75205066499986, 40.70522802799995], [-73.74863325999986, 40.6954447609999], [-73.7435928609999, 40.69621532499986], [-73.75576169199995, 40.67588590899988], [-73.76380484799996, 40.67376993299995], [-73.77469730199986, 40.680486045999906], [-73.77220275299989, 40.681610338999896], [-73.7732870399999, 40.683454731999866], [-73.77699767599997, 40.68628268899992], [-73.77966850599998, 40.685377122999945], [-73.7832662499999, 40.68999429299992], [-73.78078155499992, 40.69102994299994], [-73.77977295099997, 40.689281169999916], [-73.77789337399993, 40.68991768999992], [-73.7802372379999, 40.692972334999894], [-73.77230768699997, 40.70057150699989], [-73.77559218799982, 40.70532930899986], [-73.7749009329999, 40.707066661999875], [-73.76844546499984, 40.709637026999886], [-73.766369252, 40.70256339699992]]]}}, {\"id\": \"205\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.44565217391304346, \"OBJECTID\": 206, \"Shape_Leng\": 0.212756793386, \"Shape_Area\": 0.000944392507762, \"zone\": \"Saint George/New Brighton\", \"LocationID\": 206, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08221272914942, 40.64828016229008], [-74.07165829759784, 40.64503374643502], [-74.07093784471476, 40.64334596384062], [-74.07210952444638, 40.64246321679278], [-74.07032923715386, 40.64252612813239], [-74.07314400586021, 40.641852974480116], [-74.07019399559213, 40.6420075028053], [-74.07289005609624, 40.64092267287602], [-74.06998354653952, 40.64106894311676], [-74.07285041194942, 40.64080368844508], [-74.07258133886755, 40.63794187429597], [-74.07865269699992, 40.63830402499987], [-74.08373397299994, 40.63573714399991], [-74.08870582299987, 40.63684520599986], [-74.09242513499994, 40.63558181299992], [-74.09409312599998, 40.639266913999975], [-74.09767290799995, 40.63896067899989], [-74.09787969199995, 40.64035805499988], [-74.10974833299994, 40.63786583299993], [-74.11765722499996, 40.638062466999905], [-74.11614321199995, 40.62870226399988], [-74.1132624959999, 40.62305442499995], [-74.12125677899994, 40.62782274999989], [-74.1313098529999, 40.62632834299998], [-74.12867047299994, 40.63734826499993], [-74.1268887269999, 40.637210087999925], [-74.12705362202625, 40.64024173781949], [-74.11861635236262, 40.641530365028935], [-74.11883624616948, 40.64254186406039], [-74.1171709328468, 40.64162427104175], [-74.1172247008255, 40.64302735996259], [-74.11535082182925, 40.64247054015812], [-74.1097393256449, 40.64546372818479], [-74.09880066885808, 40.64504703431087], [-74.08570754378212, 40.648880553370454], [-74.08221272914942, 40.64828016229008]]]}}, {\"id\": \"206\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.8937827830916384, \"OBJECTID\": 207, \"Shape_Leng\": 0.0283756831492, \"Shape_Area\": 4.47230848441e-05, \"zone\": \"Saint Michaels Cemetery/Woodside\", \"LocationID\": 207, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90137640299986, 40.75964203799989], [-73.90356915599997, 40.765781788999924], [-73.90217951699996, 40.76731584199997], [-73.89349605899986, 40.765950929999924], [-73.89866137899992, 40.75992101699997], [-73.90137640299986, 40.75964203799989]]]}}, {\"id\": \"207\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5824493731918997, \"OBJECTID\": 208, \"Shape_Leng\": 0.214721009581, \"Shape_Area\": 0.00202032598969, \"zone\": \"Schuylerville/Edgewater Park\", \"LocationID\": 208, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.825986869, 40.84067575199988], [-73.82601252600003, 40.83402077799985], [-73.81552500922504, 40.83597630692601], [-73.81476387691956, 40.830816799993485], [-73.81275469994516, 40.82824823625135], [-73.80992515914596, 40.82817971199259], [-73.81384974851248, 40.8268035287504], [-73.81379523517661, 40.82529808600147], [-73.81211350556435, 40.82555263465937], [-73.81388934712506, 40.82408736742192], [-73.80766604765235, 40.825896992059], [-73.80451800414906, 40.81892768751854], [-73.79759032777322, 40.816507532070055], [-73.80140380968845, 40.8123633086586], [-73.80361390158114, 40.812521106621354], [-73.80453081609834, 40.81425461992442], [-73.80351667337368, 40.81395196364859], [-73.80704600153508, 40.816533693875236], [-73.80505213646254, 40.81289496243412], [-73.80224245334347, 40.81019739366033], [-73.79086475738406, 40.80734114044052], [-73.79028410874172, 40.80481710368888], [-73.7933898228672, 40.804204330036214], [-73.80185524738366, 40.808967826140574], [-73.80392673417204, 40.80852797081171], [-73.8099202908178, 40.81292915226176], [-73.81620056896291, 40.81384747647932], [-73.83017915330238, 40.81083454948945], [-73.8321661704888, 40.80850227767013], [-73.831615744777, 40.80493825599657], [-73.83743612112924, 40.80620264259155], [-73.84043199520461, 40.81254112811847], [-73.83974355921865, 40.816406071751366], [-73.83696070299992, 40.81634054099991], [-73.83614239499985, 40.81919890299987], [-73.83661819699992, 40.82711414899993], [-73.83709061399992, 40.82896714699996], [-73.84233077477266, 40.8290739624537], [-73.83885232384739, 40.83369066784824], [-73.83956700299987, 40.84054722799986], [-73.83771374899992, 40.840406309999885], [-73.83688925199996, 40.84337478799992], [-73.82611300699996, 40.84533869399991], [-73.825986869, 40.84067575199988]]], [[[-73.83960098545435, 40.83567071869858], [-73.83961672561374, 40.835667752627316], [-73.83962599488439, 40.83569971218477], [-73.83960850400152, 40.83570805084221], [-73.83960098545435, 40.83567071869858]]], [[[-73.83979488581072, 40.835619207246324], [-73.8396148723857, 40.83561056836579], [-73.83948374192397, 40.83457420079436], [-73.83959266810578, 40.83456047582325], [-73.83979488581072, 40.835619207246324]]]]}}, {\"id\": \"208\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3496639011687761, \"OBJECTID\": 209, \"Shape_Leng\": 0.0306953113473, \"Shape_Area\": 3.88703351209e-05, \"zone\": \"Seaport\", \"LocationID\": 209, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00499805099996, 40.71201189699987], [-73.99919451154408, 40.707947376175724], [-74.00143661179398, 40.70487217770521], [-74.00906264699988, 40.71088670399993], [-74.00499805099996, 40.71201189699987]]]}}, {\"id\": \"209\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5185512367491166, \"OBJECTID\": 210, \"Shape_Leng\": 0.114851983372, \"Shape_Area\": 0.000453155131253, \"zone\": \"Sheepshead Bay\", \"LocationID\": 210, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92759936700004, 40.5950607789999], [-73.91805571865743, 40.58643685311621], [-73.92520361514241, 40.58580627115536], [-73.92802185276429, 40.588068756350886], [-73.9239529485009, 40.59113985052502], [-73.92863061415638, 40.58857263642073], [-73.92844090493661, 40.589030428141996], [-73.9291344038264, 40.588879241681944], [-73.92875703499415, 40.589152966925], [-73.92976615773313, 40.589292069447644], [-73.93003427450557, 40.58942599650075], [-73.93057705718242, 40.589890641203176], [-73.93010325988745, 40.58960610507142], [-73.93001650952081, 40.5940945360621], [-73.93260420556844, 40.59492954452495], [-73.93121294858955, 40.58953130811921], [-73.92972846380192, 40.58861391582969], [-73.93125191267133, 40.58684982521422], [-73.92857393936458, 40.58706150237135], [-73.92552860874584, 40.58497837657265], [-73.9344471859999, 40.586652074999925], [-73.96025825899996, 40.58502930799994], [-73.9603495359999, 40.58730628599994], [-73.95619313699986, 40.58776226899994], [-73.956927197, 40.596612075999865], [-73.94248473299992, 40.598200364999855], [-73.94288954099984, 40.600318649999934], [-73.93447158899997, 40.60123151199989], [-73.9335511719998, 40.60268728099991], [-73.92813139699999, 40.59777792799987], [-73.92956858299992, 40.5968202649999], [-73.92759936700004, 40.5950607789999]]]}}, {\"id\": \"210\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3508851733003059, \"OBJECTID\": 211, \"Shape_Leng\": 0.0252345082132, \"Shape_Area\": 3.97291966087e-05, \"zone\": \"SoHo\", \"LocationID\": 211, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.001523638, 40.719820797999944], [-74.00539186699989, 40.722037365999896], [-74.00214891499996, 40.728035366999904], [-73.9967717579999, 40.7254319439999], [-74.001523638, 40.719820797999944]]]}}, {\"id\": \"211\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6535480023852117, \"OBJECTID\": 212, \"Shape_Leng\": 0.0671290653429, \"Shape_Area\": 0.000161276191858, \"zone\": \"Soundview/Bruckner\", \"LocationID\": 212, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88421976599987, 40.82299645799987], [-73.88378414099992, 40.828274291999946], [-73.87729930499995, 40.8287427739999], [-73.86085019699983, 40.833613537999874], [-73.857391213, 40.83229737199986], [-73.85615112900003, 40.82677004599987], [-73.88421976599987, 40.82299645799987]]]}}, {\"id\": \"212\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6376811594202898, \"OBJECTID\": 213, \"Shape_Leng\": 0.159958783822, \"Shape_Area\": 0.000904077906339, \"zone\": \"Soundview/Castle Hill\", \"LocationID\": 213, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.83667856399991, 40.81758876299995], [-73.83696070299992, 40.81634054099991], [-73.84003308726955, 40.81682214096213], [-73.83888231646733, 40.822140489856025], [-73.84228428387955, 40.82864660449135], [-73.83709061399992, 40.82896714699996], [-73.83667856399991, 40.81758876299995]]], [[[-73.84231712480313, 40.82718468468027], [-73.83999860407422, 40.81993494681306], [-73.84198684834051, 40.818774373908084], [-73.84617302872044, 40.81067666443039], [-73.8511447226069, 40.814396548876], [-73.85538030243693, 40.81428888208278], [-73.84903294180673, 40.81211891007852], [-73.84996696802818, 40.808578018843576], [-73.847488226419, 40.805448909149185], [-73.85019839036403, 40.80451002497395], [-73.856614854502, 40.80479692395033], [-73.85866549456955, 40.806833088596065], [-73.85895141221772, 40.810271417586556], [-73.8601269523559, 40.80960549989125], [-73.85935625734362, 40.80898803428646], [-73.85938875426787, 40.80816614120708], [-73.86065222514291, 40.80963668169183], [-73.86765646284456, 40.81058376193711], [-73.87077804885932, 40.814486895487775], [-73.8780628329999, 40.81618263199993], [-73.88244779599998, 40.81926832299992], [-73.8843985489999, 40.822965575999866], [-73.85635677499994, 40.82675208499993], [-73.85513639699997, 40.822436188999866], [-73.84289858800003, 40.82408305899986], [-73.84390624999999, 40.828407828999936], [-73.84231712480313, 40.82718468468027]]]]}}, {\"id\": \"213\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4583333333333333, \"OBJECTID\": 214, \"Shape_Leng\": 0.135300314246, \"Shape_Area\": 0.000811976523012, \"zone\": \"South Beach/Dongan Hills\", \"LocationID\": 214, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08491019899996, 40.59570472799985], [-74.08173342999991, 40.59215628599987], [-74.07043976099992, 40.59673474699988], [-74.06436003721248, 40.58826789117477], [-74.07499413073505, 40.57923655741631], [-74.07336118472098, 40.578275488289144], [-74.07508288192615, 40.579160158755506], [-74.08495715819193, 40.57125984002107], [-74.10039083499996, 40.579688665999925], [-74.09844000899997, 40.581241705999915], [-74.10525775499997, 40.58592143599986], [-74.10088251699995, 40.59017921599992], [-74.09979588999995, 40.59387136599988], [-74.09375453799998, 40.59674488999988], [-74.08983655899989, 40.60077409499996], [-74.08218729299988, 40.601941007999876], [-74.08130343799999, 40.59938765599991], [-74.08532650099991, 40.59700832199995], [-74.08491019899996, 40.59570472799985]]]}}, {\"id\": \"214\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4187683900798655, \"OBJECTID\": 215, \"Shape_Leng\": 0.103556453625, \"Shape_Area\": 0.000395647022725, \"zone\": \"South Jamaica\", \"LocationID\": 215, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7769103769999, 40.70252488499994], [-73.77395659499986, 40.699805781999906], [-73.7802372379999, 40.692972334999894], [-73.77789337399993, 40.68991768999992], [-73.77977295099997, 40.689281169999916], [-73.78078155499992, 40.69102994299994], [-73.793908417, 40.68630469099985], [-73.79561486799987, 40.688656890999916], [-73.79995042799996, 40.68682771899991], [-73.79898209099994, 40.68485873799988], [-73.80576840199991, 40.682932390999916], [-73.81063298299996, 40.69194738299988], [-73.80890216900004, 40.69256434899993], [-73.80642479399984, 40.690873857999904], [-73.79260506799987, 40.70043209999988], [-73.79239170299988, 40.7021665299999], [-73.78256582599995, 40.70542486399989], [-73.78034491200003, 40.70541241699986], [-73.7769103769999, 40.70252488499994]]]}}, {\"id\": \"215\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4507669470559129, \"OBJECTID\": 216, \"Shape_Leng\": 0.116797554681, \"Shape_Area\": 0.000816076647781, \"zone\": \"South Ozone Park\", \"LocationID\": 216, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.80857655199986, 40.688330889999925], [-73.80142836799988, 40.673666833999874], [-73.80203207699995, 40.66524802499994], [-73.81496922499986, 40.66247266799991], [-73.832705339, 40.665405772999854], [-73.84012856099982, 40.6811466509999], [-73.83697486299991, 40.6819880179999], [-73.83807769099997, 40.68423890999988], [-73.82753021399995, 40.687239482999914], [-73.82851136799994, 40.689239055999934], [-73.82599347399987, 40.689953661999894], [-73.82501128899997, 40.68795231799991], [-73.81063298299996, 40.69194738299988], [-73.80857655199986, 40.688330889999925]]]}}, {\"id\": \"216\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5170361472097849, \"OBJECTID\": 217, \"Shape_Leng\": 0.0553905251285, \"Shape_Area\": 0.000114733270758, \"zone\": \"South Williamsburg\", \"LocationID\": 217, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95554165799997, 40.70759344799989], [-73.94705205299991, 40.70366394899985], [-73.95128819399989, 40.700922364999855], [-73.95381196900003, 40.703180979999914], [-73.95745736399992, 40.70082260299993], [-73.95701993199991, 40.69897391399995], [-73.96283963699993, 40.698038667999946], [-73.96217978199998, 40.70022070999989], [-73.96838933699995, 40.70682918699989], [-73.95554165799997, 40.70759344799989]]]}}, {\"id\": \"217\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.55, \"OBJECTID\": 218, \"Shape_Leng\": 0.0837008281049, \"Shape_Area\": 0.000281293736407, \"zone\": \"Springfield Gardens North\", \"LocationID\": 218, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76561573099998, 40.6747790199999], [-73.76380484799996, 40.67376993299995], [-73.76031463999993, 40.67510997099984], [-73.75857198899986, 40.67263733099987], [-73.75636239300002, 40.67242923699999], [-73.75763129599993, 40.66645115699992], [-73.77342462799993, 40.66802761899988], [-73.78940999399987, 40.66684115899995], [-73.78808647399985, 40.6730096269999], [-73.78573730899993, 40.67238503499988], [-73.77722923600001, 40.679652720999904], [-73.77469730199986, 40.680486045999906], [-73.76561573099998, 40.6747790199999]]]}}, {\"id\": \"218\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4395918367346939, \"OBJECTID\": 219, \"Shape_Leng\": 0.155126669675, \"Shape_Area\": 0.000428176336771, \"zone\": \"Springfield Gardens South\", \"LocationID\": 219, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.761822313, 40.666877759999856], [-73.73856469500002, 40.66639689799991], [-73.74450922599979, 40.663821277999865], [-73.74853177199994, 40.65757569699993], [-73.74328714099991, 40.65613473199991], [-73.74332546899993, 40.65253780399992], [-73.74867194699979, 40.65267808799992], [-73.76165732899996, 40.658238085999855], [-73.76406290099993, 40.65435626599989], [-73.78321833499997, 40.663166283999864], [-73.80203207699995, 40.66524802499994], [-73.80155565300004, 40.6667533639999], [-73.77342462799993, 40.66802761899988], [-73.761822313, 40.666877759999856]]]}}, {\"id\": \"219\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6061746987951807, \"OBJECTID\": 220, \"Shape_Leng\": 0.106015943267, \"Shape_Area\": 0.000313035744467, \"zone\": \"Spuyten Duyvil/Kingsbridge\", \"LocationID\": 220, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89827216899992, 40.8894579429999], [-73.89972588599989, 40.88651488699985], [-73.89661546599986, 40.885955255999946], [-73.90369454799992, 40.878501617999895], [-73.90608130599988, 40.873239628999904], [-73.90746489699994, 40.87354735399985], [-73.9068187309999, 40.87661958799993], [-73.91033193599998, 40.879038046999895], [-73.91775669364831, 40.8756636279301], [-73.92490327486523, 40.87888836792653], [-73.92058641888602, 40.88718267830985], [-73.91489276999991, 40.8846247709999], [-73.90900536, 40.88950014099986], [-73.90413642399992, 40.887080708999896], [-73.9028594909999, 40.88916474399989], [-73.89827216899992, 40.8894579429999]]]}}, {\"id\": \"220\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.51875, \"OBJECTID\": 221, \"Shape_Leng\": 0.166218220846, \"Shape_Area\": 0.000890111567051, \"zone\": \"Stapleton\", \"LocationID\": 221, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.07530550099992, 40.637729958999955], [-74.07053959705199, 40.63747140462426], [-74.07341562632824, 40.63689395096554], [-74.07209483807503, 40.63664727762461], [-74.07335822045584, 40.63653609416517], [-74.0729847461233, 40.630308978981404], [-74.06802173543946, 40.62878758518541], [-74.07291177936548, 40.62999637940653], [-74.07225500955778, 40.62460875019394], [-74.06975543088323, 40.6212081398724], [-74.06544586069478, 40.61920262828925], [-74.06668207982919, 40.61806749593851], [-74.06425721751587, 40.61817168758555], [-74.06509908407213, 40.61751095612543], [-74.05685812390232, 40.608056357772554], [-74.0651709129999, 40.604213507999894], [-74.06350565699996, 40.60215005499991], [-74.07399965299999, 40.60572556699987], [-74.0815547009999, 40.60629029699987], [-74.0714060829999, 40.61943137899991], [-74.07253551699989, 40.622464021999896], [-74.08720982799996, 40.61330697299989], [-74.0878234189999, 40.615087934999885], [-74.08415419899998, 40.626267114999905], [-74.07972492199995, 40.62895068799986], [-74.08574928999992, 40.633955556999865], [-74.07878831099995, 40.63826855899991], [-74.07530550099992, 40.637729958999955]]]}}, {\"id\": \"221\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.48883374689826303, \"OBJECTID\": 222, \"Shape_Leng\": 0.0483893527976, \"Shape_Area\": 0.000130268343918, \"zone\": \"Starrett City\", \"LocationID\": 222, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.87908517399995, 40.65020211699988], [-73.8745808109999, 40.64561010599992], [-73.87841589406925, 40.638912189353306], [-73.88006714132425, 40.64200344662675], [-73.88341999203011, 40.64452381564944], [-73.88557762124442, 40.644040950146135], [-73.88764072166725, 40.64780239000726], [-73.89125147199995, 40.65002559699993], [-73.88221364899994, 40.65451863799996], [-73.87908517399995, 40.65020211699988]]], [[[-73.8882853135604, 40.64672241327608], [-73.88864137620679, 40.647079553158], [-73.88851637568528, 40.64736069152191], [-73.88820050726648, 40.647140869529046], [-73.8882853135604, 40.64672241327608]]]]}}, {\"id\": \"222\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.49356204481304666, \"OBJECTID\": 223, \"Shape_Leng\": 0.166021925275, \"Shape_Area\": 0.00076436070058, \"zone\": \"Steinway\", \"LocationID\": 223, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90649492923363, 40.7901809903447], [-73.90032629743816, 40.789007662825696], [-73.8959296297551, 40.785648944313756], [-73.89874897638398, 40.78278115576602], [-73.90114577552107, 40.78271110609054], [-73.90257929672345, 40.78028912083556], [-73.89437777934127, 40.78424320348422], [-73.89606377738421, 40.783157201805224], [-73.895284679113, 40.78154124925482], [-73.89274917876409, 40.78296570668481], [-73.89165459123714, 40.78219478153064], [-73.89300986180193, 40.78179802356213], [-73.89119589704391, 40.77856328788702], [-73.89251064542336, 40.77739286927271], [-73.89183244902246, 40.77488019994431], [-73.88945486901865, 40.77353295106357], [-73.88770661799987, 40.76683859799995], [-73.89349605899986, 40.765950929999924], [-73.91759120499997, 40.769900870999926], [-73.92398142900002, 40.77461511299991], [-73.92375761299998, 40.77654701199991], [-73.91796581399993, 40.78125629199991], [-73.91996458796496, 40.78263506876824], [-73.912599859115, 40.78937632963394], [-73.90985862925774, 40.790945493781884], [-73.90649492923363, 40.7901809903447]]]}}, {\"id\": \"223\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.30801459765711636, \"OBJECTID\": 224, \"Shape_Leng\": 0.0440204462769, \"Shape_Area\": 9.84786989135e-05, \"zone\": \"Stuy Town/Peter Cooper Village\", \"LocationID\": 224, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97511055399991, 40.735390254999864], [-73.97151882628022, 40.72935136174009], [-73.97162697285862, 40.72662845299842], [-73.98255629299986, 40.731350133999875], [-73.97849845599995, 40.73679095699989], [-73.97511055399991, 40.735390254999864]]]}}, {\"id\": \"224\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.46939986043265874, \"OBJECTID\": 225, \"Shape_Leng\": 0.0887992073281, \"Shape_Area\": 0.000310835395162, \"zone\": \"Stuyvesant Heights\", \"LocationID\": 225, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91741294299999, 40.68403405199993], [-73.9163011989999, 40.67857711199991], [-73.92183500099998, 40.67894555899991], [-73.92281374299995, 40.68341442999991], [-73.93500821699992, 40.68276376799985], [-73.93435066499993, 40.67956340799991], [-73.94032794, 40.679889974999874], [-73.94394947299996, 40.698221278999924], [-73.93817718599986, 40.69864022299987], [-73.91804607, 40.68721324799989], [-73.91741294299999, 40.68403405199993]]]}}, {\"id\": \"225\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5400358347151585, \"OBJECTID\": 226, \"Shape_Leng\": 0.168957762041, \"Shape_Area\": 0.00076194414504, \"zone\": \"Sunnyside\", \"LocationID\": 226, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90989961499994, 40.749087274999944], [-73.91187504099992, 40.74822079599992], [-73.9119412979999, 40.742382948999925], [-73.91567469599988, 40.740784929999904], [-73.91903825199986, 40.73200596599987], [-73.917342506, 40.731382825999916], [-73.91796696699996, 40.7285007709999], [-73.91205021299994, 40.72850636299993], [-73.91174734199988, 40.72991203299991], [-73.90712253099991, 40.727424727999924], [-73.91384828299984, 40.72289841199989], [-73.91132095099978, 40.71962422999992], [-73.92232773978448, 40.71659264406108], [-73.92515207271921, 40.722335339820816], [-73.9244229875485, 40.723589975986165], [-73.92037064235664, 40.723681113821044], [-73.92490316165177, 40.72450657567146], [-73.92923485876162, 40.72826016576104], [-73.93804478507161, 40.730509967846054], [-73.94162472636452, 40.73584144233816], [-73.94594410333028, 40.73751251570048], [-73.94140886345744, 40.73929957085316], [-73.93869664599998, 40.738140816999866], [-73.93726110699994, 40.745176490999896], [-73.94401030399996, 40.7436394539999], [-73.93383269399993, 40.7516047019999], [-73.91547326299991, 40.75379181799991], [-73.91035492999984, 40.75274990899987], [-73.90989961499994, 40.749087274999944]]]}}, {\"id\": \"226\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5770998746343502, \"OBJECTID\": 227, \"Shape_Leng\": 0.0828301570789, \"Shape_Area\": 0.000268316529534, \"zone\": \"Sunset Park East\", \"LocationID\": 227, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.991928124, 40.6422049039999], [-73.9955103369999, 40.63980966699993], [-73.99917564699989, 40.642025441999934], [-74.00735636699996, 40.6341636269999], [-74.00556490399993, 40.63308317599992], [-74.01000340799995, 40.630113088999906], [-74.01602010100001, 40.63331711399997], [-74.016835401, 40.63535542899995], [-74.0139608859999, 40.63815326699995], [-74.01616711499993, 40.63948244099991], [-74.00214474699993, 40.65295610699995], [-73.98905872499985, 40.64411924099995], [-73.991928124, 40.6422049039999]]]}}, {\"id\": \"227\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4474304804952344, \"OBJECTID\": 228, \"Shape_Leng\": 0.177685177706, \"Shape_Area\": 0.000993414783816, \"zone\": \"Sunset Park West\", \"LocationID\": 228, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9969179149999, 40.6691534839999], [-73.99490350099988, 40.6667854349999], [-73.99692689899985, 40.664374914999925], [-73.99063545399997, 40.660579088999896], [-73.988716511, 40.66240493799995], [-73.98725068899996, 40.66037895599992], [-73.990188415, 40.657600817999906], [-73.99516890199999, 40.659669521999945], [-74.01616711499993, 40.63948244099991], [-74.0139608859999, 40.63815326699995], [-74.016835401, 40.63535542899995], [-74.02141038399992, 40.638584902999845], [-74.02108051399998, 40.64149093699994], [-74.02475437599998, 40.6381214179999], [-74.03266558998958, 40.643720344890056], [-74.03051830863413, 40.645274425208974], [-74.02820569545007, 40.644015982890345], [-74.02604948400315, 40.64626135032611], [-74.0293899048662, 40.64880405511391], [-74.02571758804807, 40.64661549229093], [-74.02461487103106, 40.64715112511878], [-74.02622102475574, 40.64824842780924], [-74.02414566050564, 40.647615101259845], [-74.02605133196793, 40.6509943007431], [-74.02285141795983, 40.651169195161685], [-74.02522662972656, 40.652829856959634], [-74.02154138319214, 40.650671622911226], [-74.02380703903121, 40.65304472851146], [-74.02118670027885, 40.652839836895325], [-74.02320464727141, 40.654096116817925], [-74.0186223830857, 40.653491349022964], [-74.02005182454292, 40.655335948874416], [-74.01715319581021, 40.65477510436213], [-74.01970596410222, 40.656581163751035], [-74.01639437533773, 40.65562064285071], [-74.01910843803118, 40.657368884303494], [-74.01585370056823, 40.65621324055534], [-74.01786573109669, 40.65830844729852], [-74.01442130484548, 40.657608021951376], [-74.01745803087898, 40.6594501973867], [-74.01554165388997, 40.66075835547614], [-74.01183862056543, 40.65891283727472], [-74.0131555493881, 40.66198159772593], [-74.00859956968284, 40.65952092309827], [-74.00745858163884, 40.66055847620986], [-74.0103720129723, 40.66239888023248], [-74.00955074142026, 40.6632676941304], [-74.0034880110065, 40.66224055612849], [-74.00599748682693, 40.66325861082859], [-74.00772133506703, 40.66471863301977], [-74.0071253264673, 40.66491521847423], [-74.00362862508561, 40.66273484753458], [-74.00486193792649, 40.66506482193817], [-74.00083401108444, 40.66300644269171], [-73.99953152609916, 40.6643974859369], [-74.00292463516384, 40.66646231216332], [-73.99998149339123, 40.66745027490869], [-73.99902600998384, 40.66844297212693], [-73.9986188799999, 40.67183017199989], [-73.9969179149999, 40.6691534839999]]]}}, {\"id\": \"228\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.37824867930601785, \"OBJECTID\": 229, \"Shape_Leng\": 0.0425252900116, \"Shape_Area\": 9.47058230931e-05, \"zone\": \"Sutton Place/Turtle Bay North\", \"LocationID\": 229, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96162927534907, 40.75515238425915], [-73.96482214210387, 40.75159632579112], [-73.9712170449999, 40.75519332399994], [-73.9665833839999, 40.76155093499988], [-73.95877790811335, 40.75827092092435], [-73.96162927534907, 40.75515238425915]]]}}, {\"id\": \"229\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4514735032996653, \"OBJECTID\": 230, \"Shape_Leng\": 0.0310283096779, \"Shape_Area\": 5.60914463266e-05, \"zone\": \"Times Sq/Theatre District\", \"LocationID\": 230, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98362433299988, 40.75551634599986], [-73.98979085399996, 40.7572331319999], [-73.98428230099987, 40.76479181499994], [-73.97860169399989, 40.762396071999945], [-73.98362433299988, 40.75551634599986]]]}}, {\"id\": \"230\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.31503652615196714, \"OBJECTID\": 231, \"Shape_Leng\": 0.0634201165587, \"Shape_Area\": 0.000166929752992, \"zone\": \"TriBeCa/Civic Center\", \"LocationID\": 231, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00931760399995, 40.72470155099989], [-73.99995620900002, 40.71801709499987], [-74.00485768099996, 40.7124525409999], [-74.00834760999993, 40.71135854599987], [-74.01375388499997, 40.71369055499987], [-74.01244109199989, 40.719057672999874], [-74.01457231438083, 40.72027413774219], [-74.01296558526825, 40.720328675587126], [-74.01102457999994, 40.72579386199991], [-74.00931760399995, 40.72470155099989]]]}}, {\"id\": \"231\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3511347457547214, \"OBJECTID\": 232, \"Shape_Leng\": 0.0614709085331, \"Shape_Area\": 0.000216049973456, \"zone\": \"Two Bridges/Seward Park\", \"LocationID\": 232, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97635525199995, 40.71913288399983], [-73.97347975524508, 40.7188614291129], [-73.97782091638247, 40.71064544490706], [-73.9919434659999, 40.70958905299992], [-73.99256242199989, 40.71438807699996], [-73.99022012699989, 40.71440493999988], [-73.98382387299982, 40.72147287199987], [-73.97635525199995, 40.71913288399983]]]}}, {\"id\": \"232\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.34280284974617936, \"OBJECTID\": 233, \"Shape_Leng\": 0.048035918294, \"Shape_Area\": 0.000116188094682, \"zone\": \"UN/Turtle Bay South\", \"LocationID\": 233, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.96547092299994, 40.75193959699991], [-73.97123301520192, 40.743414663960074], [-73.97350835000003, 40.7437820339999], [-73.97121971999988, 40.74692154099996], [-73.9758346669999, 40.74886314599996], [-73.9712170449999, 40.75519332399994], [-73.96547092299994, 40.75193959699991]]], [[[-73.96421230395673, 40.74660431847665], [-73.96444522752618, 40.74641042576318], [-73.96458318426942, 40.746454400673315], [-73.96415980288437, 40.74686554735954], [-73.96421230395673, 40.74660431847665]]]]}}, {\"id\": \"233\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.30875189761032007, \"OBJECTID\": 234, \"Shape_Leng\": 0.0360721994984, \"Shape_Area\": 7.31054382894e-05, \"zone\": \"Union Sq\", \"LocationID\": 234, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98996936399989, 40.73490456699994], [-73.99683993899998, 40.73736088899995], [-73.99051761299985, 40.746038637999895], [-73.98407485299995, 40.74332471799989], [-73.98996936399989, 40.73490456699994]]]}}, {\"id\": \"234\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6193901035673187, \"OBJECTID\": 235, \"Shape_Leng\": 0.0761668426519, \"Shape_Area\": 0.000212844547337, \"zone\": \"University Heights/Morris Heights\", \"LocationID\": 235, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90947862999988, 40.86180812899987], [-73.91328437000003, 40.859618600999944], [-73.90976165899988, 40.857182611999896], [-73.90679734899983, 40.85937358899988], [-73.90468371399987, 40.85706030199991], [-73.91416184199993, 40.845179187999925], [-73.92808209386837, 40.84539159691672], [-73.91422824049913, 40.862490790359615], [-73.90947862999988, 40.86180812899987]]]}}, {\"id\": \"235\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3990667864186673, \"OBJECTID\": 236, \"Shape_Leng\": 0.0442519223099, \"Shape_Area\": 0.000102864345211, \"zone\": \"Upper East Side North\", \"LocationID\": 236, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95779380499984, 40.773599896999926], [-73.96468735799992, 40.77567708199994], [-73.9557773589999, 40.78791392399995], [-73.94933170599992, 40.78519312699994], [-73.95779380499984, 40.773599896999926]]]}}, {\"id\": \"236\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4185696491397384, \"OBJECTID\": 237, \"Shape_Leng\": 0.0422126003388, \"Shape_Area\": 9.59873206621e-05, \"zone\": \"Upper East Side South\", \"LocationID\": 237, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9661274729999, 40.762179299999914], [-73.97301487199995, 40.76427887899991], [-73.96468735799992, 40.77567708199994], [-73.95825534899998, 40.772965336999896], [-73.9661274729999, 40.762179299999914]]]}}, {\"id\": \"237\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3847097692019964, \"OBJECTID\": 238, \"Shape_Leng\": 0.0601093114033, \"Shape_Area\": 0.000184763693765, \"zone\": \"Upper West Side North\", \"LocationID\": 238, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96884378999985, 40.78596738899994], [-73.98188645923244, 40.7905165804675], [-73.97639951965265, 40.79827321084623], [-73.96417598599989, 40.79236204499989], [-73.96884378999985, 40.78596738899994]]]}}, {\"id\": \"238\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3883914600771428, \"OBJECTID\": 239, \"Shape_Leng\": 0.0636261152958, \"Shape_Area\": 0.000204715440774, \"zone\": \"Upper West Side South\", \"LocationID\": 239, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97501417199996, 40.78768560599987], [-73.96933573299991, 40.78529327999992], [-73.97499744000007, 40.77753254599988], [-73.98507184299989, 40.781779680999946], [-73.98567936099997, 40.780312632999944], [-73.9881274613434, 40.78140179672366], [-73.98546581197031, 40.78536070057545], [-73.98711901394246, 40.7852103190041], [-73.98542932126932, 40.78541394218458], [-73.98465507883023, 40.78653474180792], [-73.98594956155647, 40.78648711396651], [-73.98188645923244, 40.7905165804675], [-73.97501417199996, 40.78768560599987]]]}}, {\"id\": \"239\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5970464135021097, \"OBJECTID\": 240, \"Shape_Leng\": 0.146069764379, \"Shape_Area\": 0.000722130920707, \"zone\": \"Van Cortlandt Park\", \"LocationID\": 240, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87643743099994, 40.89687059299995], [-73.8771197409999, 40.895468663999964], [-73.87048672599987, 40.896640210999905], [-73.86135927899984, 40.89466231899991], [-73.86943471300005, 40.87812919899994], [-73.87094367500002, 40.87851076699998], [-73.87024100099985, 40.88022935299995], [-73.87494589799991, 40.87994766899988], [-73.87467601499999, 40.88156285199993], [-73.87812594399992, 40.88215382599989], [-73.87803562399996, 40.887118321999885], [-73.88124422499992, 40.88270963299992], [-73.89972588499982, 40.886514886999926], [-73.89643170799987, 40.89365527899989], [-73.89663333831646, 40.91141737492314], [-73.86789043706806, 40.90298695407611], [-73.87643743099994, 40.89687059299995]]]}}, {\"id\": \"240\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6392638036809816, \"OBJECTID\": 241, \"Shape_Leng\": 0.0687645777649, \"Shape_Area\": 0.00025467697592, \"zone\": \"Van Cortlandt Village\", \"LocationID\": 241, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88840157099996, 40.88441707499993], [-73.88705142599984, 40.884349575999906], [-73.88567517999988, 40.87887251099986], [-73.88937163799984, 40.87338229799993], [-73.8879350369998, 40.87203023199988], [-73.89438130299995, 40.866068850999966], [-73.89891854299988, 40.86965857399992], [-73.89942436199988, 40.86788083799993], [-73.90374975899987, 40.86959562699992], [-73.90608130599988, 40.873239628999904], [-73.89855059899999, 40.88437472699996], [-73.89661546599986, 40.885955255999946], [-73.88840157099996, 40.88441707499993]]]}}, {\"id\": \"241\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6652794292508918, \"OBJECTID\": 242, \"Shape_Leng\": 0.138136446433, \"Shape_Area\": 0.000360072450014, \"zone\": \"Van Nest/Morris Park\", \"LocationID\": 242, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83593362199987, 40.84840635599993], [-73.83771374799997, 40.84040630999987], [-73.84204144699997, 40.84064589699992], [-73.84458129199986, 40.838535507999914], [-73.85052278099987, 40.84278948899985], [-73.87359713599987, 40.83979855499992], [-73.86804200399997, 40.8448138389999], [-73.86860515599989, 40.85206864599991], [-73.86303938099996, 40.850611306999944], [-73.86250923799993, 40.84836228099987], [-73.85457388299992, 40.85150834599995], [-73.85304526899994, 40.849547331999865], [-73.84430846999992, 40.85119046399989], [-73.84494053799982, 40.8482182209999], [-73.84049767399992, 40.8541841049999], [-73.83499552599994, 40.85657120099989], [-73.8368191849999, 40.85727680899989], [-73.82847346699995, 40.860731352999906], [-73.83283502299986, 40.8560358449999], [-73.83129157699989, 40.85543410499991], [-73.83463970199993, 40.853817993999904], [-73.83593362199987, 40.84840635599993]]]}}, {\"id\": \"242\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4757057629620518, \"OBJECTID\": 243, \"Shape_Leng\": 0.0943314999364, \"Shape_Area\": 0.000438377341503, \"zone\": \"Washington Heights North\", \"LocationID\": 243, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93156536999994, 40.86958215799991], [-73.92474153399992, 40.86159363799992], [-73.92709951599986, 40.85827552099992], [-73.92467094699998, 40.8568108889999], [-73.92747680599985, 40.85069436999992], [-73.93139355399987, 40.84742823099985], [-73.94203871799984, 40.851931028999914], [-73.94317806099995, 40.84979994299991], [-73.94692626452422, 40.850528055834154], [-73.94186996426673, 40.853867739442755], [-73.93156536999994, 40.86958215799991]]]}}, {\"id\": \"243\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4486486486486487, \"OBJECTID\": 244, \"Shape_Leng\": 0.0805686044599, \"Shape_Area\": 0.000359702747951, \"zone\": \"Washington Heights South\", \"LocationID\": 244, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94068822000003, 40.85131543299985], [-73.93139355399987, 40.84742823099985], [-73.93556697199989, 40.841712527999924], [-73.93505508799984, 40.8393332359999], [-73.94034643299983, 40.8304578419999], [-73.95015521048293, 40.83439675940791], [-73.94612416421772, 40.84389249655712], [-73.9469644175959, 40.85046552581819], [-73.94317806099984, 40.84979994399989], [-73.94203871799984, 40.851931028999914], [-73.94068822000003, 40.85131543299985]]]}}, {\"id\": \"244\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.40476190476190477, \"OBJECTID\": 245, \"Shape_Leng\": 0.0959830596604, \"Shape_Area\": 0.000466175414294, \"zone\": \"West Brighton\", \"LocationID\": 245, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.09787969199995, 40.64035805499988], [-74.09767290799995, 40.63896067899989], [-74.09409312599998, 40.639266913999975], [-74.09242513499994, 40.63558181299992], [-74.08870582299987, 40.63684520599986], [-74.08373397299994, 40.63573714399991], [-74.09223971399994, 40.62574291599987], [-74.10350925299994, 40.61641607499991], [-74.1132624959999, 40.62305442499995], [-74.11614321199995, 40.62870226399988], [-74.11765722499997, 40.63806246599996], [-74.10974833299994, 40.63786583299993], [-74.09787969199995, 40.64035805499988]]]}}, {\"id\": \"245\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3119482485571439, \"OBJECTID\": 246, \"Shape_Leng\": 0.0694671937437, \"Shape_Area\": 0.000281298327076, \"zone\": \"West Chelsea/Hudson Yards\", \"LocationID\": 246, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00439976203513, 40.76267135909888], [-73.99547450199992, 40.759626856999894], [-74.00820401899988, 40.74214751799986], [-74.01213726936172, 40.74353192705926], [-74.0087278005888, 40.74485823290996], [-74.01183642872407, 40.74512596367872], [-74.00946389855962, 40.74575014365948], [-74.01150411235528, 40.74699080250515], [-74.0091373625992, 40.74735976280298], [-74.01133174441699, 40.748000575341926], [-74.0090584790135, 40.74778564533286], [-74.00894606502312, 40.74841189316866], [-74.01121562130406, 40.74869682117583], [-74.00842636447226, 40.75215785023921], [-74.0099905581974, 40.75281618714907], [-74.00997058797012, 40.75293356415314], [-74.00835352838844, 40.752350739342255], [-74.00480392560178, 40.75780984316466], [-74.00700358904015, 40.759231870341075], [-74.00390742032765, 40.75937569218354], [-74.00233528130057, 40.76154372700156], [-74.00439976203513, 40.76267135909888]]]}}, {\"id\": \"246\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6112880032867708, \"OBJECTID\": 247, \"Shape_Leng\": 0.0929678582271, \"Shape_Area\": 0.000205617157843, \"zone\": \"West Concourse\", \"LocationID\": 247, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91222180499983, 40.84235659099988], [-73.91986089399987, 40.83194722699992], [-73.92885642099992, 40.81487986599991], [-73.93027812699984, 40.813248024999965], [-73.93242632000944, 40.81411751897961], [-73.9330725108407, 40.82815615201811], [-73.92850073899989, 40.829352349999965], [-73.92399049499993, 40.83505229799987], [-73.91901668400004, 40.83838109299989], [-73.91618344399991, 40.84233489999991], [-73.91794965199988, 40.842229570999976], [-73.91729224999992, 40.84510993999991], [-73.91222180499983, 40.84235659099988]]]}}, {\"id\": \"247\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6047957371225577, \"OBJECTID\": 248, \"Shape_Leng\": 0.0569191167671, \"Shape_Area\": 0.000149592769799, \"zone\": \"West Farms/Bronx River\", \"LocationID\": 248, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86393748099981, 40.84004456599994], [-73.86271114399993, 40.83299150899997], [-73.88378414099992, 40.828274291999946], [-73.88005188499987, 40.83470205899989], [-73.87379590199993, 40.837445132999896], [-73.87277204699981, 40.83975128199995], [-73.86448011799995, 40.84108779699994], [-73.86393748099981, 40.84004456599994]]]}}, {\"id\": \"248\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.30907245039802766, \"OBJECTID\": 249, \"Shape_Leng\": 0.03638434365, \"Shape_Area\": 7.22155957729e-05, \"zone\": \"West Village\", \"LocationID\": 249, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00250642399995, 40.729016384999966], [-74.0070902079999, 40.72877225299993], [-74.00515808199998, 40.74085808099991], [-73.99683993899998, 40.73736088899995], [-74.00250642399995, 40.729016384999966]]]}}, {\"id\": \"249\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6452546296296297, \"OBJECTID\": 250, \"Shape_Leng\": 0.0796263465454, \"Shape_Area\": 0.000240975845956, \"zone\": \"Westchester Village/Unionport\", \"LocationID\": 250, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8455308949999, 40.83917330699989], [-73.83936452799996, 40.84087706699995], [-73.83959108476137, 40.834041688045254], [-73.84390624999997, 40.82840782999993], [-73.84289858800003, 40.82408305899986], [-73.85513639799981, 40.822436188999866], [-73.857391213, 40.83229737199986], [-73.86033845899995, 40.833253165999935], [-73.85168459699997, 40.8342010519999], [-73.85333524999986, 40.84247015499987], [-73.85052278099987, 40.84278948899985], [-73.8455308949999, 40.83917330699989]]]}}, {\"id\": \"250\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.4626865671641791, \"OBJECTID\": 251, \"Shape_Leng\": 0.1377111611, \"Shape_Area\": 0.000625754983157, \"zone\": \"Westerleigh\", \"LocationID\": 251, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.13107460299996, 40.631147728999956], [-74.1313098529999, 40.62632834299998], [-74.12125677899994, 40.62782274999989], [-74.10552598799994, 40.618592329999906], [-74.0977696879999, 40.61062359299988], [-74.1156683379999, 40.611175384999896], [-74.13210459699991, 40.608152923999924], [-74.14471100599997, 40.60803429699994], [-74.14954343599996, 40.60962717299988], [-74.14696864199988, 40.610151355999896], [-74.14603562099998, 40.61701277699996], [-74.13982184799993, 40.61629982499988], [-74.13826358199992, 40.624243769999936], [-74.13581008299997, 40.62506433999991], [-74.13806393799992, 40.627568470999904], [-74.13107460299996, 40.631147728999956]]]}}, {\"id\": \"251\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5337837837837838, \"OBJECTID\": 252, \"Shape_Leng\": 0.158004373564, \"Shape_Area\": 0.00102526406057, \"zone\": \"Whitestone\", \"LocationID\": 252, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82049919995306, 40.80101146781907], [-73.81369963725258, 40.79707941566509], [-73.81225365567188, 40.79812694680175], [-73.79640953426332, 40.795728824199614], [-73.79442844782817, 40.794806810498535], [-73.79375126484103, 40.78906193333896], [-73.79058298999988, 40.78819290999989], [-73.79582463099997, 40.7861305879999], [-73.80242791599977, 40.786068273999945], [-73.8037902289999, 40.77561011199992], [-73.81485850699985, 40.776485922999875], [-73.81452997799975, 40.77888817499992], [-73.82331286899986, 40.77957865999986], [-73.82224127699999, 40.781693878999924], [-73.83869921299997, 40.781676956999874], [-73.83824429099994, 40.78778591599995], [-73.84008157499997, 40.78785153099992], [-73.83895676099984, 40.79143843099996], [-73.84081532599994, 40.7915048849999], [-73.84067828300006, 40.79390789199993], [-73.83663288911686, 40.79251637576927], [-73.83728024446076, 40.78900952077265], [-73.83251063648018, 40.788624476626765], [-73.83166080927866, 40.79110259115967], [-73.8278487557192, 40.79308767321416], [-73.82945991534359, 40.79682336759143], [-73.82506496699443, 40.79737577656827], [-73.82049919995306, 40.80101146781907]]]}}, {\"id\": \"252\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5381062355658198, \"OBJECTID\": 253, \"Shape_Leng\": 0.0360514998192, \"Shape_Area\": 7.83395761191e-05, \"zone\": \"Willets Point\", \"LocationID\": 253, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83908354399988, 40.76525691299991], [-73.83679107299986, 40.76116667099992], [-73.83867321899987, 40.75701997899995], [-73.84324381399976, 40.75533680399995], [-73.84670933714987, 40.76134526931977], [-73.84272819396486, 40.76462854893166], [-73.83908354399988, 40.76525691299991]]]}}, {\"id\": \"253\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6320254506892895, \"OBJECTID\": 254, \"Shape_Leng\": 0.0858863754861, \"Shape_Area\": 0.000360040216032, \"zone\": \"Williamsbridge/Olinville\", \"LocationID\": 254, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85186563799999, 40.87905886499989], [-73.85091896099982, 40.877893440999856], [-73.85254854099988, 40.877091586999924], [-73.8506895569999, 40.87548667799993], [-73.85363638799987, 40.87330059899996], [-73.85651292399994, 40.874217792999936], [-73.85661382499993, 40.87125016699991], [-73.87056192499989, 40.87151077799995], [-73.87019944899986, 40.87631135499987], [-73.860710982, 40.89537054399996], [-73.84590568599998, 40.889741368999864], [-73.85186563799999, 40.87905886499989]]]}}, {\"id\": \"254\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.323157726432142, \"OBJECTID\": 255, \"Shape_Leng\": 0.0623841997664, \"Shape_Area\": 0.000172309184842, \"zone\": \"Williamsburg (North Side)\", \"LocationID\": 255, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96176070375392, 40.72522879205536], [-73.95477662900004, 40.72245950499994], [-73.94907816799986, 40.72254028699993], [-73.94841638599995, 40.71845553299995], [-73.94721341799986, 40.718560985999865], [-73.95547596099992, 40.7122654259999], [-73.96736391666703, 40.71648367952745], [-73.96176070375392, 40.72522879205536]]]}}, {\"id\": \"255\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.3502633655734395, \"OBJECTID\": 256, \"Shape_Leng\": 0.0679149669603, \"Shape_Area\": 0.000168611097013, \"zone\": \"Williamsburg (South Side)\", \"LocationID\": 256, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95834207500002, 40.71330630099992], [-73.95547596099992, 40.7122654259999], [-73.95433904200002, 40.714071701999906], [-73.94937231899985, 40.71406687799986], [-73.94865533499987, 40.70972498299988], [-73.95087492699996, 40.70951311499991], [-73.95023693799996, 40.70547324699994], [-73.9579043079999, 40.70842717299994], [-73.958455768, 40.707251846999895], [-73.96838933699995, 40.70682918699989], [-73.96984864823531, 40.708003041687675], [-73.96736391666705, 40.71648367902752], [-73.95834207500002, 40.71330630099992]]]}}, {\"id\": \"256\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.39972197649437635, \"OBJECTID\": 257, \"Shape_Leng\": 0.0586690259793, \"Shape_Area\": 0.00013890947321, \"zone\": \"Windsor Terrace\", \"LocationID\": 257, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97984261899994, 40.660727440999956], [-73.97436701299992, 40.65818790399992], [-73.97084113699994, 40.64637857099993], [-73.97777380600002, 40.645341876999964], [-73.97819074399992, 40.64752599199992], [-73.98029115499993, 40.647297160999855], [-73.98179568699996, 40.65524609699991], [-73.98843135299977, 40.65925121199996], [-73.9868846449999, 40.6603663889999], [-73.985086794, 40.65901328799989], [-73.982230342, 40.66214149199994], [-73.97984261899994, 40.660727440999956]]]}}, {\"id\": \"257\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.5877040639110802, \"OBJECTID\": 258, \"Shape_Leng\": 0.0890133787693, \"Shape_Area\": 0.000366209617143, \"zone\": \"Woodhaven\", \"LocationID\": 258, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8450419489999, 40.68931894699996], [-73.84119605699982, 40.68335136599989], [-73.84786817599985, 40.681486831999905], [-73.85197864599989, 40.682564210999864], [-73.85099089299987, 40.68056836699987], [-73.85392389199987, 40.679686155999875], [-73.85819836099995, 40.67943084899987], [-73.85878576899987, 40.68063046299989], [-73.86334943599995, 40.67935163999989], [-73.86410096700003, 40.68237284999988], [-73.86602666899996, 40.681918051999915], [-73.86868454399996, 40.694034691999924], [-73.85277238699999, 40.69755711199987], [-73.85443455699989, 40.70060295199994], [-73.85070842300001, 40.70020312499988], [-73.8450419489999, 40.68931894699996]]]}}, {\"id\": \"258\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6758304696449027, \"OBJECTID\": 259, \"Shape_Leng\": 0.126750305191, \"Shape_Area\": 0.000394552487366, \"zone\": \"Woodlawn/Wakefield\", \"LocationID\": 259, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.851071161919, 40.91037152011096], [-73.85347522366716, 40.90753029351422], [-73.84138689269206, 40.90417274176649], [-73.83554853999995, 40.88913350399993], [-73.84704957999995, 40.8876718159999], [-73.84590568599998, 40.889741368999864], [-73.860710982, 40.89537054399996], [-73.87048672599987, 40.896640210999905], [-73.8771197409999, 40.895468663999964], [-73.86789043706806, 40.90298695407611], [-73.85946778750369, 40.90051720913338], [-73.85957882974115, 40.9024408428857], [-73.85601023344621, 40.9053059838135], [-73.85681013619275, 40.90615582816297], [-73.85374103615364, 40.90778791636777], [-73.85458940544117, 40.9089394991491], [-73.851071161919, 40.91037152011096]]]}}, {\"id\": \"259\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.6612424781761166, \"OBJECTID\": 260, \"Shape_Leng\": 0.133514154636, \"Shape_Area\": 0.000422345326907, \"zone\": \"Woodside\", \"LocationID\": 260, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9017537339999, 40.760775474999946], [-73.90147053099994, 40.759633049999884], [-73.89866137799999, 40.759921016999904], [-73.89976862099998, 40.75666423699991], [-73.8949120899999, 40.74428123099996], [-73.90057825399985, 40.73949289599988], [-73.90926002999988, 40.736403179999854], [-73.91068876399997, 40.73347108099995], [-73.90628903499987, 40.732400751999876], [-73.90516948899983, 40.73585964299986], [-73.90107970900003, 40.73391463399987], [-73.90712253099991, 40.727424727999924], [-73.91174734199988, 40.72991203299991], [-73.91205021299994, 40.72850636299993], [-73.91796696699996, 40.7285007709999], [-73.917342506, 40.731382825999916], [-73.91903825199986, 40.73200596599987], [-73.91651190599994, 40.738944757999874], [-73.9137938869998, 40.742514491999934], [-73.9119412979999, 40.742382948999925], [-73.91190979099993, 40.748105394999875], [-73.90945182799983, 40.75065550699994], [-73.91067474899997, 40.752995076999895], [-73.90631505099988, 40.75984387599992], [-73.90779405499987, 40.760517799999896], [-73.90356915599997, 40.765781788999924], [-73.9017537339999, 40.760775474999946]]]}}, {\"id\": \"260\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.40379712206598456, \"OBJECTID\": 261, \"Shape_Leng\": 0.0271204563616, \"Shape_Area\": 3.43423231652e-05, \"zone\": \"World Trade Center\", \"LocationID\": 261, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01332610899989, 40.7050307879999], [-74.01668613099999, 40.70479933799992], [-74.01375388499997, 40.71369055499987], [-74.00860085399992, 40.711392239999945], [-74.01332610899989, 40.7050307879999]]]}}, {\"id\": \"261\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.37741681139529365, \"OBJECTID\": 262, \"Shape_Leng\": 0.0490636231541, \"Shape_Area\": 0.000122330270966, \"zone\": \"Yorkville East\", \"LocationID\": 262, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.94383256699986, 40.78285908899991], [-73.94207418803879, 40.77691784697707], [-73.94293043781381, 40.774676268035776], [-73.9474897547817, 40.77011515400953], [-73.95268514999995, 40.772339404999926], [-73.94383256699986, 40.78285908899991]]], [[[-73.93804640603422, 40.78082954427547], [-73.93759894622617, 40.78046784086141], [-73.93958378972465, 40.779576474007136], [-73.9387442308927, 40.78104387604228], [-73.93804640603422, 40.78082954427547]]]]}}, {\"id\": \"262\", \"type\": \"Feature\", \"properties\": {\"payment_type\": 0.36019673393354124, \"OBJECTID\": 263, \"Shape_Leng\": 0.0370166252994, \"Shape_Area\": 6.57697664169e-05, \"zone\": \"Yorkville West\", \"LocationID\": 263, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95218621999996, 40.7730198449999], [-73.95729409999986, 40.7742835549999], [-73.94933170599992, 40.78519312699994], [-73.94472264499986, 40.78324757199987], [-73.95218621999996, 40.7730198449999]]]}}]}"},"id":"10ea5377-ee43-4b43-9633-3c452e6ae727","type":"GeoJSONDataSource"},{"attributes":{},"id":"eee50ee7-f6fd-4709-b706-20e6342ba4eb","type":"PanTool"},{"attributes":{"dimension":1,"plot":{"id":"ae53cc6f-7f97-4812-a208-ab913d90eb10","subtype":"Figure","type":"Plot"},"ticker":{"id":"26eacfd5-0f25-4429-acdb-7da3f6282337","type":"BasicTicker"},"visible":false},"id":"36cfaa0d-2256-478f-9e32-09a8af5f4faf","type":"Grid"},{"attributes":{"color_mapper":{"id":"fe7eac4c-a6ac-449b-a2da-89f027e5010b","type":"LogColorMapper"},"formatter":{"id":"7b7b968c-bf9d-4fda-ad38-7e3a534a2390","type":"BasicTickFormatter"},"label_standoff":12,"location":[0,0],"plot":{"id":"ae53cc6f-7f97-4812-a208-ab913d90eb10","subtype":"Figure","type":"Plot"},"ticker":{"id":"cdd73a27-a419-4f2d-be6c-98f427087f04","type":"BasicTicker"}},"id":"01599911-7bd9-4ceb-93b4-1d76c6b84013","type":"ColorBar"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"6351fc86-4cb0-48f1-906d-40c4646bf983","type":"PanTool"},{"id":"f6f9dda6-1d51-4e39-80d2-22bb8a3e4a7d","type":"WheelZoomTool"},{"id":"4c82bbf0-af1f-4b96-b5c7-19931b678383","type":"BoxZoomTool"},{"id":"4a794f89-97c4-4344-b538-0ef0498b3219","type":"SaveTool"},{"id":"7f3fe531-7586-4d07-b55d-917e998d17dc","type":"ResetTool"},{"id":"2693a195-e9cc-48e0-b8c7-94ab6f8d0cf7","type":"HelpTool"},{"id":"24f7aaa5-0834-45f8-a0a9-6bf207fb7a90","type":"HoverTool"}]},"id":"c72f958d-87e1-4f05-9a9b-e2316d42004b","type":"Toolbar"},{"attributes":{"below":[{"id":"981a4af8-3ccf-4507-8498-13e2a0e4095e","type":"LinearAxis"}],"left":[{"id":"fd734f94-2867-4ad8-957e-b2b46fc077fc","type":"LinearAxis"}],"renderers":[{"id":"981a4af8-3ccf-4507-8498-13e2a0e4095e","type":"LinearAxis"},{"id":"fe6ec5d9-95fe-4fc8-b82e-9574f7c0d667","type":"Grid"},{"id":"fd734f94-2867-4ad8-957e-b2b46fc077fc","type":"LinearAxis"},{"id":"e47b8c6a-ee73-4698-8e94-0349d21c6c88","type":"Grid"},{"id":"b63fb838-d1f0-4417-b06a-2253fff02424","type":"BoxAnnotation"},{"id":"698fbc56-a4d3-478b-bd22-f23784c8fbf5","type":"GlyphRenderer"}],"title":{"id":"2860cfea-9a49-4816-b1ef-156fc5fef34d","type":"Title"},"toolbar":{"id":"e103e054-aec9-4b1c-a5c7-a9229ea8251e","type":"Toolbar"},"x_range":{"id":"488c23b0-642f-43c5-a709-d530ececa55a","type":"DataRange1d"},"x_scale":{"id":"2e37b525-1a12-4a01-9264-a099c6ca242b","type":"LinearScale"},"y_range":{"id":"be2f1631-6335-467f-9d36-88a1e10dee8c","type":"DataRange1d"},"y_scale":{"id":"26612bd9-9ba6-4d61-95ab-5284b5499ee3","type":"LinearScale"}},"id":"62ba6d47-3ae9-4aa5-be3a-40808ec4ef1a","subtype":"Figure","type":"Plot"},{"attributes":{"formatter":{"id":"f3b86a6f-66d5-48ed-adbf-ebdc3e9fd079","type":"BasicTickFormatter"},"plot":{"id":"62ba6d47-3ae9-4aa5-be3a-40808ec4ef1a","subtype":"Figure","type":"Plot"},"ticker":{"id":"7d621023-4d97-44a1-8ccf-0992cf3ef267","type":"BasicTicker"},"visible":false},"id":"fd734f94-2867-4ad8-957e-b2b46fc077fc","type":"LinearAxis"},{"attributes":{"below":[{"id":"1b83b15e-0089-4812-ad82-87c5a6ed9480","type":"LinearAxis"}],"left":[{"id":"eee6a093-e6be-4073-88b7-824d43352607","type":"LinearAxis"}],"renderers":[{"id":"1b83b15e-0089-4812-ad82-87c5a6ed9480","type":"LinearAxis"},{"id":"5ab58e62-0c50-4e83-a7f6-40cf7b7ce86c","type":"Grid"},{"id":"eee6a093-e6be-4073-88b7-824d43352607","type":"LinearAxis"},{"id":"f396af5d-43b1-43d6-ad17-82a9e1272208","type":"Grid"},{"id":"be3cc134-22fc-4afb-bd0a-a527d820071d","type":"BoxAnnotation"},{"id":"3b1fa4e4-5bf6-4bf4-85d2-091d7ff789d5","type":"GlyphRenderer"},{"id":"3f99e069-bc47-41e6-9ed0-a9e0856d105c","type":"ColorBar"}],"right":[{"id":"3f99e069-bc47-41e6-9ed0-a9e0856d105c","type":"ColorBar"}],"title":{"id":"27996c4c-da44-4dda-b78a-1aeaac693387","type":"Title"},"toolbar":{"id":"c72f958d-87e1-4f05-9a9b-e2316d42004b","type":"Toolbar"},"x_range":{"id":"5c81d120-72b6-43b4-824c-758a8d5e8726","type":"DataRange1d"},"x_scale":{"id":"c936782d-eea5-4fb4-bd79-1d11686a2dfa","type":"LinearScale"},"y_range":{"id":"f0c21fdb-3905-4866-bbf5-d191494f134c","type":"DataRange1d"},"y_scale":{"id":"9d08d013-0e4e-429d-a37a-b7275a5c84b8","type":"LinearScale"}},"id":"3058396c-651f-4f74-b4de-a7e737c8665a","subtype":"Figure","type":"Plot"},{"attributes":{"callback":null},"id":"24209739-d5c4-4529-abcf-6ff415a2dfd5","type":"DataRange1d"},{"attributes":{"palette":["#053061","#2166ac","#4393c3","#92c5de","#d1e5f0","#f7f7f7","#fddbc7","#f4a582","#d6604d","#b2182b","#67001f"]},"id":"fd673fd5-348f-4441-b09b-846332ca8ea1","type":"LinearColorMapper"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"6960645d-737b-442b-a2f0-cdb7e93d4e91","type":"PanTool"},{"id":"5150062a-68b9-4228-a3ca-5a2795f8ae24","type":"WheelZoomTool"},{"id":"ee3c44cb-3c2f-4de1-82d6-8a8ccf913021","type":"BoxZoomTool"},{"id":"a0586146-b37e-490f-a532-dc621364d57f","type":"SaveTool"},{"id":"f7b30ba4-fd79-4d13-a3c7-49803d251345","type":"ResetTool"},{"id":"d9c616aa-dc6d-4459-af43-73444362b3b2","type":"HelpTool"},{"id":"a84bfab3-9687-4112-8322-4e604f5954f2","type":"HoverTool"}]},"id":"e103e054-aec9-4b1c-a5c7-a9229ea8251e","type":"Toolbar"},{"attributes":{},"id":"7b7b968c-bf9d-4fda-ad38-7e3a534a2390","type":"BasicTickFormatter"},{"attributes":{"formatter":{"id":"d41fbb68-5ad0-439f-87ea-f1e11384576c","type":"BasicTickFormatter"},"plot":{"id":"ae53cc6f-7f97-4812-a208-ab913d90eb10","subtype":"Figure","type":"Plot"},"ticker":{"id":"26eacfd5-0f25-4429-acdb-7da3f6282337","type":"BasicTicker"},"visible":false},"id":"e0e3f2e8-d02e-461e-ad09-3f799e56ae49","type":"LinearAxis"},{"attributes":{},"id":"b868a9e2-54ac-4484-a27c-2920d66d903b","type":"LinearScale"},{"attributes":{},"id":"7d621023-4d97-44a1-8ccf-0992cf3ef267","type":"BasicTicker"},{"attributes":{"plot":null,"text":"Fraction of rides that tip in cash"},"id":"94dcf2d1-8432-4868-9512-d05dfca60e65","type":"Title"},{"attributes":{"callback":null},"id":"488c23b0-642f-43c5-a709-d530ececa55a","type":"DataRange1d"},{"attributes":{"formatter":{"id":"502eda31-9e62-48de-b5af-2bd75bfc8082","type":"BasicTickFormatter"},"plot":{"id":"ae53cc6f-7f97-4812-a208-ab913d90eb10","subtype":"Figure","type":"Plot"},"ticker":{"id":"72328d45-1f26-4e3d-9388-ba39a0b07bbd","type":"BasicTicker"},"visible":false},"id":"2a692866-0932-40ef-ab57-5680f5882b98","type":"LinearAxis"},{"attributes":{},"id":"72328d45-1f26-4e3d-9388-ba39a0b07bbd","type":"BasicTicker"},{"attributes":{},"id":"7077852c-6855-4480-8785-509b9fbeb00e","type":"LinearScale"},{"attributes":{"source":{"id":"18193dc2-af18-430d-8541-849ff6cf74a5","type":"GeoJSONDataSource"}},"id":"4bcfec7d-c05c-448a-94b4-1d2866409a23","type":"CDSView"},{"attributes":{"factors":["EWR","Queens","Bronx","Manhattan","Staten Island","Brooklyn"],"palette":["#1f77b4","#ff7f0e","#2ca02c","#d62728","#9467bd","#8c564b","#e377c2","#7f7f7f","#bcbd22","#17becf"]},"id":"6f4f819e-7190-454c-a7a3-ba5d768d5c31","type":"CategoricalColorMapper"},{"attributes":{"callback":null},"id":"d92819cb-157e-408e-979a-4aab8e0d4596","type":"DataRange1d"},{"attributes":{},"id":"62e9be12-1576-4008-82b6-0a61f5cf77ea","type":"LinearScale"},{"attributes":{},"id":"2a08cff9-d6de-4308-8dfb-94ef1c977983","type":"BasicTicker"},{"attributes":{},"id":"7eade2b2-57b6-47a4-9a00-ec5b4a408ce6","type":"BasicTickFormatter"},{"attributes":{"callback":null},"id":"be2f1631-6335-467f-9d36-88a1e10dee8c","type":"DataRange1d"},{"attributes":{"callback":null},"id":"5c81d120-72b6-43b4-824c-758a8d5e8726","type":"DataRange1d"},{"attributes":{"plot":null,"text":"Number of rides"},"id":"a4bd85d5-54d7-4339-95e8-726aa5564454","type":"Title"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"ed93bc07-ab38-4459-adf7-0bce0b3fe3b5","type":"PanTool"},{"id":"022fd809-ec00-4851-b661-c49991bfae3a","type":"WheelZoomTool"},{"id":"9bdf7e1b-f00b-4377-87b0-82d52037d7df","type":"BoxZoomTool"},{"id":"d301bef6-61e1-4d2e-85d6-2a8e369373b7","type":"SaveTool"},{"id":"0d27ecfc-706b-4857-9f4b-080d014b15e4","type":"ResetTool"},{"id":"085f3735-0ce5-463d-99a2-c12f409f4b0e","type":"HelpTool"},{"id":"05d6bea6-6455-4d7f-a047-890c32987b9a","type":"HoverTool"}]},"id":"61f82afe-2e85-4d6b-8872-1435a20bc13a","type":"Toolbar"},{"attributes":{"callback":null,"point_policy":"follow_mouse","tooltips":"&lt;div&gt;&lt;b&gt;Borough&lt;/b&gt;: @borough&lt;/div&gt;&lt;div&gt;&lt;b&gt;Zone&lt;/b&gt;: @zone&lt;/div&gt;&lt;div&gt;&lt;b&gt;Tip Fraction&lt;/b&gt;: @tip_fraction&lt;/div&gt;"},"id":"24f7aaa5-0834-45f8-a0a9-6bf207fb7a90","type":"HoverTool"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"eee50ee7-f6fd-4709-b706-20e6342ba4eb","type":"PanTool"},{"id":"6a2d253b-ce19-4ab5-8e57-e3afe7f8a209","type":"WheelZoomTool"},{"id":"5e99a9d3-a981-45fc-8281-f0e7c0c6021c","type":"BoxZoomTool"},{"id":"8b6fba3c-2dc4-43a3-9e6b-3b0aa55f67a7","type":"SaveTool"},{"id":"3b6a7eac-00a3-410a-8bc1-41a46ac7b38a","type":"ResetTool"},{"id":"aa4113e2-b3e7-4170-a486-16fa7c11995b","type":"HelpTool"},{"id":"bf29975f-fcdb-444f-ad6b-270f10ab2b06","type":"HoverTool"}]},"id":"f408f7d2-a34f-43a8-8561-003142236b1c","type":"Toolbar"},{"attributes":{"plot":{"id":"62ba6d47-3ae9-4aa5-be3a-40808ec4ef1a","subtype":"Figure","type":"Plot"},"ticker":{"id":"2a08cff9-d6de-4308-8dfb-94ef1c977983","type":"BasicTicker"},"visible":false},"id":"fe6ec5d9-95fe-4fc8-b82e-9574f7c0d667","type":"Grid"},{"attributes":{"plot":null,"text":""},"id":"2860cfea-9a49-4816-b1ef-156fc5fef34d","type":"Title"},{"attributes":{"data_source":{"id":"76c418d0-c7d9-48f9-8111-ae726987d2b2","type":"GeoJSONDataSource"},"glyph":{"id":"f1f65e0f-b376-4763-ba65-310d09a85747","type":"Patches"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"027cf8e6-f083-4db2-bf4b-6e525b959e2d","type":"Patches"},"selection_glyph":null,"view":{"id":"a00c5370-a7c6-4a7c-8839-0d6510dfa166","type":"CDSView"}},"id":"45c59d93-f760-4cc4-a732-46d46cef1f47","type":"GlyphRenderer"},{"attributes":{"callback":null,"geojson":"{\"type\": \"FeatureCollection\", \"features\": [{\"id\": \"2\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2754100159069716, \"OBJECTID\": 3, \"Shape_Leng\": 0.0843411059012, \"Shape_Area\": 0.000314414156821, \"zone\": \"Allerton/Pelham Gardens\", \"LocationID\": 3, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84792614099985, 40.87134223399991], [-73.82950892699994, 40.86184121299988], [-73.82834347399992, 40.86089202599988], [-73.8368191849999, 40.85727680899989], [-73.8566377469999, 40.8581191719999], [-73.85780378799993, 40.86303621699989], [-73.85961281199982, 40.865512717999856], [-73.86156274099987, 40.86554946399989], [-73.86137924099984, 40.8713365119999], [-73.85661382499993, 40.87125016699991], [-73.85651292399994, 40.874217792999936], [-73.84792614099985, 40.87134223399991]]]}}, {\"id\": \"3\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20128964432728239, \"OBJECTID\": 4, \"Shape_Leng\": 0.0435665270921, \"Shape_Area\": 0.000111871946192, \"zone\": \"Alphabet City\", \"LocationID\": 4, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97177410965318, 40.72582128133705], [-73.97347975574512, 40.71886142911285], [-73.98382387299982, 40.72147287199987], [-73.97802697999985, 40.729433059999984], [-73.97177410965318, 40.72582128133705]]]}}, {\"id\": \"4\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2386701533140724, \"OBJECTID\": 5, \"Shape_Leng\": 0.0921464898574, \"Shape_Area\": 0.000497957489363, \"zone\": \"Arden Heights\", \"LocationID\": 5, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.17421738099989, 40.56256808599987], [-74.17185166499996, 40.56152092299991], [-74.18018524699995, 40.54837960899992], [-74.18903145199995, 40.53815934199989], [-74.19377169299999, 40.53583384699993], [-74.20314825999996, 40.55606178499986], [-74.20046730799992, 40.55623049099987], [-74.19317388399995, 40.56260434399992], [-74.18340311699991, 40.56540496199994], [-74.17421738099989, 40.56256808599987]]]}}, {\"id\": \"5\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.35032516419868276, \"OBJECTID\": 6, \"Shape_Leng\": 0.150490542523, \"Shape_Area\": 0.000606460984581, \"zone\": \"Arrochar/Fort Wadsworth\", \"LocationID\": 6, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.06367318899999, 40.60219816599994], [-74.0651709129999, 40.604213507999894], [-74.05685812390232, 40.608056357772554], [-74.05381069805436, 40.605912718852984], [-74.05222826954189, 40.599776102303636], [-74.06503332378131, 40.58899099843518], [-74.07043976099992, 40.59673474699988], [-74.08173342999991, 40.59215628599987], [-74.08376862299998, 40.595520956999934], [-74.08587511899998, 40.59589012099985], [-74.0814942209999, 40.599081990999935], [-74.08184498199992, 40.601535196999926], [-74.08947630599992, 40.60090432199991], [-74.09172051499995, 40.59922085699985], [-74.09271908099993, 40.60636222199992], [-74.09045495899996, 40.60909452599991], [-74.06367318899999, 40.60219816599994]]]}}, {\"id\": \"6\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2124765567833194, \"OBJECTID\": 7, \"Shape_Leng\": 0.107417171123, \"Shape_Area\": 0.000389787989274, \"zone\": \"Astoria\", \"LocationID\": 7, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90413637799996, 40.76752031699986], [-73.90202516599992, 40.767416783999884], [-73.90779405499987, 40.760517799999896], [-73.90631505099988, 40.75984387599992], [-73.91023124699991, 40.75297652299985], [-73.91763192900005, 40.75385784899993], [-73.92562743799996, 40.75213724499991], [-73.92415699999985, 40.753961141999845], [-73.93679091699993, 40.75988288099991], [-73.93398092100001, 40.76282598199987], [-73.93858976899998, 40.766817358999916], [-73.93493511099992, 40.76701302399987], [-73.93344468499993, 40.768817789999915], [-73.92425288799978, 40.76472456399995], [-73.92218946199995, 40.767267356999895], [-73.91977764799998, 40.7661521839999], [-73.91849336999996, 40.76773465699995], [-73.91995596899991, 40.768440064999865], [-73.91811686499992, 40.77002848599989], [-73.90413637799996, 40.76752031699986]]]}}, {\"id\": \"7\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21507594133064187, \"OBJECTID\": 8, \"Shape_Leng\": 0.0275906911574, \"Shape_Area\": 2.6587716279e-05, \"zone\": \"Astoria Park\", \"LocationID\": 8, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92334041500001, 40.77512891199993], [-73.92827730816353, 40.77689897516217], [-73.91996458796496, 40.78263506876824], [-73.91796581399998, 40.78125629099988], [-73.92375761299998, 40.77654701199991], [-73.92334041500001, 40.77512891199993]]]}}, {\"id\": \"8\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.25747118529129204, \"OBJECTID\": 9, \"Shape_Leng\": 0.0997840924705, \"Shape_Area\": 0.000338443803197, \"zone\": \"Auburndale\", \"LocationID\": 9, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.78502434699996, 40.761036515999855], [-73.77573836999991, 40.74332564699993], [-73.79672497900003, 40.738416642999894], [-73.79723084999992, 40.74169550699993], [-73.7944636839999, 40.74497002199991], [-73.79472264899997, 40.74759932999989], [-73.79729524699995, 40.74943505299987], [-73.79471901099991, 40.75064761699991], [-73.79493246199993, 40.75779802999994], [-73.79031517, 40.75782345099988], [-73.79076004599987, 40.76098576899991], [-73.79343991199988, 40.760764127999934], [-73.79317944599984, 40.77040317799988], [-73.78502434699996, 40.761036515999855]]]}}, {\"id\": \"9\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20393170306830094, \"OBJECTID\": 10, \"Shape_Leng\": 0.0998394794152, \"Shape_Area\": 0.000435823818081, \"zone\": \"Baisley Park\", \"LocationID\": 10, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7832662499999, 40.68999429299992], [-73.77966850599998, 40.685377122999945], [-73.77699767599997, 40.68628268899992], [-73.77220275299989, 40.681610338999896], [-73.78025859499995, 40.6778409239999], [-73.78573730899993, 40.67238503499988], [-73.78808647399985, 40.6730096269999], [-73.78940999399993, 40.66684115799993], [-73.80155565300004, 40.6667533639999], [-73.80142836799988, 40.673666833999874], [-73.80576840199991, 40.682932390999916], [-73.79898209099989, 40.68485873699989], [-73.79995042799996, 40.68682771899991], [-73.79561486799987, 40.688656890999916], [-73.793908417, 40.68630469099985], [-73.7832662499999, 40.68999429299992]]]}}, {\"id\": \"10\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2038587415178357, \"OBJECTID\": 11, \"Shape_Leng\": 0.0792110389596, \"Shape_Area\": 0.00026452053504, \"zone\": \"Bath Beach\", \"LocationID\": 11, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00109809499993, 40.60303462599992], [-73.99559320799995, 40.600232221999875], [-74.00200394754667, 40.594132371137796], [-74.01041238886523, 40.60047518174621], [-74.01942760861903, 40.602871578425265], [-74.01549854999998, 40.606841523999925], [-74.01701910199995, 40.60765441699994], [-74.01050403599997, 40.613896610999866], [-74.00094408399997, 40.60863970999988], [-74.00493448899994, 40.60507440599987], [-74.00109809499993, 40.60303462599992]]]}}, {\"id\": \"11\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20504672364754467, \"OBJECTID\": 12, \"Shape_Leng\": 0.0366613013579, \"Shape_Area\": 4.15116236727e-05, \"zone\": \"Battery Park\", \"LocationID\": 12, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01565756599994, 40.70483308799993], [-74.01425861399996, 40.70448944299991], [-74.01376867799992, 40.7011432629999], [-74.014220631955, 40.70011149186829], [-74.0176479750685, 40.70349459987955], [-74.01565756599994, 40.70483308799993]]]}}, {\"id\": \"12\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.1986931306821436, \"OBJECTID\": 13, \"Shape_Leng\": 0.0502813228631, \"Shape_Area\": 0.000149358592917, \"zone\": \"Battery Park City\", \"LocationID\": 13, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01244109299991, 40.7190576729999], [-74.01668714699993, 40.704743941999865], [-74.01844770243774, 40.70416216823156], [-74.0174808615258, 40.704667487696156], [-74.01934254624483, 40.706093673029706], [-74.01777014254499, 40.71283457478916], [-74.01662424425285, 40.71215731899529], [-74.01632006627581, 40.71340798247826], [-74.01772496219284, 40.71307018162274], [-74.01671018605829, 40.718624176057965], [-74.01244109299991, 40.7190576729999]]]}}, {\"id\": \"13\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.26867475824303777, \"OBJECTID\": 14, \"Shape_Leng\": 0.175213698053, \"Shape_Area\": 0.00138177826442, \"zone\": \"Bay Ridge\", \"LocationID\": 14, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.03407329297129, 40.64431393298185], [-74.02475437599998, 40.6381214179999], [-74.02108051399998, 40.64149093699994], [-74.02141038399992, 40.638584902999845], [-74.01514630699995, 40.63322193799993], [-74.01970491599991, 40.6221849649999], [-74.02754297299998, 40.61293718299988], [-74.02637609099996, 40.61245060299993], [-74.02710660599995, 40.61024395599991], [-74.02150602899994, 40.60782209899992], [-74.02247769499994, 40.60667421599987], [-74.02078541, 40.605673685999875], [-74.02139467299999, 40.60449399799991], [-74.01869135799996, 40.60360792399991], [-74.01942760861903, 40.602871578425265], [-74.0321315955131, 40.605901836264245], [-74.04038132338673, 40.61534178232843], [-74.04189123741789, 40.62406041492981], [-74.04108656611493, 40.63009852389527], [-74.03680662048858, 40.63898422154557], [-74.03876297920752, 40.639588258980524], [-74.03672484374877, 40.63914119020668], [-74.03571276150724, 40.64064015404647], [-74.03675840317513, 40.641611762713936], [-74.03407329297129, 40.64431393298185]]]}}, {\"id\": \"14\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.254714931883147, \"OBJECTID\": 15, \"Shape_Leng\": 0.14433622262, \"Shape_Area\": 0.000925219395547, \"zone\": \"Bay Terrace/Fort Totten\", \"LocationID\": 15, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7774039129087, 40.79659824126783], [-73.77063124537418, 40.78846290603716], [-73.77426836058959, 40.78675665865473], [-73.77633728818715, 40.78848722575376], [-73.76820055460033, 40.77944417919097], [-73.76702392346498, 40.78008623415827], [-73.76677570373936, 40.77981245869834], [-73.76782207615058, 40.77881037615661], [-73.77553732199992, 40.777025597999916], [-73.77583958299994, 40.778612371999884], [-73.79565702099983, 40.773036085999905], [-73.79853009399987, 40.775190728999924], [-73.8037902289999, 40.77561011199992], [-73.80242791699989, 40.78606827399991], [-73.79491977600001, 40.78633880799987], [-73.79058298999988, 40.78819290999989], [-73.79179280401884, 40.789351616549254], [-73.79028202353197, 40.79031696339975], [-73.78174708763098, 40.79113340664446], [-73.78079182072646, 40.79403591986439], [-73.78325271025878, 40.79492032976596], [-73.7774039129087, 40.79659824126783]]]}}, {\"id\": \"15\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.25765685399940524, \"OBJECTID\": 16, \"Shape_Leng\": 0.141291873771, \"Shape_Area\": 0.000871889446182, \"zone\": \"Bayside\", \"LocationID\": 16, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7685730499999, 40.77910542899991], [-73.75864264358073, 40.767357170035446], [-73.75377036482266, 40.76530463124142], [-73.75921689799992, 40.76432485599988], [-73.75800131799996, 40.76037986699988], [-73.76239510499991, 40.75949699799992], [-73.76184343999986, 40.75553726899989], [-73.75684127399991, 40.74932207699987], [-73.77573836999991, 40.74332564699993], [-73.78546383099984, 40.76259192499988], [-73.79565702099983, 40.773036085999905], [-73.77678363299991, 40.77840958299987], [-73.77553732199992, 40.777025597999916], [-73.7685730499999, 40.77910542899991]]]}}, {\"id\": \"16\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22623919619656271, \"OBJECTID\": 17, \"Shape_Leng\": 0.093522632948, \"Shape_Area\": 0.000322957654799, \"zone\": \"Bedford\", \"LocationID\": 17, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94306406899986, 40.701424434999886], [-73.94193078899983, 40.70072523399989], [-73.94438788599984, 40.70042452299993], [-73.94032794, 40.679889974999874], [-73.9533701749999, 40.680640507999904], [-73.95468418899998, 40.68724485399985], [-73.95956770199984, 40.686682554999955], [-73.96029281699998, 40.69034624999995], [-73.95541057999996, 40.69090829199994], [-73.95614239299985, 40.694579018999896], [-73.96015854699988, 40.69411730899989], [-73.96105100699992, 40.69832607899989], [-73.95701993199991, 40.69897391399995], [-73.95745736399992, 40.70082260299993], [-73.95381196900003, 40.703180979999914], [-73.95128819399989, 40.700922364999855], [-73.94705205299991, 40.70366394899985], [-73.94306406899986, 40.701424434999886]]]}}, {\"id\": \"17\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2512246624287454, \"OBJECTID\": 18, \"Shape_Leng\": 0.0697995498569, \"Shape_Area\": 0.000148850163948, \"zone\": \"Bedford Park\", \"LocationID\": 18, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88513907699999, 40.86638287399992], [-73.88889566099989, 40.86475415299988], [-73.89573924699978, 40.85813820699986], [-73.89883814599983, 40.859307307999906], [-73.897143798, 40.86244506499991], [-73.90107966999994, 40.86275638299992], [-73.8974032329998, 40.86747418999989], [-73.89438130299995, 40.866068850999966], [-73.88785196299986, 40.87211417599992], [-73.88937163799984, 40.87338229799993], [-73.88527683199987, 40.87921766899991], [-73.88146889699985, 40.868573645999916], [-73.88513907699999, 40.86638287399992]]]}}, {\"id\": \"18\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2448542868183707, \"OBJECTID\": 19, \"Shape_Leng\": 0.101824875452, \"Shape_Area\": 0.000546661094782, \"zone\": \"Bellerose\", \"LocationID\": 19, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.72339596299987, 40.750389075999855], [-73.72258571899978, 40.74885045099991], [-73.72010148299994, 40.74977568599991], [-73.71147804899985, 40.73250140099988], [-73.71049995381028, 40.72722708720663], [-73.72990512499977, 40.723272494999954], [-73.7314721509999, 40.727411794999924], [-73.74640108899996, 40.7312928389999], [-73.74672140400006, 40.73247715099991], [-73.73933151399987, 40.739474474999916], [-73.72829094199989, 40.74745895699992], [-73.727664654, 40.749654641999946], [-73.72339596299987, 40.750389075999855]]]}}, {\"id\": \"19\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.26669397215120155, \"OBJECTID\": 20, \"Shape_Leng\": 0.0514401924362, \"Shape_Area\": 0.000134512633032, \"zone\": \"Belmont\", \"LocationID\": 20, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88386792099986, 40.8642908889999], [-73.88060896899991, 40.860357092999884], [-73.88311982600001, 40.848221091999925], [-73.88987871399985, 40.85207022099993], [-73.89109710400002, 40.856779813999886], [-73.89027555599993, 40.86110907299996], [-73.89177188799992, 40.86187171599992], [-73.88362518100003, 40.86725758799991], [-73.88267624699999, 40.86608914099984], [-73.88386792099986, 40.8642908889999]]]}}, {\"id\": \"20\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.290912805733734, \"OBJECTID\": 21, \"Shape_Leng\": 0.115973569062, \"Shape_Area\": 0.000380251345507, \"zone\": \"Bensonhurst East\", \"LocationID\": 21, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97418385499991, 40.609463501999905], [-73.97299433999983, 40.6088141419999], [-73.97291116900001, 40.599315803999964], [-73.97148350899987, 40.59717158499996], [-73.98035785499995, 40.59619154099992], [-73.97995443499985, 40.594073811999884], [-73.98601936799993, 40.594446048999906], [-73.98962150099992, 40.59098199299996], [-73.99923690396953, 40.588935328269024], [-73.99710730365224, 40.591390755943046], [-74.00004314944917, 40.59120428665461], [-73.9994830488618, 40.59296919994875], [-73.99530829599992, 40.59442758699989], [-73.99674706899995, 40.59564312599992], [-73.9868476259999, 40.60509723599994], [-73.98913191499994, 40.6064561369999], [-73.97963821799982, 40.61556454699991], [-73.97477658, 40.6126384749999], [-73.97418385499991, 40.609463501999905]]]}}, {\"id\": \"21\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2618891755765469, \"OBJECTID\": 22, \"Shape_Leng\": 0.126170229196, \"Shape_Area\": 0.000472062684608, \"zone\": \"Bensonhurst West\", \"LocationID\": 22, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99254973599997, 40.62427426799996], [-73.97963821799982, 40.61556454699991], [-73.98913191499994, 40.6064561369999], [-73.9868476259999, 40.60509723599994], [-73.99674706899995, 40.59564312599992], [-73.99530829599992, 40.59442758699989], [-74.00105748445709, 40.5925370050623], [-74.00200394804675, 40.594132371137796], [-73.99559320799995, 40.600232221999875], [-74.00493448899994, 40.60507440599987], [-74.00094408399997, 40.60863970999988], [-74.01050403599997, 40.613896610999866], [-73.99932090299998, 40.62465524699996], [-73.99771358999998, 40.62368026999996], [-73.99678847599995, 40.626826050999895], [-73.99254973599997, 40.62427426799996]]]}}, {\"id\": \"22\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.5886561171973435, \"OBJECTID\": 23, \"Shape_Leng\": 0.290556028962, \"Shape_Area\": 0.00219556576201, \"zone\": \"Bloomfield/Emerson Hill\", \"LocationID\": 23, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.19568609223377, 40.63501686464005], [-74.17459706299995, 40.622364010999874], [-74.16163010299996, 40.61793493599989], [-74.15018666499996, 40.60949781099991], [-74.14391816399994, 40.60798983699987], [-74.11742466500002, 40.609289008999916], [-74.11487058499988, 40.602136659999864], [-74.12059546299996, 40.601809757999874], [-74.12506115499997, 40.59818005599987], [-74.13091909399999, 40.598855047999905], [-74.13864458599996, 40.58906080499989], [-74.14790576499993, 40.58855634599993], [-74.16025756099997, 40.59534393799988], [-74.16189940599996, 40.59529877099993], [-74.16369888199995, 40.59225833899993], [-74.16360268199993, 40.593374198999854], [-74.16886362399997, 40.59472808699995], [-74.16719634899995, 40.602075311999926], [-74.17260229799992, 40.602971490999934], [-74.17822457299994, 40.5998280609999], [-74.17932277499995, 40.59676057899984], [-74.18730248699987, 40.588488221999896], [-74.185371048, 40.58745101499987], [-74.19423261399992, 40.58514680599991], [-74.19964981299998, 40.58226262599991], [-74.20048848899994, 40.5799497779999], [-74.20583829199992, 40.57984126099989], [-74.204646089432, 40.589285745465865], [-74.19751357718701, 40.59679898603677], [-74.20281628374593, 40.608270827967345], [-74.20244374449518, 40.61328469393197], [-74.2003834842897, 40.616428098243766], [-74.20163201104123, 40.623121565457346], [-74.2007872998309, 40.63034627446781], [-74.19568609223377, 40.63501686464005]]]}}, {\"id\": \"23\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2047782987154244, \"OBJECTID\": 24, \"Shape_Leng\": 0.0469999619287, \"Shape_Area\": 6.07235737749e-05, \"zone\": \"Bloomingdale\", \"LocationID\": 24, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95953658899998, 40.798718525999895], [-73.96004456499999, 40.79804123499991], [-73.97287179090726, 40.8033561875739], [-73.97110765876137, 40.80579013958964], [-73.95817297099987, 40.800582540999876], [-73.95953658899998, 40.798718525999895]]]}}, {\"id\": \"24\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21232970147950983, \"OBJECTID\": 25, \"Shape_Leng\": 0.0471458199319, \"Shape_Area\": 0.000124168267356, \"zone\": \"Boerum Hill\", \"LocationID\": 25, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98155298299992, 40.689146163999936], [-73.97804289599988, 40.68485661499987], [-73.98175642299985, 40.67931462399987], [-73.99490113599977, 40.6844302379999], [-73.99236366999985, 40.689690123999924], [-73.99054474599986, 40.689168104999936], [-73.98902944799994, 40.69212386099992], [-73.98155298299992, 40.689146163999936]]]}}, {\"id\": \"25\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.24551625174334002, \"OBJECTID\": 26, \"Shape_Leng\": 0.12354780707, \"Shape_Area\": 0.000534039927626, \"zone\": \"Borough Park\", \"LocationID\": 26, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98331628499983, 40.6414786819999], [-73.98042057899988, 40.64238522699991], [-73.97911188699993, 40.63544038599991], [-73.9768886119999, 40.6356748619999], [-73.97705352899999, 40.62153252199987], [-73.97539380499998, 40.62076998699995], [-73.97785009299986, 40.617287605999906], [-73.97537335500002, 40.6157547219999], [-73.97335879399992, 40.616541714999904], [-73.97290326899996, 40.61415296799988], [-73.97517176999993, 40.61472418599991], [-73.97477658, 40.6126384749999], [-74.00702302499997, 40.63210999499996], [-74.00556490399993, 40.63308317599992], [-74.00735636699996, 40.6341636269999], [-73.99917564799993, 40.642025441999856], [-73.9955103369999, 40.63980966699993], [-73.98834986699991, 40.64456224399992], [-73.98331628499983, 40.6414786819999]]]}}, {\"id\": \"26\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23818890723242828, \"OBJECTID\": 27, \"Shape_Leng\": 0.202508808518, \"Shape_Area\": 0.00134088762746, \"zone\": \"Breezy Point/Fort Tilden/Riis Beach\", \"LocationID\": 27, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86522555399998, 40.57045847199989], [-73.86268391251235, 40.56650565604353], [-73.8949181726631, 40.5570787962983], [-73.90658574923901, 40.555710625377294], [-73.94073681682697, 40.54182008707755], [-73.93985107460422, 40.55456005374592], [-73.9261572327542, 40.56155260972308], [-73.91766081900093, 40.56282614968639], [-73.91197024752503, 40.56586941753247], [-73.90683894638656, 40.562856359344565], [-73.90121507272005, 40.563008460632346], [-73.89261013445609, 40.56858724728667], [-73.87780808995733, 40.56880063006316], [-73.86288301945429, 40.576059761127304], [-73.86361132999998, 40.573823056999835], [-73.86662812000002, 40.573057648999914], [-73.86522555399998, 40.57045847199989]]]}}, {\"id\": \"27\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20950062225886823, \"OBJECTID\": 28, \"Shape_Leng\": 0.097960782214, \"Shape_Area\": 0.000291203927662, \"zone\": \"Briarwood/Jamaica Hills\", \"LocationID\": 28, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79240413399991, 40.716193040999926], [-73.79303800099989, 40.71071107499988], [-73.79776810199988, 40.7091431879999], [-73.79870150699992, 40.71066337699988], [-73.80130625, 40.709742148999965], [-73.80302414799988, 40.70813965699997], [-73.80165725599993, 40.705761474999846], [-73.8056477309999, 40.70467172799994], [-73.80694856299993, 40.70736709999989], [-73.81201248599983, 40.70594494499993], [-73.81152856899989, 40.70227287499991], [-73.81637824499992, 40.70245196399987], [-73.82591945199992, 40.71598987599992], [-73.80494148499987, 40.715996230999956], [-73.79073248899982, 40.71920875699987], [-73.79240413399991, 40.716193040999926]]]}}, {\"id\": \"28\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.26611204508792763, \"OBJECTID\": 29, \"Shape_Leng\": 0.0714083127733, \"Shape_Area\": 0.000201673837402, \"zone\": \"Brighton Beach\", \"LocationID\": 29, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96004798699995, 40.58326987199995], [-73.95414287899986, 40.58310613999989], [-73.95235846771321, 40.574274685845246], [-73.9589871897255, 40.57343065573189], [-73.95941778299985, 40.57517798299994], [-73.9688899589999, 40.57526123899986], [-73.96514385199995, 40.5911019159999], [-73.9606798409999, 40.591597582999945], [-73.96004798699995, 40.58326987199995]]]}}, {\"id\": \"29\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2900523560209424, \"OBJECTID\": 30, \"Shape_Leng\": 0.0945097669793, \"Shape_Area\": 0.000145862107626, \"zone\": \"Broad Channel\", \"LocationID\": 30, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82075892499992, 40.61523267899991], [-73.81652681189118, 40.61494701007998], [-73.81475287105245, 40.6086327967523], [-73.81739192103247, 40.60515812414338], [-73.82071206789055, 40.59517051496212], [-73.83465599507487, 40.59516845882606], [-73.83239364476839, 40.59780335319121], [-73.82806159687702, 40.59768040282441], [-73.8259798598142, 40.599472067181495], [-73.82465420648664, 40.5981492468515], [-73.82376612290614, 40.60004007091038], [-73.82146929866913, 40.59999372263383], [-73.82445218054455, 40.60079696284418], [-73.81961795381072, 40.610479041032136], [-73.82087841795061, 40.61225734590997], [-73.82161460094558, 40.60848031911391], [-73.8238246859766, 40.61149866495507], [-73.82075892499992, 40.61523267899991]]]}}, {\"id\": \"30\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20674295285310215, \"OBJECTID\": 31, \"Shape_Leng\": 0.0964245666516, \"Shape_Area\": 0.000333975927329, \"zone\": \"Bronx Park\", \"LocationID\": 31, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87094462199981, 40.85726608099992], [-73.86841539699994, 40.85778035799994], [-73.86990379599995, 40.85587302199991], [-73.86823466099995, 40.85091373699986], [-73.86889243299996, 40.848112321999906], [-73.87459175399982, 40.841520647999914], [-73.878171756, 40.84262443699987], [-73.8773292409999, 40.84398312499995], [-73.88311982600001, 40.848221091999925], [-73.88060896899991, 40.860357092999884], [-73.88394303099996, 40.86494818699985], [-73.87619224900003, 40.87047671099988], [-73.87094367500002, 40.87851076699998], [-73.86943471300005, 40.87812919899994], [-73.87094462199981, 40.85726608099992]]]}}, {\"id\": \"31\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2757900533762603, \"OBJECTID\": 32, \"Shape_Leng\": 0.05426721601, \"Shape_Area\": 0.000150879171971, \"zone\": \"Bronxdale\", \"LocationID\": 32, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85882515999995, 40.85806135699987], [-73.87104944999982, 40.857304836999965], [-73.87056192499989, 40.87151077799995], [-73.86137924099984, 40.8713365119999], [-73.86156274099987, 40.86554946399989], [-73.85961281199982, 40.865512717999856], [-73.85780378799993, 40.86303621699989], [-73.8566377469999, 40.8581191719999], [-73.85882515999995, 40.85806135699987]]]}}, {\"id\": \"32\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2051921193027173, \"OBJECTID\": 33, \"Shape_Leng\": 0.0532702931967, \"Shape_Area\": 0.000147416802448, \"zone\": \"Brooklyn Heights\", \"LocationID\": 33, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99619227264343, 40.70337714093203], [-73.99087453800003, 40.70067311699993], [-73.99097187, 40.692548799999926], [-73.98902944799994, 40.69212386099992], [-73.99054474599986, 40.689168104999936], [-74.00110519399988, 40.692056594999954], [-74.0009586845849, 40.694069083791064], [-74.00301393814541, 40.69477784423956], [-74.00026872377784, 40.69496580257801], [-74.00043147340423, 40.69705246617853], [-73.99876458027717, 40.697120733093975], [-74.0010490305752, 40.697908236697415], [-73.9983776068576, 40.698063296146074], [-74.00001840433616, 40.699466048073575], [-73.99716054243916, 40.69979281264407], [-73.99813879899439, 40.701518788248414], [-73.99619227264343, 40.70337714093203]]]}}, {\"id\": \"33\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2211178584290075, \"OBJECTID\": 34, \"Shape_Leng\": 0.0657059323545, \"Shape_Area\": 0.000173946146651, \"zone\": \"Brooklyn Navy Yard\", \"LocationID\": 34, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.97232032119902, 40.70908288315216], [-73.9702621853228, 40.706887504284005], [-73.97018075572805, 40.70492927056683], [-73.96751516691087, 40.703437356878126], [-73.96929693837701, 40.705088331229625], [-73.96838933699995, 40.70682918699989], [-73.96217978199998, 40.70022070999989], [-73.963971858, 40.69637865499991], [-73.98054419199988, 40.69824651199988], [-73.97885303360711, 40.7060655352846], [-73.97713562625376, 40.703005581402515], [-73.97596096344841, 40.704715506440465], [-73.9769274918834, 40.70279782040038], [-73.97455023388345, 40.70159034430477], [-73.97573541084542, 40.69957044770558], [-73.97411513926478, 40.70130527141927], [-73.97239201397464, 40.700166322752274], [-73.97318676670886, 40.70166980367368], [-73.97078200587988, 40.69997866301123], [-73.97283512788735, 40.70272874074404], [-73.96948655218084, 40.70051906125818], [-73.97283889974949, 40.70334642860182], [-73.96898517428966, 40.70174708878653], [-73.97465569608048, 40.70607426299424], [-73.97254880057814, 40.70620449000379], [-73.9742361696854, 40.70802438612241], [-73.97109655489108, 40.705850056558255], [-73.97232032119902, 40.70908288315216]]], [[[-73.97892920137538, 40.705931589879214], [-73.97893110285521, 40.70592490623034], [-73.97906084872884, 40.705946028470365], [-73.97905767006142, 40.705952889050316], [-73.97892920137538, 40.705931589879214]]], [[[-73.98237340600002, 40.70554334999991], [-73.98242287179853, 40.70582205595453], [-73.98102390618274, 40.705898913893535], [-73.98230205183869, 40.705736979399674], [-73.98237340600002, 40.70554334999991]]]]}}, {\"id\": \"34\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.19658802029964945, \"OBJECTID\": 35, \"Shape_Leng\": 0.085787918592, \"Shape_Area\": 0.000323825195932, \"zone\": \"Brownsville\", \"LocationID\": 35, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90105329800004, 40.66212482099994], [-73.89880698499995, 40.65740666199988], [-73.90784346399994, 40.65459152999988], [-73.90855790499995, 40.65209593799989], [-73.91012255699995, 40.655851770999945], [-73.91236458899995, 40.65551356499985], [-73.9195947979998, 40.66199567899985], [-73.92008167099986, 40.659857632999866], [-73.92637970499992, 40.6655148919999], [-73.90347422300006, 40.67550668299985], [-73.90105329800004, 40.66212482099994]]]}}, {\"id\": \"35\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22147737761861785, \"OBJECTID\": 36, \"Shape_Leng\": 0.0870507867063, \"Shape_Area\": 0.000247666139269, \"zone\": \"Bushwick North\", \"LocationID\": 36, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91254913799993, 40.70387341999989], [-73.912904041, 40.7023618909999], [-73.91067882699998, 40.701045968999914], [-73.91180820099989, 40.69993800299986], [-73.9042601839999, 40.69570037099989], [-73.90579597099993, 40.69412715499987], [-73.90123290699994, 40.69144227899997], [-73.904055772, 40.68854627799986], [-73.93115533899987, 40.70377233499987], [-73.93269784899994, 40.70317039099989], [-73.93391870799988, 40.70748702599992], [-73.92189184700005, 40.709396096999896], [-73.91254913799993, 40.70387341999989]]]}}, {\"id\": \"36\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22153532506500048, \"OBJECTID\": 37, \"Shape_Leng\": 0.142810040466, \"Shape_Area\": 0.000452062644782, \"zone\": \"Bushwick South\", \"LocationID\": 37, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93312548700004, 40.70470086199992], [-73.93269784899994, 40.70317039099989], [-73.93115533899987, 40.70377233499987], [-73.90482499699988, 40.688808569999956], [-73.90180467199995, 40.6907662979999], [-73.90116154999988, 40.68787793499991], [-73.89646625099995, 40.68233642199988], [-73.89704472199993, 40.68056181599996], [-73.90047001100004, 40.679535243999936], [-73.90262262299991, 40.6806645029999], [-73.9040463979999, 40.67922059799985], [-73.93856854799981, 40.698848135999874], [-73.94394947299996, 40.698221278999924], [-73.94438788599984, 40.70042452299993], [-73.94042057899993, 40.701076597999865], [-73.94103009699987, 40.7046361769999], [-73.94273448399997, 40.70447330599988], [-73.94378799599994, 40.710900378999945], [-73.93480281900003, 40.713372495999884], [-73.93229708600003, 40.70774944699988], [-73.93391870799988, 40.70748702599992], [-73.93312548700004, 40.70470086199992]]]}}, {\"id\": \"37\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22644071960907428, \"OBJECTID\": 38, \"Shape_Leng\": 0.0832175685234, \"Shape_Area\": 0.000327392684821, \"zone\": \"Cambria Heights\", \"LocationID\": 38, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.73624941199995, 40.702536977999884], [-73.72677759516957, 40.7030592911994], [-73.72586264191175, 40.683241080973865], [-73.7475517539999, 40.68934733599984], [-73.7435928609999, 40.69621532499986], [-73.74618157999997, 40.696121925999904], [-73.74773012199991, 40.69955313699991], [-73.74500119100004, 40.70026221299988], [-73.74590871000004, 40.70228727399988], [-73.74109293999989, 40.7034810629999], [-73.73624941199995, 40.702536977999884]]]}}, {\"id\": \"38\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.28362403401946795, \"OBJECTID\": 39, \"Shape_Leng\": 0.132369620757, \"Shape_Area\": 0.00090028293862, \"zone\": \"Canarsie\", \"LocationID\": 39, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.89121853499988, 40.64998769899987], [-73.88805587182881, 40.64511978725729], [-73.87714434884276, 40.63597649800373], [-73.88364451728226, 40.630943312138584], [-73.88263248433404, 40.62827427312689], [-73.88369231571497, 40.62755937391403], [-73.88752673706196, 40.628331824482444], [-73.89577374598754, 40.62251907138477], [-73.91860649599992, 40.632090414999894], [-73.91995062999999, 40.64470762999989], [-73.91830152299998, 40.645812262999925], [-73.90944846399994, 40.65152159899992], [-73.90398200599991, 40.64662856799988], [-73.89586154900002, 40.65188052699987], [-73.89761013299987, 40.65345381599992], [-73.89354677099988, 40.65520271399988], [-73.89000589199988, 40.651148696999925], [-73.89121853499988, 40.64998769899987]]], [[[-73.88833929256269, 40.64671022891759], [-73.88849720193863, 40.64675678600008], [-73.88853291212868, 40.646861882230674], [-73.88846787961647, 40.646899734025276], [-73.88833929256269, 40.64671022891759]]]]}}, {\"id\": \"39\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21723129248724968, \"OBJECTID\": 40, \"Shape_Leng\": 0.0518506648762, \"Shape_Area\": 0.000108949211644, \"zone\": \"Carroll Gardens\", \"LocationID\": 40, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.998192111, 40.68228732999989], [-73.99620846299992, 40.681723424999916], [-73.99490113599977, 40.6844302379999], [-73.98722413899985, 40.68144730199988], [-73.99069861899994, 40.67580741899994], [-73.99506451099984, 40.67584047499984], [-73.9986188799999, 40.67183017199989], [-74.00329231299999, 40.67964328799992], [-74.00064737999996, 40.685253946999914], [-73.99908066099994, 40.68481599199985], [-74.00004865199998, 40.682803355999965], [-73.998192111, 40.68228732999989]]]}}, {\"id\": \"40\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20351002119843972, \"OBJECTID\": 41, \"Shape_Leng\": 0.052793109453, \"Shape_Area\": 0.000143093037737, \"zone\": \"Central Harlem\", \"LocationID\": 41, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94773985499985, 40.809599720999884], [-73.94177140199992, 40.807088999999955], [-73.94523965199991, 40.80513601399989], [-73.94613131, 40.80387735799992], [-73.94459751299998, 40.803228144999906], [-73.94922045699984, 40.796909858999896], [-73.95964685399987, 40.80115642299993], [-73.954966572, 40.81006455499988], [-73.9535757779999, 40.8094766979999], [-73.95210125199992, 40.811442853999935], [-73.94773985499985, 40.809599720999884]]]}}, {\"id\": \"41\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.19878137510607055, \"OBJECTID\": 42, \"Shape_Leng\": 0.0927092274616, \"Shape_Area\": 0.000263896952605, \"zone\": \"Central Harlem North\", \"LocationID\": 42, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93436121591056, 40.83620061961779], [-73.93383089287109, 40.819521063761556], [-73.93901753299998, 40.81085655699991], [-73.93580780199986, 40.80949763799987], [-73.93854407899988, 40.80572965299991], [-73.95210125199992, 40.811442853999935], [-73.94607828700005, 40.821263215999906], [-73.93924885599988, 40.82829635099989], [-73.93864026699997, 40.82973887399988], [-73.9401708989999, 40.83038439099996], [-73.93868316299985, 40.832800788999904], [-73.93436121591056, 40.83620061961779]]]}}, {\"id\": \"42\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2120281596812563, \"OBJECTID\": 43, \"Shape_Leng\": 0.0997386183576, \"Shape_Area\": 0.000379662912054, \"zone\": \"Central Park\", \"LocationID\": 43, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97255352299985, 40.76489773199991], [-73.98164804599995, 40.76843632199994], [-73.95817297099987, 40.800582540999876], [-73.94922045699984, 40.796909858999896], [-73.97255352299985, 40.76489773199991]]]}}, {\"id\": \"43\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23410887397464583, \"OBJECTID\": 44, \"Shape_Leng\": 0.235688967594, \"Shape_Area\": 0.00194465649192, \"zone\": \"Charleston/Tottenville\", \"LocationID\": 44, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.21220034099994, 40.556443429999874], [-74.21784397899994, 40.554432717999894], [-74.22131562699998, 40.55025307399992], [-74.21776941599994, 40.54112662999992], [-74.21800676499996, 40.53229858999995], [-74.21354593399995, 40.521235715999886], [-74.22379595899992, 40.518196345999876], [-74.22690993299994, 40.5191713979999], [-74.22159300278173, 40.50250043822093], [-74.23114494958575, 40.50185121588201], [-74.23988017293566, 40.49757362656005], [-74.24917156868965, 40.49656729705339], [-74.25335538022304, 40.50041319788582], [-74.25554269163935, 40.50783774249614], [-74.25056007272197, 40.516106741535], [-74.24921528676096, 40.51508486129925], [-74.24994158808036, 40.516137488190985], [-74.24854589291671, 40.51616991714839], [-74.24940770789283, 40.516980831423645], [-74.24851449056031, 40.517475626265046], [-74.24644039651318, 40.515965028508525], [-74.2456347316994, 40.518075580745766], [-74.23991937852388, 40.52004699652993], [-74.24293235036517, 40.52122716113276], [-74.24396500758859, 40.52490536633781], [-74.24150733322897, 40.531041342795135], [-74.24204570286014, 40.53434637705037], [-74.24533651176053, 40.53690599443462], [-74.24560205698324, 40.54094959457726], [-74.24803463735688, 40.54309324044144], [-74.24336346552776, 40.54786513523582], [-74.24039182810051, 40.547663310173775], [-74.23641623448965, 40.55050862684339], [-74.23641559733107, 40.55232806634515], [-74.23334188482498, 40.55249971061369], [-74.2287242997478, 40.5562970171392], [-74.22062175949124, 40.55589882100512], [-74.21974356930723, 40.55461267207239], [-74.21921716370927, 40.55579696987855], [-74.21859325782798, 40.55467421024778], [-74.21870767560165, 40.55604101620115], [-74.21775342391726, 40.55500528477618], [-74.2056960519999, 40.55851600599987], [-74.21220034099994, 40.556443429999874]]]}}, {\"id\": \"44\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20451319801273937, \"OBJECTID\": 45, \"Shape_Leng\": 0.0459068626235, \"Shape_Area\": 9.13594688556e-05, \"zone\": \"Chinatown\", \"LocationID\": 45, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99750445299988, 40.71406913199995], [-73.99256242199989, 40.71438807699996], [-73.99190824271031, 40.70953292664334], [-73.99919451154429, 40.70794737667571], [-74.0058917509999, 40.71200715999989], [-73.99995620900002, 40.71801709499987], [-73.99605872699998, 40.7162316399999], [-73.99750445299988, 40.71406913199995]]]}}, {\"id\": \"45\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23050374219567998, \"OBJECTID\": 46, \"Shape_Leng\": 0.134475429879, \"Shape_Area\": 0.000926391677672, \"zone\": \"City Island\", \"LocationID\": 46, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.78452431937963, 40.86047706314786], [-73.78519664387113, 40.85882702363741], [-73.78743186194176, 40.858935245614425], [-73.78685886773299, 40.86004505783956], [-73.78452431937963, 40.86047706314786]]], [[[-73.76783205637173, 40.854442205742075], [-73.76938529139694, 40.85253613399675], [-73.76959674311465, 40.84759451909819], [-73.76789492672636, 40.84528943351961], [-73.76901389181107, 40.84509734272193], [-73.77157218833237, 40.8475000585803], [-73.7708892022117, 40.84975177133807], [-73.77296356611284, 40.8521592539586], [-73.77173060411904, 40.852476831319095], [-73.77296285334867, 40.853800062246464], [-73.77219223055407, 40.85998663859697], [-73.76947305423998, 40.85933013666858], [-73.76533243995276, 40.85504359512487], [-73.76783205637173, 40.854442205742075]]], [[[-73.79018745194999, 40.8586099173861], [-73.788043429758, 40.85755991768708], [-73.7884773265335, 40.8544005205322], [-73.78690734702697, 40.854063972299144], [-73.78796264288079, 40.8534747209355], [-73.78338158610377, 40.85158673391434], [-73.78420047134377, 40.8500606442406], [-73.78154449088599, 40.84940043464333], [-73.78267467078706, 40.84813967294664], [-73.78097820938946, 40.8484583479135], [-73.78263740397888, 40.848072649245836], [-73.78148431950568, 40.8478821938486], [-73.78216393435586, 40.84663633778076], [-73.780403269586, 40.846931134166056], [-73.7833653613534, 40.84444472298169], [-73.78348428748797, 40.84367854599843], [-73.78290650949755, 40.84393612584013], [-73.78052736048552, 40.84452514033816], [-73.7799270374445, 40.843883081355834], [-73.78347481249244, 40.843654738109784], [-73.78288778301486, 40.84249482884962], [-73.78022773872677, 40.84377818129771], [-73.78286499304913, 40.84247810188268], [-73.78059735822595, 40.84278659994098], [-73.78216624418964, 40.84168914411004], [-73.7804257718152, 40.842494394206874], [-73.78019599955293, 40.84204621097616], [-73.78185048197766, 40.841653946394324], [-73.7807975292347, 40.84125614604844], [-73.78282260139727, 40.836338155887844], [-73.78485009222643, 40.83747048917844], [-73.78563997845448, 40.83706443458537], [-73.78583067964128, 40.83720998000451], [-73.7857350929868, 40.83923423177581], [-73.78815921166512, 40.84049077856704], [-73.79179343625451, 40.846731672329284], [-73.78951371516173, 40.85130077171307], [-73.79311837183832, 40.851578295240834], [-73.79042053780161, 40.85302297821258], [-73.79281511966936, 40.852494819308916], [-73.79080758729923, 40.85319906006435], [-73.79287610048584, 40.8529441813284], [-73.79105721254825, 40.85391841242664], [-73.79245816844812, 40.85360070875487], [-73.79107002765942, 40.85394495666465], [-73.79253714604873, 40.85384932781285], [-73.79129282160633, 40.854584025976855], [-73.79201712228527, 40.85617694860626], [-73.79270523985137, 40.85619232890572], [-73.79268768222656, 40.856335218720375], [-73.79315700296156, 40.85635091107476], [-73.7931468608596, 40.856467828471146], [-73.79018745194999, 40.8586099173861]]], [[[-73.78283291447852, 40.85587030844573], [-73.78302371522504, 40.85509276666495], [-73.78394699722007, 40.85563043752662], [-73.78343487501385, 40.856362865958886], [-73.78283291447852, 40.85587030844573]]], [[[-73.78061730829718, 40.855735175810025], [-73.78090476851291, 40.8549681331623], [-73.7815638969994, 40.855001286433904], [-73.78122782332017, 40.856087789162764], [-73.78061730829718, 40.855735175810025]]], [[[-73.78833349834521, 40.834667129759346], [-73.78931223606624, 40.83446488655343], [-73.78951019872322, 40.835364042525754], [-73.78845700015209, 40.835309914315395], [-73.78833349834521, 40.834667129759346]]]]}}, {\"id\": \"46\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23327044941651862, \"OBJECTID\": 47, \"Shape_Leng\": 0.0898275563294, \"Shape_Area\": 0.000163198117339, \"zone\": \"Claremont/Bathgate\", \"LocationID\": 47, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89090203399996, 40.854116885999865], [-73.89431774099977, 40.851891310999854], [-73.89617712999998, 40.8466583209999], [-73.89290102299996, 40.84626832999992], [-73.89537836199995, 40.842938778999965], [-73.89666161699988, 40.84352460699992], [-73.90553116799988, 40.82910413299989], [-73.91049010699992, 40.830737372999884], [-73.90538357999992, 40.839768982999885], [-73.90483121999989, 40.84480034199987], [-73.901344046, 40.84442875099994], [-73.89106280699994, 40.8615294119999], [-73.89090203399996, 40.854116885999865]]]}}, {\"id\": \"47\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21261808278031644, \"OBJECTID\": 48, \"Shape_Leng\": 0.0437467441431, \"Shape_Area\": 9.42538425377e-05, \"zone\": \"Clinton East\", \"LocationID\": 48, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99117738199989, 40.75532984899992], [-73.99730671499998, 40.757110506999894], [-73.98806289599996, 40.769790799999925], [-73.98236545099999, 40.7673921529999], [-73.99117738199989, 40.75532984899992]]]}}, {\"id\": \"48\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21935920217712804, \"OBJECTID\": 49, \"Shape_Leng\": 0.0752900183427, \"Shape_Area\": 0.000203185525471, \"zone\": \"Clinton Hill\", \"LocationID\": 49, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96062056499981, 40.69632013999993], [-73.96015854699988, 40.69411730899989], [-73.95614239299985, 40.694579018999896], [-73.95541057999996, 40.69090829199994], [-73.96029281699998, 40.69034624999995], [-73.95956770199984, 40.686682554999955], [-73.95468418899998, 40.68724485399985], [-73.95328187399996, 40.68020003099985], [-73.95529361599996, 40.679203150999825], [-73.96868961400001, 40.68200525799988], [-73.97013154199992, 40.68950630799986], [-73.96910362499993, 40.68946752199986], [-73.9693945009999, 40.696080388999846], [-73.9645507049999, 40.696107591999855], [-73.96105100699992, 40.69832607899989], [-73.96062056499981, 40.69632013999993]]]}}, {\"id\": \"49\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20820841160617087, \"OBJECTID\": 50, \"Shape_Leng\": 0.0557479867536, \"Shape_Area\": 0.000173249283326, \"zone\": \"Clinton West\", \"LocationID\": 50, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99667705784736, 40.77351005394221], [-73.98806289599996, 40.769790799999925], [-73.99547450199992, 40.759626856999894], [-74.00375735018837, 40.763582241820224], [-74.0009730802201, 40.763420271725614], [-74.00311291631381, 40.76474559615368], [-73.99915098087908, 40.76420525958265], [-74.0020245433034, 40.76589326657037], [-73.99857638530527, 40.76499860727083], [-74.00157335385273, 40.766977192210646], [-73.9980667590535, 40.76587188113572], [-73.99737808447954, 40.76682338637958], [-74.00064950530414, 40.76824523610559], [-73.99715588949206, 40.76712869617278], [-73.9964571540504, 40.76809881579563], [-73.99973590247859, 40.76950617733152], [-73.99707176039315, 40.768732643576804], [-73.9987620691431, 40.7708250567092], [-73.99631327022087, 40.76979883633644], [-73.99493501658995, 40.77146814657798], [-73.99667705784736, 40.77351005394221]]]}}, {\"id\": \"50\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23636520312166318, \"OBJECTID\": 51, \"Shape_Leng\": 0.0953613442277, \"Shape_Area\": 0.000395756553505, \"zone\": \"Co-Op City\", \"LocationID\": 51, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82091617199997, 40.86891693399989], [-73.81578764099999, 40.8657580609999], [-73.82037552999996, 40.86103300499993], [-73.82699653199998, 40.85908778599986], [-73.83315960399985, 40.86831975599994], [-73.83565766499987, 40.86751275599992], [-73.83829221499984, 40.87295133899995], [-73.83924788999988, 40.876618737999905], [-73.8381484219999, 40.876953298999915], [-73.84324113799987, 40.879108912999904], [-73.82387140599998, 40.88778362799996], [-73.82077408699998, 40.886710602999884], [-73.81992586499996, 40.88211933299987], [-73.82329599399986, 40.87309741799988], [-73.82091617199997, 40.86891693399989]]]}}, {\"id\": \"51\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20822031786021278, \"OBJECTID\": 52, \"Shape_Leng\": 0.0289076710093, \"Shape_Area\": 4.52027509603e-05, \"zone\": \"Cobble Hill\", \"LocationID\": 52, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99588110299992, 40.68239495399995], [-74.00004865199998, 40.682803355999965], [-73.99908066099994, 40.68481599199985], [-74.00064737999996, 40.685253946999914], [-73.99776924399993, 40.691194697999954], [-73.99236366999985, 40.689690123999924], [-73.99588110299992, 40.68239495399995]]]}}, {\"id\": \"52\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.25258826219014335, \"OBJECTID\": 53, \"Shape_Leng\": 0.161500913385, \"Shape_Area\": 0.000947530980821, \"zone\": \"College Point\", \"LocationID\": 53, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.83839013354552, 40.798304872075995], [-73.83919004727183, 40.7977772664477], [-73.84025860809128, 40.79758885843212], [-73.83840083088981, 40.79675525865344], [-73.83663288911686, 40.79251637576927], [-73.84067828300006, 40.79390789199993], [-73.84081532599994, 40.7915048849999], [-73.83895676099984, 40.79143843099996], [-73.84008157499997, 40.78785153099992], [-73.83824429099994, 40.78778591599995], [-73.83869921299997, 40.781676956999874], [-73.82513740299993, 40.78169970599994], [-73.83000011099983, 40.77373240699996], [-73.83913189281712, 40.765991105784146], [-73.83975387982034, 40.767120445440874], [-73.84491332846012, 40.76552838817853], [-73.84780575104425, 40.76677294817163], [-73.84977407793218, 40.769936012942956], [-73.8486715286884, 40.770761572902536], [-73.85130108374476, 40.77180636079771], [-73.84897619867189, 40.77356166795698], [-73.84895559325399, 40.77796167373135], [-73.84963631633588, 40.77948192035157], [-73.85094705724252, 40.77906915543026], [-73.85148755480326, 40.77899297861757], [-73.84981720066014, 40.77963034805418], [-73.85232086242219, 40.77916928470371], [-73.85238832778414, 40.7795289023869], [-73.84982102259761, 40.779651339303875], [-73.84955315550077, 40.779990415384006], [-73.85169028085423, 40.779867508478176], [-73.85167099162219, 40.780035228878106], [-73.84949381208321, 40.780065828114246], [-73.85093646396534, 40.781119575156005], [-73.8492525818285, 40.782264775014035], [-73.85528242786589, 40.78197538434999], [-73.8595236950597, 40.785561954252664], [-73.85284230133867, 40.78821137173992], [-73.85480194733185, 40.788619154725914], [-73.85265830693045, 40.79110642061034], [-73.85377854910695, 40.79242127400702], [-73.8526309825624, 40.79494485424474], [-73.84906160835088, 40.79336261029852], [-73.84855988738983, 40.79549506021838], [-73.84354668938991, 40.79491394401049], [-73.84225959368331, 40.79558496893736], [-73.8428667525936, 40.79692222907191], [-73.84158721174394, 40.795401069221235], [-73.84041517975831, 40.7976784372587], [-73.83839013354552, 40.798304872075995]]], [[[-73.8678840747076, 40.78736103669457], [-73.86813032000018, 40.78723277784419], [-73.8692349173292, 40.78596991495815], [-73.8680991225789, 40.78745866181839], [-73.8678840747076, 40.78736103669457]]]]}}, {\"id\": \"53\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2130004273996172, \"OBJECTID\": 54, \"Shape_Leng\": 0.0762192448554, \"Shape_Area\": 0.000132452130536, \"zone\": \"Columbia Street\", \"LocationID\": 54, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-74.00174362072502, 40.69240674985266], [-74.00400795971088, 40.693208199632345], [-74.00127014951309, 40.69329606183691], [-74.00174362072502, 40.69240674985266]]], [[[-74.00174362072502, 40.69240674985266], [-73.99776924399993, 40.691194697999954], [-74.00329231299999, 40.67964328799992], [-74.00596041099988, 40.683362870999936], [-74.00485241699997, 40.68566813099993], [-74.00783293766679, 40.68738505516274], [-74.00459270521766, 40.688215222986614], [-74.00636461225261, 40.68966966863697], [-74.00382038919723, 40.688929644699236], [-74.0053420962491, 40.6910916029778], [-74.0009615685494, 40.69012876835003], [-74.00479415294552, 40.69176162037443], [-74.00174362072502, 40.69240674985266]]], [[[-74.01092841300002, 40.684491472999824], [-74.01217596614636, 40.68409518562848], [-74.00816320571415, 40.68617364485845], [-74.0086007239999, 40.68590956499989], [-74.01092841300002, 40.684491472999824]]]]}}, {\"id\": \"54\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22235237171253217, \"OBJECTID\": 55, \"Shape_Leng\": 0.149956807524, \"Shape_Area\": 0.000736274713586, \"zone\": \"Coney Island\", \"LocationID\": 55, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98671582299981, 40.58456849999991], [-73.97742525299988, 40.58290659299994], [-73.96661531499983, 40.58457103699992], [-73.9688899589999, 40.57526123899986], [-73.95941778299985, 40.57517798299994], [-73.95939182223685, 40.57391210937612], [-73.98367270153267, 40.57140705768737], [-73.98336058039273, 40.56952999448666], [-73.98375790448641, 40.571396227250865], [-74.00208547639728, 40.569585983985014], [-74.00303186164808, 40.57218559475904], [-74.01115668729254, 40.57416676185885], [-74.01302222952468, 40.577804459633505], [-74.0111737229432, 40.58028397882522], [-74.00604734235807, 40.58198365303581], [-73.98814208290418, 40.57886596371379], [-73.98604509916208, 40.58172068515157], [-73.98800973376738, 40.579670687155776], [-73.98995504960881, 40.58066166976562], [-73.98671582299981, 40.58456849999991]]]}}, {\"id\": \"55\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2099739474607889, \"OBJECTID\": 56, \"Shape_Leng\": 0.0568478126677, \"Shape_Area\": 0.000180907844436, \"zone\": \"Corona\", \"LocationID\": 56, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85360568399982, 40.74749682799991], [-73.85011556599989, 40.74072356299995], [-73.84750820299992, 40.73900780699995], [-73.86462193299997, 40.73428275699993], [-73.86790963899986, 40.74493371599989], [-73.85442575299999, 40.748851725999884], [-73.85360568399982, 40.74749682799991]]]}}, {\"id\": \"56\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2099739474607889, \"OBJECTID\": 57, \"Shape_Leng\": 0.0192705048557, \"Shape_Area\": 1.80259807917e-05, \"zone\": \"Corona\", \"LocationID\": 56, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8513110499998, 40.74984337599988], [-73.85442575299999, 40.748851725999884], [-73.85579909099987, 40.75440952799994], [-73.85312635499996, 40.75510934299989], [-73.8513110499998, 40.74984337599988]]]}}, {\"id\": \"57\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.33021740969461133, \"OBJECTID\": 58, \"Shape_Leng\": 0.0598554094851, \"Shape_Area\": 0.000204980931361, \"zone\": \"Country Club\", \"LocationID\": 58, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.81561382067721, 40.84818955758243], [-73.81602685114903, 40.846400458770034], [-73.81464594697542, 40.84721961352901], [-73.81570353189448, 40.84629134042384], [-73.81368634738199, 40.84727432550905], [-73.81356372861671, 40.847151411278524], [-73.8150262760428, 40.845499995058404], [-73.81284939958525, 40.84631879949822], [-73.81324326350918, 40.84523590835256], [-73.81310050626846, 40.84517241453328], [-73.8128770359758, 40.8451346679459], [-73.81315597044244, 40.84521210037631], [-73.81285536957633, 40.845186285461466], [-73.81241445328128, 40.846559380748744], [-73.81227790900483, 40.846543458279285], [-73.8128577433794, 40.845129544589135], [-73.81254457815619, 40.84497959800826], [-73.81169971823665, 40.84668170081364], [-73.81266141349049, 40.84420764950083], [-73.81254909120709, 40.84495227514607], [-73.8136558066588, 40.8450591224657], [-73.81414462049185, 40.84481817326274], [-73.81437458252545, 40.8430889259994], [-73.81569792948075, 40.8443864281129], [-73.81823722746202, 40.844355041712625], [-73.81473345817534, 40.8413757283108], [-73.81606068217324, 40.83603019622843], [-73.82601252600003, 40.83402077799985], [-73.82624132999996, 40.84299359399991], [-73.82517999, 40.842700058999874], [-73.82495697699986, 40.846105444999864], [-73.82048405599983, 40.84850102599995], [-73.81507165904237, 40.84913361979451], [-73.81561382067721, 40.84818955758243]]]}}, {\"id\": \"58\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.3125641025641025, \"OBJECTID\": 59, \"Shape_Leng\": 0.0377948070893, \"Shape_Area\": 6.28765230648e-05, \"zone\": \"Crotona Park\", \"LocationID\": 59, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88941724899998, 40.839422732999836], [-73.88659802299989, 40.839279049999895], [-73.89697220299995, 40.8343452539999], [-73.9012157839999, 40.8354101409999], [-73.89666161699988, 40.84352460699992], [-73.88941724899998, 40.839422732999836]]]}}, {\"id\": \"59\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20183312353378732, \"OBJECTID\": 60, \"Shape_Leng\": 0.0786482878226, \"Shape_Area\": 0.000161833284912, \"zone\": \"Crotona Park East\", \"LocationID\": 60, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87823856799989, 40.839985917999854], [-73.87277204699981, 40.83975128199995], [-73.87379590199993, 40.837445132999896], [-73.88005188499987, 40.83470205899989], [-73.88390094800003, 40.82980281599986], [-73.8843985489999, 40.822965575999866], [-73.88753429599987, 40.82250933899995], [-73.88601262699991, 40.828238530999926], [-73.88761307899992, 40.8293374189999], [-73.89717220999982, 40.830368832999966], [-73.89697220299995, 40.8343452539999], [-73.89408527499982, 40.8362973469999], [-73.88730373200002, 40.83812719099994], [-73.88618432099992, 40.840105043999884], [-73.88200384100001, 40.837443781999895], [-73.880061996, 40.840162234999845], [-73.87823856799989, 40.839985917999854]]]}}, {\"id\": \"60\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22889710182733514, \"OBJECTID\": 61, \"Shape_Leng\": 0.117310808097, \"Shape_Area\": 0.000510772092295, \"zone\": \"Crown Heights North\", \"LocationID\": 61, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92916130699993, 40.68343632399992], [-73.92281374299995, 40.68341442999991], [-73.92164666399992, 40.67886992299994], [-73.92275257199998, 40.667097332999894], [-73.92872204599985, 40.66449556899988], [-73.93698362399992, 40.665535507999955], [-73.93674946999998, 40.66812737099991], [-73.96131877999991, 40.67140667599988], [-73.96007428300001, 40.67487807099994], [-73.96318238899983, 40.675529503999876], [-73.96134655899995, 40.6804635239999], [-73.95529361599996, 40.679203150999825], [-73.9533701749999, 40.680640507999904], [-73.93435066499993, 40.67956340799991], [-73.93500821699992, 40.68276376799985], [-73.92916130699993, 40.68343632399992]]]}}, {\"id\": \"61\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22874497410870423, \"OBJECTID\": 62, \"Shape_Leng\": 0.0822593359838, \"Shape_Area\": 0.000158238169004, \"zone\": \"Crown Heights South\", \"LocationID\": 62, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92916752199997, 40.664299531999895], [-73.94282892199992, 40.6628474859999], [-73.94269628999989, 40.664055128999934], [-73.94546293699983, 40.66422610399989], [-73.96095595499996, 40.66328505499992], [-73.96258784899996, 40.671711596999884], [-73.93674946999998, 40.66812737099991], [-73.93698362399992, 40.665535507999955], [-73.92916752199997, 40.664299531999895]]]}}, {\"id\": \"62\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2276520478649459, \"OBJECTID\": 63, \"Shape_Leng\": 0.119100721883, \"Shape_Area\": 0.000353190910142, \"zone\": \"Cypress Hills\", \"LocationID\": 63, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86650407899994, 40.684474051999885], [-73.86602666799992, 40.681918051999936], [-73.86410096700003, 40.68237284999988], [-73.86234580499983, 40.67916478599994], [-73.86106294999989, 40.67434293499984], [-73.87231018000001, 40.67775574699987], [-73.87531627599998, 40.68011098399988], [-73.88061377799994, 40.679110031999876], [-73.88116730300005, 40.68135366299988], [-73.9000462609999, 40.676893089999936], [-73.9040463979999, 40.67922059799985], [-73.90262262299991, 40.6806645029999], [-73.89786299899986, 40.67987501099987], [-73.89417463299992, 40.6852832479999], [-73.89252316799995, 40.683424532999865], [-73.88962787599998, 40.684236453999894], [-73.87402053199992, 40.69419129499995], [-73.86891704399994, 40.69515042299987], [-73.86650407899994, 40.684474051999885]]]}}, {\"id\": \"65\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20316071145856007, \"OBJECTID\": 66, \"Shape_Leng\": 0.0546334593634, \"Shape_Area\": 0.000108378855948, \"zone\": \"DUMBO/Vinegar Hill\", \"LocationID\": 66, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97906084869959, 40.7059460290057], [-73.98050926199988, 40.69744627499991], [-73.98696892299995, 40.698503613999954], [-73.98679382799996, 40.700633222999954], [-73.99093201799984, 40.7007662159999], [-73.99544564312069, 40.70327530547179], [-73.99350881403605, 40.70462350527397], [-73.97906084869959, 40.7059460290057]]]}}, {\"id\": \"63\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.24992247891334698, \"OBJECTID\": 64, \"Shape_Leng\": 0.18445188474, \"Shape_Area\": 0.00105790284614, \"zone\": \"Douglaston\", \"LocationID\": 64, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.74412314584049, 40.77863865777706], [-73.71351828186042, 40.75983773091619], [-73.72031595299998, 40.75789725799992], [-73.72771356199995, 40.750626894999904], [-73.7310646189999, 40.744888375999885], [-73.74016951799993, 40.73848189799986], [-73.74306820799991, 40.7384408179999], [-73.74396193599988, 40.740789125999925], [-73.75098230100001, 40.740021462999884], [-73.75347299999999, 40.74240484399992], [-73.74477587099986, 40.74328109699994], [-73.74625984399985, 40.74671147499993], [-73.74384322099985, 40.74769009599988], [-73.75004764499985, 40.75840465799996], [-73.75230259399993, 40.759455198999916], [-73.75583713299987, 40.757742061999934], [-73.75921689799992, 40.76432485599988], [-73.75377036482259, 40.76530463174143], [-73.7510743225255, 40.76176367133622], [-73.74455864030779, 40.756557885832535], [-73.75507210849031, 40.76753310678742], [-73.7555733264932, 40.77152912592269], [-73.75336691999294, 40.77320784776336], [-73.75551737581792, 40.77770404836418], [-73.75080593606877, 40.782893378833954], [-73.74412314584049, 40.77863865777706]]]}}, {\"id\": \"64\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20345705416422957, \"OBJECTID\": 65, \"Shape_Leng\": 0.0446070683658, \"Shape_Area\": 8.1803882541e-05, \"zone\": \"Downtown Brooklyn/MetroTech\", \"LocationID\": 65, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98712491499988, 40.70063447999997], [-73.98696892299995, 40.698503613999954], [-73.98050926199988, 40.69744627499991], [-73.97916718399999, 40.69348832299992], [-73.98251122999989, 40.69361433399992], [-73.98147568199991, 40.689930943999876], [-73.99097186899995, 40.69254879999995], [-73.99093201799984, 40.7007662159999], [-73.98712491499988, 40.70063447999997]]]}}, {\"id\": \"66\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.8331543174013877, \"OBJECTID\": 67, \"Shape_Leng\": 0.0997470781551, \"Shape_Area\": 0.000394282272487, \"zone\": \"Dyker Heights\", \"LocationID\": 67, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.0109702639999, 40.63068186899988], [-74.00702302499997, 40.63210999499996], [-73.99724067099987, 40.62528868699994], [-73.99771358999998, 40.62368026999996], [-73.99932090299998, 40.62465524699996], [-74.01701910199995, 40.60765441699994], [-74.01549854999998, 40.606841523999925], [-74.0186913569999, 40.60360792399995], [-74.02129949199998, 40.60442832299987], [-74.02078540899996, 40.60567368599989], [-74.02247769499994, 40.60667421599987], [-74.02150602899994, 40.60782209899992], [-74.02710660599995, 40.61024395599991], [-74.02637609099996, 40.61245060299993], [-74.02754297299998, 40.61293718299988], [-74.01970491599991, 40.6221849649999], [-74.01514630699995, 40.63322193799993], [-74.0109702639999, 40.63068186899988]]]}}, {\"id\": \"67\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20994906302799735, \"OBJECTID\": 68, \"Shape_Leng\": 0.0493373795569, \"Shape_Area\": 0.000111073378655, \"zone\": \"East Chelsea\", \"LocationID\": 68, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00201937799989, 40.74044408899989], [-74.00820401899988, 40.74214751799986], [-73.99730671499998, 40.757110506999894], [-73.99163271299996, 40.75470680499996], [-74.00201937799989, 40.74044408899989]]]}}, {\"id\": \"68\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23014388796730298, \"OBJECTID\": 69, \"Shape_Leng\": 0.0789896450483, \"Shape_Area\": 0.00019905160381, \"zone\": \"East Concourse/Concourse Village\", \"LocationID\": 69, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91179361999997, 40.8264281559999], [-73.91130974499993, 40.824848557999886], [-73.91541108199998, 40.82577390899993], [-73.92518486499998, 40.818012668999884], [-73.92731653099999, 40.81855963399991], [-73.91193751499992, 40.843202387999916], [-73.90514893999989, 40.84254719499995], [-73.91049010699992, 40.830737372999884], [-73.908959903, 40.83022623299989], [-73.91179361999997, 40.8264281559999]]]}}, {\"id\": \"69\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20393045142390617, \"OBJECTID\": 70, \"Shape_Leng\": 0.0638403183367, \"Shape_Area\": 0.000195458476728, \"zone\": \"East Elmhurst\", \"LocationID\": 70, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85912375899987, 40.76127549899994], [-73.85449593299987, 40.75827863899985], [-73.87513874300004, 40.756633815999905], [-73.87608949699984, 40.77153690299987], [-73.868098735, 40.77017200899994], [-73.85912375899987, 40.76127549899994]]]}}, {\"id\": \"70\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.27721745883940074, \"OBJECTID\": 71, \"Shape_Leng\": 0.0931694551828, \"Shape_Area\": 0.000382572587364, \"zone\": \"East Flatbush/Farragut\", \"LocationID\": 71, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93177153199984, 40.65527364199989], [-73.92910623099985, 40.63979510599992], [-73.92574389399985, 40.64001446599985], [-73.92652490899994, 40.63649272299989], [-73.92744822299987, 40.635258409999864], [-73.9372670019998, 40.634650624999885], [-73.93701746200003, 40.63231097899989], [-73.94462885499995, 40.63183773499993], [-73.94537272499998, 40.638786954999865], [-73.94826499699982, 40.63860718999994], [-73.94841343400003, 40.639987463999944], [-73.94778994699976, 40.643305735999924], [-73.94586879199989, 40.64342374399988], [-73.94715896299984, 40.65559378699993], [-73.931907438, 40.656537652999916], [-73.93177153199984, 40.65527364199989]]]}}, {\"id\": \"71\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.3046321399452469, \"OBJECTID\": 72, \"Shape_Leng\": 0.0952020695166, \"Shape_Area\": 0.000323532461947, \"zone\": \"East Flatbush/Remsen Village\", \"LocationID\": 72, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92021313099984, 40.65997544099987], [-73.9195947979998, 40.66199567899985], [-73.91236458899995, 40.65551356499985], [-73.91012255699995, 40.655851770999945], [-73.90855790499995, 40.65209593799989], [-73.91995062999999, 40.64470762999989], [-73.91973736999988, 40.64277814799987], [-73.92332654599986, 40.63990021599991], [-73.92356525899983, 40.642499297999905], [-73.92538233099987, 40.64238453199987], [-73.92574389499987, 40.64001446599983], [-73.92910623199991, 40.6397951059999], [-73.93190743899989, 40.656537652999944], [-73.92799465199988, 40.65680210099988], [-73.92834017599988, 40.660060035999884], [-73.92646424299983, 40.659814361999956], [-73.93068465499985, 40.66362047699992], [-73.92637970499992, 40.6655148919999], [-73.92021313099984, 40.65997544099987]]]}}, {\"id\": \"72\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.24850480224803922, \"OBJECTID\": 73, \"Shape_Leng\": 0.0853020209129, \"Shape_Area\": 0.000291799754395, \"zone\": \"East Flushing\", \"LocationID\": 73, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79566545000003, 40.75023619999991], [-73.79729524799988, 40.74943505299992], [-73.7946756879998, 40.74721192799994], [-73.8033521419999, 40.74643196499987], [-73.80287666499987, 40.749765148999856], [-73.80750599299991, 40.750136586999936], [-73.81416172099998, 40.74613857999986], [-73.820849288, 40.75224154499988], [-73.8184078289999, 40.75383042199998], [-73.82030067699999, 40.75551497999992], [-73.81789245699991, 40.75708633799991], [-73.82063593299985, 40.75887226799989], [-73.81104963199985, 40.761417020999964], [-73.79940088899988, 40.759397218999936], [-73.79493246199993, 40.75779802999994], [-73.79566545000003, 40.75023619999991]]]}}, {\"id\": \"73\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.1997390877776161, \"OBJECTID\": 74, \"Shape_Leng\": 0.11029093625, \"Shape_Area\": 0.000295038418204, \"zone\": \"East Harlem North\", \"LocationID\": 74, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.93380589881859, 40.81651249324708], [-73.93412073484292, 40.80865645197743], [-73.92903490240305, 40.80108090385195], [-73.9290364017962, 40.79676259410132], [-73.93505300002766, 40.79168720295447], [-73.93773612099982, 40.792815803999936], [-73.93864883299987, 40.79155775499987], [-73.94327251199996, 40.79348810299993], [-73.94187604699994, 40.79540586899984], [-73.94831125500004, 40.79812921799992], [-73.94459751299998, 40.803228144999906], [-73.94613131, 40.80387735799992], [-73.94430194599991, 40.806390824999916], [-73.93854407899988, 40.80572965299991], [-73.93580780199986, 40.80949763799987], [-73.93901753299998, 40.81085655699991], [-73.93407463999996, 40.81782651399992], [-73.93440039999986, 40.81966778299989], [-73.93380589881859, 40.81651249324708]]], [[[-73.92688606022001, 40.800840648368855], [-73.92650954883642, 40.80065703126224], [-73.92704640297914, 40.800651743031324], [-73.92688606022001, 40.800840648368855]]], [[[-73.92666565490181, 40.80018271401784], [-73.92669572614771, 40.80014697707624], [-73.9272477078927, 40.800392957368906], [-73.92711464230626, 40.80056375622957], [-73.92666565490181, 40.80018271401784]]], [[[-73.92672091168967, 40.800414210943586], [-73.92653846445016, 40.80033389815394], [-73.92674263892977, 40.800403498972756], [-73.92672091168967, 40.800414210943586]]], [[[-73.92718514789182, 40.797349896890104], [-73.92767122620857, 40.79721701426623], [-73.92770694127064, 40.7972986638821], [-73.92738263624496, 40.797377063580456], [-73.92718514789182, 40.797349896890104]]]]}}, {\"id\": \"74\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.19871991798898933, \"OBJECTID\": 75, \"Shape_Leng\": 0.0876638997685, \"Shape_Area\": 0.00024056392715, \"zone\": \"East Harlem South\", \"LocationID\": 75, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94103496899987, 40.79254666899993], [-73.93505300002766, 40.79168720295447], [-73.93708104382307, 40.78936639571], [-73.93621859869943, 40.78880049757431], [-73.94354420706223, 40.78288052416259], [-73.9557773589999, 40.78791392399995], [-73.94831125500004, 40.79812921799992], [-73.94187604699994, 40.79540586899984], [-73.94327251199996, 40.79348810299993], [-73.94103496899987, 40.79254666899993]]]}}, {\"id\": \"75\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.24959927966792073, \"OBJECTID\": 76, \"Shape_Leng\": 0.241203016269, \"Shape_Area\": 0.00126660143241, \"zone\": \"East New York\", \"LocationID\": 76, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88063900599983, 40.679176984999856], [-73.87507136399991, 40.68003514599991], [-73.87231018000001, 40.67775574699987], [-73.86106294999989, 40.67434293499984], [-73.86038937900003, 40.67126877499985], [-73.85763323199998, 40.67165619399985], [-73.85568461199988, 40.66386749199993], [-73.85842950899992, 40.66345335999987], [-73.85761052799985, 40.660108355999924], [-73.86317083299996, 40.65827651199994], [-73.86327623068586, 40.656941215328104], [-73.85695870295893, 40.65058737348713], [-73.85840315341014, 40.64700390123735], [-73.85716978298207, 40.64367855888597], [-73.86289463224384, 40.64255067968245], [-73.86670411973783, 40.6400217634955], [-73.86563247391335, 40.63878676362291], [-73.86867282895084, 40.640925079203356], [-73.87061492461066, 40.64485674188673], [-73.87971979034293, 40.65437592977636], [-73.86989583386247, 40.63893452891954], [-73.86799709668125, 40.63811526520958], [-73.86960972136559, 40.63708553460871], [-73.87347939154502, 40.63628554579879], [-73.87841589406925, 40.638912189353306], [-73.8745808109999, 40.64561010599992], [-73.88221364899994, 40.65451863799996], [-73.89000589199988, 40.651148696999925], [-73.89354677099988, 40.65520271399988], [-73.89761013299987, 40.65345381599992], [-73.89586154900002, 40.65188052699987], [-73.90398200599991, 40.64662856799988], [-73.90944846399994, 40.65152159899992], [-73.90740035699999, 40.6528439899999], [-73.90784346399994, 40.65459152999988], [-73.89612587299993, 40.657814663999936], [-73.88185557299985, 40.66414998799994], [-73.88330883399975, 40.66611200499991], [-73.889906696, 40.66377828999986], [-73.89087805100002, 40.671318086999875], [-73.89271090700004, 40.67104582799992], [-73.89367036399987, 40.67482177099988], [-73.89930675399987, 40.67398204199991], [-73.90004626199993, 40.67689308999995], [-73.88116730300005, 40.68135366299988], [-73.88063900599983, 40.679176984999856]]]}}, {\"id\": \"76\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.33051263613883086, \"OBJECTID\": 77, \"Shape_Leng\": 0.075461081444, \"Shape_Area\": 0.000191917981315, \"zone\": \"East New York/Pennsylvania Avenue\", \"LocationID\": 77, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8997089949999, 40.67549425099991], [-73.89930675399987, 40.67398204199991], [-73.89367036399987, 40.67482177099988], [-73.89271090699994, 40.671045828999915], [-73.89087805100002, 40.671318086999875], [-73.889906696, 40.66377828999986], [-73.88330883399975, 40.66611200499991], [-73.88185557299985, 40.66414998799994], [-73.89880698499995, 40.65740666199988], [-73.90066118699986, 40.66058615799992], [-73.90347422300006, 40.67550668299985], [-73.90004626199993, 40.67689308999995], [-73.8997089949999, 40.67549425099991]]]}}, {\"id\": \"77\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21610668061184468, \"OBJECTID\": 78, \"Shape_Leng\": 0.0935944495806, \"Shape_Area\": 0.000191114419551, \"zone\": \"East Tremont\", \"LocationID\": 78, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88963328299991, 40.85317381799993], [-73.88987871399985, 40.85207022099993], [-73.8773292409999, 40.84398312499995], [-73.878171756, 40.84262443699987], [-73.87459175399982, 40.841520647999914], [-73.87109207899995, 40.84432478099992], [-73.86920944599997, 40.84923837199993], [-73.86785992999985, 40.84886493099987], [-73.86804200399997, 40.8448138389999], [-73.87359713599987, 40.83979855499992], [-73.880061996, 40.840162234999845], [-73.88200384100001, 40.837443781999895], [-73.88618432099993, 40.8401050449999], [-73.88941724899988, 40.83942273399985], [-73.89537836099996, 40.842938778999944], [-73.89290102299996, 40.84626832999992], [-73.89617712999998, 40.8466583209999], [-73.89307140900002, 40.85324998699989], [-73.89038953999983, 40.85468905799996], [-73.88963328299991, 40.85317381799993]]]}}, {\"id\": \"78\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20790510405693716, \"OBJECTID\": 79, \"Shape_Leng\": 0.0426249113144, \"Shape_Area\": 0.000107893068218, \"zone\": \"East Village\", \"LocationID\": 79, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98377990999991, 40.72153595399992], [-73.99260322199994, 40.72413644999987], [-73.98990295999991, 40.7344347899999], [-73.97802697999985, 40.729433059999984], [-73.98377990999991, 40.72153595399992]]]}}, {\"id\": \"79\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2205123190575919, \"OBJECTID\": 80, \"Shape_Leng\": 0.117212621448, \"Shape_Area\": 0.00040732245622, \"zone\": \"East Williamsburg\", \"LocationID\": 80, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93174841322003, 40.728053727696675], [-73.92849418540158, 40.72661460330786], [-73.92453854985803, 40.719342580204575], [-73.93220607307306, 40.71496398092936], [-73.93124067059496, 40.71372481088467], [-73.93328349320237, 40.71097870745788], [-73.9319010192085, 40.71133847142828], [-73.9303609456579, 40.7087177385352], [-73.93184868877432, 40.71256477418327], [-73.93042294313065, 40.71317255867104], [-73.93105231901723, 40.71461022392334], [-73.92460339476072, 40.717953306073376], [-73.92305500014334, 40.71634266479336], [-73.92488553507611, 40.71526718765259], [-73.92433589757124, 40.71412259346647], [-73.92074519699985, 40.71052968599989], [-73.93229708600003, 40.70774944699988], [-73.93480281900003, 40.713372495999884], [-73.94378799599994, 40.710900378999945], [-73.94273448399997, 40.70447330599988], [-73.9408928719998, 40.70391415699989], [-73.94042057899993, 40.701076597999865], [-73.94193078899983, 40.70072523399989], [-73.95023693799996, 40.705473245999876], [-73.95087492699996, 40.70951311499991], [-73.94865533499987, 40.70972498299988], [-73.94937231899985, 40.71406687799986], [-73.95348472199983, 40.71407020799988], [-73.93174841322003, 40.728053727696675]]]}}, {\"id\": \"80\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2818767298453197, \"OBJECTID\": 81, \"Shape_Leng\": 0.197592771731, \"Shape_Area\": 0.00039956655019, \"zone\": \"Eastchester\", \"LocationID\": 81, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8236684811033, 40.889903740147645], [-73.82285335053221, 40.8912068290476], [-73.79300652624502, 40.88335873744268], [-73.81209659199986, 40.88671500999993], [-73.82027432999985, 40.885550699999925], [-73.82387140599998, 40.88778362799996], [-73.84324113799987, 40.879108912999904], [-73.8381484219999, 40.876953298999915], [-73.83924788999988, 40.876618737999905], [-73.83829221499984, 40.87295133899995], [-73.83565766499987, 40.86751275599992], [-73.83315960399985, 40.86831975599994], [-73.82834347399992, 40.86089202599988], [-73.85363638799987, 40.87330059899996], [-73.8506895569999, 40.87548667799993], [-73.85254854099988, 40.877091586999924], [-73.85091896099982, 40.877893440999856], [-73.85248305800003, 40.87946370499989], [-73.84704957999995, 40.8876718159999], [-73.83554853999995, 40.88913350399993], [-73.83737956161106, 40.893924741304346], [-73.8236684811033, 40.889903740147645]]]}}, {\"id\": \"81\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20500494016299856, \"OBJECTID\": 82, \"Shape_Leng\": 0.119875649697, \"Shape_Area\": 0.000323601079994, \"zone\": \"Elmhurst\", \"LocationID\": 82, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86768504899995, 40.744071717999915], [-73.86447086199989, 40.73400727399998], [-73.88720524999994, 40.727792996999874], [-73.8885228599999, 40.734531159999904], [-73.87752195799992, 40.731593168999915], [-73.87708276299998, 40.73290931699991], [-73.88066188999987, 40.73570787399994], [-73.88033059399996, 40.73733367699989], [-73.88230252700001, 40.737707700999934], [-73.87743058400002, 40.73860051599991], [-73.88100387999987, 40.74184066199995], [-73.88551225099985, 40.74016501899988], [-73.887511321, 40.74108252699992], [-73.88798650799986, 40.743548191999935], [-73.89071647499993, 40.74312778399992], [-73.89175048799991, 40.7468117269999], [-73.86942458599987, 40.74915685199991], [-73.86768504899995, 40.744071717999915]]]}}, {\"id\": \"82\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20252139285873935, \"OBJECTID\": 83, \"Shape_Leng\": 0.105984933269, \"Shape_Area\": 0.000217463718718, \"zone\": \"Elmhurst/Maspeth\", \"LocationID\": 83, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89111026199994, 40.745202717999874], [-73.89071647499993, 40.74312778399992], [-73.88798650799986, 40.743548191999935], [-73.887511321, 40.74108252699992], [-73.88551225099985, 40.74016501899988], [-73.88100387999987, 40.74184066199995], [-73.87791831799989, 40.739394956999874], [-73.88230252700001, 40.737707700999934], [-73.88033059399996, 40.73733367699989], [-73.88066188999987, 40.73570787399994], [-73.87725122499988, 40.7317941529999], [-73.88910318999984, 40.73459239599987], [-73.89854772799987, 40.735191909999855], [-73.90107970900003, 40.73391463399987], [-73.90516948899983, 40.73585964299986], [-73.90628903499987, 40.732400751999876], [-73.91068876399997, 40.733471081999916], [-73.90926002999988, 40.736403179999854], [-73.90057825399985, 40.73949289599988], [-73.89524819599988, 40.74381972999992], [-73.89624245599988, 40.74871954499987], [-73.89139145499985, 40.74664100399984], [-73.89111026199994, 40.745202717999874]]]}}, {\"id\": \"83\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.39373205563220104, \"OBJECTID\": 84, \"Shape_Leng\": 0.233623987032, \"Shape_Area\": 0.00207375572052, \"zone\": \"Eltingville/Annadale/Prince's Bay\", \"LocationID\": 84, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.16005732699996, 40.527635041999886], [-74.1720632017547, 40.52332056936472], [-74.17756306042942, 40.519217618159864], [-74.18104081208887, 40.520705036287154], [-74.1848311647976, 40.519341358600826], [-74.19521339999068, 40.5099776529556], [-74.19967205357709, 40.511426850497486], [-74.19936127407836, 40.51301019443217], [-74.20821982874101, 40.51161821258057], [-74.21754755095681, 40.50336004949062], [-74.22153919955196, 40.50250347421612], [-74.22690993399992, 40.51917139799985], [-74.22379595899992, 40.518196345999876], [-74.21354593499994, 40.521235714999925], [-74.21409806799993, 40.52409719499991], [-74.20152661599995, 40.526265205999934], [-74.19174368799999, 40.53191669699986], [-74.19377169299999, 40.53583384699993], [-74.18903145199995, 40.53815934199989], [-74.18128532999992, 40.54695757299989], [-74.17185166499996, 40.56152092299991], [-74.16982582400001, 40.56109042099991], [-74.16540080799992, 40.54443272299989], [-74.15397419799994, 40.53426814799993], [-74.16134961699994, 40.5293734239999], [-74.16005732699996, 40.527635041999886]]]}}, {\"id\": \"84\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23689531114725296, \"OBJECTID\": 85, \"Shape_Leng\": 0.0574751473562, \"Shape_Area\": 0.000143637804933, \"zone\": \"Erasmus\", \"LocationID\": 85, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94715896299984, 40.65559378699993], [-73.94586879199989, 40.64342374399988], [-73.94778994699976, 40.643305735999924], [-73.94826499600005, 40.63860718999994], [-73.9536000469999, 40.63842234699991], [-73.95763644599988, 40.64268949899989], [-73.9587078989999, 40.65038727299989], [-73.94957113100006, 40.65080789999991], [-73.9500666489999, 40.65542299699989], [-73.94715896299984, 40.65559378699993]]]}}, {\"id\": \"85\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.3207894328727415, \"OBJECTID\": 86, \"Shape_Leng\": 0.134245282582, \"Shape_Area\": 0.000623278815249, \"zone\": \"Far Rockaway\", \"LocationID\": 86, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76669071688973, 40.61422662157113], [-73.76505961679173, 40.612035230610466], [-73.76023635756965, 40.610820864950284], [-73.76178242960594, 40.60915167005445], [-73.76155051905889, 40.60903136580077], [-73.75998446974822, 40.61087937784668], [-73.7556083701559, 40.61007791963748], [-73.7457615018933, 40.611991654374336], [-73.73815143112287, 40.60271044008338], [-73.73808925304178, 40.59768829704553], [-73.74183083531219, 40.59669555145679], [-73.73812220887619, 40.597386102069386], [-73.73763679389783, 40.594415399837956], [-73.74664920374843, 40.594280895008744], [-73.75352988157324, 40.59094648500059], [-73.76376223423699, 40.591439709016804], [-73.76440183099987, 40.59457168299989], [-73.76277845599988, 40.59475654399989], [-73.7633344969999, 40.59782287399992], [-73.76128753399995, 40.59954927099993], [-73.76654204999997, 40.599434441999904], [-73.76771811799993, 40.596261309999925], [-73.76795043655434, 40.5981585114185], [-73.77098168371995, 40.59889409104954], [-73.76895714023856, 40.60927991805636], [-73.77417180232338, 40.61182871527586], [-73.77339803484179, 40.61368296748421], [-73.76669071688973, 40.61422662157113]]]}}, {\"id\": \"86\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.19877920535763455, \"OBJECTID\": 87, \"Shape_Leng\": 0.0369015496117, \"Shape_Area\": 6.72050210692e-05, \"zone\": \"Financial District North\", \"LocationID\": 87, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00372234899996, 40.70693234399989], [-74.00197014837809, 40.704731159202304], [-74.00332307270776, 40.70562859522001], [-74.00459003883662, 40.70478775294495], [-74.00349632042315, 40.70379676926777], [-74.0053502722475, 40.70431007996148], [-74.004272587521, 40.70301566641148], [-74.00659591913875, 40.70368590245183], [-74.00530412596079, 40.70255986563306], [-74.00550166067457, 40.70243086779703], [-74.01250773999998, 40.706767065999934], [-74.00906264799988, 40.71088670399992], [-74.00372234899996, 40.70693234399989]]]}}, {\"id\": \"87\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20630995600678215, \"OBJECTID\": 88, \"Shape_Leng\": 0.0352046035327, \"Shape_Area\": 5.72999455414e-05, \"zone\": \"Financial District South\", \"LocationID\": 88, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00974639299997, 40.704775247999905], [-74.00760498189766, 40.70299314132918], [-74.00930907655483, 40.70195460302249], [-74.00759136204225, 40.70132808104815], [-74.01367479307396, 40.70012582122537], [-74.01428922899998, 40.70454907499989], [-74.01250773999998, 40.706767065999934], [-74.00974639299997, 40.704775247999905]]]}}, {\"id\": \"88\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2310376392540709, \"OBJECTID\": 89, \"Shape_Leng\": 0.122794569553, \"Shape_Area\": 0.000447548142373, \"zone\": \"Flatbush/Ditmas Park\", \"LocationID\": 89, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95374853799996, 40.63858633799993], [-73.94537272499998, 40.638786954999865], [-73.94462885499995, 40.63183773499993], [-73.94647961799998, 40.631708161999896], [-73.94415535299993, 40.629508284999886], [-73.95995565099992, 40.627778671999884], [-73.96040734299989, 40.6301625309999], [-73.96232599399991, 40.6299687169999], [-73.97136622199992, 40.62892916199996], [-73.97092126300004, 40.62656998499993], [-73.97623453700001, 40.625976349999895], [-73.97716511999984, 40.63074665399988], [-73.96986869299992, 40.63419467499989], [-73.97029036199991, 40.63640634099988], [-73.96803868499985, 40.636650984999925], [-73.97139631700004, 40.64825778599986], [-73.96465438699984, 40.650887727999894], [-73.96621321999996, 40.653189404999885], [-73.95986103199981, 40.65563478899989], [-73.95763644599988, 40.64268949899989], [-73.95374853799996, 40.63858633799993]]]}}, {\"id\": \"89\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2116001251654537, \"OBJECTID\": 90, \"Shape_Leng\": 0.0307591620819, \"Shape_Area\": 5.53132476305e-05, \"zone\": \"Flatiron\", \"LocationID\": 90, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9963368949999, 40.7380437469999], [-74.00252381499989, 40.73975264399995], [-73.99709902899994, 40.74720510199991], [-73.99141858599994, 40.74480268199988], [-73.9963368949999, 40.7380437469999]]]}}, {\"id\": \"90\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22113815404518197, \"OBJECTID\": 91, \"Shape_Leng\": 0.124996624585, \"Shape_Area\": 0.000537330013243, \"zone\": \"Flatlands\", \"LocationID\": 91, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91945472999996, 40.6401731369999], [-73.91690302699992, 40.61627422899993], [-73.9205269409999, 40.61394269499987], [-73.92640044599996, 40.61447853499988], [-73.92629604599998, 40.61311679899991], [-73.93686699599992, 40.620253280999904], [-73.94573691999982, 40.61536340699989], [-73.94651373199997, 40.61949401899995], [-73.94451833400001, 40.61971364699989], [-73.94732672199994, 40.629166566999835], [-73.94415535299993, 40.629508284999886], [-73.94647961799998, 40.631708161999896], [-73.93701746200003, 40.63231097899989], [-73.9372670019998, 40.634650624999885], [-73.92744822299987, 40.635258409999864], [-73.92538233099987, 40.64238453199987], [-73.92356525899983, 40.642499297999905], [-73.92332654599986, 40.63990021599991], [-73.91973736999988, 40.64277814799987], [-73.91945472999996, 40.6401731369999]]]}}, {\"id\": \"91\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21449369418026523, \"OBJECTID\": 92, \"Shape_Leng\": 0.117830066799, \"Shape_Area\": 0.000374946617289, \"zone\": \"Flushing\", \"LocationID\": 92, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82656578899986, 40.77064147599989], [-73.82466908999987, 40.77040760799985], [-73.82080684899987, 40.7589417829999], [-73.81789245699991, 40.75708633799991], [-73.82030067699999, 40.75551497999992], [-73.8184078289999, 40.75383042199998], [-73.820849288, 40.75224154499988], [-73.81575069499985, 40.7469794799999], [-73.82108427599998, 40.748558445999954], [-73.82426493900002, 40.75183288899994], [-73.82584440999999, 40.74934839099993], [-73.83300302099987, 40.74783363099988], [-73.8322953569999, 40.751941790999844], [-73.83560230899995, 40.75139485299991], [-73.83805974199996, 40.75466660699988], [-73.83689970199995, 40.7626981179999], [-73.8394581342306, 40.76544576822929], [-73.83000011099983, 40.77373240699996], [-73.82513740299993, 40.78169970599994], [-73.82224127699999, 40.781693878999924], [-73.82583089999997, 40.77591527099986], [-73.82656578899986, 40.77064147599989]]]}}, {\"id\": \"92\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20875280230972412, \"OBJECTID\": 93, \"Shape_Leng\": 0.170998027597, \"Shape_Area\": 0.000594278183978, \"zone\": \"Flushing Meadows-Corona Park\", \"LocationID\": 93, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8569635593979, 40.76407442786308], [-73.8566748236235, 40.763930188326135], [-73.85758285470693, 40.76307290062661], [-73.8565867538878, 40.762722840114236], [-73.85817631070509, 40.7623962942238], [-73.85661195813331, 40.761649542759294], [-73.8585852628444, 40.76191700859695], [-73.85747662435531, 40.76130891668733], [-73.85158552901191, 40.75935542065696], [-73.84930110814332, 40.76024287534997], [-73.85207200177854, 40.7603928846721], [-73.84670933714987, 40.76134526931977], [-73.84324381399976, 40.75533680399995], [-73.83867321899987, 40.75701997899995], [-73.83560230899995, 40.75139485299991], [-73.8322953569999, 40.751941790999844], [-73.83300302099987, 40.74783363099988], [-73.83699037399995, 40.74209330099992], [-73.8361277859999, 40.734241311999924], [-73.83073214599987, 40.72654932799991], [-73.8273090039999, 40.72464607699988], [-73.82531644999993, 40.71727569999994], [-73.81729568999994, 40.71757964499995], [-73.83145866999996, 40.714554448999934], [-73.8297961659998, 40.71556571899986], [-73.83488359399998, 40.719303271999934], [-73.85436928099995, 40.748738929999874], [-73.85086439699992, 40.74999564199986], [-73.85379813799992, 40.757177611999914], [-73.86109724399991, 40.76366447799989], [-73.85861915281858, 40.7619355969989], [-73.8569635593979, 40.76407442786308]]]}}, {\"id\": \"93\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2623495634389995, \"OBJECTID\": 94, \"Shape_Leng\": 0.0498326083147, \"Shape_Area\": 6.25819407393e-05, \"zone\": \"Fordham South\", \"LocationID\": 94, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89964261700001, 40.862210436999874], [-73.897143798, 40.86244506499991], [-73.89883814599983, 40.859307307999906], [-73.89573924699978, 40.85813820699986], [-73.89106059999996, 40.861471814999945], [-73.89560883299981, 40.85457036999986], [-73.89863543699985, 40.85390697499993], [-73.89816577499982, 40.85514637599989], [-73.90077922699986, 40.85604710999993], [-73.90185996099993, 40.85458614899994], [-73.90557007299988, 40.855787321999884], [-73.90351390599986, 40.86091750999989], [-73.89964261700001, 40.862210436999874]]]}}, {\"id\": \"94\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20937220078946203, \"OBJECTID\": 95, \"Shape_Leng\": 0.113629605476, \"Shape_Area\": 0.00057277033803, \"zone\": \"Forest Hills\", \"LocationID\": 95, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84732494199989, 40.738771457999896], [-73.83488359399998, 40.719303271999934], [-73.8297961659998, 40.71556571899986], [-73.8377089419999, 40.71103648399989], [-73.85593991699984, 40.7063980339999], [-73.8586809759999, 40.72261243999988], [-73.86203833799982, 40.72595678499994], [-73.85805416899993, 40.72444540599986], [-73.85727108100001, 40.72568129799988], [-73.8594198799999, 40.726464837999885], [-73.85741240699991, 40.72832247999986], [-73.85851097899999, 40.73028680199993], [-73.85625665099991, 40.73095912299993], [-73.85872465799989, 40.73570864399988], [-73.84732494199989, 40.738771457999896]]]}}, {\"id\": \"95\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.25047847935328815, \"OBJECTID\": 96, \"Shape_Leng\": 0.185180248095, \"Shape_Area\": 0.000547566253402, \"zone\": \"Forest Park/Highland Park\", \"LocationID\": 96, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83804164799996, 40.70832329499994], [-73.83696237799982, 40.70421245199992], [-73.84502472899999, 40.6993593769999], [-73.84913837299978, 40.69828274099995], [-73.85070842300001, 40.70020312499988], [-73.85443455699989, 40.70060295199994], [-73.85277238699999, 40.69755711199987], [-73.874020533, 40.69419129499994], [-73.88962787599998, 40.684236453999894], [-73.89252316899987, 40.683424532999865], [-73.89417463299992, 40.6852832479999], [-73.89646625099995, 40.68233642199988], [-73.90116155099986, 40.687877934999946], [-73.90180467199995, 40.6907662979999], [-73.9003866789998, 40.69313995999992], [-73.89736265799978, 40.6941108469999], [-73.89695653499987, 40.6919492069999], [-73.89326209899984, 40.69220094699994], [-73.8950769439999, 40.694591389999886], [-73.88642048199995, 40.696458148999945], [-73.88771297999986, 40.699678075999906], [-73.88280948499987, 40.7006834549999], [-73.88182690599986, 40.69952929899987], [-73.87639785199997, 40.70229404199989], [-73.86285374199994, 40.7034144919999], [-73.85819890699996, 40.70291107399986], [-73.85893224399979, 40.70132034499989], [-73.85725351500002, 40.70121217699989], [-73.85252019099998, 40.70229172399997], [-73.85473240499985, 40.70676414299989], [-73.84314625299982, 40.70922345599992], [-73.83804164799996, 40.70832329499994]]]}}, {\"id\": \"96\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.212340176461046, \"OBJECTID\": 97, \"Shape_Leng\": 0.0624760147423, \"Shape_Area\": 0.000163303970435, \"zone\": \"Fort Greene\", \"LocationID\": 97, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9693409769999, 40.69576842999993], [-73.96910362499993, 40.68946752199986], [-73.97013154199992, 40.68950630799986], [-73.96868961400001, 40.68200525799988], [-73.97745722099988, 40.68405228699988], [-73.98092822899991, 40.68890247699988], [-73.98372011099993, 40.69000671899992], [-73.98147568199991, 40.689930943999876], [-73.98251122999989, 40.69361433399992], [-73.97916718399999, 40.69348832299992], [-73.98054419199988, 40.69824651199988], [-73.9697278989999, 40.69780401399986], [-73.9693409769999, 40.69576842999993]]]}}, {\"id\": \"97\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20503104303252664, \"OBJECTID\": 98, \"Shape_Leng\": 0.121661018278, \"Shape_Area\": 0.000485773020954, \"zone\": \"Fresh Meadows\", \"LocationID\": 98, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76360860199983, 40.73917476499987], [-73.76273303499985, 40.73526216299995], [-73.76510920699994, 40.73455658499992], [-73.7636997919999, 40.73286411699993], [-73.76564374500005, 40.732533126999954], [-73.75669771299987, 40.7262287109999], [-73.76924074899993, 40.72590270099988], [-73.77727808000003, 40.72234633599988], [-73.77803765999981, 40.723284268999855], [-73.7752338719998, 40.725237532999884], [-73.77757506899988, 40.73019327299987], [-73.79174472699992, 40.725788849999944], [-73.79651029099988, 40.7379086369999], [-73.76772991399994, 40.74577622599993], [-73.76360860199983, 40.73917476499987]]]}}, {\"id\": \"98\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2975, \"OBJECTID\": 99, \"Shape_Leng\": 0.1833714893, \"Shape_Area\": 0.00121016463877, \"zone\": \"Freshkills Park\", \"LocationID\": 99, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.16842916199994, 40.58649792299988], [-74.16982582400001, 40.56109042099991], [-74.18295815999996, 40.56548019999992], [-74.193410946, 40.56248503899994], [-74.20091501299991, 40.55600525999987], [-74.20699885299997, 40.55756748899989], [-74.2056960519999, 40.55851600599987], [-74.21324031454323, 40.55667262386906], [-74.20596154297344, 40.580046993882775], [-74.20048848799996, 40.57994977799989], [-74.19964981299998, 40.58226262599991], [-74.19423261299991, 40.58514680599988], [-74.185371048, 40.58745101499987], [-74.18730248699987, 40.588488221999896], [-74.17488959399996, 40.602216572999914], [-74.16719634899995, 40.602075311999926], [-74.16886362299992, 40.59472808699995], [-74.16350785499993, 40.59327217999993], [-74.16842916199994, 40.58649792299988]]]}}, {\"id\": \"99\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21675614089247308, \"OBJECTID\": 100, \"Shape_Leng\": 0.0248131090342, \"Shape_Area\": 3.74700210291e-05, \"zone\": \"Garment District\", \"LocationID\": 100, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98729377099981, 40.75045160899988], [-73.99346417699986, 40.75219005499987], [-73.98979085399996, 40.7572331319999], [-73.98411754799999, 40.75484205299995], [-73.98729377099981, 40.75045160899988]]]}}, {\"id\": \"100\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.1935076788238434, \"OBJECTID\": 101, \"Shape_Leng\": 0.101709836277, \"Shape_Area\": 0.000452342528877, \"zone\": \"Glen Oaks\", \"LocationID\": 101, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.70134715908382, 40.750780580935846], [-73.70002020501624, 40.73923654186594], [-73.70766217302989, 40.727830934754685], [-73.71049995381028, 40.72722708720663], [-73.71147804899985, 40.73250140099988], [-73.72010148299994, 40.74977568599991], [-73.727664654, 40.749654641999946], [-73.728962516, 40.74763288799986], [-73.72737889099987, 40.751029566999854], [-73.7206331939999, 40.75761683499996], [-73.71612528199985, 40.75966071399991], [-73.71244940715339, 40.75924170410932], [-73.70163345846068, 40.75249332984642], [-73.70134715908382, 40.750780580935846]]]}}, {\"id\": \"101\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2244166676757176, \"OBJECTID\": 102, \"Shape_Leng\": 0.136900484646, \"Shape_Area\": 0.000296595466345, \"zone\": \"Glendale\", \"LocationID\": 102, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85595536199988, 40.70642977899992], [-73.85473240499985, 40.70676414299989], [-73.85252019099998, 40.70229172399997], [-73.85827405799986, 40.70122365899992], [-73.85819890699996, 40.70291107399986], [-73.86285374199994, 40.7034144919999], [-73.87639785199997, 40.70229404199989], [-73.88182690599986, 40.69952929899987], [-73.88280948499988, 40.70068345599986], [-73.88771297999986, 40.699678075999906], [-73.88642048199995, 40.696458148999945], [-73.8950769439999, 40.694591389999886], [-73.89326209899984, 40.69220094699994], [-73.89695653499987, 40.6919492069999], [-73.89736265799978, 40.6941108469999], [-73.900030392, 40.693437450999895], [-73.88911917799989, 40.705598876999915], [-73.86911007199988, 40.707089336999914], [-73.87016462499984, 40.70920774299987], [-73.85965416799995, 40.713379960999866], [-73.85975932900001, 40.711845058999856], [-73.85729287299985, 40.711467676999874], [-73.85595536199988, 40.70642977899992]]]}}, {\"id\": \"102\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.19578838174273858, \"OBJECTID\": 103, \"Shape_Leng\": 0.0143055167343, \"Shape_Area\": 6.33056361314e-06, \"zone\": \"Governor's Island/Ellis Island/Liberty Island\", \"LocationID\": 103, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.04388559600675, 40.690184824096946], [-74.04350596056258, 40.689687359818144], [-74.04270428413375, 40.69015520482211], [-74.04255372018648, 40.68996275914475], [-74.04438521705613, 40.68851617820147], [-74.04772962698515, 40.689915318233105], [-74.04614707204328, 40.69112264601653], [-74.04388559600675, 40.690184824096946]]]}}, {\"id\": \"103\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.19578838174273858, \"OBJECTID\": 104, \"Shape_Leng\": 0.0212208330928, \"Shape_Area\": 1.19205339715e-05, \"zone\": \"Governor's Island/Ellis Island/Liberty Island\", \"LocationID\": 103, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.03995040794246, 40.70089063032128], [-74.03771124798313, 40.69934404017382], [-74.03934037685066, 40.698115514418], [-74.04124261816106, 40.69953674121738], [-74.0399124888631, 40.69770204019913], [-74.0416605190741, 40.696452971319864], [-74.04367371215382, 40.69802040416498], [-74.03995040794246, 40.70089063032128]]]}}, {\"id\": \"104\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.19578838174273858, \"OBJECTID\": 105, \"Shape_Leng\": 0.0774253398314, \"Shape_Area\": 0.000368636392805, \"zone\": \"Governor's Island/Ellis Island/Liberty Island\", \"LocationID\": 103, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01674756096064, 40.693343368217555], [-74.0118118564723, 40.69247933506435], [-74.01333264149613, 40.69201997010238], [-74.01182575354963, 40.69106481621614], [-74.01316926402752, 40.6881547712032], [-74.02259359581036, 40.68435969594992], [-74.02213250186877, 40.683767672385194], [-74.02305574749599, 40.6829169454452], [-74.0227589902996, 40.68428444833225], [-74.026334042424, 40.6848016977242], [-74.01971454942077, 40.69313300682718], [-74.01674756096064, 40.693343368217555]]]}}, {\"id\": \"105\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22341501041707898, \"OBJECTID\": 106, \"Shape_Leng\": 0.0492110119798, \"Shape_Area\": 0.000113616826567, \"zone\": \"Gowanus\", \"LocationID\": 106, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98610388199985, 40.68100852199994], [-73.98175642299985, 40.67931462399987], [-73.99277080699981, 40.665507280999876], [-73.9986188799999, 40.67183017199989], [-73.99506451099984, 40.67584047499984], [-73.99069861899994, 40.67580741899994], [-73.9876680399999, 40.68081780999993], [-73.98610388199985, 40.68100852199994]]]}}, {\"id\": \"106\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20887280634012714, \"OBJECTID\": 107, \"Shape_Leng\": 0.0380413645908, \"Shape_Area\": 7.46112192675e-05, \"zone\": \"Gramercy\", \"LocationID\": 107, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98238628799993, 40.73141124499993], [-73.98990295999991, 40.7344347899999], [-73.98407485299995, 40.74332471799989], [-73.98085965399994, 40.74196976799989], [-73.98266497499989, 40.73949733299987], [-73.97805226499996, 40.7375539539999], [-73.98238628799993, 40.73141124499993]]]}}, {\"id\": \"107\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.35813379122664674, \"OBJECTID\": 108, \"Shape_Leng\": 0.11117141227, \"Shape_Area\": 0.000352866009465, \"zone\": \"Gravesend\", \"LocationID\": 108, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97148350899987, 40.59717158499996], [-73.97115101799994, 40.59044244399989], [-73.96514385199995, 40.5911019159999], [-73.96661531499983, 40.58457103699992], [-73.97742525299988, 40.58290659299994], [-73.98710120299997, 40.58464459199988], [-73.98654457099988, 40.583658590999946], [-73.98829948299982, 40.58346674699986], [-73.98995504960881, 40.58066166976562], [-73.99185535656628, 40.5816239785461], [-73.99026244714351, 40.5846191482053], [-73.99547188560318, 40.58221005511802], [-74.0003088081067, 40.58317703751469], [-73.99808523776004, 40.585364721581485], [-73.99956511570304, 40.5862439200957], [-73.9940790462178, 40.58869344199059], [-74.00038207107852, 40.58713305717468], [-73.98962150099992, 40.59098199299996], [-73.98601936799993, 40.594446048999906], [-73.97995443499985, 40.594073811999884], [-73.98035785499995, 40.59619154099992], [-73.97148350899987, 40.59717158499996]]]}}, {\"id\": \"108\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2538771539744302, \"OBJECTID\": 109, \"Shape_Leng\": 0.178267819599, \"Shape_Area\": 0.00116960076185, \"zone\": \"Great Kills\", \"LocationID\": 109, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.14706472699994, 40.5705086149999], [-74.13170009, 40.56453574899991], [-74.13885092699991, 40.5569511789999], [-74.13316825999992, 40.552356178999936], [-74.13861798499994, 40.548492384999875], [-74.13585712583111, 40.546451696357344], [-74.1371798712987, 40.5457928646606], [-74.13611540724054, 40.54501984567978], [-74.13794966050746, 40.54600033758068], [-74.13886951340419, 40.545130197607286], [-74.13762430745145, 40.54425356641828], [-74.13875534703455, 40.544933958426874], [-74.13786798524545, 40.54405254413092], [-74.13903307178627, 40.54470491029999], [-74.13953937915736, 40.54429514087214], [-74.13834151288823, 40.543551828814046], [-74.14038573073617, 40.54403318500786], [-74.13921993485874, 40.54316608513788], [-74.14141073560397, 40.543022415080344], [-74.14026290400692, 40.54196151152258], [-74.14134901379589, 40.54257752313157], [-74.14179719957052, 40.54220041538235], [-74.14063984172124, 40.54169454203723], [-74.14095816832888, 40.54143207796496], [-74.14229632246158, 40.54225784915546], [-74.14104157916547, 40.53734688546733], [-74.14374951249268, 40.53719106134627], [-74.15970134520207, 40.52729599283152], [-74.16134961699994, 40.5293734239999], [-74.15397419799994, 40.53426814799993], [-74.16540080799992, 40.54443272299989], [-74.16982582400001, 40.56109042099991], [-74.1649414939999, 40.56011179699991], [-74.15725974199995, 40.562553318999896], [-74.14706472699994, 40.5705086149999]]]}}, {\"id\": \"109\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22599999999999998, \"OBJECTID\": 110, \"Shape_Leng\": 0.103946292913, \"Shape_Area\": 0.000525745098785, \"zone\": \"Great Kills Park\", \"LocationID\": 110, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.11813477599993, 40.550592692999906], [-74.11338340608127, 40.54807337507532], [-74.12245306050386, 40.544770545629824], [-74.1335644349803, 40.5315111748056], [-74.1369079907634, 40.529281918882695], [-74.13883030715276, 40.529889356352996], [-74.14032475393472, 40.5352845797219], [-74.13381005153853, 40.53564293988989], [-74.12738789634466, 40.54229542996431], [-74.13011404715266, 40.54597572663567], [-74.13585712583111, 40.546451696357344], [-74.13861798499991, 40.548492383999886], [-74.12730057799996, 40.55729517199993], [-74.11813477599993, 40.550592692999906]]]}}, {\"id\": \"110\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22327868305062104, \"OBJECTID\": 111, \"Shape_Leng\": 0.0599308800658, \"Shape_Area\": 0.000208683276503, \"zone\": \"Green-Wood Cemetery\", \"LocationID\": 111, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99516890099986, 40.6596695219999], [-73.990188415, 40.657600817999906], [-73.98843135299977, 40.65925121199996], [-73.98179568699996, 40.65524609699991], [-73.98029115499993, 40.647297160999855], [-73.98905872499985, 40.64411924099995], [-74.00214474599989, 40.65295610699996], [-73.99516890099986, 40.6596695219999]]]}}, {\"id\": \"111\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2204978606181463, \"OBJECTID\": 112, \"Shape_Leng\": 0.107267875765, \"Shape_Area\": 0.000462068671429, \"zone\": \"Greenpoint\", \"LocationID\": 112, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.95439555417066, 40.73911477252253], [-73.94652352854791, 40.73692685395814], [-73.94719983367108, 40.73535517811412], [-73.94706532915383, 40.734401992819265], [-73.94557048457217, 40.736612303910405], [-73.94239443542429, 40.73542574994758], [-73.9381033496066, 40.729728519521366], [-73.93226852175046, 40.72817465801565], [-73.94841638599995, 40.71845553299995], [-73.94907816799986, 40.72254028699993], [-73.95477662900004, 40.72245950499994], [-73.95842611843923, 40.725400535832165], [-73.9616151883593, 40.725865563073995], [-73.96143187677845, 40.730965171647846], [-73.96226340423281, 40.732915551623485], [-73.96406507049178, 40.732779040653625], [-73.95837507902779, 40.73809694936863], [-73.95439555417066, 40.73911477252253]]], [[[-73.96236596889429, 40.72420906161418], [-73.96200744849497, 40.7239919013038], [-73.96207271921921, 40.72388030020147], [-73.96246790011047, 40.72413157960129], [-73.96236596889429, 40.72420906161418]]]]}}, {\"id\": \"112\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21042639149415804, \"OBJECTID\": 113, \"Shape_Leng\": 0.032745350566, \"Shape_Area\": 5.79634566998e-05, \"zone\": \"Greenwich Village North\", \"LocationID\": 113, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99136388099988, 40.72755978299991], [-73.99657214499979, 40.729552233999904], [-73.99556245499996, 40.73071521599994], [-74.000066577, 40.7329292359999], [-73.99683993899998, 40.73736088899995], [-73.98990295999991, 40.7344347899999], [-73.99136388099988, 40.72755978299991]]]}}, {\"id\": \"113\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20914651077761146, \"OBJECTID\": 114, \"Shape_Leng\": 0.0317270347711, \"Shape_Area\": 4.70207704945e-05, \"zone\": \"Greenwich Village South\", \"LocationID\": 114, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99257987399992, 40.724201554999915], [-74.00281726599992, 40.72836312899993], [-74.000066577, 40.7329292359999], [-73.99154832699986, 40.72708947099994], [-73.99257987399992, 40.724201554999915]]]}}, {\"id\": \"114\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.32119751984248307, \"OBJECTID\": 115, \"Shape_Leng\": 0.116169413964, \"Shape_Area\": 0.000373168991958, \"zone\": \"Grymes Hill/Clifton\", \"LocationID\": 115, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08069514799999, 40.62753689699988], [-74.08326107499997, 40.627194742999954], [-74.08496211899991, 40.624625232999925], [-74.08785147699994, 40.615243072999924], [-74.08720982799997, 40.61330697399994], [-74.07253551699989, 40.622464021999896], [-74.071134623, 40.62035994799992], [-74.0815547009999, 40.60629029699987], [-74.0977696879999, 40.61062359299988], [-74.10350925299994, 40.61641607499991], [-74.0922397139999, 40.62574291499989], [-74.08574928999992, 40.633955556999865], [-74.07972492199995, 40.62895068799986], [-74.08069514799999, 40.62753689699988]]]}}, {\"id\": \"115\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.19669953317303726, \"OBJECTID\": 116, \"Shape_Leng\": 0.0681164844265, \"Shape_Area\": 0.000260415337217, \"zone\": \"Hamilton Heights\", \"LocationID\": 116, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93924885699987, 40.82829635099989], [-73.94515456999994, 40.82087626499993], [-73.94808289799983, 40.82210802199994], [-73.95082880000002, 40.820827936999876], [-73.95954472552476, 40.82364494182928], [-73.95015521048293, 40.83439675940791], [-73.93864026699997, 40.82973887399988], [-73.93924885699987, 40.82829635099989]]]}}, {\"id\": \"116\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2567155021697498, \"OBJECTID\": 117, \"Shape_Leng\": 0.169885857987, \"Shape_Area\": 0.00090429427072, \"zone\": \"Hammels/Arverne\", \"LocationID\": 117, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7763584369479, 40.609655838242304], [-73.77412897394224, 40.60874659586809], [-73.77460294599074, 40.607417648882524], [-73.77484886546486, 40.60802483315967], [-73.77523851986173, 40.60855094300021], [-73.77600772152827, 40.60919075424813], [-73.77469363854065, 40.60430696481727], [-73.7818572569845, 40.59837293018506], [-73.78005493695551, 40.596861344908895], [-73.7749751364952, 40.60174817941184], [-73.77363244061033, 40.598153123466695], [-73.76809925802587, 40.59784597005487], [-73.76771811799993, 40.596261309999925], [-73.76654204999997, 40.599434441999904], [-73.76165797299986, 40.599973069999876], [-73.7633344969999, 40.59782287399992], [-73.76277845599988, 40.59475654399989], [-73.76440183099987, 40.59457168299989], [-73.76376223423699, 40.591439709016804], [-73.81730795916116, 40.58233475897], [-73.81544517899997, 40.58337349199993], [-73.81909407399996, 40.58783196099988], [-73.8176380045051, 40.58928579948519], [-73.81170344836342, 40.59159374663789], [-73.81149423585859, 40.592901326696776], [-73.81132554567534, 40.59174649614576], [-73.81081791874632, 40.59209026869603], [-73.81138593152033, 40.592442318454566], [-73.81140751227157, 40.592871257238706], [-73.81107287021051, 40.593022330572076], [-73.80991130015991, 40.59225371768247], [-73.80959170493088, 40.593580688618054], [-73.80577902839048, 40.59075949966858], [-73.8058134261234, 40.59186886150206], [-73.80430333508215, 40.59131887880457], [-73.8079775709933, 40.59415786866439], [-73.80604149807385, 40.595554555213546], [-73.8024384629031, 40.59264006383509], [-73.80234513845657, 40.59449597525689], [-73.80344017041448, 40.59443420917679], [-73.80321494251235, 40.59514564845063], [-73.80389507006726, 40.594990875367245], [-73.80331453729063, 40.59517956193947], [-73.80368468695728, 40.59519779018381], [-73.80486462646938, 40.5967336283515], [-73.80513406252756, 40.59660758897049], [-73.80524832538248, 40.596895032613546], [-73.80297943325748, 40.598844304553005], [-73.79233256449508, 40.59995439142583], [-73.78635332811808, 40.60319465903842], [-73.79034006550985, 40.59919167636005], [-73.79102536652763, 40.595527157215926], [-73.78973901889819, 40.59474366855456], [-73.7890682912148, 40.598019927382616], [-73.78350560905308, 40.60256343874268], [-73.78366696292244, 40.60532443693426], [-73.77927015444938, 40.60930736180752], [-73.7763584369479, 40.609655838242304]]]}}, {\"id\": \"117\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.3010802469135802, \"OBJECTID\": 118, \"Shape_Leng\": 0.243966217692, \"Shape_Area\": 0.00182693922711, \"zone\": \"Heartland Village/Todt Hill\", \"LocationID\": 118, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.10834417899996, 40.610234766999945], [-74.09555829799993, 40.61062477799989], [-74.09045495899996, 40.60909452599991], [-74.09271908099993, 40.60636222199992], [-74.09192827899989, 40.598504050999914], [-74.10041498999992, 40.59334309299988], [-74.10149482299994, 40.58934426099992], [-74.10790851599995, 40.58368563799992], [-74.1212571389999, 40.575254893999904], [-74.12708814699995, 40.57636910599994], [-74.13261022999991, 40.573530162999894], [-74.14418588199999, 40.57203740999988], [-74.14438744999991, 40.56970610099991], [-74.14789716899996, 40.57088728599995], [-74.15725974199995, 40.562553318999896], [-74.1649414939999, 40.56011179699991], [-74.16982582400001, 40.56109042099991], [-74.16842916199994, 40.58649792299988], [-74.16189940599996, 40.59529877099993], [-74.1481980769999, 40.588599680999934], [-74.13864458599996, 40.58906080499989], [-74.13091909399999, 40.598855047999905], [-74.12506115499997, 40.59818005599987], [-74.12059546299996, 40.601809757999874], [-74.11487058399989, 40.602136659999935], [-74.11742466500002, 40.609289008999916], [-74.12135179599994, 40.609765267999926], [-74.11508745399988, 40.61120943099993], [-74.10834417899996, 40.610234766999945]]]}}, {\"id\": \"118\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.234484146595079, \"OBJECTID\": 119, \"Shape_Leng\": 0.0835070355744, \"Shape_Area\": 0.000185779447534, \"zone\": \"Highbridge\", \"LocationID\": 119, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9247831879999, 40.84475266499989], [-73.91729224999992, 40.84510993999991], [-73.91794965199988, 40.842229570999976], [-73.91618344399991, 40.84233489999991], [-73.92802724899998, 40.82971953899987], [-73.93307251084072, 40.828156152518105], [-73.93314306709256, 40.83519412761651], [-73.92861574822331, 40.844676865029925], [-73.9247831879999, 40.84475266499989]]]}}, {\"id\": \"119\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2355613175632274, \"OBJECTID\": 120, \"Shape_Leng\": 0.0832612702158, \"Shape_Area\": 9.31462821313e-05, \"zone\": \"Highbridge Park\", \"LocationID\": 120, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92295449499989, 40.85885076199988], [-73.92227873064479, 40.85565498884376], [-73.93436121591056, 40.83620061961779], [-73.94034643299983, 40.8304578419999], [-73.93505508799984, 40.8393332359999], [-73.93556697199989, 40.841712527999924], [-73.92707982300001, 40.851432342999885], [-73.92466977999999, 40.856982610999886], [-73.92709951599986, 40.85827552099992], [-73.92499818399988, 40.86146373799991], [-73.92295449499989, 40.85885076199988]]]}}, {\"id\": \"120\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2291746213170489, \"OBJECTID\": 121, \"Shape_Leng\": 0.0969153373445, \"Shape_Area\": 0.000384563286473, \"zone\": \"Hillcrest/Pomonok\", \"LocationID\": 121, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79911195699995, 40.738206139999875], [-73.79584803299987, 40.7380746799999], [-73.79037661699995, 40.720103723999934], [-73.80780927699995, 40.715853895999906], [-73.80904032699983, 40.71991160499987], [-73.80452680199981, 40.721462545999884], [-73.80487958299983, 40.725049592999895], [-73.80652142399995, 40.7248730639999], [-73.80717042299993, 40.72826913299987], [-73.8111179729999, 40.72781641599994], [-73.81074426599986, 40.72626511299991], [-73.81510024099985, 40.72854342299991], [-73.81504998799986, 40.73894313399992], [-73.79911195699995, 40.738206139999875]]]}}, {\"id\": \"121\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22058927043739104, \"OBJECTID\": 122, \"Shape_Leng\": 0.0665318439446, \"Shape_Area\": 0.000226597104976, \"zone\": \"Hollis\", \"LocationID\": 122, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.75725671499985, 40.71813860199992], [-73.75047058199985, 40.70563993099994], [-73.76612489399999, 40.702334485999884], [-73.76825969099993, 40.709526367999956], [-73.771724447, 40.71031681899991], [-73.76943987099992, 40.711976892999864], [-73.77192122100001, 40.71568504499992], [-73.76664095199988, 40.717646613999875], [-73.765714472, 40.71639337899988], [-73.75832728699997, 40.71958208899985], [-73.75725671499985, 40.71813860199992]]]}}, {\"id\": \"122\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.3048929295890292, \"OBJECTID\": 123, \"Shape_Leng\": 0.0834211389387, \"Shape_Area\": 0.000296445173366, \"zone\": \"Homecrest\", \"LocationID\": 123, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96285973999984, 40.61254948599989], [-73.96237947799978, 40.609986373999966], [-73.95859278499987, 40.61040303099988], [-73.95619313699986, 40.58776226899994], [-73.9603495359999, 40.58730628599994], [-73.9606798409999, 40.59159758199995], [-73.97115101799994, 40.59044244399989], [-73.97299433999983, 40.6088141419999], [-73.96429649899991, 40.60977642599992], [-73.96478112799986, 40.61233092199988], [-73.96285973999984, 40.61254948599989]]]}}, {\"id\": \"123\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.24496189515275907, \"OBJECTID\": 124, \"Shape_Leng\": 0.127724410062, \"Shape_Area\": 0.000754837821479, \"zone\": \"Howard Beach\", \"LocationID\": 124, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.85096453700001, 40.671719510999885], [-73.83334819299995, 40.66677318699994], [-73.82859942399985, 40.657196972999884], [-73.82639242872236, 40.64832434880783], [-73.83133151403268, 40.65274755744263], [-73.82901417557953, 40.65610788445241], [-73.8317081441153, 40.65393117429689], [-73.83221698518575, 40.654460386292925], [-73.83149330103005, 40.65497982053967], [-73.83116100809627, 40.65591923373715], [-73.82937603410723, 40.65650633521598], [-73.82896778740296, 40.65720524792007], [-73.8291791973457, 40.65779977854427], [-73.83013028740389, 40.65840264372116], [-73.82970706838907, 40.658502513864484], [-73.82959986903019, 40.65865680691539], [-73.83014950104759, 40.6598086849573], [-73.83024427039376, 40.658349969348166], [-73.82909434825203, 40.65733421866331], [-73.83127112568108, 40.656003271494676], [-73.8318523847929, 40.65488503826691], [-73.83290309524602, 40.657483771623006], [-73.83194394578256, 40.65617142725338], [-73.832600363706, 40.65782659162732], [-73.83096907996668, 40.65823029417234], [-73.83338439472544, 40.65805185920798], [-73.83109184464118, 40.64848480675928], [-73.83561034795801, 40.64850568470053], [-73.8387360855594, 40.662454060021076], [-73.83978752612934, 40.660485998048], [-73.83592320073285, 40.645499790674954], [-73.84950699084165, 40.64413376567876], [-73.85183280737658, 40.64586005671262], [-73.85230349886227, 40.647771028550196], [-73.84939766406322, 40.65143487895031], [-73.85591932944749, 40.65130926166608], [-73.86136381950391, 40.65641475416531], [-73.85983363521494, 40.65697706322045], [-73.8596355866937, 40.65845777205655], [-73.86124943398261, 40.65872091007865], [-73.85976839588899, 40.65828193634574], [-73.86087597607579, 40.65690816207006], [-73.86317083299996, 40.65827651199994], [-73.85761052799985, 40.660108355999924], [-73.85842950899992, 40.66345335999987], [-73.85568461199988, 40.66386749199993], [-73.85763323099987, 40.67165619399989], [-73.86038937900003, 40.67126877499985], [-73.86131863599985, 40.6750212689999], [-73.85096453700001, 40.671719510999885]]], [[[-73.83032725337058, 40.65513280580328], [-73.83026601851493, 40.65542864678473], [-73.82988919856028, 40.655522822584075], [-73.82994556435834, 40.655424494353], [-73.83032725337058, 40.65513280580328]]], [[[-73.85975533686633, 40.65410417609493], [-73.86048308199986, 40.65465892799988], [-73.86098917376947, 40.65520976705256], [-73.85979242750741, 40.65447615737696], [-73.85975533686633, 40.65410417609493]]], [[[-73.85898002989568, 40.65267545791771], [-73.85975533686633, 40.65410417609493], [-73.85941022942576, 40.65386473792502], [-73.85898002989568, 40.65267545791771]]], [[[-73.85898002989568, 40.65267545791771], [-73.85731941515513, 40.65123709900783], [-73.85695870295893, 40.65058737348713], [-73.85898002989568, 40.65267545791771]]]]}}, {\"id\": \"124\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21255053776296673, \"OBJECTID\": 125, \"Shape_Leng\": 0.0497322121757, \"Shape_Area\": 7.64293209892e-05, \"zone\": \"Hudson Sq\", \"LocationID\": 125, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00269315099992, 40.72831405499989], [-74.0053098609999, 40.721824191999865], [-74.0108123829999, 40.72578980299994], [-74.0119643202455, 40.72432986461987], [-74.01165326569942, 40.725871761851344], [-74.01520426531133, 40.726361949379644], [-74.011542018976, 40.72644734692706], [-74.01067264999996, 40.72911266599987], [-74.00269315099992, 40.72831405499989]]]}}, {\"id\": \"125\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.24660924474130255, \"OBJECTID\": 126, \"Shape_Leng\": 0.12994872606, \"Shape_Area\": 0.0007032738273, \"zone\": \"Hunts Point\", \"LocationID\": 126, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88615037899993, 40.82730567799994], [-73.88753429599987, 40.82250933899995], [-73.8843985489999, 40.822965575999866], [-73.88244779599998, 40.81926832299992], [-73.87056806927927, 40.81208731762241], [-73.86811219140058, 40.806751832831786], [-73.87234684998835, 40.800174763010446], [-73.87814218819929, 40.801319290306495], [-73.87843295513449, 40.80270273469493], [-73.8850517233671, 40.80213090679797], [-73.89197929534278, 40.80638470897619], [-73.89128402413917, 40.80497628189916], [-73.89509836487399, 40.80686363917353], [-73.8952796287401, 40.805766184067906], [-73.90222284749719, 40.80494811309474], [-73.90307457199998, 40.80988061199988], [-73.89542858799986, 40.81584467199988], [-73.8970446669999, 40.8168018599999], [-73.89283653100001, 40.820974054999965], [-73.8958813589999, 40.82099257499987], [-73.89586019599982, 40.82263104599987], [-73.89320004999998, 40.82400818299992], [-73.89487088500002, 40.82393727199992], [-73.89483522899992, 40.82624684099993], [-73.88615037899993, 40.82730567799994]]]}}, {\"id\": \"126\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20417863580813, \"OBJECTID\": 127, \"Shape_Leng\": 0.0600188855815, \"Shape_Area\": 0.000163195647796, \"zone\": \"Inwood\", \"LocationID\": 127, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.911224294065, 40.87302771883775], [-73.91043824727869, 40.871377478551786], [-73.91254775130287, 40.866514943004354], [-73.91589911645785, 40.86430803808506], [-73.91538673051149, 40.863084384200725], [-73.91950213660932, 40.85879113379047], [-73.92159014848224, 40.86007280002688], [-73.92035142875443, 40.858195220619145], [-73.92282845999992, 40.8568514089999], [-73.92872203199994, 40.86674266299988], [-73.92135975499993, 40.869551853999894], [-73.91845930899993, 40.873022052999936], [-73.9140098549999, 40.871160593999946], [-73.911224294065, 40.87302771883775]]], [[[-73.92151894059094, 40.85660117062315], [-73.92229611007241, 40.855663835114505], [-73.92234459387058, 40.85568233590737], [-73.92169841904696, 40.85667230185494], [-73.92151894059094, 40.85660117062315]]]]}}, {\"id\": \"127\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2127845499894611, \"OBJECTID\": 128, \"Shape_Leng\": 0.0702090290681, \"Shape_Area\": 0.000207405982869, \"zone\": \"Inwood Hill Park\", \"LocationID\": 128, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.92359742020389, 40.878898712992644], [-73.92363695289211, 40.87881535593209], [-73.92367919311681, 40.878826074498626], [-73.92362973975199, 40.87896394312155], [-73.92359742020389, 40.878898712992644]]], [[[-73.9264055692112, 40.87762147653741], [-73.92258387170895, 40.87692253928837], [-73.921169296781, 40.87306939228805], [-73.91978312944534, 40.87360369743557], [-73.92090325515422, 40.87546616638258], [-73.91855365026461, 40.873180588115396], [-73.9175772988473, 40.87454867854727], [-73.91154281116535, 40.873253706161705], [-73.9140098549999, 40.871160593999946], [-73.91845930899993, 40.873022052999936], [-73.92135975499993, 40.869551853999894], [-73.92872203199994, 40.86674266299988], [-73.93220600531866, 40.870096445104515], [-73.9264055692112, 40.87762147653741]]]]}}, {\"id\": \"131\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2007000051677815, \"OBJECTID\": 132, \"Shape_Leng\": 0.24547851707, \"Shape_Area\": 0.00203830095472, \"zone\": \"JFK Airport\", \"LocationID\": 132, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.8250346749999, 40.663580133999844], [-73.81496922499986, 40.66247266799991], [-73.80072889200001, 40.66526937699988], [-73.780755286, 40.66240660199992], [-73.7541108669999, 40.648697664999915], [-73.75522386899995, 40.64744446899989], [-73.7474246205621, 40.6413764680137], [-73.74849297657127, 40.63567172844184], [-73.7650077554797, 40.62941398784963], [-73.77077016777163, 40.62003118215498], [-73.77082069703974, 40.62328299797239], [-73.77704080084247, 40.62766646976843], [-73.77999420781757, 40.62668929803533], [-73.78419483298926, 40.6208926699551], [-73.78949653224412, 40.62274664610792], [-73.78264543417806, 40.630247968544396], [-73.79076956011139, 40.63386374895698], [-73.79222318018806, 40.63589008813999], [-73.81830607819735, 40.64638698303703], [-73.82182513624882, 40.649899795874894], [-73.82348381110747, 40.65536991271862], [-73.82227996040434, 40.659425007931176], [-73.8186953155868, 40.661105051055245], [-73.81143921922407, 40.66074562606795], [-73.81867189288894, 40.66213298725428], [-73.82338324351308, 40.6602057470252], [-73.82498070378553, 40.65543197118367], [-73.82290717700891, 40.64809344600474], [-73.82614483059669, 40.65008184094253], [-73.82639242822242, 40.64832434880782], [-73.82859942399985, 40.657196972999884], [-73.832705339, 40.665405772999854], [-73.8250346749999, 40.663580133999844]]], [[[-73.74661040472857, 40.638724973885545], [-73.74653979797041, 40.63871136266965], [-73.74654132522075, 40.63869678799164], [-73.74664302817305, 40.638662249807076], [-73.74661040472857, 40.638724973885545]]], [[[-73.74693721618013, 40.63755610186007], [-73.74694320619099, 40.637529202088494], [-73.74704913170525, 40.63754513439954], [-73.74704316342154, 40.637566426459216], [-73.74693721618013, 40.63755610186007]]], [[[-73.74712154262465, 40.63716227570799], [-73.74706485671244, 40.637160999268374], [-73.74699391228548, 40.63715064484292], [-73.74715781614569, 40.637060524028364], [-73.74712154262465, 40.63716227570799]]]]}}, {\"id\": \"128\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21345779820790622, \"OBJECTID\": 129, \"Shape_Leng\": 0.0932727273403, \"Shape_Area\": 0.00047398833179, \"zone\": \"Jackson Heights\", \"LocationID\": 129, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87586503899996, 40.76605165399989], [-73.87513874300004, 40.756633815999905], [-73.87365721799983, 40.756803485999946], [-73.86942458599987, 40.74915685199991], [-73.89175048799984, 40.74681172799989], [-73.89624245599988, 40.74871954499987], [-73.89984592699983, 40.75756215299985], [-73.89437624799997, 40.7655265849999], [-73.87608949699984, 40.77153690299987], [-73.87586503899996, 40.76605165399989]]]}}, {\"id\": \"129\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21380360285717254, \"OBJECTID\": 130, \"Shape_Leng\": 0.142028320665, \"Shape_Area\": 0.000468322514327, \"zone\": \"Jamaica\", \"LocationID\": 130, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76962552399995, 40.71178631999993], [-73.771724447, 40.71031681899991], [-73.76844546499984, 40.709637026999886], [-73.77555451099995, 40.70607083499987], [-73.77230768699997, 40.70057150699989], [-73.77395659499993, 40.69980578099988], [-73.78034491200003, 40.70541241699986], [-73.78256582599995, 40.70542486399989], [-73.79239170299988, 40.7021665299999], [-73.79260506799987, 40.70043209999988], [-73.80642479399984, 40.690873857999904], [-73.80890216900004, 40.69256434899993], [-73.81063298299996, 40.69194738299988], [-73.81637824499992, 40.70245196399987], [-73.81152856899989, 40.70227287499991], [-73.81201248599983, 40.70594494499993], [-73.80694856299993, 40.70736709999989], [-73.8056477309999, 40.70467172799994], [-73.80165725599993, 40.705761474999846], [-73.80302414799988, 40.70813965699997], [-73.79870150699992, 40.71066337699988], [-73.79776810199988, 40.7091431879999], [-73.77192121999992, 40.715685043999905], [-73.76962552399995, 40.71178631999993]]]}}, {\"id\": \"1\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2174912891986063, \"OBJECTID\": 2, \"Shape_Leng\": 0.43346966679, \"Shape_Area\": 0.00486634037837, \"zone\": \"Jamaica Bay\", \"LocationID\": 2, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.82337597260663, 40.63898704717671], [-73.82107505533321, 40.62973377456623], [-73.81611483429545, 40.624604655173044], [-73.81891708039825, 40.62309026716333], [-73.81571468044605, 40.62148955151693], [-73.81777429834523, 40.61750112169565], [-73.81652681189122, 40.61494700958006], [-73.82141601599992, 40.615054762999925], [-73.8238246859766, 40.61149866495507], [-73.82575322105271, 40.614192081540075], [-73.83776702143372, 40.61552223141874], [-73.83496222645587, 40.61778030881177], [-73.83514355859735, 40.62548708279667], [-73.83342230809954, 40.626781983462095], [-73.83464377224975, 40.62742707942435], [-73.83115393179625, 40.63256052752952], [-73.83340607979225, 40.63850135589897], [-73.82591657926903, 40.63574741296666], [-73.82443032497055, 40.6362165210408], [-73.82419327293158, 40.63995327448778], [-73.82337597260663, 40.63898704717671]]], [[[-73.84721834832125, 40.63196568670605], [-73.8485899664393, 40.63013198680654], [-73.8585583634798, 40.63045828819559], [-73.8561374187591, 40.63508154601468], [-73.84667914840234, 40.63873171175714], [-73.84352007697278, 40.636503685327924], [-73.84721834832125, 40.63196568670605]]], [[[-73.79577948747226, 40.63215847659964], [-73.79811872226199, 40.6310375092611], [-73.79827615415297, 40.63125217936581], [-73.7907928429953, 40.63428204310243], [-73.79577948747226, 40.63215847659964]]], [[[-73.83623731597828, 40.631110861231726], [-73.84110205447426, 40.62455643651514], [-73.84445711462239, 40.62984779443998], [-73.83765174362401, 40.63283973295598], [-73.83623731597828, 40.631110861231726]]], [[[-73.81307233892618, 40.62926109964263], [-73.81352502122876, 40.62730617268596], [-73.81258556038132, 40.628371916183724], [-73.81129432207656, 40.624703779000406], [-73.81089990481607, 40.61621847134095], [-73.81679073502552, 40.62634891812118], [-73.81382899493154, 40.62613507764411], [-73.81627026043981, 40.62714580591711], [-73.8173057692762, 40.63062197379782], [-73.81506788441455, 40.63116405857793], [-73.81307233892618, 40.62926109964263]]], [[[-73.84734350666976, 40.62909473971626], [-73.84984630167418, 40.62622449074592], [-73.8472783772798, 40.626621669791945], [-73.84358010755503, 40.623178164184395], [-73.86364988794934, 40.617878773642374], [-73.86404106276488, 40.61920759671402], [-73.86151825485233, 40.619049218399226], [-73.86102883383386, 40.624138986994076], [-73.86347880947007, 40.6233886093158], [-73.86334955670225, 40.62592490877777], [-73.86041082270874, 40.62516691297755], [-73.8565055097758, 40.629097353726586], [-73.8539654419115, 40.62761094389788], [-73.85436931062769, 40.62881767654389], [-73.84734350666976, 40.62909473971626]]], [[[-73.87084603194215, 40.61696742645758], [-73.8782477932424, 40.615444608331536], [-73.87481386865049, 40.62358718918363], [-73.86684384142526, 40.62782924286026], [-73.86498193495926, 40.625660706041025], [-73.86626127665735, 40.618456348023955], [-73.87084603194215, 40.61696742645758]]], [[[-73.79783533398513, 40.627408716571516], [-73.79566256726108, 40.626114004327796], [-73.79921493427759, 40.6240847129885], [-73.7936292748135, 40.62515647619494], [-73.79809093107174, 40.62306294824432], [-73.79992057632684, 40.61857524016598], [-73.80172498640165, 40.61996700873167], [-73.80128049504093, 40.615879839747535], [-73.80270343867807, 40.61358065232171], [-73.80508906012292, 40.61343256041943], [-73.80476592616557, 40.61580099137086], [-73.80764116896117, 40.62119965469225], [-73.80698502381061, 40.623871998829976], [-73.8001117714566, 40.626927441987085], [-73.80092841565032, 40.627524104895656], [-73.79783533398513, 40.627408716571516]]], [[[-73.77348616522504, 40.625085118163526], [-73.77381024409397, 40.624925596730954], [-73.77383258726536, 40.624944848253456], [-73.77348560409436, 40.62511314810922], [-73.77348616522504, 40.625085118163526]]], [[[-73.77172760292548, 40.62358502801516], [-73.7720313143135, 40.62299121290296], [-73.77218824028017, 40.622995793795134], [-73.77511891254433, 40.619233847487365], [-73.77518045935567, 40.61926471788484], [-73.77172760292548, 40.62358502801516]]], [[[-73.79162058913477, 40.62231334348308], [-73.78419483298926, 40.6208926699551], [-73.78446087067887, 40.617027835129704], [-73.78070887195283, 40.614026809005864], [-73.79139721011913, 40.60761085524994], [-73.79480633875043, 40.61037836048121], [-73.80080125919478, 40.611933904481596], [-73.7946062540462, 40.62257170055433], [-73.79162058913477, 40.62231334348308]]], [[[-73.76670827781236, 40.61491086618553], [-73.76825288003378, 40.614877725169315], [-73.77397692018825, 40.61600357567147], [-73.76873454755717, 40.62090086822023], [-73.76745926975256, 40.620511322031255], [-73.76670827781236, 40.61491086618553]]], [[[-73.84304421296773, 40.617737696390435], [-73.84454193473684, 40.614867921675135], [-73.8409994253963, 40.613029126216134], [-73.83825667290532, 40.613847216216406], [-73.8407944240835, 40.60564326638265], [-73.8471480968766, 40.60361424219198], [-73.85179402215192, 40.60689716858368], [-73.84779013405834, 40.60923538526278], [-73.84614804513843, 40.61772842036632], [-73.84304421296773, 40.617737696390435]]], [[[-73.85523067631884, 40.61613145335797], [-73.85530325465119, 40.6131454135728], [-73.84973986306498, 40.61515605814331], [-73.85010857555366, 40.61021219334599], [-73.8551559228834, 40.606120513001486], [-73.85708638173246, 40.60646824293355], [-73.85924855452187, 40.61125483087423], [-73.85648360749232, 40.61319069486381], [-73.85858056782772, 40.61691708722184], [-73.85523067631884, 40.61613145335797]]], [[[-73.814090115668, 40.614466920099765], [-73.81503638494628, 40.6129762137494], [-73.81615626666235, 40.61294211575494], [-73.8164315038978, 40.614282334964926], [-73.814090115668, 40.614466920099765]]], [[[-73.81496830662114, 40.61042065495561], [-73.81520720849484, 40.60981673379666], [-73.81559858789666, 40.61067251616471], [-73.81450261566708, 40.61105847350542], [-73.81496830662114, 40.61042065495561]]], [[[-73.86966442958492, 40.606540123412906], [-73.87109330768071, 40.604694173994694], [-73.87300122724774, 40.61055545845987], [-73.8710387011171, 40.60972358252655], [-73.86966442958492, 40.606540123412906]]], [[[-73.79420172624032, 40.607730675613354], [-73.79352152715475, 40.60632759139594], [-73.79509002319158, 40.607271060105596], [-73.79424799033224, 40.60590038003164], [-73.79696327795033, 40.60411784876779], [-73.80428530678597, 40.604519279530564], [-73.80520676583323, 40.60673953810017], [-73.80159206744115, 40.61023096250127], [-73.79420172624032, 40.607730675613354]]], [[[-73.83244207358176, 40.605929449536745], [-73.83587601623029, 40.605636907545566], [-73.83610156628845, 40.60736279382329], [-73.83252192275347, 40.610384993603816], [-73.83032947905777, 40.60848787991649], [-73.83045815392722, 40.606669201403214], [-73.83244207358176, 40.605929449536745]]], [[[-73.82607472604538, 40.60843779954612], [-73.82771104235977, 40.608398668130704], [-73.82805077260065, 40.61004629686527], [-73.82688326414805, 40.6095966780694], [-73.82607472604538, 40.60843779954612]]], [[[-73.86702399475149, 40.60806817876942], [-73.86874317559169, 40.60703517927931], [-73.86868468826752, 40.608241158183915], [-73.86702399475149, 40.60806817876942]]], [[[-73.82718282107048, 40.607919778091194], [-73.82590092720767, 40.60646593064943], [-73.82861671457653, 40.60583848413761], [-73.82815085191197, 40.60795028983291], [-73.82718282107048, 40.607919778091194]]], [[[-73.81569926225602, 40.6064570673649], [-73.81546066483351, 40.60642813700742], [-73.81552816722274, 40.606163069945154], [-73.81552536723665, 40.60640738028192], [-73.81569926225602, 40.6064570673649]]], [[[-73.81339665223344, 40.60436407224534], [-73.80986333857152, 40.60189579000784], [-73.81157550345523, 40.59814119925956], [-73.81501471568424, 40.60205629156354], [-73.81339665223344, 40.60436407224534]]], [[[-73.86702905185578, 40.603743388853935], [-73.8676819415421, 40.603070766677334], [-73.8683080846461, 40.603139161146146], [-73.86793505372694, 40.60396113545295], [-73.86702905185578, 40.603743388853935]]], [[[-73.85243335807012, 40.59733339779527], [-73.86044420534076, 40.596471905596275], [-73.86335999147606, 40.600322202771196], [-73.86112814224546, 40.60255033107426], [-73.85482595782968, 40.601867195353634], [-73.85076748001559, 40.598839443837406], [-73.85243335807012, 40.59733339779527]]], [[[-73.87019491489939, 40.598996405901886], [-73.87108141811476, 40.598901147470094], [-73.87049411914771, 40.602604446204595], [-73.87026280564918, 40.60168122225861], [-73.87019491489939, 40.598996405901886]]], [[[-73.8231174097477, 40.60007949252535], [-73.82315293179705, 40.599976662824794], [-73.82317356168161, 40.59997800515576], [-73.82318015964296, 40.60008384902813], [-73.8231174097477, 40.60007949252535]]], [[[-73.82292757811203, 40.60006142816309], [-73.822978576463, 40.600043707224835], [-73.82303195183424, 40.59993051694214], [-73.82304007849375, 40.60006807478574], [-73.82292757811203, 40.60006142816309]]], [[[-73.82445629476595, 40.59911583868227], [-73.82395871079126, 40.59887689683851], [-73.82477088992478, 40.5983918438742], [-73.82453084258555, 40.59911949829181], [-73.82445629476595, 40.59911583868227]]], [[[-73.86952707548932, 40.598713750354406], [-73.86967070711478, 40.59668856362211], [-73.87020899836229, 40.59695423278937], [-73.86952707548932, 40.598713750354406]]], [[[-73.83668274106707, 40.5949466970158], [-73.8325749810638, 40.592118209113956], [-73.83422777067327, 40.59137381382269], [-73.83253569485043, 40.59021893579835], [-73.83495422767699, 40.589728155618005], [-73.83347866645357, 40.589368748710896], [-73.83719541135855, 40.58969911736292], [-73.8398907019092, 40.59280021149386], [-73.84385894083933, 40.59342531196059], [-73.83902964248091, 40.59625348474866], [-73.83668274106707, 40.5949466970158]]], [[[-73.86706149472118, 40.5820879767934], [-73.87000313642912, 40.582636614815954], [-73.87115922675414, 40.5868738879673], [-73.86635052164085, 40.591892676564676], [-73.86327273492819, 40.59090807569255], [-73.86125299377363, 40.58798808592123], [-73.86327471071942, 40.58387684853184], [-73.86706149472118, 40.5820879767934]]]]}}, {\"id\": \"130\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23815582345541045, \"OBJECTID\": 131, \"Shape_Leng\": 0.116547120922, \"Shape_Area\": 0.000423696152789, \"zone\": \"Jamaica Estates\", \"LocationID\": 131, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.77732562899998, 40.722392700999926], [-73.76924074899993, 40.72590270099988], [-73.75669771299987, 40.7262287109999], [-73.7523343839998, 40.72026246099993], [-73.79303800099989, 40.71071107499988], [-73.79358570299998, 40.714128677999874], [-73.790302674, 40.7208914649999], [-73.79174472699992, 40.725788849999944], [-73.77757506899988, 40.73019327299987], [-73.7752338719998, 40.725237532999884], [-73.77803765999981, 40.723284268999855], [-73.77732562899998, 40.722392700999926]]]}}, {\"id\": \"132\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21164169685230344, \"OBJECTID\": 133, \"Shape_Leng\": 0.065134863162, \"Shape_Area\": 0.000157188987507, \"zone\": \"Kensington\", \"LocationID\": 133, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97819074399992, 40.64752599199992], [-73.97777380600002, 40.645341876999964], [-73.974642245, 40.644813218999914], [-73.97084113799984, 40.64637857099989], [-73.96803868499985, 40.636650984999925], [-73.97029036199991, 40.63640634099988], [-73.96986869299992, 40.63419467499989], [-73.97604935699995, 40.63127590599986], [-73.9768886119999, 40.6356748619999], [-73.97911188699993, 40.63544038599991], [-73.98042057899988, 40.64238522699991], [-73.98299419199994, 40.64132499699987], [-73.98834986699991, 40.64456224399992], [-73.97819074399992, 40.64752599199992]]]}}, {\"id\": \"133\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.227634622322135, \"OBJECTID\": 134, \"Shape_Leng\": 0.0695878223284, \"Shape_Area\": 0.000202375182143, \"zone\": \"Kew Gardens\", \"LocationID\": 134, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82288529599995, 40.71192805799993], [-73.81637824499992, 40.70245196399987], [-73.82245890399994, 40.70365553099989], [-73.83152109599999, 40.70138962899994], [-73.83460244999996, 40.70392746699988], [-73.83661882199989, 40.70342001799988], [-73.83809584699989, 40.70616458099993], [-73.83714702999997, 40.70795704599989], [-73.84314625299982, 40.70922345599992], [-73.82591945199992, 40.71598987599992], [-73.82288529599995, 40.71192805799993]]]}}, {\"id\": \"134\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2281682886463108, \"OBJECTID\": 135, \"Shape_Leng\": 0.121793045659, \"Shape_Area\": 0.000467912240542, \"zone\": \"Kew Gardens Hills\", \"LocationID\": 135, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83126834299986, 40.74026227599991], [-73.8294675199999, 40.73702448299991], [-73.8243501109999, 40.73975360599992], [-73.81497208299996, 40.738560715999874], [-73.81510024099985, 40.72854342299991], [-73.81074426599986, 40.72626511299991], [-73.8111179729999, 40.72781641599994], [-73.80717042299993, 40.72826913299987], [-73.80652142399995, 40.7248730639999], [-73.80487958299983, 40.725049592999895], [-73.80452680199981, 40.721462545999884], [-73.80904032699983, 40.71991160499987], [-73.80780927699995, 40.715853895999906], [-73.82018883899984, 40.71673522399984], [-73.81732944799997, 40.717660456999916], [-73.82531644999993, 40.71727569999994], [-73.8273090039999, 40.72464607699988], [-73.83073214599987, 40.72654932799991], [-73.83635625099998, 40.73500859199989], [-73.83564585599999, 40.738491010999894], [-73.83699037399995, 40.74209330099992], [-73.83586131899989, 40.743227984999926], [-73.83113122799989, 40.74233799999994], [-73.83126834299986, 40.74026227599991]]]}}, {\"id\": \"135\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.25770158357179074, \"OBJECTID\": 136, \"Shape_Leng\": 0.0906287154122, \"Shape_Area\": 0.000148524994129, \"zone\": \"Kingsbridge Heights\", \"LocationID\": 136, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89996501599988, 40.86801852399986], [-73.89891854299988, 40.86965857399992], [-73.89656060599998, 40.868565199999935], [-73.90486847199993, 40.8586932249999], [-73.90777320599983, 40.85960753799984], [-73.90976165899988, 40.857182611999896], [-73.91328437100006, 40.859618600999916], [-73.90908822499988, 40.8623996959999], [-73.91422823999919, 40.862490790359615], [-73.91044282747865, 40.866527868551835], [-73.90746489699994, 40.87354735399985], [-73.90452070499992, 40.87249521099988], [-73.90318667499999, 40.86914899099988], [-73.89996501599988, 40.86801852399986]]]}}, {\"id\": \"136\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20523623109779018, \"OBJECTID\": 137, \"Shape_Leng\": 0.0461076242707, \"Shape_Area\": 0.000116455098947, \"zone\": \"Kips Bay\", \"LocationID\": 137, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97219641647804, 40.74040257875022], [-73.97248994174741, 40.73580328010761], [-73.97446647047282, 40.736278456829886], [-73.9735447289197, 40.73494432655072], [-73.98266497499989, 40.73949733299987], [-73.97812378199983, 40.745727172999885], [-73.97167921200406, 40.74305446402989], [-73.97219641647804, 40.74040257875022]]]}}, {\"id\": \"137\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22660132406930011, \"OBJECTID\": 138, \"Shape_Leng\": 0.107466933508, \"Shape_Area\": 0.000536797294367, \"zone\": \"LaGuardia Airport\", \"LocationID\": 138, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.8712485624251, 40.78603815125499], [-73.86918113468879, 40.78594348978492], [-73.86847981674143, 40.78369082919537], [-73.87255732673727, 40.78082112154256], [-73.85505106271819, 40.772195389989086], [-73.8582051105021, 40.77032345647716], [-73.85624565746174, 40.768885714198944], [-73.86268864176631, 40.76687591050486], [-73.86043337809801, 40.76386888243776], [-73.86728193799998, 40.76965000299989], [-73.87223577699984, 40.77169565899993], [-73.87743339799992, 40.77120025499991], [-73.88770661799987, 40.76683859799995], [-73.88945486901865, 40.77353295106357], [-73.88391627963797, 40.774132606591536], [-73.88493466381185, 40.77993691280691], [-73.87870562518874, 40.78058590995002], [-73.878858374856, 40.78239817868118], [-73.87974777853563, 40.782836564207], [-73.8797016367226, 40.78288281807235], [-73.8751575489337, 40.781509449653285], [-73.8712485624251, 40.78603815125499]]], [[[-73.8728719590193, 40.785975027902346], [-73.87282839783694, 40.78595446173833], [-73.8727740594054, 40.78594192145881], [-73.87280920016116, 40.78590310909999], [-73.8728719590193, 40.785975027902346]]], [[[-73.89008921683674, 40.77362643576237], [-73.89034195999996, 40.77323760499998], [-73.89183244902246, 40.77488019994431], [-73.89017369508576, 40.77785081277782], [-73.89008921683674, 40.77362643576237]]]]}}, {\"id\": \"138\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22943852755661254, \"OBJECTID\": 139, \"Shape_Leng\": 0.0932997884483, \"Shape_Area\": 0.000446669880154, \"zone\": \"Laurelton\", \"LocationID\": 139, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.74125253099989, 40.68745783899989], [-73.72718272499995, 40.68359732999987], [-73.74117817299991, 40.666402953999885], [-73.75763129599993, 40.66645115699992], [-73.75636239300002, 40.67242923699999], [-73.75857198899986, 40.67263733099987], [-73.76031463999993, 40.67510997099984], [-73.75576169199995, 40.67588590899988], [-73.747551754, 40.689347336999944], [-73.74125253099989, 40.68745783899989]]]}}, {\"id\": \"139\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.19977256954329828, \"OBJECTID\": 140, \"Shape_Leng\": 0.0475842911325, \"Shape_Area\": 0.000114203907779, \"zone\": \"Lenox Hill East\", \"LocationID\": 140, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95854329689507, 40.758495622967885], [-73.96196573699997, 40.759619460999865], [-73.95268514999995, 40.772339404999926], [-73.9474897547817, 40.77011515400953], [-73.95854329689507, 40.758495622967885]]]}}, {\"id\": \"140\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.204524443078156, \"OBJECTID\": 141, \"Shape_Leng\": 0.0415144638712, \"Shape_Area\": 7.66545579019e-05, \"zone\": \"Lenox Hill West\", \"LocationID\": 141, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96177668399997, 40.75987971599993], [-73.9665833839999, 40.76155093499988], [-73.95729409999986, 40.7742835549999], [-73.95268514999995, 40.772339404999926], [-73.96177668399997, 40.75987971599993]]]}}, {\"id\": \"141\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2107594949183379, \"OBJECTID\": 142, \"Shape_Leng\": 0.0381758942321, \"Shape_Area\": 7.56537920738e-05, \"zone\": \"Lincoln Square East\", \"LocationID\": 142, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98147898799989, 40.7677250589999], [-73.98806289599996, 40.769790799999925], [-73.980673654, 40.77992518199991], [-73.97499744000007, 40.77753254599988], [-73.98147898799989, 40.7677250589999]]]}}, {\"id\": \"142\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20312332917308562, \"OBJECTID\": 143, \"Shape_Leng\": 0.0541798538849, \"Shape_Area\": 0.00015109426901, \"zone\": \"Lincoln Square West\", \"LocationID\": 143, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.981627874, 40.780328934999936], [-73.980673654, 40.77992518199991], [-73.98806289599996, 40.769790799999925], [-73.99624065194922, 40.77378979119886], [-73.99393587681126, 40.773179512586104], [-73.98886861739992, 40.779692922911416], [-73.9915510592274, 40.779574821437386], [-73.98812746184332, 40.78140179672363], [-73.981627874, 40.780328934999936]]]}}, {\"id\": \"143\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20642499987740578, \"OBJECTID\": 144, \"Shape_Leng\": 0.0276201668505, \"Shape_Area\": 4.74789670369e-05, \"zone\": \"Little Italy/NoLiTa\", \"LocationID\": 144, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99580912300002, 40.71664465899988], [-74.00188406299992, 40.71939757099991], [-73.9967717579999, 40.7254319439999], [-73.99260322299989, 40.724136449999925], [-73.99580912300002, 40.71664465899988]]]}}, {\"id\": \"144\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21807667611524306, \"OBJECTID\": 145, \"Shape_Leng\": 0.114313844981, \"Shape_Area\": 0.000346641733797, \"zone\": \"Long Island City/Hunters Point\", \"LocationID\": 145, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94345411699997, 40.75205360799992], [-73.93663681899982, 40.748204093999895], [-73.94419561599983, 40.742882145999914], [-73.93726110699994, 40.745176490999896], [-73.9386966449998, 40.73814081699989], [-73.94140886345744, 40.73929957085316], [-73.93866991753185, 40.74253408202486], [-73.9400192297566, 40.74320827221074], [-73.94159470030033, 40.73986412616895], [-73.94548236590111, 40.738207031293705], [-73.95380883920839, 40.73982029297784], [-73.96256114655104, 40.73868995735329], [-73.95978468552276, 40.74349771658391], [-73.96064863700711, 40.74413031761407], [-73.9587753700448, 40.74455559986341], [-73.95965229755977, 40.745216588173435], [-73.95849389303115, 40.74498786491623], [-73.9567945329049, 40.748839516955584], [-73.95318382492853, 40.74773481196097], [-73.9564856879466, 40.74911169015581], [-73.95080763809156, 40.75526368005708], [-73.94345411699997, 40.75205360799992]]]}}, {\"id\": \"145\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20850320456341914, \"OBJECTID\": 146, \"Shape_Leng\": 0.0471405215717, \"Shape_Area\": 0.000104638923425, \"zone\": \"Long Island City/Queens Plaza\", \"LocationID\": 146, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93675697199996, 40.74914697199991], [-73.944052652, 40.75231325999992], [-73.93679091699993, 40.75988288099991], [-73.92415699999985, 40.753961141999845], [-73.92562743799989, 40.752137243999904], [-73.93383269399993, 40.7516047019999], [-73.93675697199996, 40.74914697199991]]]}}, {\"id\": \"146\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.3140665848583471, \"OBJECTID\": 147, \"Shape_Leng\": 0.0587654949268, \"Shape_Area\": 0.000106418172757, \"zone\": \"Longwood\", \"LocationID\": 147, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89515277899991, 40.82783428999991], [-73.89487088500002, 40.82393727199992], [-73.89320004999998, 40.82400818299992], [-73.89586019599982, 40.82263104599987], [-73.8958813589999, 40.82099257499987], [-73.89290156199988, 40.82081122299989], [-73.8970446669999, 40.8168018599999], [-73.89542858799986, 40.81584467199988], [-73.90307457199998, 40.80988061199988], [-73.9044655149999, 40.81228195999994], [-73.9006114109999, 40.822108807999875], [-73.90198758599993, 40.82244042299992], [-73.90093510699997, 40.82792810199992], [-73.89515277899991, 40.82783428999991]]]}}, {\"id\": \"147\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20466813701268594, \"OBJECTID\": 148, \"Shape_Leng\": 0.0391305015633, \"Shape_Area\": 6.97489921327e-05, \"zone\": \"Lower East Side\", \"LocationID\": 148, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98447731699999, 40.72023423899987], [-73.98675136699994, 40.71959284599997], [-73.99022012699989, 40.71440493999988], [-73.99750445299988, 40.71406913199995], [-73.99260322199994, 40.72413644999987], [-73.98382387299982, 40.72147287199987], [-73.98447731699999, 40.72023423899987]]]}}, {\"id\": \"148\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.24634533488192023, \"OBJECTID\": 149, \"Shape_Leng\": 0.0836805821837, \"Shape_Area\": 0.000270639618955, \"zone\": \"Madison\", \"LocationID\": 149, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94405560899996, 40.61199284199993], [-73.94231175899985, 40.60826819699985], [-73.93934703399997, 40.610187140999884], [-73.93753749399987, 40.60855738999986], [-73.93972613899992, 40.60713291499995], [-73.93447158899997, 40.60123151199989], [-73.94288954099984, 40.600318649999934], [-73.94248473299992, 40.598200364999855], [-73.956927197, 40.596612075999865], [-73.95859278499987, 40.61040303099988], [-73.94978380499997, 40.611371749999854], [-73.95026514899993, 40.61392435199983], [-73.9448277099998, 40.61604620899986], [-73.94405560899996, 40.61199284199993]]]}}, {\"id\": \"149\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.26958079894788756, \"OBJECTID\": 150, \"Shape_Leng\": 0.089331491183, \"Shape_Area\": 0.000332568036088, \"zone\": \"Manhattan Beach\", \"LocationID\": 150, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9301907789999, 40.58508559299986], [-73.9319281118675, 40.58286976893519], [-73.95349322048675, 40.58298819826098], [-73.94274062592493, 40.58099558955925], [-73.93258006568031, 40.58126474120341], [-73.93106028107997, 40.57626592009687], [-73.9523584672133, 40.574274685845204], [-73.95414287899986, 40.58310613999989], [-73.95900739999992, 40.58283812799991], [-73.96025825899996, 40.58502930799994], [-73.9344471859999, 40.586652074999925], [-73.9301907789999, 40.58508559299986]]]}}, {\"id\": \"150\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2055035885179721, \"OBJECTID\": 151, \"Shape_Leng\": 0.0548899410999, \"Shape_Area\": 0.000128848901918, \"zone\": \"Manhattan Valley\", \"LocationID\": 151, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96371096499993, 40.7930129339999], [-73.97639951915279, 40.79827321034625], [-73.97287179090726, 40.8033561875739], [-73.96004456499999, 40.79804123499991], [-73.96371096499993, 40.7930129339999]]]}}, {\"id\": \"151\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2016859069361659, \"OBJECTID\": 152, \"Shape_Leng\": 0.0579094768782, \"Shape_Area\": 0.000146637644064, \"zone\": \"Manhattanville\", \"LocationID\": 152, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95357262899982, 40.82197589199988], [-73.94607828700005, 40.821263215999906], [-73.95210125199992, 40.811442853999935], [-73.96203105797969, 40.81808633074085], [-73.95881892233997, 40.82151852484672], [-73.9596033246027, 40.82299468678336], [-73.95793904909117, 40.82277435354731], [-73.9595089796959, 40.82359578895142], [-73.95357262899982, 40.82197589199988]]]}}, {\"id\": \"152\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2886802831030572, \"OBJECTID\": 153, \"Shape_Leng\": 0.024737239022, \"Shape_Area\": 3.24684964817e-05, \"zone\": \"Marble Hill\", \"LocationID\": 153, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90666530999995, 40.8757156089999], [-73.90893235220783, 40.87215734798524], [-73.91578609320084, 40.87571718248032], [-73.91033193599998, 40.879038046999895], [-73.90666530999995, 40.8757156089999]]]}}, {\"id\": \"153\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2099080638471448, \"OBJECTID\": 154, \"Shape_Leng\": 0.22733157076, \"Shape_Area\": 0.00178968692433, \"zone\": \"Marine Park/Floyd Bennett Field\", \"LocationID\": 154, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.93531145199985, 40.60886003999987], [-73.92986880199989, 40.60507134099995], [-73.92624885099994, 40.60741352199989], [-73.92444144399987, 40.60579356899993], [-73.91970928699986, 40.607302162999886], [-73.91331444593793, 40.60225844819831], [-73.90903689530528, 40.60267734322018], [-73.91073123013267, 40.60202137602758], [-73.90894225138031, 40.60160704069869], [-73.8839804295303, 40.60556651972071], [-73.87671672482064, 40.584914430200875], [-73.88174676983894, 40.5791588072124], [-73.89552381007165, 40.576769818436354], [-73.89903425420563, 40.587676171745805], [-73.90426874447125, 40.58691226298856], [-73.90651874663754, 40.588025473892536], [-73.91148171990072, 40.586078718712315], [-73.91204234015626, 40.594506954288846], [-73.91513297459494, 40.59878111593029], [-73.92151689981944, 40.60192365414108], [-73.92457042932716, 40.599723602732396], [-73.92950671202776, 40.60270923350431], [-73.92927458471804, 40.60399357480484], [-73.93117476161734, 40.603850406434695], [-73.93108919528449, 40.601722258264104], [-73.91454059121104, 40.58994422917544], [-73.91819087799986, 40.58660685499996], [-73.92956858299992, 40.5968202649999], [-73.92813139699999, 40.59777792799987], [-73.93972613899992, 40.60713291499995], [-73.93531145199985, 40.60886003999987]]], [[[-73.91990064335972, 40.59960052259278], [-73.91626393108346, 40.59816113731273], [-73.91295122631982, 40.59205432996962], [-73.91524478304683, 40.59158750817847], [-73.92182277731894, 40.59733293287253], [-73.92213049353394, 40.598662596177306], [-73.91990064335972, 40.59960052259278]]], [[[-73.91513331918175, 40.58626938540072], [-73.91282142952436, 40.58589683033704], [-73.91242103069457, 40.58339278402727], [-73.91729861410717, 40.58319080174952], [-73.91191058860355, 40.58296337725262], [-73.9114996013066, 40.58181878146464], [-73.92443604094977, 40.58355820344426], [-73.93184655018902, 40.58294105529734], [-73.9320424499999, 40.5842339599999], [-73.9300961629999, 40.584502075999886], [-73.93139947599984, 40.58607989499989], [-73.92101255523748, 40.58457501880612], [-73.91513331918175, 40.58626938540072]]]]}}, {\"id\": \"154\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2529824332576436, \"OBJECTID\": 155, \"Shape_Leng\": 0.169808473032, \"Shape_Area\": 0.000868261018498, \"zone\": \"Marine Park/Mill Basin\", \"LocationID\": 155, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90464209220775, 40.62513319188153], [-73.90546206137252, 40.62577481968345], [-73.9022870793228, 40.624409853930224], [-73.90071952540437, 40.62163492049855], [-73.89644123346895, 40.621394756190135], [-73.89586112894177, 40.614518137494784], [-73.89095414564672, 40.61283404731705], [-73.89394013317667, 40.61118142665126], [-73.88971513183803, 40.611598996904085], [-73.89334621420008, 40.606743446234645], [-73.89986880363854, 40.605524640544004], [-73.90133303458649, 40.611717882905985], [-73.90249392985292, 40.611814999032276], [-73.90119253465456, 40.61056381754662], [-73.90132962659396, 40.61048104310328], [-73.90879088017641, 40.617436396721416], [-73.90797306390525, 40.615736844061175], [-73.90959740910328, 40.616392669799325], [-73.90225859966276, 40.61042277259991], [-73.90177669809985, 40.605674686555815], [-73.90524300992108, 40.60491009032442], [-73.90863959535422, 40.606658722047854], [-73.90803306097814, 40.60535860211721], [-73.9098515484547, 40.605999722177685], [-73.9083253683517, 40.60407337391859], [-73.91365119759595, 40.60398251383695], [-73.91708838503304, 40.608820854898525], [-73.91449542205973, 40.61165322366488], [-73.91428072663338, 40.61493881616294], [-73.91662804371141, 40.6134435892654], [-73.91573181927129, 40.61183084451381], [-73.91811036679019, 40.61020839405809], [-73.91813814889971, 40.607645386907706], [-73.91935398484114, 40.60804292228766], [-73.91825269092949, 40.6072293506405], [-73.91950358918922, 40.60797490397521], [-73.91794980550603, 40.60620925217101], [-73.92158429099997, 40.607639084999875], [-73.92444144399987, 40.60579356899993], [-73.92624885099994, 40.60741352199989], [-73.92986880199989, 40.60507134099995], [-73.936046277, 40.609514868999966], [-73.93753749399987, 40.60855738999986], [-73.93934703399997, 40.610187140999884], [-73.94231175899985, 40.60826819699985], [-73.94354698000001, 40.6093465549999], [-73.9448277099998, 40.61604620899986], [-73.93686699599992, 40.620253280999904], [-73.92629604599998, 40.61311679899991], [-73.92640044599996, 40.61447853499988], [-73.9205269409999, 40.61394269499987], [-73.91690302699992, 40.61627422899993], [-73.91860649599992, 40.632090414999894], [-73.90464209220775, 40.62513319188153]]]}}, {\"id\": \"155\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.5348244979658148, \"OBJECTID\": 156, \"Shape_Leng\": 0.144476890476, \"Shape_Area\": 0.0010521217443, \"zone\": \"Mariners Harbor\", \"LocationID\": 156, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-74.17989568340026, 40.64526879647802], [-74.1795871372141, 40.64191768346404], [-74.17918728067563, 40.64298753999647], [-74.1763578661754, 40.64232722056289], [-74.17436572943608, 40.64513625864542], [-74.17133587281808, 40.642855765052], [-74.17298798800311, 40.640489105066244], [-74.17105017239898, 40.642593819129914], [-74.1660998989604, 40.642406587001155], [-74.1658011075589, 40.64122451144941], [-74.16494139041178, 40.64220899988932], [-74.16519041758224, 40.64064230577938], [-74.16167534694229, 40.640610784766764], [-74.1622846525209, 40.63876565277233], [-74.16034819993155, 40.63844372645346], [-74.15986410465037, 40.639927823692716], [-74.1592271162294, 40.637796119756985], [-74.15900957887435, 40.63882257790974], [-74.15718782782525, 40.63795080522895], [-74.15710126690709, 40.63927771860018], [-74.15707077305574, 40.63794713564092], [-74.15482932461856, 40.63772385541876], [-74.15469263864514, 40.63919396811424], [-74.15459670163321, 40.63734334406899], [-74.15279703511403, 40.637132754852544], [-74.15124827966567, 40.63940934288591], [-74.14855815092196, 40.63743783752723], [-74.14877792005885, 40.6388762511277], [-74.14312802471521, 40.63968037306379], [-74.14752618999992, 40.62938928799988], [-74.14554071999994, 40.62783252199995], [-74.14650354399993, 40.62420010099991], [-74.15231170399994, 40.61716275599988], [-74.15379116699992, 40.612255611999906], [-74.1623545779999, 40.618335637999955], [-74.17459706299995, 40.622364010999874], [-74.1955597527974, 40.635023986316575], [-74.19462310145185, 40.637391183563665], [-74.18652496897599, 40.64338866217145], [-74.18483594580836, 40.644128266338726], [-74.18365067064063, 40.642239641050644], [-74.1836674587201, 40.64461048505006], [-74.17989568340026, 40.64526879647802]]], [[[-74.15945602438187, 40.641448333324036], [-74.16111242522163, 40.64183545373729], [-74.16146036002637, 40.644294969882374], [-74.15743349200979, 40.643302857779], [-74.15945602438187, 40.641448333324036]]]]}}, {\"id\": \"156\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21303884823986907, \"OBJECTID\": 157, \"Shape_Leng\": 0.131300217777, \"Shape_Area\": 0.000354370128323, \"zone\": \"Maspeth\", \"LocationID\": 157, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8978323689999, 40.73516035399992], [-73.8885228599999, 40.734531159999904], [-73.8870771209998, 40.72463770199994], [-73.90010558699989, 40.72329624099992], [-73.899816102, 40.721236899999894], [-73.89739704899986, 40.7215512099999], [-73.89736598399989, 40.72046438299989], [-73.90062644399997, 40.719764053999874], [-73.90033904299986, 40.718019947999885], [-73.89834760099983, 40.719197677999894], [-73.897267579, 40.71814715699985], [-73.89901821099994, 40.71738212699993], [-73.89750226399987, 40.716383867999916], [-73.90477244599997, 40.71286579999984], [-73.90639010099999, 40.71736369199989], [-73.90965980099996, 40.71779315499991], [-73.91283037699982, 40.71536090699993], [-73.91077807399988, 40.71318759999986], [-73.92404011299993, 40.714008312999916], [-73.92352547251075, 40.71561660836783], [-73.92064258940593, 40.715773192028834], [-73.92232773978448, 40.71659264406108], [-73.91132095099978, 40.71962422999992], [-73.91384828299984, 40.72289841199989], [-73.905180579, 40.72895657699995], [-73.90041661699985, 40.73465058799992], [-73.8978323689999, 40.73516035399992]]]}}, {\"id\": \"157\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20634919674108937, \"OBJECTID\": 158, \"Shape_Leng\": 0.0548099905515, \"Shape_Area\": 0.000185568253002, \"zone\": \"Meatpacking/West Village West\", \"LocationID\": 158, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.0077911609999, 40.74197084499982], [-74.00515808199998, 40.74085808099991], [-74.0070902079999, 40.72877225299993], [-74.014390699162, 40.728463047760016], [-74.01408693559922, 40.730663173040284], [-74.01116195696464, 40.730450676349065], [-74.01099350879154, 40.732320417940606], [-74.01399957240955, 40.733332277517704], [-74.01072514326883, 40.733459523511115], [-74.01201792010478, 40.734063744456314], [-74.01081359479103, 40.734283443647534], [-74.0103930313456, 40.739174296059204], [-74.01265928208485, 40.74074763849748], [-74.0077911609999, 40.74197084499982]]]}}, {\"id\": \"158\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21316049433835588, \"OBJECTID\": 159, \"Shape_Leng\": 0.064363233206, \"Shape_Area\": 0.00017124910387, \"zone\": \"Melrose South\", \"LocationID\": 159, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91189154999992, 40.822141481999914], [-73.9012927759998, 40.820475442999914], [-73.9044655149999, 40.81228195999994], [-73.91768626999985, 40.8160812429999], [-73.92048117399989, 40.8145636869999], [-73.92309299899993, 40.81669772799995], [-73.92399951299991, 40.81543891199991], [-73.92662718199988, 40.816118958999915], [-73.91677400599986, 40.82438708699987], [-73.91189154999992, 40.822141481999914]]]}}, {\"id\": \"159\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.24346227332445658, \"OBJECTID\": 160, \"Shape_Leng\": 0.14151914297, \"Shape_Area\": 0.000571234977994, \"zone\": \"Middle Village\", \"LocationID\": 160, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.869707654, 40.72514643399992], [-73.85962920999985, 40.713975294999926], [-73.87016462499984, 40.70920774299987], [-73.86911007199988, 40.707089336999914], [-73.88088332699985, 40.7064539769999], [-73.87761134199985, 40.70716770499993], [-73.88006267699978, 40.71265257399993], [-73.90477244599997, 40.71286579999984], [-73.89750226399987, 40.716383867999916], [-73.89901821099994, 40.71738212699993], [-73.897267579, 40.71814715699985], [-73.89834760099983, 40.719197677999894], [-73.90033904299986, 40.718019947999885], [-73.90062644399997, 40.719764053999874], [-73.89736598399989, 40.72046438299989], [-73.89739704899986, 40.7215512099999], [-73.899816102, 40.721236899999894], [-73.89903191799989, 40.723856959999864], [-73.89426494199992, 40.723143188999856], [-73.8870771209998, 40.72463770199994], [-73.88720524999994, 40.727792996999874], [-73.87509154199992, 40.73067136299992], [-73.87146074199987, 40.729326373999854], [-73.869707654, 40.72514643399992]]]}}, {\"id\": \"160\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21907665299078571, \"OBJECTID\": 161, \"Shape_Leng\": 0.0358039100611, \"Shape_Area\": 7.19130682992e-05, \"zone\": \"Midtown Center\", \"LocationID\": 161, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97534377999993, 40.7552996949999], [-73.9783696929999, 40.752416226999905], [-73.98411754799999, 40.75484205299995], [-73.977686006, 40.76364440299992], [-73.97124277300004, 40.76093641799984], [-73.97534377999993, 40.7552996949999]]]}}, {\"id\": \"161\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2176980165487151, \"OBJECTID\": 162, \"Shape_Leng\": 0.0352698146219, \"Shape_Area\": 4.78936962112e-05, \"zone\": \"Midtown East\", \"LocationID\": 162, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9712170449999, 40.75519332399994], [-73.9744494109998, 40.75076644499991], [-73.9783696929999, 40.752416226999905], [-73.96980028899992, 40.76291351199995], [-73.9665833839999, 40.76155093499988], [-73.9712170449999, 40.75519332399994]]]}}, {\"id\": \"162\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2150476451227831, \"OBJECTID\": 163, \"Shape_Leng\": 0.0341768669752, \"Shape_Area\": 4.08567804196e-05, \"zone\": \"Midtown North\", \"LocationID\": 163, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98136213699995, 40.767836738999904], [-73.9736358469999, 40.7645699129999], [-73.96980028899992, 40.76291351199995], [-73.97124277300004, 40.76093641799984], [-73.98428230099987, 40.76479181499994], [-73.98136213699995, 40.767836738999904]]]}}, {\"id\": \"163\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21359687372617775, \"OBJECTID\": 164, \"Shape_Leng\": 0.0357716119908, \"Shape_Area\": 5.56318237441e-05, \"zone\": \"Midtown South\", \"LocationID\": 164, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98362519999995, 40.7439431149999], [-73.99051761299985, 40.746038637999895], [-73.98411754799999, 40.75484205299995], [-73.98088709899987, 40.753480988999875], [-73.98407636100002, 40.74910212499993], [-73.98085054499998, 40.74775036999995], [-73.98362519999995, 40.7439431149999]]]}}, {\"id\": \"164\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23905773324731538, \"OBJECTID\": 165, \"Shape_Leng\": 0.088608036213, \"Shape_Area\": 0.000353959317019, \"zone\": \"Midwood\", \"LocationID\": 165, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96014773499996, 40.628915184999876], [-73.95995565099992, 40.627778671999884], [-73.94732672199994, 40.629166566999835], [-73.94451833400001, 40.61971364699989], [-73.94651373199997, 40.61949401899995], [-73.94573691999982, 40.61536340699989], [-73.95026514899993, 40.61392435199983], [-73.94978380499997, 40.611371749999854], [-73.96237947799978, 40.609986373999966], [-73.96471983399981, 40.62237296099993], [-73.96864510599995, 40.6219416949999], [-73.97136622199992, 40.62892916199996], [-73.96040734299989, 40.6301625309999], [-73.96014773499996, 40.628915184999876]]]}}, {\"id\": \"165\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20308903256101277, \"OBJECTID\": 166, \"Shape_Leng\": 0.0688237542871, \"Shape_Area\": 0.000255276270405, \"zone\": \"Morningside Heights\", \"LocationID\": 166, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95707738499995, 40.8142980939999], [-73.95210125199992, 40.811442853999935], [-73.9535757779999, 40.8094766979999], [-73.95496657099991, 40.81006455499994], [-73.95964685399987, 40.80115642299993], [-73.97110765876137, 40.80579013958964], [-73.96203105797969, 40.81808633074085], [-73.95707738499995, 40.8142980939999]]]}}, {\"id\": \"166\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23551928942158487, \"OBJECTID\": 167, \"Shape_Leng\": 0.0908159737292, \"Shape_Area\": 0.000167529391193, \"zone\": \"Morrisania/Melrose\", \"LocationID\": 167, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89690017899987, 40.8310333359999], [-73.8899110649999, 40.830124191999865], [-73.88609624199987, 40.82765577699989], [-73.89483522899992, 40.82624684099993], [-73.89548522799996, 40.82824854099993], [-73.90093510699997, 40.82792810199992], [-73.9012927759998, 40.820475442999914], [-73.91181615799985, 40.82203502199991], [-73.91677400599986, 40.82438708699987], [-73.91541108199998, 40.82577390899993], [-73.91130974499993, 40.824848557999886], [-73.91169430299985, 40.82775871599993], [-73.908959903, 40.83022623299989], [-73.90553116799988, 40.82910413299989], [-73.90122534999992, 40.83549991099991], [-73.89697220299995, 40.8343452539999], [-73.89690017899987, 40.8310333359999]]]}}, {\"id\": \"167\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20540890156612437, \"OBJECTID\": 168, \"Shape_Leng\": 0.114517434155, \"Shape_Area\": 0.000547058100444, \"zone\": \"Mott Haven/Port Morris\", \"LocationID\": 168, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.92480953899991, 40.81565901999993], [-73.92309299899993, 40.81669772799995], [-73.92048117399989, 40.8145636869999], [-73.91768626999985, 40.8160812429999], [-73.9044655149999, 40.81228195999994], [-73.90229231992774, 40.80456477614167], [-73.91168831241185, 40.79662376988872], [-73.91904978748826, 40.798993935843235], [-73.9229271008289, 40.80237329482137], [-73.92762788658843, 40.80269566548149], [-73.93252708785968, 40.80882328177534], [-73.93242632000944, 40.81411751897961], [-73.93027812699984, 40.813248024999965], [-73.92731653099999, 40.81855963399991], [-73.92518486499998, 40.818012668999884], [-73.92662718199988, 40.816118958999915], [-73.92480953899991, 40.81565901999993]]], [[[-73.89833036270552, 40.80241282093997], [-73.89646668834577, 40.80079047089134], [-73.90021004993142, 40.79926415589601], [-73.90003735815736, 40.800908742050225], [-73.89833036270552, 40.80241282093997]]], [[[-73.89680883223778, 40.795808445159786], [-73.89796839783742, 40.79564483916198], [-73.89919434249981, 40.796502456018175], [-73.8978825324018, 40.79711653214704], [-73.89680883223778, 40.795808445159786]]]]}}, {\"id\": \"168\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23977688244305467, \"OBJECTID\": 169, \"Shape_Leng\": 0.0601046580332, \"Shape_Area\": 0.000146027537733, \"zone\": \"Mount Hope\", \"LocationID\": 169, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89816577499982, 40.85514637599989], [-73.89863543699985, 40.85390697499993], [-73.89560883299981, 40.85457036999986], [-73.901344046, 40.84442875099994], [-73.90483121999989, 40.84480034199987], [-73.9050151499998, 40.842404697999875], [-73.91689747599989, 40.84511773399991], [-73.91416184199993, 40.845179187999925], [-73.90557007299988, 40.855787321999884], [-73.89816577499982, 40.85514637599989]]]}}, {\"id\": \"169\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21336368006251963, \"OBJECTID\": 170, \"Shape_Leng\": 0.0457690091051, \"Shape_Area\": 7.43150675972e-05, \"zone\": \"Murray Hill\", \"LocationID\": 170, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97203331099985, 40.745804967999945], [-73.97350835000003, 40.7437820339999], [-73.97812378199983, 40.745727172999885], [-73.98085965399994, 40.74196976799989], [-73.98407485299995, 40.74332471799989], [-73.98085054399985, 40.74775036999991], [-73.98407636100002, 40.74910212499993], [-73.98088709899987, 40.753480988999875], [-73.9744494109998, 40.75076644499991], [-73.9758346669999, 40.74886314599996], [-73.97121971999988, 40.74692154099996], [-73.97203331099985, 40.745804967999945]]]}}, {\"id\": \"170\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2337356586482648, \"OBJECTID\": 171, \"Shape_Leng\": 0.107353876244, \"Shape_Area\": 0.000520099538161, \"zone\": \"Murray Hill-Queens\", \"LocationID\": 171, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79317944599988, 40.77040317699989], [-73.79343991199988, 40.760764127999934], [-73.79076004599987, 40.76098576899991], [-73.79031517, 40.75782345099988], [-73.81104963199985, 40.761417020999964], [-73.82063593299985, 40.75887226799989], [-73.82657481899986, 40.772345397999885], [-73.82331286899986, 40.77957865999986], [-73.81452997799975, 40.77888817499992], [-73.81485850699985, 40.776485922999875], [-73.79853009399987, 40.775190728999924], [-73.79317944599988, 40.77040317699989]]]}}, {\"id\": \"171\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.281725262992855, \"OBJECTID\": 172, \"Shape_Leng\": 0.118476116148, \"Shape_Area\": 0.000658402501406, \"zone\": \"New Dorp/Midland Beach\", \"LocationID\": 172, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08654858694766, 40.56963231913558], [-74.08621632224266, 40.56853269814713], [-74.09054960106333, 40.567083597211294], [-74.09938031432728, 40.5589641933612], [-74.11376161199998, 40.56649718899993], [-74.11636336399992, 40.56391563799986], [-74.12489904399992, 40.568949141999916], [-74.12150213699994, 40.571721558999904], [-74.1215372899999, 40.57515325299986], [-74.10525775499997, 40.58592143599986], [-74.09844000899997, 40.581241705999915], [-74.10039083499996, 40.579688665999925], [-74.08597647499992, 40.57223697099986], [-74.08495715819193, 40.57125984002107], [-74.08654858694766, 40.56963231913558]]]}}, {\"id\": \"0\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2185816410601061, \"OBJECTID\": 1, \"Shape_Leng\": 0.116357453189, \"Shape_Area\": 0.0007823067885, \"zone\": \"Newark Airport\", \"LocationID\": 1, \"borough\": \"EWR\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.18445299999996, 40.6949959999999], [-74.18284199999994, 40.70346499999988], [-74.17565999999994, 40.707379999999866], [-74.16081899999995, 40.70764099999991], [-74.15306999999996, 40.7052829999999], [-74.17283399999997, 40.67656399999988], [-74.17862999999994, 40.671037999999896], [-74.18503799999996, 40.67335999999989], [-74.18991599999998, 40.68154099999988], [-74.18922299999997, 40.688623999999905], [-74.18563199999994, 40.69164799999987], [-74.18445299999996, 40.6949959999999]]]}}, {\"id\": \"172\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2269099119196972, \"OBJECTID\": 173, \"Shape_Leng\": 0.0600639661414, \"Shape_Area\": 0.000178329851357, \"zone\": \"North Corona\", \"LocationID\": 173, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85540366099997, 40.75834030899989], [-73.85312635499996, 40.75510934299989], [-73.85579909099987, 40.75440952799994], [-73.85442575299999, 40.748851725999884], [-73.86790963899986, 40.74493371599989], [-73.87365721799983, 40.756803485999946], [-73.85540366099997, 40.75834030899989]]]}}, {\"id\": \"173\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.25843157826105, \"OBJECTID\": 174, \"Shape_Leng\": 0.0678177559602, \"Shape_Area\": 0.000155928664422, \"zone\": \"Norwood\", \"LocationID\": 174, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87772817699984, 40.88345419499995], [-73.87812594399992, 40.88215382599989], [-73.87467601499999, 40.88156285199993], [-73.87494589799991, 40.87994766899988], [-73.87024100099985, 40.88022935299995], [-73.87619224999979, 40.87047671099987], [-73.8830900319999, 40.86659150899993], [-73.88146889699985, 40.868573645999916], [-73.88705142599984, 40.884349575999906], [-73.88124422499992, 40.88270963299992], [-73.87803562399996, 40.887118321999885], [-73.87772817699984, 40.88345419499995]]]}}, {\"id\": \"174\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2016697843747185, \"OBJECTID\": 175, \"Shape_Leng\": 0.134897665102, \"Shape_Area\": 0.000504920726134, \"zone\": \"Oakland Gardens\", \"LocationID\": 175, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7562560639999, 40.7584304689999], [-73.75230259399993, 40.759455198999916], [-73.75004764499985, 40.75840465799996], [-73.74384322099985, 40.74769009599988], [-73.74625984399985, 40.74671147499993], [-73.74477587099986, 40.74328109699994], [-73.75347299999999, 40.74240484399992], [-73.75098230099987, 40.74002146199996], [-73.7439619359999, 40.74078912699986], [-73.74306820799991, 40.7384408179999], [-73.74016951799993, 40.73848189799986], [-73.75048295699996, 40.72922129399996], [-73.75669771299987, 40.7262287109999], [-73.76564374500005, 40.732533126999954], [-73.7636997919999, 40.73286411699993], [-73.76510920699994, 40.73455658499992], [-73.76248609299982, 40.736730981999884], [-73.76772991399994, 40.74577622599993], [-73.75684127399992, 40.74932207799991], [-73.76184343999986, 40.75553726899989], [-73.76239510599996, 40.759496997999854], [-73.75800131799996, 40.76037986699988], [-73.7562560639999, 40.7584304689999]]]}}, {\"id\": \"175\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.17177692481372772, \"OBJECTID\": 176, \"Shape_Leng\": 0.151995190703, \"Shape_Area\": 0.000657782126267, \"zone\": \"Oakwood\", \"LocationID\": 176, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.12187198599999, 40.57510757499989], [-74.12150213699994, 40.571721558999904], [-74.12489904399992, 40.568949141999916], [-74.11636336299996, 40.56391563799985], [-74.11376161199998, 40.56649718899993], [-74.09992914200652, 40.55907068582615], [-74.10140516910823, 40.555538526164405], [-74.10309839111532, 40.555904641451136], [-74.10484447636338, 40.55302581455626], [-74.10768623464054, 40.553976671477514], [-74.11301046102231, 40.547780065214994], [-74.12730057799996, 40.55729517199993], [-74.13321002499998, 40.552383875999936], [-74.13885092699991, 40.5569511789999], [-74.13170009, 40.56453574899991], [-74.1443874499999, 40.56970609999989], [-74.14418588199999, 40.57203740999988], [-74.13261022999991, 40.573530162999894], [-74.12708814799998, 40.57636910599995], [-74.12187198599999, 40.57510757499989]]]}}, {\"id\": \"176\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.28871928482258397, \"OBJECTID\": 177, \"Shape_Leng\": 0.0715805407056, \"Shape_Area\": 0.000198894762248, \"zone\": \"Ocean Hill\", \"LocationID\": 177, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90287079599995, 40.6786314149999], [-73.90004626199993, 40.67689308999995], [-73.92275257199998, 40.667097332999894], [-73.92164666399992, 40.67886992299994], [-73.9163011989999, 40.67857711199991], [-73.91804607, 40.68721324799989], [-73.90287079599995, 40.6786314149999]]]}}, {\"id\": \"177\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21526255079957998, \"OBJECTID\": 178, \"Shape_Leng\": 0.0682528770739, \"Shape_Area\": 0.000175805409143, \"zone\": \"Ocean Parkway South\", \"LocationID\": 178, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97080408199997, 40.62593266699988], [-73.96864510599995, 40.6219416949999], [-73.96471983399981, 40.62237296099993], [-73.96285973999984, 40.61254948599989], [-73.96478112799986, 40.61233092199988], [-73.96429649899991, 40.60977642599992], [-73.97403097799986, 40.608702309999906], [-73.97517176999993, 40.61472418599991], [-73.97290326899996, 40.61415296799988], [-73.97335879399992, 40.616541714999904], [-73.97537335500002, 40.6157547219999], [-73.97785009299986, 40.617287605999906], [-73.97539380499998, 40.62076998699995], [-73.97705352899999, 40.62153252199987], [-73.9772614999999, 40.625852772999906], [-73.97080408199997, 40.62593266699988]]]}}, {\"id\": \"178\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20926966950433043, \"OBJECTID\": 179, \"Shape_Leng\": 0.0713808644377, \"Shape_Area\": 0.000183893478476, \"zone\": \"Old Astoria\", \"LocationID\": 179, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93290347544752, 40.77794449857707], [-73.92782147400004, 40.776697526999946], [-73.91770217299991, 40.7703931119999], [-73.91995596899991, 40.768440064999865], [-73.91849336999996, 40.76773465699995], [-73.91977764799998, 40.7661521839999], [-73.92218946199995, 40.767267356999895], [-73.92425288799978, 40.76472456399995], [-73.93344468499993, 40.768817789999915], [-73.93493511099992, 40.76701302399987], [-73.93618379699991, 40.767691800999934], [-73.93424609600005, 40.771029275999894], [-73.93746638537067, 40.77253149512518], [-73.93765483090819, 40.775085685383594], [-73.93508232492144, 40.77794423372776], [-73.93290347544752, 40.77794449857707]]]}}, {\"id\": \"179\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22245118569541122, \"OBJECTID\": 180, \"Shape_Leng\": 0.0805277671523, \"Shape_Area\": 0.000247682193353, \"zone\": \"Ozone Park\", \"LocationID\": 180, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84119605699982, 40.68335136599989], [-73.83334819299995, 40.66677318699994], [-73.86131863599985, 40.6750212689999], [-73.86334943599995, 40.67935163999989], [-73.85878576899987, 40.68063046299989], [-73.85819836099995, 40.67943084899987], [-73.85392389199987, 40.679686155999875], [-73.85099089299987, 40.68056836699987], [-73.85197864599989, 40.682564210999864], [-73.84786817599985, 40.681486831999905], [-73.84119605699982, 40.68335136599989]]]}}, {\"id\": \"180\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21747093307404936, \"OBJECTID\": 181, \"Shape_Leng\": 0.0895372399547, \"Shape_Area\": 0.000306890292807, \"zone\": \"Park Slope\", \"LocationID\": 181, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97583132299987, 40.66581507999991], [-73.97965353499985, 40.661243232999915], [-73.98223034199995, 40.66214149299991], [-73.985086794, 40.65901328799989], [-73.988716511, 40.66240493799995], [-73.9906354539999, 40.66057908799997], [-73.99692689899985, 40.664374914999925], [-73.99490350199993, 40.66678543499988], [-73.99277080699981, 40.665507280999876], [-73.97804289599988, 40.68485661499987], [-73.97375598499991, 40.68303401999988], [-73.97618051399984, 40.67806432099995], [-73.96949738899991, 40.67590637999996], [-73.96871822199984, 40.67392423399988], [-73.97583132299987, 40.66581507999991]]]}}, {\"id\": \"181\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20946044788320664, \"OBJECTID\": 182, \"Shape_Leng\": 0.0393606369479, \"Shape_Area\": 9.10644156732e-05, \"zone\": \"Parkchester\", \"LocationID\": 182, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85465135700002, 40.83388802699987], [-73.86271114399993, 40.83299150899997], [-73.86448011799995, 40.84108779699994], [-73.85333524999986, 40.84247015499987], [-73.85168459699997, 40.8342010519999], [-73.85465135700002, 40.83388802699987]]]}}, {\"id\": \"182\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2847370571411895, \"OBJECTID\": 183, \"Shape_Leng\": 0.0398262960348, \"Shape_Area\": 9.51929451066e-05, \"zone\": \"Pelham Bay\", \"LocationID\": 183, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83676643699985, 40.84338596999994], [-73.83463970299987, 40.853817993999904], [-73.82860197199996, 40.85571784799988], [-73.82721751200002, 40.853250512999935], [-73.82799676999998, 40.84862431399989], [-73.82611300699996, 40.84533869399991], [-73.83676643699985, 40.84338596999994]]]}}, {\"id\": \"183\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.1773082366260968, \"OBJECTID\": 184, \"Shape_Leng\": 0.260815683043, \"Shape_Area\": 0.00198883363103, \"zone\": \"Pelham Bay Park\", \"LocationID\": 184, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.80728647099995, 40.885931813999875], [-73.79322870891143, 40.88282259495222], [-73.80087270458245, 40.87159011989393], [-73.80252284756597, 40.87050664469679], [-73.80339117523091, 40.87250503763108], [-73.80431849468228, 40.87115974106797], [-73.80302203685277, 40.86915511979648], [-73.80588410012888, 40.869068502184916], [-73.80841596818787, 40.87118969336752], [-73.80464604502912, 40.86664711643685], [-73.80300388108948, 40.86704183656889], [-73.80225816157765, 40.8656082064109], [-73.79780377219292, 40.87225613227397], [-73.794790446816, 40.873200484033966], [-73.79384702788171, 40.87859445979086], [-73.7895587294431, 40.881261284915404], [-73.789523093688, 40.87899812706048], [-73.7843816810755, 40.878420531670514], [-73.78719087560506, 40.872730553729134], [-73.78508139658601, 40.87358996322568], [-73.78526152075709, 40.87174532233236], [-73.78358738486025, 40.87378095468029], [-73.78320175426292, 40.87067036085376], [-73.78560245305444, 40.86834165222368], [-73.78795446145405, 40.86984541906357], [-73.79139032586131, 40.868141307466146], [-73.79190454927502, 40.86082956271207], [-73.79454562318031, 40.85888825916121], [-73.79435005934273, 40.85652702380392], [-73.7987225560008, 40.85491440117259], [-73.79780522990592, 40.85194959859714], [-73.80040387554423, 40.84810822100142], [-73.80417557932397, 40.853340120189074], [-73.80478410547944, 40.86154633053462], [-73.80592716427573, 40.86194119699028], [-73.80612390783546, 40.85981220661901], [-73.80783343870087, 40.860318133305384], [-73.80768048107562, 40.858538063289515], [-73.81411414337536, 40.86373145821164], [-73.81641392102723, 40.86118062770394], [-73.81273146495035, 40.85887754194771], [-73.81230602708418, 40.854162026473176], [-73.81684183599076, 40.85362521315728], [-73.81712409042197, 40.85126957344483], [-73.81507165904237, 40.84913361979451], [-73.82495697699986, 40.846105444999864], [-73.82517999, 40.842700058999874], [-73.82624132899997, 40.84299359399996], [-73.82799676999987, 40.8486243129999], [-73.82723026100001, 40.853350756999895], [-73.82860197199996, 40.85571784799988], [-73.83283502299986, 40.8560358449999], [-73.82834347399992, 40.86089202599988], [-73.82699653199998, 40.85908778599986], [-73.82037552999996, 40.86103300499993], [-73.81578764099999, 40.8657580609999], [-73.821266938, 40.869071605999885], [-73.82329599399986, 40.87309741799988], [-73.81992586499996, 40.88211933299987], [-73.82027432999985, 40.885550699999925], [-73.81442762200005, 40.88677196499989], [-73.80728647099995, 40.885931813999875]]], [[[-73.78650554049733, 40.880940134479225], [-73.78578002611279, 40.88036396266133], [-73.78742039384403, 40.87977089028999], [-73.787127834652, 40.88092346317125], [-73.78650554049733, 40.880940134479225]]], [[[-73.78103351104939, 40.87648400204775], [-73.78120649649391, 40.87628502546013], [-73.78132704118002, 40.87636132654635], [-73.78112767478156, 40.87665268050809], [-73.78103351104939, 40.87648400204775]]], [[[-73.78241811865315, 40.87492327042177], [-73.78265785360905, 40.87479265669398], [-73.78280207510406, 40.87494886620542], [-73.78247604125525, 40.87516813337884], [-73.78241811865315, 40.87492327042177]]], [[[-73.77435244645233, 40.874169955874805], [-73.77453640510954, 40.87471001543833], [-73.77409538292126, 40.87489618448013], [-73.7740318153443, 40.874394774956215], [-73.77435244645233, 40.874169955874805]]], [[[-73.78605394964902, 40.873782646192794], [-73.78618924358646, 40.87359972769974], [-73.78633325996694, 40.87367772488745], [-73.7861946371647, 40.87388531941464], [-73.78605394964902, 40.873782646192794]]], [[[-73.78648510546586, 40.87320925495041], [-73.7867268909535, 40.87307453926568], [-73.78626202153426, 40.87339368663393], [-73.78632869488987, 40.873271144554096], [-73.78648510546586, 40.87320925495041]]], [[[-73.77080975398195, 40.871549946847985], [-73.76990710668544, 40.87047694447026], [-73.77288042665229, 40.871245263430254], [-73.77217786492288, 40.87198111068936], [-73.77080975398195, 40.871549946847985]]], [[[-73.76668965841925, 40.86709778162448], [-73.76709907439783, 40.8667045727482], [-73.76779449382023, 40.86678117376544], [-73.76760072096499, 40.86749638654143], [-73.76668965841925, 40.86709778162448]]], [[[-73.76964988627088, 40.86548557242442], [-73.76979148260493, 40.86512823356445], [-73.77022931630273, 40.865138939396005], [-73.770068972998, 40.86552587618341], [-73.76964988627088, 40.86548557242442]]], [[[-73.78401249138903, 40.863131994074394], [-73.78376770348979, 40.86261060846434], [-73.78469165850701, 40.86254278990822], [-73.7842865050008, 40.86320508878048], [-73.78401249138903, 40.863131994074394]]], [[[-73.78312589594455, 40.862856167300684], [-73.7828819783119, 40.862406433091266], [-73.7831450428813, 40.86206867890555], [-73.78328053505923, 40.86278070671258], [-73.78312589594455, 40.862856167300684]]], [[[-73.77460156350928, 40.86206904745972], [-73.7749432115575, 40.861391499889486], [-73.77529297955127, 40.86123200821588], [-73.77485681624742, 40.86198363433426], [-73.77460156350928, 40.86206904745972]]], [[[-73.77290231992438, 40.86120858327854], [-73.77323150244695, 40.86074716979338], [-73.77351264165688, 40.861241184087085], [-73.77310632921937, 40.861614575032775], [-73.77290231992438, 40.86120858327854]]], [[[-73.8022229535527, 40.84163481314409], [-73.80263811156135, 40.841081153267076], [-73.80694608641589, 40.84146244718637], [-73.80680801372397, 40.84248913998752], [-73.8022229535527, 40.84163481314409]]]]}}, {\"id\": \"184\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.24992585526126967, \"OBJECTID\": 185, \"Shape_Leng\": 0.0861619681636, \"Shape_Area\": 0.000228537100218, \"zone\": \"Pelham Parkway\", \"LocationID\": 185, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84004614299985, 40.857806268999866], [-73.83499552599994, 40.85657120099989], [-73.84049767399992, 40.8541841049999], [-73.84494053799982, 40.8482182209999], [-73.84430846999992, 40.85119046399989], [-73.85304526899994, 40.849547331999865], [-73.85457388299992, 40.85150834599995], [-73.86250923799993, 40.84836228099987], [-73.86303938099996, 40.850611306999944], [-73.86860515599989, 40.85206864599991], [-73.86990421600002, 40.85577329299987], [-73.8681883059999, 40.858063833999964], [-73.84004614299985, 40.857806268999866]]]}}, {\"id\": \"185\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21923492475874293, \"OBJECTID\": 186, \"Shape_Leng\": 0.0246963902234, \"Shape_Area\": 3.70729416953e-05, \"zone\": \"Penn Station/Madison Sq West\", \"LocationID\": 186, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99096832799995, 40.74542088999985], [-73.99709902899994, 40.74720510199991], [-73.99346417699986, 40.75219005499987], [-73.98776905899994, 40.749787028999926], [-73.99096832799995, 40.74542088999985]]]}}, {\"id\": \"186\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2644006948465547, \"OBJECTID\": 187, \"Shape_Leng\": 0.126868431324, \"Shape_Area\": 0.000421195755741, \"zone\": \"Port Richmond\", \"LocationID\": 187, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-74.13435269573621, 40.64188679740515], [-74.12890729033458, 40.64119482758617], [-74.1268887269999, 40.637210087999925], [-74.12867047299994, 40.63734826499993], [-74.13047428999985, 40.6308855089999], [-74.13806393799992, 40.627568470999904], [-74.13581008299997, 40.62506433999991], [-74.13826358199992, 40.624243769999936], [-74.13982184799993, 40.61629982499988], [-74.14603562099998, 40.61701277699996], [-74.14696864199988, 40.610151355999896], [-74.15018666499996, 40.60949781099991], [-74.15379116699992, 40.612255611999906], [-74.15231170399994, 40.61716275599988], [-74.14622812499992, 40.6249253619999], [-74.14554071999994, 40.62783252199995], [-74.14752618999992, 40.62938928799988], [-74.14478228999994, 40.637953779999904], [-74.14229766980965, 40.64030580709358], [-74.13435269573621, 40.64188679740515]]], [[[-74.12117246275159, 40.64166914557247], [-74.12128074797168, 40.64143249599354], [-74.12131194814832, 40.64145253116248], [-74.12117246275159, 40.64166914557247]]], [[[-74.1215534360538, 40.64161162832474], [-74.12176052602824, 40.64157431506241], [-74.12175764698422, 40.64142342422718], [-74.12177911197513, 40.641592547117035], [-74.1215534360538, 40.64161162832474]]], [[[-74.12107727405213, 40.64160742058869], [-74.12118039503834, 40.64138344518971], [-74.12121470277087, 40.64139683999368], [-74.12110124737936, 40.64161328229901], [-74.12107727405213, 40.64160742058869]]], [[[-74.12130529083001, 40.64160070313003], [-74.12137332919554, 40.64147647512165], [-74.12132617111645, 40.64160656922885], [-74.12130529083001, 40.64160070313003]]], [[[-74.12185329597821, 40.64158069227959], [-74.12180661091023, 40.64141215903768], [-74.1218414704778, 40.641405894387034], [-74.12199083807539, 40.64155463949942], [-74.12185329597821, 40.64158069227959]]], [[[-74.12270319595869, 40.64137664893253], [-74.12272114424832, 40.64134762377306], [-74.12276203509289, 40.641426635297144], [-74.12271103075855, 40.641433755252706], [-74.12270319595869, 40.64137664893253]]], [[[-74.12613276551227, 40.6412643880282], [-74.1259954246122, 40.640231242814394], [-74.12636808688976, 40.64133238048696], [-74.12564026870598, 40.64141849208226], [-74.12613276551227, 40.6412643880282]]]]}}, {\"id\": \"188\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21968928524172474, \"OBJECTID\": 189, \"Shape_Leng\": 0.0486743306502, \"Shape_Area\": 0.000101427594033, \"zone\": \"Prospect Heights\", \"LocationID\": 189, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96070974499996, 40.673121393999985], [-73.96131877999991, 40.67140667599988], [-73.96892332199981, 40.67293755299988], [-73.96949738899991, 40.67590637999996], [-73.97618051399984, 40.67806432099995], [-73.97375598499991, 40.68303401999988], [-73.96134655899995, 40.6804635239999], [-73.96318238899983, 40.675529503999876], [-73.96007428300001, 40.67487807099994], [-73.96070974499996, 40.673121393999985]]]}}, {\"id\": \"189\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2160607750894813, \"OBJECTID\": 190, \"Shape_Leng\": 0.0743294660829, \"Shape_Area\": 0.000270873666522, \"zone\": \"Prospect Park\", \"LocationID\": 190, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96307724199983, 40.66213019499987], [-73.96190028499997, 40.65487096199992], [-73.96621321999996, 40.653189404999885], [-73.96465438699984, 40.650887727999894], [-73.97139631700004, 40.64825778599986], [-73.97436701299992, 40.65818790399992], [-73.98022902599985, 40.660985325999896], [-73.96865541400003, 40.673023838999924], [-73.96258784899996, 40.671711596999884], [-73.96095595499996, 40.66328505499992], [-73.96307724199983, 40.66213019499987]]]}}, {\"id\": \"187\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2188398180407007, \"OBJECTID\": 188, \"Shape_Leng\": 0.0977983968367, \"Shape_Area\": 0.000313003032771, \"zone\": \"Prospect-Lefferts Gardens\", \"LocationID\": 188, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95729172699996, 40.6643026319999], [-73.94269628999989, 40.664055128999934], [-73.94282892199992, 40.6628474859999], [-73.93068465499985, 40.66362047699992], [-73.92646424299983, 40.659814361999956], [-73.92834017599988, 40.660060035999884], [-73.92799465199988, 40.65680210099988], [-73.9500666489999, 40.65542299699989], [-73.94957113100006, 40.65080789999991], [-73.9587078989999, 40.65038727299989], [-73.95986103199981, 40.65563478899989], [-73.96190028499997, 40.65487096199992], [-73.96307724199983, 40.66213019499987], [-73.95729172699996, 40.6643026319999]]]}}, {\"id\": \"190\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.30035481171136735, \"OBJECTID\": 191, \"Shape_Leng\": 0.130932637664, \"Shape_Area\": 0.000688460603185, \"zone\": \"Queens Village\", \"LocationID\": 191, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.73016587199996, 40.72395859599988], [-73.72816233910697, 40.7230841398913], [-73.73032628984046, 40.722157296892334], [-73.7269714679152, 40.710714504016735], [-73.72677759516957, 40.7030592911994], [-73.74590871000004, 40.70228727399988], [-73.74500119100004, 40.70026221299988], [-73.74773012199991, 40.69955313699991], [-73.74618157999997, 40.696121925999904], [-73.74863325999986, 40.6954447609999], [-73.75205066499986, 40.70522802799995], [-73.75047058199985, 40.70563993099994], [-73.75725671499985, 40.71813860199992], [-73.7523343839998, 40.72026246099993], [-73.75669771299987, 40.7262287109999], [-73.74672140400006, 40.73247715099991], [-73.74555667199989, 40.73063689799994], [-73.7314721509999, 40.727411794999924], [-73.73016587199996, 40.72395859599988]]]}}, {\"id\": \"191\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20838730672371972, \"OBJECTID\": 192, \"Shape_Leng\": 0.11196461971, \"Shape_Area\": 0.000373808019026, \"zone\": \"Queensboro Hill\", \"LocationID\": 192, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82239598999983, 40.74986422299986], [-73.81416171999987, 40.74613857999992], [-73.80750599299991, 40.750136586999936], [-73.80287666499983, 40.749765149999895], [-73.8033521419999, 40.74643196499987], [-73.7946756879998, 40.74721192799994], [-73.79453507400001, 40.74450763799989], [-73.79723084999992, 40.74169550699993], [-73.79640218700004, 40.73828235799987], [-73.8243501109999, 40.73975360599992], [-73.82946751999994, 40.737024481999896], [-73.83143025500006, 40.73913573299994], [-73.83113122799989, 40.74233799999994], [-73.83586131899989, 40.743227984999926], [-73.83392218299996, 40.74753207499989], [-73.82584440999999, 40.74934839099993], [-73.82426493799994, 40.75183288799992], [-73.82239598999983, 40.74986422299986]]]}}, {\"id\": \"192\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20413753185240252, \"OBJECTID\": 193, \"Shape_Leng\": 0.0655299584801, \"Shape_Area\": 0.000145591657944, \"zone\": \"Queensbridge/Ravenswood\", \"LocationID\": 193, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93522718099996, 40.76952572899989], [-73.93858976899998, 40.766817358999916], [-73.93398092199995, 40.76282598199987], [-73.944052652, 40.75231325999992], [-73.95080763809156, 40.75526368005708], [-73.9413117494144, 40.76691800476961], [-73.93424609599985, 40.771029274999925], [-73.93522718099996, 40.76952572899989]]]}}, {\"id\": \"193\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21631961892463786, \"OBJECTID\": 194, \"Shape_Leng\": 0.0817938353532, \"Shape_Area\": 0.000273481702719, \"zone\": \"Randalls Island\", \"LocationID\": 194, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91639852914044, 40.79768484203896], [-73.91378385370595, 40.793836285482065], [-73.92412113996787, 40.78252307401082], [-73.92810118993664, 40.78092032002105], [-73.93598226003293, 40.7839423680532], [-73.93084178719639, 40.79099143000079], [-73.9261041024332, 40.79073160730996], [-73.92560814781015, 40.79189695828146], [-73.9278688922052, 40.79095483831721], [-73.9283194359912, 40.79309126608271], [-73.9254929354651, 40.80195616882528], [-73.92263378594883, 40.80186873814113], [-73.91639852914044, 40.79768484203896]]]}}, {\"id\": \"194\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2211924311571713, \"OBJECTID\": 195, \"Shape_Leng\": 0.131308244067, \"Shape_Area\": 0.000801651953985, \"zone\": \"Red Hook\", \"LocationID\": 195, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.0071710969999, 40.68635700999991], [-74.00485241599992, 40.685668130999886], [-74.00596041099988, 40.683362870999936], [-73.9986188799999, 40.67183017199989], [-74.00289483876944, 40.66734846594933], [-74.0058636618068, 40.66799376258568], [-74.00683236078251, 40.666049645526776], [-74.00696707834247, 40.666052182356665], [-74.00543926529353, 40.6709214682384], [-74.00754821618384, 40.66705174573358], [-74.00708724221813, 40.668591358535764], [-74.00985411855632, 40.668523628335336], [-74.01154017543327, 40.6650859197566], [-74.01574872882654, 40.6645683362916], [-74.01690267361069, 40.66484660233888], [-74.01933946871777, 40.671624948422625], [-74.01751037086694, 40.671034530984144], [-74.01648665554487, 40.664930508953816], [-74.01227967872792, 40.66573241415882], [-74.01033845947926, 40.669078195448], [-74.01162527521356, 40.670581028275876], [-74.01424926931875, 40.66972860473708], [-74.01175350509699, 40.670657757578326], [-74.0151998057007, 40.6707540276644], [-74.01423289452782, 40.67213000051297], [-74.01568625497515, 40.671038834928844], [-74.0129758779219, 40.67332791872093], [-74.01632099922307, 40.6729082695637], [-74.01496392184576, 40.67467763029424], [-74.0188009477692, 40.6722507648487], [-74.01727138470127, 40.67360864580548], [-74.0188577695892, 40.6751022924089], [-74.01791198226178, 40.676510462801936], [-74.01995197864673, 40.67710301044993], [-74.01825267004557, 40.67855288672587], [-74.01928128101008, 40.67964814029763], [-74.01282000316418, 40.68362241624865], [-74.01410549647976, 40.68171251937541], [-74.013018354221, 40.68041237401427], [-74.00967108558879, 40.683268727715], [-74.01193259955522, 40.683887749077535], [-74.0071710969999, 40.68635700999991]]]}}, {\"id\": \"195\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20640880709365245, \"OBJECTID\": 196, \"Shape_Leng\": 0.0832421106332, \"Shape_Area\": 0.0001974713336, \"zone\": \"Rego Park\", \"LocationID\": 196, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85762923599995, 40.728606921999884], [-73.8594198799999, 40.726464837999885], [-73.85727108100001, 40.72568129799988], [-73.85805416899993, 40.72444540599986], [-73.86203833799982, 40.72595678499994], [-73.8586809759999, 40.72261243999988], [-73.85729287299985, 40.711467676999874], [-73.85975932900001, 40.711845058999856], [-73.86003789899996, 40.71493223599987], [-73.86929270499994, 40.72443592899991], [-73.87146074199987, 40.729326373999854], [-73.87509154199992, 40.73067136299992], [-73.85886555799988, 40.73593913899987], [-73.85625665099991, 40.73095912299993], [-73.85851097899999, 40.73028680199993], [-73.85762923599995, 40.728606921999884]]]}}, {\"id\": \"196\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21599472468336775, \"OBJECTID\": 197, \"Shape_Leng\": 0.108568532229, \"Shape_Area\": 0.000504689160432, \"zone\": \"Richmond Hill\", \"LocationID\": 197, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83696237799982, 40.70421245199992], [-73.83152109599999, 40.70138962899994], [-73.82245890399994, 40.70365553099989], [-73.81637824499992, 40.70245196399987], [-73.81063298299996, 40.69194738299988], [-73.82501128899997, 40.68795231799991], [-73.82599347399987, 40.689953661999894], [-73.82851136799994, 40.689239055999934], [-73.82753021399995, 40.687239482999914], [-73.83807769099997, 40.68423890999988], [-73.83697486299991, 40.6819880179999], [-73.84012856099982, 40.6811466509999], [-73.84947763499991, 40.69819580699987], [-73.83696237799982, 40.70421245199992]]]}}, {\"id\": \"197\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2320048953352985, \"OBJECTID\": 198, \"Shape_Leng\": 0.134094242763, \"Shape_Area\": 0.000499809882564, \"zone\": \"Ridgewood\", \"LocationID\": 198, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90768387299997, 40.71737969499987], [-73.90639010099999, 40.71736369199989], [-73.90537582199983, 40.71287658099986], [-73.88006267699978, 40.71265257399993], [-73.87761134199985, 40.70716770499993], [-73.88911917799989, 40.705598876999915], [-73.90123290699994, 40.69144227899997], [-73.90579597099993, 40.69412715499987], [-73.9042601839999, 40.69570037099989], [-73.91180820099989, 40.69993800299986], [-73.91067882699998, 40.701045968999914], [-73.91290404099983, 40.70236189199989], [-73.91180710099992, 40.703434952999864], [-73.92189184699987, 40.709396095999836], [-73.92074519699985, 40.71052968599989], [-73.92404011299993, 40.714008312999916], [-73.91077807399988, 40.71318759999986], [-73.91283037699992, 40.71536090799986], [-73.90965980099996, 40.71779315499991], [-73.90768387299997, 40.71737969499987]]]}}, {\"id\": \"198\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22719946272666225, \"OBJECTID\": 199, \"Shape_Leng\": 0.0778085005169, \"Shape_Area\": 0.000288747549303, \"zone\": \"Rikers Island\", \"LocationID\": 199, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88885148496314, 40.79870632895875], [-73.88665108524579, 40.79803819669988], [-73.88375741590156, 40.79570856541983], [-73.87288457445655, 40.79145210820714], [-73.87080888583708, 40.78789669184152], [-73.87308761694455, 40.78585495239161], [-73.8783068007881, 40.785356620754236], [-73.88905212421602, 40.78737256012899], [-73.89282283610926, 40.79281708195147], [-73.89179289686568, 40.79677524575864], [-73.88885148496314, 40.79870632895875]]]}}, {\"id\": \"199\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22788726971129314, \"OBJECTID\": 200, \"Shape_Leng\": 0.112661735435, \"Shape_Area\": 0.000744643168558, \"zone\": \"Riverdale/North Riverdale/Fieldston\", \"LocationID\": 200, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.896411335, 40.90450452099991], [-73.89643170799987, 40.89365527899989], [-73.89787887499983, 40.89027610099993], [-73.9028594909999, 40.88916474399989], [-73.90413642399992, 40.887080708999896], [-73.90975930299997, 40.889250027999886], [-73.91489276999991, 40.8846247709999], [-73.92058641888602, 40.88718267830985], [-73.9103325682445, 40.91553277700258], [-73.89663333881637, 40.911417374923055], [-73.896411335, 40.90450452099991]]]}}, {\"id\": \"200\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.26130578966267454, \"OBJECTID\": 201, \"Shape_Leng\": 0.130404117686, \"Shape_Area\": 0.000619350747819, \"zone\": \"Rockaway Park\", \"LocationID\": 201, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.81849901800001, 40.5873623529999], [-73.81544517899997, 40.58337349199993], [-73.81730795916116, 40.58233475897], [-73.86255286029154, 40.56681001230082], [-73.86662812000002, 40.573057648999914], [-73.8504438514919, 40.582130243581396], [-73.83929597982016, 40.581863545478186], [-73.82480116623435, 40.587154615369926], [-73.81849901800001, 40.5873623529999]]]}}, {\"id\": \"201\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20122313630249722, \"OBJECTID\": 202, \"Shape_Leng\": 0.0699215712618, \"Shape_Area\": 0.000106175396322, \"zone\": \"Roosevelt Island\", \"LocationID\": 202, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94180032729437, 40.76904692662474], [-73.95254079336881, 40.757095314537246], [-73.96158304969282, 40.74941430105318], [-73.94472478498723, 40.76978627176733], [-73.94007665725826, 40.77292618617896], [-73.94180032729437, 40.76904692662474]]]}}, {\"id\": \"202\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2045135499499479, \"OBJECTID\": 203, \"Shape_Leng\": 0.189937815868, \"Shape_Area\": 0.000615129861659, \"zone\": \"Rosedale\", \"LocationID\": 203, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7275350030123, 40.673930284146124], [-73.72833071887973, 40.66304394386548], [-73.72522608103498, 40.652001000073724], [-73.74143706021165, 40.646889178160706], [-73.74227387212949, 40.64012338122567], [-73.739471216134, 40.63570643191466], [-73.74246642615259, 40.635119228252094], [-73.74096487686366, 40.63730477503007], [-73.74397128496219, 40.63788001261484], [-73.7437342150463, 40.6390221176639], [-73.7464954765674, 40.63651030383521], [-73.74516219973704, 40.63811218797305], [-73.74704226024105, 40.636932365240085], [-73.74631467883957, 40.641394313090785], [-73.7487767478731, 40.64511819098248], [-73.75522386899995, 40.64744446899989], [-73.7541108669999, 40.648697664999915], [-73.76406290099993, 40.65435626599989], [-73.76165732899996, 40.658238085999855], [-73.74867194699979, 40.65267808799992], [-73.74332546799988, 40.652537803999905], [-73.74328714099991, 40.65613473199991], [-73.74853177199994, 40.65757569699993], [-73.74281884699987, 40.66530554899986], [-73.73891022900001, 40.66595449199986], [-73.74117817299991, 40.666402953999885], [-73.72718272499995, 40.68359732999987], [-73.72586264191175, 40.683241080973865], [-73.72563005109944, 40.67958795089065], [-73.7275350030123, 40.673930284146124]]]}}, {\"id\": \"203\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.24250863084994317, \"OBJECTID\": 204, \"Shape_Leng\": 0.101912418311, \"Shape_Area\": 0.000641384265972, \"zone\": \"Rossville/Woodrow\", \"LocationID\": 204, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.2032648999999, 40.55562847699989], [-74.19174368799999, 40.53191669699986], [-74.20152661599995, 40.526265205999934], [-74.2156723039999, 40.52395762799989], [-74.21800676499996, 40.53229858999995], [-74.21776941599994, 40.54112662999992], [-74.22131562699998, 40.55025307399992], [-74.21784397799998, 40.55443271799988], [-74.21050789400002, 40.557064181999934], [-74.205854485, 40.55750245699993], [-74.2032648999999, 40.55562847699989]]]}}, {\"id\": \"204\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2272722620789881, \"OBJECTID\": 205, \"Shape_Leng\": 0.141953388327, \"Shape_Area\": 0.000766216513505, \"zone\": \"Saint Albans\", \"LocationID\": 205, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.766369252, 40.70256339699992], [-73.75205066499986, 40.70522802799995], [-73.74863325999986, 40.6954447609999], [-73.7435928609999, 40.69621532499986], [-73.75576169199995, 40.67588590899988], [-73.76380484799996, 40.67376993299995], [-73.77469730199986, 40.680486045999906], [-73.77220275299989, 40.681610338999896], [-73.7732870399999, 40.683454731999866], [-73.77699767599997, 40.68628268899992], [-73.77966850599998, 40.685377122999945], [-73.7832662499999, 40.68999429299992], [-73.78078155499992, 40.69102994299994], [-73.77977295099997, 40.689281169999916], [-73.77789337399993, 40.68991768999992], [-73.7802372379999, 40.692972334999894], [-73.77230768699997, 40.70057150699989], [-73.77559218799982, 40.70532930899986], [-73.7749009329999, 40.707066661999875], [-73.76844546499984, 40.709637026999886], [-73.766369252, 40.70256339699992]]]}}, {\"id\": \"205\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21166864022103807, \"OBJECTID\": 206, \"Shape_Leng\": 0.212756793386, \"Shape_Area\": 0.000944392507762, \"zone\": \"Saint George/New Brighton\", \"LocationID\": 206, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08221272914942, 40.64828016229008], [-74.07165829759784, 40.64503374643502], [-74.07093784471476, 40.64334596384062], [-74.07210952444638, 40.64246321679278], [-74.07032923715386, 40.64252612813239], [-74.07314400586021, 40.641852974480116], [-74.07019399559213, 40.6420075028053], [-74.07289005609624, 40.64092267287602], [-74.06998354653952, 40.64106894311676], [-74.07285041194942, 40.64080368844508], [-74.07258133886755, 40.63794187429597], [-74.07865269699992, 40.63830402499987], [-74.08373397299994, 40.63573714399991], [-74.08870582299987, 40.63684520599986], [-74.09242513499994, 40.63558181299992], [-74.09409312599998, 40.639266913999975], [-74.09767290799995, 40.63896067899989], [-74.09787969199995, 40.64035805499988], [-74.10974833299994, 40.63786583299993], [-74.11765722499996, 40.638062466999905], [-74.11614321199995, 40.62870226399988], [-74.1132624959999, 40.62305442499995], [-74.12125677899994, 40.62782274999989], [-74.1313098529999, 40.62632834299998], [-74.12867047299994, 40.63734826499993], [-74.1268887269999, 40.637210087999925], [-74.12705362202625, 40.64024173781949], [-74.11861635236262, 40.641530365028935], [-74.11883624616948, 40.64254186406039], [-74.1171709328468, 40.64162427104175], [-74.1172247008255, 40.64302735996259], [-74.11535082182925, 40.64247054015812], [-74.1097393256449, 40.64546372818479], [-74.09880066885808, 40.64504703431087], [-74.08570754378212, 40.648880553370454], [-74.08221272914942, 40.64828016229008]]]}}, {\"id\": \"206\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22106390335282308, \"OBJECTID\": 207, \"Shape_Leng\": 0.0283756831492, \"Shape_Area\": 4.47230848441e-05, \"zone\": \"Saint Michaels Cemetery/Woodside\", \"LocationID\": 207, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90137640299986, 40.75964203799989], [-73.90356915599997, 40.765781788999924], [-73.90217951699996, 40.76731584199997], [-73.89349605899986, 40.765950929999924], [-73.89866137899992, 40.75992101699997], [-73.90137640299986, 40.75964203799989]]]}}, {\"id\": \"207\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.19966128241805375, \"OBJECTID\": 208, \"Shape_Leng\": 0.214721009581, \"Shape_Area\": 0.00202032598969, \"zone\": \"Schuylerville/Edgewater Park\", \"LocationID\": 208, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.825986869, 40.84067575199988], [-73.82601252600003, 40.83402077799985], [-73.81552500922504, 40.83597630692601], [-73.81476387691956, 40.830816799993485], [-73.81275469994516, 40.82824823625135], [-73.80992515914596, 40.82817971199259], [-73.81384974851248, 40.8268035287504], [-73.81379523517661, 40.82529808600147], [-73.81211350556435, 40.82555263465937], [-73.81388934712506, 40.82408736742192], [-73.80766604765235, 40.825896992059], [-73.80451800414906, 40.81892768751854], [-73.79759032777322, 40.816507532070055], [-73.80140380968845, 40.8123633086586], [-73.80361390158114, 40.812521106621354], [-73.80453081609834, 40.81425461992442], [-73.80351667337368, 40.81395196364859], [-73.80704600153508, 40.816533693875236], [-73.80505213646254, 40.81289496243412], [-73.80224245334347, 40.81019739366033], [-73.79086475738406, 40.80734114044052], [-73.79028410874172, 40.80481710368888], [-73.7933898228672, 40.804204330036214], [-73.80185524738366, 40.808967826140574], [-73.80392673417204, 40.80852797081171], [-73.8099202908178, 40.81292915226176], [-73.81620056896291, 40.81384747647932], [-73.83017915330238, 40.81083454948945], [-73.8321661704888, 40.80850227767013], [-73.831615744777, 40.80493825599657], [-73.83743612112924, 40.80620264259155], [-73.84043199520461, 40.81254112811847], [-73.83974355921865, 40.816406071751366], [-73.83696070299992, 40.81634054099991], [-73.83614239499985, 40.81919890299987], [-73.83661819699992, 40.82711414899993], [-73.83709061399992, 40.82896714699996], [-73.84233077477266, 40.8290739624537], [-73.83885232384739, 40.83369066784824], [-73.83956700299987, 40.84054722799986], [-73.83771374899992, 40.840406309999885], [-73.83688925199996, 40.84337478799992], [-73.82611300699996, 40.84533869399991], [-73.825986869, 40.84067575199988]]], [[[-73.83960098545435, 40.83567071869858], [-73.83961672561374, 40.835667752627316], [-73.83962599488439, 40.83569971218477], [-73.83960850400152, 40.83570805084221], [-73.83960098545435, 40.83567071869858]]], [[[-73.83979488581072, 40.835619207246324], [-73.8396148723857, 40.83561056836579], [-73.83948374192397, 40.83457420079436], [-73.83959266810578, 40.83456047582325], [-73.83979488581072, 40.835619207246324]]]]}}, {\"id\": \"208\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20156080292580283, \"OBJECTID\": 209, \"Shape_Leng\": 0.0306953113473, \"Shape_Area\": 3.88703351209e-05, \"zone\": \"Seaport\", \"LocationID\": 209, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00499805099996, 40.71201189699987], [-73.99919451154408, 40.707947376175724], [-74.00143661179398, 40.70487217770521], [-74.00906264699988, 40.71088670399993], [-74.00499805099996, 40.71201189699987]]]}}, {\"id\": \"209\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.25580338073910863, \"OBJECTID\": 210, \"Shape_Leng\": 0.114851983372, \"Shape_Area\": 0.000453155131253, \"zone\": \"Sheepshead Bay\", \"LocationID\": 210, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92759936700004, 40.5950607789999], [-73.91805571865743, 40.58643685311621], [-73.92520361514241, 40.58580627115536], [-73.92802185276429, 40.588068756350886], [-73.9239529485009, 40.59113985052502], [-73.92863061415638, 40.58857263642073], [-73.92844090493661, 40.589030428141996], [-73.9291344038264, 40.588879241681944], [-73.92875703499415, 40.589152966925], [-73.92976615773313, 40.589292069447644], [-73.93003427450557, 40.58942599650075], [-73.93057705718242, 40.589890641203176], [-73.93010325988745, 40.58960610507142], [-73.93001650952081, 40.5940945360621], [-73.93260420556844, 40.59492954452495], [-73.93121294858955, 40.58953130811921], [-73.92972846380192, 40.58861391582969], [-73.93125191267133, 40.58684982521422], [-73.92857393936458, 40.58706150237135], [-73.92552860874584, 40.58497837657265], [-73.9344471859999, 40.586652074999925], [-73.96025825899996, 40.58502930799994], [-73.9603495359999, 40.58730628599994], [-73.95619313699986, 40.58776226899994], [-73.956927197, 40.596612075999865], [-73.94248473299992, 40.598200364999855], [-73.94288954099984, 40.600318649999934], [-73.93447158899997, 40.60123151199989], [-73.9335511719998, 40.60268728099991], [-73.92813139699999, 40.59777792799987], [-73.92956858299992, 40.5968202649999], [-73.92759936700004, 40.5950607789999]]]}}, {\"id\": \"210\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2083806733799869, \"OBJECTID\": 211, \"Shape_Leng\": 0.0252345082132, \"Shape_Area\": 3.97291966087e-05, \"zone\": \"SoHo\", \"LocationID\": 211, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.001523638, 40.719820797999944], [-74.00539186699989, 40.722037365999896], [-74.00214891499996, 40.728035366999904], [-73.9967717579999, 40.7254319439999], [-74.001523638, 40.719820797999944]]]}}, {\"id\": \"211\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22523481603235468, \"OBJECTID\": 212, \"Shape_Leng\": 0.0671290653429, \"Shape_Area\": 0.000161276191858, \"zone\": \"Soundview/Bruckner\", \"LocationID\": 212, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88421976599987, 40.82299645799987], [-73.88378414099992, 40.828274291999946], [-73.87729930499995, 40.8287427739999], [-73.86085019699983, 40.833613537999874], [-73.857391213, 40.83229737199986], [-73.85615112900003, 40.82677004599987], [-73.88421976599987, 40.82299645799987]]]}}, {\"id\": \"212\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2409995051733899, \"OBJECTID\": 213, \"Shape_Leng\": 0.159958783822, \"Shape_Area\": 0.000904077906339, \"zone\": \"Soundview/Castle Hill\", \"LocationID\": 213, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.83667856399991, 40.81758876299995], [-73.83696070299992, 40.81634054099991], [-73.84003308726955, 40.81682214096213], [-73.83888231646733, 40.822140489856025], [-73.84228428387955, 40.82864660449135], [-73.83709061399992, 40.82896714699996], [-73.83667856399991, 40.81758876299995]]], [[[-73.84231712480313, 40.82718468468027], [-73.83999860407422, 40.81993494681306], [-73.84198684834051, 40.818774373908084], [-73.84617302872044, 40.81067666443039], [-73.8511447226069, 40.814396548876], [-73.85538030243693, 40.81428888208278], [-73.84903294180673, 40.81211891007852], [-73.84996696802818, 40.808578018843576], [-73.847488226419, 40.805448909149185], [-73.85019839036403, 40.80451002497395], [-73.856614854502, 40.80479692395033], [-73.85866549456955, 40.806833088596065], [-73.85895141221772, 40.810271417586556], [-73.8601269523559, 40.80960549989125], [-73.85935625734362, 40.80898803428646], [-73.85938875426787, 40.80816614120708], [-73.86065222514291, 40.80963668169183], [-73.86765646284456, 40.81058376193711], [-73.87077804885932, 40.814486895487775], [-73.8780628329999, 40.81618263199993], [-73.88244779599998, 40.81926832299992], [-73.8843985489999, 40.822965575999866], [-73.85635677499994, 40.82675208499993], [-73.85513639699997, 40.822436188999866], [-73.84289858800003, 40.82408305899986], [-73.84390624999999, 40.828407828999936], [-73.84231712480313, 40.82718468468027]]]]}}, {\"id\": \"213\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2380262504761795, \"OBJECTID\": 214, \"Shape_Leng\": 0.135300314246, \"Shape_Area\": 0.000811976523012, \"zone\": \"South Beach/Dongan Hills\", \"LocationID\": 214, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08491019899996, 40.59570472799985], [-74.08173342999991, 40.59215628599987], [-74.07043976099992, 40.59673474699988], [-74.06436003721248, 40.58826789117477], [-74.07499413073505, 40.57923655741631], [-74.07336118472098, 40.578275488289144], [-74.07508288192615, 40.579160158755506], [-74.08495715819193, 40.57125984002107], [-74.10039083499996, 40.579688665999925], [-74.09844000899997, 40.581241705999915], [-74.10525775499997, 40.58592143599986], [-74.10088251699995, 40.59017921599992], [-74.09979588999995, 40.59387136599988], [-74.09375453799998, 40.59674488999988], [-74.08983655899989, 40.60077409499996], [-74.08218729299988, 40.601941007999876], [-74.08130343799999, 40.59938765599991], [-74.08532650099991, 40.59700832199995], [-74.08491019899996, 40.59570472799985]]]}}, {\"id\": \"214\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20337276167364543, \"OBJECTID\": 215, \"Shape_Leng\": 0.103556453625, \"Shape_Area\": 0.000395647022725, \"zone\": \"South Jamaica\", \"LocationID\": 215, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7769103769999, 40.70252488499994], [-73.77395659499986, 40.699805781999906], [-73.7802372379999, 40.692972334999894], [-73.77789337399993, 40.68991768999992], [-73.77977295099997, 40.689281169999916], [-73.78078155499992, 40.69102994299994], [-73.793908417, 40.68630469099985], [-73.79561486799987, 40.688656890999916], [-73.79995042799996, 40.68682771899991], [-73.79898209099994, 40.68485873799988], [-73.80576840199991, 40.682932390999916], [-73.81063298299996, 40.69194738299988], [-73.80890216900004, 40.69256434899993], [-73.80642479399984, 40.690873857999904], [-73.79260506799987, 40.70043209999988], [-73.79239170299988, 40.7021665299999], [-73.78256582599995, 40.70542486399989], [-73.78034491200003, 40.70541241699986], [-73.7769103769999, 40.70252488499994]]]}}, {\"id\": \"215\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2098709574326453, \"OBJECTID\": 216, \"Shape_Leng\": 0.116797554681, \"Shape_Area\": 0.000816076647781, \"zone\": \"South Ozone Park\", \"LocationID\": 216, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.80857655199986, 40.688330889999925], [-73.80142836799988, 40.673666833999874], [-73.80203207699995, 40.66524802499994], [-73.81496922499986, 40.66247266799991], [-73.832705339, 40.665405772999854], [-73.84012856099982, 40.6811466509999], [-73.83697486299991, 40.6819880179999], [-73.83807769099997, 40.68423890999988], [-73.82753021399995, 40.687239482999914], [-73.82851136799994, 40.689239055999934], [-73.82599347399987, 40.689953661999894], [-73.82501128899997, 40.68795231799991], [-73.81063298299996, 40.69194738299988], [-73.80857655199986, 40.688330889999925]]]}}, {\"id\": \"216\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21720397511326678, \"OBJECTID\": 217, \"Shape_Leng\": 0.0553905251285, \"Shape_Area\": 0.000114733270758, \"zone\": \"South Williamsburg\", \"LocationID\": 217, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95554165799997, 40.70759344799989], [-73.94705205299991, 40.70366394899985], [-73.95128819399989, 40.700922364999855], [-73.95381196900003, 40.703180979999914], [-73.95745736399992, 40.70082260299993], [-73.95701993199991, 40.69897391399995], [-73.96283963699993, 40.698038667999946], [-73.96217978199998, 40.70022070999989], [-73.96838933699995, 40.70682918699989], [-73.95554165799997, 40.70759344799989]]]}}, {\"id\": \"217\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22582323002551435, \"OBJECTID\": 218, \"Shape_Leng\": 0.0837008281049, \"Shape_Area\": 0.000281293736407, \"zone\": \"Springfield Gardens North\", \"LocationID\": 218, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76561573099998, 40.6747790199999], [-73.76380484799996, 40.67376993299995], [-73.76031463999993, 40.67510997099984], [-73.75857198899986, 40.67263733099987], [-73.75636239300002, 40.67242923699999], [-73.75763129599993, 40.66645115699992], [-73.77342462799993, 40.66802761899988], [-73.78940999399987, 40.66684115899995], [-73.78808647399985, 40.6730096269999], [-73.78573730899993, 40.67238503499988], [-73.77722923600001, 40.679652720999904], [-73.77469730199986, 40.680486045999906], [-73.76561573099998, 40.6747790199999]]]}}, {\"id\": \"218\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20879320172163654, \"OBJECTID\": 219, \"Shape_Leng\": 0.155126669675, \"Shape_Area\": 0.000428176336771, \"zone\": \"Springfield Gardens South\", \"LocationID\": 219, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.761822313, 40.666877759999856], [-73.73856469500002, 40.66639689799991], [-73.74450922599979, 40.663821277999865], [-73.74853177199994, 40.65757569699993], [-73.74328714099991, 40.65613473199991], [-73.74332546899993, 40.65253780399992], [-73.74867194699979, 40.65267808799992], [-73.76165732899996, 40.658238085999855], [-73.76406290099993, 40.65435626599989], [-73.78321833499997, 40.663166283999864], [-73.80203207699995, 40.66524802499994], [-73.80155565300004, 40.6667533639999], [-73.77342462799993, 40.66802761899988], [-73.761822313, 40.666877759999856]]]}}, {\"id\": \"219\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.24984324430717308, \"OBJECTID\": 220, \"Shape_Leng\": 0.106015943267, \"Shape_Area\": 0.000313035744467, \"zone\": \"Spuyten Duyvil/Kingsbridge\", \"LocationID\": 220, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89827216899992, 40.8894579429999], [-73.89972588599989, 40.88651488699985], [-73.89661546599986, 40.885955255999946], [-73.90369454799992, 40.878501617999895], [-73.90608130599988, 40.873239628999904], [-73.90746489699994, 40.87354735399985], [-73.9068187309999, 40.87661958799993], [-73.91033193599998, 40.879038046999895], [-73.91775669364831, 40.8756636279301], [-73.92490327486523, 40.87888836792653], [-73.92058641888602, 40.88718267830985], [-73.91489276999991, 40.8846247709999], [-73.90900536, 40.88950014099986], [-73.90413642399992, 40.887080708999896], [-73.9028594909999, 40.88916474399989], [-73.89827216899992, 40.8894579429999]]]}}, {\"id\": \"220\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.19856595619290837, \"OBJECTID\": 221, \"Shape_Leng\": 0.166218220846, \"Shape_Area\": 0.000890111567051, \"zone\": \"Stapleton\", \"LocationID\": 221, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.07530550099992, 40.637729958999955], [-74.07053959705199, 40.63747140462426], [-74.07341562632824, 40.63689395096554], [-74.07209483807503, 40.63664727762461], [-74.07335822045584, 40.63653609416517], [-74.0729847461233, 40.630308978981404], [-74.06802173543946, 40.62878758518541], [-74.07291177936548, 40.62999637940653], [-74.07225500955778, 40.62460875019394], [-74.06975543088323, 40.6212081398724], [-74.06544586069478, 40.61920262828925], [-74.06668207982919, 40.61806749593851], [-74.06425721751587, 40.61817168758555], [-74.06509908407213, 40.61751095612543], [-74.05685812390232, 40.608056357772554], [-74.0651709129999, 40.604213507999894], [-74.06350565699996, 40.60215005499991], [-74.07399965299999, 40.60572556699987], [-74.0815547009999, 40.60629029699987], [-74.0714060829999, 40.61943137899991], [-74.07253551699989, 40.622464021999896], [-74.08720982799996, 40.61330697299989], [-74.0878234189999, 40.615087934999885], [-74.08415419899998, 40.626267114999905], [-74.07972492199995, 40.62895068799986], [-74.08574928999992, 40.633955556999865], [-74.07878831099995, 40.63826855899991], [-74.07530550099992, 40.637729958999955]]]}}, {\"id\": \"221\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21978536087443468, \"OBJECTID\": 222, \"Shape_Leng\": 0.0483893527976, \"Shape_Area\": 0.000130268343918, \"zone\": \"Starrett City\", \"LocationID\": 222, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.87908517399995, 40.65020211699988], [-73.8745808109999, 40.64561010599992], [-73.87841589406925, 40.638912189353306], [-73.88006714132425, 40.64200344662675], [-73.88341999203011, 40.64452381564944], [-73.88557762124442, 40.644040950146135], [-73.88764072166725, 40.64780239000726], [-73.89125147199995, 40.65002559699993], [-73.88221364899994, 40.65451863799996], [-73.87908517399995, 40.65020211699988]]], [[[-73.8882853135604, 40.64672241327608], [-73.88864137620679, 40.647079553158], [-73.88851637568528, 40.64736069152191], [-73.88820050726648, 40.647140869529046], [-73.8882853135604, 40.64672241327608]]]]}}, {\"id\": \"222\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2201778840572781, \"OBJECTID\": 223, \"Shape_Leng\": 0.166021925275, \"Shape_Area\": 0.00076436070058, \"zone\": \"Steinway\", \"LocationID\": 223, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90649492923363, 40.7901809903447], [-73.90032629743816, 40.789007662825696], [-73.8959296297551, 40.785648944313756], [-73.89874897638398, 40.78278115576602], [-73.90114577552107, 40.78271110609054], [-73.90257929672345, 40.78028912083556], [-73.89437777934127, 40.78424320348422], [-73.89606377738421, 40.783157201805224], [-73.895284679113, 40.78154124925482], [-73.89274917876409, 40.78296570668481], [-73.89165459123714, 40.78219478153064], [-73.89300986180193, 40.78179802356213], [-73.89119589704391, 40.77856328788702], [-73.89251064542336, 40.77739286927271], [-73.89183244902246, 40.77488019994431], [-73.88945486901865, 40.77353295106357], [-73.88770661799987, 40.76683859799995], [-73.89349605899986, 40.765950929999924], [-73.91759120499997, 40.769900870999926], [-73.92398142900002, 40.77461511299991], [-73.92375761299998, 40.77654701199991], [-73.91796581399993, 40.78125629199991], [-73.91996458796496, 40.78263506876824], [-73.912599859115, 40.78937632963394], [-73.90985862925774, 40.790945493781884], [-73.90649492923363, 40.7901809903447]]]}}, {\"id\": \"223\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2023397908303294, \"OBJECTID\": 224, \"Shape_Leng\": 0.0440204462769, \"Shape_Area\": 9.84786989135e-05, \"zone\": \"Stuy Town/Peter Cooper Village\", \"LocationID\": 224, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97511055399991, 40.735390254999864], [-73.97151882628022, 40.72935136174009], [-73.97162697285862, 40.72662845299842], [-73.98255629299986, 40.731350133999875], [-73.97849845599995, 40.73679095699989], [-73.97511055399991, 40.735390254999864]]]}}, {\"id\": \"224\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2502146360440248, \"OBJECTID\": 225, \"Shape_Leng\": 0.0887992073281, \"Shape_Area\": 0.000310835395162, \"zone\": \"Stuyvesant Heights\", \"LocationID\": 225, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91741294299999, 40.68403405199993], [-73.9163011989999, 40.67857711199991], [-73.92183500099998, 40.67894555899991], [-73.92281374299995, 40.68341442999991], [-73.93500821699992, 40.68276376799985], [-73.93435066499993, 40.67956340799991], [-73.94032794, 40.679889974999874], [-73.94394947299996, 40.698221278999924], [-73.93817718599986, 40.69864022299987], [-73.91804607, 40.68721324799989], [-73.91741294299999, 40.68403405199993]]]}}, {\"id\": \"225\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20844430739268255, \"OBJECTID\": 226, \"Shape_Leng\": 0.168957762041, \"Shape_Area\": 0.00076194414504, \"zone\": \"Sunnyside\", \"LocationID\": 226, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90989961499994, 40.749087274999944], [-73.91187504099992, 40.74822079599992], [-73.9119412979999, 40.742382948999925], [-73.91567469599988, 40.740784929999904], [-73.91903825199986, 40.73200596599987], [-73.917342506, 40.731382825999916], [-73.91796696699996, 40.7285007709999], [-73.91205021299994, 40.72850636299993], [-73.91174734199988, 40.72991203299991], [-73.90712253099991, 40.727424727999924], [-73.91384828299984, 40.72289841199989], [-73.91132095099978, 40.71962422999992], [-73.92232773978448, 40.71659264406108], [-73.92515207271921, 40.722335339820816], [-73.9244229875485, 40.723589975986165], [-73.92037064235664, 40.723681113821044], [-73.92490316165177, 40.72450657567146], [-73.92923485876162, 40.72826016576104], [-73.93804478507161, 40.730509967846054], [-73.94162472636452, 40.73584144233816], [-73.94594410333028, 40.73751251570048], [-73.94140886345744, 40.73929957085316], [-73.93869664599998, 40.738140816999866], [-73.93726110699994, 40.745176490999896], [-73.94401030399996, 40.7436394539999], [-73.93383269399993, 40.7516047019999], [-73.91547326299991, 40.75379181799991], [-73.91035492999984, 40.75274990899987], [-73.90989961499994, 40.749087274999944]]]}}, {\"id\": \"226\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.6067164249108072, \"OBJECTID\": 227, \"Shape_Leng\": 0.0828301570789, \"Shape_Area\": 0.000268316529534, \"zone\": \"Sunset Park East\", \"LocationID\": 227, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.991928124, 40.6422049039999], [-73.9955103369999, 40.63980966699993], [-73.99917564699989, 40.642025441999934], [-74.00735636699996, 40.6341636269999], [-74.00556490399993, 40.63308317599992], [-74.01000340799995, 40.630113088999906], [-74.01602010100001, 40.63331711399997], [-74.016835401, 40.63535542899995], [-74.0139608859999, 40.63815326699995], [-74.01616711499993, 40.63948244099991], [-74.00214474699993, 40.65295610699995], [-73.98905872499985, 40.64411924099995], [-73.991928124, 40.6422049039999]]]}}, {\"id\": \"227\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2527603720000839, \"OBJECTID\": 228, \"Shape_Leng\": 0.177685177706, \"Shape_Area\": 0.000993414783816, \"zone\": \"Sunset Park West\", \"LocationID\": 228, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9969179149999, 40.6691534839999], [-73.99490350099988, 40.6667854349999], [-73.99692689899985, 40.664374914999925], [-73.99063545399997, 40.660579088999896], [-73.988716511, 40.66240493799995], [-73.98725068899996, 40.66037895599992], [-73.990188415, 40.657600817999906], [-73.99516890199999, 40.659669521999945], [-74.01616711499993, 40.63948244099991], [-74.0139608859999, 40.63815326699995], [-74.016835401, 40.63535542899995], [-74.02141038399992, 40.638584902999845], [-74.02108051399998, 40.64149093699994], [-74.02475437599998, 40.6381214179999], [-74.03266558998958, 40.643720344890056], [-74.03051830863413, 40.645274425208974], [-74.02820569545007, 40.644015982890345], [-74.02604948400315, 40.64626135032611], [-74.0293899048662, 40.64880405511391], [-74.02571758804807, 40.64661549229093], [-74.02461487103106, 40.64715112511878], [-74.02622102475574, 40.64824842780924], [-74.02414566050564, 40.647615101259845], [-74.02605133196793, 40.6509943007431], [-74.02285141795983, 40.651169195161685], [-74.02522662972656, 40.652829856959634], [-74.02154138319214, 40.650671622911226], [-74.02380703903121, 40.65304472851146], [-74.02118670027885, 40.652839836895325], [-74.02320464727141, 40.654096116817925], [-74.0186223830857, 40.653491349022964], [-74.02005182454292, 40.655335948874416], [-74.01715319581021, 40.65477510436213], [-74.01970596410222, 40.656581163751035], [-74.01639437533773, 40.65562064285071], [-74.01910843803118, 40.657368884303494], [-74.01585370056823, 40.65621324055534], [-74.01786573109669, 40.65830844729852], [-74.01442130484548, 40.657608021951376], [-74.01745803087898, 40.6594501973867], [-74.01554165388997, 40.66075835547614], [-74.01183862056543, 40.65891283727472], [-74.0131555493881, 40.66198159772593], [-74.00859956968284, 40.65952092309827], [-74.00745858163884, 40.66055847620986], [-74.0103720129723, 40.66239888023248], [-74.00955074142026, 40.6632676941304], [-74.0034880110065, 40.66224055612849], [-74.00599748682693, 40.66325861082859], [-74.00772133506703, 40.66471863301977], [-74.0071253264673, 40.66491521847423], [-74.00362862508561, 40.66273484753458], [-74.00486193792649, 40.66506482193817], [-74.00083401108444, 40.66300644269171], [-73.99953152609916, 40.6643974859369], [-74.00292463516384, 40.66646231216332], [-73.99998149339123, 40.66745027490869], [-73.99902600998384, 40.66844297212693], [-73.9986188799999, 40.67183017199989], [-73.9969179149999, 40.6691534839999]]]}}, {\"id\": \"228\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20838347666004042, \"OBJECTID\": 229, \"Shape_Leng\": 0.0425252900116, \"Shape_Area\": 9.47058230931e-05, \"zone\": \"Sutton Place/Turtle Bay North\", \"LocationID\": 229, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96162927534907, 40.75515238425915], [-73.96482214210387, 40.75159632579112], [-73.9712170449999, 40.75519332399994], [-73.9665833839999, 40.76155093499988], [-73.95877790811335, 40.75827092092435], [-73.96162927534907, 40.75515238425915]]]}}, {\"id\": \"229\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22121627185013748, \"OBJECTID\": 230, \"Shape_Leng\": 0.0310283096779, \"Shape_Area\": 5.60914463266e-05, \"zone\": \"Times Sq/Theatre District\", \"LocationID\": 230, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98362433299988, 40.75551634599986], [-73.98979085399996, 40.7572331319999], [-73.98428230099987, 40.76479181499994], [-73.97860169399989, 40.762396071999945], [-73.98362433299988, 40.75551634599986]]]}}, {\"id\": \"230\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20480204546289757, \"OBJECTID\": 231, \"Shape_Leng\": 0.0634201165587, \"Shape_Area\": 0.000166929752992, \"zone\": \"TriBeCa/Civic Center\", \"LocationID\": 231, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00931760399995, 40.72470155099989], [-73.99995620900002, 40.71801709499987], [-74.00485768099996, 40.7124525409999], [-74.00834760999993, 40.71135854599987], [-74.01375388499997, 40.71369055499987], [-74.01244109199989, 40.719057672999874], [-74.01457231438083, 40.72027413774219], [-74.01296558526825, 40.720328675587126], [-74.01102457999994, 40.72579386199991], [-74.00931760399995, 40.72470155099989]]]}}, {\"id\": \"231\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2053885710041709, \"OBJECTID\": 232, \"Shape_Leng\": 0.0614709085331, \"Shape_Area\": 0.000216049973456, \"zone\": \"Two Bridges/Seward Park\", \"LocationID\": 232, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97635525199995, 40.71913288399983], [-73.97347975524508, 40.7188614291129], [-73.97782091638247, 40.71064544490706], [-73.9919434659999, 40.70958905299992], [-73.99256242199989, 40.71438807699996], [-73.99022012699989, 40.71440493999988], [-73.98382387299982, 40.72147287199987], [-73.97635525199995, 40.71913288399983]]]}}, {\"id\": \"232\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2093244003439885, \"OBJECTID\": 233, \"Shape_Leng\": 0.048035918294, \"Shape_Area\": 0.000116188094682, \"zone\": \"UN/Turtle Bay South\", \"LocationID\": 233, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.96547092299994, 40.75193959699991], [-73.97123301520192, 40.743414663960074], [-73.97350835000003, 40.7437820339999], [-73.97121971999988, 40.74692154099996], [-73.9758346669999, 40.74886314599996], [-73.9712170449999, 40.75519332399994], [-73.96547092299994, 40.75193959699991]]], [[[-73.96421230395673, 40.74660431847665], [-73.96444522752618, 40.74641042576318], [-73.96458318426942, 40.746454400673315], [-73.96415980288437, 40.74686554735954], [-73.96421230395673, 40.74660431847665]]]]}}, {\"id\": \"233\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21232037209124394, \"OBJECTID\": 234, \"Shape_Leng\": 0.0360721994984, \"Shape_Area\": 7.31054382894e-05, \"zone\": \"Union Sq\", \"LocationID\": 234, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98996936399989, 40.73490456699994], [-73.99683993899998, 40.73736088899995], [-73.99051761299985, 40.746038637999895], [-73.98407485299995, 40.74332471799989], [-73.98996936399989, 40.73490456699994]]]}}, {\"id\": \"234\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2623165632922229, \"OBJECTID\": 235, \"Shape_Leng\": 0.0761668426519, \"Shape_Area\": 0.000212844547337, \"zone\": \"University Heights/Morris Heights\", \"LocationID\": 235, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90947862999988, 40.86180812899987], [-73.91328437000003, 40.859618600999944], [-73.90976165899988, 40.857182611999896], [-73.90679734899983, 40.85937358899988], [-73.90468371399987, 40.85706030199991], [-73.91416184199993, 40.845179187999925], [-73.92808209386837, 40.84539159691672], [-73.91422824049913, 40.862490790359615], [-73.90947862999988, 40.86180812899987]]]}}, {\"id\": \"235\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20648591384041076, \"OBJECTID\": 236, \"Shape_Leng\": 0.0442519223099, \"Shape_Area\": 0.000102864345211, \"zone\": \"Upper East Side North\", \"LocationID\": 236, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95779380499984, 40.773599896999926], [-73.96468735799992, 40.77567708199994], [-73.9557773589999, 40.78791392399995], [-73.94933170599992, 40.78519312699994], [-73.95779380499984, 40.773599896999926]]]}}, {\"id\": \"236\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21094145359954042, \"OBJECTID\": 237, \"Shape_Leng\": 0.0422126003388, \"Shape_Area\": 9.59873206621e-05, \"zone\": \"Upper East Side South\", \"LocationID\": 237, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9661274729999, 40.762179299999914], [-73.97301487199995, 40.76427887899991], [-73.96468735799992, 40.77567708199994], [-73.95825534899998, 40.772965336999896], [-73.9661274729999, 40.762179299999914]]]}}, {\"id\": \"237\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20696143472837536, \"OBJECTID\": 238, \"Shape_Leng\": 0.0601093114033, \"Shape_Area\": 0.000184763693765, \"zone\": \"Upper West Side North\", \"LocationID\": 238, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96884378999985, 40.78596738899994], [-73.98188645923244, 40.7905165804675], [-73.97639951965265, 40.79827321084623], [-73.96417598599989, 40.79236204499989], [-73.96884378999985, 40.78596738899994]]]}}, {\"id\": \"238\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21181264080136739, \"OBJECTID\": 239, \"Shape_Leng\": 0.0636261152958, \"Shape_Area\": 0.000204715440774, \"zone\": \"Upper West Side South\", \"LocationID\": 239, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97501417199996, 40.78768560599987], [-73.96933573299991, 40.78529327999992], [-73.97499744000007, 40.77753254599988], [-73.98507184299989, 40.781779680999946], [-73.98567936099997, 40.780312632999944], [-73.9881274613434, 40.78140179672366], [-73.98546581197031, 40.78536070057545], [-73.98711901394246, 40.7852103190041], [-73.98542932126932, 40.78541394218458], [-73.98465507883023, 40.78653474180792], [-73.98594956155647, 40.78648711396651], [-73.98188645923244, 40.7905165804675], [-73.97501417199996, 40.78768560599987]]]}}, {\"id\": \"239\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.3395252354336962, \"OBJECTID\": 240, \"Shape_Leng\": 0.146069764379, \"Shape_Area\": 0.000722130920707, \"zone\": \"Van Cortlandt Park\", \"LocationID\": 240, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87643743099994, 40.89687059299995], [-73.8771197409999, 40.895468663999964], [-73.87048672599987, 40.896640210999905], [-73.86135927899984, 40.89466231899991], [-73.86943471300005, 40.87812919899994], [-73.87094367500002, 40.87851076699998], [-73.87024100099985, 40.88022935299995], [-73.87494589799991, 40.87994766899988], [-73.87467601499999, 40.88156285199993], [-73.87812594399992, 40.88215382599989], [-73.87803562399996, 40.887118321999885], [-73.88124422499992, 40.88270963299992], [-73.89972588499982, 40.886514886999926], [-73.89643170799987, 40.89365527899989], [-73.89663333831646, 40.91141737492314], [-73.86789043706806, 40.90298695407611], [-73.87643743099994, 40.89687059299995]]]}}, {\"id\": \"240\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.280474301843297, \"OBJECTID\": 241, \"Shape_Leng\": 0.0687645777649, \"Shape_Area\": 0.00025467697592, \"zone\": \"Van Cortlandt Village\", \"LocationID\": 241, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88840157099996, 40.88441707499993], [-73.88705142599984, 40.884349575999906], [-73.88567517999988, 40.87887251099986], [-73.88937163799984, 40.87338229799993], [-73.8879350369998, 40.87203023199988], [-73.89438130299995, 40.866068850999966], [-73.89891854299988, 40.86965857399992], [-73.89942436199988, 40.86788083799993], [-73.90374975899987, 40.86959562699992], [-73.90608130599988, 40.873239628999904], [-73.89855059899999, 40.88437472699996], [-73.89661546599986, 40.885955255999946], [-73.88840157099996, 40.88441707499993]]]}}, {\"id\": \"241\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23964565561431272, \"OBJECTID\": 242, \"Shape_Leng\": 0.138136446433, \"Shape_Area\": 0.000360072450014, \"zone\": \"Van Nest/Morris Park\", \"LocationID\": 242, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83593362199987, 40.84840635599993], [-73.83771374799997, 40.84040630999987], [-73.84204144699997, 40.84064589699992], [-73.84458129199986, 40.838535507999914], [-73.85052278099987, 40.84278948899985], [-73.87359713599987, 40.83979855499992], [-73.86804200399997, 40.8448138389999], [-73.86860515599989, 40.85206864599991], [-73.86303938099996, 40.850611306999944], [-73.86250923799993, 40.84836228099987], [-73.85457388299992, 40.85150834599995], [-73.85304526899994, 40.849547331999865], [-73.84430846999992, 40.85119046399989], [-73.84494053799982, 40.8482182209999], [-73.84049767399992, 40.8541841049999], [-73.83499552599994, 40.85657120099989], [-73.8368191849999, 40.85727680899989], [-73.82847346699995, 40.860731352999906], [-73.83283502299986, 40.8560358449999], [-73.83129157699989, 40.85543410499991], [-73.83463970199993, 40.853817993999904], [-73.83593362199987, 40.84840635599993]]]}}, {\"id\": \"242\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20627447361808154, \"OBJECTID\": 243, \"Shape_Leng\": 0.0943314999364, \"Shape_Area\": 0.000438377341503, \"zone\": \"Washington Heights North\", \"LocationID\": 243, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93156536999994, 40.86958215799991], [-73.92474153399992, 40.86159363799992], [-73.92709951599986, 40.85827552099992], [-73.92467094699998, 40.8568108889999], [-73.92747680599985, 40.85069436999992], [-73.93139355399987, 40.84742823099985], [-73.94203871799984, 40.851931028999914], [-73.94317806099995, 40.84979994299991], [-73.94692626452422, 40.850528055834154], [-73.94186996426673, 40.853867739442755], [-73.93156536999994, 40.86958215799991]]]}}, {\"id\": \"243\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.19844548136535045, \"OBJECTID\": 244, \"Shape_Leng\": 0.0805686044599, \"Shape_Area\": 0.000359702747951, \"zone\": \"Washington Heights South\", \"LocationID\": 244, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94068822000003, 40.85131543299985], [-73.93139355399987, 40.84742823099985], [-73.93556697199989, 40.841712527999924], [-73.93505508799984, 40.8393332359999], [-73.94034643299983, 40.8304578419999], [-73.95015521048293, 40.83439675940791], [-73.94612416421772, 40.84389249655712], [-73.9469644175959, 40.85046552581819], [-73.94317806099984, 40.84979994399989], [-73.94203871799984, 40.851931028999914], [-73.94068822000003, 40.85131543299985]]]}}, {\"id\": \"244\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.31326437153170633, \"OBJECTID\": 245, \"Shape_Leng\": 0.0959830596604, \"Shape_Area\": 0.000466175414294, \"zone\": \"West Brighton\", \"LocationID\": 245, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.09787969199995, 40.64035805499988], [-74.09767290799995, 40.63896067899989], [-74.09409312599998, 40.639266913999975], [-74.09242513499994, 40.63558181299992], [-74.08870582299987, 40.63684520599986], [-74.08373397299994, 40.63573714399991], [-74.09223971399994, 40.62574291599987], [-74.10350925299994, 40.61641607499991], [-74.1132624959999, 40.62305442499995], [-74.11614321199995, 40.62870226399988], [-74.11765722499997, 40.63806246599996], [-74.10974833299994, 40.63786583299993], [-74.09787969199995, 40.64035805499988]]]}}, {\"id\": \"245\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20686532766408525, \"OBJECTID\": 246, \"Shape_Leng\": 0.0694671937437, \"Shape_Area\": 0.000281298327076, \"zone\": \"West Chelsea/Hudson Yards\", \"LocationID\": 246, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00439976203513, 40.76267135909888], [-73.99547450199992, 40.759626856999894], [-74.00820401899988, 40.74214751799986], [-74.01213726936172, 40.74353192705926], [-74.0087278005888, 40.74485823290996], [-74.01183642872407, 40.74512596367872], [-74.00946389855962, 40.74575014365948], [-74.01150411235528, 40.74699080250515], [-74.0091373625992, 40.74735976280298], [-74.01133174441699, 40.748000575341926], [-74.0090584790135, 40.74778564533286], [-74.00894606502312, 40.74841189316866], [-74.01121562130406, 40.74869682117583], [-74.00842636447226, 40.75215785023921], [-74.0099905581974, 40.75281618714907], [-74.00997058797012, 40.75293356415314], [-74.00835352838844, 40.752350739342255], [-74.00480392560178, 40.75780984316466], [-74.00700358904015, 40.759231870341075], [-74.00390742032765, 40.75937569218354], [-74.00233528130057, 40.76154372700156], [-74.00439976203513, 40.76267135909888]]]}}, {\"id\": \"246\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21327531999755805, \"OBJECTID\": 247, \"Shape_Leng\": 0.0929678582271, \"Shape_Area\": 0.000205617157843, \"zone\": \"West Concourse\", \"LocationID\": 247, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91222180499983, 40.84235659099988], [-73.91986089399987, 40.83194722699992], [-73.92885642099992, 40.81487986599991], [-73.93027812699984, 40.813248024999965], [-73.93242632000944, 40.81411751897961], [-73.9330725108407, 40.82815615201811], [-73.92850073899989, 40.829352349999965], [-73.92399049499993, 40.83505229799987], [-73.91901668400004, 40.83838109299989], [-73.91618344399991, 40.84233489999991], [-73.91794965199988, 40.842229570999976], [-73.91729224999992, 40.84510993999991], [-73.91222180499983, 40.84235659099988]]]}}, {\"id\": \"247\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23975959776939262, \"OBJECTID\": 248, \"Shape_Leng\": 0.0569191167671, \"Shape_Area\": 0.000149592769799, \"zone\": \"West Farms/Bronx River\", \"LocationID\": 248, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86393748099981, 40.84004456599994], [-73.86271114399993, 40.83299150899997], [-73.88378414099992, 40.828274291999946], [-73.88005188499987, 40.83470205899989], [-73.87379590199993, 40.837445132999896], [-73.87277204699981, 40.83975128199995], [-73.86448011799995, 40.84108779699994], [-73.86393748099981, 40.84004456599994]]]}}, {\"id\": \"248\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.3739594338742877, \"OBJECTID\": 249, \"Shape_Leng\": 0.03638434365, \"Shape_Area\": 7.22155957729e-05, \"zone\": \"West Village\", \"LocationID\": 249, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00250642399995, 40.729016384999966], [-74.0070902079999, 40.72877225299993], [-74.00515808199998, 40.74085808099991], [-73.99683993899998, 40.73736088899995], [-74.00250642399995, 40.729016384999966]]]}}, {\"id\": \"249\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22424487863230827, \"OBJECTID\": 250, \"Shape_Leng\": 0.0796263465454, \"Shape_Area\": 0.000240975845956, \"zone\": \"Westchester Village/Unionport\", \"LocationID\": 250, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8455308949999, 40.83917330699989], [-73.83936452799996, 40.84087706699995], [-73.83959108476137, 40.834041688045254], [-73.84390624999997, 40.82840782999993], [-73.84289858800003, 40.82408305899986], [-73.85513639799981, 40.822436188999866], [-73.857391213, 40.83229737199986], [-73.86033845899995, 40.833253165999935], [-73.85168459699997, 40.8342010519999], [-73.85333524999986, 40.84247015499987], [-73.85052278099987, 40.84278948899985], [-73.8455308949999, 40.83917330699989]]]}}, {\"id\": \"250\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2908846826274664, \"OBJECTID\": 251, \"Shape_Leng\": 0.1377111611, \"Shape_Area\": 0.000625754983157, \"zone\": \"Westerleigh\", \"LocationID\": 251, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.13107460299996, 40.631147728999956], [-74.1313098529999, 40.62632834299998], [-74.12125677899994, 40.62782274999989], [-74.10552598799994, 40.618592329999906], [-74.0977696879999, 40.61062359299988], [-74.1156683379999, 40.611175384999896], [-74.13210459699991, 40.608152923999924], [-74.14471100599997, 40.60803429699994], [-74.14954343599996, 40.60962717299988], [-74.14696864199988, 40.610151355999896], [-74.14603562099998, 40.61701277699996], [-74.13982184799993, 40.61629982499988], [-74.13826358199992, 40.624243769999936], [-74.13581008299997, 40.62506433999991], [-74.13806393799992, 40.627568470999904], [-74.13107460299996, 40.631147728999956]]]}}, {\"id\": \"251\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.19230468294425673, \"OBJECTID\": 252, \"Shape_Leng\": 0.158004373564, \"Shape_Area\": 0.00102526406057, \"zone\": \"Whitestone\", \"LocationID\": 252, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82049919995306, 40.80101146781907], [-73.81369963725258, 40.79707941566509], [-73.81225365567188, 40.79812694680175], [-73.79640953426332, 40.795728824199614], [-73.79442844782817, 40.794806810498535], [-73.79375126484103, 40.78906193333896], [-73.79058298999988, 40.78819290999989], [-73.79582463099997, 40.7861305879999], [-73.80242791599977, 40.786068273999945], [-73.8037902289999, 40.77561011199992], [-73.81485850699985, 40.776485922999875], [-73.81452997799975, 40.77888817499992], [-73.82331286899986, 40.77957865999986], [-73.82224127699999, 40.781693878999924], [-73.83869921299997, 40.781676956999874], [-73.83824429099994, 40.78778591599995], [-73.84008157499997, 40.78785153099992], [-73.83895676099984, 40.79143843099996], [-73.84081532599994, 40.7915048849999], [-73.84067828300006, 40.79390789199993], [-73.83663288911686, 40.79251637576927], [-73.83728024446076, 40.78900952077265], [-73.83251063648018, 40.788624476626765], [-73.83166080927866, 40.79110259115967], [-73.8278487557192, 40.79308767321416], [-73.82945991534359, 40.79682336759143], [-73.82506496699443, 40.79737577656827], [-73.82049919995306, 40.80101146781907]]]}}, {\"id\": \"252\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.1968457090250007, \"OBJECTID\": 253, \"Shape_Leng\": 0.0360514998192, \"Shape_Area\": 7.83395761191e-05, \"zone\": \"Willets Point\", \"LocationID\": 253, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83908354399988, 40.76525691299991], [-73.83679107299986, 40.76116667099992], [-73.83867321899987, 40.75701997899995], [-73.84324381399976, 40.75533680399995], [-73.84670933714987, 40.76134526931977], [-73.84272819396486, 40.76462854893166], [-73.83908354399988, 40.76525691299991]]]}}, {\"id\": \"253\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.27103593179914537, \"OBJECTID\": 254, \"Shape_Leng\": 0.0858863754861, \"Shape_Area\": 0.000360040216032, \"zone\": \"Williamsbridge/Olinville\", \"LocationID\": 254, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85186563799999, 40.87905886499989], [-73.85091896099982, 40.877893440999856], [-73.85254854099988, 40.877091586999924], [-73.8506895569999, 40.87548667799993], [-73.85363638799987, 40.87330059899996], [-73.85651292399994, 40.874217792999936], [-73.85661382499993, 40.87125016699991], [-73.87056192499989, 40.87151077799995], [-73.87019944899986, 40.87631135499987], [-73.860710982, 40.89537054399996], [-73.84590568599998, 40.889741368999864], [-73.85186563799999, 40.87905886499989]]]}}, {\"id\": \"254\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.21100623323921086, \"OBJECTID\": 255, \"Shape_Leng\": 0.0623841997664, \"Shape_Area\": 0.000172309184842, \"zone\": \"Williamsburg (North Side)\", \"LocationID\": 255, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96176070375392, 40.72522879205536], [-73.95477662900004, 40.72245950499994], [-73.94907816799986, 40.72254028699993], [-73.94841638599995, 40.71845553299995], [-73.94721341799986, 40.718560985999865], [-73.95547596099992, 40.7122654259999], [-73.96736391666703, 40.71648367952745], [-73.96176070375392, 40.72522879205536]]]}}, {\"id\": \"255\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.211436788357516, \"OBJECTID\": 256, \"Shape_Leng\": 0.0679149669603, \"Shape_Area\": 0.000168611097013, \"zone\": \"Williamsburg (South Side)\", \"LocationID\": 256, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95834207500002, 40.71330630099992], [-73.95547596099992, 40.7122654259999], [-73.95433904200002, 40.714071701999906], [-73.94937231899985, 40.71406687799986], [-73.94865533499987, 40.70972498299988], [-73.95087492699996, 40.70951311499991], [-73.95023693799996, 40.70547324699994], [-73.9579043079999, 40.70842717299994], [-73.958455768, 40.707251846999895], [-73.96838933699995, 40.70682918699989], [-73.96984864823531, 40.708003041687675], [-73.96736391666705, 40.71648367902752], [-73.95834207500002, 40.71330630099992]]]}}, {\"id\": \"256\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.23592163517446302, \"OBJECTID\": 257, \"Shape_Leng\": 0.0586690259793, \"Shape_Area\": 0.00013890947321, \"zone\": \"Windsor Terrace\", \"LocationID\": 257, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97984261899994, 40.660727440999956], [-73.97436701299992, 40.65818790399992], [-73.97084113699994, 40.64637857099993], [-73.97777380600002, 40.645341876999964], [-73.97819074399992, 40.64752599199992], [-73.98029115499993, 40.647297160999855], [-73.98179568699996, 40.65524609699991], [-73.98843135299977, 40.65925121199996], [-73.9868846449999, 40.6603663889999], [-73.985086794, 40.65901328799989], [-73.982230342, 40.66214149199994], [-73.97984261899994, 40.660727440999956]]]}}, {\"id\": \"257\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.22047778849365984, \"OBJECTID\": 258, \"Shape_Leng\": 0.0890133787693, \"Shape_Area\": 0.000366209617143, \"zone\": \"Woodhaven\", \"LocationID\": 258, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8450419489999, 40.68931894699996], [-73.84119605699982, 40.68335136599989], [-73.84786817599985, 40.681486831999905], [-73.85197864599989, 40.682564210999864], [-73.85099089299987, 40.68056836699987], [-73.85392389199987, 40.679686155999875], [-73.85819836099995, 40.67943084899987], [-73.85878576899987, 40.68063046299989], [-73.86334943599995, 40.67935163999989], [-73.86410096700003, 40.68237284999988], [-73.86602666899996, 40.681918051999915], [-73.86868454399996, 40.694034691999924], [-73.85277238699999, 40.69755711199987], [-73.85443455699989, 40.70060295199994], [-73.85070842300001, 40.70020312499988], [-73.8450419489999, 40.68931894699996]]]}}, {\"id\": \"258\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.27829088737692265, \"OBJECTID\": 259, \"Shape_Leng\": 0.126750305191, \"Shape_Area\": 0.000394552487366, \"zone\": \"Woodlawn/Wakefield\", \"LocationID\": 259, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.851071161919, 40.91037152011096], [-73.85347522366716, 40.90753029351422], [-73.84138689269206, 40.90417274176649], [-73.83554853999995, 40.88913350399993], [-73.84704957999995, 40.8876718159999], [-73.84590568599998, 40.889741368999864], [-73.860710982, 40.89537054399996], [-73.87048672599987, 40.896640210999905], [-73.8771197409999, 40.895468663999964], [-73.86789043706806, 40.90298695407611], [-73.85946778750369, 40.90051720913338], [-73.85957882974115, 40.9024408428857], [-73.85601023344621, 40.9053059838135], [-73.85681013619275, 40.90615582816297], [-73.85374103615364, 40.90778791636777], [-73.85458940544117, 40.9089394991491], [-73.851071161919, 40.91037152011096]]]}}, {\"id\": \"259\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20437281528599283, \"OBJECTID\": 260, \"Shape_Leng\": 0.133514154636, \"Shape_Area\": 0.000422345326907, \"zone\": \"Woodside\", \"LocationID\": 260, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9017537339999, 40.760775474999946], [-73.90147053099994, 40.759633049999884], [-73.89866137799999, 40.759921016999904], [-73.89976862099998, 40.75666423699991], [-73.8949120899999, 40.74428123099996], [-73.90057825399985, 40.73949289599988], [-73.90926002999988, 40.736403179999854], [-73.91068876399997, 40.73347108099995], [-73.90628903499987, 40.732400751999876], [-73.90516948899983, 40.73585964299986], [-73.90107970900003, 40.73391463399987], [-73.90712253099991, 40.727424727999924], [-73.91174734199988, 40.72991203299991], [-73.91205021299994, 40.72850636299993], [-73.91796696699996, 40.7285007709999], [-73.917342506, 40.731382825999916], [-73.91903825199986, 40.73200596599987], [-73.91651190599994, 40.738944757999874], [-73.9137938869998, 40.742514491999934], [-73.9119412979999, 40.742382948999925], [-73.91190979099993, 40.748105394999875], [-73.90945182799983, 40.75065550699994], [-73.91067474899997, 40.752995076999895], [-73.90631505099988, 40.75984387599992], [-73.90779405499987, 40.760517799999896], [-73.90356915599997, 40.765781788999924], [-73.9017537339999, 40.760775474999946]]]}}, {\"id\": \"260\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20724657615053085, \"OBJECTID\": 261, \"Shape_Leng\": 0.0271204563616, \"Shape_Area\": 3.43423231652e-05, \"zone\": \"World Trade Center\", \"LocationID\": 261, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01332610899989, 40.7050307879999], [-74.01668613099999, 40.70479933799992], [-74.01375388499997, 40.71369055499987], [-74.00860085399992, 40.711392239999945], [-74.01332610899989, 40.7050307879999]]]}}, {\"id\": \"261\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.20094480581780036, \"OBJECTID\": 262, \"Shape_Leng\": 0.0490636231541, \"Shape_Area\": 0.000122330270966, \"zone\": \"Yorkville East\", \"LocationID\": 262, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.94383256699986, 40.78285908899991], [-73.94207418803879, 40.77691784697707], [-73.94293043781381, 40.774676268035776], [-73.9474897547817, 40.77011515400953], [-73.95268514999995, 40.772339404999926], [-73.94383256699986, 40.78285908899991]]], [[[-73.93804640603422, 40.78082954427547], [-73.93759894622617, 40.78046784086141], [-73.93958378972465, 40.779576474007136], [-73.9387442308927, 40.78104387604228], [-73.93804640603422, 40.78082954427547]]]]}}, {\"id\": \"262\", \"type\": \"Feature\", \"properties\": {\"tip_fraction\": 0.2030808607390993, \"OBJECTID\": 263, \"Shape_Leng\": 0.0370166252994, \"Shape_Area\": 6.57697664169e-05, \"zone\": \"Yorkville West\", \"LocationID\": 263, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95218621999996, 40.7730198449999], [-73.95729409999986, 40.7742835549999], [-73.94933170599992, 40.78519312699994], [-73.94472264499986, 40.78324757199987], [-73.95218621999996, 40.7730198449999]]]}}]}"},"id":"79833421-4b37-4874-b257-e91f6932982f","type":"GeoJSONDataSource"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"xs":{"field":"xs"},"ys":{"field":"ys"}},"id":"027cf8e6-f083-4db2-bf4b-6e525b959e2d","type":"Patches"},{"attributes":{},"id":"7297bf13-8681-427a-88cb-a60cd7da3cc9","type":"BasicTicker"},{"attributes":{"plot":{"id":"ae53cc6f-7f97-4812-a208-ab913d90eb10","subtype":"Figure","type":"Plot"},"ticker":{"id":"72328d45-1f26-4e3d-9388-ba39a0b07bbd","type":"BasicTicker"},"visible":false},"id":"8ee03255-55ef-4637-8819-7dae96f8299a","type":"Grid"},{"attributes":{},"id":"aa4113e2-b3e7-4170-a486-16fa7c11995b","type":"HelpTool"},{"attributes":{"formatter":{"id":"7eade2b2-57b6-47a4-9a00-ec5b4a408ce6","type":"BasicTickFormatter"},"plot":{"id":"62ba6d47-3ae9-4aa5-be3a-40808ec4ef1a","subtype":"Figure","type":"Plot"},"ticker":{"id":"2a08cff9-d6de-4308-8dfb-94ef1c977983","type":"BasicTicker"},"visible":false},"id":"981a4af8-3ccf-4507-8498-13e2a0e4095e","type":"LinearAxis"},{"attributes":{"formatter":{"id":"4863a02c-a14f-44ad-b059-4fa0a7811065","type":"BasicTickFormatter"},"plot":{"id":"3058396c-651f-4f74-b4de-a7e737c8665a","subtype":"Figure","type":"Plot"},"ticker":{"id":"7297bf13-8681-427a-88cb-a60cd7da3cc9","type":"BasicTicker"},"visible":false},"id":"1b83b15e-0089-4812-ad82-87c5a6ed9480","type":"LinearAxis"},{"attributes":{},"id":"26612bd9-9ba6-4d61-95ab-5284b5499ee3","type":"LinearScale"},{"attributes":{},"id":"9d08d013-0e4e-429d-a37a-b7275a5c84b8","type":"LinearScale"},{"attributes":{"below":[{"id":"2a692866-0932-40ef-ab57-5680f5882b98","type":"LinearAxis"}],"left":[{"id":"e0e3f2e8-d02e-461e-ad09-3f799e56ae49","type":"LinearAxis"}],"renderers":[{"id":"2a692866-0932-40ef-ab57-5680f5882b98","type":"LinearAxis"},{"id":"8ee03255-55ef-4637-8819-7dae96f8299a","type":"Grid"},{"id":"e0e3f2e8-d02e-461e-ad09-3f799e56ae49","type":"LinearAxis"},{"id":"36cfaa0d-2256-478f-9e32-09a8af5f4faf","type":"Grid"},{"id":"3872cd96-a3c4-4d3c-a8d3-d671c5190f24","type":"BoxAnnotation"},{"id":"45c59d93-f760-4cc4-a732-46d46cef1f47","type":"GlyphRenderer"},{"id":"01599911-7bd9-4ceb-93b4-1d76c6b84013","type":"ColorBar"}],"right":[{"id":"01599911-7bd9-4ceb-93b4-1d76c6b84013","type":"ColorBar"}],"title":{"id":"a4bd85d5-54d7-4339-95e8-726aa5564454","type":"Title"},"toolbar":{"id":"f408f7d2-a34f-43a8-8561-003142236b1c","type":"Toolbar"},"x_range":{"id":"24209739-d5c4-4529-abcf-6ff415a2dfd5","type":"DataRange1d"},"x_scale":{"id":"a41f3949-e45f-4f6e-b29c-14b4306bdc8a","type":"LinearScale"},"y_range":{"id":"d92819cb-157e-408e-979a-4aab8e0d4596","type":"DataRange1d"},"y_scale":{"id":"b868a9e2-54ac-4484-a27c-2920d66d903b","type":"LinearScale"}},"id":"ae53cc6f-7f97-4812-a208-ab913d90eb10","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"a41f3949-e45f-4f6e-b29c-14b4306bdc8a","type":"LinearScale"},{"attributes":{"formatter":{"id":"b7d58927-6ebb-4002-8dd1-f651f3cd7cc3","type":"BasicTickFormatter"},"plot":{"id":"3058396c-651f-4f74-b4de-a7e737c8665a","subtype":"Figure","type":"Plot"},"ticker":{"id":"2a7eb668-a58d-46ed-9e0b-0f25d7fff4c5","type":"BasicTicker"},"visible":false},"id":"eee6a093-e6be-4073-88b7-824d43352607","type":"LinearAxis"},{"attributes":{"callback":null,"geojson":"{\"type\": \"FeatureCollection\", \"features\": [{\"id\": \"0\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 1, \"Shape_Leng\": 0.116357453189, \"Shape_Area\": 0.0007823067885, \"zone\": \"Newark Airport\", \"LocationID\": 1, \"borough\": \"EWR\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.18445299999996, 40.6949959999999], [-74.18448899999999, 40.69509499999987], [-74.18449799999996, 40.69518499999987], [-74.18438099999997, 40.69587799999988], [-74.18428199999994, 40.6962109999999], [-74.18402099999997, 40.69707499999986], [-74.18391299999996, 40.697506999999845], [-74.18375099999997, 40.69779499999988], [-74.18363399999998, 40.6983259999999], [-74.18356199999994, 40.69845199999987], [-74.18354399999998, 40.69855999999987], [-74.18350799999996, 40.69870399999992], [-74.18327399999998, 40.70008999999988], [-74.18315699999994, 40.701214999999884], [-74.18316599999997, 40.70238499999987], [-74.18313899999998, 40.702627999999876], [-74.18309399999998, 40.7028529999999], [-74.18299499999995, 40.703158999999864], [-74.18284199999994, 40.70346499999988], [-74.18264399999998, 40.703734999999874], [-74.18242799999996, 40.703950999999904], [-74.18220299999996, 40.70413999999989], [-74.18203199999994, 40.70425699999987], [-74.18180699999994, 40.704391999999885], [-74.18157299999996, 40.704499999999875], [-74.18132099999997, 40.70460799999991], [-74.18080799999996, 40.704787999999894], [-74.179467, 40.70534599999992], [-74.17887299999995, 40.705543999999875], [-74.17831499999994, 40.70572399999986], [-74.17776599999996, 40.70589499999988], [-74.17709099999996, 40.706092999999896], [-74.17699199999998, 40.70613799999988], [-74.17689299999995, 40.70619199999987], [-74.17664999999994, 40.70641699999988], [-74.17642499999994, 40.706695999999916], [-74.17628999999994, 40.70689399999986], [-74.17608299999995, 40.70710999999988], [-74.17599299999995, 40.70719099999991], [-74.17589399999997, 40.7072629999999], [-74.17565999999994, 40.707379999999866], [-74.17538099999996, 40.707469999999894], [-74.17515599999996, 40.70751499999988], [-74.17475999999994, 40.70759599999993], [-74.17417499999993, 40.7076679999999], [-74.17388699999998, 40.70773099999992], [-74.17347299999994, 40.707748999999865], [-74.17275299999994, 40.707802999999906], [-74.17188899999996, 40.707910999999854], [-74.17163699999998, 40.707955999999875], [-74.17133999999999, 40.707964999999895], [-74.17120499999999, 40.707955999999875], [-74.16994499999998, 40.70797399999988], [-74.16888299999994, 40.7079379999999], [-74.16681299999993, 40.70785699999988], [-74.16442799999999, 40.70779399999987], [-74.16401399999995, 40.70777599999992], [-74.16233999999997, 40.707721999999876], [-74.16081899999995, 40.70764099999991], [-74.16057599999993, 40.70760499999988], [-74.16033299999998, 40.707568999999864], [-74.160063, 40.7074879999999], [-74.15938799999998, 40.7072629999999], [-74.15904599999999, 40.707145999999916], [-74.15891999999997, 40.70710999999989], [-74.15827199999995, 40.706875999999916], [-74.15459099999998, 40.705651999999866], [-74.15409599999998, 40.70544499999989], [-74.15401499999997, 40.70538199999988], [-74.15387999999996, 40.70532799999989], [-74.15376299999997, 40.70540899999987], [-74.15323199999995, 40.70524699999987], [-74.15317799999997, 40.70531899999987], [-74.15306999999996, 40.7052829999999], [-74.15359199999995, 40.70437399999987], [-74.15386199999995, 40.70384299999989], [-74.15513999999996, 40.70155699999985], [-74.15544599999998, 40.70108899999987], [-74.15575199999995, 40.7006659999999], [-74.15600399999994, 40.70026099999991], [-74.15635499999996, 40.69975699999986], [-74.15745299999998, 40.69809199999987], [-74.15754299999998, 40.6979389999999], [-74.15758799999998, 40.69781299999988], [-74.15762399999994, 40.69767799999991], [-74.15829899999994, 40.696705999999885], [-74.15951399999994, 40.694887999999885], [-74.15958599999993, 40.69476199999984], [-74.16014399999995, 40.69410499999987], [-74.16057599999993, 40.693222999999875], [-74.16262799999998, 40.69028899999989], [-74.16279899999995, 40.690027999999884], [-74.16290699999996, 40.689874999999866], [-74.16292499999997, 40.689874999999866], [-74.16295199999996, 40.689874999999866], [-74.16306899999995, 40.68989299999987], [-74.16309599999994, 40.689928999999886], [-74.16322199999996, 40.68998299999987], [-74.16331199999996, 40.68999199999992], [-74.16341099999994, 40.6900009999999], [-74.16352799999999, 40.69000999999986], [-74.16380699999996, 40.69004599999989], [-74.16410399999995, 40.6900819999999], [-74.16417599999994, 40.690081999999904], [-74.16422999999998, 40.69005499999986], [-74.16436499999998, 40.6900369999999], [-74.16450899999995, 40.68998299999986], [-74.16467099999994, 40.68988399999989], [-74.16479699999996, 40.689757999999884], [-74.16491399999995, 40.689586999999904], [-74.16499499999998, 40.689388999999885], [-74.16528299999999, 40.68891199999988], [-74.16542699999997, 40.68875899999989], [-74.16548099999994, 40.688632999999854], [-74.16560699999997, 40.688425999999865], [-74.16576899999995, 40.68802999999986], [-74.16587699999997, 40.68787699999991], [-74.16583199999997, 40.68757999999987], [-74.16582299999999, 40.68748999999985], [-74.16580499999998, 40.687156999999914], [-74.16582299999999, 40.68703999999986], [-74.16589499999998, 40.68684199999989], [-74.16604799999999, 40.68655399999988], [-74.16639899999996, 40.686022999999864], [-74.16650699999997, 40.68588799999986], [-74.16674099999994, 40.6854919999999], [-74.16695699999997, 40.68523099999986], [-74.16738899999996, 40.68454699999991], [-74.16781199999997, 40.68394399999988], [-74.16791099999995, 40.683790999999864], [-74.16804599999995, 40.68360199999991], [-74.16816299999994, 40.68347599999988], [-74.16822599999995, 40.683349999999905], [-74.16848699999997, 40.682998999999896], [-74.16886499999998, 40.68239599999987], [-74.16916199999997, 40.681999999999896], [-74.16929699999997, 40.68178399999987], [-74.16947699999997, 40.68155899999989], [-74.16981899999996, 40.681018999999885], [-74.16995399999996, 40.68087499999991], [-74.17005299999994, 40.68066799999987], [-74.17041299999994, 40.68015499999989], [-74.17051199999997, 40.67999299999987], [-74.17067399999996, 40.679650999999865], [-74.17093499999999, 40.67929099999988], [-74.17144799999994, 40.67847199999989], [-74.17151999999999, 40.678381999999885], [-74.17160999999999, 40.678255999999884], [-74.17193399999996, 40.67782399999988], [-74.17200599999995, 40.67773399999988], [-74.17283399999997, 40.67656399999988], [-74.17314899999997, 40.67619499999991], [-74.17322999999999, 40.6760779999999], [-74.17329299999994, 40.67601499999989], [-74.17358999999993, 40.67571799999991], [-74.17423799999995, 40.67493499999991], [-74.17437299999995, 40.674817999999895], [-74.17484999999994, 40.67432299999992], [-74.17500299999995, 40.67416999999991], [-74.17538999999995, 40.67375599999988], [-74.17604699999998, 40.673044999999895], [-74.17630799999995, 40.67276599999986], [-74.17641599999996, 40.67262199999986], [-74.17663199999998, 40.672396999999876], [-74.17678499999994, 40.67218099999991], [-74.17697399999997, 40.671937999999884], [-74.17709099999996, 40.67178499999988], [-74.17734299999995, 40.67155999999989], [-74.17754999999994, 40.67142499999989], [-74.17778399999997, 40.67131699999989], [-74.17802699999999, 40.671208999999855], [-74.17862999999994, 40.671037999999896], [-74.17888199999999, 40.671001999999866], [-74.17912499999994, 40.670992999999896], [-74.17933199999999, 40.67101099999991], [-74.17979099999997, 40.67115499999987], [-74.17997999999994, 40.671208999999884], [-74.18010599999997, 40.67126299999989], [-74.18030399999998, 40.67129899999986], [-74.18133899999998, 40.67170399999987], [-74.18213999999996, 40.67202799999989], [-74.18384999999995, 40.672648999999865], [-74.18437199999994, 40.6729099999999], [-74.18458799999996, 40.67302699999989], [-74.18492099999997, 40.673269999999896], [-74.18503799999996, 40.67335999999989], [-74.18513699999994, 40.673458999999866], [-74.18547899999999, 40.67390899999987], [-74.18594699999994, 40.67466499999989], [-74.18670299999997, 40.67578999999991], [-74.18733299999997, 40.67674399999987], [-74.18767499999996, 40.67729299999991], [-74.18795399999995, 40.67761699999989], [-74.18819699999995, 40.67792299999991], [-74.18852099999998, 40.67848099999987], [-74.18877299999997, 40.67885899999989], [-74.18905199999995, 40.679335999999864], [-74.18935799999997, 40.67975899999988], [-74.18949299999997, 40.680091999999895], [-74.18969999999996, 40.68079399999987], [-74.18977199999995, 40.68113599999987], [-74.189781, 40.681198999999886], [-74.18983499999996, 40.68131599999987], [-74.18991599999998, 40.68154099999988], [-74.18996999999996, 40.6818019999999], [-74.18999699999995, 40.682251999999885], [-74.18999699999995, 40.68262999999992], [-74.18996999999996, 40.68295399999991], [-74.18998799999997, 40.6831789999999], [-74.18995199999995, 40.68352099999987], [-74.18993399999994, 40.68370999999992], [-74.189871, 40.684078999999876], [-74.189781, 40.68481699999991], [-74.18976299999997, 40.685032999999876], [-74.18962799999997, 40.6861039999999], [-74.18955599999998, 40.68689599999987], [-74.18951999999996, 40.6872019999999], [-74.18947499999996, 40.68748999999985], [-74.18939399999994, 40.68773299999989], [-74.18939399999994, 40.687831999999894], [-74.18941199999995, 40.687939999999855], [-74.18940299999997, 40.68809299999987], [-74.18934899999994, 40.68826399999989], [-74.18922299999997, 40.688623999999905], [-74.18898899999994, 40.68904699999991], [-74.18870099999998, 40.689442999999855], [-74.18779199999994, 40.69018999999985], [-74.18723399999999, 40.69059499999986], [-74.18636999999995, 40.69118899999991], [-74.18591099999998, 40.69144999999987], [-74.18563199999994, 40.69164799999987], [-74.18445299999996, 40.6949959999999]]]}}, {\"id\": \"1\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 2, \"Shape_Leng\": 0.43346966679, \"Shape_Area\": 0.00486634037837, \"zone\": \"Jamaica Bay\", \"LocationID\": 2, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.82337597260663, 40.63898704717671], [-73.82277105438693, 40.63557691408512], [-73.82265046764824, 40.63536884414308], [-73.82253791037438, 40.63515817971118], [-73.82243347854202, 40.63494510056532], [-73.8223372611914, 40.63472978853998], [-73.82224934035098, 40.63451242737345], [-73.8221697909667, 40.63429320255108], [-73.82209868083864, 40.6340723011465], [-73.82203607056276, 40.63384991166226], [-73.82198201348179, 40.633626223869015], [-73.82193655563646, 40.633401428643005], [-73.82189973572993, 40.63317571780407], [-73.82187158509225, 40.632949283951106], [-73.82185212765629, 40.63272232029805], [-73.82184137993583, 40.63249502050889], [-73.82183935101227, 40.63226757853259], [-73.8218460425281, 40.63204018843732], [-73.82107505533321, 40.62973377456623], [-73.82081417085445, 40.629390439641234], [-73.82059122816396, 40.62892422501728], [-73.8202234085928, 40.62844358255744], [-73.8200841811797, 40.628261639526286], [-73.8199971542669, 40.62805115103772], [-73.81991014068153, 40.62784065311699], [-73.81973384601805, 40.627712304087076], [-73.81947843088552, 40.62762863286645], [-73.81796414839563, 40.62710556569678], [-73.81800752228978, 40.62647634385673], [-73.81769796593258, 40.62596162249622], [-73.81710538785215, 40.625384851107945], [-73.81611483429545, 40.624604655173044], [-73.8160864347078, 40.62431541030936], [-73.81664198730377, 40.62393220944394], [-73.8167299805857, 40.62387151407256], [-73.81795284720685, 40.62409328011744], [-73.81891708039825, 40.62309026716333], [-73.81889389047457, 40.62262915803264], [-73.81825303631376, 40.62209423858474], [-73.81778633288076, 40.62193777763477], [-73.81778718310667, 40.621626331776106], [-73.8169700782087, 40.62149155710105], [-73.81691065059803, 40.621869647095345], [-73.81653074259579, 40.62200251874798], [-73.81571468044605, 40.62148955151693], [-73.81557294001617, 40.62073484077254], [-73.81543174704264, 40.62011653124222], [-73.81639131315181, 40.61962195550727], [-73.81682455918784, 40.61946156485392], [-73.81718053878453, 40.619267274493204], [-73.81744426110988, 40.61891175591167], [-73.81744645138826, 40.61811089699635], [-73.81764711074565, 40.617611944960316], [-73.81777429834523, 40.61750112169565], [-73.81775726750763, 40.61730714881576], [-73.81760298632462, 40.616983832581454], [-73.81751717814184, 40.61639641166187], [-73.8171684613709, 40.61584319772224], [-73.81711893622217, 40.61528219234544], [-73.81652681189122, 40.61494700958006], [-73.81768666599999, 40.61493299799986], [-73.82075892599994, 40.61523267899991], [-73.82141601599992, 40.615054762999925], [-73.8215706029998, 40.61490868999993], [-73.82163921899993, 40.614870614999845], [-73.82170208100001, 40.61482419099985], [-73.82176857999997, 40.61476625499992], [-73.82234961099986, 40.61426002599995], [-73.82379847999995, 40.61212942099992], [-73.8238246859766, 40.61149866495507], [-73.82397101271293, 40.61192149929154], [-73.82403572267913, 40.61213048090672], [-73.82410048459043, 40.61231956879447], [-73.82424356049395, 40.61250877658137], [-73.82432166316987, 40.61258847036726], [-73.8243870281721, 40.61254878272031], [-73.82436121280683, 40.61243932957914], [-73.82444821239913, 40.61234112665787], [-73.82458320331664, 40.61239988154398], [-73.82467451637949, 40.612419914122576], [-73.82473943706756, 40.612549320589395], [-73.82460797477792, 40.61290720498259], [-73.82468584347531, 40.61307641875661], [-73.824907237176, 40.61326574495243], [-73.82520683961239, 40.61349497719592], [-73.82525834646223, 40.613763619289685], [-73.82540161085667, 40.61388319788558], [-73.82555758997881, 40.61413210471074], [-73.82575322105271, 40.614192081540075], [-73.82593603373292, 40.614162518209255], [-73.82609287085626, 40.614083182229585], [-73.826197496476, 40.61400376542788], [-73.82619778134861, 40.61389435100908], [-73.82635390865978, 40.61378615702355], [-73.82662889857586, 40.613745800855575], [-73.82673157371033, 40.613664956123905], [-73.82703834217757, 40.6133950021526], [-73.82715194745357, 40.613378557154796], [-73.82725636632989, 40.61337871412455], [-73.82737373432043, 40.61341867797081], [-73.82747351895246, 40.61355236332681], [-73.82747779183, 40.61355808935558], [-73.82758192904869, 40.61366765928358], [-73.82767204227477, 40.613624148862804], [-73.82772568489919, 40.61359824867005], [-73.82787525471795, 40.613697628598395], [-73.82769886106767, 40.61387671827217], [-73.82758123538642, 40.613936221503465], [-73.8273200321807, 40.61399551069022], [-73.82724153726997, 40.61406502057191], [-73.82750222609539, 40.61420466672019], [-73.82773709441845, 40.614234859242316], [-73.82807653538532, 40.614205527127346], [-73.82845503182497, 40.614216039151735], [-73.82900287951898, 40.61435611007713], [-73.82932878428791, 40.61451574332958], [-73.82964171605141, 40.6146455161249], [-73.83009815098022, 40.614805340060016], [-73.83054575285917, 40.61493306078065], [-73.83091996001404, 40.61500548703347], [-73.8314812223769, 40.615006311228164], [-73.83170324361234, 40.61495690244544], [-73.8320297090652, 40.61489770036318], [-73.83246091965542, 40.614709339811256], [-73.83331105440271, 40.61449331081881], [-73.83401888200429, 40.61438116642064], [-73.83405492385512, 40.61436188793414], [-73.83406504547996, 40.61435647443239], [-73.83410581152143, 40.61435944275289], [-73.8341784916688, 40.61436199003059], [-73.83434168122284, 40.61436543719744], [-73.83445946628927, 40.614392450421285], [-73.8346032336605, 40.61441390158446], [-73.83472391325719, 40.61445052666856], [-73.83484247863916, 40.6144679338274], [-73.83490183563694, 40.614468157475095], [-73.83496190251225, 40.61447290875004], [-73.83501220878382, 40.61449233293344], [-73.83504177385295, 40.61450939365018], [-73.83506620220827, 40.61452021523822], [-73.83512098542279, 40.61453569210867], [-73.83517134812571, 40.61454663430032], [-73.83520623957126, 40.61453597904436], [-73.83525501050032, 40.61454974496719], [-73.83532281744199, 40.614570675723115], [-73.83538891957515, 40.614596598377865], [-73.83542288683323, 40.61461045443579], [-73.83545449273417, 40.614632458068606], [-73.83546955939202, 40.614657264070445], [-73.83547876382073, 40.61469339938509], [-73.83545814612324, 40.61471384545857], [-73.83543987928485, 40.61473067291295], [-73.83542879838902, 40.614754701507835], [-73.83542479353247, 40.614772372280925], [-73.83542777742117, 40.614774599832785], [-73.83546600150866, 40.614873466279484], [-73.83551649420116, 40.61494874641214], [-73.83558894560753, 40.61502891644492], [-73.83565778576532, 40.61509867837635], [-73.83574792728226, 40.6151629251546], [-73.83579322194048, 40.61518439875596], [-73.83585572623771, 40.615208999052946], [-73.83590450244563, 40.6152205139514], [-73.83598591313452, 40.61523188491708], [-73.83604642983012, 40.61522977594597], [-73.83610164292601, 40.61523130833409], [-73.83615138042873, 40.61521842487933], [-73.83620170280982, 40.61520235666928], [-73.83622173494702, 40.6151850655438], [-73.83622104005505, 40.61517466115949], [-73.83621145405357, 40.61516204947466], [-73.83615370058651, 40.61513926713333], [-73.83614289101911, 40.61512618712993], [-73.83614816967186, 40.61511801504228], [-73.83619025265256, 40.61511239276597], [-73.83623192892556, 40.615125765525484], [-73.83625516805921, 40.61513787463743], [-73.83627296218822, 40.61513779075455], [-73.83629664347178, 40.6151308795602], [-73.83631971425355, 40.615122623237674], [-73.83634403495043, 40.615119337102726], [-73.83634298945309, 40.61513473338826], [-73.83630165045997, 40.61515978999668], [-73.8362555309456, 40.615180832315154], [-73.836256190453, 40.61519123497351], [-73.83625868102743, 40.61520342593361], [-73.83628443659826, 40.615232255019855], [-73.83632259460953, 40.615255586365365], [-73.8363821447316, 40.6152801816666], [-73.83642080855896, 40.61528902058307], [-73.83647071166415, 40.61529647509992], [-73.83650810164664, 40.61529584208576], [-73.83654842492393, 40.61528840551515], [-73.83658509337694, 40.61527286721724], [-73.83660620420999, 40.61524245801393], [-73.83660293659774, 40.61520855451133], [-73.83659202032008, 40.61517966383299], [-73.83660663290642, 40.61515470684256], [-73.83663626355332, 40.61514552697837], [-73.83667370236468, 40.61515392380227], [-73.8367397195242, 40.61517081523142], [-73.83679106388888, 40.61520810759664], [-73.83684841155214, 40.61524988249358], [-73.83686341804386, 40.61527016091644], [-73.83688967112587, 40.61528677736101], [-73.83691998426377, 40.61529250094956], [-73.83697872940104, 40.61529313377217], [-73.83702921808582, 40.61529740498504], [-73.83707080859173, 40.61530171834586], [-73.8371224799956, 40.61530870901978], [-73.83717716917288, 40.615317953526734], [-73.83721530341752, 40.615336754552786], [-73.83724567284065, 40.61534925887809], [-73.83726642394453, 40.61534689979378], [-73.83726283632416, 40.61534104878436], [-73.83727813973981, 40.61532877380782], [-73.83729720614221, 40.61533815998089], [-73.83733296587165, 40.61535607975402], [-73.83736778999722, 40.615402955273666], [-73.83738923070878, 40.615413250620804], [-73.83742250503579, 40.61541670120903], [-73.83746760280816, 40.61541558474026], [-73.83750808709809, 40.615430765838326], [-73.83753077231013, 40.61544830972719], [-73.83757377957497, 40.615477932533985], [-73.83761124299842, 40.61549088655009], [-73.83770572740917, 40.615502629926624], [-73.83774137083236, 40.61550923917903], [-73.83776702143372, 40.61552223141874], [-73.83778556555642, 40.6155393313238], [-73.83779167484144, 40.615559650342874], [-73.83778783177006, 40.61559991222926], [-73.83775556201697, 40.61564304180018], [-73.8377251129955, 40.615692486933575], [-73.83771154036306, 40.615704325318845], [-73.83767959707981, 40.615717125850004], [-73.8376222064016, 40.61573730141596], [-73.83749594902788, 40.615756474960634], [-73.83742299815893, 40.615761778476745], [-73.83729967326363, 40.61577414773579], [-73.83724993001131, 40.61578931091298], [-73.83721865108845, 40.615810264393254], [-73.83717327566708, 40.61585074017479], [-73.83704926990326, 40.615994693811], [-73.83701342650974, 40.6160410031713], [-73.83695526084304, 40.61611275149117], [-73.83690989149997, 40.6161509496178], [-73.83688206247315, 40.61615969401999], [-73.83682027132109, 40.616152304059945], [-73.83674770660578, 40.6161318351782], [-73.83661044646236, 40.61611305900015], [-73.8365605989011, 40.616112383691814], [-73.83653038538397, 40.61612141008841], [-73.83646539870269, 40.616156714051925], [-73.83644029720082, 40.616170402327306], [-73.83633821743527, 40.6162437078502], [-73.83621028429496, 40.616343380215234], [-73.8361498330899, 40.61639228980907], [-73.83594162529525, 40.61654526191713], [-73.83574510959822, 40.616717848584244], [-73.83558888700725, 40.61688072107752], [-73.83554316307831, 40.616931852746745], [-73.83543187671067, 40.61705630065517], [-73.83527378301078, 40.61724722195531], [-73.83520018779365, 40.61735205002421], [-73.83515679945094, 40.617418632078284], [-73.8351060716087, 40.61749651113463], [-73.83506760060499, 40.61755555184745], [-73.83503079738357, 40.61762461418816], [-73.8349818113314, 40.61774073002495], [-73.83496222645587, 40.61778030881177], [-73.83495416881976, 40.61783362863525], [-73.83494283780968, 40.617885709738324], [-73.83495035743844, 40.61791830026055], [-73.83496031607699, 40.617940849237], [-73.83496600704768, 40.61795901756522], [-73.83496699909188, 40.61796218646273], [-73.83497854383712, 40.617969668301804], [-73.83501307295018, 40.617960824747996], [-73.8350509052779, 40.61795635072771], [-73.83508222781771, 40.617961939542305], [-73.83511105765169, 40.61797128630448], [-73.8351473448356, 40.61799442268986], [-73.83516472778436, 40.618015059898376], [-73.83516892325414, 40.618037024292306], [-73.83515913111307, 40.618064622904704], [-73.83514848023428, 40.618074681660424], [-73.83512875541668, 40.61808288776387], [-73.83510899173636, 40.61807852259004], [-73.83508428232517, 40.61807859637329], [-73.83505386643535, 40.618084289936675], [-73.83503172539164, 40.61810879665894], [-73.83501865754488, 40.61813518284088], [-73.83500397387171, 40.61817345172101], [-73.83500412587884, 40.61821423958553], [-73.83498947570031, 40.61826820816863], [-73.8349682566798, 40.61831341308498], [-73.83497341954912, 40.618381134093426], [-73.83496138913816, 40.618468950331085], [-73.83496079783818, 40.61853293069665], [-73.83497585229192, 40.61859185621791], [-73.83503367077738, 40.61864817839469], [-73.83508817813792, 40.61870326344785], [-73.8351220063932, 40.61873015666991], [-73.83515004883148, 40.61873763417457], [-73.83516730584809, 40.61873634197401], [-73.83520566728862, 40.61870740421659], [-73.83521290614456, 40.61870794381802], [-73.83525802361956, 40.61870029561122], [-73.8353150144414, 40.61869719337912], [-73.83534349182173, 40.61870357464657], [-73.83535650781197, 40.61871309017983], [-73.83536182798973, 40.618732091245384], [-73.8353404200944, 40.618751055704635], [-73.83533027499944, 40.61877546876064], [-73.8353355951704, 40.618794471502646], [-73.83535629616419, 40.6188283988267], [-73.83538885612354, 40.61885784279428], [-73.83541435759753, 40.61887324960184], [-73.83543510318425, 40.618889115661545], [-73.8354611268023, 40.61892620767172], [-73.83549183618312, 40.61900585240989], [-73.83550350266252, 40.61910989580282], [-73.83549383449827, 40.61920350812297], [-73.83549488479503, 40.61927404976679], [-73.83549481789157, 40.619301196254895], [-73.83550310305235, 40.61931567324388], [-73.8355207781153, 40.61937899354336], [-73.83554976450334, 40.619427424622074], [-73.83557218292678, 40.61944571066124], [-73.83560872508038, 40.61947310115986], [-73.83565958049579, 40.61950495916942], [-73.835714468994, 40.619537261008624], [-73.83572870016225, 40.61955051178868], [-73.83574513689678, 40.61957556780851], [-73.83574706558579, 40.61961152632981], [-73.83572583140196, 40.61966265954252], [-73.83571454012318, 40.6196989844889], [-73.83570733216746, 40.619729166951956], [-73.8356673493013, 40.619758753071764], [-73.83564923064897, 40.619787987299176], [-73.83563870292694, 40.61980589568313], [-73.83560626485749, 40.61982849387304], [-73.83556282642932, 40.61985673059368], [-73.83554696400546, 40.61989126496709], [-73.8355369686483, 40.61991268752325], [-73.83553327031747, 40.6199376318377], [-73.83551236293482, 40.61995810614527], [-73.83549391596809, 40.619959753532186], [-73.83538308531075, 40.619942000135865], [-73.83531699866492, 40.61993724046734], [-73.83526739330783, 40.619953172108715], [-73.83522937910507, 40.61995819493326], [-73.83520875852072, 40.61995017859354], [-73.8351961685202, 40.61994310472235], [-73.83514645819189, 40.61989977659646], [-73.83511766376013, 40.61989062255555], [-73.83508243198399, 40.61989150465222], [-73.83504962373301, 40.61988863103023], [-73.83501268056406, 40.61988267613292], [-73.8349920065259, 40.61986722071551], [-73.83495815063704, 40.61983928773372], [-73.83495620861927, 40.61982089259598], [-73.83495585537514, 40.619817487562536], [-73.83495550518768, 40.61981400211941], [-73.83495511701948, 40.619810405885126], [-73.83495473079734, 40.619806644972414], [-73.83495434339326, 40.619802828605884], [-73.83495392079347, 40.61979893093737], [-73.8349534982628, 40.61979500529139], [-73.83495311224588, 40.619791161452135], [-73.83495272588691, 40.619787455991435], [-73.83495267403252, 40.619787176478376], [-73.8349530577307, 40.61978377786393], [-73.83495342746937, 40.61978015641562], [-73.83495375978538, 40.619776642969875], [-73.8349538231922, 40.61977607379697], [-73.83495466899879, 40.61977313152508], [-73.8349559415767, 40.61976857639302], [-73.83495605003769, 40.619768191734494], [-73.83495721478977, 40.619764120104065], [-73.83495757467031, 40.61976341850631], [-73.83495779121424, 40.619762979723426], [-73.83496003675234, 40.61975856319457], [-73.83496028948841, 40.619758069514056], [-73.83496249817028, 40.619753709557045], [-73.83496284723834, 40.61975302385905], [-73.83496513817671, 40.61974954069981], [-73.83496781315462, 40.619745454819714], [-73.83497048872262, 40.61974139691764], [-73.83497072365829, 40.619741077776865], [-73.834970814756, 40.6197409576218], [-73.83497446314935, 40.61973709276949], [-73.834974859619, 40.619736709026846], [-73.83497850647574, 40.619732842999525], [-73.83497886776435, 40.61973246088167], [-73.83498258899671, 40.61972856865893], [-73.83498291263044, 40.61972821061124], [-73.83498663238724, 40.619724292586724], [-73.83498695749064, 40.61972396251848], [-73.83498732003198, 40.61972360737459], [-73.83498764366566, 40.61972324932688], [-73.83499095388885, 40.619719752217684], [-73.8349913280428, 40.61971941300569], [-73.83499168832026, 40.61971908416062], [-73.83499573129845, 40.619715330108114], [-73.83499605765442, 40.619715027013896], [-73.83500010062926, 40.61971127413395], [-73.83500038938872, 40.61971097148827], [-73.83500447068995, 40.61970718951313], [-73.83500475739861, 40.61970691601449], [-73.83500887401334, 40.61970307863788], [-73.83500988633324, 40.619702147456245], [-73.83501013817067, 40.61970192818702], [-73.83501421668167, 40.619698118230254], [-73.83501465125362, 40.619697707234984], [-73.83501872969266, 40.61969392626034], [-73.83503035670653, 40.61968310046283], [-73.83503295469998, 40.619680716231215], [-73.83503562771511, 40.619678222375775], [-73.8350388417635, 40.61967520761084], [-73.8350420905473, 40.6196721935658], [-73.83504541136438, 40.619669122496774], [-73.83504600631076, 40.619668582063824], [-73.83504866200768, 40.61966588949363], [-73.83505187585239, 40.61966260081804], [-73.8350527483464, 40.619661757220676], [-73.83505566434685, 40.61965977853022], [-73.8350599580499, 40.619656875213074], [-73.8350628545967, 40.619654939381846], [-73.83506468473206, 40.61965367113188], [-73.83506995210881, 40.619650083012125], [-73.83507612289257, 40.619645863765754], [-73.83507655587348, 40.6196455624994], [-73.83508222142333, 40.61964170104017], [-73.8350881028403, 40.61963767403644], [-73.8350938384386, 40.619633757727634], [-73.83509986472069, 40.61962964917692], [-73.83510571089066, 40.61962564959658], [-73.83510776823056, 40.61962425384717], [-73.83511087117782, 40.619621622730016], [-73.83511563653995, 40.61961762159892], [-73.83511787957178, 40.61961572654381], [-73.83511986031176, 40.61961356591393], [-73.83512339906771, 40.619609755179], [-73.8351257453126, 40.61960719601896], [-73.8351265079198, 40.619605890050686], [-73.83512882201632, 40.61960174869952], [-73.83513117502804, 40.61959760690171], [-73.83513348964644, 40.6195935210033], [-73.83513580587247, 40.61958940712991], [-73.8351392553219, 40.61958329944254], [-73.83514423938809, 40.61957273031025], [-73.83514489049413, 40.61957134896137], [-73.83514551135299, 40.61956850571017], [-73.83514642197723, 40.619564417798145], [-73.83514715832642, 40.619561102615606], [-73.83514733267062, 40.619560301908926], [-73.83514824336787, 40.61955618434434], [-73.83514915406101, 40.61955206845503], [-73.83514942572108, 40.61955085509625], [-73.83514909177528, 40.61954808954956], [-73.83514863265606, 40.61954410839969], [-73.83514832854563, 40.61954145815572], [-73.83514795827979, 40.61954004568952], [-73.83514688852236, 40.61953592696025], [-73.83514610712336, 40.61953291366635], [-73.83514570968407, 40.619531643393394], [-73.83514499386496, 40.61952949481127], [-73.83514377633992, 40.61952716658857], [-73.83514276321685, 40.61952522012284], [-73.83514069003044, 40.61952244052232], [-73.83513829404646, 40.619519236107664], [-73.83513659621009, 40.61951741279902], [-73.83513451176393, 40.619515186530826], [-73.83513270384385, 40.6195132483068], [-73.83513109800163, 40.619511831890556], [-73.83512599216328, 40.6195072593997], [-73.83512498779872, 40.61950666289585], [-73.83512039717206, 40.61950383411642], [-73.83511725433776, 40.61950223322396], [-73.83511367982304, 40.61950040772576], [-73.83511143853501, 40.619499551452805], [-73.83510843704316, 40.61949876160357], [-73.83510807841144, 40.61949869189964], [-73.83510471762445, 40.61949783402076], [-73.83510247642766, 40.61949783080596], [-73.83509911353333, 40.619497825982094], [-73.83509575041911, 40.61949782115781], [-73.83509013557159, 40.619499586231306], [-73.83508234969528, 40.619502045948664], [-73.8350822889517, 40.61950207166105], [-73.83507654554161, 40.61950390406364], [-73.8350720393931, 40.619505325449445], [-73.83506882681762, 40.61950632433968], [-73.83506854261785, 40.61950647269773], [-73.83506630662428, 40.61950756680533], [-73.8350643232077, 40.619508497766795], [-73.83506259282973, 40.61950934566194], [-73.83506107884065, 40.61951008396828], [-73.8350564819437, 40.61951228473611], [-73.83505494634642, 40.61951331518214], [-73.8350538626772, 40.619514027803866], [-73.83505285389694, 40.619514684578256], [-73.83505195181418, 40.619515314030984], [-73.83505108598817, 40.61951586228417], [-73.83505029146416, 40.61951638266211], [-73.83504749952071, 40.61951825112877], [-73.83503401904368, 40.61953019034733], [-73.83501812440286, 40.619545924822], [-73.8350174383641, 40.61954658373455], [-73.83501678750642, 40.61954724102232], [-73.83501617306054, 40.619547843412874], [-73.83501501883251, 40.61954899452402], [-73.83501444057265, 40.61954954436226], [-73.83501389947668, 40.61955009090331], [-73.83501335800345, 40.61955061214684], [-73.83501285170817, 40.619551132938355], [-73.83501176920393, 40.619552174420875], [-73.83501122810381, 40.61955272263716], [-73.83500817335559, 40.619555777669376], [-73.83500797686946, 40.61955592933623], [-73.83500462000539, 40.619558998174725], [-73.83499739945185, 40.61956560387044], [-73.83499693894599, 40.61956601164536], [-73.83499353773406, 40.619568343124314], [-73.83498967769185, 40.619571026928355], [-73.83498581617673, 40.6195736839255], [-73.83498202816111, 40.61957631338561], [-73.83497823933318, 40.61957891553737], [-73.83497398161872, 40.61958184688058], [-73.8349722351652, 40.61958305979717], [-73.83496957971555, 40.6195845853582], [-73.83496499971633, 40.61958724015046], [-73.83496034584253, 40.619589896009245], [-73.83495565555567, 40.61959260710008], [-73.83495093009276, 40.61959531763774], [-73.83494616931696, 40.619598083073946], [-73.83494173227032, 40.619600629344255], [-73.83493968118961, 40.619601804463855], [-73.83493743655649, 40.61960265597438], [-73.8349366472464, 40.61960284532163], [-73.83493407154735, 40.6196035042009], [-73.8349318282333, 40.61960435571321], [-73.83492954791917, 40.619604344229906], [-73.83492846599323, 40.61960435088531], [-73.83491650941178, 40.61960388741827], [-73.83490492692965, 40.61960346234883], [-73.83490307284829, 40.619603922569006], [-73.8349015619159, 40.619604312249635], [-73.8349014869658, 40.61960430393302], [-73.83490141551309, 40.6196043038304], [-73.83490130690464, 40.619604303674414], [-73.83490105473129, 40.619604303312244], [-73.83489819967585, 40.619604307420836], [-73.83489694687438, 40.61960459997019], [-73.83489483466322, 40.61960515681886], [-73.83489371362325, 40.61960515520877], [-73.8348925904722, 40.61960600715647], [-73.83489197374509, 40.619606020007964], [-73.8348914696521, 40.61960600554669], [-73.83488922611876, 40.619606856555194], [-73.83488922400447, 40.619607711285646], [-73.83488810107427, 40.61960856273096], [-73.83488782426414, 40.619608923023556], [-73.83488684804733, 40.61961010186395], [-73.83488659512682, 40.6196104033881], [-73.83488655959444, 40.61961045828664], [-73.83488652215, 40.61961048621021], [-73.83488648749562, 40.61961054161261], [-73.83488294676292, 40.619614790425906], [-73.8348824800685, 40.61961538867216], [-73.83487998302958, 40.61961917727434], [-73.83487998296161, 40.61961920474897], [-73.83487797702384, 40.6196222162189], [-73.83487723359994, 40.619623675334644], [-73.83487719740646, 40.61962373073483], [-73.83487480869455, 40.619628255112715], [-73.83487477131926, 40.61962825505903], [-73.83487347349393, 40.61963075038366], [-73.83487336078586, 40.619631163181], [-73.83487317748829, 40.61963184911696], [-73.83487154282814, 40.6196379944168], [-73.83487154262299, 40.6196380773434], [-73.83487120962234, 40.619639288938124], [-73.83487052091054, 40.61964307142602], [-73.83486780155245, 40.61965807494431], [-73.83486805894447, 40.61966272005846], [-73.8348680582627, 40.61966299564281], [-73.83486833459253, 40.619668072841606], [-73.8348683682949, 40.61966840225223], [-73.83486860929065, 40.61967345410345], [-73.83486864373661, 40.61967383846478], [-73.8348688705577, 40.61967857684858], [-73.83486891960808, 40.61967883441131], [-73.83486895523896, 40.6196792730536], [-73.8348694830632, 40.61968426986476], [-73.83486955322564, 40.619684791483515], [-73.83486996756154, 40.61968882802232], [-73.83487026126434, 40.619689815525625], [-73.83487179751529, 40.61969492251164], [-73.83487308349402, 40.619699260514516], [-73.83487333383293, 40.619700002525484], [-73.83487458329964, 40.61970425821917], [-73.83487483468762, 40.619705109460575], [-73.83487608523482, 40.61970928390425], [-73.83487636955081, 40.61971024442188], [-73.8348766377702, 40.61971104626632], [-73.83487690090588, 40.61971416972173], [-73.8348770072858, 40.61971515963651], [-73.8348773571777, 40.61971902922375], [-73.83487746188544, 40.619720072912834], [-73.83487756799781, 40.61972117105097], [-73.83487767295904, 40.619722378919036], [-73.83487772768987, 40.61972300639408], [-73.83487774135509, 40.61972361471197], [-73.83487772981483, 40.61972827971068], [-73.8348777264198, 40.61972965210416], [-73.83487770655927, 40.619731548169995], [-73.83487735609071, 40.61973404385018], [-73.83487713717255, 40.619735442236205], [-73.83487656803999, 40.619739233607476], [-73.83487640579283, 40.61973958635823], [-73.834874311164, 40.6197452096468], [-73.83487206295995, 40.6197477689426], [-73.83486981541657, 40.61975032773673], [-73.83486307480958, 40.6197562973351], [-73.8348540914557, 40.619762264213854], [-73.83485325984896, 40.6197626644189], [-73.83485160003306, 40.619763513417276], [-73.83484986957967, 40.61976438811365], [-73.83484766933802, 40.619765511418485], [-73.83484543326422, 40.61976663299639], [-73.834843232584, 40.619767755797824], [-73.83484139316472, 40.61976868578969], [-73.83483958986686, 40.6197695895313], [-73.83483778657002, 40.61977049277032], [-73.83483605545379, 40.619771368638254], [-73.83483208731447, 40.619773394059855], [-73.83482967163266, 40.619774598306385], [-73.8348271829766, 40.619775884202035], [-73.83482458695994, 40.61977719859099], [-73.83482184592403, 40.61977856705094], [-73.83482042054405, 40.61977929894784], [-73.83481921278555, 40.61978004539738], [-73.8348167235161, 40.61978157890005], [-73.83480807158676, 40.619786968778634], [-73.83480499749777, 40.619789302397336], [-73.83480160512086, 40.619791876632924], [-73.83480021058838, 40.619792936260815], [-73.83479853640046, 40.61979461701979], [-73.83479346785425, 40.61979976108818], [-73.83478202158658, 40.61981641545874], [-73.83477996605193, 40.61981938614613], [-73.83477872965373, 40.61982148803197], [-73.8347756193006, 40.619826835446226], [-73.83477546438883, 40.61982706674876], [-73.83477427491415, 40.619831528684536], [-73.83477320071925, 40.619835604798844], [-73.83477310957899, 40.61983627495178], [-73.83477206006914, 40.6198441454682], [-73.83477203469464, 40.61985439620219], [-73.8347752830575, 40.61987536395475], [-73.83477534387069, 40.619875754384886], [-73.8347767126989, 40.619879839539045], [-73.83478171846966, 40.6198946417306], [-73.83478275512489, 40.619897826608415], [-73.83478386448336, 40.619901039568134], [-73.83478424558758, 40.61990224565519], [-73.83478963457533, 40.61991296035216], [-73.8347898237671, 40.619913358003075], [-73.83479124855907, 40.6199157074684], [-73.83479297066026, 40.61991856447163], [-73.83479476431347, 40.61992155945395], [-73.83479652145589, 40.619924471457], [-73.83479670104235, 40.61992466387095], [-73.83479943094244, 40.61992779740505], [-73.83480223326198, 40.61993098482005], [-73.83480503427865, 40.61993425465736], [-73.83480546312809, 40.61993473557952], [-73.83480812350956, 40.619937442987016], [-73.83481143130801, 40.61994079698195], [-73.8348148431988, 40.61994428833264], [-73.83481552717186, 40.61994499963882], [-73.83481894145231, 40.6199476143131], [-73.83482318391319, 40.619950859248895], [-73.83482746235862, 40.619954133386436], [-73.8348278341032, 40.61995441335898], [-73.83483199380255, 40.61995707517439], [-73.83483652603833, 40.61995996368912], [-73.8348409850447, 40.61996277034422], [-73.83484126464018, 40.619962973288594], [-73.83484537325917, 40.61996557639494], [-73.83484968839795, 40.61996832739093], [-73.83485393281136, 40.61997105081036], [-73.83485469539761, 40.61997153438967], [-73.8348581422215, 40.6199731709071], [-73.83486220641197, 40.61997507066187], [-73.83486605747945, 40.619976914658146], [-73.83486925121079, 40.61997838881105], [-73.83487911894214, 40.619982696923394], [-73.83488448117433, 40.61998503880824], [-73.83488973311268, 40.6199873508817], [-73.83489020092962, 40.61998754421212], [-73.83489070539567, 40.619987764567284], [-73.8348958868922, 40.619990023265096], [-73.8349006027422, 40.619992100194985], [-73.8349058543044, 40.61999420922202], [-73.83491071255162, 40.619996136919994], [-73.83491553445205, 40.61999809321307], [-73.83492035598107, 40.620000022030695], [-73.8349241199416, 40.62000152933402], [-73.8349251798322, 40.62000181196983], [-73.83493007474962, 40.62000313644744], [-73.83494497840654, 40.62000713951085], [-73.83494652564518, 40.62000754145676], [-73.83495037894774, 40.62000821660289], [-73.83495138570291, 40.62000838222683], [-73.83495682336319, 40.62000929636612], [-73.83496669339895, 40.62001098699956], [-73.8349729909799, 40.6200114324535], [-73.83497889815727, 40.62001185271982], [-73.83498473196929, 40.62001224590833], [-73.8349903869734, 40.620012665142056], [-73.83499593255537, 40.62001305741369], [-73.83503648675614, 40.62003029806967], [-73.83506194119016, 40.62003508370845], [-73.83511140979229, 40.620059693162624], [-73.83514945421035, 40.620086261214205], [-73.8351892113884, 40.62011966873837], [-73.83521267286356, 40.62012988546007], [-73.83524713409838, 40.6201344633323], [-73.8352833721993, 40.620134240681075], [-73.8352988919852, 40.62013698090934], [-73.83531201853388, 40.62014537181385], [-73.83530504599716, 40.620153649991664], [-73.83525703590811, 40.620180122834086], [-73.835228061921, 40.62019965339754], [-73.83517947408095, 40.62022697676222], [-73.83511109381281, 40.620260583712735], [-73.83505992869569, 40.62029591875947], [-73.83501945449157, 40.62033368338847], [-73.8349766806337, 40.620384565412664], [-73.83494291247693, 40.62043414290278], [-73.83487095582524, 40.62074744644516], [-73.83498716099349, 40.62101129988884], [-73.83497579529411, 40.62101949048852], [-73.8349325067799, 40.62102980425339], [-73.83488425945177, 40.621035169107174], [-73.83484210233186, 40.62106920004676], [-73.8348018333844, 40.62112568442332], [-73.83479426014556, 40.62117255353669], [-73.83477106595734, 40.621215174948134], [-73.83475117363801, 40.62126213775865], [-73.83477206223073, 40.621293211422895], [-73.83480605640716, 40.62132603264879], [-73.83485905038059, 40.62136747339013], [-73.83487573996096, 40.62139230975941], [-73.83486860012296, 40.62140983929493], [-73.83486007514385, 40.62144861103434], [-73.83485302732606, 40.62147239774434], [-73.8348617532606, 40.621512896024406], [-73.83486110670388, 40.621526647054374], [-73.8348400277696, 40.621543662018155], [-73.83481229773928, 40.62155509540437], [-73.83478297712946, 40.62156905116115], [-73.83474970830171, 40.62159238855885], [-73.83475426084244, 40.62162983516908], [-73.8347735869859, 40.621666505909815], [-73.83480360476744, 40.621705606942754], [-73.83480411594866, 40.62174620382317], [-73.83479301877969, 40.6217762439309], [-73.8347526375955, 40.62182023816353], [-73.834706371938, 40.621854290620504], [-73.8346676730216, 40.62190202129716], [-73.83464866926714, 40.62195398041147], [-73.8346451380572, 40.6219977273897], [-73.83466547391477, 40.622048755499776], [-73.8346921164322, 40.62208351605373], [-73.83472608413201, 40.62211323613151], [-73.83476020875996, 40.62215231560311], [-73.8347597397065, 40.62218165680579], [-73.83476098931946, 40.62221473285871], [-73.83478255229285, 40.62223515833474], [-73.83482124702645, 40.62224797713946], [-73.83485757192346, 40.622270810596916], [-73.83487846094775, 40.62230185743407], [-73.83488283700238, 40.62232368421127], [-73.83485615308435, 40.62234947551252], [-73.83484340044038, 40.62237890829777], [-73.8348497664996, 40.62242632062673], [-73.83489935765645, 40.62246028951774], [-73.83497320567402, 40.6225327758058], [-73.8349712094714, 40.622567741614176], [-73.8349690567791, 40.62259334751715], [-73.83498020634333, 40.62263008890535], [-73.83500259350858, 40.62265240876774], [-73.83502173941585, 40.62267348828023], [-73.83502020554216, 40.622682241984705], [-73.83500804316691, 40.622691694612286], [-73.83499413194808, 40.622694310213376], [-73.83534733990399, 40.62369577884393], [-73.83514355859735, 40.62548708279667], [-73.83342230809954, 40.626781983462095], [-73.83464377224975, 40.62742707942435], [-73.83304100907748, 40.628261271060616], [-73.83286528731682, 40.628408350262575], [-73.83286508734435, 40.62848820264789], [-73.83287721797437, 40.62859207399194], [-73.83288560857211, 40.62866390752471], [-73.83289548956779, 40.628903478732965], [-73.83268559314394, 40.6290149678853], [-73.8326635580086, 40.62902359237241], [-73.83236036671222, 40.62914225774722], [-73.83219207032315, 40.629381569075036], [-73.83196080768154, 40.62965272987937], [-73.83188649602349, 40.63002792793848], [-73.83183334645541, 40.63032330422833], [-73.83189666398192, 40.630572394052024], [-73.83171090635092, 40.63126149809776], [-73.83166394135613, 40.63142672049938], [-73.83160808634373, 40.63159032946402], [-73.83154343599053, 40.63175204747618], [-73.83147009989457, 40.63191160022591], [-73.8313882023895, 40.63206871707372], [-73.83129788233452, 40.632223131510294], [-73.83119929287734, 40.63237458160876], [-73.8311726114447, 40.63246698807974], [-73.83115393179625, 40.63256052752952], [-73.83114333418939, 40.632654798233716], [-73.83114086417227, 40.632749395325924], [-73.83114653238748, 40.63284391253632], [-73.83116031452592, 40.6329379439361], [-73.83118215143037, 40.633031085681694], [-73.83121194934962, 40.63312293774828], [-73.83124958033936, 40.63321310564855], [-73.83128625882125, 40.63341587766314], [-73.83133112864137, 40.633617691427716], [-73.8315519507211, 40.63384438316006], [-73.83168397540554, 40.63397811777474], [-73.83193878472684, 40.63423622542843], [-73.8319116731582, 40.63457621843594], [-73.83190631367316, 40.634643426189925], [-73.83195787786688, 40.63497888056586], [-73.83212076365058, 40.63512543734713], [-73.8323134547505, 40.635298808974646], [-73.83249094435719, 40.63557855183555], [-73.83245895783561, 40.63579410645499], [-73.83263580697175, 40.63632937448189], [-73.83290683934878, 40.63692866084925], [-73.83309423850194, 40.63743998965629], [-73.83340607979225, 40.63850135589897], [-73.83294956256923, 40.638823494236775], [-73.83289974361197, 40.63886268506689], [-73.832812666356, 40.63888193721898], [-73.8322165881916, 40.638928490220756], [-73.82938003995758, 40.637074927368005], [-73.82866782109734, 40.63661471844797], [-73.82808567353285, 40.63587972683653], [-73.82706379083446, 40.635811456523534], [-73.82665496763366, 40.635810840934084], [-73.8265099853732, 40.63578398166053], [-73.82636306756064, 40.635764144352805], [-73.82621479958787, 40.6357514080522], [-73.8260657722256, 40.63574582350641], [-73.82591657926903, 40.63574741296666], [-73.82576781517317, 40.635756170100024], [-73.82562007268534, 40.63577206001383], [-73.8254739404821, 40.63579501939495], [-73.82533000082624, 40.63582495676232], [-73.82518882724635, 40.63586175283105], [-73.82505098225157, 40.63590526098733], [-73.8249170150923, 40.63595530787298], [-73.82478745957022, 40.636011694076004], [-73.82466283191333, 40.63607419492457], [-73.82454362871778, 40.63614256138235], [-73.82443032497055, 40.6362165210408], [-73.82413577690048, 40.638787996704544], [-73.82457379890309, 40.639353873239195], [-73.82419327293158, 40.63995327448778], [-73.82337597260663, 40.63898704717671]]], [[[-73.84721834832125, 40.63196568670605], [-73.8485899664393, 40.63013198680654], [-73.8585583634798, 40.63045828819559], [-73.8561374187591, 40.63508154601468], [-73.84667914840234, 40.63873171175714], [-73.84352007697278, 40.636503685327924], [-73.84504859278633, 40.63317483899965], [-73.84721834832125, 40.63196568670605]]], [[[-73.79577948747226, 40.63215847659964], [-73.79803751834721, 40.631196239807494], [-73.79796829437942, 40.63110130747364], [-73.79811872226199, 40.6310375092611], [-73.79827615415297, 40.63125217936581], [-73.79812347917935, 40.631317681629135], [-73.7980553716909, 40.63122445859815], [-73.79581864007577, 40.63217769654574], [-73.79365017276974, 40.63310177435078], [-73.79364942874943, 40.633102090315504], [-73.79119486069375, 40.63414821967486], [-73.7908111127525, 40.63431035575525], [-73.7907928429953, 40.63428204310243], [-73.79356534109542, 40.63310193937666], [-73.79356608423642, 40.63310162341087], [-73.79577948747226, 40.63215847659964]]], [[[-73.83623731597828, 40.631110861231726], [-73.84110205447426, 40.62455643651514], [-73.84445711462239, 40.62984779443998], [-73.842423745765, 40.63153254924423], [-73.83765174362401, 40.63283973295598], [-73.83623731597828, 40.631110861231726]]], [[[-73.81307233892618, 40.62926109964263], [-73.81352502122876, 40.62730617268596], [-73.81258556038132, 40.628371916183724], [-73.81129432207656, 40.624703779000406], [-73.81135581995851, 40.62293751504066], [-73.81185181296344, 40.622097618373914], [-73.81210045334295, 40.62110059681489], [-73.81171450418677, 40.620703573736186], [-73.81171923661802, 40.62009423414049], [-73.81150542539498, 40.61979681095144], [-73.81109451157876, 40.61791025228185], [-73.81024066504992, 40.616487640654206], [-73.81089990481607, 40.61621847134095], [-73.81200717773355, 40.617179371483296], [-73.81211344524338, 40.61792554683542], [-73.81288213914506, 40.61891091965568], [-73.81370354000288, 40.62138674458606], [-73.81449543302412, 40.622394945305274], [-73.81418278559646, 40.6236946315581], [-73.81518203155235, 40.624744171143405], [-73.8161270318486, 40.62534899918069], [-73.81679073502552, 40.62634891812118], [-73.81383288662273, 40.62592623399761], [-73.81382899493154, 40.62613507764411], [-73.81627026043981, 40.62714580591711], [-73.81720935727392, 40.62910635115234], [-73.8173057692762, 40.63062197379782], [-73.81637181200166, 40.63137272824912], [-73.81506788441455, 40.63116405857793], [-73.81391733219806, 40.630776392218095], [-73.81307233892618, 40.62926109964263]]], [[[-73.84734350666976, 40.62909473971626], [-73.84984630167418, 40.62622449074592], [-73.8472783772798, 40.626621669791945], [-73.84504991718353, 40.62541106188362], [-73.84358010755503, 40.623178164184395], [-73.8461475400832, 40.622223609607744], [-73.84847781581286, 40.62241878479054], [-73.85665510191748, 40.61999364918834], [-73.85947697626024, 40.61866521683819], [-73.86107132214268, 40.61771207937058], [-73.86364988794934, 40.617878773642374], [-73.86404106276488, 40.61920759671402], [-73.86343910789503, 40.61894217513062], [-73.86151825485233, 40.619049218399226], [-73.86147000631671, 40.62123924718382], [-73.86102883383386, 40.624138986994076], [-73.86347880947007, 40.6233886093158], [-73.86334955670225, 40.62592490877777], [-73.86137632465496, 40.625979153993065], [-73.86041082270874, 40.62516691297755], [-73.86008391106512, 40.6256745170403], [-73.85861338934723, 40.62796202232805], [-73.8576777902821, 40.62798375894847], [-73.85786309317032, 40.628444902667404], [-73.8565055097758, 40.629097353726586], [-73.8561347984124, 40.628280911272874], [-73.8539654419115, 40.62761094389788], [-73.85436931062769, 40.62881767654389], [-73.85071696108574, 40.62800299210944], [-73.84846868196166, 40.6296433307532], [-73.84734350666976, 40.62909473971626]]], [[[-73.87084603194215, 40.61696742645758], [-73.8782477932424, 40.615444608331536], [-73.87481386865049, 40.62358718918363], [-73.86684384142526, 40.62782924286026], [-73.86498193495926, 40.625660706041025], [-73.86606533106142, 40.6214933267491], [-73.86626127665735, 40.618456348023955], [-73.87084603194215, 40.61696742645758]]], [[[-73.79783533398513, 40.627408716571516], [-73.79783848156961, 40.62701346630369], [-73.79744863612672, 40.627110476260945], [-73.79566256726108, 40.626114004327796], [-73.79921493427759, 40.6240847129885], [-73.79698347650404, 40.62486468974474], [-73.79403741688607, 40.62563519595493], [-73.7936292748135, 40.62515647619494], [-73.79809093107174, 40.62306294824432], [-73.79951085673736, 40.62117553302096], [-73.79992057632684, 40.61857524016598], [-73.80172498640165, 40.61996700873167], [-73.80096292242341, 40.61777236939703], [-73.80128049504093, 40.615879839747535], [-73.80270343867807, 40.61358065232171], [-73.8033982096119, 40.613172104348564], [-73.80508906012292, 40.61343256041943], [-73.80466968986522, 40.614301827300864], [-73.80476592616557, 40.61580099137086], [-73.80575302222282, 40.61669485753826], [-73.80565945060563, 40.617600255343724], [-73.80764116896117, 40.62119965469225], [-73.80728331329274, 40.62166643562193], [-73.80768917829106, 40.6222776604247], [-73.80698502381061, 40.623871998829976], [-73.80557147888442, 40.62496900157379], [-73.80516248846952, 40.62465622418597], [-73.80271345029576, 40.62593478734804], [-73.80251577946906, 40.6263291376923], [-73.8001117714566, 40.626927441987085], [-73.80092841565032, 40.627524104895656], [-73.79783533398513, 40.627408716571516]]], [[[-73.77348616522504, 40.625085118163526], [-73.77381024409397, 40.624925596730954], [-73.77383258726536, 40.624944848253456], [-73.77348560409436, 40.62511314810922], [-73.77348616522504, 40.625085118163526]]], [[[-73.77172760292548, 40.62358502801516], [-73.77178742171202, 40.62350408187312], [-73.77168780941253, 40.62345775824798], [-73.7716990840563, 40.623438989211046], [-73.77180205088992, 40.6234878803381], [-73.7718065553057, 40.62348190999068], [-73.77197318324889, 40.623272116308335], [-73.77187021304596, 40.62322407888819], [-73.77188149343576, 40.623203602050566], [-73.77198669843922, 40.62325335218967], [-73.77214546970639, 40.62304610520043], [-73.7720313143135, 40.62299121290296], [-73.77207296880073, 40.62293919036048], [-73.7721490467794, 40.622976724329334], [-73.77218824028017, 40.622995793795134], [-73.77272418290725, 40.62230839938326], [-73.77308562752837, 40.62183676009294], [-73.77347337495445, 40.62134402610005], [-73.77363729197724, 40.621135724360485], [-73.77411466469975, 40.620522520213164], [-73.77456049454024, 40.619951959126276], [-73.7750332232728, 40.619343178695], [-73.77503335134442, 40.619343013761736], [-73.77511891254433, 40.619233847487365], [-73.77518045935567, 40.61926471788484], [-73.77515794285773, 40.61929371606763], [-73.77511877728904, 40.61927399379186], [-73.77506473853114, 40.61934307518107], [-73.77506460979983, 40.6193432401131], [-73.77495328183132, 40.61948635632548], [-73.77500588175907, 40.6195112300096], [-73.77499461579019, 40.61952828992289], [-73.77493976825987, 40.61950426656821], [-73.77476640595854, 40.61972259439369], [-73.77482012770623, 40.61974746935469], [-73.77480885456588, 40.61976623803491], [-73.77475625738923, 40.6197405112075], [-73.77458288591141, 40.6199605455242], [-73.77463437522347, 40.61998285399991], [-73.77462198893457, 40.61999905929049], [-73.77457162552267, 40.619975898617575], [-73.77439825015809, 40.62019678670988], [-73.77445085909794, 40.620219098134356], [-73.774438476271, 40.62023444800687], [-73.7743858730955, 40.62021042879396], [-73.77421473327387, 40.62043302878822], [-73.77426958798799, 40.620454491646065], [-73.77425720928375, 40.62046898727483], [-73.77420347429072, 40.62044752778313], [-73.77402897875453, 40.620667560052944], [-73.77408383225163, 40.62068987655831], [-73.77407143925994, 40.62070778889978], [-73.77401659017951, 40.6206846193378], [-73.77384546527352, 40.62090209425039], [-73.77389694638455, 40.62092696573332], [-73.77388568146198, 40.62094317298173], [-73.77383308349965, 40.62091744405923], [-73.7736608298803, 40.621136625440286], [-73.7737167938199, 40.621161506331575], [-73.77370215913446, 40.621180268452285], [-73.77364843599236, 40.621155392468694], [-73.77350067294029, 40.62134390917439], [-73.7734773057808, 40.621373721048805], [-73.77353102837378, 40.6213985976132], [-73.77352087855442, 40.62141651548622], [-73.77346603422149, 40.621391637209406], [-73.77329266580485, 40.621608253307166], [-73.77334974791422, 40.6216348443745], [-73.7733362347926, 40.62165275560001], [-73.77328363248037, 40.62162788115485], [-73.77310914042, 40.62184534831895], [-73.77316286325559, 40.62187022505424], [-73.77315159799728, 40.62188643172646], [-73.77309787515313, 40.62186155498594], [-73.77292561377533, 40.6220815899726], [-73.7729815822394, 40.622105616492675], [-73.77296470321912, 40.62212437458906], [-73.77291322087405, 40.622099502187915], [-73.77274432125824, 40.62232039659964], [-73.7727935624733, 40.62234440991182], [-73.77278118331373, 40.622358906551966], [-73.77272857395504, 40.62233574030543], [-73.77255744039574, 40.62255321426784], [-73.7726089222986, 40.62257808682594], [-73.7725965330078, 40.622595145451754], [-73.77254616993127, 40.62257112983641], [-73.77237278964559, 40.622789451129925], [-73.772425392748, 40.622814327664635], [-73.77241188430918, 40.62283052932034], [-73.77236152244429, 40.622806512454694], [-73.77220950703806, 40.62300523248699], [-73.7724031363917, 40.62309530229876], [-73.7723626016271, 40.62314732600432], [-73.7721689786699, 40.62305554733859], [-73.77215455823358, 40.62307319410719], [-73.77215434472937, 40.62307345553245], [-73.7720046057531, 40.62326192901904], [-73.77215464477041, 40.623326795120775], [-73.77218482383529, 40.62334001519293], [-73.77217018334778, 40.62335963116915], [-73.77215360357187, 40.62335285173819], [-73.77199669270388, 40.62328155948756], [-73.77182107912483, 40.62349645992333], [-73.77181882771771, 40.62349901798295], [-73.77182442403672, 40.62350159111614], [-73.77199904490342, 40.623577104443584], [-73.77198776448255, 40.623597582464626], [-73.77185569638743, 40.623535820446456], [-73.77181316781842, 40.62351523514535], [-73.77173897590608, 40.6236057260367], [-73.77172760292548, 40.62358502801516]]], [[[-73.79162058913477, 40.62231334348308], [-73.79142802632143, 40.62227765834795], [-73.79075818793622, 40.62232836832426], [-73.79059984430631, 40.62244948577123], [-73.79048062923107, 40.62252084743856], [-73.79030507811355, 40.622582563102306], [-73.79017983899541, 40.62257756252419], [-73.79007341842517, 40.62256305186657], [-73.79005044260067, 40.622563009909435], [-73.78988553708297, 40.62256270863351], [-73.78981035453674, 40.62257211483743], [-73.78967870292792, 40.622614822524326], [-73.78949653224412, 40.62274664610792], [-73.78419483298926, 40.6208926699551], [-73.78426326908115, 40.62083272323078], [-73.7843383029774, 40.620777550807404], [-73.78441936332378, 40.62072757273768], [-73.78450583289406, 40.62068316956702], [-73.78459705327867, 40.6206446793915], [-73.78469232989883, 40.62061239528353], [-73.78477773362741, 40.62057023304006], [-73.7848576953853, 40.62052227307279], [-73.78493153947468, 40.62046892066322], [-73.78499864189892, 40.62041062666032], [-73.78505843563585, 40.62034788367028], [-73.78511041542829, 40.62028122189329], [-73.78515414205441, 40.62021120464334], [-73.78518924603695, 40.620138423587036], [-73.78521543076681, 40.62006349374441], [-73.78507784902251, 40.61996661358056], [-73.78512331439762, 40.61981492104341], [-73.78517524443524, 40.61965100037344], [-73.78514372469022, 40.61946244452387], [-73.78516032368115, 40.619296009374345], [-73.78520870265943, 40.61923734791775], [-73.7851833013958, 40.61914427645058], [-73.7851545601093, 40.61909036591641], [-73.7851711510987, 40.618926379687885], [-73.78516495572417, 40.61885537554957], [-73.78514920785406, 40.61875742472632], [-73.78515920006004, 40.61864728248332], [-73.7850632794476, 40.618505118554204], [-73.78488368265464, 40.61840930965986], [-73.78478111816777, 40.61833567667866], [-73.78476856019043, 40.61824507642237], [-73.78477534795478, 40.61813248093754], [-73.78478291732561, 40.61811635181475], [-73.78481093058895, 40.6180566587725], [-73.78481106475007, 40.61801504289331], [-73.78478547486449, 40.61798072183545], [-73.7847244609295, 40.61797326360713], [-73.78467752923336, 40.61797939255476], [-73.78465054000075, 40.61798291740893], [-73.78458950259879, 40.617982803048655], [-73.78451882678749, 40.61798267058927], [-73.78450284231191, 40.6179581614057], [-73.78449655136116, 40.61791653247601], [-73.78455126699085, 40.617884811163975], [-73.78458184988712, 40.61785505060126], [-73.78453859900476, 40.61782848281919], [-73.7844101694661, 40.61780620957503], [-73.7843556902111, 40.6177644905874], [-73.78434624252294, 40.61770572099269], [-73.78438813142866, 40.617666630704974], [-73.78445579298352, 40.61760555728037], [-73.78445273734563, 40.617556590931486], [-73.78440144915797, 40.617522222601856], [-73.78432120689806, 40.61750004012531], [-73.78429575194136, 40.61742410357805], [-73.78429919287694, 40.617353117964186], [-73.78433788632171, 40.617309125484866], [-73.78433998512607, 40.61730479761577], [-73.78437346044376, 40.617235752376715], [-73.78442813736525, 40.617216270743896], [-73.78444768097869, 40.617133074620874], [-73.78445326249802, 40.61708854087404], [-73.78446087067887, 40.617027835129704], [-73.78438089785311, 40.616922419251615], [-73.78426867510038, 40.61685611283726], [-73.78418536377005, 40.61678985942237], [-73.78414720948078, 40.61666738664297], [-73.78415237624955, 40.61663524595187], [-73.78415704405519, 40.61660620428706], [-73.78412181771587, 40.616571865985456], [-73.78406670589821, 40.61654371889476], [-73.78405446729549, 40.61653746850265], [-73.7839679358515, 40.61647365614667], [-73.78390060938848, 40.61643191346047], [-73.78390327278183, 40.61636910751812], [-73.78390756529659, 40.61626790874861], [-73.78381802921346, 40.61614044420899], [-73.78370885300865, 40.616125550865654], [-73.78366473079498, 40.616114250907856], [-73.78366070504741, 40.616113220062005], [-73.78366002987642, 40.61610922673384], [-73.7836449441638, 40.61602016468612], [-73.7835543329955, 40.615996299924284], [-73.78353260458239, 40.61599057725003], [-73.78344597109687, 40.61595859086472], [-73.78333396543637, 40.61582618521764], [-73.7832782049081, 40.61570128109888], [-73.78326717962132, 40.61561797810338], [-73.78322554530014, 40.61557873072326], [-73.78310021264996, 40.61559318344571], [-73.7829884433602, 40.61564124591981], [-73.78298118565476, 40.61564436669286], [-73.78275635553686, 40.615631701809015], [-73.7825669578421, 40.61558972756902], [-73.78248993127501, 40.615567549886805], [-73.78238409631444, 40.61551349272969], [-73.78231042656971, 40.615447257365275], [-73.78230713854042, 40.61543656008269], [-73.78228181716396, 40.61535417796198], [-73.78231752089329, 40.61524163684286], [-73.78229527471422, 40.61516815367754], [-73.78221175978906, 40.61516554770508], [-73.7821313541849, 40.61519477128253], [-73.78205109149752, 40.61517993165954], [-73.78199656877105, 40.61515289959362], [-73.78197744644285, 40.615106350463265], [-73.78189090283789, 40.61504743389118], [-73.78186828033726, 40.615089008115866], [-73.7818790370823, 40.61516485585312], [-73.78184222128019, 40.61519911963102], [-73.7818100802744, 40.61520395366908], [-73.78171700912468, 40.61517684879661], [-73.78164014281379, 40.6151057102113], [-73.78158240022272, 40.61508112127921], [-73.78149577687142, 40.61504668348637], [-73.78147617328904, 40.61501039771566], [-73.78144149492915, 40.614946211580346], [-73.7814159727366, 40.61489230670212], [-73.78127785533971, 40.61488714843479], [-73.78122023166848, 40.61488130926985], [-73.78120399415778, 40.61487966338073], [-73.78115269211109, 40.61485018910313], [-73.7811399071764, 40.614830580573596], [-73.78114968573595, 40.61480474370714], [-73.78115934985756, 40.61477920968273], [-73.78113050286747, 40.614759570571785], [-73.78111067894662, 40.614762408299036], [-73.7810630097626, 40.614769233967756], [-73.78098597748865, 40.61474950353707], [-73.78085789171315, 40.61462440929852], [-73.7807937650353, 40.614587567092435], [-73.7808103408408, 40.61443092653004], [-73.78080415768376, 40.614357473192506], [-73.78076582541503, 40.614291303032175], [-73.78072433857183, 40.614207991596345], [-73.78075878329834, 40.614177856769054], [-73.78076621936054, 40.6141713515442], [-73.7807535478932, 40.61411747051919], [-73.78070887195283, 40.614026809005864], [-73.78085046476025, 40.6139511902222], [-73.78097249377586, 40.613963663010324], [-73.7811490607288, 40.613998271982446], [-73.78114590422712, 40.61398112905265], [-73.78112055120368, 40.61387581600957], [-73.7810403451562, 40.613813496770575], [-73.78100199089187, 40.61378501389626], [-73.78092185869777, 40.61373100434156], [-73.78088360834359, 40.61364035463625], [-73.78090621621647, 40.61360367661871], [-73.78098980054023, 40.61358425171302], [-73.78105720259778, 40.61360151703611], [-73.78115999778085, 40.61360171278711], [-73.78119226513, 40.61355771028799], [-73.78119576665524, 40.61346958792765], [-73.78122166520576, 40.61340843637568], [-73.78127952806562, 40.61339630628866], [-73.7812605110188, 40.61331793350504], [-73.78137959932596, 40.61324716678573], [-73.78160136846203, 40.613210868422776], [-73.78177836610467, 40.61311328429783], [-73.78197506772605, 40.612883542643644], [-73.78231954307886, 40.61265163449765], [-73.78239337732361, 40.61266646227556], [-73.78239023652235, 40.6126444243652], [-73.78261557467621, 40.612497968636845], [-73.78272796594388, 40.612510421340204], [-73.78278570606835, 40.61253501137741], [-73.7828790134917, 40.61248867479037], [-73.7829146271154, 40.61240306079756], [-73.78292121972007, 40.61235166533054], [-73.78296005311707, 40.612263609630624], [-73.78302450498049, 40.6122000831892], [-73.78315942025627, 40.61220033778338], [-73.78322696610739, 40.61217353650431], [-73.78330429802995, 40.6121002429373], [-73.78335575780689, 40.61208075524395], [-73.78345861352133, 40.61206136486561], [-73.7835263575679, 40.61197336343408], [-73.78355221437404, 40.61192445198088], [-73.78371299347518, 40.611873346707824], [-73.7837605485183, 40.61193510310074], [-73.78382814992248, 40.612022891547255], [-73.78394373669919, 40.61204024579138], [-73.78422271946083, 40.61194038033933], [-73.78436963663604, 40.6118877889443], [-73.78447437567272, 40.61185029594851], [-73.78458968364842, 40.61195332910705], [-73.78458789348959, 40.61195962067106], [-73.7845799363939, 40.61198758300819], [-73.78464086652882, 40.612019520908184], [-73.78472760664935, 40.612017235618886], [-73.78475981475188, 40.61199036775516], [-73.7847728543081, 40.61193163909378], [-73.78485638084535, 40.61192934769011], [-73.78497523353337, 40.61192957008814], [-73.78501066349975, 40.61190026040854], [-73.78507185385064, 40.61185141469719], [-73.78509129264212, 40.61180004195303], [-73.78505931114388, 40.61175591822391], [-73.78506792206909, 40.611684101720336], [-73.78507251590148, 40.61164578191831], [-73.78516282917359, 40.6115308935415], [-73.78521116182439, 40.61148447099247], [-73.78522423150612, 40.61141595124298], [-73.78544973958921, 40.61121318664408], [-73.78551080380369, 40.611203508345326], [-73.78553984649244, 40.61116194688065], [-73.78577809315796, 40.61099347707069], [-73.78621843179218, 40.610911064582744], [-73.78646972800934, 40.610678968981624], [-73.78682445265207, 40.6107883309109], [-73.78684518697295, 40.61079472358821], [-73.78710253758436, 40.610677695554664], [-73.78719897412161, 40.61065584216966], [-73.78721512862111, 40.61062649494941], [-73.78719602603874, 40.6105726038908], [-73.78722835948798, 40.610506566677444], [-73.78722999351474, 40.61047392407473], [-73.78723178914876, 40.610438028181576], [-73.78719984810371, 40.61038166491338], [-73.78726107424681, 40.61032057789604], [-73.78741544568005, 40.6102621115996], [-73.78757610833092, 40.610245271710774], [-73.78759578527854, 40.610281761814086], [-73.7876144287936, 40.61031633464022], [-73.78770742637307, 40.61036546826493], [-73.78783925815156, 40.61032409384312], [-73.7879132621454, 40.6102850621979], [-73.78790602838673, 40.61027675011098], [-73.78787485581836, 40.61024092722476], [-73.78785894330079, 40.610194384978335], [-73.78824140912991, 40.61012654656824], [-73.78829264835399, 40.61017560060223], [-73.78832480067743, 40.61016586788916], [-73.78836030518765, 40.61011207640171], [-73.78837012007149, 40.610055790335615], [-73.78853725349592, 40.610024274213146], [-73.78859475483435, 40.61012474855982], [-73.78865910627547, 40.61009059422679], [-73.78871402336283, 40.60999277489203], [-73.78870777636307, 40.609936458768004], [-73.7887080088047, 40.60986301813367], [-73.78876926429395, 40.60979213860528], [-73.78879514030761, 40.60973588201696], [-73.78875680409138, 40.609669714493045], [-73.78876011660967, 40.60963789567892], [-73.78880507094011, 40.60964287450448], [-73.78884041217606, 40.60964049216583], [-73.7888662029074, 40.609611163587566], [-73.78893706313524, 40.609550092870656], [-73.78897525486545, 40.60955358796784], [-73.78899165373358, 40.609555089820525], [-73.78908160165909, 40.60955280596736], [-73.78909787128777, 40.60948673874685], [-73.78911184344186, 40.609413459520624], [-73.78911746868255, 40.60938395768333], [-73.78910162466049, 40.609315384936835], [-73.78903435482209, 40.609256508399184], [-73.78901840238645, 40.60922220694455], [-73.78916957340999, 40.609158836202404], [-73.78928528024618, 40.60913701641445], [-73.78938805806573, 40.609130085288214], [-73.78939130309898, 40.609129866744794], [-73.78948118868126, 40.60914716666045], [-73.78953913741638, 40.609105656619974], [-73.78960657372537, 40.6090710693147], [-73.78964852690937, 40.60904955272555], [-73.78984776252041, 40.609022988992386], [-73.78995389362336, 40.60898156632591], [-73.78998612257973, 40.60894735304491], [-73.78999907091941, 40.60891555179987], [-73.79006660916056, 40.60888874699258], [-73.79011167083348, 40.60885945333057], [-73.7900957887504, 40.60880312018514], [-73.79001236552747, 40.608773590842034], [-73.79001264272644, 40.608685463306934], [-73.79007706133319, 40.60862927633309], [-73.7901736401271, 40.60856090783165], [-73.79028941414941, 40.60851705451778], [-73.79029670742352, 40.60851625044823], [-73.79039866370154, 40.608505014261475], [-73.79047269491804, 40.60845618791433], [-73.7904812468437, 40.608409233934886], [-73.79048250009374, 40.60840235008777], [-73.79046997488345, 40.60829951022694], [-73.79038020614568, 40.60824549017297], [-73.79054444426578, 40.608111147653055], [-73.79071592085809, 40.6081304766207], [-73.79072104674593, 40.60813105354399], [-73.79077575139577, 40.60809932990773], [-73.79085621325443, 40.608048067245385], [-73.7908659950587, 40.60800157215921], [-73.79085658903483, 40.60792811514515], [-73.79083984839724, 40.60787640669498], [-73.79089225900364, 40.607820466797165], [-73.79088916294998, 40.607783741214334], [-73.79087659100047, 40.607695589287495], [-73.79091191580129, 40.60769810132761], [-73.79098584610873, 40.60768110004131], [-73.79098892075747, 40.60772516957295], [-73.79104666876955, 40.60774730654251], [-73.79104723143563, 40.60774732733384], [-73.79118798343292, 40.60775245958023], [-73.79133899604432, 40.60773804585382], [-73.79139058757958, 40.607674491367234], [-73.79139192215438, 40.60766166622427], [-73.79139721011913, 40.60761085524994], [-73.79147416230792, 40.60765505882776], [-73.7914796291121, 40.607661825377406], [-73.79156369380654, 40.60776587244002], [-73.79159895008118, 40.607809510119296], [-73.79151114690586, 40.60815452141681], [-73.7915097486129, 40.608155516350855], [-73.79140813105519, 40.60822777557577], [-73.79128604191624, 40.60823734570321], [-73.79115751304454, 40.608251801389216], [-73.791115556099, 40.60831537327887], [-73.79107360039092, 40.60837894565771], [-73.79117287672476, 40.60847459910882], [-73.79126583521303, 40.608535968322805], [-73.79133325013775, 40.60854833043536], [-73.79141980894755, 40.608602343748586], [-73.7914741007453, 40.60870281201171], [-73.7915832433979, 40.6087250414358], [-73.7917598105094, 40.60875718636972], [-73.7919771311963, 40.60872949173802], [-73.79214499541327, 40.60874948499667], [-73.79274948626171, 40.60864636687806], [-73.792920727321, 40.6086751632376], [-73.79292606044487, 40.608676060584486], [-73.79293361676645, 40.60868158407414], [-73.79299656635982, 40.60872759550929], [-73.79305747543663, 40.60876687407996], [-73.79311408510226, 40.60877147197761], [-73.79321161830435, 40.608779391207065], [-73.7932724972324, 40.608828461873294], [-73.7932854153539, 40.60885680763679], [-73.79329482918058, 40.60887746218556], [-73.79330754229287, 40.60892154898023], [-73.79340051851256, 40.60897802195192], [-73.79351938111789, 40.60897333932672], [-73.7935579122261, 40.60897830475483], [-73.79365104095642, 40.60898581496914], [-73.79372793482922, 40.6090496021196], [-73.79372780624465, 40.60909121804704], [-73.79377580688372, 40.60915005608764], [-73.79381432923432, 40.60915746918647], [-73.7938527911191, 40.609184466356325], [-73.79394893616826, 40.60925563254132], [-73.79394239014113, 40.60929478801782], [-73.79388132265457, 40.60930691922945], [-73.7938799776454, 40.609308174461006], [-73.79383944411542, 40.609346012506236], [-73.7938456491542, 40.60941701574549], [-73.79389688394807, 40.609468517220755], [-73.79397714240278, 40.609483348078506], [-73.79406359128787, 40.60957408030573], [-73.7941544822748, 40.609592483388184], [-73.79418557615644, 40.60959877853627], [-73.7942466746913, 40.6095768559605], [-73.79433651754034, 40.609608841147214], [-73.79443270053278, 40.609667765111794], [-73.79454166514711, 40.609748744979655], [-73.79454478656235, 40.609778126510626], [-73.79446700696276, 40.60980986612718], [-73.79446116506642, 40.609812249676345], [-73.7944674687868, 40.60985142933027], [-73.79452510633077, 40.60991028495337], [-73.79452178213785, 40.6099469994519], [-73.79455369214612, 40.61001560132704], [-73.79467600571685, 40.610106854540525], [-73.79467868056405, 40.6101088487307], [-73.79475872103917, 40.61019467293104], [-73.79480995093128, 40.610248621254094], [-73.79478325170489, 40.610262116499385], [-73.79476169204892, 40.610273014828095], [-73.79476157163172, 40.61031218299261], [-73.79480633875043, 40.61037836048121], [-73.79487372764495, 40.610400511721046], [-73.79494752459134, 40.61042757140562], [-73.79505655097407, 40.61048896619427], [-73.79508550143615, 40.61048901780995], [-73.79514006718176, 40.610489115075225], [-73.79523965260138, 40.610486845252964], [-73.79529736025292, 40.610523667976324], [-73.79537441438853, 40.61053604538012], [-73.79555747865007, 40.610546164067614], [-73.79566026938924, 40.61054634685171], [-73.79579178957168, 40.6106053324849], [-73.79603232973153, 40.61072816075954], [-73.79624422255735, 40.61076525650763], [-73.79637893337657, 40.61083159228128], [-73.79655209352363, 40.610929819676294], [-73.79679919436889, 40.61100859285839], [-73.79689556095903, 40.61100876317943], [-73.79692441928965, 40.61102595108638], [-73.79691788228041, 40.61106265896639], [-73.79691732271, 40.611070426818635], [-73.79691452867704, 40.611109165904274], [-73.79696581925703, 40.611143529177376], [-73.79701077653216, 40.61114850479514], [-73.79711999198145, 40.61114869761805], [-73.7971624469114, 40.61114877254543], [-73.79722278233363, 40.611148879002506], [-73.79731593657868, 40.61114904330363], [-73.79736731722681, 40.61115402893564], [-73.79737045480938, 40.611178515370305], [-73.79736385663233, 40.61121314479721], [-73.79735739829847, 40.61124703717324], [-73.79745676195705, 40.61131820439622], [-73.79752750476399, 40.611293849301305], [-73.79771064649509, 40.61127948261544], [-73.79794184478439, 40.61130681744336], [-73.79808315944648, 40.61131441143469], [-73.7982470203558, 40.61130245846462], [-73.7983634556353, 40.61131132153457], [-73.79837869240349, 40.611312481263546], [-73.79839061713035, 40.611316369093494], [-73.7985070549989, 40.61135432249593], [-73.79848442127278, 40.61140324294632], [-73.79848744997311, 40.6114644490874], [-73.79852584119676, 40.61151592428682], [-73.79864139333537, 40.61154550388557], [-73.79876353846713, 40.61151878962435], [-73.79886957949955, 40.61150673452405], [-73.79898529394339, 40.61148245771915], [-73.79903982792398, 40.61150703230239], [-73.79912946875938, 40.61160755768692], [-73.7991259695635, 40.6117030255618], [-73.79912554232706, 40.61184500898044], [-73.79911239018125, 40.61194535519122], [-73.79917318059519, 40.6120262469318], [-73.79932094418534, 40.612026505004486], [-73.79937883934394, 40.612002125167955], [-73.79938235953564, 40.61198821297142], [-73.79939184177391, 40.61195073879188], [-73.79944972926401, 40.61192880785067], [-73.7996105031139, 40.61187523140935], [-73.79970685649201, 40.61188029556174], [-73.7997262371791, 40.611868790745774], [-73.79978086995465, 40.61183636059688], [-73.79988381658069, 40.6117851308024], [-73.79988394133282, 40.611743514879436], [-73.8000159585105, 40.61163848040781], [-73.80007703500371, 40.61162389868435], [-73.80017340117134, 40.61162406625498], [-73.80025700070239, 40.6115972828881], [-73.80030502354087, 40.61165122317155], [-73.80034349615052, 40.61167576973235], [-73.80039810464088, 40.611675864574025], [-73.800436695706, 40.61166124361109], [-73.80049472403928, 40.61164318147944], [-73.80052668996203, 40.61164426408016], [-73.8005395167722, 40.61165163032446], [-73.80057149294561, 40.611700645932906], [-73.80057144723526, 40.61171594698154], [-73.80057138324905, 40.61173736566871], [-73.80056162959758, 40.61177651761589], [-73.8006033005496, 40.611805965851936], [-73.8006546021935, 40.61183787921729], [-73.80069947118977, 40.61187222968045], [-73.80074112012176, 40.61190902181293], [-73.80080125919478, 40.611933904481596], [-73.80073775474803, 40.61196042387409], [-73.8007119835992, 40.6119848595921], [-73.80067977373251, 40.612014179682696], [-73.80062192312624, 40.612023872280965], [-73.80056088361394, 40.61202621415962], [-73.8004774226873, 40.61200648463071], [-73.80041950019094, 40.61204065672905], [-73.80035197001881, 40.612065019851826], [-73.80023948919161, 40.612081960169164], [-73.80019944719878, 40.61213616188071], [-73.80019433453432, 40.612143082510855], [-73.80021346073205, 40.61219207587892], [-73.80019080019999, 40.612250789542124], [-73.80007134482238, 40.61245131953739], [-73.80007125684408, 40.61248069532333], [-73.80005830484173, 40.6125149449467], [-73.80004543342157, 40.61252610380153], [-73.79999067071368, 40.612573580330825], [-73.80001625835416, 40.612610344774794], [-73.80006058206881, 40.61264427738756], [-73.80007074363093, 40.61265205569257], [-73.80008992919088, 40.612681464999554], [-73.80008978990149, 40.6127279776089], [-73.80007677854819, 40.61278181181506], [-73.80006848765422, 40.6128016631935], [-73.80005736632921, 40.612828290398596], [-73.80001550298633, 40.61286248972253], [-73.80001213766509, 40.61291389175862], [-73.80000872033965, 40.61298243061675], [-73.79999243799428, 40.613055843302526], [-73.7999343527159, 40.61314387003032], [-73.79987003947546, 40.61316579119973], [-73.79978007368867, 40.613172978511734], [-73.79972220057734, 40.61319001343029], [-73.7996867102713, 40.61324136065051], [-73.79960319061053, 40.61324121507318], [-73.799542126579, 40.61325089985135], [-73.79947902190645, 40.61328836008456], [-73.7994681113871, 40.61329483582882], [-73.7993748959427, 40.61331425785442], [-73.79931694077328, 40.61335822059154], [-73.79930387847645, 40.61342918985625], [-73.7993517832537, 40.613522298682746], [-73.79939200146262, 40.6135474925859], [-73.79942232263285, 40.6135664856972], [-73.79947157403302, 40.61356969956874], [-73.79953794488276, 40.613574031379144], [-73.79959252486924, 40.61358391830638], [-73.7996502507532, 40.61361584331301], [-73.7996595570501, 40.61372602046493], [-73.799685101531, 40.613777474043054], [-73.7996367993324, 40.61381655754866], [-73.79959903264584, 40.61384520724655], [-73.79951767033212, 40.61390692658454], [-73.79947510832075, 40.614173686542635], [-73.79950698933646, 40.614254527145995], [-73.79947799745693, 40.614281405240085], [-73.79944259530328, 40.61430337542756], [-73.79950678346574, 40.61432307158209], [-73.79951311218878, 40.614354907498125], [-73.79951021374198, 40.61437755693212], [-73.79950653347994, 40.61440630390767], [-73.79950266109694, 40.614408401321874], [-73.79946148689586, 40.614430704547196], [-73.79945817122423, 40.61446497258375], [-73.79947688999992, 40.61451366178858], [-73.79949960873981, 40.61457275684785], [-73.79945762700558, 40.61464612461835], [-73.79944032859643, 40.61466299362397], [-73.79936741505662, 40.61473409616842], [-73.7993768387615, 40.614805104676854], [-73.79939159947278, 40.61486201971181], [-73.79940224366943, 40.614903069235034], [-73.79929581592357, 40.61504242030885], [-73.79928524480067, 40.615048588038725], [-73.79917031100385, 40.61511564261107], [-73.79912189571496, 40.61519144628431], [-73.79909594590342, 40.61527463368828], [-73.79906364553028, 40.61533333026594], [-73.79904620025569, 40.615337673794166], [-73.79902991524453, 40.615344123060886], [-73.79901522475771, 40.61535250608832], [-73.79900252053618, 40.61536259933242], [-73.79900205053465, 40.61538772682638], [-73.79900623216679, 40.615412654018094], [-73.79901498196996, 40.6154368833502], [-73.7990259290895, 40.615456639688794], [-73.79903991670028, 40.6154752484791], [-73.79905674147689, 40.615492439219196], [-73.79905492951963, 40.61551244394932], [-73.7990489771576, 40.61553197650975], [-73.79903903505591, 40.615550542484], [-73.79902535487066, 40.615567671921646], [-73.79900828287819, 40.61558293123431], [-73.79897187441813, 40.61556987336722], [-73.79893881229017, 40.61555239464188], [-73.79891001234513, 40.61553097923683], [-73.7989024492685, 40.61552399310073], [-73.798882156658, 40.61552117121894], [-73.79886153142323, 40.615520927515696], [-73.79884113376117, 40.61552326861022], [-73.79882151768798, 40.61552813091637], [-73.7988032159917, 40.6155353823705], [-73.7987867257609, 40.61554482601728], [-73.79877249488439, 40.615556205359745], [-73.79876090988529, 40.615569211325955], [-73.79875228542326, 40.61558349066319], [-73.79874685574764, 40.615598655532864], [-73.79874476833575, 40.615614294044015], [-73.7987596034431, 40.615665809649045], [-73.79877337735275, 40.61571471261737], [-73.79872463344243, 40.61573029717817], [-73.7986799466203, 40.61575181460492], [-73.79864054950353, 40.61577867137472], [-73.79860752880491, 40.61581012668706], [-73.79858179535775, 40.61584531289663], [-73.79856405899156, 40.6158832594461], [-73.79855468966309, 40.61588701778433], [-73.79854465671518, 40.61588959618309], [-73.79853421412332, 40.6158909293723], [-73.79852362623315, 40.615890983603514], [-73.79851316106756, 40.61588975750392], [-73.79850308354331, 40.61588728211109], [-73.79849364876466, 40.61588362008753], [-73.79848509556415, 40.61587886413417], [-73.79847764045894, 40.61587313464396], [-73.79847147216802, 40.61586657665371], [-73.79846674683574, 40.61585935617338], [-73.798463546398, 40.61585152959327], [-73.79846204259528, 40.615843412317126], [-73.79846227472295, 40.61583521645854], [-73.79846423671493, 40.61582715618434], [-73.79846787730132, 40.61581944211858], [-73.7984411026312, 40.61579187521958], [-73.79840921414502, 40.61576767343401], [-73.79837293055456, 40.6157473822248], [-73.7983330696275, 40.61573145891766], [-73.79829052975519, 40.6157202623939], [-73.79824626970642, 40.61571404500148], [-73.79820740884634, 40.615712793192124], [-73.79816866238929, 40.61571538603761], [-73.79816300611748, 40.615716090283584], [-73.79812948535049, 40.6157463350454], [-73.79810231859612, 40.6157800792528], [-73.79808291105947, 40.61581478786383], [-73.7980702376693, 40.615851258973024], [-73.79806456056252, 40.615888738242774], [-73.7980659971718, 40.61592645048177], [-73.79808371498052, 40.61592289435658], [-73.79810195949871, 40.61592161118434], [-73.79812023213249, 40.61592263603208], [-73.79813803352123, 40.61592594089244], [-73.79815487718109, 40.61593143544886], [-73.79817030280194, 40.61593896954392], [-73.79818388882569, 40.61594833728307], [-73.79819526396729, 40.615959282660796], [-73.79820411736088, 40.615971506557294], [-73.7982102070556, 40.61598467491258], [-73.79821336662795, 40.61599842785603], [-73.7982135097299, 40.61601238954078], [-73.79821099289485, 40.616060366072105], [-73.79819977174908, 40.61610761337566], [-73.79818006300701, 40.61615321888526], [-73.79815224732067, 40.616196301744026], [-73.7981168619278, 40.61623602981712], [-73.79807459027694, 40.61627163576555], [-73.79807726070206, 40.616281150942704], [-73.79807792215901, 40.61629086822486], [-73.79807655798858, 40.616300542881916], [-73.79807320254595, 40.61630993125736], [-73.79806794033756, 40.61631879690427], [-73.7980609038916, 40.61632691654107], [-73.79805227042077, 40.616334085674126], [-73.7980422573597, 40.61634012374823], [-73.79799600560548, 40.61635410137157], [-73.79789585988114, 40.616388899727255], [-73.79779911547128, 40.61642889740473], [-73.79770623653002, 40.61647390250824], [-73.79761766866868, 40.6165236991177], [-73.79755006077956, 40.61656692375326], [-73.7974865779239, 40.61657159119655], [-73.79742284775487, 40.61656971617212], [-73.79738616088451, 40.61657964154207], [-73.7973522037096, 40.616594149876335], [-73.79732195358513, 40.61661282359746], [-73.79729628117188, 40.61663512523925], [-73.79727592537725, 40.61666041291677], [-73.79726147208783, 40.616687958800156], [-73.79725333730627, 40.616716970062946], [-73.79725175517643, 40.61674661170099], [-73.79725677124432, 40.61677603056553], [-73.79726824114535, 40.61680437991874], [-73.7972858347583, 40.6168308438045], [-73.79730344910976, 40.61682809755179], [-73.79732140914075, 40.61682746430595], [-73.79733928101942, 40.61682895936337], [-73.79735663304317, 40.616832546610304], [-73.79737304606743, 40.61683813939532], [-73.79738812362802, 40.61684560262253], [-73.79740150151996, 40.616854756014526], [-73.79741285659374, 40.61686537846716], [-73.7974219145618, 40.616877213390644], [-73.79742873273153, 40.616890678809106], [-73.79743257606708, 40.61690481155978], [-73.79743334153846, 40.61691923278799], [-73.79742164092688, 40.61696737851773], [-73.79740304786137, 40.61701748579913], [-73.79737828290916, 40.617066017734686], [-73.79738846778457, 40.617071526947086], [-73.79739744399028, 40.61707814508095], [-73.79740500561536, 40.6170857203192], [-73.79741097919928, 40.61709407888925], [-73.79741567900955, 40.61710431943163], [-73.79741799741426, 40.61711502334536], [-73.79741786531413, 40.6171258716079], [-73.79741528664454, 40.61713654089428], [-73.79741033826045, 40.61714671321377], [-73.7974031676439, 40.6171560853871], [-73.79739398850936, 40.617164378082954], [-73.79738307443444, 40.61717134414276], [-73.79737437783018, 40.61717539776419], [-73.79736509127339, 40.61717860264889], [-73.7973025665911, 40.61723142723815], [-73.79723481710543, 40.61728036799297], [-73.7971622569712, 40.61732512572864], [-73.79714541720053, 40.61735256304975], [-73.79713317353263, 40.617381380164495], [-73.79716194894681, 40.61743281446217], [-73.7971950402285, 40.61748271879482], [-73.79723230812465, 40.61753088314312], [-73.79723859239994, 40.61753931321156], [-73.79724333558096, 40.61754830955737], [-73.79724685822482, 40.617559541699144], [-73.79724798226711, 40.61757105833647], [-73.7972466788679, 40.617582563992634], [-73.7972429814662, 40.61759376347242], [-73.79723698492384, 40.617604369435774], [-73.79722884309027, 40.61761410977018], [-73.79722082448347, 40.617607828187424], [-73.79720876210737, 40.617600780727976], [-73.7971953419327, 40.6175953424115], [-73.79718093007504, 40.61759166160027], [-73.79716591970326, 40.6175898387103], [-73.79715072031405, 40.617589923471726], [-73.79713574656108, 40.617591913572056], [-73.79711152049389, 40.617615720530324], [-73.79708298940882, 40.61763657451953], [-73.79705076574594, 40.61765402789129], [-73.79701554121124, 40.61766770599419], [-73.7969780719283, 40.617677315215566], [-73.7969518297396, 40.61768142573818], [-73.79692505707091, 40.61768217807085], [-73.79689848885276, 40.61767955156143], [-73.79687285440359, 40.617673618309794], [-73.79684885740897, 40.61766454118809], [-73.79682715660415, 40.61765256937057], [-73.79681528937235, 40.61765844504974], [-73.79680488647284, 40.617665756673006], [-73.79679624371698, 40.6176742963312], [-73.79678960686611, 40.617683821195136], [-73.7967851646435, 40.61769406042085], [-73.79678304336666, 40.61770472285115], [-73.79678330335601, 40.61771550529431], [-73.79678593722032, 40.617726101146225], [-73.79679087006495, 40.61773620910837], [-73.7967979616229, 40.61774554175527], [-73.79680711815642, 40.617749199716755], [-73.79681536108711, 40.61775395483375], [-73.7968224661755, 40.61775967774874], [-73.79682824013456, 40.617766212775834], [-73.79683252589022, 40.61777338213689], [-73.79683520685202, 40.61778099079728], [-73.79683621008719, 40.61778883177156], [-73.79683550830332, 40.61779669175436], [-73.79683312059072, 40.61780435692334], [-73.79681130421892, 40.61783395529965], [-73.79679645333708, 40.617865961080696], [-73.79678900569206, 40.61789943088135], [-73.79678918081787, 40.6179333781625], [-73.79679432022968, 40.61795855030528], [-73.79680366406913, 40.617983010023146], [-73.79681506826151, 40.61803887028155], [-73.79683551278102, 40.618093213468164], [-73.79685530183276, 40.618130335621814], [-73.79687931758346, 40.61816598032531], [-73.79690098172009, 40.61819271142592], [-73.79692502482435, 40.61821823422007], [-73.79697639665079, 40.61822811671141], [-73.79698912798531, 40.61826730754102], [-73.7970049966818, 40.618330984124846], [-73.79700484778857, 40.61837994395792], [-73.79697270043968, 40.61838723116663], [-73.79689881105924, 40.61838710064378], [-73.7968570552419, 40.6183845790924], [-73.79682174691476, 40.61837472544337], [-73.79678892958304, 40.618369400147465], [-73.79676073741287, 40.618364824671666], [-73.79670932209949, 40.618369629969294], [-73.79665464913575, 40.61838911794599], [-73.79660959004038, 40.61841596638524], [-73.79655815251495, 40.61842811505436], [-73.79651316930017, 40.61843048370957], [-73.79650023582766, 40.61845738896819], [-73.796455245125, 40.61846220602791], [-73.7964103073449, 40.618449885756306], [-73.7963556790422, 40.618454684535365], [-73.7963362531319, 40.61850361138954], [-73.79632956787496, 40.61858927954574], [-73.79635192027044, 40.618633383030996], [-73.79634215632429, 40.618674983523206], [-73.7962938552147, 40.61871161784586], [-73.79631300438106, 40.618753267891556], [-73.79628721338942, 40.618782598113334], [-73.79623256238474, 40.618794741444866], [-73.79622603236346, 40.618829002505706], [-73.79628373194903, 40.618870720884715], [-73.79628362735241, 40.61890499283249], [-73.79621915173756, 40.618978319534456], [-73.79620007770853, 40.61901561336169], [-73.79618784068263, 40.61905453804855], [-73.79618266830488, 40.619094369517356], [-73.79618465680534, 40.61913436682054], [-73.79619376920586, 40.61917378592504], [-73.79620983600927, 40.61921189355172], [-73.79623255834962, 40.61924798081692], [-73.79625038803628, 40.61926968869612], [-73.79629197255309, 40.61932851423564], [-73.79623393627779, 40.61939695614389], [-73.79615340374144, 40.619467806573454], [-73.79612434843372, 40.61951426669914], [-73.79615312712687, 40.61955838278801], [-73.79618830890925, 40.61960985302919], [-73.79615920936664, 40.61967100171486], [-73.79602718518987, 40.61977113716199], [-73.7959703456468, 40.61978065515557], [-73.79596932719024, 40.61978082573557], [-73.79592429747481, 40.6197978827093], [-73.79593052903357, 40.61986154232402], [-73.79559534308251, 40.620211013607275], [-73.79555957594641, 40.62035048679241], [-73.79557680732991, 40.620526585312554], [-73.7954917359122, 40.62075602931512], [-73.79547474221752, 40.620776290894604], [-73.79549708296938, 40.62080707886347], [-73.7955172338287, 40.62083873209171], [-73.79553742958896, 40.62087569380565], [-73.79555463970607, 40.620913522525065], [-73.79555062776872, 40.620921877976166], [-73.79554530911442, 40.620929798484546], [-73.79553649126696, 40.62093932848453], [-73.79552579101806, 40.620947664467955], [-73.79551348247503, 40.62095459289186], [-73.79549988094514, 40.62095993627105], [-73.79545176147752, 40.62095970637506], [-73.79540418923348, 40.620965225133986], [-73.79535833969949, 40.62097635618212], [-73.79531534579557, 40.62099282447685], [-73.79527626988178, 40.62101422309462], [-73.79524207751014, 40.62104002328583], [-73.79521361356514, 40.62106958753898], [-73.79519736218977, 40.62113075936381], [-73.79515200782235, 40.62115464508866], [-73.79511148378306, 40.62118312978857], [-73.79507658678963, 40.62121565344854], [-73.79504800293363, 40.6212515766458], [-73.79502629419169, 40.62129019312082], [-73.79501188737623, 40.621330743662135], [-73.7950050657438, 40.62137243103301], [-73.79500514352377, 40.62140576001699], [-73.79501007899763, 40.621438876202525], [-73.79503337737094, 40.621472753364685], [-73.79506306561069, 40.62150359299404], [-73.79509847534585, 40.621530700789556], [-73.7951388093929, 40.62155346646536], [-73.79518315970257, 40.62157137749001], [-73.79520301100749, 40.62159021267853], [-73.79521891255902, 40.62161111134561], [-73.7952305015144, 40.62163359662908], [-73.79523751343378, 40.62165715546329], [-73.79516938062133, 40.621695014715996], [-73.7950962193355, 40.62172696487349], [-73.7950189085003, 40.62175262209942], [-73.79493837689395, 40.62177167815794], [-73.79488671677727, 40.62185481870539], [-73.79487299710861, 40.62186695103254], [-73.79485692002076, 40.62187726656349], [-73.7948388946738, 40.62188550276833], [-73.79481937981173, 40.62189145003592], [-73.79479887208713, 40.6218949570082], [-73.79477789342168, 40.62189593443298], [-73.79475697772168, 40.62189435743464], [-73.79473665729232, 40.621890266147815], [-73.79471744928763, 40.62188376469572], [-73.7946998425511, 40.62187501854031], [-73.79468428517264, 40.62186425027076], [-73.7946806977154, 40.621850909137535], [-73.79467422297094, 40.621838215910586], [-73.79466504457541, 40.62182653059095], [-73.79465342284485, 40.62181618459348], [-73.79463968739142, 40.62180747134773], [-73.79462422777566, 40.621800637975596], [-73.7946074824578, 40.621795878282455], [-73.79458992636214, 40.62179332726087], [-73.79457205740796, 40.62179305726193], [-73.7945498866044, 40.62179299670093], [-73.79453008043839, 40.62179537252888], [-73.79451097265147, 40.62180000106543], [-73.79449300186825, 40.62180677606106], [-73.79447658061319, 40.621815541993946], [-73.79446208584234, 40.62182609763951], [-73.79444985028817, 40.6218382006897], [-73.79444015482304, 40.621851573315205], [-73.79443425229591, 40.62189321840058], [-73.79442386285947, 40.62194964635316], [-73.7944107655114, 40.622005746156155], [-73.79442001628661, 40.62203446985182], [-73.7944353710467, 40.62206163324823], [-73.79445640734178, 40.62208648899569], [-73.79448254640309, 40.6221083532343], [-73.79451306906626, 40.622126624409596], [-73.79454713555884, 40.62214079982292], [-73.79458380860342, 40.622150489463145], [-73.79460957198225, 40.622158726001054], [-73.7946331233919, 40.62217017909578], [-73.79465377756067, 40.62218451549909], [-73.79467093351647, 40.62220131806769], [-73.79468409207296, 40.622220097900176], [-73.79469287035404, 40.62224030856271], [-73.7946970129351, 40.62226136198848], [-73.79469703112095, 40.622276965022344], [-73.79469349867145, 40.622296150456435], [-73.79468611203347, 40.62231468817971], [-73.79468715884306, 40.6223683225769], [-73.79467951459112, 40.62242164572645], [-73.79466329602539, 40.62247384309935], [-73.79463875087076, 40.622524117361976], [-73.7946062540462, 40.62257170055433], [-73.79456630193827, 40.62261586582284], [-73.79451950481986, 40.62265593852227], [-73.79446657752885, 40.62269130652243], [-73.79440832854812, 40.6227214295587], [-73.79434564765546, 40.62274584748536], [-73.79421064090884, 40.622765875014835], [-73.79407428620424, 40.62277961327007], [-73.79393708771487, 40.62278701145347], [-73.7937995527352, 40.62278804221028], [-73.79366218980341, 40.62278270172928], [-73.79352550682152, 40.62277100975673], [-73.79339000917749, 40.62275300952429], [-73.792883580545, 40.622559220696914], [-73.79260056336351, 40.622462721580554], [-73.79226351352034, 40.62236908907301], [-73.79187800516166, 40.622361047834765], [-73.79162058913477, 40.62231334348308]]], [[[-73.76670827781236, 40.61491086618553], [-73.76825288003378, 40.614877725169315], [-73.77397692018825, 40.61600357567147], [-73.76873454755717, 40.62090086822023], [-73.76745926975256, 40.620511322031255], [-73.7667193110134, 40.61500363286468], [-73.76670827781236, 40.61491086618553]]], [[[-73.84304421296773, 40.617737696390435], [-73.84283573528793, 40.61660943616361], [-73.84400850201834, 40.61704136615355], [-73.84454193473684, 40.614867921675135], [-73.8409994253963, 40.613029126216134], [-73.83825667290532, 40.613847216216406], [-73.8407944240835, 40.60564326638265], [-73.8471480968766, 40.60361424219198], [-73.85179402215192, 40.60689716858368], [-73.85124092910449, 40.607719613414964], [-73.84779013405834, 40.60923538526278], [-73.84816769367677, 40.61253422829745], [-73.84731105470064, 40.613438461926826], [-73.84639522711923, 40.616036814801745], [-73.84614804513843, 40.61772842036632], [-73.84304421296773, 40.617737696390435]]], [[[-73.85523067631884, 40.61613145335797], [-73.85530325465119, 40.6131454135728], [-73.85236704384849, 40.61315598907671], [-73.85376845962072, 40.61408759651678], [-73.85263195046835, 40.61509408190896], [-73.84973986306498, 40.61515605814331], [-73.85010857555366, 40.61021219334599], [-73.8551559228834, 40.606120513001486], [-73.85708638173246, 40.60646824293355], [-73.85704052727723, 40.60829948887597], [-73.85901755318278, 40.60947575392508], [-73.85924855452187, 40.61125483087423], [-73.85802093146347, 40.61264381691059], [-73.85648360749232, 40.61319069486381], [-73.85858056782772, 40.61691708722184], [-73.85523067631884, 40.61613145335797]]], [[[-73.814090115668, 40.614466920099765], [-73.81413268075418, 40.61436629043908], [-73.81416449897701, 40.61429106964075], [-73.81441441488526, 40.614101424008155], [-73.8146826267648, 40.61397515785498], [-73.81486725972127, 40.61401064975278], [-73.81486681073604, 40.61417254228434], [-73.814857222891, 40.614299226214776], [-73.81504193462983, 40.61430656181428], [-73.81535634872849, 40.61417332968294], [-73.81549526637583, 40.61403981513316], [-73.81535703015994, 40.61392697127488], [-73.81536653934614, 40.6138284438187], [-73.81519143554301, 40.61368738556275], [-73.81499776068877, 40.61358149086307], [-73.81487519159299, 40.61336984895391], [-73.81479563255313, 40.613194029981294], [-73.81485122699522, 40.61313077018466], [-73.81487213107738, 40.61311332049703], [-73.81503638494628, 40.6129762137494], [-73.81523956494507, 40.61298357925722], [-73.81528551351116, 40.61306811885985], [-73.81527590714306, 40.61320184104876], [-73.81532195326875, 40.61325118690909], [-73.81546948317194, 40.61334292900841], [-73.81560806899893, 40.61332907375156], [-73.81571906149262, 40.61327294163629], [-73.81578420222384, 40.61309707529883], [-73.81588601679647, 40.613019810910956], [-73.81598763622685, 40.61301293489119], [-73.81615626666235, 40.61294211575494], [-73.8164315038978, 40.614282334964926], [-73.81623366684653, 40.61423104619861], [-73.8161412617739, 40.61424497658887], [-73.81598412428654, 40.61428695826425], [-73.81582696674197, 40.614335978434866], [-73.81569763477397, 40.614342809885336], [-73.81554994675568, 40.61430737811123], [-73.81538349309263, 40.6143774996555], [-73.81523555167247, 40.61443357256184], [-73.8150691571859, 40.61448257634695], [-73.81495837543912, 40.61446128234239], [-73.81471812627856, 40.61449608855192], [-73.81458844285291, 40.61462961822811], [-73.81444032345223, 40.61474903867063], [-73.81420926754464, 40.614703443269974], [-73.81408062432926, 40.61455840923629], [-73.814090115668, 40.614466920099765]]], [[[-73.81496830662114, 40.61042065495561], [-73.8150251664767, 40.61035320555015], [-73.81507481822895, 40.610282527928845], [-73.8151169482724, 40.61020906853107], [-73.8151512905161, 40.61013329136652], [-73.81517762806351, 40.61005567508305], [-73.81519579458158, 40.60997670994407], [-73.81520567535078, 40.60989689473104], [-73.81520720849484, 40.60981673379666], [-73.81559858789666, 40.61067251616471], [-73.8154036618984, 40.610384802015474], [-73.81526445493755, 40.61049963818177], [-73.8150387432527, 40.61072453111434], [-73.81484699639005, 40.61081905690231], [-73.814613646295, 40.61093407380671], [-73.81450261566708, 40.61105847350542], [-73.81449876827786, 40.61073545365928], [-73.81458991478213, 40.610695427156486], [-73.81467660241397, 40.61065001075345], [-73.8147582836064, 40.61059949133032], [-73.81483444241965, 40.6105441880006], [-73.81490459780065, 40.61048445009514], [-73.81496830662114, 40.61042065495561]]], [[[-73.86966442958492, 40.606540123412906], [-73.87109330768071, 40.604694173994694], [-73.87105656646135, 40.606285088494246], [-73.87245579811875, 40.60818553034854], [-73.87268959311653, 40.60917799830844], [-73.87300122724774, 40.61055545845987], [-73.87180082867997, 40.61028041358032], [-73.8710387011171, 40.60972358252655], [-73.87076008541273, 40.60870540090801], [-73.86966442958492, 40.606540123412906]]], [[[-73.79420172624032, 40.607730675613354], [-73.79352152715475, 40.60632759139594], [-73.79399754952593, 40.60608015346263], [-73.79509002319158, 40.607271060105596], [-73.79424799033224, 40.60590038003164], [-73.79696327795033, 40.60411784876779], [-73.8012001966788, 40.60391220065208], [-73.8018028511679, 40.604211431452256], [-73.80428530678597, 40.604519279530564], [-73.80520676583323, 40.60673953810017], [-73.80419695590167, 40.608744199035584], [-73.80159206744115, 40.61023096250127], [-73.79878198680099, 40.61033327568909], [-73.79630654215427, 40.60933803253817], [-73.79609453981675, 40.6088264839427], [-73.79559832739619, 40.60872535630222], [-73.79529977914471, 40.608229878318475], [-73.79420172624032, 40.607730675613354]]], [[[-73.83244207358176, 40.605929449536745], [-73.83454144109655, 40.60547738376589], [-73.83587601623029, 40.605636907545566], [-73.83610156628845, 40.60736279382329], [-73.83428342685944, 40.60918568185396], [-73.83373275973192, 40.60928603307339], [-73.83380311515587, 40.609550591858046], [-73.83252192275347, 40.610384993603816], [-73.83140871257932, 40.609421755813216], [-73.83067824786619, 40.60878967826978], [-73.83032947905777, 40.60848787991649], [-73.83045815392722, 40.606669201403214], [-73.83244207358176, 40.605929449536745]]], [[[-73.82607472604538, 40.60843779954612], [-73.82661051008785, 40.60819620584165], [-73.82771104235977, 40.608398668130704], [-73.82805077260065, 40.61004629686527], [-73.82688326414805, 40.6095966780694], [-73.82607472604538, 40.60843779954612]]], [[[-73.86702399475149, 40.60806817876942], [-73.86874317559169, 40.60703517927931], [-73.86868468826752, 40.608241158183915], [-73.86777501561137, 40.60853948877828], [-73.86702399475149, 40.60806817876942]]], [[[-73.82718282107048, 40.607919778091194], [-73.82679537497145, 40.607720440593134], [-73.82629763015808, 40.607817072229714], [-73.82617361009859, 40.60755411178232], [-73.8263911775284, 40.60735743365122], [-73.82598212776448, 40.60715799759935], [-73.82590092720767, 40.60646593064943], [-73.82649618795769, 40.60622446371768], [-73.8269052458468, 40.606423897977294], [-73.82861671457653, 40.60583848413761], [-73.82866060090204, 40.606256158374734], [-73.82815085191197, 40.60795028983291], [-73.82718282107048, 40.607919778091194]]], [[[-73.81569926225602, 40.6064570673649], [-73.8155212646286, 40.60641964237777], [-73.8155168973555, 40.60643588521717], [-73.81546066483351, 40.60642813700742], [-73.81552816722274, 40.606163069945154], [-73.81558381764519, 40.60617290276205], [-73.81552536723665, 40.60640738028192], [-73.81570815854799, 40.606439294195894], [-73.81572726392088, 40.606354262277904], [-73.81576751589343, 40.60637159210934], [-73.81574174081905, 40.60646524511771], [-73.81571005865179, 40.60645862779], [-73.81569926225602, 40.6064570673649]]], [[[-73.81339665223344, 40.60436407224534], [-73.81184130264785, 40.603134665085776], [-73.81085365186647, 40.60322371301433], [-73.81053636758658, 40.60249598881469], [-73.80986333857152, 40.60189579000784], [-73.80992146111767, 40.60178709534227], [-73.80999658353151, 40.601696439012215], [-73.8100342593758, 40.6016109084604], [-73.81003453242589, 40.60151494013756], [-73.8100244640379, 40.60146564385276], [-73.80994982919876, 40.601385114165566], [-73.80991258166577, 40.60132020993271], [-73.8099433702624, 40.601263199129335], [-73.80999459024659, 40.60120103437945], [-73.81003914574595, 40.60108957743949], [-73.81003591240686, 40.60102991701338], [-73.81004293231338, 40.60095471054518], [-73.81002618357127, 40.60086130844196], [-73.80997877227131, 40.60078082493845], [-73.80993821798249, 40.60068219691062], [-73.80992817216713, 40.60062511756395], [-73.80989100609673, 40.600531682170455], [-73.80986053496292, 40.6004771638921], [-73.80985064502396, 40.600365616758594], [-73.80988145450942, 40.6003008250928], [-73.80993943621954, 40.60025423403637], [-73.80999055209632, 40.60022838130371], [-73.81000088663791, 40.600184305288366], [-73.81001471187018, 40.600109109564016], [-73.80997059406056, 40.60006753738019], [-73.81000132948452, 40.60002868281523], [-73.81006930440077, 40.60005732629201], [-73.81014417523036, 40.60005485564211], [-73.81014430798412, 40.60000816889966], [-73.81010697922734, 40.59997179570274], [-73.81004927959846, 40.59991982516422], [-73.80998813232418, 40.599883412557276], [-73.80997465251733, 40.59983670280388], [-73.81005295429873, 40.59982386412303], [-73.81010053621723, 40.59984469179265], [-73.81013110919791, 40.599862898655715], [-73.81018896434578, 40.599860400365955], [-73.81022311058264, 40.59981895701237], [-73.81024025805883, 40.59977229836463], [-73.81024380045234, 40.599723024061134], [-73.81024048657294, 40.599691893612174], [-73.81022695590293, 40.59966334015216], [-73.81026778270058, 40.599666001924916], [-73.81029493857316, 40.59968938965007], [-73.81030183970356, 40.59965568220466], [-73.81030198708322, 40.59960380843507], [-73.81028510566179, 40.59955709360473], [-73.81025462859844, 40.59950516927546], [-73.81025479075838, 40.59944810730722], [-73.81025503399489, 40.599362515359445], [-73.81033001323586, 40.59932113988604], [-73.81035740538844, 40.59926152899451], [-73.81034740297027, 40.59918888826432], [-73.81033056583131, 40.59912661051977], [-73.81030695226997, 40.59905394732009], [-73.81029011586071, 40.598991670072344], [-73.81033106684644, 40.59895023791682], [-73.81038898814394, 40.59892439673993], [-73.8104163272087, 40.59888294259712], [-73.81040964716155, 40.59883883796755], [-73.8104880810805, 40.59877931173957], [-73.81058684830285, 40.59874834959015], [-73.8106856684498, 40.59869923159592], [-73.81078441560808, 40.59867605082374], [-73.81085937905794, 40.59863986152541], [-73.81090036686123, 40.59858546109986], [-73.8109616972456, 40.59855703088099], [-73.81104335601387, 40.59855975903601], [-73.8111181504355, 40.59858322539088], [-73.81117974133485, 40.59858489283772], [-73.81122022555856, 40.59858598697425], [-73.81131560541287, 40.59854983208716], [-73.81138719612315, 40.5985032621373], [-73.81141798216156, 40.598446250923026], [-73.81148965192016, 40.59837115042115], [-73.81152379592731, 40.598329707176575], [-73.81153417375009, 40.598270068095545], [-73.81150861997385, 40.598216183527164], [-73.81157550345523, 40.59814119925956], [-73.81172811474971, 40.59827557421149], [-73.81178922582542, 40.598324953967484], [-73.81193874830761, 40.59839523079278], [-73.81205766969288, 40.59845767362346], [-73.81222073287758, 40.598553908477776], [-73.81222404674801, 40.59858503769893], [-73.81221034899123, 40.59861614029549], [-73.81222386439053, 40.59864988071621], [-73.81224417176405, 40.598688819784165], [-73.8122678748163, 40.59873035775287], [-73.81228818953024, 40.59876670363495], [-73.81239671422856, 40.59889656680437], [-73.81253246018585, 40.59902647427727], [-73.81261044089611, 40.59912775679624], [-73.81272920416659, 40.59924726207325], [-73.81297362359823, 40.59945515849486], [-73.81305369627488, 40.59950371201523], [-73.81329372961245, 40.59981659270371], [-73.81444738136224, 40.60054316885366], [-73.81501471568424, 40.60205629156354], [-73.8142869369858, 40.60445566622167], [-73.81339665223344, 40.60436407224534]]], [[[-73.86702905185578, 40.603743388853935], [-73.8676819415421, 40.603070766677334], [-73.8683080846461, 40.603139161146146], [-73.86793505372694, 40.60396113545295], [-73.86702607287029, 40.6041715840082], [-73.86702905185578, 40.603743388853935]]], [[[-73.85243335807012, 40.59733339779527], [-73.85269029351811, 40.597170750463306], [-73.85286117431974, 40.597244308232646], [-73.85365316421074, 40.59683787744518], [-73.853995595994, 40.59668348634672], [-73.85436974704125, 40.596692110402316], [-73.85465863090106, 40.596578388927576], [-73.85492648581442, 40.596301664373286], [-73.85595300562048, 40.59618887618966], [-73.85673364394975, 40.59607576435409], [-73.85716132350703, 40.59604369786466], [-73.85778119968528, 40.59611781491794], [-73.86044420534076, 40.596471905596275], [-73.86054977985192, 40.59706302019168], [-73.86141485166918, 40.597753696377694], [-73.86177276370987, 40.598299178571146], [-73.86274820792643, 40.59907120592458], [-73.86328663134141, 40.599810042163234], [-73.86335999147606, 40.600322202771196], [-73.86297506260962, 40.601018272615576], [-73.86187189847233, 40.602164325815906], [-73.86154081260408, 40.60238035986936], [-73.86117495608606, 40.60248664842694], [-73.86112814224546, 40.60255033107426], [-73.86078169166855, 40.60260840025625], [-73.86048884213515, 40.60233934739461], [-73.85905914327755, 40.602061431573915], [-73.85482595782968, 40.601867195353634], [-73.85471200729276, 40.60164902532504], [-73.85427387360807, 40.601430436697015], [-73.85372090340685, 40.60140066517852], [-73.85276838155842, 40.600948854025916], [-73.85257851635212, 40.600570688348604], [-73.8520453763197, 40.600206622800954], [-73.85176045305586, 40.59969751381257], [-73.85076748001559, 40.598839443837406], [-73.85153122893772, 40.59843393622049], [-73.85200481909777, 40.597748438274294], [-73.85243335807012, 40.59733339779527]]], [[[-73.87019491489939, 40.598996405901886], [-73.87108141811476, 40.598901147470094], [-73.87049411914771, 40.602604446204595], [-73.87026280564918, 40.60168122225861], [-73.87019491489939, 40.598996405901886]]], [[[-73.8231174097477, 40.60007949252535], [-73.82311789977163, 40.60005688484153], [-73.82313294399587, 40.600057236353436], [-73.82313384374159, 40.600042165446055], [-73.82314760323396, 40.60004087570288], [-73.82315293179705, 40.599976662824794], [-73.82317356168161, 40.59997800515576], [-73.82316692767215, 40.60004811439995], [-73.82317337306215, 40.60004910637507], [-73.82317376718535, 40.60006286885787], [-73.82318408196349, 40.60006321191322], [-73.82318015964296, 40.60008384902813], [-73.8231174097477, 40.60007949252535]]], [[[-73.82292757811203, 40.60006142816309], [-73.8229361266004, 40.60004040516013], [-73.822978576463, 40.600043707224835], [-73.82300436542411, 40.59992561919807], [-73.82303195183424, 40.59993051694214], [-73.82300403883895, 40.60004860044176], [-73.82304012567126, 40.600050274348874], [-73.82304007849375, 40.60006807478574], [-73.82292757811203, 40.60006142816309]]], [[[-73.82445629476595, 40.59911583868227], [-73.82439004315057, 40.59909811302643], [-73.82439431638863, 40.599087873285136], [-73.82446713343744, 40.599107464858584], [-73.82450212274327, 40.599008662160735], [-73.8245173365546, 40.59896580873303], [-73.82436255896116, 40.59895043633271], [-73.82435472080273, 40.59894946809494], [-73.82433824076556, 40.59894743204931], [-73.82422585197662, 40.59893354607241], [-73.82417340595445, 40.59892573150956], [-73.82412749012028, 40.59891622247147], [-73.82407791355021, 40.59890920383301], [-73.82406984190722, 40.598922481127985], [-73.82395853199995, 40.59888785399994], [-73.82395871079126, 40.59887689683851], [-73.82405944857535, 40.598889422330345], [-73.82410071394857, 40.59889732688409], [-73.82418471900705, 40.59891282190698], [-73.82427026010049, 40.598925932927], [-73.82434640845658, 40.59893433752879], [-73.82452271276402, 40.598951263990585], [-73.82453702420634, 40.59891221796511], [-73.82461363488147, 40.598723790015384], [-73.824617199115, 40.59871502361059], [-73.82461901353119, 40.59871056004336], [-73.82463684641318, 40.59866669670713], [-73.82463881301405, 40.59866186229396], [-73.8246517839916, 40.59862995900571], [-73.8246632500266, 40.59860175816713], [-73.82468443801113, 40.598553328229684], [-73.82469739000635, 40.59852068593453], [-73.82470460364472, 40.59850250658538], [-73.82470658714921, 40.598497508688], [-73.82471777465584, 40.59846931294721], [-73.82472773089512, 40.598444221491945], [-73.82474292407974, 40.59839899042229], [-73.82474761015331, 40.59840015536492], [-73.82474907564067, 40.598389453972615], [-73.82477088992478, 40.5983918438742], [-73.82476908857363, 40.598396277308574], [-73.82474806935706, 40.598448025943796], [-73.8247464666553, 40.59845196999208], [-73.8247324778652, 40.598486409140925], [-73.82471455960446, 40.59853052097605], [-73.82470430756192, 40.59855576191769], [-73.82470216485918, 40.598561038006395], [-73.82468323745927, 40.59860709767021], [-73.82468179156804, 40.59861083220977], [-73.82468019919393, 40.59861494162449], [-73.82466068784454, 40.598665332706396], [-73.82465836742638, 40.59867132437647], [-73.82464109016925, 40.59871594445675], [-73.82463894522901, 40.598721482891946], [-73.82462212690153, 40.59876491622056], [-73.82462028423028, 40.59876967677444], [-73.82460303016, 40.598814235903426], [-73.82460174601978, 40.59881755354045], [-73.82458387696438, 40.59886370040984], [-73.82458200685116, 40.598868530613565], [-73.82456502381463, 40.59891238753149], [-73.82456317472412, 40.59891716231478], [-73.82454580594191, 40.598962020467106], [-73.82454404027547, 40.59896657959944], [-73.82450563123234, 40.599065769777305], [-73.82450339642126, 40.5990715419436], [-73.82448805568868, 40.59911116118601], [-73.82453084258555, 40.59911949829181], [-73.82452624858546, 40.599129027223235], [-73.82447690960616, 40.59912000982335], [-73.82445629476595, 40.59911583868227]]], [[[-73.86952707548932, 40.598713750354406], [-73.86967070711478, 40.59668856362211], [-73.87020899836229, 40.59695423278937], [-73.86952707548932, 40.598713750354406]]], [[[-73.83668274106707, 40.5949466970158], [-73.83678395742554, 40.594887574408126], [-73.83677470021847, 40.59483025309939], [-73.83670920760002, 40.59471196329247], [-73.83647454514829, 40.59460059573057], [-73.83624456552002, 40.59449639775504], [-73.83599100458636, 40.59442798314016], [-73.83587798121873, 40.594531692254336], [-73.83581100477463, 40.59453372437038], [-73.83576520416725, 40.5945351125751], [-73.83565342677645, 40.59447730803854], [-73.83559157397217, 40.59444532110318], [-73.83548957637792, 40.59445475446702], [-73.83541276835358, 40.59454893562655], [-73.8352719938995, 40.594473517514494], [-73.83511720319443, 40.59436584359023], [-73.83505672124198, 40.59412219884756], [-73.83513188170217, 40.59412947010124], [-73.83514512951908, 40.594097700379216], [-73.83516024257754, 40.5940614577799], [-73.83511348532718, 40.593968265052986], [-73.83496333298632, 40.59388566970452], [-73.8348693335679, 40.59389628051196], [-73.83433212684879, 40.59366076774679], [-73.83431911140642, 40.59361700835926], [-73.83430516205212, 40.593557889352816], [-73.83433345400043, 40.59351763526879], [-73.8343018578103, 40.59346923616374], [-73.83431256331623, 40.593415525880886], [-73.83422463139402, 40.59334824199578], [-73.83410834072814, 40.59334807417262], [-73.83401323492436, 40.59333181881704], [-73.83387239778445, 40.59328326224798], [-73.83385114672677, 40.59332621139611], [-73.8337313334338, 40.593326038098006], [-73.83361163952554, 40.59327751230226], [-73.83352017803828, 40.59321290776129], [-73.83349924180611, 40.59312960217402], [-73.833481783144, 40.593065105808606], [-73.833362250994, 40.59295210791204], [-73.83329545012083, 40.59289022545178], [-73.83324963971388, 40.59289015898433], [-73.83323193224741, 40.59292505599578], [-73.83322124739877, 40.592970706867405], [-73.83314367447208, 40.59298939894405], [-73.83308377330324, 40.59298662645063], [-73.83303817170851, 40.5929032839407], [-73.83303504965276, 40.592742101730266], [-73.83307051685698, 40.59265081882316], [-73.83313425607417, 40.59252734092462], [-73.83316620690034, 40.59243336671119], [-73.83312766467343, 40.59234466298055], [-73.83312434323057, 40.59226407007009], [-73.83279328686963, 40.59218837240355], [-73.83271223681659, 40.59218825443817], [-73.83265939228929, 40.592182803648186], [-73.83262065649224, 40.592172001885004], [-73.8325749810638, 40.592118209113956], [-73.83260558943994, 40.59208255751101], [-73.83263966877404, 40.592042862938094], [-73.83262993706295, 40.591983974348985], [-73.83260892825395, 40.59193021801814], [-73.83267596985205, 40.591895394120044], [-73.83274648102432, 40.59188206475373], [-73.83276840016067, 40.591892922751676], [-73.83280631172961, 40.59191170135953], [-73.83281672314727, 40.59197618808564], [-73.83288011866262, 40.59198971114878], [-73.83298585493124, 40.5919818067063], [-73.83306701801895, 40.59193625658477], [-73.83309890085683, 40.591869145318185], [-73.833049721272, 40.59180728930483], [-73.83305697727148, 40.591724024246005], [-73.83307487165115, 40.59161391211072], [-73.83314190531755, 40.59158177393057], [-73.83324747991823, 40.591638338917456], [-73.83321910088924, 40.59171351532739], [-73.83319072245837, 40.59178869055789], [-73.83322927027885, 40.59187470777639], [-73.83330658794601, 40.59195809505975], [-73.83344030034272, 40.592036192044176], [-73.83355999167891, 40.59208471902894], [-73.8336726955662, 40.59210905841047], [-73.83375728840898, 40.592101122149224], [-73.83367997692342, 40.59201504901109], [-73.83369071615022, 40.59194790630625], [-73.83370144153317, 40.59188613742612], [-73.83378258395456, 40.59184864661736], [-73.83387770851942, 40.59185684234878], [-73.83393396278849, 40.59190796439203], [-73.83399721888573, 40.59197789953869], [-73.83395473218756, 40.592058427245924], [-73.83396870763654, 40.59210680049318], [-73.83408153119412, 40.59208278609028], [-73.83412045489972, 40.59201837121284], [-73.83422638358282, 40.59193256281543], [-73.83414197115772, 40.591867969430936], [-73.83410693316273, 40.591787329758176], [-73.83407541783315, 40.591706695160696], [-73.833994523274, 40.59164479442239], [-73.83392421303131, 40.591577534575535], [-73.83398778268769, 40.59152121353043], [-73.83409020087223, 40.59143002868278], [-73.83422777067327, 40.59137381382269], [-73.83420674164263, 40.59132811592586], [-73.8341504267253, 40.591301171794086], [-73.83406576840979, 40.591335971784574], [-73.83403034912305, 40.591408450384016], [-73.83388588500925, 40.591402868961204], [-73.83376258339992, 40.59138925980156], [-73.83360056172245, 40.591359475252], [-73.83349484594545, 40.59135932211596], [-73.83343130283038, 40.59140489758093], [-73.83335740358869, 40.59136449555018], [-73.833283482777, 40.59133215278572], [-73.83320601241849, 40.59131055016864], [-73.8331144130332, 40.591302358496705], [-73.83305089657865, 40.591337187773185], [-73.83299440094652, 40.59138277323608], [-73.83292381035615, 40.591428338680565], [-73.83287440201715, 40.59145781519542], [-73.83283563342253, 40.591460444804795], [-73.83281812203546, 40.59141743844208], [-73.83277600359993, 40.59135022007501], [-73.83268085970118, 40.59135008157465], [-73.8326495671695, 40.591343717459395], [-73.83256110970056, 40.591325729776635], [-73.83256476938875, 40.59127200937696], [-73.83260021625141, 40.59118878591861], [-73.83260741887503, 40.59112701082998], [-73.83260052601841, 40.591065216882264], [-73.83253006957867, 40.59105705437029], [-73.8325197333496, 40.59096301885267], [-73.83248832294223, 40.59084208957843], [-73.83247096606243, 40.59073729947652], [-73.83247465948892, 40.59067014660304], [-73.83249597709224, 40.59060033392233], [-73.8324996885007, 40.590525122900004], [-73.8324822588798, 40.59044988105836], [-73.83245774684436, 40.59038805973487], [-73.83246148069871, 40.59030479057107], [-73.83253569485043, 40.59021893579835], [-73.83262739367129, 40.59018683386004], [-73.83274377975108, 40.59014670910927], [-73.83284962070249, 40.590095823552915], [-73.83298352367508, 40.59009601825906], [-73.83315739581462, 40.5900997583145], [-73.83328307099791, 40.590085708917634], [-73.83342772609485, 40.59001338777054], [-73.83354043983051, 40.5900323546316], [-73.83358956500702, 40.59011570156727], [-73.83367775384623, 40.590078221033835], [-73.8337730080737, 40.59003269130758], [-73.83387892890823, 40.589949569879124], [-73.83400232738634, 40.58992288484003], [-73.83411510118847, 40.589917675027834], [-73.83423133832333, 40.58993664680044], [-73.83439339168011, 40.58995299799172], [-73.83450948849541, 40.5900283816892], [-73.83463980875666, 40.590052746674175], [-73.83478435641427, 40.5900234045573], [-73.83483035177443, 40.58994825424876], [-73.83485519070757, 40.58987844619105], [-73.83488359302017, 40.589792525728036], [-73.83495422767699, 40.589728155618005], [-73.83491913647808, 40.589669006797095], [-73.8348134040974, 40.5896769141929], [-73.83466524585045, 40.58974117219387], [-73.83454181557454, 40.589781288746636], [-73.8344571724885, 40.58981071637507], [-73.83436564163769, 40.58977566225632], [-73.83430585060809, 40.589729908656416], [-73.83418252598175, 40.589727044586326], [-73.83405563128245, 40.58974297942082], [-73.83400263455297, 40.589799315785065], [-73.83394606071681, 40.58987713597731], [-73.83382965577461, 40.58992532126432], [-73.8337487303853, 40.58987685064007], [-73.83363953352385, 40.58986057507662], [-73.83353381408156, 40.589863108639946], [-73.83339288407699, 40.589854845039156], [-73.8333280477052, 40.58983659582867], [-73.833227415183, 40.5897955060834], [-73.83323112660207, 40.58972029503289], [-73.83329476213193, 40.58963711276133], [-73.83340774259358, 40.58954862815871], [-73.83344322834957, 40.589449286964715], [-73.83347866645357, 40.589368748710896], [-73.83359681259313, 40.589328625478906], [-73.8336903586853, 40.58933015090568], [-73.83384520036506, 40.58933267559812], [-73.8339371220968, 40.58933417485337], [-73.8339931251685, 40.589372179119046], [-73.83409883714177, 40.589372331795744], [-73.83427132532549, 40.58933962327683], [-73.83448611494346, 40.58934312397328], [-73.83458883487054, 40.589292449605004], [-73.83470507858038, 40.58930873440548], [-73.83484966329664, 40.58934904939524], [-73.83493035837527, 40.58940576539278], [-73.83501486800172, 40.58943006413598], [-73.83511708989475, 40.589416778340805], [-73.83512404459879, 40.589454396538436], [-73.83522266937197, 40.58947065603458], [-73.83536721419101, 40.58944131485623], [-73.8354800532563, 40.58940923992457], [-73.83561393634105, 40.58941749021539], [-73.8356985057161, 40.58941761117123], [-73.83576901366463, 40.58940428162246], [-73.83583637795317, 40.589365125531586], [-73.83594705454765, 40.58936692773617], [-73.83600508350207, 40.589412676459816], [-73.83607198232245, 40.589434262616834], [-73.83612483726172, 40.589434338009504], [-73.83620230045881, 40.58945862465619], [-73.83626573378614, 40.58945602906099], [-73.83632197535155, 40.58951252260318], [-73.83640642113407, 40.58956368142386], [-73.83646274185439, 40.589587938461406], [-73.8365296600677, 40.589601465032516], [-73.83659313295577, 40.589582752295364], [-73.83664249161001, 40.58957207712938], [-73.83668114048537, 40.58961779845867], [-73.83675859162032, 40.58964745855902], [-73.83684663235576, 40.589669073794745], [-73.83689939038598, 40.5897094429111], [-73.83693453583638, 40.58974710100794], [-73.83704731566715, 40.58973920232521], [-73.83707912070494, 40.589701639209316], [-73.83719541135855, 40.58969911736292], [-73.83724816918233, 40.58973948698858], [-73.83725161485465, 40.589771727912854], [-73.83725148391838, 40.589825453461884], [-73.83724433877009, 40.589865737558476], [-73.83721603048043, 40.58991405051864], [-73.83725123513788, 40.58992753240636], [-73.83731456482138, 40.58996791630324], [-73.83731798282331, 40.59001090203083], [-73.83735312278405, 40.59005124666217], [-73.8374128965695, 40.59010505698781], [-73.83744455749473, 40.59012659247734], [-73.8375536054168, 40.59020465029495], [-73.83766267949973, 40.59027196102248], [-73.83780353914231, 40.590309768152196], [-73.8381061416056, 40.59049286328023], [-73.83809538729548, 40.59056806455161], [-73.83815153368617, 40.59066485076516], [-73.83825709155998, 40.59072947047191], [-73.83829578946876, 40.59073788784501], [-73.83846841001579, 40.59077543528394], [-73.83859154196814, 40.590858884579525], [-73.83865480129, 40.59092881716831], [-73.83866857222813, 40.591063151323155], [-73.83870346633577, 40.5912055738016], [-73.83870659463203, 40.591369443045245], [-73.8386927270338, 40.591427450960076], [-73.83866743647685, 40.59153325175092], [-73.83946303818477, 40.59225656848909], [-73.83949754038912, 40.59216838222548], [-73.83962809824376, 40.59209603359914], [-73.8397304844814, 40.59201558736428], [-73.83984000695787, 40.591897543281654], [-73.83996340087272, 40.59187353901191], [-73.83999494116478, 40.59194611217134], [-73.83992068316992, 40.59205346069026], [-73.83977954149731, 40.592131167564695], [-73.83960308258222, 40.59224106050579], [-73.83946895572447, 40.59233220595793], [-73.83943345735968, 40.5924396094736], [-73.83948958143614, 40.59254713982214], [-73.83961625566648, 40.592625219102246], [-73.83977816861827, 40.592703348051096], [-73.8398907019092, 40.59280021149386], [-73.84006318617527, 40.5928783540975], [-73.84023583948621, 40.59288665361502], [-73.84039433124553, 40.59292179539158], [-73.84059858452798, 40.592978491292754], [-73.84081016081028, 40.59291968620368], [-73.84093026245439, 40.59279896857427], [-73.84094103158732, 40.59271570890763], [-73.84093419484857, 40.59262705102477], [-73.8409450429299, 40.59251155543275], [-73.84098756626545, 40.59241222051886], [-73.84111823141744, 40.592294203977254], [-73.84116420777474, 40.59222442380349], [-73.84122784858577, 40.59213586333136], [-73.84135823835017, 40.592133357919614], [-73.8414637761373, 40.5922087199808], [-73.84141426957216, 40.59228118257597], [-73.84134359949014, 40.592361673598944], [-73.84131169366367, 40.5924422186197], [-73.84116650852377, 40.592446064371934], [-73.84143751160617, 40.59249874471396], [-73.84142831850568, 40.592553276795854], [-73.84168567500943, 40.59263809149378], [-73.84184022507651, 40.59271336809901], [-73.84187475901565, 40.59276266394533], [-73.84190980778224, 40.59284330124291], [-73.84193426610223, 40.592931983304595], [-73.84200478923243, 40.592913275604324], [-73.84206458684868, 40.592959025349444], [-73.84213139591954, 40.59302090153838], [-73.84216295246586, 40.59308810261414], [-73.84220507443722, 40.59315800469281], [-73.84230728725518, 40.593150086834534], [-73.8424024335184, 40.59315021736392], [-73.84245517820459, 40.59319864272924], [-73.84255328808544, 40.59318713590552], [-73.84257613724341, 40.59321741767998], [-73.84265231583105, 40.59328487457759], [-73.84269009181736, 40.593303666149296], [-73.84272271259046, 40.59331989247711], [-73.84279664017409, 40.59335222916294], [-73.84282990630474, 40.59338495431444], [-73.8428867995623, 40.59340791061614], [-73.84293744603644, 40.59341689275641], [-73.84300206150367, 40.59343765386186], [-73.84304660524084, 40.593451964100964], [-73.8430878848422, 40.59345426838658], [-73.84314526308786, 40.59345747148239], [-73.84320943161804, 40.59347450114619], [-73.84326567535612, 40.593484011627936], [-73.84330729609289, 40.59348724139782], [-73.84340242979927, 40.59349274375278], [-73.84345911349182, 40.59349122074316], [-73.84355660708614, 40.59348894427195], [-73.8436504238803, 40.59348603112926], [-73.84372313493482, 40.59348243500604], [-73.84380073754829, 40.593450304991435], [-73.84385894083933, 40.59342531196059], [-73.84382165075462, 40.593488948465804], [-73.84378161417668, 40.59354420676003], [-73.84375969765586, 40.59360778607443], [-73.84376315736597, 40.593679696452824], [-73.84374124012379, 40.59374327576204], [-73.84373384895089, 40.59380134416975], [-73.84371192494658, 40.593867689710706], [-73.84363933259718, 40.59388141817311], [-73.843606530162, 40.59394498281135], [-73.84361726620143, 40.59400860653123], [-73.84361712370776, 40.59406945037141], [-73.84358788589665, 40.59416067579145], [-73.8435260073294, 40.5942463261109], [-73.8434424400089, 40.59429875875416], [-73.84336623167985, 40.59430695129887], [-73.84335158303408, 40.59436500978682], [-73.84331153839406, 40.594423032977005], [-73.84324615524353, 40.59445613123199], [-73.84313364052117, 40.59447533740426], [-73.843046580238, 40.59446968720597], [-73.84291602237036, 40.594447384028406], [-73.84283967082496, 40.594516419558296], [-73.84283218742793, 40.59461320702788], [-73.84271954116224, 40.59468772364996], [-73.84264325365281, 40.59472910303237], [-73.84255616699606, 40.594734515069426], [-73.8424604916988, 40.59471583843021], [-73.84231495048506, 40.59463024852525], [-73.84220803023986, 40.594667662710265], [-73.84218630991286, 40.59471020698651], [-73.84216429081826, 40.59475333770383], [-73.8421060387757, 40.594838991065316], [-73.84196074544143, 40.59491069750994], [-73.84187111474984, 40.594906117756175], [-73.84179387366368, 40.59490217082855], [-73.84164143508538, 40.59492685189063], [-73.84152874189412, 40.595020727491764], [-73.84131074866141, 40.59515041061996], [-73.84131049188757, 40.59525457172557], [-73.8412377057435, 40.595352199349705], [-73.84114504755935, 40.59535740225562], [-73.84104539054935, 40.59536299613645], [-73.84096797936242, 40.59546745557802], [-73.84091435785372, 40.59553981457469], [-73.84078351504536, 40.59563642989195], [-73.84059825592088, 40.59573020401426], [-73.84043404351124, 40.595799312548166], [-73.84031973891216, 40.595870355647605], [-73.84006799452713, 40.59596177972253], [-73.83988646171512, 40.59601407289271], [-73.83970857749765, 40.59605807497322], [-73.83951254398809, 40.59610481650751], [-73.8393926765489, 40.59616272699143], [-73.83918213035949, 40.59620944818109], [-73.83902964248091, 40.59625348474866], [-73.83892079765629, 40.596253332232386], [-73.83871038533843, 40.59624474020713], [-73.83847829729167, 40.59619739892336], [-73.83834418982916, 40.59614189764269], [-73.8381920292834, 40.596050418288996], [-73.8380433968333, 40.59600042802676], [-73.83788751406561, 40.595947661264695], [-73.83772796402187, 40.59591148290797], [-73.83759018910307, 40.595872569529114], [-73.83743435500348, 40.59580044301385], [-73.83734112299042, 40.59570141859261], [-73.83719111158142, 40.595557936776565], [-73.83707862933531, 40.59543958010494], [-73.83701776530886, 40.59534995062072], [-73.83695207059688, 40.59531403945314], [-73.83685364242194, 40.59521361223885], [-73.83671758013152, 40.59513103920251], [-73.83666574166003, 40.59509335783976], [-73.83662377269687, 40.59506285287462], [-73.83660044560382, 40.59499476553555], [-73.83668274106707, 40.5949466970158]]], [[[-73.86706149472118, 40.5820879767934], [-73.86768677970458, 40.58196329228612], [-73.86814728191008, 40.581988903219305], [-73.86896292210412, 40.58169379478866], [-73.86925758051196, 40.58190964107345], [-73.86968631043702, 40.58230878579189], [-73.87000313642912, 40.582636614815954], [-73.87034778401303, 40.58306413703523], [-73.8705059791835, 40.583341953013736], [-73.87051388095846, 40.5840823274032], [-73.87061497794427, 40.58491535106038], [-73.87084743947547, 40.58542105463095], [-73.87099791647937, 40.586061798365314], [-73.87115922675414, 40.5868738879673], [-73.86981356482106, 40.58946092678817], [-73.86849671647204, 40.59008985028954], [-73.8675927577521, 40.591005791690655], [-73.86690268173955, 40.59147303544145], [-73.86635052164085, 40.591892676564676], [-73.86476222962085, 40.59149221858279], [-73.86430826692539, 40.59141474432638], [-73.86371118798182, 40.591129281983044], [-73.86327273492819, 40.59090807569255], [-73.86274156028652, 40.590366407099964], [-73.86272348498059, 40.59007451075339], [-73.86231374617182, 40.58951874680793], [-73.86193216221744, 40.588898945374616], [-73.86149401111527, 40.58854247360196], [-73.86125299377363, 40.58798808592123], [-73.86137584125022, 40.58737342003284], [-73.86150824362481, 40.58698481999754], [-73.86159140175833, 40.58654599627411], [-73.86167447867456, 40.58614479499366], [-73.86192220398715, 40.58566854822555], [-73.86218624579826, 40.58525502227703], [-73.86249965640731, 40.58482901537917], [-73.86282948605064, 40.58441556714235], [-73.86317563293608, 40.584064840860336], [-73.86327471071942, 40.58387684853184], [-73.86383442889513, 40.58365178217413], [-73.86449245335686, 40.58361493817876], [-73.86503589278036, 40.583302060085636], [-73.86557925292553, 40.5830268024549], [-73.86584278235404, 40.58285154128586], [-73.8661228348405, 40.58263867613857], [-73.86651827430362, 40.582300538285615], [-73.86706149472118, 40.5820879767934]]]]}}, {\"id\": \"2\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 3, \"Shape_Leng\": 0.0843411059012, \"Shape_Area\": 0.000314414156821, \"zone\": \"Allerton/Pelham Gardens\", \"LocationID\": 3, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84792614099985, 40.87134223399991], [-73.84725144800001, 40.870987051999926], [-73.84699273699992, 40.87085288799987], [-73.84640766399981, 40.870554300999885], [-73.84608957099984, 40.870389771999946], [-73.84578147499997, 40.870227695999894], [-73.845172567, 40.86991469799994], [-73.84487737499977, 40.86976050699997], [-73.84442133700004, 40.86952229899989], [-73.84423146999998, 40.86942034999993], [-73.84358581499981, 40.86908763699989], [-73.84315715499991, 40.868863265999934], [-73.84280443499989, 40.868680752999886], [-73.842406294, 40.86847474399989], [-73.84185186499998, 40.86819192599989], [-73.84122362600002, 40.867858150999915], [-73.84064364799987, 40.86755417599992], [-73.83974233399985, 40.867087930999894], [-73.83954475699979, 40.86698673799988], [-73.83922085399998, 40.866820839999924], [-73.83875223499997, 40.866572776999945], [-73.83784380199984, 40.86609918799994], [-73.83697021099982, 40.8656431359999], [-73.83678824299999, 40.86554590599999], [-73.8361860139999, 40.86522409899994], [-73.83578436699997, 40.86501601099999], [-73.83162007899985, 40.86265937499991], [-73.83151964200005, 40.86259766899985], [-73.83089329699999, 40.86221284899995], [-73.83012015999996, 40.862049281999894], [-73.82972614799992, 40.86195274499995], [-73.82950892699994, 40.86184121299988], [-73.82924089599999, 40.8616318679999], [-73.82911929099998, 40.86150997699985], [-73.82841963500002, 40.86093193999992], [-73.82834347399992, 40.86089202599988], [-73.82847346699995, 40.860731352999906], [-73.82881978099991, 40.8604491129999], [-73.82945155499976, 40.860000568999894], [-73.82960951799994, 40.85988842099988], [-73.82973719699993, 40.85980973499988], [-73.829874887, 40.859740419999895], [-73.83002075599987, 40.85968150899988], [-73.83017269399993, 40.85963375399996], [-73.83103340899994, 40.85934303399997], [-73.83190625000005, 40.859070198999895], [-73.83308051999994, 40.85867901899998], [-73.83348699499999, 40.85852949499987], [-73.83437708299991, 40.858190848999854], [-73.83533372599987, 40.8578313009999], [-73.83628873399996, 40.85747692099994], [-73.8368191849999, 40.85727680899989], [-73.83724089099992, 40.85735732299986], [-73.838176221, 40.85750772399996], [-73.83911116599994, 40.85765790099987], [-73.84004614299985, 40.857806268999866], [-73.84097144699983, 40.857949078999916], [-73.84191078799992, 40.858091531999925], [-73.8432955319999, 40.85829037599991], [-73.84458712899988, 40.858445809999914], [-73.8455246059999, 40.85853848699988], [-73.84645854099998, 40.8586363739999], [-73.84739746999989, 40.85868254599995], [-73.84833709099989, 40.85870184799994], [-73.84930893499991, 40.85871750299992], [-73.85028234799984, 40.85865726599995], [-73.8512286699999, 40.85857584399987], [-73.85217337899977, 40.858499155999944], [-73.85310947900003, 40.8584097979999], [-73.85405423899992, 40.85832373099985], [-73.85585355899978, 40.858171764999966], [-73.85601437899996, 40.85815817999989], [-73.8566377469999, 40.8581191719999], [-73.85735319699984, 40.859711605999934], [-73.85739481999993, 40.859984102999896], [-73.85746244199993, 40.860426811999915], [-73.85747219599985, 40.860495676999925], [-73.85747885699989, 40.8605427009999], [-73.85752224299985, 40.86084898099988], [-73.85752991799988, 40.86090316799989], [-73.85756682699994, 40.8611637229999], [-73.85759960699981, 40.86141838899994], [-73.8576092669999, 40.861493440999894], [-73.85767084800001, 40.86197187299994], [-73.85767885599986, 40.86203407799992], [-73.85769181699992, 40.8621347729999], [-73.85769842199993, 40.86218608099991], [-73.85770552799988, 40.862241284999925], [-73.85771028399981, 40.862278229999895], [-73.85780378799993, 40.86303621699989], [-73.85806098599983, 40.86304235999992], [-73.85844789699989, 40.86376138999988], [-73.85871220299993, 40.86425255299985], [-73.85890956799982, 40.864636925999875], [-73.85961281199982, 40.865512717999856], [-73.86055100799985, 40.86552969499993], [-73.86156274099987, 40.86554946399989], [-73.86154239799991, 40.86615362299991], [-73.86150246599986, 40.8675514329999], [-73.86149826099988, 40.86778221599998], [-73.8614338389999, 40.869603467999866], [-73.86137924099984, 40.8713365119999], [-73.86036598199989, 40.87131862199991], [-73.85942680499996, 40.8713006969999], [-73.85848731699984, 40.87128405999989], [-73.85754854399994, 40.87126631799993], [-73.85661382499993, 40.87125016699991], [-73.85655118199992, 40.87303042899989], [-73.85651292399994, 40.874217792999936], [-73.85577052399987, 40.87398371299992], [-73.85551758400005, 40.87390395599987], [-73.854508051, 40.87358094199986], [-73.85363638799987, 40.87330059899996], [-73.85357628499997, 40.87327602099987], [-73.85265751899979, 40.872992186999966], [-73.85170987299989, 40.872697458999944], [-73.85108463399987, 40.87249279099995], [-73.85078983299995, 40.87239629199989], [-73.84987645199996, 40.8721018809999], [-73.84859676099995, 40.871670784999864], [-73.84792614099985, 40.87134223399991]]]}}, {\"id\": \"3\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 4, \"Shape_Leng\": 0.0435665270921, \"Shape_Area\": 0.000111871946192, \"zone\": \"Alphabet City\", \"LocationID\": 4, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97177410965318, 40.72582128133705], [-73.97179142961485, 40.72581422114909], [-73.97181650587731, 40.72581496675256], [-73.97185935830923, 40.72558296903031], [-73.97187437695213, 40.72550311607974], [-73.97188490951473, 40.725424405065766], [-73.97191120732055, 40.72529037069991], [-73.97193374501082, 40.725176290351165], [-73.9719397545633, 40.72515803861004], [-73.97196154778592, 40.72503997881088], [-73.97200511841213, 40.72484148970817], [-73.97203666966882, 40.72468407002008], [-73.97206595867397, 40.72455915832874], [-73.97207196960339, 40.72453635085808], [-73.97208098151926, 40.72449756793038], [-73.97208400057282, 40.72446790554567], [-73.97209751290906, 40.7244234152347], [-73.97215609001734, 40.7241701782781], [-73.97216585996632, 40.724131963273145], [-73.9722169237619, 40.723930054646765], [-73.9722334467696, 40.72384677992629], [-73.97226499868246, 40.72371445733361], [-73.97232507731592, 40.723463499327295], [-73.97236115744644, 40.723312060476445], [-73.97239945748277, 40.72314881636666], [-73.97251011180383, 40.722696886006716], [-73.97288971396678, 40.721088297219346], [-73.9730466728251, 40.720436229833325], [-73.97312959090407, 40.72009119241493], [-73.9732107544226, 40.7197848577304], [-73.97332646360798, 40.7193611047339], [-73.97347975574512, 40.71886142911285], [-73.97464720799998, 40.718842349999896], [-73.97468687499985, 40.718841701999914], [-73.97474239999998, 40.71884079399995], [-73.97485919999995, 40.71885547999994], [-73.97491298899988, 40.71886244299986], [-73.97497644799995, 40.71887065199991], [-73.97509520199993, 40.71888074599987], [-73.97545167499995, 40.71894655899996], [-73.97551012699985, 40.71895861299989], [-73.97609249599991, 40.71907870199988], [-73.97635525199995, 40.71913288399983], [-73.97648182399995, 40.719159627999915], [-73.97674495699998, 40.719215233999925], [-73.97722379199992, 40.71930977499991], [-73.978031227, 40.71963834799987], [-73.97875234999982, 40.71993370599995], [-73.97966021699983, 40.72021282999992], [-73.98130824199984, 40.72070878599992], [-73.98216615299982, 40.72096885499996], [-73.98300308699991, 40.721222680999894], [-73.98382387299982, 40.72147287199987], [-73.98377990999991, 40.72153595399992], [-73.98344363599992, 40.72201850699989], [-73.9829641439999, 40.72267132499997], [-73.98252449499985, 40.723277976999945], [-73.9820824239999, 40.72388204399991], [-73.981636639, 40.72449348099985], [-73.9812112959998, 40.72507279999992], [-73.98076713900005, 40.7256855949999], [-73.9803329819999, 40.72628033599988], [-73.97990650199998, 40.72686577299986], [-73.97946913000003, 40.72746767699993], [-73.97899861999981, 40.7281057469999], [-73.97853582899992, 40.728746084999905], [-73.97805542899995, 40.7293830859999], [-73.97802697999985, 40.729433059999984], [-73.97739386799988, 40.72916504799991], [-73.97690436400002, 40.72895782399994], [-73.976269971, 40.7286892589999], [-73.976152041, 40.72864520999985], [-73.9761329, 40.728638271999934], [-73.97580243799992, 40.72851847999986], [-73.97569651100005, 40.728475052999926], [-73.97329796799995, 40.72742800799989], [-73.97205015299988, 40.726880945999866], [-73.97197685099988, 40.726830355999944], [-73.97191244199995, 40.72679100899989], [-73.97185671300005, 40.72675868499989], [-73.97179602099988, 40.72671367999988], [-73.97162697285862, 40.72662845299842], [-73.97166972701174, 40.726392327154294], [-73.97169587451613, 40.72624182930235], [-73.97171894361502, 40.726121993259504], [-73.97173843379353, 40.72599966755553], [-73.97176755410212, 40.72583330345458], [-73.97177410965318, 40.72582128133705]]]}}, {\"id\": \"4\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 5, \"Shape_Leng\": 0.0921464898574, \"Shape_Area\": 0.000497957489363, \"zone\": \"Arden Heights\", \"LocationID\": 5, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.17421738099989, 40.56256808599987], [-74.17348507999998, 40.56226748999993], [-74.17225507699992, 40.56170483699991], [-74.17191921399996, 40.56155171899995], [-74.17185166499996, 40.56152092299991], [-74.17222993499993, 40.561135857999915], [-74.17245326199999, 40.56088825699991], [-74.17321541599998, 40.560077661999884], [-74.17358785199997, 40.55969842799995], [-74.17437344399991, 40.55870866599984], [-74.17475740499992, 40.55816798199986], [-74.1750629909999, 40.55771126399991], [-74.17512402999996, 40.557620037999925], [-74.1754729799999, 40.557065311999914], [-74.17559957599995, 40.55685071199989], [-74.17579913099998, 40.55651243799986], [-74.1761165979999, 40.55593754099987], [-74.17661991899996, 40.55499100699991], [-74.17746276899993, 40.553411145999846], [-74.17815548499992, 40.55211278999988], [-74.17874485199995, 40.5509958689999], [-74.1793348139999, 40.54988835199988], [-74.17991198599996, 40.548834452999905], [-74.18018524699995, 40.54837960899992], [-74.18128532999992, 40.54695757299989], [-74.18206740899994, 40.54604483899989], [-74.18285329599989, 40.545145824999885], [-74.18369101699987, 40.54417358399989], [-74.1855985999999, 40.541957592999914], [-74.18657647099997, 40.54085328399988], [-74.1870889679999, 40.54024712299989], [-74.18738120699997, 40.53989224299986], [-74.18770381799999, 40.5395269479999], [-74.18826921099989, 40.53886614199993], [-74.18903145199995, 40.53815934199989], [-74.189510226, 40.537788579999905], [-74.19030442599997, 40.53724645299995], [-74.19135323799996, 40.53666485899984], [-74.19377169299999, 40.53583384699993], [-74.19409661699994, 40.53650724499989], [-74.19444077699995, 40.5371565929999], [-74.19468316799997, 40.53766189199994], [-74.19476262699999, 40.5378314079999], [-74.19507682399993, 40.53848646999991], [-74.19542518899992, 40.53917847399987], [-74.19576408099995, 40.53983810199987], [-74.19609298399995, 40.540497735999914], [-74.19640185999997, 40.54117319999993], [-74.19671334599992, 40.541846973999895], [-74.19702175199988, 40.54252684899992], [-74.19730759199997, 40.54326204499988], [-74.19840857099993, 40.545306446999895], [-74.19860457899993, 40.545679908999915], [-74.19912044799996, 40.54670870699988], [-74.19985031599995, 40.54813241899989], [-74.20012791899997, 40.54875987399988], [-74.20018262899993, 40.548906645999864], [-74.20097039299995, 40.55079013899987], [-74.20185089099988, 40.552689430999926], [-74.20216129599993, 40.55335994099991], [-74.20234825099999, 40.55375861799991], [-74.20262828599995, 40.554343511999946], [-74.20291038499992, 40.55494466699989], [-74.2032648999999, 40.55562847699989], [-74.20325826899995, 40.55570233899991], [-74.20324645499988, 40.55577582299988], [-74.20322948999994, 40.55584871799988], [-74.20320742499993, 40.555920814999894], [-74.20318032199998, 40.5559919049999], [-74.20314825999996, 40.55606178499986], [-74.20301604799995, 40.55602844299987], [-74.20288252399993, 40.555998296999846], [-74.2027478209999, 40.5559713759999], [-74.20261207199997, 40.55594770599989], [-74.20247541199996, 40.55592731199989], [-74.20233797799996, 40.55591021399987], [-74.2022083119999, 40.55589290699985], [-74.20207764, 40.55588077499988], [-74.2019463179999, 40.55587385199995], [-74.20181469999991, 40.5558721559999], [-74.20168314599995, 40.555875692999926], [-74.20155201099993, 40.55588445199986], [-74.20142165399993, 40.55589840999992], [-74.20129242599992, 40.55591752899991], [-74.20116467999998, 40.55594175599992], [-74.20103876099996, 40.55597102699991], [-74.20091501299993, 40.556005260999854], [-74.20079377199987, 40.55604436599988], [-74.20062607299995, 40.55613226099993], [-74.20046730799992, 40.55623049099987], [-74.20031905499991, 40.5563382259999], [-74.20018273999989, 40.5564544339999], [-74.20005953999996, 40.556577940999894], [-74.19946683299995, 40.557181180999905], [-74.19769481199992, 40.55888705399984], [-74.19407564499997, 40.56207847399989], [-74.19386270699994, 40.562221584999925], [-74.19386255299989, 40.56222168499994], [-74.19386239999993, 40.56222177399991], [-74.19364100699988, 40.56235719999993], [-74.19364075899988, 40.56235735399986], [-74.19364051199993, 40.56235749799986], [-74.19341094699993, 40.56248503999991], [-74.19317388399995, 40.56260434399992], [-74.19307925099996, 40.5626277349999], [-74.19298216699998, 40.56264428199988], [-74.1928834799999, 40.56265383999989], [-74.19278405499998, 40.5626563249999], [-74.192684759, 40.562651716999945], [-74.19258646299993, 40.562640053999885], [-74.19237007799995, 40.56258854699988], [-74.19214711699996, 40.562550214999916], [-74.19191967499992, 40.56252570499986], [-74.19168998299995, 40.5625154089999], [-74.19146034299997, 40.5625194289999], [-74.19123302399998, 40.56253757599988], [-74.19101021699997, 40.562569401999916], [-74.19085177899994, 40.562618932999946], [-74.1907004609999, 40.56268463499989], [-74.19056041299997, 40.56276545599995], [-74.19043537599994, 40.56285950699993], [-74.19032832899994, 40.56296424499989], [-74.19024124699997, 40.56307668499993], [-74.19011908599994, 40.56319841199997], [-74.18997973099991, 40.56331160499996], [-74.18982494799998, 40.56341415199991], [-74.18982482999994, 40.563414223999885], [-74.18982471199989, 40.56341428799993], [-74.18965655100001, 40.563504491999886], [-74.18947754599994, 40.56358098999991], [-74.1892907519999, 40.56364274199991], [-74.18909923999989, 40.56368927199991], [-74.18726897599993, 40.56404169399984], [-74.18677554199998, 40.56419095799989], [-74.18627446199993, 40.564325526999866], [-74.18576707899994, 40.564445037999924], [-74.18576687799995, 40.56444508299994], [-74.18576667699995, 40.56444512799989], [-74.18525395999991, 40.5645493549999], [-74.18502077599994, 40.5647047109999], [-74.18502052799994, 40.56470486499994], [-74.18502026899988, 40.56470502799989], [-74.18477472999997, 40.56485049699993], [-74.18451788299991, 40.56498530499989], [-74.18451768299992, 40.564985404999916], [-74.18451749399999, 40.56498549599994], [-74.18425093099988, 40.56510871899992], [-74.18425062399993, 40.56510885399991], [-74.18425030599994, 40.5651089899999], [-74.18397521799992, 40.56522011499991], [-74.18397488799997, 40.56522024099992], [-74.18397454599993, 40.56522036799989], [-74.18369189099995, 40.5653190679999], [-74.18340311699991, 40.56540496199994], [-74.18318459699992, 40.56545009399985], [-74.18295815999996, 40.56548019999992], [-74.18272650699997, 40.56549428299992], [-74.1824925989999, 40.56549172899984], [-74.18241339499998, 40.56548514599992], [-74.18225953199997, 40.565472349999936], [-74.18203038699987, 40.56543640499994], [-74.18180816599994, 40.56538457899992], [-74.1815955649999, 40.56531796099987], [-74.18139492499988, 40.56523797799991], [-74.17794544199997, 40.56398336699993], [-74.17758036699996, 40.56375543999989], [-74.17659366099998, 40.563241328999894], [-74.17539949099991, 40.56296466199991], [-74.17488750399997, 40.56282641899986], [-74.17421738099989, 40.56256808599987]]]}}, {\"id\": \"5\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 6, \"Shape_Leng\": 0.150490542523, \"Shape_Area\": 0.000606460984581, \"zone\": \"Arrochar/Fort Wadsworth\", \"LocationID\": 6, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.06367318899999, 40.60219816599994], [-74.06350565699998, 40.602150053999914], [-74.06383152699988, 40.60258896799995], [-74.0651709129999, 40.604213507999894], [-74.06046429699992, 40.60640582399985], [-74.06042867199996, 40.6063527299999], [-74.05801067099993, 40.607368791999924], [-74.05778710399987, 40.607449590999906], [-74.0578901689999, 40.607580753999905], [-74.05685812390232, 40.608056357772554], [-74.05677183367786, 40.607899470387395], [-74.05674934830238, 40.607798757335814], [-74.05665999852928, 40.607555930875925], [-74.05649565936758, 40.60724764825619], [-74.05640693287154, 40.60718873802621], [-74.0560104141088, 40.606780566131995], [-74.05552126122357, 40.6064828925687], [-74.05543198004166, 40.60648249104071], [-74.05537932434086, 40.606494223522326], [-74.05528719697126, 40.60653889007871], [-74.05516001122312, 40.60673049135086], [-74.05498867839447, 40.60666069691174], [-74.05499916476879, 40.606645744784], [-74.05503373135794, 40.606659826645995], [-74.05505194098775, 40.60663386047028], [-74.0550195433675, 40.60662066296658], [-74.05505591012927, 40.60656880569464], [-74.05510403532153, 40.60658841067159], [-74.05517250950514, 40.60649076734958], [-74.05518218103546, 40.60647456683434], [-74.05517955854631, 40.606402334922386], [-74.05516855690021, 40.60636554730313], [-74.05515251060758, 40.606329045897695], [-74.05512022975005, 40.60630536171677], [-74.05505974095801, 40.606265420452814], [-74.05381069805436, 40.605912718852984], [-74.05371769396106, 40.60560458157246], [-74.05360729740819, 40.605238809417735], [-74.05367256354405, 40.60518834332228], [-74.0540203995905, 40.60493472091635], [-74.05385099498753, 40.604530274454824], [-74.0537815276551, 40.60436142291274], [-74.05375474319875, 40.604276931421374], [-74.05369607985057, 40.60414810383785], [-74.0536712685173, 40.604004785905076], [-74.05359133556057, 40.60376168855161], [-74.05356753570281, 40.603696466325864], [-74.05353964480109, 40.603633557567264], [-74.05320486119456, 40.60205979370229], [-74.05315799190798, 40.60197598489976], [-74.05311861801997, 40.601889964830484], [-74.05308691752415, 40.6018021224205], [-74.05306303371839, 40.60171285483406], [-74.05304707455683, 40.60162256567726], [-74.05303911216359, 40.60153166317351], [-74.05303330741077, 40.60146647069907], [-74.05303612675755, 40.60140116360679], [-74.05304754163846, 40.6013364030034], [-74.0530674364756, 40.60127284446126], [-74.05309560984993, 40.60121113138266], [-74.05331857342289, 40.60101058824162], [-74.05335468423449, 40.60095402513734], [-74.0533839675499, 40.60089521381433], [-74.05340618556725, 40.60083463190023], [-74.05342115786627, 40.60077277140067], [-74.05342876287239, 40.600710134704016], [-74.05342893884374, 40.600647230500904], [-74.05342168437245, 40.60058456965282], [-74.0534070583946, 40.600522661043904], [-74.05337605959711, 40.60046855454402], [-74.05333852878636, 40.60041690720496], [-74.05329479798284, 40.60036817591043], [-74.05324525405089, 40.60032279174719], [-74.05319033527633, 40.60028115619137], [-74.05300717979289, 40.600160417609814], [-74.05240319632722, 40.59986949892358], [-74.05225411910509, 40.59985116741017], [-74.05222983028327, 40.59982388359393], [-74.05222826954189, 40.599776102303636], [-74.05224531733288, 40.59974801415288], [-74.0522677927031, 40.59973493984094], [-74.05229099291411, 40.59972044052657], [-74.05232494851855, 40.59971490532035], [-74.0523724578026, 40.599718782418186], [-74.05242827330757, 40.59972815401489], [-74.05249178950837, 40.59973684307158], [-74.05260660501526, 40.5998145753706], [-74.0527173663352, 40.599854472448875], [-74.05279416485439, 40.59987346465481], [-74.05287153843383, 40.59986725546714], [-74.05291816194962, 40.5998462571782], [-74.0530228827237, 40.59977868359759], [-74.05309724728707, 40.59973106712562], [-74.05434118619358, 40.59881917533658], [-74.0543115048126, 40.598777505760154], [-74.05433644573098, 40.5987604455301], [-74.05447422333906, 40.5987555395484], [-74.05681139913331, 40.59672692304163], [-74.05719369904648, 40.59639507881197], [-74.05972690577232, 40.59390821671267], [-74.05974734759447, 40.593848611567246], [-74.05964120596147, 40.59370938839296], [-74.05966255766188, 40.593703175710345], [-74.05976660687638, 40.59383525751018], [-74.05984505851372, 40.5937805319415], [-74.06055353637058, 40.593379755423555], [-74.06133778103231, 40.59271541906451], [-74.06215274432542, 40.59202504054202], [-74.06388709125392, 40.590207466209264], [-74.06503332378131, 40.58899099843518], [-74.06563611899992, 40.589628104999896], [-74.06580036099993, 40.589746609999885], [-74.06575530299992, 40.58978965399993], [-74.06572536499986, 40.589845295999915], [-74.06571834999994, 40.58990810299984], [-74.06573690899995, 40.589969516999936], [-74.06577682899997, 40.590021390999915], [-74.0658293969999, 40.59005916499991], [-74.06651182899995, 40.59034492199989], [-74.06662559499993, 40.59034201999989], [-74.06666723999994, 40.590382761999905], [-74.06728410699992, 40.590986202999865], [-74.06751477499988, 40.591334642999925], [-74.06795227299993, 40.59208368299986], [-74.06814585499988, 40.59245576399995], [-74.06831810799993, 40.59278576599994], [-74.06964532799996, 40.59526401099989], [-74.06970478499989, 40.59537635999994], [-74.07043976099992, 40.59673474699988], [-74.07118634999992, 40.59648999199993], [-74.07281699299993, 40.59595221499992], [-74.073679591, 40.59566839599994], [-74.07453730499996, 40.595386779999885], [-74.07539899099994, 40.59510414799992], [-74.07626202999995, 40.5948225699999], [-74.07714922399995, 40.594534080999885], [-74.07802261099994, 40.59425482399996], [-74.07888191699986, 40.593942790999925], [-74.0784669379999, 40.5933000119999], [-74.07936282299997, 40.59299407399992], [-74.08029682099989, 40.59266864199986], [-74.08173342999991, 40.59215628599987], [-74.08177066499995, 40.59221445099991], [-74.08239591899998, 40.59319116199994], [-74.08304163599998, 40.59422972099989], [-74.0832075799999, 40.59453373499988], [-74.08330235099992, 40.59471632799989], [-74.08376862299998, 40.595520956999934], [-74.08406968699998, 40.59549626999994], [-74.08491019899996, 40.59570472799985], [-74.08578780799998, 40.59588407999995], [-74.08587511899998, 40.59589012099985], [-74.08585350599992, 40.5959409809999], [-74.08547413999999, 40.596833816999876], [-74.08532650099991, 40.59700832199995], [-74.08515829199987, 40.59717273799993], [-74.08515824499993, 40.597172791999874], [-74.0851581859999, 40.59717283699992], [-74.08497132899998, 40.59732510999985], [-74.08476818599993, 40.59746346299987], [-74.08476811499993, 40.59746351699994], [-74.08476803199989, 40.597463562999955], [-74.08469129699996, 40.597507131999926], [-74.08360775099997, 40.597912413999936], [-74.08248303399992, 40.59832980799991], [-74.0823092379999, 40.59845665499994], [-74.08199990199995, 40.59868243799987], [-74.08160410699992, 40.59902445899997], [-74.0814942209999, 40.599081990999935], [-74.08144216599995, 40.5991592199999], [-74.08130343799999, 40.59938765599991], [-74.08116420899995, 40.600206553999946], [-74.08119595999997, 40.600415172999874], [-74.081216903, 40.600547931999905], [-74.08125785899989, 40.60067884799985], [-74.081318208, 40.60080532899989], [-74.0813965889999, 40.60092495199989], [-74.08184498199992, 40.601535196999926], [-74.08218729299988, 40.601941007999876], [-74.082815161, 40.60197418199989], [-74.08383802499996, 40.60192576699993], [-74.08432821299992, 40.6018726829999], [-74.08457153399995, 40.601822479999946], [-74.08520894199994, 40.60160095799991], [-74.085837971, 40.60142777299994], [-74.08625420399993, 40.601324188999854], [-74.08647535899999, 40.60126896199992], [-74.08666235099997, 40.6012335389999], [-74.0883067039999, 40.60110777999993], [-74.08920169499996, 40.600963789999916], [-74.08947630599992, 40.60090432199991], [-74.08983655899989, 40.60077409499996], [-74.09010463499996, 40.600592348999896], [-74.09036073899993, 40.60040036399994], [-74.09060390399996, 40.60019882399991], [-74.09083323299996, 40.59998852399986], [-74.09147742399989, 40.599408489999895], [-74.0916133049999, 40.59926156199984], [-74.09172051499995, 40.59922085699985], [-74.091798925, 40.599318209999886], [-74.09197184899992, 40.59982426299994], [-74.09211869899993, 40.600086968999946], [-74.09227904299993, 40.60034540999989], [-74.0924525679999, 40.60059897899986], [-74.09263886099998, 40.60084710199989], [-74.09283746599989, 40.60108923399986], [-74.0928674959999, 40.60117779299992], [-74.092888504, 40.601267850999896], [-74.09290036399996, 40.60135887199993], [-74.09290300799996, 40.601450319999834], [-74.09289641899991, 40.60154165099987], [-74.09288063499999, 40.6016323259999], [-74.09285575199992, 40.60172180699987], [-74.09282191499993, 40.60180956599985], [-74.09272060299999, 40.60249806099995], [-74.09260921099994, 40.60323012299987], [-74.09244145299986, 40.604119585999875], [-74.09258665899993, 40.60514053599992], [-74.09262561899992, 40.60545055399992], [-74.09268272599991, 40.60591244899991], [-74.09271376799991, 40.606059182999914], [-74.09271379199996, 40.606059272999886], [-74.092713803, 40.60605936299993], [-74.09272620699986, 40.60620985699994], [-74.092719093, 40.6063620239999], [-74.09271909299996, 40.60636212299986], [-74.09271908099993, 40.60636222199992], [-74.09269213899994, 40.60651384699985], [-74.09264560299997, 40.60666272199991], [-74.09264557999991, 40.606662785999916], [-74.092645556, 40.60666283899991], [-74.09258024999994, 40.60680678599988], [-74.09249749399994, 40.606943849999894], [-74.09239912899989, 40.607072288999916], [-74.09203043499996, 40.607426039999915], [-74.09187212199998, 40.60756836799993], [-74.09172937299998, 40.60772119199991], [-74.09160388599999, 40.60788297299994], [-74.09149702999991, 40.60805196799984], [-74.0914097369999, 40.60822629799987], [-74.09096036099994, 40.608629682999876], [-74.09080114099994, 40.60877808899989], [-74.09055711299992, 40.60900115099989], [-74.09045495899996, 40.60909452599991], [-74.08949370099997, 40.60868693299991], [-74.08874638699992, 40.60837004599988], [-74.08796268699999, 40.608051827999915], [-74.0871642159999, 40.60772760199992], [-74.08635759599996, 40.60740005899988], [-74.08542756299994, 40.60702238999989], [-74.084900025, 40.60689013199992], [-74.08406781699996, 40.60668148499985], [-74.08355131599996, 40.60655198599987], [-74.08341933399991, 40.60652250899992], [-74.0828094989999, 40.606410978999904], [-74.0815547009999, 40.60629029699987], [-74.07778089599998, 40.60617100899986], [-74.07656734699997, 40.606121663999915], [-74.07606892799996, 40.60606295199986], [-74.07551713999996, 40.60599795099993], [-74.07543239699991, 40.605987966999884], [-74.0747894209999, 40.60587021199996], [-74.0739996529999, 40.60572556799985], [-74.07358326599991, 40.60562106699992], [-74.07335207399997, 40.60556304399993], [-74.07298106899998, 40.605469929999835], [-74.07268510599995, 40.60538461299993], [-74.07242904799989, 40.605302701999946], [-74.07206682799995, 40.60518634699984], [-74.07172956299996, 40.60505808999994], [-74.0691403099999, 40.60403971899988], [-74.06898118299992, 40.60398191899988], [-74.06571018099996, 40.60279382299992], [-74.06550663199994, 40.602719880999935], [-74.06472638299991, 40.602497866999904], [-74.0642173399999, 40.60235301499994], [-74.06367318899999, 40.60219816599994]]]}}, {\"id\": \"6\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 7, \"Shape_Leng\": 0.107417171123, \"Shape_Area\": 0.000389787989274, \"zone\": \"Astoria\", \"LocationID\": 7, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90413637799996, 40.76752031699986], [-73.90325211599989, 40.767504322999905], [-73.9030052879999, 40.767509662999885], [-73.90275780099984, 40.76750354699987], [-73.90251104099987, 40.76748594099992], [-73.90226637199996, 40.76745694399991], [-73.90202516599992, 40.767416783999884], [-73.90217951699996, 40.76731584199997], [-73.90255919500005, 40.766940786999896], [-73.90341888199997, 40.765957306999944], [-73.90351227799994, 40.76584945999989], [-73.90356915599997, 40.765781788999924], [-73.90361058299999, 40.765726577999935], [-73.90370771099997, 40.765597148999845], [-73.90451229899999, 40.76458906399989], [-73.90519286699994, 40.763750940999934], [-73.90566075299986, 40.7631679879999], [-73.90606811199996, 40.7626604449999], [-73.90625691699984, 40.76242520699992], [-73.90651032099997, 40.76210946999992], [-73.90677808699982, 40.7617774819999], [-73.90703976399989, 40.76145304399992], [-73.907419729, 40.76098192599993], [-73.90753042700004, 40.7608446779999], [-73.90779405499987, 40.760517799999896], [-73.90740334999987, 40.76033397899992], [-73.90699741899994, 40.76014299099993], [-73.90631505099988, 40.75984387599992], [-73.9074261499998, 40.75810005299993], [-73.90782817499984, 40.75699926499987], [-73.9079488889999, 40.75667581499994], [-73.90798755699986, 40.756572205999916], [-73.90802408399996, 40.75647433199986], [-73.90806081699996, 40.75637590799995], [-73.90810013899984, 40.75627054199987], [-73.90813601099995, 40.75617442399993], [-73.90817188199993, 40.75607830499995], [-73.90820623799995, 40.75598624799986], [-73.90824210899991, 40.75589012999991], [-73.90827537899996, 40.7558009829999], [-73.90830736999996, 40.7557152619999], [-73.90835663299995, 40.7555832579999], [-73.90840546799983, 40.755452399999896], [-73.90846544599992, 40.75529168599991], [-73.908509019, 40.75521110499989], [-73.90853125399997, 40.75516998399991], [-73.90854932099997, 40.75513657099991], [-73.90860265599999, 40.7550379379999], [-73.90862474299999, 40.75499709099988], [-73.90865671500002, 40.75493796099989], [-73.90868067299994, 40.75489365399992], [-73.90871371100005, 40.7548325549999], [-73.908736451, 40.75479049999993], [-73.90876818500003, 40.754731814999936], [-73.90879316499999, 40.75468561599994], [-73.90901259099992, 40.75427981499996], [-73.90917451199998, 40.75406416699992], [-73.90922139799983, 40.754001722999895], [-73.90922902699994, 40.75399264199997], [-73.90944851199998, 40.75373135399991], [-73.90969314899992, 40.75346973299989], [-73.90995439699995, 40.753217822999936], [-73.91023124699991, 40.75297652299985], [-73.91067474899997, 40.752995076999895], [-73.91256954199997, 40.75330421499995], [-73.9135418349999, 40.7534687139999], [-73.91446450799998, 40.75367641199993], [-73.91468429799998, 40.753680068999856], [-73.91547326299991, 40.75379181799991], [-73.91650534599997, 40.75389159099995], [-73.91763192900005, 40.75385784899993], [-73.91892348099987, 40.75363612399991], [-73.920276847, 40.75333811799993], [-73.92047926000001, 40.75328523699993], [-73.92161909499991, 40.75302765399987], [-73.92204020499989, 40.75293762799992], [-73.92296643399989, 40.752728801999844], [-73.9244174849999, 40.75240430199992], [-73.92562743799996, 40.75213724499991], [-73.92415699999985, 40.753961141999845], [-73.92495400099993, 40.75433675199992], [-73.92575429299997, 40.75471089899986], [-73.92655413100006, 40.75508563499993], [-73.92735228500001, 40.75546007599991], [-73.92815372599988, 40.755835855999905], [-73.92895587499993, 40.75621258399985], [-73.92975052199998, 40.75658057699988], [-73.93055267399994, 40.756958942999916], [-73.93135128699987, 40.75733408499986], [-73.93215200299993, 40.757710384999925], [-73.93335306599997, 40.75827433399995], [-73.93425020699995, 40.75869513399993], [-73.93496053299995, 40.75902508599996], [-73.93503410199992, 40.75905925799991], [-73.93505987399978, 40.759071228999865], [-73.93579879199996, 40.759418044999904], [-73.93588175099988, 40.75945697999986], [-73.93679091699993, 40.75988288099991], [-73.93605584499984, 40.76065330999995], [-73.93601701999981, 40.760694001999894], [-73.93593276099996, 40.7607823119999], [-73.93589256999996, 40.7608244349999], [-73.93584407599987, 40.760875259999935], [-73.93581185199994, 40.7609090339999], [-73.93577489099982, 40.7609477719999], [-73.93569960399994, 40.7610266769999], [-73.93566314299997, 40.76106489099995], [-73.93560854899978, 40.761122109999846], [-73.93556528699992, 40.76116745099992], [-73.93550408999991, 40.76123158999994], [-73.93546712899983, 40.761270327999945], [-73.93543323399999, 40.76130585199992], [-73.93538955299994, 40.76135538399994], [-73.93480791700001, 40.761960415999894], [-73.93472503099987, 40.762047169999875], [-73.93463877499988, 40.762137449999926], [-73.93457323599995, 40.762206044999935], [-73.9343717929999, 40.7624168849999], [-73.93429712599992, 40.76249503399991], [-73.9341081699999, 40.76269280199993], [-73.93404264799992, 40.762761377999915], [-73.93398092100001, 40.76282598199987], [-73.93541372899996, 40.763496787999884], [-73.93614654199985, 40.763842828999906], [-73.93616861799997, 40.76385325299988], [-73.93619184999997, 40.763864223999846], [-73.93621639299991, 40.76387581299986], [-73.9362557499999, 40.7638943969999], [-73.93684249199987, 40.764165082999924], [-73.93687031099991, 40.76417791699989], [-73.93700478599986, 40.76423995399997], [-73.93600532399995, 40.76588489499987], [-73.93682006599995, 40.76618273599989], [-73.93763535399994, 40.76647487199988], [-73.93846096299997, 40.766768302999935], [-73.93858976899998, 40.766817358999916], [-73.93836125899992, 40.76691075499987], [-73.93709253699996, 40.767360392999905], [-73.93618379699991, 40.767691800999934], [-73.93594942700004, 40.76757822499996], [-73.93493511099992, 40.76701302399987], [-73.93422865800001, 40.767854682999875], [-73.93344468499993, 40.768817789999915], [-73.93202371699994, 40.768256495999886], [-73.9298564789999, 40.76732655199985], [-73.92855636599988, 40.76671423899995], [-73.92643792699984, 40.765732814999936], [-73.92506062099999, 40.76509731499991], [-73.92425288799978, 40.76472456399995], [-73.92353865299998, 40.76560349799989], [-73.92218946199995, 40.767267356999895], [-73.92139066099995, 40.76689410499988], [-73.92058727399994, 40.76652069899995], [-73.91977764799998, 40.7661521839999], [-73.918906013, 40.76722677599987], [-73.91849336999996, 40.76773465699995], [-73.91911161599978, 40.76802545099991], [-73.91995596899991, 40.768440064999865], [-73.91828990899992, 40.76987856799984], [-73.91811686499992, 40.77002848599989], [-73.91759120499997, 40.769900870999926], [-73.91682128799984, 40.76979176099995], [-73.91643979099995, 40.76972971399988], [-73.915848706, 40.76963357499988], [-73.91487376500001, 40.76947041699985], [-73.91388850199984, 40.769323929999906], [-73.91291678, 40.76916129699992], [-73.91194026099988, 40.769010152999954], [-73.91094174899979, 40.76885247299993], [-73.909980063, 40.76869723499987], [-73.90900867299995, 40.76854754699986], [-73.90803079099996, 40.76838473399991], [-73.90708728999995, 40.76823623099996], [-73.90611305099988, 40.76806770499989], [-73.90577352999988, 40.76802858899988], [-73.9055484799998, 40.76797657999987], [-73.90531717799979, 40.76789864999989], [-73.90490866399992, 40.76774556099986], [-73.90446527099982, 40.767583778999914], [-73.90427336799988, 40.76754510499992], [-73.90418895299992, 40.767530284999964], [-73.90413637799996, 40.76752031699986]]]}}, {\"id\": \"7\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 8, \"Shape_Leng\": 0.0275906911574, \"Shape_Area\": 2.6587716279e-05, \"zone\": \"Astoria Park\", \"LocationID\": 8, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92334041500001, 40.77512891199993], [-73.92398142899997, 40.774615111999886], [-73.92523246199984, 40.77529885399988], [-73.92531479799985, 40.7753438509999], [-73.92602208199996, 40.77573039899992], [-73.92658081899985, 40.7760392589999], [-73.92737641899994, 40.77646661099989], [-73.92782147400004, 40.776697526999946], [-73.92827730816353, 40.77689897516217], [-73.92708229009109, 40.77771978109686], [-73.92631833401951, 40.77824177509243], [-73.9258220122297, 40.778385906131554], [-73.92574521623308, 40.77840537091916], [-73.9256507618914, 40.77843081191646], [-73.92557304281429, 40.77845885141435], [-73.92481615523091, 40.77875769181163], [-73.92423363003755, 40.7791011938827], [-73.92318694398986, 40.780248393062635], [-73.92295652695057, 40.780441008061004], [-73.92263013815422, 40.78056478536394], [-73.92209965499714, 40.78085187481845], [-73.92097562099832, 40.78176769125319], [-73.91996458796496, 40.78263506876824], [-73.91975969999989, 40.782514177999886], [-73.91796581399998, 40.78125629099988], [-73.91861823099985, 40.780732867999866], [-73.9190647659999, 40.780359563999916], [-73.91939411500002, 40.78009035599996], [-73.92012731100007, 40.7794889519999], [-73.92078680499985, 40.778952266999944], [-73.92139486599993, 40.77845444399985], [-73.92200155699992, 40.77795459099984], [-73.92264888599983, 40.77742265099986], [-73.92331041499993, 40.776883294999934], [-73.92375761299998, 40.77654701199991], [-73.92302571499998, 40.77538430899993], [-73.92314698499983, 40.77528580399993], [-73.92320981399997, 40.775234768999916], [-73.923277837, 40.77517762299988], [-73.92334041500001, 40.77512891199993]]]}}, {\"id\": \"8\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 9, \"Shape_Leng\": 0.0997840924705, \"Shape_Area\": 0.000338443803197, \"zone\": \"Auburndale\", \"LocationID\": 9, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.78502434699996, 40.761036515999855], [-73.78486059299999, 40.760644362999905], [-73.78467742500004, 40.76025720799989], [-73.78447529299982, 40.75987586199995], [-73.78430628699978, 40.759483644999904], [-73.78428991899993, 40.7594402989999], [-73.78425940899996, 40.759382645999914], [-73.78407792999997, 40.75903964399989], [-73.78382076099993, 40.75853003399991], [-73.78346111099987, 40.757864063999904], [-73.78338071499984, 40.75775389699993], [-73.78316157699985, 40.75732618499991], [-73.78303191299983, 40.75708319099989], [-73.78281576399985, 40.75666634299991], [-73.78264852999988, 40.756331903999865], [-73.78248130799982, 40.75599745499987], [-73.78212461999979, 40.75533931199993], [-73.78165171799995, 40.754707752999956], [-73.78113322699984, 40.7540893259999], [-73.78052037799992, 40.753330742999886], [-73.78009357400003, 40.75250750399991], [-73.77980639299984, 40.75189500499989], [-73.77944066999987, 40.751122695999946], [-73.77906558899984, 40.75033257399991], [-73.77831180799991, 40.748745013999944], [-73.77753347599982, 40.74714834199988], [-73.77679196999986, 40.74554428899989], [-73.77616785699998, 40.744240022999904], [-73.77596627999999, 40.74381044799988], [-73.77585237799993, 40.74356469299995], [-73.77575892599995, 40.74336722699992], [-73.77573836999991, 40.74332564699993], [-73.77579713399984, 40.743317665999925], [-73.77805835699986, 40.74298648899991], [-73.778599248, 40.742908122999886], [-73.77965641299991, 40.742729309999895], [-73.78036603499997, 40.74259166299987], [-73.78068584999994, 40.74250560699994], [-73.78100566399996, 40.74241954099993], [-73.78121857699983, 40.742362250999925], [-73.78137698899997, 40.74231999399989], [-73.78176526799983, 40.74221640599993], [-73.7821600969999, 40.742115548999905], [-73.7831101189999, 40.741848304999905], [-73.78349697199984, 40.74174135999992], [-73.78400944099984, 40.74159935699984], [-73.78484244599998, 40.74136852999993], [-73.78577051299996, 40.7411106799999], [-73.78664844699985, 40.74086674899993], [-73.78669085300001, 40.74085298599988], [-73.78673516799984, 40.74084177599985], [-73.78759444399994, 40.74061795799991], [-73.78848243499986, 40.7403682779999], [-73.7886511029998, 40.7403257139999], [-73.78934727299989, 40.74012633799987], [-73.79020750099997, 40.739894702999855], [-73.7911152649998, 40.73964983899984], [-73.79202817599997, 40.73940358099989], [-73.79285797099998, 40.73917973299994], [-73.79291603199995, 40.73916776099987], [-73.79297881699995, 40.73915188099988], [-73.79369341399999, 40.73895006199987], [-73.79440801799996, 40.73874822999989], [-73.79533772999987, 40.738493672999915], [-73.79588681699978, 40.73837637499989], [-73.79611584799986, 40.73833613999987], [-73.79640218700004, 40.73828235799987], [-73.79641178399994, 40.738325325999895], [-73.79643940099992, 40.73843894799993], [-73.79672497900003, 40.738416642999894], [-73.79629297299985, 40.73855230599992], [-73.79640833699986, 40.738593419999894], [-73.7971768669999, 40.739986684999934], [-73.79722362499994, 40.740097800999884], [-73.79722373099987, 40.74009803499988], [-73.79722380199985, 40.740098277999905], [-73.79725664999995, 40.74021283499994], [-73.79725670899992, 40.74021304299991], [-73.79725674299983, 40.740213241999896], [-73.79727515299997, 40.740330033999896], [-73.79727517699985, 40.740330185999895], [-73.79727517699983, 40.74033034899986], [-73.79727886799999, 40.74044760099988], [-73.79727887899985, 40.74044786199991], [-73.79727885499992, 40.74044812399993], [-73.79726795099995, 40.74056448599986], [-73.79723084999992, 40.74169550699993], [-73.79695065300002, 40.741933034999896], [-73.79666622899994, 40.74213522099994], [-73.79666601799985, 40.742135371999886], [-73.79666581899986, 40.742135522999895], [-73.79639432300002, 40.742348348999904], [-73.79639423599997, 40.74234842499992], [-73.79639414999993, 40.74234849999986], [-73.79613638599983, 40.742571410999865], [-73.79613631199986, 40.74257146699995], [-73.79613624999985, 40.74257152399986], [-73.79589335199982, 40.742803526999886], [-73.7958931919999, 40.74280368699995], [-73.79589302899994, 40.74280385699993], [-73.79566566899987, 40.74304423599989], [-73.79543059299992, 40.74326428999993], [-73.7952125779999, 40.74349556899989], [-73.79501312, 40.74373670699995], [-73.79501298499999, 40.74373688099991], [-73.79501285899983, 40.743737044999854], [-73.79483295499979, 40.74398694199992], [-73.79467356599999, 40.7442440949999], [-73.79467340799984, 40.74424434599993], [-73.79467327199988, 40.74424460699987], [-73.79453507400001, 40.74450763799989], [-73.7944912069999, 40.744737567999955], [-73.79449117099993, 40.74473774799991], [-73.7944911469999, 40.744737937999965], [-73.7944636839999, 40.74497002199991], [-73.79445282999983, 40.7452030899999], [-73.79445281699995, 40.74520343199987], [-73.79445282799998, 40.74520375699995], [-73.79445869199989, 40.74543695399997], [-73.79448120499993, 40.74566936999989], [-73.79448121599987, 40.74566953199994], [-73.79448123999998, 40.745669693999915], [-73.79452019299998, 40.7459001589999], [-73.79457976099988, 40.746402682999914], [-73.7946756879998, 40.74721192799994], [-73.79472264899997, 40.74759932999989], [-73.79598269399985, 40.74853189299996], [-73.796581036, 40.74897739799995], [-73.79704716099987, 40.7493323179999], [-73.79729524699995, 40.74943505299987], [-73.79644617199999, 40.74985921699992], [-73.79566545000003, 40.75023619999991], [-73.79471901099991, 40.75064761699991], [-73.79468324699988, 40.75127504499995], [-73.79465583899984, 40.75198814499991], [-73.794627692, 40.75270267499999], [-73.79457726999996, 40.7535630109999], [-73.79474720599988, 40.755628187999925], [-73.79493246199993, 40.75779802999994], [-73.79402473999977, 40.75773596899993], [-73.79303015799997, 40.75775619399999], [-73.79212059099993, 40.75777758199993], [-73.79201171599978, 40.757780149999896], [-73.79122151599987, 40.75779876599992], [-73.79031517, 40.75782345099988], [-73.79053450999999, 40.759373037999886], [-73.79076004599987, 40.76098576899991], [-73.791654662, 40.76091206299987], [-73.79255337999999, 40.76084029399992], [-73.79343991199988, 40.760764127999934], [-73.79344033499987, 40.76082252699995], [-73.79344299999985, 40.761185336999866], [-73.793438002, 40.7615135269999], [-73.79343673799991, 40.76159683099993], [-73.79334546300007, 40.76229534299991], [-73.79322483899982, 40.763142007999946], [-73.79302551399985, 40.76462119199986], [-73.7939591469999, 40.7646953199999], [-73.79361279599985, 40.76722335799996], [-73.793422277, 40.76859990799989], [-73.79321182299991, 40.77033648899987], [-73.79317944599984, 40.77040317799988], [-73.79238243799978, 40.7694838389999], [-73.79235243699989, 40.76944963799987], [-73.79190204099989, 40.768936211999915], [-73.79149870499994, 40.76844739099989], [-73.79137684499982, 40.7683242969999], [-73.79059858199989, 40.76741868799998], [-73.79021960799982, 40.76695117499985], [-73.78990502999987, 40.7665115969999], [-73.78963649799978, 40.76614161099991], [-73.78871017999994, 40.76485425199996], [-73.7886108559999, 40.764709593999925], [-73.78849669799996, 40.764570552999885], [-73.78836856600003, 40.76443842899987], [-73.78822763099994, 40.76431439599985], [-73.78807523699999, 40.76419948999993], [-73.78733166699999, 40.76380781299986], [-73.78714131699988, 40.76370782099992], [-73.78595810599988, 40.76310224399986], [-73.78571993599994, 40.762921464999856], [-73.78546383099984, 40.76259192499988], [-73.78542241999988, 40.76251310799994], [-73.78535911599988, 40.7622563179999], [-73.78530175399992, 40.76197850399994], [-73.78527359199997, 40.761842105999925], [-73.785251708, 40.761756958999904], [-73.78516838299994, 40.76143281799993], [-73.78502434699996, 40.761036515999855]]]}}, {\"id\": \"9\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 10, \"Shape_Leng\": 0.0998394794152, \"Shape_Area\": 0.000435823818081, \"zone\": \"Baisley Park\", \"LocationID\": 10, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7832662499999, 40.68999429299992], [-73.78234452799988, 40.68840743699997], [-73.78148675699995, 40.6886971829999], [-73.78051569899989, 40.6870365199999], [-73.77976509199979, 40.685743601999924], [-73.77966850599998, 40.685377122999945], [-73.77957347199998, 40.68541503299992], [-73.77870794599994, 40.685700419999876], [-73.77784971099992, 40.68598927299989], [-73.77699767599997, 40.68628268899992], [-73.77697494099996, 40.686246713999935], [-73.7767991239999, 40.68596846499985], [-73.77632402199986, 40.6852165559999], [-73.77567289899983, 40.68474158899993], [-73.77501231399985, 40.684275578999845], [-73.77462061100005, 40.68399096999984], [-73.7745579039998, 40.683885621999906], [-73.77441975399977, 40.68365352199991], [-73.77418414899984, 40.68323792099991], [-73.773499153, 40.68348222699995], [-73.77328703999997, 40.68345473099989], [-73.77297749099996, 40.68293379299995], [-73.77256209599992, 40.682209496999846], [-73.77220275299989, 40.681610338999896], [-73.77232211299984, 40.68149096799993], [-73.77299539399996, 40.6812088249999], [-73.77356539499996, 40.68096392599992], [-73.77408130199989, 40.68076015399986], [-73.77469730199986, 40.680486045999906], [-73.77475342099996, 40.68046728099991], [-73.77549588399995, 40.680219045999934], [-73.77636248100002, 40.679935908999916], [-73.77722923600001, 40.679652720999904], [-73.77749031599994, 40.67955740299992], [-73.77774211299992, 40.67944657599985], [-73.77798227199999, 40.67932109299989], [-73.77820864799988, 40.67918207599993], [-73.77841935399988, 40.6790308799999], [-73.7793120879999, 40.6784368039999], [-73.77947817299986, 40.67836454999986], [-73.78025859499995, 40.6778409239999], [-73.7809512629998, 40.677353507999946], [-73.78128277899985, 40.67710972999991], [-73.78157185499998, 40.676901330999954], [-73.782218084, 40.676437536999906], [-73.78304214600001, 40.67584371699991], [-73.7832954899999, 40.675679444999915], [-73.783536133, 40.675503090999904], [-73.78376243999988, 40.675315685999905], [-73.78376261799986, 40.67531553299985], [-73.78376278399993, 40.67531538999988], [-73.78397345, 40.675117994999944], [-73.78397363999976, 40.67511780699991], [-73.78397384099986, 40.67511760899993], [-73.78416807199993, 40.67491103399994], [-73.78434496299991, 40.6746964399999], [-73.78484333899993, 40.67390332299989], [-73.78511037999995, 40.673478363999905], [-73.78532371499995, 40.67312502099992], [-73.78573730899993, 40.67238503499988], [-73.78686902099984, 40.672823923999864], [-73.78808647399985, 40.6730096269999], [-73.78847621499992, 40.67132650099996], [-73.7888310799999, 40.6698234859999], [-73.78915781999983, 40.66833809599989], [-73.78940999399993, 40.66684115799993], [-73.79039544099997, 40.66688770399988], [-73.79129821899986, 40.666970784999904], [-73.79218446699987, 40.66704188899988], [-73.79306694299999, 40.667067293999935], [-73.79319502099987, 40.66705844699985], [-73.79392731599994, 40.667006752999946], [-73.79445715, 40.666981055999926], [-73.79498745799991, 40.66696182699995], [-73.79551810099989, 40.6669490719999], [-73.7960489459999, 40.666942792999855], [-73.7965798539999, 40.666942991999896], [-73.79711068999998, 40.66694966899994], [-73.79739484699991, 40.6669398859999], [-73.798072072, 40.66671288499993], [-73.79828522699991, 40.66664725099992], [-73.79850596999994, 40.66659438099988], [-73.79873226699989, 40.666554982999905], [-73.79896198599992, 40.666529485999874], [-73.79919289299993, 40.666518074999885], [-73.79942279899994, 40.666520663999925], [-73.79968140399984, 40.66656547099993], [-73.80027044399995, 40.66666752699987], [-73.80102008199994, 40.66672907999993], [-73.80139045299981, 40.66674665899994], [-73.80148383499996, 40.66675015199992], [-73.80155565300004, 40.6667533639999], [-73.80153052599994, 40.66683715899995], [-73.80141266299995, 40.66735840999995], [-73.80134286799985, 40.66777379699988], [-73.801250796, 40.66838594499995], [-73.80120616599989, 40.66894442399993], [-73.80117909099995, 40.66910026499992], [-73.80116000899989, 40.66980042099991], [-73.80113236199993, 40.67054866799994], [-73.8011343009999, 40.671294461999906], [-73.80113697299981, 40.67135852899988], [-73.80114238199995, 40.67148794299991], [-73.80115538600003, 40.67179912099987], [-73.80115934999986, 40.67189388999986], [-73.80117234299993, 40.67220503199988], [-73.801194616, 40.67242246499994], [-73.80130166499995, 40.67312152199991], [-73.80140210699993, 40.673561318999894], [-73.80142836799988, 40.673666833999874], [-73.80168194799982, 40.67438384799994], [-73.80202521799998, 40.675114272999934], [-73.80238906400001, 40.67588847899992], [-73.80267769599996, 40.67649796699987], [-73.8030752049999, 40.67734560099988], [-73.80323027699988, 40.677681056999894], [-73.80410226599996, 40.67942454999991], [-73.80428288199992, 40.679817355999916], [-73.80464517099982, 40.68057988999991], [-73.80492976299996, 40.68117697399989], [-73.80522682799987, 40.68179814999995], [-73.80551811199996, 40.68240820999994], [-73.80576840199991, 40.682932390999916], [-73.80568769599995, 40.68295563299993], [-73.80519209299995, 40.683096057999876], [-73.80432351099998, 40.683341794999855], [-73.80344528099988, 40.68359185699996], [-73.80256430899988, 40.68384243899986], [-73.80168572400002, 40.68409195599986], [-73.80077394899995, 40.684349927999904], [-73.79986119199985, 40.684610142999944], [-73.79943031399989, 40.68473198799994], [-73.79898209099989, 40.68485873699989], [-73.79995042799996, 40.68682771899991], [-73.79913531199985, 40.687181957999904], [-73.79828588299998, 40.68752401899992], [-73.79746057199985, 40.68787230199989], [-73.7966293329999, 40.68822145199994], [-73.79561486799987, 40.688656890999916], [-73.7952109489999, 40.688094665999884], [-73.79477837399985, 40.68747068399995], [-73.79437087299995, 40.68691258199987], [-73.793908417, 40.68630469099985], [-73.79317220799987, 40.68659668599987], [-73.79248043499994, 40.68688962999991], [-73.79210699399984, 40.687047535999945], [-73.79038468299987, 40.68775361399986], [-73.78962631299987, 40.68793905899987], [-73.78920529899993, 40.688053212999904], [-73.78867911999995, 40.68818018399991], [-73.78764209899992, 40.68843484999997], [-73.78698344499996, 40.68860847699995], [-73.78643950599998, 40.688781709999894], [-73.78580499899994, 40.68899689699988], [-73.78479240699981, 40.689361599999856], [-73.78409903999982, 40.68964775499987], [-73.7832662499999, 40.68999429299992]]]}}, {\"id\": \"10\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 11, \"Shape_Leng\": 0.0792110389596, \"Shape_Area\": 0.00026452053504, \"zone\": \"Bath Beach\", \"LocationID\": 11, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00109809499993, 40.60303462599992], [-74.00032353799993, 40.60262029199991], [-74.00002266099992, 40.60290731599988], [-73.99927902000005, 40.60245801699991], [-73.998563851, 40.602028433999905], [-73.99782332499991, 40.601578178999915], [-73.99708036399986, 40.60112882899989], [-73.9963651649999, 40.600696053999904], [-73.99559320799995, 40.600232221999875], [-73.99733919599988, 40.5985516119999], [-73.9986558659999, 40.59726856199988], [-73.99931921399997, 40.59664075099992], [-73.99951049399985, 40.59644180899987], [-73.99996502999996, 40.59599430299989], [-73.999997695, 40.59596779199991], [-74.00004233899988, 40.59594037099991], [-74.00027871999994, 40.59570266899989], [-74.00200394754667, 40.594132371137796], [-74.00219143075543, 40.594356454281495], [-74.00230298841333, 40.5945433405649], [-74.00242215707557, 40.59472747670866], [-74.00254882028027, 40.59490868266266], [-74.0026828542343, 40.59508678124064], [-74.0028241279343, 40.59526159829328], [-74.00297250329423, 40.59543296287876], [-74.00312783528081, 40.59560070742998], [-74.00328997205449, 40.59576466791808], [-74.00345875511843, 40.59592468401364], [-74.00363401947315, 40.59608059924292], [-74.00552621986131, 40.5975391540707], [-74.00774585717092, 40.59893572292123], [-74.0084840914256, 40.599391458642174], [-74.00921723904898, 40.59984404515234], [-74.00961563255356, 40.600039964139015], [-74.00977479602606, 40.60013149053109], [-74.00993545102718, 40.60022149043687], [-74.01017239979778, 40.600349996195334], [-74.01041238886523, 40.60047518174621], [-74.01065533816518, 40.60059700531704], [-74.01090145053502, 40.60071555985398], [-74.01115036636867, 40.60083066431848], [-74.01140200242962, 40.60094228021335], [-74.0116363253687, 40.60104205715398], [-74.0118728222616, 40.601138808863034], [-74.01206343136853, 40.60121952230082], [-74.01236981602695, 40.6013492571162], [-74.0129988703337, 40.60155920754662], [-74.0130216004983, 40.60156586710369], [-74.01394137729592, 40.601835344373335], [-74.01478363716735, 40.6020821029708], [-74.0153028898328, 40.60219124627411], [-74.0158240482871, 40.60229498086631], [-74.01617431336636, 40.60235057841938], [-74.0165263578993, 40.60239921156565], [-74.01687994353071, 40.60244084737736], [-74.01723483086067, 40.6024754576642], [-74.0175907796064, 40.60250301899235], [-74.01799227936888, 40.60249195235379], [-74.01829286301476, 40.60244735523035], [-74.01832347216846, 40.60241765113048], [-74.0191356390289, 40.60261275450496], [-74.01913408409737, 40.60264855305148], [-74.01932826647813, 40.60281080817145], [-74.01942760861903, 40.602871578425265], [-74.01939343499993, 40.60291176099992], [-74.01923150299993, 40.603102173999915], [-74.01918088999996, 40.60314959499995], [-74.01917685099995, 40.603154421999875], [-74.01871382199994, 40.60358207499994], [-74.01869135799996, 40.60360792399991], [-74.018660218, 40.60364106799989], [-74.01823145099998, 40.604108472999926], [-74.01821458599989, 40.60418195799988], [-74.01817646699998, 40.604252608999936], [-74.01811904999991, 40.60431522299989], [-74.01804702499993, 40.60436573499991], [-74.0175086849999, 40.60485049699985], [-74.01746711399997, 40.60496510399988], [-74.01728366699999, 40.60513782499991], [-74.01671283899998, 40.60568263499987], [-74.01640553099998, 40.605975924999875], [-74.01549854999998, 40.606841523999925], [-74.0162363669999, 40.60723326599993], [-74.01701910199995, 40.60765441699994], [-74.01522379399994, 40.6093563839999], [-74.01348255899993, 40.61103739799988], [-74.01171201599988, 40.6127350289999], [-74.01108648099991, 40.61333904399986], [-74.01050403599997, 40.613896610999866], [-74.01019858699995, 40.613712166999925], [-74.00830383399986, 40.612567999999925], [-74.00788398499994, 40.61234007199985], [-74.00708800599996, 40.61190794499986], [-74.00601438399993, 40.61132508199992], [-74.00342538899996, 40.609969092999926], [-74.00266468099994, 40.60956996599992], [-74.00185301899995, 40.60913755999995], [-74.00094408399997, 40.60863970999988], [-74.00130866099992, 40.60834356899987], [-74.00171808600001, 40.60800297799994], [-74.00195011999993, 40.60781768299991], [-74.00264412399989, 40.60725717499991], [-74.00442477199992, 40.60556859699992], [-74.00493448899994, 40.60507440599987], [-74.00444636799998, 40.60480184699984], [-74.00416249599996, 40.60465473899992], [-74.00340902699993, 40.60425938099994], [-74.00263069599995, 40.6038464489999], [-74.00184603099991, 40.60343143199989], [-74.00109809499993, 40.60303462599992]]]}}, {\"id\": \"11\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 12, \"Shape_Leng\": 0.0366613013579, \"Shape_Area\": 4.15116236727e-05, \"zone\": \"Battery Park\", \"LocationID\": 12, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01565756599994, 40.70483308799993], [-74.01539758699995, 40.70480014899991], [-74.01527219799995, 40.70483003699996], [-74.01516257, 40.704805094999884], [-74.01496125199992, 40.70475979699996], [-74.01489542099996, 40.7046975179999], [-74.01478059499996, 40.70467411299991], [-74.0146670989999, 40.704647212999895], [-74.01451712299998, 40.704605674999854], [-74.01437031200001, 40.70455803799994], [-74.01428922899998, 40.70454907499989], [-74.01427065899998, 40.70452022899989], [-74.01425861399996, 40.70448944299991], [-74.01425342599994, 40.70445756399993], [-74.01425523899994, 40.704425471999905], [-74.014391769, 40.70359878299993], [-74.01442983199998, 40.70315064799991], [-74.014192151, 40.70275671099995], [-74.01389890399994, 40.702270669999976], [-74.01379120799997, 40.70130547599994], [-74.01378191199989, 40.70122215499992], [-74.01377528899994, 40.70118271299988], [-74.01376867799992, 40.7011432629999], [-74.01415893499998, 40.70109628999988], [-74.01395152778245, 40.70099098222993], [-74.0140919838548, 40.70055161832718], [-74.01410490300086, 40.7005108283869], [-74.01409144949483, 40.700508336726905], [-74.01412821190817, 40.70039328205465], [-74.0141372638859, 40.70039495521028], [-74.01414860637784, 40.7003205717237], [-74.01417467279444, 40.70026089129383], [-74.014220631955, 40.70011149186829], [-74.0143098131129, 40.70012129606874], [-74.01430981586151, 40.70013410151564], [-74.01417787931776, 40.70054147355368], [-74.01416495763299, 40.70053905203081], [-74.014152387873, 40.70057786578352], [-74.01428920741124, 40.700644144765086], [-74.01430157023238, 40.70063221177271], [-74.0142965168929, 40.700629186523706], [-74.01431530264765, 40.70061103797418], [-74.01431796468076, 40.70061249730781], [-74.01436783583218, 40.70054612266748], [-74.01444813150715, 40.70044737239987], [-74.01455999973689, 40.70030978677959], [-74.01457513332028, 40.70031691633122], [-74.01458425440417, 40.70030846895508], [-74.01458234409432, 40.70028387552862], [-74.01463294333476, 40.700197430412395], [-74.01467119992019, 40.70021040169973], [-74.01466717213704, 40.700239380907306], [-74.01465386124688, 40.700242641175116], [-74.01460863003733, 40.700309708245804], [-74.01460825448504, 40.70032365711382], [-74.01455219325995, 40.70038298075157], [-74.01449342549978, 40.70044516925065], [-74.01449246893418, 40.70045466900283], [-74.01448492126369, 40.70046115359205], [-74.01447857931363, 40.70046208215578], [-74.01440947394667, 40.7005513314123], [-74.01439689988958, 40.70056348189321], [-74.01439480693733, 40.70057047909727], [-74.01432384363005, 40.70066130412713], [-74.01464323234902, 40.70081730432509], [-74.01472424834412, 40.70071429389424], [-74.01477142439134, 40.700663651906844], [-74.01481236514043, 40.70061084993608], [-74.01485926659423, 40.70055544464551], [-74.01493032611235, 40.70047926986344], [-74.01499883053579, 40.70040612057649], [-74.0150681826931, 40.70031825755492], [-74.0150983227013, 40.70033426552348], [-74.01503579735073, 40.70042083101053], [-74.01502271060043, 40.70041953605862], [-74.01500850517252, 40.70043770992607], [-74.01499314575906, 40.700436847319885], [-74.01496757071692, 40.70046541487967], [-74.01497495555579, 40.70047017772347], [-74.01494881182722, 40.70049441360121], [-74.01494255505823, 40.70050610329131], [-74.01492322289359, 40.70051519147152], [-74.0149175465081, 40.70052557571268], [-74.01492379463954, 40.70053033870716], [-74.01493687947057, 40.70052297938124], [-74.01494199155965, 40.70052384332575], [-74.0148859896547, 40.70058011443573], [-74.01482004284685, 40.700660619780244], [-74.01472681782191, 40.700774015849575], [-74.01470265033873, 40.70079911602562], [-74.014673656285, 40.70083460953097], [-74.01481072560155, 40.700903831982195], [-74.01482834225268, 40.70089474286971], [-74.01502113661684, 40.700992504013406], [-74.01514231812091, 40.700842892354245], [-74.01567778184732, 40.70125602102741], [-74.01579351711148, 40.701346905807576], [-74.01591034233417, 40.7014609484439], [-74.01611609060468, 40.701630801625356], [-74.01613890635998, 40.70164601710269], [-74.0162444906067, 40.70175272143678], [-74.01639205370195, 40.70189770100826], [-74.01651515260785, 40.70201729867141], [-74.01653258703746, 40.702036539683036], [-74.01681859599911, 40.70233857508047], [-74.01689657041247, 40.70242187995947], [-74.01700396789508, 40.702554535627485], [-74.01713705889382, 40.702711340267676], [-74.01714508062653, 40.70270534997615], [-74.01725182133947, 40.70283555726373], [-74.01724571591124, 40.70283845773918], [-74.01729346221033, 40.70288743870684], [-74.01731837763397, 40.702918871771594], [-74.01739839332275, 40.703019140804905], [-74.01744003546048, 40.70307751454187], [-74.01749381162765, 40.7031548434601], [-74.01754038297949, 40.703236160429874], [-74.01757548561497, 40.70330750305602], [-74.0175964796706, 40.70336138730908], [-74.01761681217123, 40.70342324846537], [-74.01762601904944, 40.70342276122709], [-74.0176479750685, 40.70349459987955], [-74.01766045074616, 40.70354997909756], [-74.01766881307267, 40.703591313467626], [-74.01767132020255, 40.70360642738378], [-74.01768310568107, 40.703711131225276], [-74.01767264581422, 40.70371179104597], [-74.01767689629425, 40.70377050363851], [-74.0176742874818, 40.70383436757152], [-74.01766578442252, 40.703899233718545], [-74.01765792483437, 40.703932661786354], [-74.01764306532696, 40.70398897346138], [-74.01765378898192, 40.70399838081254], [-74.01765461989744, 40.704009682569755], [-74.01764390293607, 40.70402224590194], [-74.01763399890548, 40.70402475178324], [-74.01762657963023, 40.704023500821606], [-74.01757035703022, 40.70418388099316], [-74.01713123899997, 40.70428253499989], [-74.01684600699993, 40.70463488599989], [-74.01675906699995, 40.704742282999845], [-74.01668714699993, 40.704743941999865], [-74.01668613099999, 40.70479933799992], [-74.01662761199991, 40.704799430999856], [-74.01652040399995, 40.70479971799988], [-74.01637519799986, 40.704838484999925], [-74.01629888499991, 40.70484641499997], [-74.01621736899986, 40.7048530269999], [-74.01587396199992, 40.70488211299991], [-74.01565582199999, 40.704882633999915], [-74.01565756599994, 40.70483308799993]]]}}, {\"id\": \"12\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 13, \"Shape_Leng\": 0.0502813228631, \"Shape_Area\": 0.000149358592917, \"zone\": \"Battery Park City\", \"LocationID\": 13, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01244109299991, 40.7190576729999], [-74.01281566799992, 40.71735629799986], [-74.01286663199996, 40.717176100999936], [-74.0130514639999, 40.7164547709999], [-74.01323384899992, 40.71565618799984], [-74.0133512149999, 40.71523331199992], [-74.01347524299993, 40.71511071099989], [-74.01359239999995, 40.7144943159999], [-74.01375388499997, 40.71369055499987], [-74.01404744699995, 40.7124378149999], [-74.01417753399997, 40.71188266599996], [-74.0142689369999, 40.71146781599992], [-74.01445559399995, 40.71090715299986], [-74.01466507099994, 40.71042693799992], [-74.01482432799999, 40.70982336799991], [-74.01487257099997, 40.70968487399991], [-74.01495415399995, 40.70943269199994], [-74.01534330799996, 40.70849681699988], [-74.01559220199994, 40.70788324299986], [-74.015643777, 40.707756098999894], [-74.01568829399997, 40.70766382499986], [-74.01571815199996, 40.70758531099992], [-74.01584985499994, 40.70724726799987], [-74.01586395999995, 40.707200182999905], [-74.01588574099995, 40.70714217599992], [-74.01590498299998, 40.70709269499987], [-74.01593869399987, 40.70700757399988], [-74.0160596679999, 40.70670512999991], [-74.01619519599998, 40.70625848499991], [-74.0162515959999, 40.70607261399987], [-74.01640305499988, 40.70566620199989], [-74.01662217899997, 40.70509448299992], [-74.01665417299996, 40.7049638209999], [-74.01668613099999, 40.70479933799992], [-74.01668714699993, 40.704743941999865], [-74.01675906699995, 40.704742282999845], [-74.01684600699993, 40.70463488599989], [-74.01713123899997, 40.70428253499989], [-74.01757035703022, 40.70418388099316], [-74.017794545203, 40.70423489810132], [-74.01785585682943, 40.70407884761557], [-74.01775054986874, 40.70405488289344], [-74.01779139303439, 40.70395091258355], [-74.01781758697946, 40.70393792249663], [-74.01783782825969, 40.70396266613854], [-74.01780821746945, 40.70403196447959], [-74.01791229326216, 40.704054497150985], [-74.01794269363918, 40.703973176576255], [-74.01795473982617, 40.703975785608584], [-74.01784441106726, 40.7042420399379], [-74.01844770243774, 40.70416216823156], [-74.01866830837966, 40.704435240621244], [-74.01865606299125, 40.704440968684594], [-74.01751639597092, 40.704580844793384], [-74.0174808615258, 40.704667487696156], [-74.01753166144591, 40.70467109085019], [-74.01754791723494, 40.70466387515681], [-74.01755808298708, 40.70466593603325], [-74.01757026976702, 40.704662836429215], [-74.01757365505411, 40.704667996083565], [-74.01756688836936, 40.70467159878248], [-74.01759804780863, 40.704684480713354], [-74.0176342910399, 40.70467891041742], [-74.01767567049036, 40.704680875362584], [-74.01768666268582, 40.704673985963076], [-74.01771834047513, 40.704679887260674], [-74.01773321304685, 40.70467152000495], [-74.01774937712389, 40.7046754518885], [-74.01776552858085, 40.70467249656343], [-74.01776682871572, 40.70466511126367], [-74.01778363303418, 40.70467101601077], [-74.01779785539696, 40.70466707943758], [-74.0178062672323, 40.70466216121492], [-74.01782889269187, 40.7046651016672], [-74.01783988433819, 40.704661165086556], [-74.01790194580029, 40.70465180844165], [-74.01791034926417, 40.70465919223341], [-74.01794138637455, 40.70465573717841], [-74.01794397538687, 40.704648361727315], [-74.01796918021806, 40.70464736737476], [-74.01797888053436, 40.704640971684015], [-74.01798922454327, 40.7046468779127], [-74.01799827820122, 40.70464982983339], [-74.01800797806544, 40.70463851001485], [-74.01802414308348, 40.704645890908516], [-74.01804288218334, 40.70462817432251], [-74.01805387593451, 40.70462964047163], [-74.01806504689, 40.704635960523234], [-74.01811296129115, 40.704553727487166], [-74.01886287526735, 40.70471961588463], [-74.01888746649367, 40.70472914883739], [-74.0189257238241, 40.704773762570255], [-74.01934155704531, 40.70607053820401], [-74.01934254624483, 40.706093673029706], [-74.0192224228398, 40.706573088725996], [-74.019122759634, 40.70697581433742], [-74.01910255488423, 40.70700043764151], [-74.01909155084701, 40.7070044731845], [-74.0190661714014, 40.707013770621415], [-74.01903338019694, 40.70701319087609], [-74.01895359463364, 40.70699676946552], [-74.01894579167332, 40.7070177709414], [-74.01893740557156, 40.70703420689672], [-74.018926604438, 40.70704425778585], [-74.01890921570659, 40.70706434245121], [-74.01888462028889, 40.70708580553676], [-74.01885943006913, 40.70710041568372], [-74.01882943775024, 40.70711548664006], [-74.01879644936784, 40.707127819847635], [-74.0187610576787, 40.70713786652501], [-74.01872626987436, 40.70714335568639], [-74.01868966870467, 40.707142443753945], [-74.0186554807563, 40.70714153912505], [-74.01863058372689, 40.70713971526927], [-74.0185807840372, 40.707123748102724], [-74.01854718475336, 40.707107770264166], [-74.01857117399432, 40.70707992238869], [-74.01860717601117, 40.70709544097519], [-74.01863417023104, 40.7071031988087], [-74.01864437101163, 40.7071054758605], [-74.01864796262215, 40.70708173802993], [-74.0186869537992, 40.707084009237526], [-74.01872115249107, 40.70708172614977], [-74.01875054622958, 40.70707989330998], [-74.01876194037423, 40.707075785866905], [-74.01874993323987, 40.707049762489504], [-74.01877393087639, 40.70704428278756], [-74.01879432757899, 40.70703241167225], [-74.01882011914348, 40.70701597288301], [-74.01883870432944, 40.70699496965148], [-74.01884829724429, 40.70698309041019], [-74.01884889814508, 40.70697031235021], [-74.01881110035015, 40.7069662117742], [-74.01879851485626, 40.70698218837616], [-74.01877571598487, 40.70699725864644], [-74.01874212416382, 40.70700776337878], [-74.01872292981695, 40.70700685750043], [-74.0186947319076, 40.707010967669895], [-74.01867853372192, 40.70701873300243], [-74.0186479447413, 40.70701781957846], [-74.01862814488057, 40.70701097846493], [-74.01861254498962, 40.70699728514591], [-74.0186047441128, 40.706989064177236], [-74.01860294181307, 40.7069749179781], [-74.01859153053353, 40.70695939491844], [-74.01857952850992, 40.70694981480905], [-74.01857172609165, 40.70693839792395], [-74.0185693319175, 40.70692059520576], [-74.01856452356972, 40.706906440109265], [-74.01854892759579, 40.70690827067913], [-74.01853513269664, 40.70691832201406], [-74.01853033752076, 40.70695073241034], [-74.01840649088626, 40.707257872158465], [-74.01821726389763, 40.707742206471714], [-74.01821699384473, 40.70775130976829], [-74.01819029259642, 40.70782162610998], [-74.01865557049648, 40.70792394888325], [-74.01864689079073, 40.70794681354695], [-74.0188006306842, 40.707980621637354], [-74.0188319588392, 40.70799916729172], [-74.01883395951484, 40.70800221914011], [-74.01885347760366, 40.70802201228401], [-74.01885500911972, 40.70804753444736], [-74.01874689830457, 40.70848058025357], [-74.01863142468959, 40.7089411000542], [-74.01846882134515, 40.70958957100872], [-74.01834136609794, 40.71010647694362], [-74.01826409180671, 40.71040899662683], [-74.01814480361767, 40.71088865018895], [-74.01811057789173, 40.711029628214455], [-74.01804609687575, 40.71134450148705], [-74.01797853231488, 40.711756319208014], [-74.01792224257028, 40.712039242700754], [-74.01787333422655, 40.712315144456404], [-74.01777014254499, 40.71283457478916], [-74.01775292705896, 40.712848327555946], [-74.01773183836696, 40.71284931234664], [-74.01772021563546, 40.71284473093366], [-74.01772148946118, 40.71278414511305], [-74.01780037888389, 40.71234512415861], [-74.01662424425285, 40.71215731899529], [-74.0165446381253, 40.712456037930906], [-74.01659227582768, 40.71254817107677], [-74.01661817655314, 40.71264976356906], [-74.01661840078299, 40.71275615002966], [-74.01659149520512, 40.712861563719784], [-74.0165554889345, 40.71292909550437], [-74.01654112171721, 40.71295556481934], [-74.01647724204135, 40.713029493073954], [-74.01639169485804, 40.713099997950714], [-74.01632006627581, 40.71340798247826], [-74.01654114686625, 40.713441052742326], [-74.01691068695474, 40.71349632851291], [-74.01754443334657, 40.71361122266428], [-74.01756811041477, 40.71352370347795], [-74.01757889322138, 40.71344274342789], [-74.01764810719082, 40.713206185951634], [-74.01765509447189, 40.71316106266698], [-74.01767041664047, 40.71306920796511], [-74.01767270851062, 40.7130659962747], [-74.01767565436691, 40.71306310981558], [-74.01767917626104, 40.71306062496484], [-74.01768318100166, 40.71305860747276], [-74.0176875626216, 40.71305711072315], [-74.01769220518092, 40.71305617432077], [-74.0176969858353, 40.713055823043184], [-74.01770177808676, 40.71305606618535], [-74.01770610821879, 40.713056814196534], [-74.01771024196847, 40.71305804785401], [-74.017714085846, 40.713059739257254], [-74.01771755291756, 40.7130618501529], [-74.01772056477111, 40.7130643328008], [-74.01772305328981, 40.71306713105258], [-74.01772496219284, 40.71307018162274], [-74.01770687258355, 40.713169282968664], [-74.01761833310017, 40.7136543239355], [-74.01753721790737, 40.714097542247806], [-74.01740871360494, 40.71480680040797], [-74.01738223187851, 40.71494446127923], [-74.01726421509528, 40.71559268236992], [-74.01712122293634, 40.716371793579704], [-74.01698274294627, 40.71713064366814], [-74.0168629737726, 40.71778695428783], [-74.01671018605829, 40.718624176057965], [-74.01669419095931, 40.718648843354394], [-74.01665946969602, 40.71866602122463], [-74.01662996338182, 40.71866536845022], [-74.01594695204268, 40.718575548906244], [-74.01594382984199, 40.71858925504418], [-74.01566883666864, 40.718554489265706], [-74.01567062060764, 40.718537748736196], [-74.01555087754495, 40.71852193318381], [-74.01500060682406, 40.71844923605504], [-74.01499619731617, 40.718473001316646], [-74.0149898133956, 40.718507347327694], [-74.01498333230728, 40.71852379143296], [-74.01497036258002, 40.718535850864214], [-74.01494515549334, 40.718543535077174], [-74.01470270021802, 40.718516154884156], [-74.01321322719183, 40.7183157198958], [-74.01306805492324, 40.71892219382553], [-74.013072377692, 40.718938618670066], [-74.01303068113208, 40.71913145590955], [-74.01270167899996, 40.71909177499986], [-74.01255468899996, 40.71906986299998], [-74.01244109299991, 40.7190576729999]]]}}, {\"id\": \"13\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 14, \"Shape_Leng\": 0.175213698053, \"Shape_Area\": 0.00138177826442, \"zone\": \"Bay Ridge\", \"LocationID\": 14, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.03407329297129, 40.64431393298185], [-74.03284292006926, 40.64356371017527], [-74.03266558998958, 40.643720344890056], [-74.03260644399995, 40.643698910999944], [-74.02905968899994, 40.64172667299992], [-74.02889666799992, 40.64121402199986], [-74.02890829899998, 40.641173410999905], [-74.02839277799997, 40.640963136999865], [-74.02802775299989, 40.6407934899999], [-74.02742711399993, 40.64051433699994], [-74.02703439399988, 40.640349985999876], [-74.02691981299992, 40.64030850799986], [-74.02674292499995, 40.64024446799992], [-74.02662323799991, 40.6402011359999], [-74.02626267299999, 40.64009438099988], [-74.02628611999998, 40.64007452899986], [-74.02635007599996, 40.64001846499988], [-74.02646434999998, 40.639918292999916], [-74.02659984399993, 40.63978656099993], [-74.02664251499989, 40.63974743299988], [-74.02686129399991, 40.639536290999885], [-74.02695244699993, 40.63944606499998], [-74.02475437599998, 40.6381214179999], [-74.0245384189999, 40.63832493699993], [-74.02417094699996, 40.63866937099985], [-74.02392610699991, 40.638898857999926], [-74.02354939199988, 40.63932033399992], [-74.02351787199997, 40.63944477399987], [-74.02348203699995, 40.63953321299994], [-74.02346821799988, 40.639567334999875], [-74.02340085399995, 40.63968573199987], [-74.02331682999991, 40.63979778199991], [-74.02321780199998, 40.63990151499989], [-74.02310593199999, 40.639995325999934], [-74.02259016799994, 40.64043910599985], [-74.02246253899987, 40.640538898999836], [-74.02228790899991, 40.64063883599989], [-74.02222562199997, 40.64067782299986], [-74.02180903199998, 40.64095040899988], [-74.02172675999999, 40.641004941999924], [-74.02152001099994, 40.64115336799986], [-74.02120118199991, 40.64138693999995], [-74.0211362599999, 40.6414339679999], [-74.02111136799995, 40.64145200099989], [-74.02108051399998, 40.64149093699994], [-74.0210580239999, 40.641475982999935], [-74.02108520199992, 40.6414403539999], [-74.0211051969999, 40.641414135999895], [-74.02114153299986, 40.64136239499993], [-74.02126975099996, 40.64117350099991], [-74.021471331, 40.64082961299996], [-74.02159214600002, 40.64056920899986], [-74.02161540399997, 40.64050926299993], [-74.02166136899989, 40.64037438299987], [-74.0217092389999, 40.64020009799985], [-74.02172617299989, 40.640113319999934], [-74.02176040899997, 40.639824345999884], [-74.02174189000002, 40.63946629199994], [-74.02173880499991, 40.63944050699994], [-74.02170856799994, 40.63930090299994], [-74.02170098699993, 40.6392654599999], [-74.02167777199992, 40.63919125699987], [-74.02164685099997, 40.63909104799991], [-74.02160867799991, 40.63897677999994], [-74.02154646799997, 40.638830162999916], [-74.02150876399996, 40.63874907199992], [-74.02141038399992, 40.638584902999845], [-74.02129372099995, 40.63842824199984], [-74.02116029799993, 40.6382807979999], [-74.021114445, 40.638235535999904], [-74.02107776199989, 40.63820262199994], [-74.02084657599987, 40.637983350999924], [-74.020598444, 40.63777414199989], [-74.02032883799993, 40.63757296899991], [-74.02002934999997, 40.6373793659999], [-74.01976495999995, 40.63721743499991], [-74.01962150999994, 40.63713106399992], [-74.01955410699993, 40.637091226999914], [-74.01899546799997, 40.63675410399992], [-74.01799939199995, 40.636157921999875], [-74.01755212599994, 40.63588725099987], [-74.01750323299989, 40.635856163999904], [-74.01727269499997, 40.63569567599991], [-74.01705196399995, 40.63552755399995], [-74.016835401, 40.63535542899995], [-74.01667603, 40.63518670499994], [-74.01660869499997, 40.635118833999876], [-74.01650516099988, 40.63499150699987], [-74.01646522999985, 40.63494240699986], [-74.01633863299998, 40.63475616099995], [-74.01626768999998, 40.634628405999855], [-74.01623055699994, 40.6345615299999], [-74.0161423149999, 40.634360157999915], [-74.01607491199997, 40.63415382299994], [-74.01602896399994, 40.633944430999854], [-74.016004681, 40.63373391199987], [-74.01600191100002, 40.63352419099988], [-74.01602010100001, 40.63331711399997], [-74.01591473499991, 40.633365548999905], [-74.01560204999993, 40.63351697799989], [-74.01514630699995, 40.63322193799993], [-74.01542409800001, 40.63251313899987], [-74.01574804399993, 40.63176426999991], [-74.01604820399999, 40.631032888999876], [-74.01634322299998, 40.63028559599991], [-74.01664336499988, 40.62957607899994], [-74.0169264479999, 40.62883563899991], [-74.01696955099997, 40.628728785999925], [-74.01722288099991, 40.62810068899994], [-74.01784193399993, 40.626590693999944], [-74.0187518069999, 40.62431751099996], [-74.0189099719999, 40.62408698799987], [-74.01907578299992, 40.62361467399988], [-74.01938702899992, 40.62289181899993], [-74.01970491599991, 40.6221849649999], [-74.02008667300001, 40.62242095199986], [-74.02062330999992, 40.62180625199994], [-74.02090368299993, 40.62152616599993], [-74.02117792099988, 40.621252195999865], [-74.02167353899998, 40.62073479999989], [-74.02172925499997, 40.6206794519999], [-74.02226919899991, 40.620109624999934], [-74.02287598699992, 40.61946614799993], [-74.02363350499998, 40.61853764799987], [-74.02387367099986, 40.618259319999865], [-74.02462461299996, 40.61738902399985], [-74.02525327199992, 40.616678457999946], [-74.02574989099996, 40.616058222999904], [-74.02607013199996, 40.615486666999885], [-74.02624732399988, 40.61510478099994], [-74.02651463099991, 40.61467032699992], [-74.02736880500001, 40.61323451599991], [-74.02754297299998, 40.61293718299988], [-74.02653228199998, 40.61242537099991], [-74.02637609099996, 40.61245060299993], [-74.02710660599995, 40.61024395599991], [-74.02497932499995, 40.609741974999906], [-74.02478376299999, 40.60969962899984], [-74.0245785549999, 40.610223301999916], [-74.02459593799989, 40.61010922499989], [-74.02459441799985, 40.60999204099988], [-74.02457307699996, 40.6098745279999], [-74.02453195999996, 40.60975959499989], [-74.02447207599994, 40.60965008299994], [-74.02439534399993, 40.60954853399993], [-74.02430437599992, 40.609456968999936], [-74.02150602899994, 40.60782209899992], [-74.02247769499994, 40.60667421599987], [-74.02078541, 40.605673685999875], [-74.02122508399991, 40.604738061999974], [-74.02139467299999, 40.60449399799991], [-74.02129949199998, 40.60442832299987], [-74.0205358359999, 40.60419641799992], [-74.01924239499986, 40.60377850099991], [-74.019189715, 40.60376183999993], [-74.01912316599994, 40.60373976999991], [-74.01909859999992, 40.60373226399985], [-74.01869135799996, 40.60360792399991], [-74.01871382299996, 40.60358207499993], [-74.01917685099995, 40.603154421999875], [-74.01918088999996, 40.60314959499995], [-74.01923150399992, 40.603102173999915], [-74.01939343599999, 40.60291176099992], [-74.01942760861903, 40.602871578425265], [-74.01974772844801, 40.60308480926381], [-74.02010437562546, 40.603259552473915], [-74.02024692223878, 40.60332431782445], [-74.0203938499666, 40.60338312118084], [-74.02054472997047, 40.603435790909074], [-74.02069912187268, 40.60348217327761], [-74.0208565750425, 40.60352213290627], [-74.0210166299108, 40.60355555316099], [-74.0211788193127, 40.60358233649454], [-74.02134266985102, 40.603602404731895], [-74.02150770327854, 40.603615699297556], [-74.02514668248197, 40.604083124229994], [-74.02930882862765, 40.604599064564454], [-74.03005146547684, 40.604759173811956], [-74.03088511216566, 40.60518456375306], [-74.03101612271274, 40.60525141411518], [-74.03143471659999, 40.60552946681069], [-74.03179633794443, 40.60578027459593], [-74.03186039997433, 40.60581118472868], [-74.0321315955131, 40.605901836264245], [-74.0327068147988, 40.606331066038074], [-74.03359224082001, 40.60708260743882], [-74.03378350679236, 40.607254350388594], [-74.03427280539496, 40.60771747923557], [-74.034355360413, 40.60782297472988], [-74.03443004631626, 40.6079318233239], [-74.03449662936796, 40.60804368442151], [-74.03453929685496, 40.60814055535553], [-74.03459014036846, 40.608235106410824], [-74.03464894466927, 40.60832693725557], [-74.0347154608099, 40.60841565907369], [-74.0347894071866, 40.6085008962114], [-74.03487047073257, 40.608582287767895], [-74.03495830824197, 40.608659489123674], [-74.03505254782415, 40.60873217339959], [-74.03515279047687, 40.608800032841515], [-74.03525861177654, 40.60886278012296], [-74.03530371251365, 40.608888235269525], [-74.03535170243764, 40.60891041067642], [-74.03540217219016, 40.60892911718412], [-74.03547715437021, 40.60894504883701], [-74.03554940984297, 40.608967117033416], [-74.03561807682696, 40.608995058569896], [-74.03568233633708, 40.60902854019212], [-74.03574142195207, 40.60906716256955], [-74.03579462895571, 40.609110465057874], [-74.03603977777905, 40.60929563900184], [-74.03608058166901, 40.6093468863874], [-74.03611335501427, 40.60939472941205], [-74.0363093104175, 40.60980339016717], [-74.03634220893493, 40.609963852315985], [-74.0363839712695, 40.61012311228578], [-74.03643452294632, 40.610280885828004], [-74.03649377380209, 40.61043689134577], [-74.03656161814445, 40.61059085039554], [-74.03663793494069, 40.61074248818415], [-74.03672258803354, 40.61089153406034], [-74.03681542638358, 40.61103772199684], [-74.03691628433882, 40.6111807910655], [-74.03702498192945, 40.61132048590398], [-74.0371413251896, 40.611456557170285], [-74.03726510650235, 40.61158876198925], [-74.03739610497087, 40.611716864385166], [-74.03753408681214, 40.6118406357037], [-74.03873050931418, 40.612943761342144], [-74.03892016103417, 40.613141671586604], [-74.03920557647402, 40.61343951273367], [-74.03944150515275, 40.613775944191865], [-74.03982365089743, 40.6143273886638], [-74.04023427278453, 40.615028256649374], [-74.04038132338673, 40.61534178232843], [-74.04047612021522, 40.615625263664015], [-74.04072760517815, 40.617050416395266], [-74.04086140769245, 40.617630765787595], [-74.04101750654688, 40.61830780552467], [-74.0411277728829, 40.618786056113024], [-74.04137522891591, 40.619993552965234], [-74.04143385995191, 40.62008328218532], [-74.04148540020998, 40.6201755043379], [-74.04152966802239, 40.62026989441518], [-74.04156650734973, 40.62036611976727], [-74.04159578832969, 40.6204638412752], [-74.04161740773601, 40.6205627145458], [-74.04162219761896, 40.62066875832983], [-74.0416233689152, 40.620774861150096], [-74.04161785575602, 40.6209422590333], [-74.04160333814035, 40.621109343824465], [-74.04157984042503, 40.62127583441747], [-74.0415996813113, 40.62146631518381], [-74.04167119741669, 40.62209180591905], [-74.0417493234626, 40.62245699566951], [-74.04169490179855, 40.62280245622834], [-74.04170832517198, 40.62324493419511], [-74.04171015812464, 40.6233053206118], [-74.0417579401569, 40.6234283209218], [-74.0417987682504, 40.62355276698131], [-74.04183256751773, 40.62367843064929], [-74.04185927595628, 40.623805081551446], [-74.04187884456147, 40.623932487501996], [-74.04189123741789, 40.62406041492981], [-74.04189643176426, 40.624188629306566], [-74.04189513118017, 40.62429902703854], [-74.0418884929145, 40.62440931329303], [-74.04187652595625, 40.62451933829385], [-74.0417566930148, 40.62505506019891], [-74.04168321534894, 40.6253835421745], [-74.04164207726816, 40.626009357924765], [-74.04156624733882, 40.62674359415526], [-74.04142918710238, 40.62807066515698], [-74.04141650353655, 40.628149960545464], [-74.04130293385225, 40.62886001369012], [-74.04127701609347, 40.62902206057], [-74.04109062117575, 40.62972570042867], [-74.04110307519313, 40.629818740500966], [-74.04110655512133, 40.62991222556074], [-74.04110104224472, 40.63000565384534], [-74.04108656611493, 40.63009852389527], [-74.04106320439342, 40.630190337245], [-74.04103108243446, 40.63028060109854], [-74.04099037261425, 40.63036883097465], [-74.04094129340537, 40.630454553307736], [-74.04088410820547, 40.630537307988796], [-74.04081912392392, 40.63061665083624], [-74.04074668933455, 40.630692155979894], [-74.04065885393268, 40.63088875293372], [-74.04028911535313, 40.63171630457623], [-74.04015481009853, 40.632016901563], [-74.03991790625045, 40.632534443707065], [-74.03929388690396, 40.63388635429643], [-74.03843230568432, 40.6356851891421], [-74.03816058598187, 40.63624398058439], [-74.03783376084706, 40.636916083833164], [-74.03745424937436, 40.637696515307255], [-74.03708957847103, 40.63844640994759], [-74.03680662048858, 40.63898422154557], [-74.03876297920752, 40.639588258980524], [-74.0386880469073, 40.63973825602953], [-74.03672484374877, 40.63914119020668], [-74.03660641448076, 40.63936301858892], [-74.03647949179155, 40.639582104484745], [-74.0363441848906, 40.63979825924039], [-74.03620061021064, 40.64001129672453], [-74.03608560952524, 40.64017159149988], [-74.03596592750286, 40.64032988534885], [-74.0358416240794, 40.64048609895574], [-74.03571276150724, 40.64064015404647], [-74.03597990789547, 40.640940050544415], [-74.03634712296504, 40.64114331106443], [-74.03673442685152, 40.64145842171883], [-74.03675181360583, 40.64147755284508], [-74.03676491282464, 40.64149857295399], [-74.03677338009166, 40.64152092937311], [-74.03677699277581, 40.64154403429451], [-74.03677565588535, 40.641567280229424], [-74.03676940456552, 40.641590055980984], [-74.03675840317513, 40.641611762713936], [-74.03674294096601, 40.64163182969993], [-74.03672342447773, 40.641649729323426], [-74.03670036684926, 40.64166499095433], [-74.03445193181358, 40.64382678668235], [-74.034538297394, 40.64387638151011], [-74.03407329297129, 40.64431393298185]]]}}, {\"id\": \"14\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 15, \"Shape_Leng\": 0.14433622262, \"Shape_Area\": 0.000925219395547, \"zone\": \"Bay Terrace/Fort Totten\", \"LocationID\": 15, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7774039129087, 40.79659824126783], [-73.77738937457792, 40.79659450183191], [-73.77737533353579, 40.79659063854807], [-73.77736151739849, 40.79658715463961], [-73.77734722706326, 40.79658341668249], [-73.77732802081982, 40.796579507397446], [-73.77730831725226, 40.79657527248403], [-73.77729522076002, 40.796572608989926], [-73.77727883366029, 40.79656898558112], [-73.77726240942357, 40.79656565694303], [-73.77724580769197, 40.79656247270167], [-73.77722588125384, 40.79655751782156], [-73.77720481055027, 40.796551497944655], [-73.77718793343199, 40.79654536873484], [-73.77717293031019, 40.796538703545885], [-73.77716048389526, 40.79653321245801], [-73.77714079232376, 40.79652546721037], [-73.77712309051303, 40.79651796670836], [-73.77710551097655, 40.7965129900437], [-73.77709075818875, 40.79650944934304], [-73.77707133827573, 40.79650570143153], [-73.77705491487234, 40.79650213169628], [-73.77703946639896, 40.79649759822766], [-73.77702305739517, 40.79649361355523], [-73.7770085428856, 40.79648968581853], [-73.77699143346936, 40.79648579025017], [-73.77697245275725, 40.796481691870845], [-73.77695981862223, 40.796478812287695], [-73.77694225105256, 40.79647388939687], [-73.77692817826858, 40.79646925152675], [-73.77691718770845, 40.79646388133537], [-73.77690167683939, 40.79645679128878], [-73.77688598311, 40.79645124949161], [-73.77687050489574, 40.7964452046955], [-73.77685666384485, 40.79643845058072], [-73.77684188553589, 40.796428327890354], [-73.7768261853407, 40.79641761673879], [-73.77681399145276, 40.79641092058888], [-73.77679777685923, 40.796401560368025], [-73.77678418987156, 40.79639332999943], [-73.77676727795311, 40.796383797212805], [-73.7767499357965, 40.79637483937548], [-73.77673442631956, 40.79636754291753], [-73.77672082896045, 40.796362212052244], [-73.7767037306246, 40.79635529078489], [-73.77668638249598, 40.79634851226521], [-73.77667185244594, 40.79634213556841], [-73.77665423802641, 40.79632997125392], [-73.77663779999116, 40.79632000731809], [-73.77661855053992, 40.796307921749644], [-73.7766030466047, 40.79629858867732], [-73.7765878071623, 40.79628834762485], [-73.77657209394047, 40.79627832142478], [-73.77655799331602, 40.796270964516765], [-73.77654790396049, 40.796265729889726], [-73.77653362678076, 40.796258435290916], [-73.77651555681871, 40.79625117840095], [-73.77650408698263, 40.79624748249545], [-73.7764911641103, 40.79624283809], [-73.77647569741559, 40.79623680147143], [-73.77645834823085, 40.79622995975812], [-73.77644100212493, 40.79622259603883], [-73.77642436508934, 40.796215783175704], [-73.77640490129258, 40.796207154378905], [-73.7763880218685, 40.79619867499949], [-73.7763743933529, 40.796192138016664], [-73.77635820000582, 40.79618370553079], [-73.77634039397623, 40.79617552187176], [-73.77632679821677, 40.79616986713598], [-73.77631272968648, 40.796163995205845], [-73.77629750991538, 40.79615841856499], [-73.77628271935279, 40.79615224820701], [-73.77626443652422, 40.79614465698604], [-73.77624895989177, 40.7961381795581], [-73.77623536554736, 40.796132174015895], [-73.77621823846742, 40.79612308148995], [-73.77620134180472, 40.796112071409446], [-73.77618988034116, 40.79610268464799], [-73.77617885953198, 40.79609605336272], [-73.77616429083893, 40.79608723584757], [-73.77615232522406, 40.796079558706055], [-73.77613963747997, 40.796071546282526], [-73.77612628144364, 40.79606147953481], [-73.77611429231715, 40.79605030876701], [-73.77610276926637, 40.79603774241092], [-73.7760912856223, 40.79602781432162], [-73.77607828619544, 40.79601744252682], [-73.77606737098047, 40.79600734432826], [-73.77605582008007, 40.79599598443177], [-73.77604725794872, 40.7959842790402], [-73.77603866568093, 40.795973997053494], [-73.7760293623414, 40.79596769409022], [-73.77601318102688, 40.795958990355096], [-73.77599720655718, 40.7959491523693], [-73.77598539923815, 40.795940106332054], [-73.77597432156729, 40.79592555967183], [-73.77596822218737, 40.79591418406017], [-73.775953774968, 40.79590446596955], [-73.77594061502012, 40.79589582304519], [-73.77593061168318, 40.79588608661824], [-73.7759196649338, 40.79587508806209], [-73.77590880304822, 40.79586356816308], [-73.77589905470377, 40.79585168974544], [-73.77588612311689, 40.79583893065471], [-73.77587233417897, 40.79582704437839], [-73.77586326528844, 40.79581803112735], [-73.77585239587621, 40.795804889561275], [-73.77584465498843, 40.7957953289049], [-73.77584290844239, 40.79579317364331], [-73.77583110101452, 40.79578096671862], [-73.77582084085186, 40.79577013198782], [-73.77581022054922, 40.79576039451781], [-73.77579847271905, 40.79575117837077], [-73.77578634075365, 40.79574036629749], [-73.77577227211225, 40.795727434775905], [-73.77576359025652, 40.79571929625015], [-73.775752639847, 40.79570900797877], [-73.77574126696302, 40.7956980353803], [-73.77573108370488, 40.79568898376489], [-73.7757177588325, 40.795676774545896], [-73.7757093058664, 40.79566737432418], [-73.77569673601043, 40.79565601257565], [-73.77568453628999, 40.795644318672885], [-73.77567574597047, 40.79563664699041], [-73.77566926949147, 40.79562815206704], [-73.7756646012886, 40.79562095731017], [-73.77564699764488, 40.795612574581384], [-73.77563103732929, 40.7956017825139], [-73.77562134647184, 40.79559432601978], [-73.77561001797245, 40.79558066152715], [-73.77560040629238, 40.79557070955792], [-73.77558981591773, 40.795562837354396], [-73.77557656568186, 40.7955529590464], [-73.77556708750666, 40.79554558354153], [-73.77555882387453, 40.79553299717454], [-73.7755552874506, 40.795520364561085], [-73.77554058341056, 40.795513177563215], [-73.77553146710926, 40.79550733327298], [-73.77553345786804, 40.795504131884314], [-73.77553195267372, 40.79550061678085], [-73.77552296493784, 40.795488351655145], [-73.77551457646572, 40.795481086489495], [-73.77550330387254, 40.79547170823503], [-73.77548828146153, 40.795460622124196], [-73.77547684312017, 40.79545790670384], [-73.77546627197862, 40.7954446031817], [-73.77546441562504, 40.79543645045462], [-73.77545298499133, 40.795421191071874], [-73.77543937631013, 40.79540878308427], [-73.77542669336691, 40.795399672125725], [-73.77541469117227, 40.79539218363637], [-73.77540246897253, 40.79538332670378], [-73.7753918834261, 40.795374300575375], [-73.77538072181515, 40.795363661057834], [-73.77536976075994, 40.79534963774722], [-73.77535993733348, 40.79533935213663], [-73.77534953272698, 40.79532890406519], [-73.77533948316074, 40.79531880798594], [-73.7753293011994, 40.79530940553838], [-73.77532069352814, 40.79530054575829], [-73.77531105858056, 40.795290458693394], [-73.77530079651218, 40.79527997470998], [-73.77528735896209, 40.795269521222934], [-73.77527655563807, 40.79526187255653], [-73.77526177777118, 40.79525176725157], [-73.77524930736936, 40.79524285604281], [-73.77524109301233, 40.79523666218698], [-73.77522841161719, 40.79522711044956], [-73.77521822784352, 40.79521805761426], [-73.77520778681627, 40.7952081584399], [-73.77519630388718, 40.79519770875406], [-73.77518559691597, 40.795189537754766], [-73.77517458092494, 40.79518135827802], [-73.77515952443582, 40.79516969392657], [-73.7751495765414, 40.79516116472738], [-73.77513845803097, 40.79515194873436], [-73.77512785665888, 40.79514377894077], [-73.77511846078075, 40.79513667428005], [-73.77510333604457, 40.795124272005545], [-73.77509364674852, 40.79511613028997], [-73.77507955910173, 40.79510531432475], [-73.77506900584751, 40.79509699970999], [-73.77505736680486, 40.79508742368608], [-73.77504829571028, 40.79507873368923], [-73.77491790665806, 40.79499012265496], [-73.77479353901879, 40.79489664607736], [-73.77467550675657, 40.79479853995565], [-73.77456410783664, 40.79469605197661], [-73.77445962347299, 40.79458944088831], [-73.77443578887555, 40.79456347302087], [-73.77437784429105, 40.7944951882745], [-73.77432664447697, 40.79442385815788], [-73.77428246509835, 40.794349866750565], [-73.7742455440165, 40.79427361246051], [-73.77421608000793, 40.7941955058792], [-73.7742044616393, 40.7941654454196], [-73.77418705214467, 40.79413704580953], [-73.7741642440045, 40.79411094727948], [-73.77413655140263, 40.794087738184714], [-73.774117023877, 40.79408403853672], [-73.77409859116338, 40.7940778992209], [-73.77408177679733, 40.79406949460936], [-73.77406705834852, 40.794059063414146], [-73.77405485385675, 40.794046901907656], [-73.77404550995897, 40.79403335550693], [-73.7740392920429, 40.79401880896351], [-73.77403637671088, 40.79400367543496], [-73.77403684676305, 40.79398838475068], [-73.77404068884668, 40.793973371203144], [-73.7740477938349, 40.79395906121335], [-73.77405795992715, 40.79394586121933], [-73.77403999017578, 40.79385679444769], [-73.7740182219482, 40.79376821845375], [-73.77398067975606, 40.793643014950185], [-73.77393555059898, 40.7935192666697], [-73.77388293130392, 40.79339723900761], [-73.77382293476086, 40.79327719366772], [-73.77375568967857, 40.79315938810081], [-73.77374905422273, 40.7931207896111], [-73.77374710611448, 40.79308189124953], [-73.77374986177655, 40.793043021100154], [-73.77368083470303, 40.79286575659477], [-73.77363248188914, 40.79274975621505], [-73.77356729524408, 40.79266382842165], [-73.77349649263613, 40.792580507019025], [-73.77342025408298, 40.792500003830064], [-73.77337545326392, 40.79245990149119], [-73.77333601221568, 40.79241666960013], [-73.77330230698017, 40.792370720353105], [-73.77327465890835, 40.792322491854684], [-73.77320467172578, 40.79224253223184], [-73.77307986542115, 40.7921446089333], [-73.77287743738252, 40.791997693320845], [-73.77286059684761, 40.79197709653282], [-73.77277910650301, 40.79189753714522], [-73.77272012268155, 40.79184435699735], [-73.77266865746701, 40.79178684357741], [-73.77260041750584, 40.79172620880269], [-73.77252757063758, 40.79166875332356], [-73.77246875261999, 40.79162818020105], [-73.77241379404522, 40.791584593294274], [-73.77236296162013, 40.79153820413016], [-73.7723264817354, 40.79148082462651], [-73.77229688964472, 40.79142119847368], [-73.7722744217891, 40.791359802123246], [-73.77225925768093, 40.79129712617071], [-73.77225151846997, 40.79123367143485], [-73.77225126597583, 40.791169944956124], [-73.77225850219455, 40.791106455944906], [-73.77227316928341, 40.79104371171239], [-73.77227075371162, 40.790944463780626], [-73.77226051428751, 40.790845504394674], [-73.77224248787607, 40.79074718949841], [-73.77223841615907, 40.79065920649175], [-73.77224206821533, 40.79057121284437], [-73.77225342777682, 40.790483599787535], [-73.77223303752142, 40.79040534229604], [-73.77220607891066, 40.790328241164374], [-73.7721726631949, 40.7902526145021], [-73.77213292826468, 40.79017877433443], [-73.77210255348926, 40.79013000015939], [-73.77202595346799, 40.790063692567394], [-73.77194403848256, 40.790001162427444], [-73.77189574144926, 40.78998417732687], [-73.77185143898816, 40.789961768318506], [-73.77181217739995, 40.78993446464287], [-73.7717788839305, 40.78990291113913], [-73.77175234487125, 40.789867853014314], [-73.77173680635669, 40.789825494472346], [-73.77173042355776, 40.78978178881643], [-73.771733362281, 40.789737871514184], [-73.77174554616062, 40.78969488352967], [-73.77171860487624, 40.78966141348849], [-73.77166811281884, 40.78959191109706], [-73.77144337542126, 40.78934263237258], [-73.7711276990068, 40.78901277421275], [-73.77087816042668, 40.78877673022536], [-73.77063124537418, 40.78846290603716], [-73.77064195008725, 40.78837887603174], [-73.77066138329643, 40.78829575516882], [-73.77068942271313, 40.78821406636863], [-73.77072589190864, 40.78813432354018], [-73.77077056142382, 40.788057028348895], [-73.77082315021427, 40.78798266705989], [-73.7711714496876, 40.787763720001394], [-73.77124371742936, 40.78773161177086], [-73.77131348065242, 40.7876964488439], [-73.77138051609693, 40.787658343752696], [-73.77149672842718, 40.787590496346304], [-73.77161578634065, 40.78752556176706], [-73.77173756350668, 40.787463608911835], [-73.77190205680833, 40.7873865959972], [-73.77199320470947, 40.78735611430665], [-73.77208769981934, 40.78733223636572], [-73.77218471602598, 40.7873151709236], [-73.77228340517985, 40.787305067172205], [-73.77379096552599, 40.786828957833734], [-73.77396728847471, 40.786781774218426], [-73.77412967826919, 40.78676409915586], [-73.77426836058959, 40.78675665865473], [-73.7743460921796, 40.78677223413504], [-73.77439231197853, 40.78681496092917], [-73.77438976901033, 40.78685714529808], [-73.77426439520684, 40.78692887354648], [-73.77420002662137, 40.78696216313965], [-73.77419980868142, 40.78702642524989], [-73.7742301148687, 40.787065041675966], [-73.7743241010546, 40.787082253572294], [-73.77444617141909, 40.78717856502961], [-73.77501922678299, 40.78739447438411], [-73.77628944855437, 40.7885136481994], [-73.77633728818715, 40.78848722575376], [-73.77587233688504, 40.78802031762283], [-73.7750481043881, 40.78719259407269], [-73.77486248240422, 40.78707913013461], [-73.774806953694, 40.78697068727885], [-73.77474124745507, 40.786927235879425], [-73.7746308254633, 40.78691941868919], [-73.7745858466651, 40.78687809258887], [-73.77456236644174, 40.78682149734369], [-73.77451195192242, 40.78672886084857], [-73.77445462503042, 40.7866799092223], [-73.77435661274104, 40.78666172510186], [-73.77426466694423, 40.78665122914519], [-73.7740960989511, 40.78629656530393], [-73.77407638297778, 40.78591728401922], [-73.7726172080581, 40.7841910491489], [-73.7720920025976, 40.78356969320761], [-73.77160731979487, 40.7829962651855], [-73.77060974454855, 40.782079793940724], [-73.76981014523498, 40.78129770845182], [-73.76930405710117, 40.78078195798143], [-73.76908807714565, 40.780532832334075], [-73.76902445493923, 40.780450591996654], [-73.76858058549459, 40.779876824912144], [-73.76833795673872, 40.779634872200724], [-73.76827333570864, 40.77962028411314], [-73.76815915444048, 40.77963595970004], [-73.76813458838144, 40.77958964044201], [-73.76823190254096, 40.77950163764268], [-73.76820055460033, 40.77944417919097], [-73.76716193687588, 40.779782875660686], [-73.76720729367467, 40.779866828611894], [-73.76699783535086, 40.77993147052596], [-73.7670734335662, 40.78007042899436], [-73.76702392346498, 40.78008623415827], [-73.7669275026482, 40.779918316052], [-73.7669580014477, 40.77989958115397], [-73.76688994703834, 40.779779433943396], [-73.76677570373936, 40.77981245869834], [-73.76675868631202, 40.77978350558992], [-73.76686912971435, 40.77974902775042], [-73.76686345436596, 40.779740341271875], [-73.76708815912728, 40.77966705475085], [-73.767135449099, 40.77974233714537], [-73.76816173051576, 40.77939682124199], [-73.76807788861774, 40.77925206154374], [-73.76782207615058, 40.77881037615661], [-73.76796073100003, 40.77877600899994], [-73.76807192199989, 40.77874844899989], [-73.76825889099999, 40.778702104999915], [-73.76833788999983, 40.7786825249999], [-73.7685730499999, 40.77910542899991], [-73.76864150099995, 40.77908686699987], [-73.76877614499993, 40.77904284199991], [-73.76934982499989, 40.77887476899986], [-73.77113180399985, 40.77834080999992], [-73.7720360819999, 40.778069802999866], [-73.772903019, 40.77780428099986], [-73.7737825709999, 40.77754746799989], [-73.77465548700002, 40.777285608999925], [-73.77553732199992, 40.777025597999916], [-73.77572440800002, 40.777900433999896], [-73.77583958299994, 40.778612371999884], [-73.77678363299988, 40.778409583999874], [-73.77769354899995, 40.77816041699992], [-73.77858771199995, 40.77791239199995], [-73.77947656099995, 40.777664169999895], [-73.78036180399994, 40.77738704699991], [-73.78279772899998, 40.77666603599993], [-73.78407645399997, 40.77628882299991], [-73.78440918799984, 40.77619021899989], [-73.78448330699995, 40.77616825899986], [-73.78459339299995, 40.77614122499991], [-73.78491489199986, 40.776040509999895], [-73.78569433800004, 40.775812774999885], [-73.78626498299998, 40.77564607999991], [-73.78655073399982, 40.77555774699991], [-73.78740991599994, 40.77531789199992], [-73.78826972999987, 40.77507535699988], [-73.78893975799984, 40.77490059199992], [-73.78913873699987, 40.774848689999935], [-73.7898420409999, 40.7746873259999], [-73.79034971799999, 40.774573093999926], [-73.79163526099994, 40.774238655999895], [-73.7922679529999, 40.77399004199987], [-73.79271259399978, 40.773787526999875], [-73.7936523029999, 40.77336043899989], [-73.79499129399991, 40.77314334799993], [-73.79557358499984, 40.77304924499992], [-73.79565702099983, 40.773036085999905], [-73.79571707399992, 40.77306964299988], [-73.79652490499979, 40.7735032779999], [-73.79672685099997, 40.77360922899989], [-73.79724339799986, 40.77389389699988], [-73.79730005599994, 40.773932285999884], [-73.79741514499983, 40.774010264999916], [-73.79741522800005, 40.774010318999885], [-73.79741530999986, 40.77401038199986], [-73.79757522899995, 40.77413684199995], [-73.79772190099987, 40.774272372999846], [-73.79785400499989, 40.77441566899992], [-73.79823295499997, 40.77484830499985], [-73.79853009399987, 40.775190728999924], [-73.7986519199999, 40.77520055899988], [-73.800989741, 40.77538911599989], [-73.8037902289999, 40.77561011199992], [-73.80369287499998, 40.77631968799989], [-73.8035964729998, 40.77703047199987], [-73.80346278599977, 40.778008173999865], [-73.80342253099997, 40.77833770799985], [-73.80332690999992, 40.778995479999935], [-73.8032260339999, 40.779702924999846], [-73.80320064599984, 40.77988363099988], [-73.80313481699993, 40.780415805999894], [-73.803066233, 40.78077013599987], [-73.8030331849998, 40.78094086999989], [-73.80324974699984, 40.781275478999916], [-73.80314563299987, 40.78132556799988], [-73.80299781999996, 40.78141047599991], [-73.80293830099997, 40.78183503399991], [-73.8028401069999, 40.78255003699992], [-73.80274225299986, 40.783253667999894], [-73.80254724699995, 40.78467334699992], [-73.80241070199989, 40.78581721099992], [-73.80242243499985, 40.78601897999991], [-73.80242791699989, 40.78606827399991], [-73.80215169199994, 40.78605649899994], [-73.80157320499991, 40.78603473299989], [-73.80067959399986, 40.786001101999894], [-73.79973844499978, 40.78595286999994], [-73.7992594699999, 40.785934836999935], [-73.79880021900001, 40.7859175479999], [-73.7978420549998, 40.78589425899987], [-73.79685935799988, 40.78599452199992], [-73.79582463099997, 40.7861305879999], [-73.79491977600001, 40.78633880799987], [-73.79454742299993, 40.78646433199993], [-73.79392031799995, 40.78667573999992], [-73.7933258969999, 40.78693204899985], [-73.79153624300002, 40.78775191299995], [-73.7906355999999, 40.78816951899994], [-73.79058298999988, 40.78819290999989], [-73.790653771, 40.788249452999864], [-73.79084786600005, 40.78840689799993], [-73.79133754899989, 40.78885774099988], [-73.79156371699993, 40.789103095999884], [-73.79179280401884, 40.789351616549254], [-73.79175254252796, 40.78938643776097], [-73.79162795617712, 40.7894861231687], [-73.79028202353197, 40.79031696339975], [-73.78988738668919, 40.79037193097968], [-73.78930907131766, 40.79038006955028], [-73.78883143352806, 40.790357187582224], [-73.78835429027176, 40.79032784777117], [-73.7879935835311, 40.79034968837414], [-73.78750199944683, 40.79030881800571], [-73.78568101301407, 40.79034067165998], [-73.7838384251435, 40.79064095467152], [-73.78317940975151, 40.79086646470491], [-73.78253080663518, 40.79127678943192], [-73.78252265694827, 40.79135222690715], [-73.78440669003494, 40.79167583933253], [-73.78439845368956, 40.79171891686168], [-73.78434153059591, 40.79174935379487], [-73.78227330183385, 40.791373715717924], [-73.78174708763098, 40.79113340664446], [-73.7817021394188, 40.79117525786651], [-73.78160545881975, 40.791217921223286], [-73.78143260460877, 40.79136512329363], [-73.78123362112055, 40.79148460246174], [-73.78107845970229, 40.79167732177907], [-73.78089784791108, 40.79219370535159], [-73.78077503139014, 40.79299070534052], [-73.78077892481954, 40.793189785863845], [-73.78085015490862, 40.793252479377415], [-73.78083949014304, 40.79327744017283], [-73.78158881545012, 40.793463541012535], [-73.78158781636597, 40.79344838542981], [-73.78158153246125, 40.793444386581825], [-73.78163864257793, 40.793309707768074], [-73.78168372830307, 40.7933217570108], [-73.78162662328144, 40.793454840007925], [-73.78160879379377, 40.793451615702814], [-73.78161607678727, 40.79347077130438], [-73.78166011555379, 40.79348202031304], [-73.7816421343347, 40.79352505337762], [-73.78082154869789, 40.79331942900066], [-73.78080808737927, 40.7933509327133], [-73.7807243506983, 40.793333618824185], [-73.78070025516259, 40.79364850590945], [-73.78079182072646, 40.79403591986439], [-73.78100539667301, 40.79438481862711], [-73.7813176840371, 40.79460466456844], [-73.78152857545292, 40.79472485577791], [-73.78181003762921, 40.79480185241986], [-73.78190385421033, 40.794901203894135], [-73.78258165591822, 40.79488335802936], [-73.78265517138459, 40.79490070120977], [-73.78284362044448, 40.79490185399306], [-73.78307475337424, 40.79491681769891], [-73.78310976963976, 40.79491446050581], [-73.78314000854334, 40.79489731287008], [-73.78317026275754, 40.79487586588125], [-73.78321522572556, 40.794825295774366], [-73.78324789295391, 40.79478855571322], [-73.78328948833689, 40.79475996026348], [-73.78331796684243, 40.794758356474155], [-73.78332856842805, 40.79476160260946], [-73.78332528563587, 40.79479224087093], [-73.7832965245875, 40.79482605743941], [-73.78328163720504, 40.79483570739623], [-73.78327094437915, 40.79486068745444], [-73.78327193181352, 40.794883269736395], [-73.78325271025878, 40.79492032976596], [-73.78318780792462, 40.794973434233306], [-73.78311558348871, 40.794993458539004], [-73.78305415481076, 40.79499109708873], [-73.78286814787282, 40.79498657863788], [-73.78236678802342, 40.794959614703636], [-73.78200326103361, 40.794975462503565], [-73.78182459908278, 40.79507728971345], [-73.78172625575411, 40.79542750602256], [-73.78165761051014, 40.79555862334205], [-73.78154150032744, 40.79565973731996], [-73.77989837558063, 40.79623809836545], [-73.7798030505206, 40.79620552893678], [-73.77944961456502, 40.79657623806387], [-73.77946869192893, 40.796608441296094], [-73.77943301516063, 40.79666153857653], [-73.7793624680482, 40.796685392534904], [-73.77928446350832, 40.79667469847259], [-73.7792728777499, 40.796637683818055], [-73.77871478762599, 40.796597360195996], [-73.77868968820113, 40.79660527281772], [-73.77868530361944, 40.79660919102858], [-73.77867108995639, 40.79661810557603], [-73.778653137277, 40.79662911153307], [-73.7786374737861, 40.79663550136388], [-73.77862394962096, 40.79664044721123], [-73.77860456300029, 40.79664792835134], [-73.77858658990885, 40.79665439498891], [-73.7785723534402, 40.796659555234854], [-73.77855506807938, 40.79666583438733], [-73.77853520981658, 40.79667268437904], [-73.77851818617542, 40.79667861323066], [-73.7784973595596, 40.7966842553393], [-73.77848101850053, 40.79668803312994], [-73.77846700036604, 40.796691437110304], [-73.77844831121799, 40.79669579605253], [-73.77843218488232, 40.79669950171049], [-73.77841209444438, 40.79670470445934], [-73.77839550549669, 40.7967080497132], [-73.77837960824762, 40.79671021607783], [-73.77836745440197, 40.7967125514675], [-73.77834924005742, 40.79671684831703], [-73.77833287754164, 40.796720113752656], [-73.77831089780186, 40.79672303249394], [-73.77829124888358, 40.79672406736736], [-73.77827066301263, 40.796725125731335], [-73.77825056887954, 40.79672772576945], [-73.77823373526176, 40.796730134564555], [-73.77822228238244, 40.79673194877572], [-73.7782013295322, 40.79673279115116], [-73.77818474615981, 40.796734920988136], [-73.77816768905346, 40.79673658938364], [-73.77815154791757, 40.79673738606178], [-73.77813539455161, 40.79673814820391], [-73.77811877759432, 40.79673924215685], [-73.7781028854632, 40.79673986809647], [-73.77808486122399, 40.79674003208208], [-73.77806662937581, 40.79673872730426], [-73.77805024184082, 40.7967354080633], [-73.77803334795568, 40.79673056788638], [-73.77801974867874, 40.79672615554589], [-73.77800593512394, 40.79672148212149], [-73.77799095770179, 40.796717914802265], [-73.77797479179517, 40.7967154979345], [-73.77795864859094, 40.79671321445842], [-73.77794084652679, 40.796711010206906], [-73.7779218631978, 40.79670765964538], [-73.7779050120194, 40.79670449378134], [-73.77788558987741, 40.79670125109234], [-73.77787108189364, 40.796698972430256], [-73.77785421688698, 40.79669651684017], [-73.77783898494677, 40.796694379510484], [-73.77782400143708, 40.796692838880894], [-73.77780760959772, 40.79669041987882], [-73.77778725197105, 40.79668689593743], [-73.77777223683373, 40.79668406680164], [-73.77775422709308, 40.79668002419901], [-73.77773780843233, 40.79667527435354], [-73.77772279891418, 40.79667030156635], [-73.77770874128566, 40.796664727353715], [-73.77769491559815, 40.79666019877806], [-73.77767875245776, 40.79665676298415], [-73.77766142646871, 40.79665392887425], [-73.77764012114096, 40.79665068251131], [-73.77762351135634, 40.79664963325345], [-73.77760947214593, 40.79664897543398], [-73.77759425962752, 40.79664828837734], [-73.77757812464012, 40.79664705122794], [-73.77756101083169, 40.79664471204753], [-73.77754158666977, 40.79664182009476], [-73.777521697539, 40.79664013375986], [-73.77750623891349, 40.79663881665584], [-73.77748973756474, 40.79663737943148], [-73.77747147920505, 40.796633202451844], [-73.77745764333974, 40.79662499915687], [-73.77744564507037, 40.79661646620007], [-73.77743367405546, 40.79661026575175], [-73.77742032900714, 40.79660400837061], [-73.7774039129087, 40.79659824126783]]]}}, {\"id\": \"15\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 16, \"Shape_Leng\": 0.141291873771, \"Shape_Area\": 0.000871889446182, \"zone\": \"Bayside\", \"LocationID\": 16, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7685730499999, 40.77910542899991], [-73.76833788999983, 40.7786825249999], [-73.76825889099999, 40.778702104999915], [-73.76807192199989, 40.77874844899989], [-73.76796073100003, 40.77877600899994], [-73.76782207615058, 40.77881037615661], [-73.76760454633784, 40.77829903466069], [-73.7676189407549, 40.77799469391587], [-73.76747817366723, 40.77765554086643], [-73.76713717312357, 40.776937753291556], [-73.76709113484746, 40.776844841109174], [-73.76700068069776, 40.77669521396545], [-73.76658827278406, 40.776172594165985], [-73.76632823258105, 40.775843054330814], [-73.76627787969173, 40.77574118632524], [-73.76614233293911, 40.77546695978642], [-73.76600398326563, 40.77506700693253], [-73.76572796903913, 40.7746033280921], [-73.76510436820205, 40.77379903872306], [-73.76469385424426, 40.773340474854514], [-73.76436067006205, 40.77302852084781], [-73.76417633114528, 40.77282999044271], [-73.76358456590272, 40.77235821590673], [-73.76223863110448, 40.77122066177051], [-73.7612630125983, 40.770395965074215], [-73.76085900055723, 40.77002264272332], [-73.76060524066253, 40.76976593345716], [-73.76052349507854, 40.76961682085475], [-73.76044056848673, 40.76948518693146], [-73.76001142296055, 40.76909583162673], [-73.75960078092824, 40.76872325671303], [-73.75937389428368, 40.76863638878568], [-73.75907352924408, 40.76845378633801], [-73.75893512093685, 40.768334328515934], [-73.75880776561003, 40.76818265272536], [-73.75883847871388, 40.768038287739046], [-73.75882204462802, 40.76794933440106], [-73.7588273600204, 40.767767523872216], [-73.75879456995813, 40.76765389115854], [-73.75873403546348, 40.76754922615335], [-73.75864264358073, 40.767357170035446], [-73.75842156550789, 40.767199067448146], [-73.75802387192715, 40.76707532904307], [-73.75787030567056, 40.76694642845211], [-73.75760683799241, 40.7669699190069], [-73.75752194375056, 40.76710675613373], [-73.7573902080387, 40.76716667235849], [-73.75734523174002, 40.76706980756656], [-73.7571843001946, 40.76695200038971], [-73.75703114820047, 40.766823254341304], [-73.75692940163003, 40.76679768260604], [-73.75665982851439, 40.76665032915951], [-73.75641084289745, 40.76651422276766], [-73.75640921140914, 40.766510148930536], [-73.75640836159586, 40.76650604089554], [-73.75640830611397, 40.766501872052565], [-73.75640903262868, 40.766497641202925], [-73.7564131832044, 40.766486492556076], [-73.75642003660344, 40.76647464752031], [-73.75644838018687, 40.76643884892769], [-73.75646019017901, 40.7664253664105], [-73.75649903977315, 40.76638619525873], [-73.75650884816422, 40.76637283532363], [-73.75651240090923, 40.76636639373759], [-73.75651468312392, 40.76636104961193], [-73.75651512522731, 40.766359997643185], [-73.7565155091402, 40.76635871017682], [-73.75651572652936, 40.76635746826089], [-73.75651577784062, 40.76635627055586], [-73.75651563993463, 40.76635511801813], [-73.75651534873262, 40.76635400904841], [-73.75651487825853, 40.766352937058144], [-73.75651424303084, 40.76635190861166], [-73.75651342915802, 40.76635092652746], [-73.75651244924389, 40.7663499786028], [-73.75651129245526, 40.76634907486635], [-73.75650846744547, 40.766347393967386], [-73.75650494248836, 40.76634587358723], [-73.75650071937817, 40.76634450501809], [-73.75649107536171, 40.766342377656294], [-73.75647876240242, 40.76634072126506], [-73.7564737783002, 40.76634035992999], [-73.75643582839564, 40.76633764971248], [-73.75640357156823, 40.76633574525942], [-73.75638758604588, 40.76633479308035], [-73.75634312226629, 40.76633520297428], [-73.75631567038226, 40.76633788196008], [-73.75628664411393, 40.76634340455195], [-73.75625559323588, 40.76635184870079], [-73.75622064503258, 40.76636370615331], [-73.75620578099215, 40.76637041143858], [-73.75619103618445, 40.76637963606818], [-73.75617702002063, 40.76639061121284], [-73.75616353456147, 40.76640233347391], [-73.75614489705069, 40.76641051622121], [-73.75602210333291, 40.76646442121023], [-73.75588294950307, 40.76641229346199], [-73.75547310439798, 40.76636558041649], [-73.7549367434127, 40.7660932521388], [-73.75425553419626, 40.765532892788315], [-73.75425235038242, 40.76552911246537], [-73.75410065488441, 40.765349353964524], [-73.75401961912938, 40.76530745082464], [-73.75397498244763, 40.76530695093847], [-73.75388234874703, 40.76534929289707], [-73.75385085906869, 40.76534212937308], [-73.75382270603563, 40.76533572114292], [-73.75377036482266, 40.76530463124142], [-73.75405049899983, 40.7651753319999], [-73.75435817399999, 40.76505094399988], [-73.75467572399998, 40.76493975999991], [-73.75500165499989, 40.76484240599992], [-73.75533439, 40.764759409999954], [-73.75567226899989, 40.76469114899985], [-73.75601361099996, 40.76463784199992], [-73.75635669699987, 40.764599557999915], [-73.7583941619999, 40.7644052469999], [-73.75850831399994, 40.76439435899995], [-73.75856122399995, 40.76438654599987], [-73.75862488399996, 40.764383148999855], [-73.75921689799992, 40.76432485599988], [-73.759097561, 40.76405669599989], [-73.7588643139999, 40.76327454099993], [-73.75859320499987, 40.764088229999906], [-73.75851382899985, 40.76410483099993], [-73.75844672099987, 40.76411514599987], [-73.75843838499995, 40.763928487999856], [-73.75844568799998, 40.76368152999987], [-73.75836296599996, 40.76349738299991], [-73.75823811499988, 40.763238765999944], [-73.75809857499982, 40.76298453899995], [-73.75794474699994, 40.7627354049999], [-73.75788761399987, 40.76265702199986], [-73.75784858699993, 40.76256679999993], [-73.75783301499989, 40.76246859299991], [-73.75784418199986, 40.76236794099986], [-73.75788220699995, 40.76227115299988], [-73.75794391899996, 40.76218407699988], [-73.75802364199994, 40.76211096099997], [-73.75811452099987, 40.76205389699988], [-73.75848075099995, 40.7619947369999], [-73.75800131799996, 40.76037986699988], [-73.76031788499999, 40.75990455099988], [-73.76125540599999, 40.75971470799989], [-73.76239510499991, 40.75949699799992], [-73.76199745999992, 40.75810219099993], [-73.76176692899985, 40.757278801999846], [-73.76185572299987, 40.756516974999926], [-73.76192713800003, 40.755772852999876], [-73.76189611099981, 40.755654002999904], [-73.76184343999986, 40.75553726899989], [-73.76176970599987, 40.755426020999884], [-73.76167669899993, 40.75532347699988], [-73.7615672369999, 40.75523235199987], [-73.76144497499985, 40.75515465699987], [-73.76128938199999, 40.7550522149999], [-73.76114556499998, 40.7549389489999], [-73.76101539999992, 40.754816091999864], [-73.76090047000001, 40.75468513599987], [-73.76080194199993, 40.754547729999956], [-73.76048623699985, 40.75383961699987], [-73.76018801899988, 40.75316049899985], [-73.76013367999985, 40.75302838899988], [-73.76013364499984, 40.75302828899988], [-73.76013358599997, 40.753028199999925], [-73.76005811999997, 40.75289976499995], [-73.76005800199988, 40.752899574999894], [-73.76005787299981, 40.752899403999926], [-73.75996192099983, 40.752777446999865], [-73.7599618029999, 40.75277730199993], [-73.75996168499987, 40.75277718499993], [-73.75984680199994, 40.75266435499989], [-73.7598466729999, 40.75266422799993], [-73.7598465069999, 40.75266410199989], [-73.75971521799991, 40.75256288999989], [-73.75971508799988, 40.7525627919999], [-73.75971494599995, 40.75256269999991], [-73.75957032200002, 40.75247497699995], [-73.75957016899991, 40.75247487799994], [-73.75957000299998, 40.752474795999895], [-73.75948272499986, 40.75243381299991], [-73.75910049199996, 40.752253820999954], [-73.75782746199997, 40.75075826999989], [-73.75737714099989, 40.750228101999895], [-73.75698991500002, 40.74957500999989], [-73.7568655229999, 40.749361488999966], [-73.75684127399991, 40.74932207699987], [-73.75714080500002, 40.74923725899987], [-73.75744314899997, 40.74915673299992], [-73.7577393699999, 40.7490634429999], [-73.75802805599993, 40.74895781599986], [-73.75805782299985, 40.74894194899988], [-73.75845857499999, 40.748763093999884], [-73.75885932599982, 40.748584227999885], [-73.75972553599988, 40.748188660999894], [-73.76073612099987, 40.74773738499994], [-73.76138355799993, 40.74752917099989], [-73.76145571899991, 40.747507951999864], [-73.76151846699999, 40.74749054799987], [-73.76240045799992, 40.74724325399988], [-73.76297777799992, 40.74708162199994], [-73.76414324400004, 40.74676404599995], [-73.76501623199981, 40.746522881999894], [-73.7658920239999, 40.74628062699993], [-73.76635520899988, 40.74615249699984], [-73.76681551899995, 40.7460300569999], [-73.76729011899992, 40.74590381399993], [-73.76772991399994, 40.74577622599993], [-73.76857701199982, 40.74552220399994], [-73.76937164099995, 40.745220308999876], [-73.770389824, 40.74473791399991], [-73.77137678500003, 40.74425103599991], [-73.77145218799996, 40.7442215229999], [-73.77153402299992, 40.744191625999854], [-73.77156671899986, 40.74417951599989], [-73.77162097999992, 40.744159731999915], [-73.7724146099999, 40.743884844999926], [-73.7728126769999, 40.74378924899988], [-73.77321534799998, 40.74370439599988], [-73.77362185199992, 40.74363046299994], [-73.77403143100003, 40.74356760099993], [-73.77444330499989, 40.74351589999993], [-73.77486382100001, 40.74345333599993], [-73.77551238399978, 40.74335684199993], [-73.77569182399996, 40.74333358899985], [-73.77573836999991, 40.74332564699993], [-73.77575892599995, 40.74336722699992], [-73.77585237799993, 40.74356469299995], [-73.77596627999999, 40.74381044799988], [-73.77616785699998, 40.744240022999904], [-73.77679196999986, 40.74554428899989], [-73.77753347599982, 40.74714834199988], [-73.77831180799991, 40.748745013999944], [-73.77906558899984, 40.75033257399991], [-73.77944066999987, 40.751122695999946], [-73.77980639299984, 40.75189500499989], [-73.78009357400003, 40.75250750399991], [-73.78052037799992, 40.753330742999886], [-73.78113322699984, 40.7540893259999], [-73.78165171799995, 40.754707752999956], [-73.78212461999979, 40.75533931199993], [-73.78248130799982, 40.75599745499987], [-73.78264852999988, 40.756331903999865], [-73.78281576399985, 40.75666634299991], [-73.78303191299983, 40.75708319099989], [-73.78316157699985, 40.75732618499991], [-73.78338071499984, 40.75775389699993], [-73.78346111099987, 40.757864063999904], [-73.78382076099993, 40.75853003399991], [-73.78407792999997, 40.75903964399989], [-73.78425940899996, 40.759382645999914], [-73.78428991899993, 40.7594402989999], [-73.78430628699978, 40.759483644999904], [-73.78447529299982, 40.75987586199995], [-73.78467742500004, 40.76025720799989], [-73.78486059299999, 40.760644362999905], [-73.78502434699996, 40.761036515999855], [-73.78516838299994, 40.76143281799993], [-73.785251708, 40.761756958999904], [-73.78527359199997, 40.761842105999925], [-73.78530175299996, 40.76197850399993], [-73.78535911599988, 40.7622563179999], [-73.78542241999988, 40.76251310799994], [-73.78546383099984, 40.76259192499988], [-73.78571993599994, 40.762921464999856], [-73.78595810599988, 40.76310224399986], [-73.787141317, 40.763707819999865], [-73.78733166699999, 40.76380781299986], [-73.78807523699999, 40.76419948999993], [-73.78822763099994, 40.76431439599985], [-73.78836856600003, 40.76443842899987], [-73.78849669799996, 40.764570552999885], [-73.7886108559999, 40.764709593999925], [-73.78871017999994, 40.76485425199996], [-73.78963649799978, 40.76614161099991], [-73.78990502999987, 40.7665115969999], [-73.79021960799982, 40.76695117499985], [-73.79059858199989, 40.76741868799998], [-73.79137684499982, 40.7683242969999], [-73.79149870499994, 40.76844739099989], [-73.79190204099989, 40.768936211999915], [-73.79235243699989, 40.76944963799987], [-73.79238243799978, 40.7694838389999], [-73.79317944599988, 40.77040317699989], [-73.79323711899993, 40.77046772099994], [-73.79394905799995, 40.7712673749999], [-73.79415923499985, 40.77151603799994], [-73.79480546099995, 40.77225687499992], [-73.794936368, 40.77240452999993], [-73.79507190299978, 40.77254212499992], [-73.7952155069999, 40.77267490699989], [-73.79565702099983, 40.773036085999905], [-73.79557358499984, 40.77304924499992], [-73.79499129399991, 40.77314334799993], [-73.7936523029999, 40.77336043899989], [-73.79271259399978, 40.773787526999875], [-73.7922679529999, 40.77399004199987], [-73.79163526099994, 40.774238655999895], [-73.79034971799999, 40.774573093999926], [-73.78984203999995, 40.77468732599991], [-73.78913873699987, 40.774848689999935], [-73.78893975799984, 40.77490059199992], [-73.78826972999987, 40.77507535699988], [-73.7874099169999, 40.77531789199993], [-73.78655073399982, 40.77555774699991], [-73.78626498299998, 40.77564607999991], [-73.78569433899995, 40.77581277499992], [-73.78491489299977, 40.776040509999845], [-73.78459339399987, 40.776141224999925], [-73.78448330699995, 40.77616825899986], [-73.78440918799984, 40.77619021899989], [-73.78407645399997, 40.77628882299991], [-73.78279772899998, 40.77666603599993], [-73.78036180399994, 40.77738704699991], [-73.77947656099995, 40.777664169999895], [-73.77858771199995, 40.77791239199995], [-73.77769354999994, 40.77816041699992], [-73.77678363299991, 40.77840958299987], [-73.77583958299994, 40.778612371999884], [-73.77572440800002, 40.777900433999896], [-73.77553732199992, 40.777025597999916], [-73.77465548700002, 40.777285608999925], [-73.7737825709999, 40.77754746799989], [-73.772903019, 40.77780428099986], [-73.7720360819999, 40.778069802999866], [-73.77113180399985, 40.77834080999992], [-73.76934982499989, 40.77887476899986], [-73.76877614499993, 40.77904284199991], [-73.76864150099995, 40.77908686699987], [-73.7685730499999, 40.77910542899991]]]}}, {\"id\": \"16\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 17, \"Shape_Leng\": 0.093522632948, \"Shape_Area\": 0.000322957654799, \"zone\": \"Bedford\", \"LocationID\": 17, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94306406899986, 40.701424434999886], [-73.94220058700002, 40.700890666999896], [-73.94193078899983, 40.70072523399989], [-73.94438788599984, 40.70042452299993], [-73.94424286099989, 40.69969927999991], [-73.94409591299996, 40.698972954999924], [-73.94394947200006, 40.698221278999895], [-73.94391750199989, 40.69805620199984], [-73.94380383199999, 40.697469264999945], [-73.94378455600005, 40.69736972899993], [-73.9437430669999, 40.69715549899995], [-73.9437245359998, 40.6970598119999], [-73.94368427099982, 40.69685190199992], [-73.9436380689999, 40.696613317999905], [-73.94362121299996, 40.69652627899993], [-73.94355634200002, 40.696191282999926], [-73.94354024099975, 40.696108141999964], [-73.94352527499989, 40.6960308549999], [-73.943388021, 40.69528898999993], [-73.94324249099988, 40.69455748499995], [-73.94312826799985, 40.69396703799988], [-73.9431142779999, 40.693894719999896], [-73.94310040999994, 40.69382302899988], [-73.942951361, 40.693090783999864], [-73.94280765199987, 40.69235779399988], [-73.94266181699986, 40.691624343999855], [-73.94251587899991, 40.69089200099991], [-73.94236932699991, 40.690159944999884], [-73.94222203500003, 40.689427978999916], [-73.9420768489998, 40.6886972019999], [-73.94193245099987, 40.687962958999904], [-73.94178527599988, 40.68722837199987], [-73.94163933199995, 40.686497269999954], [-73.94149491799988, 40.685764528999904], [-73.94134827200003, 40.68503120299987], [-73.94120399299996, 40.68429983899994], [-73.94105783799992, 40.683566871999844], [-73.940913343, 40.682833616999865], [-73.94076893299986, 40.6821008389999], [-73.94062005399995, 40.68137012999991], [-73.94047634999994, 40.68063569499985], [-73.94032794, 40.679889974999874], [-73.9412086429999, 40.67993835399988], [-73.94326176899995, 40.68005060699992], [-73.94397347799995, 40.680088127999916], [-73.94627471000001, 40.68021332599986], [-73.94674916000001, 40.68023966099991], [-73.94773023599987, 40.6802918459999], [-73.94955688699993, 40.68039040299995], [-73.95115828499993, 40.68047861499993], [-73.95155682700002, 40.680498847999935], [-73.9533701749999, 40.680640507999904], [-73.95351616899988, 40.6813825999999], [-73.95366256199999, 40.68211490399989], [-73.95380893500004, 40.68284800799992], [-73.953954531, 40.683580778999946], [-73.95410042600001, 40.68431310799984], [-73.95424647699987, 40.685046247999914], [-73.95439296899998, 40.68577971999988], [-73.95453798699995, 40.686511174999936], [-73.95468418899998, 40.68724485399985], [-73.95684165199995, 40.68699607899989], [-73.95956770199984, 40.686682554999955], [-73.95971374799981, 40.687415633999926], [-73.959859395, 40.6881474509999], [-73.96000519799986, 40.688881033999955], [-73.96009714599994, 40.68934520999991], [-73.96012172900004, 40.689469306999825], [-73.96017256100002, 40.68972986199988], [-73.96018691899992, 40.6898034559999], [-73.96022304499992, 40.6899886269999], [-73.96023740299987, 40.690062221999874], [-73.96029281699998, 40.69034624999995], [-73.96013760799985, 40.69036437999989], [-73.96008336799996, 40.69037071599989], [-73.95957591899987, 40.69042998799992], [-73.95942791399997, 40.69044727499993], [-73.959348624, 40.69045646099992], [-73.95928133699987, 40.69046425699991], [-73.9589933049999, 40.69049762899987], [-73.95886713600001, 40.69051224799996], [-73.95857844799987, 40.69054569399986], [-73.95818048699985, 40.690590729999876], [-73.95808563499989, 40.690601463999876], [-73.95727249099987, 40.69069347799987], [-73.95635602999991, 40.69079978199989], [-73.95541057999996, 40.69090829199994], [-73.95582662799985, 40.692992382999925], [-73.95614239299985, 40.694579018999896], [-73.95708701900001, 40.69447043999992], [-73.95799732999993, 40.694365838999914], [-73.95931047999983, 40.69421508699989], [-73.96015854699988, 40.69411730899989], [-73.96062056499981, 40.69632013999993], [-73.96092543799982, 40.697736506999895], [-73.96105100699992, 40.69832607899989], [-73.96019688899986, 40.69846272699991], [-73.95885874699991, 40.69867732599986], [-73.9579593819998, 40.69882000299991], [-73.95701993199991, 40.69897391399995], [-73.95716730100004, 40.69970786799989], [-73.95722517499998, 40.69999934999987], [-73.95745736399992, 40.70082260299993], [-73.95572360999994, 40.70194576999991], [-73.95381196900003, 40.703180979999914], [-73.95318085199992, 40.702616904999935], [-73.95255052799989, 40.70205166699994], [-73.95192018899999, 40.70148754899992], [-73.95128819399989, 40.700922364999855], [-73.9502742399999, 40.70157924199991], [-73.94937873499998, 40.70215888999993], [-73.94753858099985, 40.70335064999991], [-73.94705205299991, 40.70366394899985], [-73.94625107799986, 40.703208746999934], [-73.945854991, 40.70298720699987], [-73.94544988199996, 40.70276063599993], [-73.94463910199993, 40.702313693999926], [-73.94431460199992, 40.70213334499988], [-73.94400503999998, 40.70196179199989], [-73.94382473799989, 40.70186200799988], [-73.94322685899992, 40.7015207089999], [-73.94306406899986, 40.701424434999886]]]}}, {\"id\": \"17\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 18, \"Shape_Leng\": 0.0697995498569, \"Shape_Area\": 0.000148850163948, \"zone\": \"Bedford Park\", \"LocationID\": 18, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88513907699999, 40.86638287399992], [-73.88593703500003, 40.86592439799994], [-73.88683316999993, 40.86538999399989], [-73.8881227879999, 40.86425457099988], [-73.88889566099989, 40.86475415299988], [-73.890161637, 40.86347636299995], [-73.8906539299999, 40.86295483399993], [-73.89114803099996, 40.862476137999934], [-73.8917325659999, 40.861909814999905], [-73.89177188799992, 40.86187171599992], [-73.89138023399983, 40.86170058799992], [-73.89141802399983, 40.86166298399996], [-73.89256392599998, 40.860522728999925], [-73.89321520799987, 40.85987420599986], [-73.89381026099987, 40.858986564999924], [-73.8946932639999, 40.85942628199991], [-73.89573924699978, 40.85813820699986], [-73.89636314599984, 40.85837531899991], [-73.89735765499988, 40.85876273499986], [-73.89818244699991, 40.85908368099995], [-73.89866157499996, 40.859246901999896], [-73.89883814599983, 40.859307307999906], [-73.89880457699982, 40.8593642709999], [-73.898590972, 40.859766782999934], [-73.89828973399992, 40.86033442099995], [-73.89822728999994, 40.860448863999885], [-73.89802453099999, 40.86082046099989], [-73.89773815399992, 40.86135113699993], [-73.897143798, 40.86244506499991], [-73.89737274799985, 40.86243236399985], [-73.89798099399988, 40.86240253999991], [-73.89841038399996, 40.862296966999935], [-73.89943670799998, 40.862116911999856], [-73.89964261700001, 40.862210436999874], [-73.89996177599984, 40.86235027999989], [-73.90028368099995, 40.86247183699992], [-73.90107966999994, 40.86275638299992], [-73.90027587299991, 40.863787634999944], [-73.89887282999977, 40.86558402699991], [-73.89797842699994, 40.86672770799987], [-73.8974032329998, 40.86747418999989], [-73.89631315699987, 40.8670385749999], [-73.89544263399986, 40.866602088999855], [-73.89536151600007, 40.86656133099995], [-73.8945304879999, 40.86614380899987], [-73.89438130299995, 40.866068850999966], [-73.89436438800003, 40.86609167199985], [-73.89431825299997, 40.8661539259999], [-73.89408697099991, 40.86645343299995], [-73.89397201799989, 40.8666023039999], [-73.8937613899999, 40.866875057999955], [-73.89368435599988, 40.86698082099989], [-73.89300269899991, 40.86788106799988], [-73.89243509099991, 40.868620597999936], [-73.89221258799988, 40.868915499999936], [-73.8919171829999, 40.869302109999886], [-73.8917463349999, 40.86951054799996], [-73.89157400099995, 40.86968425199998], [-73.89138946999998, 40.869850993999904], [-73.89119350399992, 40.8700100329999], [-73.89098696999984, 40.870160714999905], [-73.89042763899991, 40.870487053999916], [-73.88991744500004, 40.870780631999864], [-73.88950049199994, 40.87102055599993], [-73.88884301599997, 40.87139806699986], [-73.88868449900004, 40.87149511299988], [-73.88833546999989, 40.871709830999905], [-73.8879350369998, 40.87203023199988], [-73.88785196299986, 40.87211417599992], [-73.88798774400006, 40.872227490999855], [-73.88857987899992, 40.87272163999993], [-73.88863187299991, 40.87276503099986], [-73.88937163799984, 40.87338229799993], [-73.88855699899982, 40.8751973129999], [-73.88824333699998, 40.875892194999935], [-73.88808032199985, 40.87627274599987], [-73.88773577599993, 40.87703388199987], [-73.88567517999988, 40.87887251099986], [-73.88541068099991, 40.87910168399991], [-73.88527683199987, 40.87921766899991], [-73.88505419400002, 40.87889287599994], [-73.88489171499985, 40.87860601699993], [-73.88477074799978, 40.87839502799989], [-73.8846776199998, 40.87823547699989], [-73.88382977799998, 40.8767828419999], [-73.88355294399997, 40.87625222899987], [-73.88349725699989, 40.876156739999885], [-73.88345586399996, 40.876056256999945], [-73.88345579299984, 40.87605609499993], [-73.88345575699994, 40.87605593399992], [-73.88342964999991, 40.87595216999989], [-73.88341932399999, 40.87584701799994], [-73.88342472699996, 40.875742665999866], [-73.88343632799992, 40.87538431899991], [-73.88344008999987, 40.87479468399987], [-73.883400353, 40.87394662599995], [-73.88334994299991, 40.87375884999991], [-73.88327982500005, 40.87357380999987], [-73.88319052799997, 40.87339337899994], [-73.88319048200002, 40.87339327999999], [-73.8831904229999, 40.87339318999993], [-73.88308277499985, 40.87321902499988], [-73.88308273999986, 40.873218971999904], [-73.88308270399986, 40.87321891799995], [-73.882957783, 40.87305245199983], [-73.8828305549999, 40.87291666499986], [-73.88283042399992, 40.87291651999991], [-73.88283030599987, 40.87291636699989], [-73.88271951399993, 40.87277078099993], [-73.88262696899983, 40.87261718899993], [-73.88262687499989, 40.87261702799994], [-73.88262679199994, 40.872616855999865], [-73.88255399799999, 40.87245745799991], [-73.88255391499996, 40.872457287999914], [-73.88255385599989, 40.87245711599994], [-73.88250150999994, 40.87229386699992], [-73.8825014739999, 40.87229374999986], [-73.8825014509999, 40.872293623999944], [-73.88246984299988, 40.87212868499994], [-73.88222049399987, 40.871227247999855], [-73.88197649899989, 40.87038728399996], [-73.88174189599988, 40.8695527139999], [-73.88146889699985, 40.868573645999916], [-73.88188379899985, 40.86832083599991], [-73.88284356199999, 40.86773622599992], [-73.88362518100003, 40.86725758799991], [-73.88442861399994, 40.86680037899986], [-73.88513907699999, 40.86638287399992]]]}}, {\"id\": \"18\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 19, \"Shape_Leng\": 0.101824875452, \"Shape_Area\": 0.000546661094782, \"zone\": \"Bellerose\", \"LocationID\": 19, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.72339596299987, 40.750389075999855], [-73.72258571899978, 40.74885045099991], [-73.720144294, 40.74976283099995], [-73.72010148299994, 40.74977568599991], [-73.71988664100002, 40.74936951099994], [-73.71896368299994, 40.74699473999995], [-73.71894406299995, 40.74694425599994], [-73.71892134799997, 40.74689486899994], [-73.71869037499981, 40.746392765999936], [-73.71853063599995, 40.7460393299999], [-73.71823250099993, 40.74539666299989], [-73.71764302999979, 40.744070413999964], [-73.71762516399986, 40.74402847899988], [-73.71760306099992, 40.743974627999876], [-73.71740598499986, 40.74351092999989], [-73.71656758399989, 40.74164908599988], [-73.71578267699999, 40.73989578599985], [-73.71544946799989, 40.73921680399994], [-73.71511121299982, 40.73856840999988], [-73.71496199799985, 40.738276409999855], [-73.71494063099992, 40.73823459999989], [-73.71476982799997, 40.7379003479999], [-73.71436629299991, 40.737243891999896], [-73.71370155699985, 40.73616012199993], [-73.71328662099987, 40.735473124999935], [-73.71289825099977, 40.73485113699988], [-73.71283004500002, 40.73473962799994], [-73.71278769000006, 40.7346703809999], [-73.71250100899985, 40.734201632999905], [-73.7122862199999, 40.73385241799987], [-73.71208085999997, 40.73351852599989], [-73.71186191499989, 40.73315683699991], [-73.71147804899985, 40.73250140099988], [-73.710893249, 40.730971041999936], [-73.71075217299996, 40.73052537499992], [-73.71066275899989, 40.72949571099991], [-73.710655187, 40.729382965999925], [-73.71061871899991, 40.72883987399995], [-73.7105550639999, 40.728020849999915], [-73.71050351799981, 40.72727838999988], [-73.71049995381028, 40.72722708720663], [-73.7105628560812, 40.7272188714254], [-73.71191821920186, 40.72704180707466], [-73.71407555595218, 40.726759930163915], [-73.71496944887915, 40.72664661018486], [-73.71587971997315, 40.7265419058296], [-73.71828706789265, 40.726041111135174], [-73.72038627275316, 40.72547455419105], [-73.7211926570586, 40.725266546195385], [-73.72204226687052, 40.7250126029472], [-73.722578793026, 40.72487767601808], [-73.72291170303338, 40.7247939467331], [-73.72295712084203, 40.724782448805506], [-73.72296233421183, 40.724781127902155], [-73.72315163514787, 40.72473320701826], [-73.7234275409709, 40.72466335095062], [-73.72419675486013, 40.72445460193022], [-73.72454518798916, 40.72436003595222], [-73.72463422510643, 40.72433644820514], [-73.72471792493599, 40.72431260475058], [-73.72473123486797, 40.72430839576523], [-73.72493417712295, 40.72424417017321], [-73.72508489078929, 40.72419561380042], [-73.72567196979708, 40.724037658996544], [-73.72647559477359, 40.723752496193974], [-73.72732826508961, 40.72341196312062], [-73.72797157476118, 40.72315911624762], [-73.72816233910697, 40.7230841398913], [-73.72818574899999, 40.723145636999924], [-73.72836415399992, 40.72361435199985], [-73.72990512499977, 40.723272494999954], [-73.73016587199996, 40.72395859599988], [-73.73043003899987, 40.724640254999876], [-73.73069257599997, 40.72532826699992], [-73.73095653299988, 40.726014939999835], [-73.73121918199986, 40.72669819599986], [-73.73144806399984, 40.72734384999995], [-73.7314721509999, 40.727411794999924], [-73.73161735999996, 40.72744912699996], [-73.73272577999994, 40.727734069999904], [-73.73361727799993, 40.727933573999955], [-73.73424214199999, 40.7280753799999], [-73.73450816499994, 40.7281364099999], [-73.73498719899999, 40.72824220599991], [-73.73545493899995, 40.7283456249999], [-73.73632483599988, 40.72854566199993], [-73.73715501999992, 40.728732010999884], [-73.73816144099993, 40.7289580909999], [-73.73840175399988, 40.729011523999866], [-73.73904904900003, 40.72916463899987], [-73.73968059999994, 40.72930668299992], [-73.74040469999989, 40.72946953699989], [-73.74080920999988, 40.729560507999956], [-73.74200329399991, 40.72982592299992], [-73.74230785999983, 40.72989362199998], [-73.74242143799988, 40.72991886499994], [-73.74376398999986, 40.730227197999966], [-73.74383878399982, 40.7302445999999], [-73.74385189999987, 40.73024764599996], [-73.74398330999986, 40.73027821199988], [-73.74411105400003, 40.73030792499984], [-73.74417482499997, 40.730322763999936], [-73.74488800699997, 40.73048310599994], [-73.74543152500001, 40.73061791799988], [-73.74555667199989, 40.73063689799994], [-73.74629555799991, 40.731180177999875], [-73.74640108899996, 40.7312928389999], [-73.74649162199995, 40.731414588999925], [-73.74656523699987, 40.73154360499996], [-73.74659290699987, 40.73161070999988], [-73.74662058899987, 40.73167780499995], [-73.7466568949999, 40.73181492399991], [-73.74667404599982, 40.73195264999993], [-73.74669850599986, 40.73221018799995], [-73.74671314399986, 40.73236436899988], [-73.74672140400006, 40.73247715099991], [-73.7461625949999, 40.73292721999995], [-73.74555368099982, 40.73340431799991], [-73.74472417799997, 40.734011029999905], [-73.742947859, 40.7354565519999], [-73.74228431199994, 40.736086407999935], [-73.74158944499997, 40.736782008999946], [-73.74151636899985, 40.73686036099986], [-73.74144628700002, 40.736938745999964], [-73.74016951799993, 40.73848189799986], [-73.73986995499992, 40.73886690099992], [-73.73972244199989, 40.739036089999914], [-73.73933151399987, 40.739474474999916], [-73.73901210399985, 40.73974647899997], [-73.738571759, 40.74009212299992], [-73.73816047999985, 40.74037643499986], [-73.73751038099981, 40.74071544799988], [-73.73685894199984, 40.741050400999875], [-73.73560786699984, 40.74169621999989], [-73.73539272699988, 40.74181104999989], [-73.7343696589998, 40.74235159899991], [-73.73375484899985, 40.742747627999975], [-73.73370171599998, 40.7427851569999], [-73.7336646159999, 40.74281318399988], [-73.73241694699988, 40.74369457599995], [-73.73232731699994, 40.74377143299988], [-73.7322691809999, 40.74381896199988], [-73.7310646189999, 40.744888375999885], [-73.7306280189999, 40.745364157999866], [-73.73043134700004, 40.74558483599989], [-73.72993169099983, 40.74618812499993], [-73.72964619299997, 40.74656191399992], [-73.72953448999986, 40.746716395999925], [-73.729145939, 40.74727379999993], [-73.72906670999978, 40.74739095899988], [-73.728962516, 40.74763288799986], [-73.72888621599986, 40.74756663599992], [-73.72829094199989, 40.74745895699992], [-73.72822968899995, 40.747830803999875], [-73.72815098799995, 40.74820123199985], [-73.72805493499999, 40.74856952199987], [-73.7279417009999, 40.74893498799988], [-73.72781151199987, 40.74929691999998], [-73.727664654, 40.749654641999946], [-73.72627213299992, 40.74930649099986], [-73.72339596299987, 40.750389075999855]]]}}, {\"id\": \"19\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 20, \"Shape_Leng\": 0.0514401924362, \"Shape_Area\": 0.000134512633032, \"zone\": \"Belmont\", \"LocationID\": 20, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88386792099986, 40.8642908889999], [-73.88379386499992, 40.86417182299991], [-73.8837937109999, 40.86417157099987], [-73.88200794099986, 40.862046852999946], [-73.88103469099984, 40.861275294999885], [-73.88092572600007, 40.86116228799991], [-73.88083055899997, 40.86104034399993], [-73.88075099099991, 40.86091098799991], [-73.88068839499986, 40.86077599299989], [-73.8806437029999, 40.86063731599991], [-73.88061726900006, 40.860497010999936], [-73.88060896899991, 40.860357092999884], [-73.88075871500001, 40.85949442499993], [-73.880965862, 40.85854520499987], [-73.88098942799995, 40.85845935899989], [-73.88100678999979, 40.85826802299992], [-73.88100464399997, 40.85807565699992], [-73.88098293799995, 40.85788399199989], [-73.88094199099993, 40.857694990999974], [-73.88094193200001, 40.85769473899994], [-73.88094184999994, 40.85769449599996], [-73.88088225100002, 40.85750954499986], [-73.88080810099991, 40.857198447999885], [-73.88079155299991, 40.85712750799994], [-73.88077132199986, 40.8570397609999], [-73.880729541, 40.85680727499991], [-73.88073042799992, 40.85667660499989], [-73.88103358999996, 40.85559995199987], [-73.8816727419998, 40.8533496789999], [-73.88200299099991, 40.85218935799987], [-73.88232256599994, 40.851060791999906], [-73.88255596199991, 40.85022427099987], [-73.88284931500006, 40.84919252699994], [-73.88288269199988, 40.84907404699991], [-73.88311982600001, 40.848221091999925], [-73.88322877699981, 40.84834587799986], [-73.88377755699985, 40.848682183999884], [-73.88480193499986, 40.84930945899987], [-73.88581418499989, 40.84993079699989], [-73.88643555899993, 40.850309427999875], [-73.88751770499984, 40.85094946799988], [-73.88800539899992, 40.85097610299994], [-73.88823974099995, 40.85103536599995], [-73.8887824569999, 40.851341184999924], [-73.88895256299988, 40.85144078099994], [-73.88951837699989, 40.85180274499989], [-73.88963044999998, 40.85188593199995], [-73.88987871399985, 40.85207022099993], [-73.88993961599995, 40.85259076299992], [-73.88963328299991, 40.85317381799993], [-73.88960978099988, 40.85362187999995], [-73.89003039600001, 40.854220991999945], [-73.89038953999983, 40.85468905799996], [-73.89044259299995, 40.85504635099993], [-73.8907742849999, 40.85581193999993], [-73.89108676599993, 40.85659725699984], [-73.89109710400002, 40.856779813999886], [-73.8911055689998, 40.85742738299995], [-73.89111318899985, 40.857850998999865], [-73.89111115400003, 40.85838432199987], [-73.89111943300001, 40.85865896199989], [-73.89112461399988, 40.8588700059999], [-73.89112568799987, 40.860061824999924], [-73.89059489199987, 40.86071569999991], [-73.89027555599993, 40.86110907299996], [-73.89036823899993, 40.86115807599985], [-73.89067455199995, 40.86132004099988], [-73.89082169799998, 40.86139939599989], [-73.89106280699994, 40.8615294119999], [-73.89138023399983, 40.86170058799992], [-73.89177188799992, 40.86187171599992], [-73.8917325659999, 40.861909814999905], [-73.8906539299999, 40.86295483399993], [-73.890161637, 40.86347636299995], [-73.88889566099989, 40.86475415299988], [-73.8881227879999, 40.86425457099988], [-73.88683316999992, 40.865389992999894], [-73.88593703500003, 40.86592439799994], [-73.88513907699999, 40.86638287399992], [-73.88442861399994, 40.86680037899986], [-73.88362518100003, 40.86725758799991], [-73.8830900319999, 40.86659150899993], [-73.88297537099986, 40.86644973199989], [-73.88271794699989, 40.86613296599992], [-73.88267624699999, 40.86608914099984], [-73.88273658899989, 40.86604857299993], [-73.88319816099992, 40.86576631399988], [-73.88344168299979, 40.86561633199988], [-73.883563778, 40.86552635199991], [-73.88367321300004, 40.86542528299993], [-73.88376749199992, 40.86531475299994], [-73.88384464399992, 40.86519681299989], [-73.88390334899988, 40.86507380199995], [-73.88394303099996, 40.86494818699985], [-73.88396922299982, 40.864817740999825], [-73.88397514999986, 40.86468399799992], [-73.88396002399999, 40.864549718999896], [-73.88392395799984, 40.86441776099988], [-73.88386792099986, 40.8642908889999]]]}}, {\"id\": \"20\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 21, \"Shape_Leng\": 0.115973569062, \"Shape_Area\": 0.000380251345507, \"zone\": \"Bensonhurst East\", \"LocationID\": 21, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97418385499991, 40.609463501999905], [-73.97403097799986, 40.608702309999906], [-73.97316643599986, 40.60879557699993], [-73.97299433999983, 40.6088141419999], [-73.97258982399998, 40.60667153299988], [-73.97239735499981, 40.60565188999986], [-73.97217742800002, 40.604194711999845], [-73.97235152099995, 40.60334811899986], [-73.972452617, 40.60261705399988], [-73.9725653369999, 40.601794372999905], [-73.97291116900001, 40.599315803999964], [-73.9719108049999, 40.599423058999896], [-73.97176105299991, 40.59865193199989], [-73.97163038199993, 40.5979440169999], [-73.97148350899987, 40.59717158499996], [-73.97323983499987, 40.59697881799983], [-73.97408390599992, 40.59688294399995], [-73.974926956, 40.596793537999936], [-73.97503940399993, 40.59678002699995], [-73.97558819299998, 40.59671920599991], [-73.97650934500001, 40.59661610899983], [-73.97750674199997, 40.596506452999904], [-73.9785016919999, 40.596394852999914], [-73.97890571399992, 40.59635272999989], [-73.9794258679999, 40.596297522999876], [-73.98035785499995, 40.59619154099992], [-73.98014301299985, 40.595066175999904], [-73.97995443499985, 40.594073811999884], [-73.98071099999997, 40.59399739599998], [-73.98179084800002, 40.593867455999906], [-73.98273346399986, 40.593763698999915], [-73.98303296899982, 40.593730292999915], [-73.9834412439999, 40.593687772999864], [-73.98354622399994, 40.59367650999994], [-73.98366766399984, 40.593663481999926], [-73.98407040399994, 40.593617363999876], [-73.98412888499986, 40.593610666999915], [-73.98463965399982, 40.59355217599988], [-73.98467794099993, 40.5936586499999], [-73.98601936799993, 40.594446048999906], [-73.98798856399989, 40.59255277799986], [-73.98962150099992, 40.59098199299996], [-73.99036425099996, 40.591431931999935], [-73.99066311999997, 40.5911671669999], [-73.99147342099985, 40.590449305999904], [-73.99153934399993, 40.59039090299989], [-73.99161412399994, 40.59035644499992], [-73.99255747799991, 40.5899216749999], [-73.99286067699995, 40.58978269799995], [-73.99291591499997, 40.589757378999884], [-73.99297835999997, 40.58973003299996], [-73.99330855, 40.58958389899994], [-73.99375761899992, 40.590331843999856], [-73.99801867061471, 40.588425400020526], [-73.99867293375554, 40.5891955499396], [-73.99923690396953, 40.588935328269024], [-73.99946446811052, 40.58923326938065], [-73.99837834882116, 40.58977044508854], [-73.99842061695254, 40.5898140545596], [-73.99883618014556, 40.5896254895423], [-73.9988658617368, 40.58966697528812], [-73.99844387079486, 40.58986039917759], [-73.99851059611983, 40.58996013293428], [-73.99855344167928, 40.590017683146215], [-73.99858515364288, 40.59004738497782], [-73.99862485126727, 40.59008456707945], [-73.99868197911874, 40.59012967417432], [-73.99874318832653, 40.590190336045396], [-73.99877991358242, 40.5902323318472], [-73.99879419437912, 40.5902805499443], [-73.99877800589593, 40.59031435534189], [-73.99873298307851, 40.590334987929545], [-73.99868809619367, 40.590325655709115], [-73.99865341212593, 40.59029143644465], [-73.9985942438702, 40.59022766353232], [-73.99853303406093, 40.59020899816093], [-73.99849018691935, 40.59021677414818], [-73.99844325936857, 40.59024010549443], [-73.99839633111722, 40.590282100254925], [-73.99835552304194, 40.59033498314752], [-73.99775972701352, 40.59095246961378], [-73.99711168571537, 40.591277977443745], [-73.99710562330513, 40.59133427927689], [-73.99710730365224, 40.591390755943046], [-73.99711670841539, 40.59144679034145], [-73.99713373485126, 40.5915017702026], [-73.99715819693523, 40.59155509477774], [-73.99718982739483, 40.59160618140396], [-73.9972282806271, 40.59165447187027], [-73.99806438267662, 40.59120946473145], [-73.99812869461262, 40.5912886642579], [-73.99740518642399, 40.59164508498032], [-73.99728269703776, 40.59171481486809], [-73.99754851058395, 40.59191757917553], [-73.99856404800653, 40.591492325309545], [-73.99916761862079, 40.59114913581574], [-73.99957329199391, 40.591092566583356], [-73.99986219746393, 40.59112803340917], [-73.99992669140542, 40.59114738407374], [-73.9999873500428, 40.591172945649255], [-74.00004314944917, 40.59120428665461], [-74.00009314771766, 40.59124087805044], [-74.00013650086177, 40.591282102170034], [-74.00017247706104, 40.59132726314402], [-74.00020046901545, 40.59137559864704], [-74.000220004197, 40.59142629276542], [-74.00023075282738, 40.591478489769656], [-74.00023253344592, 40.59153130855935], [-74.00005812170888, 40.59171484748445], [-73.99967223219585, 40.59196375866878], [-73.99930612934223, 40.592178726406004], [-73.99903131441663, 40.59231280344756], [-73.99902051717862, 40.592323348987264], [-73.99901217208547, 40.59233511698889], [-73.99900651177178, 40.592347779399724], [-73.99900369403004, 40.59236098323367], [-73.99900379741118, 40.59237436041128], [-73.99900681903495, 40.59238753802081], [-73.99901267467028, 40.59240014871337], [-73.99902120108229, 40.59241184094382], [-73.99903216058368, 40.592422288770635], [-73.9990452476599, 40.592431200941924], [-73.9990600974859, 40.59243832901494], [-73.99907629609663, 40.592443474281794], [-73.99909339192617, 40.59244649330885], [-73.99930029587152, 40.592559122665], [-73.99934775356314, 40.592622309479], [-73.99938869468849, 40.592688091910205], [-73.99942287662074, 40.59275608014794], [-73.99945009678441, 40.59282587130941], [-73.99947019385698, 40.59289705182673], [-73.9994830488618, 40.59296919994875], [-73.9991972768818, 40.59317814395959], [-73.99897464310263, 40.59330636943554], [-73.99838831091802, 40.59353139180308], [-73.99736802499984, 40.59397431099996], [-73.99633951299984, 40.59362247499993], [-73.99630983500005, 40.593667751999895], [-73.99629319699989, 40.593716441999916], [-73.99619345599999, 40.593992197999896], [-73.99535761899992, 40.59440332799991], [-73.99530829599992, 40.59442758699989], [-73.99549868199995, 40.594594406999946], [-73.99587725299996, 40.59491413699993], [-73.99611613099994, 40.59511588599987], [-73.99639126099986, 40.595345801999905], [-73.99669451199998, 40.595599205999875], [-73.99674706899995, 40.59564312599992], [-73.99630015799983, 40.59606105399993], [-73.99522558800007, 40.59709332799984], [-73.99336225799983, 40.598888688999885], [-73.99159842599994, 40.60058036599988], [-73.99097171199985, 40.60118455999986], [-73.99038998499982, 40.60174340299995], [-73.989807573, 40.60230291099993], [-73.9892234389999, 40.60286567499987], [-73.98869008799991, 40.603425306999966], [-73.98806312999994, 40.603979451999834], [-73.98747998199978, 40.60453971199996], [-73.9868476259999, 40.60509723599994], [-73.98864528, 40.60616665399989], [-73.98913191499994, 40.6064561369999], [-73.98855078899993, 40.60700228299991], [-73.98842267599987, 40.60711914899985], [-73.98796510300004, 40.60756553599988], [-73.9873779529998, 40.60812811799987], [-73.9873391769999, 40.60816828599996], [-73.98671329999988, 40.60876626099987], [-73.98612915399993, 40.6093298639999], [-73.98575987199992, 40.60968408399989], [-73.9855485319999, 40.60988724299989], [-73.98535587499991, 40.610072013999876], [-73.98496383499996, 40.61044799399991], [-73.98438244899988, 40.611005419999884], [-73.984162011, 40.61121730099988], [-73.98380893599982, 40.61155987299987], [-73.98335142399998, 40.61199230099991], [-73.98322226999993, 40.61212220899992], [-73.98265927499983, 40.61269247099993], [-73.98215550199991, 40.613149614999934], [-73.98206106199993, 40.6132384189999], [-73.98174843299985, 40.613554623999946], [-73.98143750099993, 40.613837687999926], [-73.98080506199987, 40.614447129999846], [-73.98022027499997, 40.61500494899987], [-73.97963821799982, 40.61556454699991], [-73.977408165, 40.614218112999886], [-73.975208294, 40.612885257999906], [-73.97477658, 40.6126384749999], [-73.97458407499984, 40.61161357599985], [-73.97451543899999, 40.61125680099987], [-73.974390729, 40.6105792129999], [-73.97418385499991, 40.609463501999905]]]}}, {\"id\": \"21\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 22, \"Shape_Leng\": 0.126170229196, \"Shape_Area\": 0.000472062684608, \"zone\": \"Bensonhurst West\", \"LocationID\": 22, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99254973599997, 40.62427426799996], [-73.99009356099988, 40.62278895299992], [-73.98785095499996, 40.62143503199991], [-73.9856935779999, 40.6201315139999], [-73.98627524699981, 40.61956661699987], [-73.98406791799982, 40.61824073899987], [-73.98186818299985, 40.61691212699987], [-73.98087912799988, 40.61631443499988], [-73.98003930399985, 40.61580692999995], [-73.97963821799982, 40.61556454699991], [-73.98022027499997, 40.61500494899987], [-73.98080506199987, 40.614447129999846], [-73.98143750099993, 40.613837687999926], [-73.98174843299985, 40.613554623999946], [-73.98206106199993, 40.6132384189999], [-73.98215550199991, 40.613149614999934], [-73.98265927499983, 40.61269247099993], [-73.98322226999993, 40.61212220899992], [-73.98335142399998, 40.61199230099991], [-73.98380893599982, 40.61155987299987], [-73.984162011, 40.61121730099988], [-73.98438244899988, 40.611005419999884], [-73.98496383499996, 40.61044799399991], [-73.98535587499991, 40.610072013999876], [-73.9855485319999, 40.60988724299989], [-73.98575987199992, 40.60968408399989], [-73.98612915399993, 40.6093298639999], [-73.98671329999988, 40.60876626099987], [-73.9873391769999, 40.60816828599996], [-73.9873779529998, 40.60812811799987], [-73.98796510300004, 40.60756553599988], [-73.98842267599987, 40.60711914899985], [-73.98855078899993, 40.60700228299991], [-73.98913191499994, 40.6064561369999], [-73.98864528, 40.60616665399989], [-73.9868476259999, 40.60509723599994], [-73.98747998199978, 40.60453971199996], [-73.98806312999994, 40.603979451999834], [-73.98869008799991, 40.603425306999966], [-73.9892234389999, 40.60286567499987], [-73.989807573, 40.60230291099993], [-73.99038998499982, 40.60174340299995], [-73.99097171199985, 40.60118455999986], [-73.99159842599994, 40.60058036599988], [-73.99336225799983, 40.598888688999885], [-73.99522558800007, 40.59709332799984], [-73.99630015799983, 40.59606105399993], [-73.99674706899995, 40.59564312599992], [-73.99669451199998, 40.595599205999875], [-73.99639126099986, 40.595345801999905], [-73.99611613099994, 40.59511588599987], [-73.99587725299996, 40.59491413699993], [-73.99549868199995, 40.594594406999946], [-73.99530829599992, 40.59442758699989], [-73.99535761899992, 40.59440332799991], [-73.99619345599999, 40.593992197999896], [-73.99629319699989, 40.593716441999916], [-73.99630983500005, 40.593667751999895], [-73.99633951299984, 40.59362247499993], [-73.99736802499984, 40.59397431099996], [-73.99838831091802, 40.59353139180308], [-73.99854421315825, 40.59359676275944], [-74.00030548953626, 40.59280477839979], [-74.00105748445709, 40.5925370050623], [-74.00164017530518, 40.59348065580983], [-74.00193320208321, 40.59389469541273], [-74.00184415135462, 40.593977667237965], [-74.00200394804675, 40.594132371137796], [-74.00027871999994, 40.59570266899989], [-74.00004233999992, 40.59594037099991], [-73.99999769599995, 40.59596779199991], [-73.9999650309999, 40.59599430299989], [-73.99951049399985, 40.59644180899987], [-73.99931921399997, 40.59664075099992], [-73.9986558659999, 40.59726856199988], [-73.99733919599988, 40.5985516119999], [-73.99559320799995, 40.600232221999875], [-73.9963651649999, 40.600696053999904], [-73.99708036399986, 40.60112882899989], [-73.99782332599999, 40.601578178999944], [-73.998563851, 40.602028433999905], [-73.99927902000005, 40.60245801699991], [-74.00002266099992, 40.60290731599988], [-74.00032353899998, 40.60262029199991], [-74.00109809599998, 40.60303462599992], [-74.00184603099991, 40.60343143199989], [-74.002630697, 40.60384644899989], [-74.00340902699993, 40.60425938099994], [-74.00416249599996, 40.60465473899992], [-74.00444636799998, 40.60480184699984], [-74.00493448899994, 40.60507440599987], [-74.00442477199992, 40.60556859699992], [-74.00264412399989, 40.60725717499991], [-74.00195011999993, 40.60781768299991], [-74.00171808600001, 40.60800297799994], [-74.00130866099992, 40.60834356899987], [-74.00094408399997, 40.60863970999988], [-74.00185301899995, 40.60913755999995], [-74.00266468099994, 40.60956996599992], [-74.00342538899996, 40.609969092999926], [-74.00601438399993, 40.61132508199992], [-74.00708800599996, 40.61190794499986], [-74.00788398499994, 40.61234007199985], [-74.00830383399986, 40.612567999999925], [-74.01019858699995, 40.613712166999925], [-74.01050403599997, 40.613896610999866], [-74.0099237199999, 40.61445664299991], [-74.00934164499988, 40.615015095999865], [-74.00876039999996, 40.61557582299988], [-74.00817687599992, 40.61613318099995], [-74.00759806899993, 40.61669446099994], [-74.00701531599991, 40.6172539459999], [-74.00643394999992, 40.617814716999874], [-74.00585319899992, 40.61837047699991], [-74.00522665999992, 40.618973153999875], [-74.00459907, 40.619576852999955], [-74.00401906899995, 40.62013618899991], [-74.0034355699999, 40.62069540399994], [-74.00285701599996, 40.621253642999896], [-74.00227586299998, 40.621814273999874], [-74.00169380199993, 40.622372426999966], [-74.00120222399994, 40.62284271199989], [-74.00052987499996, 40.623489938999946], [-73.99994665000001, 40.624050621999864], [-73.99932090299998, 40.62465524699996], [-73.99771358999998, 40.62368026999996], [-73.99748403999989, 40.624519845999934], [-73.99724067099987, 40.62528868699994], [-73.99701633599989, 40.62606161499991], [-73.99687778399999, 40.626530782999964], [-73.99678847599995, 40.626826050999895], [-73.99507814899985, 40.62580673699993], [-73.99464972999995, 40.62569717899992], [-73.99473485699997, 40.6256147409999], [-73.99489280599995, 40.625461793999925], [-73.99432497399985, 40.62534290799988], [-73.99398197100003, 40.62514139899989], [-73.99254973599997, 40.62427426799996]]]}}, {\"id\": \"22\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 23, \"Shape_Leng\": 0.290556028962, \"Shape_Area\": 0.00219556576201, \"zone\": \"Bloomfield/Emerson Hill\", \"LocationID\": 23, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.19568609223377, 40.63501686464005], [-74.1955768805797, 40.635004218570565], [-74.1955597527974, 40.635023986316575], [-74.19248359599999, 40.633405071999945], [-74.1921598649999, 40.633230970999946], [-74.18649341199989, 40.63021905099987], [-74.18368103499995, 40.62874067099991], [-74.18190235699991, 40.62775580699991], [-74.18033424799995, 40.626834629999905], [-74.17993706099998, 40.626601296999894], [-74.1795447069999, 40.62636235399988], [-74.17924623499991, 40.626180580999886], [-74.17918377399995, 40.626137884999906], [-74.17888390599995, 40.62593287099991], [-74.17828177399998, 40.62545136099991], [-74.1780882019999, 40.62526042699992], [-74.1778150209999, 40.62499096299995], [-74.17775298599997, 40.62493844999993], [-74.17610052899997, 40.623406241999895], [-74.17519625699995, 40.62270624399988], [-74.17490328599996, 40.622529106999885], [-74.17459706299995, 40.622364010999874], [-74.17427887999997, 40.62221179199985], [-74.17395016799996, 40.622073168999854], [-74.17361243999996, 40.621948740999855], [-74.1732673079999, 40.621838983999915], [-74.17270752199991, 40.62171153799988], [-74.17236260999994, 40.62163579899988], [-74.17207112899989, 40.62157862099991], [-74.17181177999997, 40.62154338399994], [-74.17095546499992, 40.621442180999935], [-74.16985740699987, 40.621312386999925], [-74.16953656499994, 40.621270499999916], [-74.16923624499995, 40.6212164329999], [-74.16888894299989, 40.62115114299992], [-74.16878471999995, 40.621124028999866], [-74.16847311299996, 40.62104581499989], [-74.16808086299993, 40.62093625599991], [-74.16769710399988, 40.62079027499995], [-74.16742666699999, 40.620677283999896], [-74.16603193299996, 40.62009453499998], [-74.16570682499993, 40.61995403599987], [-74.16484190399989, 40.61957240499988], [-74.16394094899994, 40.619143261999945], [-74.16337391799988, 40.61885458999991], [-74.1623545779999, 40.618335637999955], [-74.16163010299996, 40.61793493599989], [-74.16052177399989, 40.61722978899994], [-74.15986704899998, 40.61681322499988], [-74.15912798199999, 40.61629637499986], [-74.15861741699992, 40.61593931499988], [-74.15792673899992, 40.615457424999896], [-74.157373837, 40.615031910999924], [-74.15651187899992, 40.61435448199988], [-74.15479265199997, 40.61303223099987], [-74.15474080199988, 40.6129923179999], [-74.15386832299995, 40.612320627999964], [-74.15379116699992, 40.612255611999906], [-74.15371834199986, 40.612206280999864], [-74.15317275299992, 40.611761521999874], [-74.15120348199993, 40.61026499499993], [-74.15031900699995, 40.60959281499989], [-74.15018666499996, 40.60949781099991], [-74.14998477099994, 40.6095384179999], [-74.14954343599997, 40.60962717199986], [-74.148783295, 40.609199986999904], [-74.14852820199994, 40.60908079499994], [-74.14826911999995, 40.60896670999988], [-74.14800622599995, 40.608857807999875], [-74.14773969799992, 40.608754164999915], [-74.14746971699995, 40.60865584899989], [-74.14719646699989, 40.60856292999989], [-74.14690609299997, 40.60846833999988], [-74.14661053099996, 40.608382978999906], [-74.14631056999997, 40.60830708799991], [-74.14600697999994, 40.60824083499989], [-74.14575166799996, 40.60818646199988], [-74.14549412999996, 40.608138571999866], [-74.14523464999989, 40.608097218999895], [-74.14497351299995, 40.60806244799991], [-74.14471100599997, 40.60803429699994], [-74.14444741899989, 40.608012796999915], [-74.14418304099992, 40.60799797199993], [-74.14391816399994, 40.60798983699987], [-74.142832906, 40.60799259499987], [-74.14188196399996, 40.60804294099991], [-74.14092528299996, 40.60808963199984], [-74.13997406999988, 40.60814639199986], [-74.13903904999992, 40.608195812999924], [-74.13810528799995, 40.60824232499986], [-74.1371740599999, 40.60829269799993], [-74.1359058139999, 40.6083589069999], [-74.13497712799997, 40.60841117899987], [-74.13404435499996, 40.60845319899988], [-74.13311710799991, 40.608505831999864], [-74.13215051199998, 40.608551963999936], [-74.13211741399986, 40.60826434899993], [-74.13210459699991, 40.608152923999924], [-74.131568734, 40.60818117199994], [-74.13094002299995, 40.60821431199985], [-74.12983323699997, 40.60830394299994], [-74.12874897499987, 40.608391740999934], [-74.12784267099993, 40.60850028899994], [-74.12742960799994, 40.60854975999994], [-74.12674361599996, 40.60863191499992], [-74.12650911499996, 40.60866290799986], [-74.12630497799988, 40.608699642999966], [-74.12520040399997, 40.60889840499993], [-74.1251456319999, 40.608908470999914], [-74.12452618799996, 40.60902231899992], [-74.12398126100001, 40.60913521899988], [-74.12350639099996, 40.609233601999975], [-74.12326521999991, 40.60929134799988], [-74.12252324799988, 40.609468999999905], [-74.12247214699988, 40.609481234999905], [-74.12135179599994, 40.609765267999926], [-74.1213358189999, 40.609666497999875], [-74.12127253299997, 40.60936216999995], [-74.12116879899992, 40.60883437599989], [-74.12079903099993, 40.60887493299986], [-74.12066342799993, 40.60888980599989], [-74.1205232869999, 40.60890517599993], [-74.120254056, 40.6089347039999], [-74.11913216599991, 40.6090577409999], [-74.11897246899993, 40.60907525399993], [-74.11883362299994, 40.60909048099991], [-74.1186902039999, 40.60910620799989], [-74.11848942399992, 40.60912822499994], [-74.11839688799994, 40.60913837299989], [-74.11834442099989, 40.609144125999904], [-74.11742466500002, 40.609289008999916], [-74.11706965299993, 40.60820624999994], [-74.116493189, 40.60765675199984], [-74.1163204779999, 40.6075755289999], [-74.11625356799993, 40.60753644099988], [-74.11615857899993, 40.60748094799987], [-74.1160100819999, 40.6073743499999], [-74.11587716399993, 40.60725747099985], [-74.1158562429999, 40.60715118399992], [-74.11579141499993, 40.60686116499993], [-74.11567482099998, 40.60613313799993], [-74.11554728399992, 40.60539592799987], [-74.11544265199991, 40.60469494899995], [-74.11537754999995, 40.60398418699989], [-74.11530272499996, 40.603868255999906], [-74.11511342599988, 40.60253381699985], [-74.11487058499988, 40.602136659999864], [-74.11500030999999, 40.60207428599988], [-74.11684368799999, 40.60189480699987], [-74.11777623899994, 40.60186024799992], [-74.11790155199996, 40.601856453999865], [-74.11957594399998, 40.60181702899996], [-74.12059546299996, 40.601809757999874], [-74.12055576699996, 40.6010452919999], [-74.12056668799994, 40.60095567899986], [-74.12056671099998, 40.60095547199994], [-74.12056678099997, 40.60095524599989], [-74.12059572199999, 40.6008671179999], [-74.12059578099999, 40.600866936999886], [-74.12059587499996, 40.60086676699992], [-74.12064219099997, 40.600782950999914], [-74.1206422609999, 40.60078282499992], [-74.12064234399995, 40.60078271599989], [-74.12070424299995, 40.600705980999884], [-74.12229424999991, 40.59968717799989], [-74.12264348599997, 40.59941858499993], [-74.123185791, 40.59919511599987], [-74.12322499799997, 40.59909695299992], [-74.12399136699995, 40.598714810999894], [-74.12454297999987, 40.598395257999876], [-74.12462019999995, 40.59834793499992], [-74.12470164799997, 40.5983049259999], [-74.12478690899994, 40.59826644999988], [-74.12487554999991, 40.59823270299993], [-74.12496712099988, 40.5982038559999], [-74.12506115499997, 40.59818005599987], [-74.12515717499994, 40.5981614239999], [-74.12525469300002, 40.59814805499989], [-74.12535321299993, 40.598140016999864], [-74.12545223399988, 40.59813734999987], [-74.12714563699993, 40.59833036199988], [-74.12798105699996, 40.59842483599991], [-74.12888881199997, 40.59854783299993], [-74.12982988799996, 40.59867157699992], [-74.13080319499996, 40.5988420859999], [-74.13091909399999, 40.598855047999905], [-74.13095007699992, 40.59873715199991], [-74.13099847699996, 40.59862154199993], [-74.13106374499998, 40.598510423999905], [-74.13114463199999, 40.59840588899992], [-74.13123935899998, 40.598309748999924], [-74.13205430899994, 40.59781870599993], [-74.1328130649999, 40.59724097399986], [-74.13385485799996, 40.596357845999904], [-74.13510176799998, 40.594535032999936], [-74.13544074099998, 40.593816942999844], [-74.13549571999992, 40.59370046799996], [-74.13639761399997, 40.591973456999895], [-74.13659957099995, 40.59166794799989], [-74.13681568999996, 40.59136787899989], [-74.13704557999992, 40.591073817999856], [-74.13728879499998, 40.590786314999875], [-74.13849744199997, 40.58931370499991], [-74.13864458599996, 40.58906080499989], [-74.14168277499994, 40.58874118499995], [-74.14230337999994, 40.588725478999955], [-74.14387326699992, 40.588615298999926], [-74.14583447999993, 40.58856291899987], [-74.14636402099994, 40.58855786899987], [-74.14645464799997, 40.58855885299993], [-74.14684720299991, 40.58857108499991], [-74.1477604, 40.58855595599992], [-74.14790576499993, 40.58855634599993], [-74.14805282699994, 40.588570806999904], [-74.1481980769999, 40.588599680999934], [-74.14833787899993, 40.588642607999915], [-74.14846882599996, 40.588698540999886], [-74.14858798599987, 40.58876582799994], [-74.14869312199986, 40.5888423539999], [-74.14972777099995, 40.589492503999885], [-74.15096968899994, 40.59017657399992], [-74.152162103, 40.590836192999845], [-74.15235208199991, 40.59093696199996], [-74.15450856599989, 40.59213609399994], [-74.15537259699995, 40.59259733799994], [-74.1561030049999, 40.59304747599991], [-74.15636568699993, 40.5931951279999], [-74.15795821599993, 40.59408704499986], [-74.15800268899991, 40.59412020399985], [-74.1584322029999, 40.59435961999991], [-74.15891672599996, 40.59462968499985], [-74.15989227899996, 40.595170140999855], [-74.16025756099997, 40.59534393799988], [-74.16180204399988, 40.59530144799985], [-74.16189940599996, 40.59529877099993], [-74.16188699199998, 40.59500712699986], [-74.16188546199997, 40.5949903349999], [-74.16188841099995, 40.59483745799984], [-74.16190818299995, 40.59468531299989], [-74.16194448999997, 40.594535495999885], [-74.16200333499995, 40.59432685099992], [-74.16208738699994, 40.594083113999865], [-74.1621900049999, 40.5938431059999], [-74.1623107339999, 40.59360803399994], [-74.16244891299996, 40.59337908599984], [-74.16260370599998, 40.5931573659999], [-74.16279187799998, 40.59295768699991], [-74.16299668999994, 40.59276682799989], [-74.16321702299987, 40.59258595999989], [-74.16345157099991, 40.592416147999856], [-74.16369888199995, 40.59225833899993], [-74.16363304299993, 40.59240052699992], [-74.16362405299995, 40.59241993699985], [-74.16356570299993, 40.592586526999874], [-74.16352460800002, 40.59275663099986], [-74.16350123699995, 40.59292868299986], [-74.16349571699995, 40.59310106099989], [-74.163507855, 40.59327217899987], [-74.16360268199993, 40.593374198999854], [-74.16439050499997, 40.59348246099993], [-74.166390049, 40.59373792599991], [-74.167784501, 40.59418566599993], [-74.16886362399997, 40.59472808699995], [-74.16828458099997, 40.59643316499995], [-74.1679964909999, 40.597922359999885], [-74.16762086799996, 40.599849696999925], [-74.16719634899995, 40.602075311999926], [-74.16900518099992, 40.60227337499987], [-74.17062978699994, 40.60243131199995], [-74.1709288839999, 40.60245585699987], [-74.17122684699993, 40.60249861199986], [-74.17152081799995, 40.602559508999875], [-74.17180796899991, 40.60263814499992], [-74.1720855709999, 40.602733770999926], [-74.172351093, 40.60284533499991], [-74.17260229799992, 40.602971490999934], [-74.17488959499995, 40.60221657299994], [-74.17582749599997, 40.60152685299988], [-74.17822457299994, 40.5998280609999], [-74.17932277499995, 40.59676057899984], [-74.18183084799989, 40.59409424599988], [-74.18468224999998, 40.59147895099995], [-74.18683246699995, 40.5890713609999], [-74.18730248699987, 40.588488221999896], [-74.185371048, 40.58745101499987], [-74.18551828799991, 40.58743794299994], [-74.18566449699989, 40.58741932799991], [-74.18580931799991, 40.58739521799987], [-74.18595239599988, 40.587365671999876], [-74.18609337999993, 40.58733075999993], [-74.18623192299994, 40.5872905709999], [-74.18636768699997, 40.587245200999895], [-74.18650033699991, 40.58719476299993], [-74.18681414999996, 40.58710057399991], [-74.18705055599993, 40.587083020999955], [-74.18719250099991, 40.58704623999991], [-74.18733173099997, 40.587003856999935], [-74.18746786899996, 40.586955989999886], [-74.18760054199991, 40.58690276699988], [-74.18772938699996, 40.58684433499992], [-74.18785405399997, 40.58678085299987], [-74.18797420199998, 40.58671249399986], [-74.18808950399989, 40.586639443999914], [-74.18819964399991, 40.586561903999936], [-74.1883043219999, 40.586480084999884], [-74.18859601799987, 40.586106195999946], [-74.18877046599995, 40.586005800999864], [-74.18895013099997, 40.58591090399989], [-74.18913471499995, 40.58582166299985], [-74.18932391199996, 40.58573822599987], [-74.18951740799994, 40.58566073099988], [-74.18971488199998, 40.58558930699995], [-74.19078902199995, 40.58518836699991], [-74.19131783599987, 40.585704488999966], [-74.19226048499998, 40.5852069689999], [-74.1923211319999, 40.5849132419999], [-74.19251747899993, 40.585158981999875], [-74.193324197, 40.585038280999875], [-74.19423261399992, 40.58514680599991], [-74.19964981299998, 40.58226262599991], [-74.20048848899994, 40.5799497779999], [-74.20051512399985, 40.579960381999875], [-74.2005382919999, 40.579966845999884], [-74.20055646999992, 40.57996640299996], [-74.20058554299995, 40.579961575999945], [-74.20062395199992, 40.57994860899994], [-74.20065720799997, 40.57993400299995], [-74.20069287199986, 40.57991832299988], [-74.20073234099989, 40.5799105679999], [-74.200797772, 40.579904826999915], [-74.200851071, 40.57991170499991], [-74.20094091499986, 40.57991956299993], [-74.20103374199996, 40.57992585099987], [-74.20109537699989, 40.579927500999915], [-74.20115662599996, 40.57993156499993], [-74.20121027499995, 40.57993503899993], [-74.20125254599999, 40.57993721699993], [-74.20128744299997, 40.579941655999875], [-74.2013187079999, 40.57994725599987], [-74.20135420199988, 40.5799583649999], [-74.2013805979999, 40.57997327799995], [-74.20140358699999, 40.57999220499992], [-74.20143434499994, 40.58002080599988], [-74.2014525309999, 40.5800347179999], [-74.20148133499991, 40.58004855599993], [-74.20154141399996, 40.580082349999884], [-74.20158593699988, 40.580103048999874], [-74.201626822, 40.58011151499991], [-74.2016748199999, 40.58011480599988], [-74.20173840199996, 40.58011683599987], [-74.20176039899995, 40.58011715299987], [-74.20179998599995, 40.58012471499987], [-74.20184248899996, 40.58014407399991], [-74.20187663199988, 40.580161496999885], [-74.20191105199993, 40.58017458399995], [-74.20195345799995, 40.58018581899987], [-74.20200397499998, 40.58018899599993], [-74.20204828299998, 40.58018625699991], [-74.20210029899994, 40.58018503899992], [-74.20214420399996, 40.58019190599988], [-74.2021713749999, 40.58020105299987], [-74.20230969599996, 40.580239456999934], [-74.20240475199986, 40.580257130999875], [-74.20251156099994, 40.580279916999906], [-74.20254855399999, 40.58028665799987], [-74.20260239099991, 40.58029205299991], [-74.20264545999996, 40.58029656099989], [-74.20269703699992, 40.580304951999935], [-74.20276536999991, 40.58031819799994], [-74.20281236899989, 40.580324427999884], [-74.20285677399994, 40.58032983999995], [-74.20300418199997, 40.58035008299991], [-74.20309425099991, 40.58036082099994], [-74.20314675299993, 40.58036531099985], [-74.20319223099993, 40.58036874499986], [-74.20323381699988, 40.58037045599987], [-74.20327432799998, 40.580374143999904], [-74.20332116299994, 40.580385643999925], [-74.20337531899995, 40.580402016999905], [-74.20343081399989, 40.58041929399991], [-74.20346962599996, 40.58043264699992], [-74.20350663899988, 40.580436159999884], [-74.20354150699988, 40.580436097999936], [-74.20356291499995, 40.58043415399991], [-74.20366188299988, 40.580420565999894], [-74.2037591199999, 40.5804010189999], [-74.20385400899988, 40.58037563699988], [-74.20394595099991, 40.580344580999906], [-74.20403100999988, 40.580333158999906], [-74.20420118899996, 40.580322231999894], [-74.20437042899988, 40.5803047879999], [-74.20453830399995, 40.58028087099992], [-74.20456367299995, 40.58028061599988], [-74.2045886649999, 40.5802772819999], [-74.2046126399999, 40.580270952999875], [-74.20463498599996, 40.580261791999895], [-74.20465513199991, 40.58025003199989], [-74.20467256099992, 40.58023597499986], [-74.20468682999994, 40.58021997899994], [-74.20469757199993, 40.58020245399992], [-74.20470451399991, 40.580183847999876], [-74.20476467899992, 40.5801428909999], [-74.20482987599989, 40.580106686999926], [-74.20489946199996, 40.580075591999915], [-74.20497275099989, 40.580049913999886], [-74.20504901899987, 40.58002990699992], [-74.20512751399997, 40.580015766999956], [-74.20520746099992, 40.5800076339999], [-74.20523194099997, 40.580010506999926], [-74.20525670800001, 40.58001031599985], [-74.20528110699988, 40.58000706799989], [-74.20530449199997, 40.58000084699988], [-74.20532624599998, 40.579991818999915], [-74.20534579299994, 40.579980220999836], [-74.20536261699996, 40.579966361999915], [-74.2053762709999, 40.5799506069999], [-74.2053863959999, 40.57993337199984], [-74.20543842599989, 40.57990221299989], [-74.20549524799993, 40.5798763739999], [-74.20555593099994, 40.57985627799989], [-74.20561948299995, 40.57984225399984], [-74.20568486299997, 40.57983453099987], [-74.20575100199994, 40.57983323599993], [-74.20583829199992, 40.57984126099989], [-74.2059328709999, 40.5798924379999], [-74.205950418, 40.57995433999984], [-74.20596154297344, 40.580046993882775], [-74.20598641215831, 40.58007698896933], [-74.206003098644, 40.58020374451813], [-74.20601270293663, 40.58025256025641], [-74.20601164437174, 40.58027822941258], [-74.20599824576975, 40.58030169740582], [-74.20598907320168, 40.580318126254994], [-74.20598602922253, 40.580338538196315], [-74.20598768121006, 40.580366848459654], [-74.20599414424598, 40.58039520069104], [-74.20599255081491, 40.5804103839359], [-74.20598341988727, 40.5804239333732], [-74.20594660290547, 40.58045087268133], [-74.20592159850466, 40.580469002521866], [-74.20591861496264, 40.580485222586766], [-74.20591753329506, 40.58051246237593], [-74.2058970544074, 40.58053231652651], [-74.20589079815034, 40.580555207968146], [-74.205820692153, 40.58060282864582], [-74.2057755076905, 40.5806380821521], [-74.20576494221854, 40.58065581116426], [-74.20575982767907, 40.58067672805645], [-74.20575734259258, 40.58070605258599], [-74.2057549623511, 40.58072804280349], [-74.20575332946562, 40.58074584527761], [-74.20576270054559, 40.580763216256166], [-74.20577197243135, 40.58078740068532], [-74.20577441108945, 40.58080890464438], [-74.20576787123194, 40.5808334807646], [-74.20574692749514, 40.580855837444304], [-74.20574262948877, 40.58086785251191], [-74.20574379921223, 40.580882010882334], [-74.2057420614763, 40.58090714934314], [-74.20573808837854, 40.5809443222968], [-74.20572403289702, 40.58096568893218], [-74.20569702651666, 40.58097961010295], [-74.20565901048455, 40.58099448746782], [-74.20561757191867, 40.581008287361946], [-74.20557469907166, 40.58102626850585], [-74.20544302999238, 40.58105345690725], [-74.2054173705693, 40.581069486480324], [-74.20541159080935, 40.581091130787414], [-74.2054067987052, 40.58112048355695], [-74.20539291124268, 40.58113787151845], [-74.20537706597383, 40.58114039319174], [-74.20535699520654, 40.581146311210425], [-74.20532229370895, 40.58116907939564], [-74.20528796822211, 40.58120288159002], [-74.20527220884874, 40.58122913856347], [-74.20526958813123, 40.58125844908295], [-74.2052693620202, 40.58129433506051], [-74.20527202154237, 40.58135114237636], [-74.20526700378379, 40.58137387509614], [-74.20524920561914, 40.581404036139375], [-74.20521287714367, 40.58144284734671], [-74.20516545262045, 40.5814746998999], [-74.20513750637808, 40.5815045113187], [-74.20512521495195, 40.58152628417256], [-74.20512231077548, 40.581547318379506], [-74.20511158049024, 40.58157237186299], [-74.20508136987225, 40.58159946880262], [-74.20506947324093, 40.581632826307484], [-74.20506822436222, 40.581659900026324], [-74.20507853137714, 40.581686193370565], [-74.20508976059689, 40.58169151635698], [-74.2051080040074, 40.58169896450181], [-74.20512209441733, 40.58172298223061], [-74.20512367296969, 40.5817812193346], [-74.20511888390284, 40.581818629769494], [-74.20509095824308, 40.581853408567596], [-74.20507422735601, 40.581882290764085], [-74.20507012398778, 40.581914890524736], [-74.20510186514834, 40.58198215692221], [-74.20512510854586, 40.58202005089318], [-74.20513077033327, 40.582038205882334], [-74.20513225919491, 40.58206652147029], [-74.20512670482992, 40.582083628816626], [-74.20510505123276, 40.582106643192574], [-74.20509671037522, 40.58212909904501], [-74.20509542302041, 40.582165967465954], [-74.20509339544918, 40.58219054972914], [-74.20508854474977, 40.58220819107532], [-74.20506897543812, 40.582225322262474], [-74.20505645907878, 40.58225740345466], [-74.2050551441379, 40.58228572454965], [-74.20504263125663, 40.58231887289908], [-74.20503506784873, 40.58236590499715], [-74.20501903173691, 40.582393183520814], [-74.20500227903955, 40.58241458503253], [-74.20499119588473, 40.58245681563508], [-74.20498291791678, 40.58249957364146], [-74.20498022302412, 40.582534843320424], [-74.20496559453855, 40.58256372107799], [-74.20488662232019, 40.58263439150841], [-74.2048768754869, 40.58265578050752], [-74.20486430388429, 40.58266969383118], [-74.20483141014492, 40.582684178643454], [-74.20480198335079, 40.582685834194976], [-74.20477815309634, 40.58268480833289], [-74.20476976291052, 40.5826912351574], [-74.20476280082798, 40.58270620731277], [-74.20476216027058, 40.58272597764913], [-74.20476221924581, 40.582745213156144], [-74.20475597011422, 40.582764459402675], [-74.20473569721067, 40.582780524117794], [-74.20472240233877, 40.58278695967866], [-74.2047091190272, 40.58279766940677], [-74.20470286889555, 40.582816380896766], [-74.20470294422896, 40.58284095890195], [-74.2047163529982, 40.582871389763234], [-74.20472623268154, 40.5828938137903], [-74.20474594990972, 40.582924766989535], [-74.20475094130767, 40.58295254208531], [-74.20474959018405, 40.58296910783595], [-74.20473981443627, 40.58298088015733], [-74.20472511173142, 40.58298518171855], [-74.20471181593868, 40.58299161727955], [-74.20470483416038, 40.58300017809808], [-74.20470347483324, 40.583014071426284], [-74.20470352560267, 40.58303063518114], [-74.20471128061838, 40.58304558278611], [-74.20472604415846, 40.58306105047994], [-74.20473588871609, 40.583071719500325], [-74.20474573327542, 40.58308238801734], [-74.20475417505162, 40.58309305853082], [-74.20475561223768, 40.58310427678487], [-74.20474934341483, 40.58311711169745], [-74.2047445073956, 40.58314009485838], [-74.20474316697732, 40.58315986761382], [-74.20473901192635, 40.58317590193004], [-74.20472712204311, 40.5831839390858], [-74.20470539425226, 40.58318290895939], [-74.20466404415026, 40.58318191378417], [-74.20463814971185, 40.583193714808], [-74.2046241736578, 40.58320656293811], [-74.20461932144067, 40.583224203762704], [-74.20461938037161, 40.58324343809621], [-74.20461943766831, 40.58326213901755], [-74.20460617051465, 40.58327765697892], [-74.2045970827756, 40.58328515337046], [-74.20458940764408, 40.58329585309735], [-74.20458104289362, 40.583310829413534], [-74.20454341730591, 40.58338195838172], [-74.20453023988733, 40.583426862754415], [-74.20451503396664, 40.58349634887388], [-74.20451027225019, 40.58354284236165], [-74.20449984308463, 40.58357011082869], [-74.20447402884271, 40.58360809223264], [-74.20442721397755, 40.58365359003326], [-74.20440076433692, 40.58371294584188], [-74.20439877211804, 40.58374874716517], [-74.20439752718507, 40.583800043334854], [-74.20440188742126, 40.58385079477768], [-74.20441389194124, 40.58388015932912], [-74.2044470555811, 40.58395383355967], [-74.20445697556904, 40.58398961536097], [-74.20446835234766, 40.58404302457249], [-74.20448598348868, 40.58407932556044], [-74.20453808215798, 40.58415670784161], [-74.20461476082346, 40.58425167683368], [-74.20463030601356, 40.58429332454314], [-74.20463390208104, 40.5843232388013], [-74.20462977804404, 40.58434942720774], [-74.20460253447358, 40.584378328049965], [-74.20442145408362, 40.584519172555986], [-74.2043977140958, 40.58454806711535], [-74.20437680805169, 40.58458657384673], [-74.20436433899089, 40.584634149370665], [-74.20436167594352, 40.58468010435888], [-74.20436741459896, 40.584723372637235], [-74.20439629985479, 40.58477301110756], [-74.20441392548736, 40.58480717510365], [-74.204424569888, 40.584850434650846], [-74.20442328538282, 40.58488837239203], [-74.20441781684238, 40.58493379828689], [-74.20441939116981, 40.5849898977091], [-74.20444200119024, 40.585050232886786], [-74.20443857102863, 40.58507428404445], [-74.20442602919894, 40.585097815150746], [-74.20441210120488, 40.58512722633832], [-74.20441081363548, 40.58516409524352], [-74.2044221001769, 40.58519629868874], [-74.2044004957285, 40.585456911678804], [-74.20441040927199, 40.58549002072008], [-74.20441471663784, 40.585523675327394], [-74.20440428309311, 40.58554933968978], [-74.2043636722888, 40.585561701582], [-74.2043412730567, 40.585570289797026], [-74.20432588794984, 40.585581002712374], [-74.20431982332869, 40.58559826957136], [-74.2043189388622, 40.58560078425905], [-74.20431867099312, 40.58560760721997], [-74.20431828965745, 40.585617349254726], [-74.20431854925825, 40.58561845381977], [-74.20432254891573, 40.5856355079445], [-74.20434224520208, 40.58565898271225], [-74.20435071667102, 40.58567927159017], [-74.20435009526345, 40.58570491849559], [-74.20433621964578, 40.585751428174994], [-74.20430757734005, 40.585781399264775], [-74.20424528742339, 40.585808759236635], [-74.2041738351581, 40.58581957345847], [-74.20413954560352, 40.585836197053474], [-74.20411999295423, 40.58585920699764], [-74.20411940258332, 40.58589554107575], [-74.20414124971829, 40.58593504013657], [-74.20417076912078, 40.58596277116557], [-74.20420542329487, 40.5860135582314], [-74.20422356877546, 40.58604015115936], [-74.20425943665474, 40.58608122937394], [-74.2042749428259, 40.586110054126955], [-74.20427713544649, 40.586139436131056], [-74.20425570487673, 40.58623618316044], [-74.20425857092111, 40.58625648200864], [-74.20425865426664, 40.58628373175722], [-74.20425381709462, 40.586306180807014], [-74.20423984358409, 40.586320097721035], [-74.20422026477975, 40.58633509276627], [-74.20421541115921, 40.58635166523287], [-74.20420228531195, 40.586414202142414], [-74.20420240620957, 40.58645374030289], [-74.20421693369822, 40.58662068340185], [-74.20423545614075, 40.58671896233078], [-74.20423706725386, 40.58679107819999], [-74.20423708035298, 40.58679162499403], [-74.20424560837007, 40.58683007936899], [-74.2042457129535, 40.58686427469632], [-74.20424585675818, 40.58691129402538], [-74.20425040456472, 40.58702348818806], [-74.20425812875469, 40.587257499513534], [-74.20426533002895, 40.587320535042984], [-74.204292152031, 40.5873824661778], [-74.20435412247495, 40.58747960015626], [-74.20437820037156, 40.587560771251454], [-74.2043826647372, 40.58764518375068], [-74.20438579358988, 40.58775203789977], [-74.20438319127823, 40.58781722736447], [-74.20437917821558, 40.58788028165798], [-74.20438633322478, 40.58792835684866], [-74.20439762297147, 40.587952914893705], [-74.2044256759391, 40.587959275849784], [-74.2044901874033, 40.587969848118036], [-74.20454212228326, 40.58799326496621], [-74.2045646538937, 40.588027419694434], [-74.20457893934689, 40.58811502004493], [-74.20458065694802, 40.58821760368262], [-74.20455143291892, 40.588287115234415], [-74.20449833794214, 40.5883427761876], [-74.20443395174978, 40.58837281241694], [-74.20441023417295, 40.5884091871309], [-74.20440523678165, 40.588419602133186], [-74.20437829682214, 40.58842228554259], [-74.20435234806935, 40.588428423012125], [-74.20432814669547, 40.58843783568988], [-74.20430639795251, 40.58845024928142], [-74.20428773562212, 40.58846530204245], [-74.20427270354543, 40.588482555320375], [-74.20426173977556, 40.58850150633709], [-74.20425516381164, 40.58852160283994], [-74.20425316728853, 40.588542259195634], [-74.20425580839121, 40.588562873455004], [-74.20426123461236, 40.58859481624041], [-74.20426068074417, 40.58862702300179], [-74.2042439920589, 40.58864017779802], [-74.20422490770723, 40.58865127921354], [-74.20420385801572, 40.58866007692551], [-74.20418131762762, 40.58866637255661], [-74.20415692245034, 40.58872020425905], [-74.20413021376582, 40.588773393924285], [-74.20412195360132, 40.58882725073073], [-74.20412200974495, 40.58888147441249], [-74.20413038144704, 40.588935321209455], [-74.20414695389636, 40.58898805252939], [-74.20417149979477, 40.589038945078705], [-74.20420368247356, 40.58908730078406], [-74.20432302142383, 40.58913661747327], [-74.204574220731, 40.58871913982805], [-74.20470529848623, 40.58877237083056], [-74.20446810314508, 40.58918269561869], [-74.20443548401086, 40.58921839829688], [-74.204646089432, 40.589285745465865], [-74.20440892551585, 40.589706763811556], [-74.20420300609156, 40.58964297174176], [-74.2041007534238, 40.58984275741241], [-74.20385886609272, 40.590249525361656], [-74.20369974335807, 40.59019990647488], [-74.20394167611974, 40.589807396399316], [-74.20388550456514, 40.5897861107414], [-74.20265866972028, 40.59072214628492], [-74.20263401602924, 40.59069814942304], [-74.20260466555042, 40.59067745053108], [-74.20257136688997, 40.59066057754702], [-74.20253496935247, 40.5906479608263], [-74.20249640127933, 40.59063992216579], [-74.20245664637172, 40.59063666659586], [-74.20241671860074, 40.59063827715165], [-74.20237763634617, 40.590644712755164], [-74.20234039642337, 40.59065580926256], [-74.20230594865892, 40.59067128365125], [-74.20227517166538, 40.59069074123784], [-74.2022168883613, 40.590746953548624], [-74.20215374436117, 40.59080002731068], [-74.20208602968194, 40.59084971875003], [-74.20201405533584, 40.590895799627575], [-74.20246775956575, 40.59083882485734], [-74.2025862272926, 40.59087671114387], [-74.20305254911503, 40.59114054180979], [-74.20296340195651, 40.59123092277659], [-74.20245495063132, 40.59095112585197], [-74.202027339207, 40.59099907929197], [-74.20206727337371, 40.59114453377126], [-74.20208857970688, 40.59117859215291], [-74.20210693333235, 40.59121363537014], [-74.20211289703634, 40.591217990444136], [-74.20211784377658, 40.59122303174096], [-74.20212163976946, 40.59122862292027], [-74.20212418235339, 40.591234612770414], [-74.20212540276478, 40.59124083929775], [-74.20212526799746, 40.591247134107675], [-74.20212378169593, 40.591253326958885], [-74.20212098405622, 40.5912592503675], [-74.20211695073955, 40.59126474413673], [-74.20210582304722, 40.59128520454924], [-74.20209021838782, 40.5913038881921], [-74.20207060766732, 40.591320231239756], [-74.20202398004938, 40.591332929165844], [-74.20197947682976, 40.59134947438202], [-74.20193764229154, 40.5913696645379], [-74.20189898808032, 40.59139325270468], [-74.20187170905275, 40.59143112837132], [-74.20185112965837, 40.591471388735236], [-74.2018376001094, 40.59151334868536], [-74.2018282943085, 40.59154386227341], [-74.2018257314502, 40.591575128655975], [-74.20182998202394, 40.591606287991794], [-74.20184092914558, 40.591636483381876], [-74.2018582717722, 40.59166488443563], [-74.20188153297842, 40.5916907101064], [-74.20192266920789, 40.591715255029854], [-74.20195886491075, 40.59174396877873], [-74.2019893996557, 40.59177627984606], [-74.20200349813028, 40.591780764795], [-74.20230284591744, 40.5914693217743], [-74.20231743569836, 40.59147698486388], [-74.20228727877038, 40.59150864109828], [-74.2023164576832, 40.59152396510266], [-74.20230864003041, 40.59153252120722], [-74.20228057909593, 40.59151634116633], [-74.20210009097973, 40.59170390185628], [-74.20202032457773, 40.59178671501954], [-74.20205960871606, 40.59180885418954], [-74.20200944557338, 40.59186029996112], [-74.2019436762175, 40.59182367412003], [-74.20175478948575, 40.592017324818435], [-74.201824436096, 40.59205295636962], [-74.20178248226532, 40.59209513428535], [-74.20172160160581, 40.5920530216791], [-74.2013033939439, 40.59253148864053], [-74.20128925984358, 40.59252433525516], [-74.20092540496347, 40.59286637930488], [-74.200748808064, 40.59275033941975], [-74.20068976791988, 40.59274995343729], [-74.20063127854338, 40.59275609877191], [-74.20057457669009, 40.59276864548078], [-74.2005208613197, 40.59278732826419], [-74.2004712682456, 40.59281175207506], [-74.20042684611835, 40.59284140047192], [-74.20042886474769, 40.59288645457012], [-74.2004234595963, 40.59293134624284], [-74.20041071589982, 40.5929753674559], [-74.20039083463888, 40.5930178239023], [-74.20036412936977, 40.59305804595363], [-74.20033102128066, 40.59309539922136], [-74.20029203254796, 40.59312929456295], [-74.20022540741584, 40.59316618693115], [-74.20015656826536, 40.59320063639575], [-74.20004403597565, 40.59328578496619], [-74.19993703426019, 40.5933749804701], [-74.19983581328245, 40.59346801439222], [-74.19974060969567, 40.59356466924327], [-74.19965164608844, 40.59366471906851], [-74.19956913046464, 40.59376792997536], [-74.19949325575736, 40.59387406068025], [-74.19942419937705, 40.59398286307282], [-74.19936212279688, 40.59409408279548], [-74.19938898826084, 40.59413827009382], [-74.19942322768752, 40.59417938809712], [-74.19946424899246, 40.59421672575841], [-74.19947352134893, 40.59422115240753], [-74.19948165559656, 40.59422672975697], [-74.19948840886074, 40.594233291276836], [-74.19949357950024, 40.59424064105151], [-74.19949701312818, 40.59424855962953], [-74.19949860722171, 40.59425681057569], [-74.19949831418322, 40.59426514753086], [-74.19949736363506, 40.59428427800823], [-74.19949236458696, 40.594303039103316], [-74.19948344880574, 40.59432093629295], [-74.19947085129925, 40.594337497824924], [-74.19945632117715, 40.59435115761505], [-74.19943168321554, 40.59438458967391], [-74.19939999862336, 40.5944143735898], [-74.1993621758987, 40.594439655354826], [-74.19932799056618, 40.59445624324398], [-74.19929116182526, 40.59446913040079], [-74.1992108242351, 40.59454360947802], [-74.19902465726125, 40.59467644801813], [-74.19894864455006, 40.59478000776039], [-74.19889498982688, 40.59487881648165], [-74.19888888242043, 40.594904756789106], [-74.19887443541094, 40.594944927760366], [-74.19886101146594, 40.594952638759565], [-74.19879713413988, 40.59495018658111], [-74.1987714026433, 40.5949638974913], [-74.19876805458141, 40.59496817478595], [-74.1987323255373, 40.59501179831048], [-74.19869662340403, 40.59506396411391], [-74.19867324216078, 40.595114400808036], [-74.19865548667394, 40.59517251436708], [-74.19863211254362, 40.59522551307075], [-74.19862556263999, 40.59528360775762], [-74.19863022280653, 40.59534339203769], [-74.1986281079241, 40.59538610318732], [-74.19860922087038, 40.595440801926415], [-74.19858473171841, 40.59549551201839], [-74.19856137143587, 40.59555278101238], [-74.19852902294922, 40.595602378432076], [-74.19849556327543, 40.59565539452864], [-74.19845423987307, 40.595701590661854], [-74.19842750755885, 40.5957554493536], [-74.19840296649916, 40.59579307465476], [-74.19835042032791, 40.59583331199777], [-74.19830796614374, 40.595876092257], [-74.19825315828169, 40.595909501097005], [-74.19822305871897, 40.59596165540639], [-74.19819183483902, 40.59601295975829], [-74.19815500349986, 40.596062564800974], [-74.1981204131753, 40.596112165968044], [-74.19808246637238, 40.59616348187958], [-74.19804451037392, 40.59621223576351], [-74.19800880011675, 40.59626183715075], [-74.19800105800269, 40.59629601782259], [-74.19799668454347, 40.59633275472038], [-74.1979654616297, 40.59638405733148], [-74.1979622670406, 40.59644043839561], [-74.19795002851592, 40.59647035480373], [-74.19791431640616, 40.59651910477752], [-74.19788395473405, 40.596548332681856], [-74.19787729703556, 40.59655474226969], [-74.19787520651842, 40.596556754549454], [-74.1978729711544, 40.59655846619547], [-74.19783497175446, 40.59659269895008], [-74.19780037992399, 40.59664144528383], [-74.19776795444966, 40.59666541828954], [-74.19772992710946, 40.596690254488315], [-74.19768636210588, 40.59673645419567], [-74.19763717716219, 40.596776683749475], [-74.19760471617818, 40.59678869984606], [-74.1975935096188, 40.59678871911552], [-74.19755768499311, 40.596799882895525], [-74.1975487296359, 40.596803316062214], [-74.19751847587472, 40.59680336806573], [-74.19751357718701, 40.59679898603677], [-74.19751448668937, 40.59734918605724], [-74.19752237639196, 40.59736454836389], [-74.1975145700716, 40.59737737394275], [-74.19747652740341, 40.597396230279735], [-74.19745761291965, 40.59744238753408], [-74.19745442542755, 40.597501330595534], [-74.19745235168168, 40.597557709712724], [-74.197454761376, 40.59761493418693], [-74.19748291970386, 40.59766357375691], [-74.19748864223308, 40.59770370887348], [-74.19749654625022, 40.59772419571973], [-74.19748090154937, 40.597738742563074], [-74.19743279194556, 40.597764450720845], [-74.19738360796272, 40.59780553606761], [-74.19737584196214, 40.59783202796156], [-74.19736144895344, 40.59789099026308], [-74.19734368335155, 40.59794654041523], [-74.19731244404159, 40.597992719476686], [-74.19729907669395, 40.59802007571798], [-74.1973059170801, 40.59805935586228], [-74.1973340858026, 40.598111411544835], [-74.19737568141731, 40.598156610293344], [-74.19741728170408, 40.59820351882564], [-74.19743845280497, 40.59846063769394], [-74.19833088558589, 40.600656979597765], [-74.19875958205294, 40.60171197517623], [-74.19918333889221, 40.602358736987064], [-74.20014866759907, 40.603683350098635], [-74.20138527472514, 40.60463696620847], [-74.20221849851052, 40.605969679616436], [-74.20281628374593, 40.608270827967345], [-74.20273927047037, 40.60883221126093], [-74.20267299169078, 40.60982786123189], [-74.20230179014108, 40.61187564203906], [-74.20244374449518, 40.61328469393197], [-74.20064229730261, 40.61603323707614], [-74.2003834842897, 40.616428098243766], [-74.20058986446027, 40.617028927389455], [-74.20069001303489, 40.6170415714279], [-74.20076290801349, 40.617053401544396], [-74.20080892377666, 40.617072967329534], [-74.20082804386193, 40.617094288048285], [-74.2008304096492, 40.61713528402795], [-74.20083723821055, 40.61716943818839], [-74.20084742522633, 40.61720273241604], [-74.20084535241513, 40.61725911148558], [-74.20084328605802, 40.61731719833676], [-74.20084680818461, 40.617370150367776], [-74.20086267588432, 40.61742820588297], [-74.20087629624591, 40.61748455752935], [-74.20087646075417, 40.617539223219325], [-74.20087662783489, 40.617594743638776], [-74.20089136968642, 40.61765109282013], [-74.20089938404898, 40.61770659952501], [-74.20090739710211, 40.61776210505858], [-74.20091423238277, 40.61779882239855], [-74.20090986705057, 40.61783812184212], [-74.20087858490696, 40.61787234209637], [-74.20087301283809, 40.61788345734883], [-74.20089110022624, 40.617933820892574], [-74.20089346564374, 40.61797481737089], [-74.2009002735371, 40.61800213820937], [-74.20089138557003, 40.6180286322463], [-74.20087690516628, 40.61805940754891], [-74.20087478740122, 40.61810041085589], [-74.20084680817138, 40.61811498084624], [-74.20081543372865, 40.618119305979896], [-74.20078290449888, 40.618111675720286], [-74.2007637976405, 40.6180946254613], [-74.20074009349516, 40.618040000380496], [-74.20070855668939, 40.61798966046441], [-74.20069507526787, 40.61797943372331], [-74.20064125204006, 40.6179735500668], [-74.20058410068826, 40.617978774357205], [-74.20055275649818, 40.6179924942406], [-74.20054156331808, 40.61799763949357], [-74.20048674969225, 40.61803531795103], [-74.20043867109608, 40.61807554666407], [-74.20041529080609, 40.61812854728366], [-74.20038742362465, 40.618180698137294], [-74.20036292502164, 40.618234553753965], [-74.20034850652021, 40.61828582926975], [-74.20035203856249, 40.6183421985547], [-74.20036566101716, 40.61839940230609], [-74.20039607974422, 40.61845059899734], [-74.20042313487245, 40.61850094798325], [-74.20044231269767, 40.61854191414937], [-74.20048393619727, 40.61859138298328], [-74.20051657149038, 40.61863403463199], [-74.20056712603832, 40.61867152928309], [-74.20063669653429, 40.61869532553015], [-74.20070737819523, 40.61871655559208], [-74.20077247709379, 40.618744629382356], [-74.20080841793948, 40.61876762952103], [-74.20083991450633, 40.61880430260727], [-74.2008344149703, 40.61883933370438], [-74.20081542374747, 40.61886072032839], [-74.20075835642854, 40.618894132527224], [-74.20071136905621, 40.61892496457095], [-74.2006979927439, 40.61894975863334], [-74.20069592745396, 40.61900869919873], [-74.20069049525623, 40.619065938183596], [-74.20068053886425, 40.61911037192587], [-74.20068071349013, 40.61916845485066], [-74.20069092927473, 40.619210291352054], [-74.20070789218401, 40.61925980264958], [-74.20071472324886, 40.61929481103738], [-74.20073954753296, 40.61934943365837], [-74.20075311533593, 40.619387848713856], [-74.20075316068288, 40.61938801381847], [-74.20075764740828, 40.61940406969274], [-74.20077574921811, 40.61945955827678], [-74.20079832794164, 40.61951247751582], [-74.20082091698058, 40.619568813491085], [-74.20085806260202, 40.619619997298756], [-74.20089632653438, 40.61967032658365], [-74.20093569892526, 40.61971638192252], [-74.20096499633247, 40.61976758159085], [-74.2009729628065, 40.619806859469776], [-74.20097089773077, 40.61986579936476], [-74.20097552827605, 40.61991447794746], [-74.20099363018717, 40.61996996699726], [-74.20101618955653, 40.62001605287505], [-74.20102868350264, 40.620070697493205], [-74.20103548328741, 40.620095455805206], [-74.20102657539782, 40.620115117536365], [-74.20100533126885, 40.62013309136913], [-74.20099525859429, 40.62013823520794], [-74.20098303339508, 40.6201732768937], [-74.20098217389514, 40.62019625184807], [-74.20098095847078, 40.620228800715154], [-74.20098449202942, 40.62028516996776], [-74.2009868438432, 40.620321893960075], [-74.2010037523626, 40.62035346912193], [-74.20104077573244, 40.62036365348459], [-74.2010688259972, 40.62037214674219], [-74.2010699746938, 40.620381540777615], [-74.20105206983467, 40.62039182169896], [-74.20099715575674, 40.62039618804552], [-74.20092318554512, 40.620400588846124], [-74.20084920460141, 40.62040157235658], [-74.20077407925909, 40.6203948700693], [-74.20070002640486, 40.62037193646621], [-74.20064051727076, 40.620339582561094], [-74.20057989124972, 40.62030808480545], [-74.20053613507298, 40.62029534783707], [-74.20050588129898, 40.62029967210122], [-74.20046674061955, 40.62033049085503], [-74.20045783099043, 40.62035015204183], [-74.20046240094837, 40.620378331033535], [-74.2004905276734, 40.620412448564615], [-74.20050515347535, 40.620430359238036], [-74.20049514742536, 40.6204577110937], [-74.20047052940691, 40.62047227562743], [-74.20043245103147, 40.62048429891995], [-74.2004280019192, 40.62049541101681], [-74.2004370934674, 40.620536394761025], [-74.20045955296372, 40.620550021963794], [-74.20052688004162, 40.62057296745373], [-74.20056725122792, 40.62057802157745], [-74.2006412284865, 40.620576184693725], [-74.20071518908162, 40.62056836900845], [-74.20077347485226, 40.62056655943544], [-74.20084750825812, 40.6205826573301], [-74.2008733158235, 40.620591155726274], [-74.20087674494054, 40.62061335623617], [-74.20087689917773, 40.62066460634049], [-74.20089935755497, 40.62067737922776], [-74.20092966672648, 40.62069184735474], [-74.20092969500968, 40.620701242846934], [-74.20092192004205, 40.62072517341098], [-74.20093094529145, 40.620743949054905], [-74.2009826019348, 40.6207746087449], [-74.20099047807766, 40.62078399101907], [-74.200986158669, 40.62083866452976], [-74.20100089721804, 40.62089415945369], [-74.20102119611141, 40.62093426882064], [-74.20106836913772, 40.62096493681455], [-74.20108983459889, 40.6210204199609], [-74.20112138194689, 40.621073322938194], [-74.20112483433921, 40.621103212152576], [-74.2011271819975, 40.62113823002875], [-74.20113848207161, 40.621168105512524], [-74.20114977443346, 40.62119541848165], [-74.20117906131713, 40.62124234611224], [-74.20119716912669, 40.62129954340837], [-74.20121079614697, 40.621357602272916], [-74.2012311382086, 40.62141223329329], [-74.20123130559404, 40.62146775368989], [-74.20123366361715, 40.62150618662762], [-74.20125847310123, 40.62155483087214], [-74.20128883426491, 40.621586381431754], [-74.20133145358044, 40.62159399488333], [-74.20138303786867, 40.62159988381704], [-74.20143239998035, 40.62161260896536], [-74.20143578875664, 40.6216211448395], [-74.20142240959545, 40.62164508475218], [-74.20140225479616, 40.62165280831418], [-74.20132939502028, 40.621654643728235], [-74.20130810779165, 40.62165809826718], [-74.2013014781412, 40.62169056765575], [-74.20129941204375, 40.621748654488464], [-74.20129622163314, 40.62180588972821], [-74.20128966370712, 40.62186142022304], [-74.20128300465908, 40.621883640559446], [-74.20126628533909, 40.62191527404903], [-74.20126518935844, 40.6219238177795], [-74.20125970678117, 40.621963973909246], [-74.20124305586253, 40.62201866901081], [-74.20123089203481, 40.62207421098946], [-74.20122657597591, 40.62212973923008], [-74.20123122680886, 40.62218525177478], [-74.20124484631499, 40.62224074861939], [-74.20126182328333, 40.622294530618895], [-74.20127768566681, 40.62235002353239], [-74.2012946695401, 40.622406368711225], [-74.20131276554288, 40.62245929518175], [-74.20132414578988, 40.622515648995446], [-74.20132319150866, 40.6225711718526], [-74.20131102145736, 40.62262415132081], [-74.20128760353833, 40.622665191912695], [-74.20127534263133, 40.622688276286034], [-74.20128435897738, 40.622704488885454], [-74.20130229725306, 40.62270531271301], [-74.20137624811906, 40.62269322412822], [-74.20145020839274, 40.622684552739756], [-74.20152419915728, 40.62268613202665], [-74.20154553334775, 40.62269805202311], [-74.20156132642177, 40.62273048378607], [-74.2015670984474, 40.62278599434843], [-74.20157400062897, 40.6228440637799], [-74.20158537726877, 40.622899565183864], [-74.20160235599974, 40.62295334762954], [-74.20161933837996, 40.62300883852891], [-74.20162847442838, 40.62306434318696], [-74.20163201104123, 40.623121565457346], [-74.2016321787644, 40.62317708584358], [-74.20163346671013, 40.62323260493476], [-74.20163139695073, 40.623289836366894], [-74.20162484432517, 40.62334707750371], [-74.20161267733178, 40.623401764786756], [-74.20159939399505, 40.62345730875961], [-74.20158494435559, 40.62349833368137], [-74.20156258549943, 40.623518873765526], [-74.20151667919954, 40.62353689041871], [-74.20144495652622, 40.62354470486809], [-74.20137209129265, 40.623544832522406], [-74.20134408905345, 40.62355256864429], [-74.20133404548383, 40.62356796088904], [-74.20133748521867, 40.62359357979315], [-74.2013801947626, 40.62363023425228], [-74.20140831691215, 40.62366178852963], [-74.20141176249858, 40.62368911688868], [-74.20139733073569, 40.623736121038014], [-74.20136497069723, 40.62378571911503], [-74.20132924492235, 40.62383446834302], [-74.20127892559044, 40.62387641076836], [-74.20122634786158, 40.623913232071956], [-74.20117823304412, 40.62394321152425], [-74.20113576602957, 40.62398684793748], [-74.2010989240153, 40.624037308010244], [-74.20107662790329, 40.624078346589776], [-74.20108128389592, 40.62413556808517], [-74.20109042715195, 40.62419363529246], [-74.20108835947418, 40.62425172210849], [-74.20110421636812, 40.62430550557715], [-74.20113912157518, 40.62435584063679], [-74.2011448901549, 40.6244104964828], [-74.20114394150578, 40.62446772777967], [-74.20113962867144, 40.624524109558074], [-74.20113418927706, 40.62457963974589], [-74.20111978609876, 40.624636039845775], [-74.20110537966264, 40.62469158571791], [-74.20108872991594, 40.62474628077798], [-74.2010709587324, 40.62480183253036], [-74.20105655441857, 40.624857378391575], [-74.20104439249307, 40.62491377506318], [-74.20103671160895, 40.624969309163575], [-74.20103015573068, 40.62502569435437], [-74.20102247739634, 40.62508208318255], [-74.20101367409276, 40.62513676618451], [-74.20100853518737, 40.62517392375305], [-74.20100599469657, 40.625192300278854], [-74.20099607727869, 40.62524954440204], [-74.20098504239097, 40.625307648059035], [-74.20097848041523, 40.62536232596601], [-74.20098195802754, 40.62540075693589], [-74.2010100798537, 40.62543231080371], [-74.20103706793152, 40.625459598004554], [-74.20105176402082, 40.62550057188019], [-74.2010609057428, 40.62555778435066], [-74.20106108071656, 40.62561586723568], [-74.20106236889201, 40.62567138564773], [-74.20106365794909, 40.625726904058006], [-74.2010615850543, 40.62578328257846], [-74.20105950832702, 40.62583795214059], [-74.20106080033608, 40.62589518001203], [-74.20107666274832, 40.6259506729334], [-74.20109587274136, 40.626001033773036], [-74.20113189608449, 40.62605051331071], [-74.20117014779946, 40.62609571627478], [-74.20118252353812, 40.62611021570943], [-74.20120831355544, 40.626111878368796], [-74.2012161553177, 40.626053771572145], [-74.20141670667907, 40.62607051389567], [-74.20139260637572, 40.62624889054192], [-74.20139258099198, 40.62624907671134], [-74.20098625499442, 40.62934414810753], [-74.2008047968127, 40.62933257144424], [-74.20075483895435, 40.62959463411348], [-74.20062123354481, 40.62958763810686], [-74.20061121332972, 40.629610718480016], [-74.20057987864897, 40.62962956454984], [-74.20052497747263, 40.62964076464405], [-74.2004768171585, 40.629657077504966], [-74.20045119063903, 40.6297100803016], [-74.20043565877464, 40.62976477328592], [-74.20042573785118, 40.629821165953366], [-74.2004236656081, 40.629877543932885], [-74.2004249542151, 40.62993306232537], [-74.20042510042924, 40.62998174919864], [-74.20042414785866, 40.63003812623056], [-74.2004231933758, 40.63009364735962], [-74.20042223823503, 40.63014916966209], [-74.20041343977793, 40.63020555986685], [-74.20040812771613, 40.63025010069453], [-74.20043065947087, 40.63025148374015], [-74.20043452432516, 40.63025172075748], [-74.2004340873544, 40.63023625624201], [-74.20046414832258, 40.630237690989155], [-74.20046497314343, 40.630252262233434], [-74.20047491385961, 40.6302530304447], [-74.20052900571375, 40.630257206431644], [-74.20052899588633, 40.63025393577904], [-74.20061528535054, 40.6302606252933], [-74.20061725250865, 40.63026567721769], [-74.20072423728168, 40.63027500781987], [-74.2007261630681, 40.63026608219535], [-74.2007286257561, 40.630261691547645], [-74.20073195150815, 40.63025764457701], [-74.20073606002079, 40.63025403900084], [-74.20074085209063, 40.63025096187887], [-74.20074621200907, 40.63024848751069], [-74.20075201035624, 40.630246675642184], [-74.20075810712619, 40.63024557002223], [-74.20076435510724, 40.6302451973471], [-74.2007689976836, 40.630245305811385], [-74.20077355840719, 40.63024597590625], [-74.20077792098232, 40.63024719054442], [-74.20078197416555, 40.63024891875351], [-74.200785614603, 40.630251116464954], [-74.2007887494656, 40.63025372763847], [-74.20079129881583, 40.630256685690526], [-74.20079319764672, 40.63025991519234], [-74.20079439753887, 40.630263333793486], [-74.20079486789567, 40.63026685432143], [-74.20079407663107, 40.630275153752514], [-74.2007872998309, 40.63034627446781], [-74.20078612299848, 40.630350226878434], [-74.20078422588257, 40.630354013218], [-74.20078164644988, 40.63035755771076], [-74.20077843632254, 40.63036078942092], [-74.20077465974444, 40.630363643672105], [-74.20076973879723, 40.63036637465731], [-74.20076430389027, 40.63036846483531], [-74.20075849749085, 40.63036985941536], [-74.2007524718044, 40.630370521840916], [-74.20074638478476, 40.63037043474747], [-74.20073858608478, 40.630368794944104], [-74.20073124932135, 40.6303661974001], [-74.20072457555139, 40.63036271329854], [-74.20071874766295, 40.63035843811795], [-74.2007139253638, 40.63035348901557], [-74.20071024080411, 40.63034800161688], [-74.20070779495533, 40.63034212629854], [-74.2007066548433, 40.63033602406799], [-74.20070685171133, 40.630329862151], [-74.20070838016382, 40.63032380940878], [-74.20040311643059, 40.630292362267326], [-74.20038928301837, 40.63037472640552], [-74.20037487473546, 40.63043027215239], [-74.20036382796738, 40.63048495730575], [-74.20035614844026, 40.63054134555402], [-74.20034398232109, 40.63059688738853], [-74.20033518820867, 40.630654986373095], [-74.20032975474884, 40.630712223762785], [-74.20032768364649, 40.63076945747148], [-74.2003222411613, 40.63082413285406], [-74.2003067104728, 40.63087882631436], [-74.20028669542474, 40.63093438131187], [-74.20027229016036, 40.630990781769896], [-74.20025788164706, 40.63104632749833], [-74.20024347054529, 40.631101018495954], [-74.20023018131249, 40.6311557075376], [-74.20021577082558, 40.63121039903252], [-74.20020248154825, 40.631265088070386], [-74.20018694655916, 40.631318927958624], [-74.20018695424578, 40.6313214904694], [-74.20018462159042, 40.63132866946418], [-74.20035346710812, 40.63137723655127], [-74.20036347745564, 40.63135675727859], [-74.20036791091931, 40.63135283394711], [-74.20037313669167, 40.631349524804016], [-74.20037901017413, 40.631346921413794], [-74.20038536884705, 40.631345095812705], [-74.20039203676454, 40.63134409851535], [-74.2003988294248, 40.6313439571169], [-74.20040555887397, 40.631344675530165], [-74.20041203890698, 40.63134623387633], [-74.20041809022051, 40.631348589035866], [-74.20042354537391, 40.6313516758412], [-74.20042825342215, 40.631355408879855], [-74.20043208409267, 40.63135968485808], [-74.20043493139015, 40.631364385459094], [-74.20043671652917, 40.6313693806163], [-74.20043739011423, 40.63137453211302], [-74.20041405776921, 40.631430844447806], [-74.2004117785669, 40.63143625814105], [-74.20040832722417, 40.63144129890702], [-74.20040380373979, 40.63144582069401], [-74.20039833917748, 40.631449692487756], [-74.20039209186776, 40.63145280210655], [-74.20038524282045, 40.63145505945216], [-74.20037799048004, 40.631456399120054], [-74.20037054497672, 40.63145678229455], [-74.20036312203683, 40.63145619787362], [-74.20035593673342, 40.63145466279015], [-74.20034919725371, 40.63145222152195], [-74.200343098868, 40.63144894480252], [-74.2003378182712, 40.63144492757146], [-74.2003335084633, 40.63144028622456], [-74.2003302943169, 40.63143515524033], [-74.20032826895856, 40.631429683284225], [-74.2003274910707, 40.63142402890096], [-74.20032798319177, 40.63141835592081], [-74.20032973106245, 40.631412828712904], [-74.20033268403984, 40.631407607422595], [-74.20033675656403, 40.631402843331756], [-74.2001726540772, 40.631361080530986], [-74.20015700674327, 40.63142916660896], [-74.2001381058405, 40.63148215761576], [-74.20010798593165, 40.63153260612368], [-74.20006776011, 40.631577946658005], [-74.20002079954769, 40.63162073620256], [-74.19997047608236, 40.63166353224651], [-74.19991789471179, 40.63170120597339], [-74.19986082809585, 40.63173888914792], [-74.19980488565277, 40.63177742339801], [-74.19975342724327, 40.63181595033311], [-74.19969860102778, 40.63185277480094], [-74.19967512075753, 40.6318741689966], [-74.19958138195453, 40.631943687757676], [-74.19922396189709, 40.632250661395354], [-74.19888910947763, 40.6325143578124], [-74.19883067612237, 40.63256853505975], [-74.19869022038306, 40.632698760933735], [-74.19856881854885, 40.63276814773782], [-74.19852346205487, 40.632783049024674], [-74.19850080496614, 40.63279791127553], [-74.1984835284849, 40.632828118936764], [-74.1984749664375, 40.63283290963337], [-74.19842463212467, 40.63287314131435], [-74.19838103570827, 40.63291677977717], [-74.19834081295598, 40.63296382798489], [-74.198335259167, 40.6329809201619], [-74.19835215694141, 40.63300822353622], [-74.19839937094993, 40.633049997522555], [-74.19840057855035, 40.633079035920225], [-74.19838273313529, 40.63311067139133], [-74.19837490082801, 40.6331158099516], [-74.19837451123031, 40.63311612792367], [-74.19834358263705, 40.63314148770751], [-74.19828426341229, 40.633175756719254], [-74.1982249336577, 40.63320746369747], [-74.1981599865825, 40.63323490883927], [-74.19809389789796, 40.633255521398304], [-74.19803227375793, 40.633269295570585], [-74.19795938941562, 40.63326685738021], [-74.19791561643243, 40.6332507038661], [-74.19785945083686, 40.63321492603911], [-74.1978032825518, 40.633178294628884], [-74.19774935751734, 40.63314165816059], [-74.19770944064285, 40.63311727343526], [-74.19770892162028, 40.633116957027795], [-74.19768982830384, 40.633105885532935], [-74.19762359861335, 40.63307866501908], [-74.19759109214782, 40.633081284581166], [-74.19756087372008, 40.63309927316573], [-74.19754747163212, 40.6331172345268], [-74.19754725355178, 40.63311755220139], [-74.19752514502034, 40.633149730620595], [-74.19748941010829, 40.633198478608854], [-74.19746144974295, 40.633221589508004], [-74.19739537162705, 40.63324562007614], [-74.19732704477023, 40.63326794432946], [-74.19728675222868, 40.63329193108205], [-74.19727448930175, 40.63331501399527], [-74.19728474265855, 40.6333705170167], [-74.1972871082701, 40.633412366556634], [-74.19726255931032, 40.633451700421], [-74.19722682789016, 40.633501303052284], [-74.19722689857569, 40.63352522054139], [-74.19724948428885, 40.633579847175056], [-74.19726870369098, 40.63363362585192], [-74.19728567947706, 40.63368740954743], [-74.19728584361084, 40.633742929877464], [-74.19727252112973, 40.6337873690232], [-74.19723678514949, 40.63383526387301], [-74.19719319610093, 40.63388231744665], [-74.19714960710607, 40.63392851677314], [-74.19710932737318, 40.63395677325026], [-74.19707345771366, 40.63395939675497], [-74.1970341703711, 40.63394408877805], [-74.19701172021942, 40.63393558661944], [-74.19693997159443, 40.63393827205498], [-74.1968861882006, 40.63394946798221], [-74.19684924611059, 40.63396917735558], [-74.19680564860418, 40.634012814042094], [-74.19677318194454, 40.63402909847446], [-74.19670031696526, 40.63403349463375], [-74.19662858901309, 40.63404301217345], [-74.19658156160821, 40.63406444649691], [-74.19652335974168, 40.63409871268364], [-74.19646965159124, 40.63413553368534], [-74.19642830481342, 40.634182583134894], [-74.19640266897571, 40.63423387772307], [-74.19639381410525, 40.63427318291348], [-74.19639167415882, 40.63430820799612], [-74.19638286583896, 40.63436288891421], [-74.19636957213456, 40.634417577994135], [-74.19635291542002, 40.634472272312514], [-74.19633289229948, 40.63452611931889], [-74.19625489591704, 40.63456079627186], [-74.19618464614511, 40.634625724563676], [-74.19620359550302, 40.634686050233725], [-74.19619879725332, 40.63473021017521], [-74.19616074043309, 40.63477810529293], [-74.19614123036082, 40.63479275899455], [-74.19611983392684, 40.63480696358221], [-74.19609834576855, 40.63482774869209], [-74.19603303929244, 40.63487718727505], [-74.19602261159346, 40.634910511483554], [-74.19603260182834, 40.634973358518344], [-74.19602904691446, 40.63499584880699], [-74.19596386706435, 40.63503051962415], [-74.1958835533078, 40.63506696183833], [-74.19580605679684, 40.63505839723762], [-74.19568609223377, 40.63501686464005]]]}}, {\"id\": \"23\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 24, \"Shape_Leng\": 0.0469999619287, \"Shape_Area\": 6.07235737749e-05, \"zone\": \"Bloomingdale\", \"LocationID\": 24, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95953658899998, 40.798718525999895], [-73.96004456499999, 40.79804123499991], [-73.96147779999993, 40.79865415599994], [-73.96286980099991, 40.79923967699995], [-73.96571144299992, 40.80043806999986], [-73.96775900399992, 40.80130351599994], [-73.96787379699998, 40.80135169799993], [-73.96798415999996, 40.80139826599985], [-73.96858360799983, 40.801635469999916], [-73.97004742199996, 40.80226500999989], [-73.9702159479999, 40.802335850999924], [-73.97027400499987, 40.80235903599987], [-73.97032589799987, 40.80238456099995], [-73.97150381000002, 40.80283773599995], [-73.97250022199995, 40.80321661299996], [-73.97257779799997, 40.80324718399991], [-73.9726574479999, 40.80327651399991], [-73.97279907800002, 40.803329158999894], [-73.97287179090726, 40.8033561875739], [-73.97255429829634, 40.803798634650114], [-73.97217237241792, 40.8043190769591], [-73.97205095521862, 40.80453037600999], [-73.9719529243853, 40.80461970905157], [-73.97181849483539, 40.80471754320449], [-73.97179980150743, 40.80478561548601], [-73.97171200727101, 40.804931691276515], [-73.9716204956932, 40.80504229439937], [-73.97161113391445, 40.80509618747434], [-73.97144210961531, 40.80533442683979], [-73.9712908330033, 40.80552870753318], [-73.97110765876137, 40.80579013958964], [-73.97098767999996, 40.80574290299997], [-73.97084148399995, 40.80568534399987], [-73.97076362299994, 40.80565737299987], [-73.97069326499995, 40.80563026799993], [-73.96978722000001, 40.80530049699996], [-73.96860707399988, 40.804870949999895], [-73.96855913399985, 40.80485358399993], [-73.96849506699984, 40.80483226999993], [-73.96804849299988, 40.804683699999906], [-73.96680327299997, 40.80420756099992], [-73.96670106899995, 40.804168475999916], [-73.96659731599986, 40.804122764999946], [-73.96386126299988, 40.80297203799993], [-73.96107793999975, 40.801800357999845], [-73.95964685399987, 40.80115642299993], [-73.95848111500001, 40.800670477999894], [-73.95817297099987, 40.800582540999876], [-73.95833304999988, 40.80036505399987], [-73.95861879299989, 40.79997702599994], [-73.95907669099992, 40.79935223299986], [-73.95953658899998, 40.798718525999895]]]}}, {\"id\": \"24\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 25, \"Shape_Leng\": 0.0471458199319, \"Shape_Area\": 0.000124168267356, \"zone\": \"Boerum Hill\", \"LocationID\": 25, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98155298299992, 40.689146163999936], [-73.98101786899988, 40.68893792699991], [-73.98092822899991, 40.68890247699988], [-73.9808430809999, 40.688761361999916], [-73.980691182, 40.68851581099992], [-73.98064972099985, 40.688459476999874], [-73.98012756199996, 40.68775000899993], [-73.97984055799984, 40.687344331999896], [-73.97959037699985, 40.68700390199995], [-73.97936297699995, 40.686682000999916], [-73.97909065799995, 40.68629386599988], [-73.97899402899994, 40.686156131999894], [-73.97827416099987, 40.685195951999944], [-73.97804289599988, 40.68485661499987], [-73.97838862599984, 40.68442334199987], [-73.97842970699993, 40.684355612999866], [-73.97876022499986, 40.6837654439999], [-73.97918390599999, 40.68312883799991], [-73.97961592499983, 40.682493556999916], [-73.98003510099997, 40.68185262699988], [-73.98046155400003, 40.68122254699992], [-73.98088932299994, 40.6805933489999], [-73.98132413699994, 40.67995210099991], [-73.98175642299985, 40.67931462399987], [-73.98185715599982, 40.67935261499984], [-73.98413878099997, 40.68024537099991], [-73.98610388199985, 40.68100852199994], [-73.98653302999993, 40.680372638999884], [-73.98729306099992, 40.6806696979999], [-73.9876680399999, 40.68081780999993], [-73.98722413899985, 40.68144730199988], [-73.98827189199999, 40.68186309999987], [-73.99042374299992, 40.68269393299993], [-73.99262292599985, 40.68354371199993], [-73.99490113599977, 40.6844302379999], [-73.99457806799994, 40.68510014199988], [-73.99440802399982, 40.685452495999925], [-73.99425367999989, 40.685777996999924], [-73.99409147700004, 40.68611343899993], [-73.9939292959999, 40.686452870999865], [-73.99378736999994, 40.68675596699995], [-73.99360423199991, 40.6871272299999], [-73.99333822999985, 40.687678796999876], [-73.99326829999998, 40.68782445099991], [-73.99300451699985, 40.68836463599994], [-73.99293238499986, 40.68851457599991], [-73.99268552799995, 40.68903324699992], [-73.99262286099999, 40.68916054899988], [-73.99236366999985, 40.689690123999924], [-73.99070669599995, 40.68913674399992], [-73.99054474599986, 40.689168104999936], [-73.99016617899996, 40.689729533999916], [-73.98973690099999, 40.690359972999886], [-73.98930768499997, 40.690991687999926], [-73.98930270499983, 40.691060225999884], [-73.98902944799994, 40.69212386099992], [-73.98893356299986, 40.69208664099989], [-73.9882070139999, 40.69180695999995], [-73.98734787400004, 40.69145766799992], [-73.98646436299995, 40.69110361099993], [-73.98633134999982, 40.69105050999989], [-73.98536391899995, 40.69066661299989], [-73.98510411999996, 40.69056114499992], [-73.98443216299995, 40.690292171999936], [-73.9840703399998, 40.69014307899988], [-73.98372011099993, 40.69000671899992], [-73.98295657299988, 40.689699730999905], [-73.9820973789999, 40.68935800999991], [-73.98155298299992, 40.689146163999936]]]}}, {\"id\": \"25\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 26, \"Shape_Leng\": 0.12354780707, \"Shape_Area\": 0.000534039927626, \"zone\": \"Borough Park\", \"LocationID\": 26, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98331628499983, 40.6414786819999], [-73.98299419199994, 40.64132499699987], [-73.98244454199998, 40.6415386259999], [-73.98204321799996, 40.64170758199988], [-73.98123240499983, 40.642044912999935], [-73.98042057899988, 40.64238522699991], [-73.98007428599995, 40.640539699999906], [-73.98000693099998, 40.64020883699988], [-73.9799098319998, 40.639686357999906], [-73.97987360300003, 40.63951820599989], [-73.9796770499999, 40.6384647999999], [-73.97955441199994, 40.63781925899984], [-73.9794759209999, 40.63743136399991], [-73.97938644699988, 40.636932162999855], [-73.97928591699991, 40.63641419399991], [-73.97911188699993, 40.63544038599991], [-73.97802159000001, 40.63555270299984], [-73.9768886119999, 40.6356748619999], [-73.97663655999995, 40.6343538209999], [-73.97655064099989, 40.633903508999886], [-73.97643342199997, 40.63328912299986], [-73.97604935699995, 40.63127590599986], [-73.97716511999984, 40.63074665399988], [-73.97699848899988, 40.62987149599992], [-73.97684964299987, 40.62908858099994], [-73.97669604399998, 40.62835456399994], [-73.97657775699989, 40.627573185999886], [-73.9765146209999, 40.62729448999989], [-73.97644970399986, 40.62700825499989], [-73.97623453700001, 40.625976349999895], [-73.9772614999999, 40.625852772999906], [-73.97719665599993, 40.62510197899993], [-73.97710959299997, 40.62494825999989], [-73.976943824, 40.624052749999855], [-73.97683465199992, 40.623481041999895], [-73.97675209599991, 40.623013311999884], [-73.97656583899989, 40.621994902999894], [-73.97695070799993, 40.62163000699997], [-73.97705352899999, 40.62153252199987], [-73.97539380499998, 40.62076998699995], [-73.97682433599995, 40.61867579999989], [-73.97752301899989, 40.61767654099989], [-73.97785009299986, 40.617287605999906], [-73.97562954199996, 40.61594539099984], [-73.97537335500002, 40.6157547219999], [-73.975442767, 40.616114089999925], [-73.97548096999985, 40.616311924999906], [-73.97444328399989, 40.61642164299991], [-73.97335879399992, 40.616541714999904], [-73.97290326899996, 40.61415296799988], [-73.97399319500002, 40.61402989399992], [-73.97517176999993, 40.61472418599991], [-73.97497264699994, 40.61366810999988], [-73.97489295799981, 40.61318636799989], [-73.97477658, 40.6126384749999], [-73.975208294, 40.612885257999906], [-73.977408165, 40.614218112999886], [-73.97963821799982, 40.61556454699991], [-73.98003930399985, 40.61580692999995], [-73.98087912799988, 40.61631443499988], [-73.98186818299985, 40.61691212699987], [-73.98406791799982, 40.61824073899987], [-73.98627524699981, 40.61956661699987], [-73.9856935779999, 40.6201315139999], [-73.98785095499996, 40.62143503199991], [-73.99009356099988, 40.62278895299992], [-73.99254973599997, 40.62427426799996], [-73.99398197100003, 40.62514139899989], [-73.99432497399985, 40.62534290799988], [-73.99489280599995, 40.625461793999925], [-73.99473485699997, 40.6256147409999], [-73.99464972999995, 40.62569717899992], [-73.99507814899985, 40.62580673699993], [-73.99678847599995, 40.626826050999895], [-73.99687778399999, 40.626530782999964], [-73.99701633599989, 40.62606161499991], [-73.99724067099987, 40.62528868699994], [-73.99811614300002, 40.625815706999944], [-74.00031369499995, 40.62714458299991], [-74.00251454799998, 40.62847420999986], [-74.00222793499994, 40.62875116999988], [-74.00193011399998, 40.629032766999885], [-74.00306153999993, 40.6297151749999], [-74.00368597199991, 40.63009178099986], [-74.00413727999998, 40.630363965999905], [-74.00633870699993, 40.63169362599995], [-74.00702302499997, 40.63210999499996], [-74.006806779, 40.63225039099994], [-74.00661914999998, 40.63237545799995], [-74.00630615499989, 40.632584087999916], [-74.00556490399993, 40.63308317599992], [-74.00735636699996, 40.6341636269999], [-74.00675122699992, 40.634744786999875], [-74.00641784699992, 40.635064849999885], [-74.00616926899994, 40.63530350099989], [-74.00558613, 40.63586622899986], [-74.00500346499996, 40.636425234999905], [-74.00442048399997, 40.63698498599989], [-74.00411104099993, 40.63728296099992], [-74.00383793099998, 40.63754593099991], [-74.00325273099993, 40.63810426199991], [-74.00267214299998, 40.63866700999992], [-74.00236943499988, 40.63895767799987], [-74.00209075399988, 40.63922525399994], [-74.00150771299988, 40.63978496199994], [-74.00092541799997, 40.64034590199991], [-74.00066078199991, 40.64059930899991], [-74.00034122099999, 40.6409053159999], [-73.99975690799992, 40.64146714999988], [-73.99917564799993, 40.642025441999856], [-73.99697357399992, 40.64069410499995], [-73.9955103369999, 40.63980966699993], [-73.99479539499987, 40.64028848699991], [-73.99433124099983, 40.64060028799991], [-73.99407691999981, 40.640770401999944], [-73.99336253699994, 40.641247798999885], [-73.99264456099989, 40.64172828999991], [-73.991928124, 40.6422049039999], [-73.99121277399998, 40.64268589099995], [-73.99049634100004, 40.64316444099995], [-73.98977890299996, 40.64364488699989], [-73.98905872499985, 40.64411924099995], [-73.98834986699991, 40.64456224399992], [-73.98749940600001, 40.64404550999989], [-73.98673080599988, 40.64357399299988], [-73.9858964449999, 40.643067594999906], [-73.985095196, 40.642573055999925], [-73.9845871949999, 40.64226200399988], [-73.98430293099989, 40.64208456299985], [-73.98331628499983, 40.6414786819999]]]}}, {\"id\": \"26\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 27, \"Shape_Leng\": 0.202508808518, \"Shape_Area\": 0.00134088762746, \"zone\": \"Breezy Point/Fort Tilden/Riis Beach\", \"LocationID\": 27, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86522555399998, 40.57045847199989], [-73.86454176099993, 40.56924292199994], [-73.86410310099991, 40.569354609999934], [-73.86406802399988, 40.569297978999934], [-73.86316930199989, 40.56773927199992], [-73.86308767499982, 40.5675977289999], [-73.86286892288632, 40.56683057177312], [-73.86267372829568, 40.56651956610252], [-73.86267607982793, 40.5665130138054], [-73.86268391251235, 40.56650565604353], [-73.86269356691223, 40.56650219217743], [-73.86270424269793, 40.5665019973689], [-73.8627141083374, 40.56650509874633], [-73.86277353301229, 40.56657186205274], [-73.86295920590028, 40.566815983072175], [-73.86300482943778, 40.56687595773083], [-73.86319883665385, 40.56697222108389], [-73.86335482591628, 40.56701963007009], [-73.86342360986391, 40.56703174353393], [-73.86349306360215, 40.56703431173402], [-73.86355862677969, 40.567027194390754], [-73.8637303990016, 40.56697901282819], [-73.86387639829695, 40.5669342852578], [-73.86402791878817, 40.56688561150596], [-73.86449981054494, 40.5667240744525], [-73.86459949094905, 40.56669299776952], [-73.86475323748434, 40.566652574352474], [-73.86482648734001, 40.56664084586797], [-73.8648457640142, 40.566637761374906], [-73.8649546750932, 40.566631477600865], [-73.86508926857456, 40.56663218475607], [-73.86525534672059, 40.56662474418802], [-73.86537824806315, 40.56660813782069], [-73.86553757257411, 40.56657161043734], [-73.86557630398882, 40.56655997659382], [-73.86565210793115, 40.56653377766778], [-73.86572546406634, 40.56650378580237], [-73.86579602875463, 40.566470135127446], [-73.86586350645223, 40.5664329680382], [-73.86636470299644, 40.5661813321767], [-73.86644810921857, 40.56613891596792], [-73.86653974501574, 40.566107909500026], [-73.86661720554638, 40.566091942910326], [-73.86653832191413, 40.56595727655682], [-73.86630105072975, 40.565552214298094], [-73.86630941232445, 40.56554606412482], [-73.86654955895546, 40.565953525384245], [-73.86663005715053, 40.566090111654], [-73.86668580404546, 40.56608473784669], [-73.86674197887773, 40.566083433306375], [-73.86679216232595, 40.566084806288956], [-73.86686111965143, 40.566086471070065], [-73.86688401059845, 40.566083976276055], [-73.86696357079242, 40.56606874198119], [-73.86702972224312, 40.56605622921264], [-73.86709181078302, 40.56603859651582], [-73.8671429879069, 40.56602136563421], [-73.86732622021515, 40.56595559545229], [-73.86738139335078, 40.565935398105374], [-73.8674257709762, 40.56591687194776], [-73.86747429909371, 40.56589584615776], [-73.86765236683206, 40.56581373349282], [-73.86768770930918, 40.565796124603466], [-73.86770878117876, 40.56578742257893], [-73.86775063505787, 40.56577283704188], [-73.86785317398673, 40.5657399059368], [-73.86788155843954, 40.56572877272331], [-73.86792367047542, 40.56570957616081], [-73.86797245198414, 40.56568511944022], [-73.86807455478316, 40.56563374527903], [-73.86809782439515, 40.565623947214], [-73.86817969316189, 40.56559372097314], [-73.86823531210628, 40.56556896674802], [-73.86832677099366, 40.56553115811213], [-73.86836484239736, 40.565518909915205], [-73.86837690378205, 40.56551601491614], [-73.8683950442179, 40.5655143882133], [-73.86843272443515, 40.56551492592719], [-73.86845140122098, 40.56551692031767], [-73.86846705188553, 40.565520621216976], [-73.86848241215057, 40.565527663332496], [-73.8684869484588, 40.56552701030156], [-73.86848953699123, 40.565519385463155], [-73.86838610254688, 40.56534185454267], [-73.86823181879357, 40.56507704077842], [-73.8682511755017, 40.565069489459624], [-73.86840555232219, 40.56533537549579], [-73.86852753205115, 40.565545463433686], [-73.86854986356495, 40.565544696910926], [-73.868623218271, 40.56553800877494], [-73.8686901563823, 40.56552808931248], [-73.86893605282155, 40.56547413219297], [-73.86905175794233, 40.56544086306031], [-73.8691282463427, 40.565419625860656], [-73.86924996432192, 40.56538476045332], [-73.86948488954816, 40.56527065197848], [-73.86954872397517, 40.56524111509826], [-73.86968095022635, 40.565187927537636], [-73.86988639456341, 40.56511342587334], [-73.87003028797032, 40.56506022365147], [-73.87031973070592, 40.56493365714443], [-73.87039480972099, 40.56489591950433], [-73.8704872406497, 40.564855428594704], [-73.87062214815259, 40.56481330949413], [-73.87051611850829, 40.56463238223506], [-73.87025285397128, 40.56418310191552], [-73.87027588464693, 40.56417465359739], [-73.87053524869275, 40.564626011194136], [-73.87064094150604, 40.56480994521735], [-73.87064565391573, 40.564809409738025], [-73.87065866951325, 40.56481436836889], [-73.87066630743064, 40.56481525968412], [-73.87079074288935, 40.56479094187672], [-73.87091316206462, 40.56476458551071], [-73.87099970074952, 40.564736415587], [-73.87111211955774, 40.56469089341756], [-73.87116249832623, 40.5646762624261], [-73.87124151827355, 40.56464577884291], [-73.87128982771225, 40.564626210380986], [-73.87136289489302, 40.564596593376976], [-73.87144543470181, 40.56455926799005], [-73.87160225040432, 40.56450287240941], [-73.87170264487447, 40.56445901269151], [-73.87187304106037, 40.564376480329514], [-73.87232617089947, 40.56418954025002], [-73.87250831174484, 40.56412128483286], [-73.8725776351144, 40.56409750600724], [-73.87263971280453, 40.564084184026754], [-73.87264671264475, 40.564041795506355], [-73.87258749014757, 40.56393863747306], [-73.87237878047286, 40.56357511656151], [-73.87239869822375, 40.56357077001371], [-73.87256879318456, 40.56386147776902], [-73.87260883087148, 40.563931033341184], [-73.87266353398269, 40.56402605512359], [-73.87266787468027, 40.564034254826765], [-73.87268736318256, 40.56403769755697], [-73.87270809641154, 40.56405759510705], [-73.8727238972499, 40.56408745638789], [-73.87273512756167, 40.56409968929783], [-73.87274925481708, 40.56410955604112], [-73.8727606175667, 40.564113963607575], [-73.87277678060882, 40.56411502622753], [-73.87283461996127, 40.564113793595965], [-73.87288655736036, 40.56411048323227], [-73.87294196168004, 40.564104259482484], [-73.8729839745517, 40.564098479260004], [-73.87302519640879, 40.56409210308157], [-73.87307650520094, 40.564084334302365], [-73.87314057697263, 40.56407806625708], [-73.87321952830246, 40.56406390618561], [-73.87329889953693, 40.56404075184445], [-73.87333383849683, 40.56402856991333], [-73.87341959004098, 40.56399718206549], [-73.87344692115289, 40.56398714430683], [-73.87347711337766, 40.56397542585057], [-73.87352028757074, 40.563961037853815], [-73.87356559873494, 40.5639462921715], [-73.87361039032855, 40.56393225672879], [-73.87367333727143, 40.56391432438932], [-73.87375288230227, 40.56389248470005], [-73.87379999120962, 40.56387666871009], [-73.87384821426248, 40.563858414354414], [-73.87392010655763, 40.563830469396976], [-73.87398333570002, 40.563807476287614], [-73.87396234686531, 40.56376070639328], [-73.87397732709955, 40.563756544132715], [-73.87399917445005, 40.563805395016864], [-73.87402612883245, 40.56380039114732], [-73.8740716377515, 40.563787599215736], [-73.87412371693783, 40.56377252684616], [-73.8741500028522, 40.5637655322951], [-73.87419364441172, 40.56375361160226], [-73.87426973237943, 40.56373022733147], [-73.87431230105985, 40.56371723389802], [-73.87437714732536, 40.56369965456142], [-73.87443483573853, 40.56368431930004], [-73.87447106976656, 40.56367456058566], [-73.87452022105322, 40.563658170564594], [-73.87455252668062, 40.56364618474654], [-73.87459337507963, 40.56363116305626], [-73.87464053117007, 40.563609196751464], [-73.87465199208484, 40.56359853743912], [-73.8746666375388, 40.56357782411208], [-73.87466953104926, 40.56355873657037], [-73.8746506447683, 40.563511356438624], [-73.87464481920868, 40.56350146330547], [-73.87466540729139, 40.5634976665944], [-73.87468471054004, 40.56353074615861], [-73.87473658078231, 40.56353824072976], [-73.87477351950852, 40.56352994271122], [-73.87484278763579, 40.56351648246642], [-73.87489963722413, 40.563506935889876], [-73.8749561458776, 40.56350301691279], [-73.87500676147575, 40.56349906569543], [-73.87504255093543, 40.56349364684841], [-73.87508297419413, 40.56348490136332], [-73.87512975898886, 40.563471283137496], [-73.87517848346008, 40.5634561347724], [-73.87522720738546, 40.563441031451525], [-73.87524985801689, 40.56343364503572], [-73.8752717516301, 40.56342650925574], [-73.87529181474282, 40.56341936327524], [-73.87530945564228, 40.563412988649645], [-73.87534115876268, 40.56340130769767], [-73.87535587961015, 40.56339632592228], [-73.87538638520446, 40.563387164480275], [-73.87534551646789, 40.56329906659987], [-73.87536087323856, 40.5632958504412], [-73.87540121346792, 40.563382534125715], [-73.87541872053987, 40.56337805929456], [-73.87545505162277, 40.56336697672159], [-73.87547439144372, 40.5633611803831], [-73.87551004856199, 40.56335021450054], [-73.87552758218385, 40.56334468625361], [-73.87555952995204, 40.56333453538936], [-73.87559506972467, 40.56332275096978], [-73.8756290273301, 40.563311368740564], [-73.87565650831084, 40.56330286100388], [-73.87568355361338, 40.56329344377209], [-73.87571355454236, 40.56328274243264], [-73.8757454201404, 40.56327221331274], [-73.87577452358008, 40.56326145704173], [-73.87580575386734, 40.56324963103947], [-73.87583528292514, 40.56323797390256], [-73.8758855622984, 40.563218703170655], [-73.87590988032137, 40.56320917421155], [-73.87595295265639, 40.56319171487047], [-73.87597470538526, 40.56318430003461], [-73.87599724911752, 40.563176660720956], [-73.87603797533089, 40.56316358457966], [-73.87606765576666, 40.56315364740809], [-73.87603074813882, 40.56307447802597], [-73.8760461294172, 40.563070794058625], [-73.87608271947352, 40.563148512787194], [-73.87610050107055, 40.56314306597241], [-73.87612944192581, 40.5631367660658], [-73.87616566871453, 40.56313061825068], [-73.87619188002725, 40.56312557738809], [-73.87623177412057, 40.5631154343994], [-73.87625899418441, 40.563107169993465], [-73.87631318582261, 40.56309172129477], [-73.87633039968337, 40.56308655327291], [-73.87637686168809, 40.56306889923475], [-73.87639149048142, 40.563062655727485], [-73.87641778562529, 40.56305039218808], [-73.87642834027622, 40.56304529706564], [-73.87644818600278, 40.56303453083122], [-73.87646000618406, 40.56302776160095], [-73.87647953341072, 40.563016859989474], [-73.87650201161124, 40.56300598835245], [-73.87653101431876, 40.56299196109574], [-73.87654600306143, 40.56298325208348], [-73.87656188792157, 40.56297473166594], [-73.87657784435744, 40.562966347524736], [-73.8765944878989, 40.56295690131019], [-73.87660652200594, 40.56294942029262], [-73.87662032765036, 40.56294139352161], [-73.87663304695474, 40.56293423925047], [-73.8766627617093, 40.562917746350436], [-73.87667729045369, 40.5629091443813], [-73.87669317480369, 40.562901119665824], [-73.8767132289167, 40.5628921354571], [-73.87673545637608, 40.562883297987966], [-73.8767434906596, 40.562880615074945], [-73.87672645978108, 40.562844521751664], [-73.87674369135848, 40.56284208140833], [-73.8767592984439, 40.56287562490441], [-73.8767730849189, 40.56287213681385], [-73.87678968296791, 40.56286779582757], [-73.87680946313873, 40.562860594328605], [-73.87682916055601, 40.56285288059863], [-73.8768736030291, 40.562835249993334], [-73.87689802684656, 40.562825615059545], [-73.876924708713, 40.562815458171706], [-73.8769513298522, 40.56280511190377], [-73.87696544963367, 40.56280002222658], [-73.87699432547852, 40.56279096616609], [-73.8770082670926, 40.56278647856263], [-73.87704837900323, 40.562772986266374], [-73.87706415438072, 40.56276707762034], [-73.87711007301019, 40.56274879312667], [-73.87712614278729, 40.5627424961178], [-73.87714098654575, 40.56273586625367], [-73.87715607666624, 40.562729650617484], [-73.87717178189162, 40.562722698338064], [-73.87718888135463, 40.56271550328863], [-73.87720520197054, 40.56270777599936], [-73.87722076596584, 40.56270056992197], [-73.8772362246945, 40.562693147951535], [-73.87725221358004, 40.56268579037451], [-73.87726906383583, 40.562679000636635], [-73.87728427215485, 40.56267300139095], [-73.87729838041301, 40.56266793746007], [-73.87732946042871, 40.562660010140895], [-73.87735635919601, 40.56265320331182], [-73.87734281968088, 40.56263229637029], [-73.87736084534804, 40.562628263740656], [-73.87737403170412, 40.562649017013236], [-73.8773836474245, 40.56264675791915], [-73.87740724013115, 40.56264033554793], [-73.87752204544438, 40.562602824449314], [-73.87761725904532, 40.562566914051835], [-73.87769976955727, 40.56253606905333], [-73.87780227013106, 40.56249936530093], [-73.87789681427573, 40.562460598245124], [-73.87797558561535, 40.562427443359745], [-73.87805690415122, 40.56239632678013], [-73.87810489087754, 40.56237092892275], [-73.87819091216764, 40.56233136100081], [-73.87827637443897, 40.56230015802658], [-73.8783713531596, 40.56226309448449], [-73.87849030785846, 40.56221617598844], [-73.87863332705666, 40.56216225056558], [-73.87874838460596, 40.5621214530793], [-73.87886213323743, 40.56207351146075], [-73.87893541975764, 40.5620429796811], [-73.87903836918255, 40.56199864749559], [-73.87920381335462, 40.56191702682255], [-73.87937898313005, 40.561836929087455], [-73.87954024329774, 40.56179666565886], [-73.87954475732901, 40.56179488737246], [-73.87956142278068, 40.56179177907632], [-73.87957863349105, 40.561788032893034], [-73.87959637702109, 40.56178335948525], [-73.87961485607637, 40.56177717388321], [-73.87963567395806, 40.56177046987883], [-73.87965868676253, 40.561763991487865], [-73.87968186281925, 40.561758829210376], [-73.87970635043087, 40.561753091329045], [-73.87973011928683, 40.56174613726036], [-73.87974563530074, 40.56173931778465], [-73.87975883079315, 40.56172967869506], [-73.87977032503349, 40.56171939769128], [-73.87978478609632, 40.56170873279764], [-73.87980665020133, 40.56169739045226], [-73.87982058327975, 40.561691065700856], [-73.87983442103652, 40.56168448821299], [-73.87984807048495, 40.56167855065822], [-73.87986099695851, 40.5616731326935], [-73.87987504764712, 40.561666996864815], [-73.87989003151243, 40.56166059245462], [-73.8799052272357, 40.561654619150794], [-73.87991978375338, 40.561649367063076], [-73.87993381993422, 40.56164408645201], [-73.87994793990492, 40.56163899473306], [-73.87996253102727, 40.56163379595053], [-73.87998694116628, 40.561625168205296], [-73.87999703259649, 40.5616212344257], [-73.88001748502982, 40.561613951824725], [-73.88003818600468, 40.56160612115427], [-73.88005879307917, 40.56159810090572], [-73.8800820945289, 40.56158838119541], [-73.88010722783451, 40.561578249426354], [-73.88013158004648, 40.56156944149529], [-73.8801561087673, 40.56156007436105], [-73.88017659797245, 40.561551865162215], [-73.88019510089887, 40.561544796776765], [-73.88021646533025, 40.56153629095068], [-73.88024009597919, 40.561527202301704], [-73.88026267382189, 40.561519923548374], [-73.88028402529329, 40.56151140831457], [-73.88029663880127, 40.56150310142431], [-73.88029945871983, 40.56149839325516], [-73.88030219429331, 40.56149380394435], [-73.88030376241444, 40.56148194580274], [-73.88029971929609, 40.56147151352376], [-73.88029125467662, 40.5614573387097], [-73.88028328768723, 40.5614426247999], [-73.88028035877844, 40.56142982547907], [-73.88027612406724, 40.561400913782784], [-73.88027653869281, 40.56139974385115], [-73.88028171347875, 40.56138518804984], [-73.88029675110954, 40.561374955249434], [-73.88032213845268, 40.56136801195929], [-73.88033427510743, 40.56136861916831], [-73.88035535042575, 40.56136965886821], [-73.88037097709187, 40.5613797162167], [-73.88038652381759, 40.56140191269535], [-73.88039813555842, 40.561425544993895], [-73.88041490293793, 40.56147157034809], [-73.88044061582691, 40.56147372156319], [-73.88045748805685, 40.561474063630364], [-73.88047833156288, 40.561472581741526], [-73.88050508130863, 40.56146906162662], [-73.88052044713967, 40.56146716006488], [-73.880535067378, 40.56146577874338], [-73.88055180135724, 40.56146444568915], [-73.88056789802403, 40.56146290389764], [-73.88058466554952, 40.561462588114914], [-73.88059874050708, 40.56146210752924], [-73.88062471827277, 40.56146194572576], [-73.88065503286174, 40.561459204441675], [-73.88067122541169, 40.56145686127235], [-73.88068750016801, 40.56145473446717], [-73.88070388287355, 40.561451816193106], [-73.8807208433218, 40.56144900556832], [-73.88073916280933, 40.56144583733556], [-73.88075795457834, 40.56144254277009], [-73.8807759545183, 40.56143970624408], [-73.88079254859147, 40.56143724771121], [-73.88080875189407, 40.56143490503621], [-73.88082570091095, 40.56143209438401], [-73.88084176466914, 40.561429536451215], [-73.88085587891946, 40.56142721857465], [-73.88087985492963, 40.56142345224179], [-73.8809059096621, 40.56141966075196], [-73.88092118030214, 40.561417542756736], [-73.88093924873357, 40.56141583224443], [-73.88095900626, 40.56141431228607], [-73.88097801947205, 40.56141235078193], [-73.88099687882973, 40.56141105554915], [-73.88101282281468, 40.56140926274484], [-73.88104769888399, 40.56140443564484], [-73.88106581723754, 40.56140182485871], [-73.88108276623433, 40.56139901467111], [-73.881099358105, 40.56139741133172], [-73.88111436629285, 40.561396742340264], [-73.88112995376133, 40.561396181500584], [-73.88114925192757, 40.561393815678294], [-73.88116962349282, 40.56139150423785], [-73.88118575602273, 40.5613900354398], [-73.88120008258839, 40.56138813271206], [-73.88121897799168, 40.56138595476507], [-73.88123683590801, 40.56138291932948], [-73.88125385752036, 40.56137928963561], [-73.88126707541673, 40.561376286757216], [-73.88128249221315, 40.56137213387953], [-73.8813042410815, 40.56136548263687], [-73.88132655862354, 40.56135872542872], [-73.88134871078579, 40.56135157954342], [-73.88136894897035, 40.56134482889048], [-73.8813898367198, 40.56133831244134], [-73.8814120452841, 40.561332301617135], [-73.88143293141147, 40.5613266946783], [-73.88145183346806, 40.5613212481918], [-73.88147293606477, 40.56131322820112], [-73.88149581334554, 40.56130271618776], [-73.8815191542814, 40.56129124001096], [-73.88153201189785, 40.561284770034455], [-73.88155470392729, 40.56127305059788], [-73.88157549384806, 40.56126159875125], [-73.88159656803151, 40.5612497607023], [-73.88161925961303, 40.56123803304354], [-73.88164018844435, 40.561227761576404], [-73.8816634567856, 40.56121706045621], [-73.88168899263773, 40.56120577598256], [-73.88170253568676, 40.561199612435054], [-73.88171737737864, 40.5611929545025], [-73.88173260920927, 40.56118616143791], [-73.88174658812723, 40.561179837002754], [-73.8817586184783, 40.56117471543492], [-73.88178192122048, 40.56116404180099], [-73.88180537902329, 40.561152755162624], [-73.88182947646317, 40.5611407313759], [-73.88184233483787, 40.561134295876684], [-73.88185623325523, 40.56112690032671], [-73.88187025034674, 40.56111971950298], [-73.88188268294091, 40.56111344556264], [-73.88189455988073, 40.56110711777507], [-73.88190695714788, 40.561100819168836], [-73.88192107865262, 40.56109470946488], [-73.88193490525215, 40.561088167396484], [-73.88194830614006, 40.5610817512062], [-73.88196126979652, 40.56107551566507], [-73.88197456513642, 40.561068910054466], [-73.88198752963675, 40.56106269143171], [-73.88200050406658, 40.561056446180324], [-73.88203604916899, 40.56104105788076], [-73.88205880591771, 40.561032273702146], [-73.88208199965021, 40.561023319254794], [-73.88210707368597, 40.56101299700819], [-73.8821216897349, 40.56100694323953], [-73.88213620094551, 40.561000672577734], [-73.88214913940051, 40.56099525604307], [-73.8821604122492, 40.56099065634909], [-73.88218306409571, 40.56098167385033], [-73.88220615093074, 40.56097253096465], [-73.8822294874467, 40.56096380112383], [-73.88225563517344, 40.560953588672774], [-73.88228206413865, 40.56094482748039], [-73.88230928304074, 40.56093654354631], [-73.8823345270254, 40.560930338136416], [-73.88235267047222, 40.56092634873712], [-73.88237074147563, 40.56092320578925], [-73.8823851748646, 40.56092056471894], [-73.88241586728287, 40.56091089700157], [-73.8824162120251, 40.56091009589146], [-73.88241602369006, 40.56090951989842], [-73.88235271249685, 40.56078809926085], [-73.882356357125, 40.56077687346529], [-73.88235923039828, 40.56076796180767], [-73.88237441679736, 40.560760430809786], [-73.88238549043537, 40.56076102733196], [-73.88239808836536, 40.56076171554471], [-73.8824195896144, 40.560768401897214], [-73.88242584458052, 40.56077034444328], [-73.8824378051493, 40.560790950968475], [-73.88243908710649, 40.560793816204104], [-73.88245337703015, 40.56082575465579], [-73.88247328322596, 40.56087296216574], [-73.88248668159774, 40.56087529030036], [-73.88250750252948, 40.560879517774076], [-73.88252057000595, 40.5608813954134], [-73.88253888350766, 40.56088149422307], [-73.88255540237668, 40.56088121425246], [-73.88257394133495, 40.560880765966886], [-73.88259575058406, 40.56088059007429], [-73.88261752366901, 40.56088036857243], [-73.88263861248006, 40.56087978684802], [-73.88265599601887, 40.56087821039512], [-73.88267018015226, 40.56087652311708], [-73.88268796496037, 40.56087522650897], [-73.88270678652788, 40.560874831262495], [-73.88272323540367, 40.56087437981323], [-73.882741879012, 40.56087412828751], [-73.88276124168169, 40.56087519260241], [-73.88277761838575, 40.5608755618006], [-73.88280046617912, 40.560875395137984], [-73.88282652775801, 40.56087440398432], [-73.88284902255931, 40.56087360787722], [-73.88287626049771, 40.56087470667475], [-73.88289644771169, 40.56087631209161], [-73.88291645747425, 40.5608790257017], [-73.88293962972233, 40.56088223657724], [-73.88295837856984, 40.560883102212905], [-73.88297874541257, 40.56088411440746], [-73.88300900265625, 40.56088694549323], [-73.88302771502269, 40.560888198577054], [-73.8830503248985, 40.560889427649066], [-73.88307447541696, 40.56088693091244], [-73.88309826218214, 40.56088329912061], [-73.88310982640823, 40.56088034862673], [-73.88313764957451, 40.56087765675655], [-73.8831443819454, 40.560877060331165], [-73.8831633133852, 40.560874504991375], [-73.88318433715789, 40.5608710404139], [-73.88320353254964, 40.56086614293299], [-73.88321840479821, 40.56086286207248], [-73.88323280320117, 40.560859725641855], [-73.88324741562234, 40.56085592148518], [-73.88326552705627, 40.56085009611661], [-73.88328688961892, 40.56084155403374], [-73.8833094351043, 40.56083235542637], [-73.88333046640618, 40.560824190108676], [-73.8833522544527, 40.560815513416806], [-73.8833739347147, 40.56080754861338], [-73.88339201329119, 40.5608006499951], [-73.88340857891116, 40.560793832262824], [-73.88342590281917, 40.56078561927137], [-73.88343539982534, 40.560781003912375], [-73.88344315695069, 40.560777234320746], [-73.88345903958856, 40.56076918186295], [-73.88347557387985, 40.56076041906728], [-73.88349337258666, 40.56075112564446], [-73.88351285915205, 40.560742094104555], [-73.88353281763692, 40.56073288411802], [-73.88355510425974, 40.560722316648295], [-73.88357654153918, 40.56071207281864], [-73.88359905432155, 40.56070091988511], [-73.88362146087441, 40.56068958674489], [-73.8836406196835, 40.56067903367992], [-73.8836593548208, 40.56066768860375], [-73.88368014419221, 40.56065623637554], [-73.88370244284773, 40.56064566888932], [-73.8837275769731, 40.560634546059624], [-73.88374036229729, 40.56062890369553], [-73.8837524963408, 40.56062400869173], [-73.88377569269934, 40.56061408993795], [-73.88380001160759, 40.56060247925819], [-73.88382328178228, 40.56059077805145], [-73.88384669395788, 40.56057850939229], [-73.88386109809427, 40.56057202262581], [-73.88387561981135, 40.56056575125234], [-73.88389070942661, 40.56055956203819], [-73.8839075224937, 40.560552744484184], [-73.88392794090461, 40.560544633889954], [-73.88394940861666, 40.5605367845262], [-73.88396907019593, 40.56052897839929], [-73.88399082036796, 40.5605216432948], [-73.88401473247602, 40.56051327436476], [-73.88403597455215, 40.56050670435022], [-73.88405726297333, 40.56050013387597], [-73.88410568844773, 40.5604844868742], [-73.8841324117759, 40.560476228310336], [-73.8841587684461, 40.56046735788629], [-73.8841840883879, 40.560458189380356], [-73.8842050830714, 40.56045112403434], [-73.88422171797328, 40.56044491773886], [-73.88423691281278, 40.56043894386111], [-73.8842516820894, 40.56043314143822], [-73.88426677146676, 40.560426925369605], [-73.88428154007914, 40.5604211229422], [-73.88429717297159, 40.560414987496486], [-73.8843137017093, 40.5604085565906], [-73.8843310946317, 40.560401847112786], [-73.88434848693704, 40.560395111329605], [-73.88436422437736, 40.56038922861499], [-73.88437913564492, 40.56038361512484], [-73.88439325430404, 40.560378550333034], [-73.88440740771495, 40.56037345809958], [-73.88443206430397, 40.56036525233582], [-73.8844540744506, 40.56035791673506], [-73.88447607272956, 40.560350581788356], [-73.88449720857403, 40.56034352574498], [-73.88451499964208, 40.56033799638523], [-73.88453390201965, 40.56033167538847], [-73.88455439118883, 40.56032346540885], [-73.88457420687197, 40.56031500278364], [-73.88459797973535, 40.56030513822021], [-73.88461097723092, 40.560299937246], [-73.8846249896923, 40.560294656540464], [-73.88463856501663, 40.56028950941811], [-73.88465332350869, 40.56028367996053], [-73.88467004324299, 40.560276636538234], [-73.88468617287215, 40.56026952065097], [-73.88470191292048, 40.5602617104613], [-73.88471693288132, 40.56025443145421], [-73.88473148047956, 40.56024733206578], [-73.88474405393285, 40.56024128412235], [-73.88475532648981, 40.56023667596438], [-73.88477491440844, 40.560229698563596], [-73.88479112502597, 40.560223626789124], [-73.88480416654231, 40.56021287858729], [-73.8848168532933, 40.56020328280155], [-73.88483601878171, 40.56019503576012], [-73.88485800842345, 40.56019240195546], [-73.88488436886665, 40.560182368041204], [-73.88485674870846, 40.56013566602369], [-73.88482942339473, 40.56008946051857], [-73.88482639762242, 40.56008434162391], [-73.88482738580119, 40.56007304171813], [-73.88483809042465, 40.560054717142414], [-73.88485949479873, 40.56004280745799], [-73.88488243929883, 40.560041164372485], [-73.88489701008322, 40.560047599056425], [-73.88490715629744, 40.56006638597498], [-73.88492030518748, 40.56008942529685], [-73.8849443679716, 40.560144678817345], [-73.8849511180102, 40.560160174555385], [-73.88497701148718, 40.56016056939058], [-73.88499384958591, 40.56016039744602], [-73.8850127096114, 40.560157697319156], [-73.88502923309899, 40.56015551637084], [-73.88505227175698, 40.56015381018705], [-73.88508182676415, 40.56015342529315], [-73.88511277798887, 40.56015096173725], [-73.88513088351232, 40.56014836788791], [-73.88514881564575, 40.560145307794244], [-73.8851658160028, 40.56014249052201], [-73.88518128795546, 40.5601398072511], [-73.8851987482445, 40.5601368812044], [-73.8852174405887, 40.560133324463564], [-73.8852341912882, 40.56013006448527], [-73.88524858662313, 40.5601281832679], [-73.88527386894698, 40.560124769303464], [-73.88529648951729, 40.560122201381795], [-73.8853141363416, 40.560121035405395], [-73.88533015232493, 40.56011876829939], [-73.88534667832201, 40.56011562551876], [-73.8853632401859, 40.56011244541259], [-73.88538374267821, 40.560108918596406], [-73.88540837775066, 40.56010429388806], [-73.88543277625296, 40.56009928093931], [-73.88545309071588, 40.56009536575647], [-73.88547531123174, 40.560091153764645], [-73.88550302533781, 40.56008545790457], [-73.88551666325391, 40.560082304011516], [-73.88554427202199, 40.56007641805759], [-73.88555827758914, 40.560072911540416], [-73.88558241626406, 40.56006748332943], [-73.88559871654877, 40.56006478223174], [-73.88561210551558, 40.56006256507402], [-73.88563340220868, 40.56005804638965], [-73.8856508311303, 40.560053658879696], [-73.88567038650775, 40.560049309167745], [-73.8856904900261, 40.56004503354367], [-73.88570870779338, 40.56004202574859], [-73.88572526989816, 40.56003868425911], [-73.88574489795874, 40.56003382950418], [-73.88576087880901, 40.560031534325674], [-73.88577690695864, 40.56002855349206], [-73.88579419469708, 40.56002347961752], [-73.885808948384, 40.560018954394295], [-73.88582778528144, 40.56001342084513], [-73.88584691761402, 40.560008403746096], [-73.88586762480838, 40.56000344081322], [-73.88589285465979, 40.55999623545903], [-73.88590609226951, 40.559992339467115], [-73.88593359689607, 40.55998449522671], [-73.88594540242342, 40.55998079767123], [-73.88597470624889, 40.5599719357883], [-73.88598972144771, 40.559966941382854], [-73.88600862950243, 40.55996049667945], [-73.8860289513432, 40.55995272704177], [-73.88604894163463, 40.55994435982744], [-73.88607110952485, 40.559936177712395], [-73.88608492815555, 40.55993135332302], [-73.88611109684452, 40.5599219190292], [-73.88614853091195, 40.559906710922064], [-73.88616863902541, 40.55989853478779], [-73.88618585685326, 40.55989178213551], [-73.88620281397749, 40.559885524775694], [-73.88622110842529, 40.559878854939434], [-73.88623901467702, 40.5598714435634], [-73.8862589101418, 40.559862878700876], [-73.88628023867297, 40.55985404553071], [-73.88631124047652, 40.559842404749475], [-73.88633210557335, 40.55983469875457], [-73.88634885109194, 40.559828081142044], [-73.88636686065661, 40.55982086853882], [-73.88638642583061, 40.55981262816794], [-73.88640556342533, 40.559804586731325], [-73.8864200715227, 40.55979756013298], [-73.88643674773688, 40.55978970556589], [-73.88644624442207, 40.55978559003428], [-73.88646991785976, 40.55977579249977], [-73.88648678162845, 40.559769373171946], [-73.88650786008945, 40.55976107999615], [-73.88652795651807, 40.55975287748531], [-73.8865440262614, 40.559746945379146], [-73.88656470274908, 40.55973786675081], [-73.8865848474998, 40.55972875190983], [-73.88660159386502, 40.55972108736699], [-73.88661008661221, 40.559716889076086], [-73.88663477870321, 40.55970609939591], [-73.88664823292167, 40.55970005713933], [-73.88666264554954, 40.559693860359204], [-73.88667740176875, 40.559687375764824], [-73.88669180276406, 40.55968117025798], [-73.88671025250878, 40.55967327350798], [-73.88672558815315, 40.55966693271705], [-73.88674273568994, 40.55965998222653], [-73.88676002423266, 40.559653427075666], [-73.88677742056535, 40.55964703620761], [-73.88680568666378, 40.55963611448223], [-73.88682510812184, 40.55962862325619], [-73.88684618561702, 40.55962033052007], [-73.88686545531702, 40.559611521682534], [-73.88688214219336, 40.55960366823392], [-73.88689517092206, 40.559596829427264], [-73.88690777389455, 40.559590153876584], [-73.88692159769377, 40.55958180841585], [-73.88693506601751, 40.559573761309196], [-73.88694127351818, 40.55957027492404], [-73.88696170683093, 40.5595581541445], [-73.88697348578656, 40.55954931044137], [-73.8869831680338, 40.55954089891125], [-73.8869889206047, 40.559532745363036], [-73.88699238967502, 40.55952481456039], [-73.88699518208136, 40.559517732468485], [-73.88699547207565, 40.55951471838837], [-73.8869526003802, 40.559432161692335], [-73.88695038899543, 40.55942845241411], [-73.88694833771328, 40.55942501938266], [-73.88697510801153, 40.55940048299849], [-73.8870058914951, 40.55940590791807], [-73.88701103333392, 40.55941662869161], [-73.88701797484295, 40.55943110725562], [-73.88702989018371, 40.5594582420669], [-73.88706563479928, 40.559539603771576], [-73.88706621334039, 40.55954014562952], [-73.88707182997231, 40.559542104885786], [-73.88708924527616, 40.55954187004284], [-73.8871069574882, 40.559541806202056], [-73.88712214223418, 40.559541271628646], [-73.88714024448572, 40.55954047036431], [-73.88715841708209, 40.559539766836295], [-73.88718163115941, 40.55953938455414], [-73.88719563399836, 40.55953958778435], [-73.8872120089974, 40.55954039644993], [-73.88722917637295, 40.55954077299398], [-73.88724694515936, 40.55954128716527], [-73.8872650100889, 40.55954138447507], [-73.88729132758633, 40.559541987277065], [-73.88731003010243, 40.55954239262288], [-73.88733151943774, 40.55954213426556], [-73.88735455628775, 40.55954186050148], [-73.88737553751076, 40.559541627939694], [-73.88739594207813, 40.55954129713888], [-73.88741472854424, 40.55954076605785], [-73.88743154221267, 40.55954039554417], [-73.88744783717925, 40.559540078296685], [-73.88746871260936, 40.55953966618983], [-73.88750700361999, 40.55953950551237], [-73.88753042924644, 40.559540024175064], [-73.88755174014868, 40.559540459510615], [-73.88757175204198, 40.55954138929165], [-73.88759312195924, 40.55954239059326], [-73.88761179944223, 40.559543184032236], [-73.88762863594864, 40.559543470061634], [-73.88764645226922, 40.55954406446454], [-73.8876609625663, 40.55954423998936], [-73.8876688394054, 40.55954403141244], [-73.88769425074301, 40.55954285021504], [-73.88771534057113, 40.55954138484503], [-73.88773229808184, 40.55954024915129], [-73.8877467994364, 40.55953894855066], [-73.88776681567525, 40.55953729264541], [-73.88778395211988, 40.55953503064802], [-73.88780158577295, 40.559531779367056], [-73.88781954941862, 40.55952861786679], [-73.88783280130949, 40.55952629829073], [-73.88785342476315, 40.55952184270132], [-73.88787524355067, 40.55951557895216], [-73.88789256106391, 40.559510651379526], [-73.88791143846906, 40.559504744271536], [-73.88792461201481, 40.55950028156606], [-73.88794172129407, 40.55949286026788], [-73.88795601960904, 40.55948557142159], [-73.8879666430407, 40.559480646018855], [-73.88798040932177, 40.55947428493372], [-73.88799632718069, 40.55946606870868], [-73.88800753009538, 40.55946073057104], [-73.8880237288675, 40.55945410011914], [-73.88804760878529, 40.55944383086408], [-73.88806295796348, 40.55943651555387], [-73.88807775364894, 40.559429172227354], [-73.88809382369216, 40.55942122736799], [-73.88811183072283, 40.55941364458273], [-73.8881272377495, 40.5594074546186], [-73.88814416728067, 40.55940201449128], [-73.88816231056654, 40.559397665157825], [-73.8881751281128, 40.55939353462369], [-73.88818845551663, 40.55938833356594], [-73.88821164791288, 40.55937973806882], [-73.88823408223281, 40.55937221318295], [-73.88825247898284, 40.55936484703594], [-73.88826990484918, 40.55935866018959], [-73.88828704675458, 40.55935283208105], [-73.8883044874921, 40.559345103801405], [-73.88831780453516, 40.559338335978985], [-73.88833477455884, 40.55933020248467], [-73.88834640400576, 40.559323676774106], [-73.88835120127615, 40.559321465854765], [-73.88837425445327, 40.559310583071266], [-73.8883934805104, 40.55930182168098], [-73.88841196066112, 40.559293554111164], [-73.88843460892724, 40.55928547185446], [-73.88845783242984, 40.55927899190529], [-73.88847432349478, 40.55927392958595], [-73.88848599622588, 40.55927001468934], [-73.88850010155922, 40.55926484115546], [-73.88851497721903, 40.55925914584185], [-73.8885450129397, 40.55924633483122], [-73.88856043165234, 40.559239676071755], [-73.88857518948606, 40.559233764857495], [-73.88858819830041, 40.55922844650834], [-73.88860163108474, 40.55922297544663], [-73.8886153128972, 40.5592179360143], [-73.88864142483696, 40.559207506966594], [-73.88866346050536, 40.5591981893755], [-73.8886823662905, 40.55918978882082], [-73.88870563469126, 40.559177815392125], [-73.88872095207054, 40.559168411200375], [-73.8887338823092, 40.55916093095282], [-73.88874756681224, 40.55915386640597], [-73.88876356449067, 40.559147343879125], [-73.88877861484137, 40.559142504443344], [-73.88879445734517, 40.55913722449878], [-73.88881184628468, 40.559131910365224], [-73.88882876320554, 40.559126712708014], [-73.8888431745663, 40.55912266539673], [-73.88886870236347, 40.55911514354833], [-73.88888522950117, 40.55910961228872], [-73.88889898140337, 40.559104798553264], [-73.88891371347253, 40.55909981505114], [-73.88892976811798, 40.559094418357304], [-73.88894827995512, 40.55908881764011], [-73.88896538647242, 40.55908292623725], [-73.88898168957054, 40.559077511683064], [-73.88899879608117, 40.55907162077792], [-73.88901342180179, 40.55906598831686], [-73.88903236176469, 40.559058650044825], [-73.88904806278256, 40.55905256896803], [-73.88906459552393, 40.559044570642705], [-73.88907860002212, 40.559036703227164], [-73.88908973099831, 40.55903120408575], [-73.88910490444815, 40.55902350029977], [-73.88912460287192, 40.559013640749676], [-73.88914307298448, 40.55900443656022], [-73.88915842352391, 40.55899709430007], [-73.8891750585287, 40.558990329580226], [-73.88920044973277, 40.55897995331932], [-73.8892171555851, 40.55897337696537], [-73.88923433400562, 40.55896661276099], [-73.88924992947923, 40.558960838469666], [-73.88926519408469, 40.55895536792398], [-73.88928182463887, 40.558951673504886], [-73.88929867917054, 40.558948286214004], [-73.88931527232135, 40.55894555521843], [-73.8893351879128, 40.55894061284399], [-73.88933571933883, 40.55894043275875], [-73.88934793480718, 40.55893633887288], [-73.88936900021129, 40.55892812858948], [-73.88938592166666, 40.558920400147834], [-73.88939953358431, 40.55891372194484], [-73.88941636213822, 40.55890478685904], [-73.8894280985923, 40.558897467720286], [-73.8894400605295, 40.558889545689766], [-73.88945256487835, 40.55888223616687], [-73.88946571895634, 40.55887466659123], [-73.88947903966285, 40.55886595478817], [-73.88949428876107, 40.55885589102979], [-73.8895032919375, 40.55885181212977], [-73.8895241314313, 40.55884470009413], [-73.88953821130872, 40.55884100318434], [-73.88955600226384, 40.55883544556875], [-73.88957202650458, 40.55882750984012], [-73.8895834926073, 40.55881961362015], [-73.88959408512527, 40.55881106051208], [-73.88960513997493, 40.55880239861796], [-73.88961883994106, 40.5587923889739], [-73.88959262563444, 40.558698140938716], [-73.88963947542904, 40.55866435179396], [-73.88966114737742, 40.558694192269215], [-73.88971251660583, 40.5587723978264], [-73.88973183055423, 40.558773532957865], [-73.88974665924717, 40.55877433877252], [-73.88976684651834, 40.55877655563952], [-73.8897884365622, 40.55877953912548], [-73.88981016755976, 40.55878282965744], [-73.88982638760736, 40.55878493398074], [-73.88984272640757, 40.55878615570057], [-73.88986547856697, 40.55878639380159], [-73.88988292695555, 40.55878781594024], [-73.88990545270576, 40.55878962055579], [-73.88992527143604, 40.55879315181626], [-73.88994362517927, 40.55879634845245], [-73.88996071980075, 40.5587975807641], [-73.88997827768068, 40.55879720941802], [-73.88999497422743, 40.5587971441591], [-73.89001495135712, 40.558797460321856], [-73.89003085569685, 40.55879734052204], [-73.89004747960237, 40.55879763638038], [-73.89006522538631, 40.55879813920445], [-73.89008167051098, 40.55879909143648], [-73.89010045639388, 40.55879864116776], [-73.89011859438818, 40.55879786593821], [-73.89013605771044, 40.5587967571785], [-73.89015182107171, 40.55879687981027], [-73.8901698816206, 40.558798987018065], [-73.89017732999744, 40.558800138202194], [-73.89018413011773, 40.55880118824803], [-73.89019106912332, 40.558803536783074], [-73.89019916422593, 40.55880625532376], [-73.89020264144463, 40.55880928390291], [-73.89021653553505, 40.55881154962421], [-73.89023464758462, 40.55880464124942], [-73.89025218137611, 40.55879713825806], [-73.89026622960283, 40.55879155065402], [-73.8902820358403, 40.55878616241276], [-73.8902961529898, 40.55878153816775], [-73.89031160456129, 40.558776501396466], [-73.89032621782147, 40.55877179586696], [-73.89034504805443, 40.55876626659655], [-73.89037454574526, 40.55875691190846], [-73.89039100139331, 40.558751848107434], [-73.89040687782477, 40.55874757750802], [-73.89042333184365, 40.55874350397011], [-73.89044251425915, 40.55873872270572], [-73.8904563463809, 40.55873497166995], [-73.89047102896177, 40.55873097936286], [-73.8904887144675, 40.558725231523454], [-73.89050235858414, 40.558720661249225], [-73.89051863774655, 40.558715246454135], [-73.8905491641333, 40.55870489305075], [-73.89057314703759, 40.55869638779369], [-73.89059425787754, 40.55868926644073], [-73.89061099796507, 40.558683653883655], [-73.89062392295904, 40.55867919046423], [-73.89063850105025, 40.55867393653616], [-73.89065244339335, 40.5586679167243], [-73.8906680755495, 40.55866068258737], [-73.89068514879413, 40.55865375556194], [-73.89070326159215, 40.558646333802265], [-73.89071751952281, 40.55864195289293], [-73.89073738740578, 40.55863651517912], [-73.89075892304187, 40.558630187454135], [-73.89077963004345, 40.55862485691262], [-73.89079589630344, 40.558619378907174], [-73.890810771714, 40.558613224934135], [-73.89082208107335, 40.55860758088106], [-73.89083940306162, 40.55859963682764], [-73.89085496498267, 40.55859221261952], [-73.89087643516281, 40.5585822375168], [-73.89089516097698, 40.558574546251144], [-73.89091301383365, 40.55856715218892], [-73.8909297671219, 40.55856054932864], [-73.89094582323652, 40.55855461156789], [-73.89096025963967, 40.55854960122033], [-73.89097938744426, 40.558542153576006], [-73.89099331738521, 40.558536160515736], [-73.89101216480755, 40.55852755965913], [-73.89102836401084, 40.55851993901709], [-73.89104144405664, 40.55851424167256], [-73.89105494917361, 40.55850838390105], [-73.89106808825646, 40.55850284911347], [-73.89108116693396, 40.55849717856786], [-73.89109520382699, 40.55849132079333], [-73.89111285503942, 40.55848403523582], [-73.89112776621204, 40.55847787019866], [-73.89114339681828, 40.558471663121374], [-73.89115908759332, 40.55846505972316], [-73.8911772826573, 40.55845777299581], [-73.89118391048598, 40.5584550860648], [-73.89120803770966, 40.5584448532435], [-73.89122525414977, 40.55843665539153], [-73.89124131255271, 40.55842873755561], [-73.8912568743287, 40.558421890265954], [-73.89127004696967, 40.55841636486876], [-73.89128423749976, 40.558410830549214], [-73.89130299743927, 40.55840370734326], [-73.89131718794901, 40.55839818223389], [-73.89133357396578, 40.558391940161634], [-73.89134873135693, 40.55838629852463], [-73.89137023241307, 40.558379403061025], [-73.89138923834852, 40.558373765058555], [-73.89140483105456, 40.55836949355042], [-73.8914188761831, 40.55836569753456], [-73.89143378338977, 40.558361128343805], [-73.89145192790853, 40.558356254793445], [-73.89146564052045, 40.55835283674026], [-73.89148184843577, 40.55834777305136], [-73.89149498721612, 40.55834224809906], [-73.89150868211286, 40.558335668567075], [-73.89151945556257, 40.558325466958095], [-73.89152846414585, 40.55831834338345], [-73.8915482632456, 40.558311742639354], [-73.89156731550995, 40.5583071773453], [-73.89158297770366, 40.558303518535425], [-73.89159799271951, 40.55829814678933], [-73.89161127463389, 40.558291846797864], [-73.89162655615469, 40.558282369644104], [-73.89164055934089, 40.558275026283454], [-73.89165446820805, 40.55826787046567], [-73.8916682591017, 40.55826058938992], [-73.89168619603952, 40.55825176424637], [-73.89170233654164, 40.55824395230224], [-73.8917141761112, 40.55823892131808], [-73.8917278339071, 40.558233386944096], [-73.89174360874942, 40.558225937016665], [-73.89175592486993, 40.558217708163916], [-73.89176760137714, 40.55821028017018], [-73.89178229273934, 40.55820115374988], [-73.89179238639566, 40.55819468940188], [-73.89180333059387, 40.55818833223582], [-73.8918173438662, 40.55818192535524], [-73.89182486838432, 40.55817957244886], [-73.89185168389997, 40.558178607433305], [-73.89186809765137, 40.558177965165406], [-73.89188170127139, 40.55817643870762], [-73.89189898298577, 40.55817192496629], [-73.89191180091986, 40.55816672299687], [-73.89192354828387, 40.55816042152499], [-73.89193651477026, 40.55815150924031], [-73.89194793452896, 40.55814255160612], [-73.8919576292185, 40.55813476103659], [-73.89197200524211, 40.55812351923029], [-73.89198479627267, 40.558113210748324], [-73.89199719571869, 40.55810463114042], [-73.89200886207534, 40.55809720311257], [-73.8920211869688, 40.558090578338486], [-73.8920300739883, 40.558085228249276], [-73.8920445263986, 40.55807821847862], [-73.89206134338015, 40.558068211305454], [-73.89207391762217, 40.55806100978695], [-73.89208699685538, 40.55804822513736], [-73.89211373696473, 40.55791654611905], [-73.89213864396103, 40.55787154199658], [-73.89216471320687, 40.55786825290236], [-73.8921993963761, 40.557911357569026], [-73.89220367738248, 40.55795112129285], [-73.89226637138374, 40.55802542116516], [-73.89227848852535, 40.55803087222673], [-73.89229729856612, 40.55803709366762], [-73.8923172563169, 40.55804193942072], [-73.89233895484402, 40.5580436347269], [-73.8923581722535, 40.55804650778437], [-73.89237238317288, 40.55804961889782], [-73.8923883154821, 40.5580537037986], [-73.89240443414792, 40.55805977946199], [-73.89240849233514, 40.558061899501595], [-73.89242549285805, 40.558077693727896], [-73.89243810858285, 40.558088907095694], [-73.8924492860879, 40.5580985798506], [-73.89246225917607, 40.55810789526906], [-73.89247739622482, 40.55811424827028], [-73.89249384089474, 40.558116218570916], [-73.89250945943235, 40.55811779054769], [-73.89252745099384, 40.55811929354253], [-73.89254519501212, 40.55812024713994], [-73.89255904386211, 40.55812069215576], [-73.89256782772809, 40.558121276172855], [-73.89259676604209, 40.558121475123855], [-73.89261533023675, 40.558119565051186], [-73.89263311420149, 40.55811807809675], [-73.89265316574712, 40.55811598009951], [-73.89266882387994, 40.55811437354665], [-73.89268761034235, 40.55811384160342], [-73.89270938425692, 40.55811265589837], [-73.89272522039116, 40.55811086019516], [-73.89274005394665, 40.55810870453696], [-73.89275822850786, 40.55810649654415], [-73.8927839978274, 40.558102486315704], [-73.89280034483944, 40.55809865609336], [-73.8928155345823, 40.5580946364851], [-73.89283174003617, 40.55809100431415], [-73.89284949188885, 40.55808748180786], [-73.89286694672911, 40.55808480588546], [-73.89288343253995, 40.55808235371434], [-73.89290109973307, 40.558080119426094], [-73.8929185314161, 40.55807791071519], [-73.89293684750216, 40.558075514019976], [-73.8929565712635, 40.558071795649965], [-73.89297166705921, 40.55806800042365], [-73.89298807345267, 40.558063837013016], [-73.89300506972839, 40.55806028680634], [-73.8930230319097, 40.55805763830519], [-73.89304052257285, 40.55805507832052], [-73.89305931230533, 40.55805207927418], [-73.89307393396348, 40.55804945462687], [-73.89308984296201, 40.55804630848276], [-73.89310568151681, 40.55804298334897], [-73.89312594855188, 40.55803879603688], [-73.89315180317428, 40.55803292153244], [-73.89316803285013, 40.55802895611955], [-73.89318481921103, 40.55802342498507], [-73.89319530403395, 40.558016193925795], [-73.89320861760801, 40.55801153238895], [-73.8932276638708, 40.558011009980724], [-73.893248790743, 40.55800852588513], [-73.89326280142214, 40.55800363162097], [-73.89327827868684, 40.557997107981656], [-73.8932950656039, 40.55799163060862], [-73.89331270247462, 40.55798580271254], [-73.89332746870785, 40.557981484607396], [-73.89334402797203, 40.55797757375552], [-73.89335990249572, 40.55797385174166], [-73.89337632196165, 40.55796917497612], [-73.89339216208175, 40.557964848477944], [-73.89340847281092, 40.55796100054482], [-73.89342424263211, 40.55795654012065], [-73.8934359490259, 40.55795322752472], [-73.89345614580793, 40.557949400781055], [-73.89347065050556, 40.55794557863864], [-73.89348635361986, 40.557937902239985], [-73.89349979678227, 40.5579268838673], [-73.89351478690286, 40.55791479373087], [-73.89353242776691, 40.55790643509797], [-73.8935451768539, 40.55790063010987], [-73.89354955190153, 40.557896762381354], [-73.89355458792144, 40.55789231879493], [-73.8935595172113, 40.5578893781875], [-73.89356665523418, 40.55788511547052], [-73.89358631459312, 40.55787667862058], [-73.89360532636421, 40.55786799623861], [-73.89361865369972, 40.55786279455059], [-73.8936355454664, 40.557857506701225], [-73.89365297118835, 40.557851263585704], [-73.8936676264106, 40.557842136890784], [-73.89367995542506, 40.55783292745856], [-73.89369137329186, 40.55782497750927], [-73.89370398329113, 40.55781738935402], [-73.8937191808256, 40.55780922561365], [-73.89373488375027, 40.55780163043292], [-73.89374749199403, 40.557795554563654], [-73.8937616438375, 40.557790930594926], [-73.89378003300159, 40.55778767792253], [-73.89379029427897, 40.55778656261638], [-73.89381008575477, 40.557784518456536], [-73.8938275574881, 40.55777929310122], [-73.89384242170522, 40.55777214845591], [-73.89386234461092, 40.55776217019205], [-73.89387427890601, 40.55775625635762], [-73.89387797702273, 40.55775442230676], [-73.8938941053769, 40.55774710693721], [-73.89391154277077, 40.557740396049745], [-73.89392471499531, 40.5577358782076], [-73.89394099359602, 40.55773049090422], [-73.89395525321527, 40.55772506554817], [-73.8939708512668, 40.55771722648225], [-73.89398576390695, 40.55770958547193], [-73.89399779211266, 40.55770386051754], [-73.8940138143366, 40.55769586100857], [-73.8940258456031, 40.55768917258961], [-73.89404137259272, 40.55768070269839], [-73.8940541575225, 40.55767440196597], [-73.89407049888413, 40.55766709665221], [-73.8940831766383, 40.557661614871385], [-73.89409993015835, 40.55765452453946], [-73.89411487596625, 40.55764797382773], [-73.89412708245119, 40.557641590799165], [-73.89414340108867, 40.55763364532236], [-73.89415434322012, 40.557627838601135], [-73.89416716493481, 40.55762062770001], [-73.89418431266425, 40.557610792406386], [-73.89419667280882, 40.5576041845194], [-73.89421276567148, 40.55759690458869], [-73.89422328361127, 40.55759098003173], [-73.89423647214896, 40.557582924104615], [-73.89425270890521, 40.55757437526173], [-73.89426830527626, 40.5575675538981], [-73.89427586644453, 40.5575647063165], [-73.8943040279781, 40.557555790160436], [-73.8943238969548, 40.55754927142693], [-73.8943430567223, 40.55754341679638], [-73.89436062314624, 40.55753741745889], [-73.8943777294342, 40.55753153396132], [-73.89439432946315, 40.55752493045477], [-73.8944106466766, 40.557517291853465], [-73.894428581734, 40.557509022814216], [-73.89444046916708, 40.55750351396518], [-73.89445645580385, 40.55749600890995], [-73.89447226487341, 40.55748855813667], [-73.89449181506372, 40.55748247985239], [-73.89450707869523, 40.557477000400205], [-73.89452105398466, 40.55747213241962], [-73.89453427277226, 40.557467695802316], [-73.8945501135457, 40.557462415062396], [-73.89456551849206, 40.557457296926685], [-73.89457849050979, 40.557452364356614], [-73.89459382460936, 40.557446065903875], [-73.89460757906444, 40.55743971222238], [-73.89463210764855, 40.55742818019064], [-73.89464726684704, 40.55742149457613], [-73.89466465983715, 40.557413171225676], [-73.89467979669054, 40.55740546834485], [-73.89469513329593, 40.55739812466174], [-73.89471090600708, 40.55739169156647], [-73.89472783505299, 40.55738547580988], [-73.89474633523552, 40.55737829752368], [-73.8947661476704, 40.55737066039972], [-73.89477504316352, 40.55736740004149], [-73.8948011434219, 40.55735592486592], [-73.89481652964469, 40.557347185185385], [-73.89482859594897, 40.55734005477039], [-73.89484124113268, 40.55733244022019], [-73.89485719504914, 40.55732334036199], [-73.8948763610934, 40.55731342556071], [-73.89489400002807, 40.557306083958345], [-73.89490822437855, 40.55730063097663], [-73.89492507893114, 40.55729678332568], [-73.89494140837533, 40.55729651832173], [-73.89494848366209, 40.5572950393059], [-73.89495142499406, 40.5572939254094], [-73.89495276013942, 40.55729301694315], [-73.89495419117144, 40.5572918929536], [-73.89495517340907, 40.557289822009714], [-73.8949559942491, 40.55728646479082], [-73.89495649475008, 40.55728302435171], [-73.89495626172017, 40.557281475492154], [-73.89489756403287, 40.55714972853696], [-73.8948819661663, 40.5571053905375], [-73.89489010400668, 40.55708833355588], [-73.89489014015683, 40.55708825233697], [-73.8949181726631, 40.5570787962983], [-73.89494218163914, 40.55709000172235], [-73.89495564136901, 40.55710520598147], [-73.89496631694892, 40.557133464806725], [-73.89501734885607, 40.5572601081249], [-73.895017937896, 40.55726084646501], [-73.89501879965948, 40.55726121782891], [-73.89502000255709, 40.55726157794511], [-73.8950457120951, 40.55726554626917], [-73.8950640624076, 40.55727179431212], [-73.89507707602817, 40.55727794801569], [-73.89509113472049, 40.55728677661734], [-73.89510797247821, 40.55729417721755], [-73.89512221087577, 40.55730215124209], [-73.89513439494031, 40.557309951000626], [-73.89514345051357, 40.55731747901628], [-73.89514753304947, 40.55732706428157], [-73.89514852671894, 40.5573405191808], [-73.89514692835664, 40.557351414034585], [-73.89515159156545, 40.55735923439968], [-73.89516164657343, 40.55736885208905], [-73.89517340416194, 40.55737814799899], [-73.89518458696945, 40.557385037150844], [-73.89519982761882, 40.557393318492444], [-73.89521720563008, 40.55740234008633], [-73.89523064412063, 40.55740819712954], [-73.89524370640552, 40.55741317932196], [-73.89526805469517, 40.55741962160901], [-73.89528651630843, 40.55742202540892], [-73.89530166347386, 40.55742297571036], [-73.89531893658403, 40.55742354095342], [-73.89533908996137, 40.5574236943498], [-73.89535719235813, 40.5574223697679], [-73.89537230601208, 40.557421717103324], [-73.89538789129564, 40.55742146006947], [-73.89540595765068, 40.557420567506725], [-73.89542036374307, 40.55741953590894], [-73.89543626972052, 40.55741843380473], [-73.89544354339156, 40.557418053432635], [-73.89547081878321, 40.557416484045575], [-73.89549445853949, 40.55741546033939], [-73.89551188771064, 40.557414270149664], [-73.8955279008104, 40.557412366667016], [-73.89554482506938, 40.55740916732931], [-73.89556565952896, 40.557404575518156], [-73.89558391866811, 40.557401052380484], [-73.89560237805892, 40.55739785392745], [-73.89561699906224, 40.557395211365225], [-73.89563232828618, 40.55739294705717], [-73.89565436553595, 40.55738920285144], [-73.89567357950362, 40.55738595079279], [-73.89569085963721, 40.55738201328512], [-73.89570692406242, 40.55737761530376], [-73.89572107424323, 40.557374053399265], [-73.89573877925031, 40.5573699269612], [-73.89575393356245, 40.557366293423726], [-73.895766501129, 40.557363171350985], [-73.89578748916927, 40.5573583908326], [-73.8958042376851, 40.557354344266095], [-73.89582644044448, 40.557350653954664], [-73.89583376321649, 40.55734103399283], [-73.89583913360062, 40.55733530280698], [-73.89585185203113, 40.557326138392895], [-73.8958669424309, 40.55731784024725], [-73.89587965836685, 40.557310412777895], [-73.89589515186488, 40.55730109775243], [-73.895905849487, 40.55729278640562], [-73.89591892385134, 40.557282559496855], [-73.89593506810212, 40.55727173108796], [-73.89594551358121, 40.557267211548584], [-73.89596128166882, 40.557262803881514], [-73.89598835415671, 40.55725635454412], [-73.89600401650891, 40.55725227948989], [-73.89602108451805, 40.55724784668942], [-73.89604177905677, 40.557242074416926], [-73.8960648951546, 40.557235647782555], [-73.89608718331206, 40.55722941037322], [-73.89610882361058, 40.55722366524659], [-73.89612955327631, 40.557217894162804], [-73.89617571486743, 40.55720522021451], [-73.896193716343, 40.55719967114288], [-73.89621316114835, 40.55719312422436], [-73.89623375248901, 40.55718532621989], [-73.89625477931412, 40.55717828500269], [-73.89627753339919, 40.55716944473003], [-73.8963028574274, 40.55715491460955], [-73.89633257498987, 40.557140172670785], [-73.89635374860089, 40.55712996190202], [-73.89636887458332, 40.55712173676558], [-73.89638072903632, 40.55711411848442], [-73.89639189314381, 40.55710946402104], [-73.89640888902028, 40.55710547253934], [-73.89642634856298, 40.55710035550458], [-73.89644593365945, 40.55709438397091], [-73.89647070447474, 40.557086419481834], [-73.89649489769143, 40.55707839951718], [-73.89652079133806, 40.55706958782534], [-73.8965351208921, 40.55706487304129], [-73.89654869476571, 40.55705964493324], [-73.89657303082267, 40.557050859301526], [-73.89659881984007, 40.557040805766256], [-73.89662413630357, 40.557031939754296], [-73.8966502984294, 40.557025759499666], [-73.89667381225632, 40.55702141315194], [-73.89669126886511, 40.55701731381856], [-73.89670686171098, 40.55701252710642], [-73.89672018771319, 40.5570073525323], [-73.8967319909397, 40.55700228414152], [-73.89674603923494, 40.556995488021734], [-73.89676232388463, 40.55698592162546], [-73.89677993051063, 40.55697655460287], [-73.89679286790525, 40.556970549560404], [-73.89680827343786, 40.55696444035251], [-73.89682509817398, 40.55695701697257], [-73.89684263133437, 40.556949981890874], [-73.89684674230215, 40.55694826622373], [-73.89686257314483, 40.55694167859004], [-73.89688782001124, 40.556931606409684], [-73.89690139405178, 40.556926377087386], [-73.89691461391158, 40.55692149992849], [-73.89694316598658, 40.55691189073978], [-73.89695825050372, 40.55690668856569], [-73.89697236759687, 40.55690154264956], [-73.89699522942668, 40.55689179163874], [-73.8970085611632, 40.5568825820784], [-73.89701959656192, 40.55687031772734], [-73.89702194052781, 40.5568662679505], [-73.8970272086246, 40.55685712319525], [-73.89703572008942, 40.55684239774318], [-73.89704812159891, 40.5568323321039], [-73.89706181021296, 40.55682171811885], [-73.89707345559772, 40.55681113798495], [-73.89708358566833, 40.556795964999594], [-73.89708396311498, 40.55678079148345], [-73.8970794853591, 40.55676738725561], [-73.89707637644929, 40.55675498457543], [-73.89707449198416, 40.55674435612907], [-73.89706961439545, 40.556729718522305], [-73.89706825020895, 40.55671828019991], [-73.89706586648651, 40.55670237578893], [-73.89706380003445, 40.55669496241343], [-73.89706459160598, 40.55666362447431], [-73.89707956116763, 40.55665743308464], [-73.89709684679295, 40.55665631372607], [-73.89711859133806, 40.55665886341114], [-73.89712552628787, 40.5566638591667], [-73.89712899705194, 40.5566722097887], [-73.89712965360629, 40.55667490358769], [-73.89713317547921, 40.55668786501365], [-73.89713692363696, 40.55669965560804], [-73.89713909670532, 40.55671457944371], [-73.8971421344785, 40.5567273971977], [-73.89714624648263, 40.55674125996009], [-73.89715174000722, 40.55675416987645], [-73.89715560272309, 40.556768311855095], [-73.89715972516333, 40.55678239040479], [-73.89716376799583, 40.556794533744885], [-73.8971685651108, 40.55680724317386], [-73.89717146548982, 40.55681814325643], [-73.8971742583602, 40.55682868087554], [-73.89717686335517, 40.556839696456755], [-73.89718544253134, 40.55684974521011], [-73.89718596194871, 40.55685035012306], [-73.89719491170374, 40.556857525399444], [-73.89720641046021, 40.55686628829197], [-73.89722312098311, 40.556879235718824], [-73.89723833191195, 40.55689164872123], [-73.89725586404428, 40.556907667199134], [-73.89726846764735, 40.556919718848434], [-73.89727249952617, 40.5569235761454], [-73.89728569121945, 40.55693657369006], [-73.89729639319526, 40.55694871309642], [-73.89730646248513, 40.556957916644514], [-73.89731702574228, 40.55696813787503], [-73.89733107996251, 40.55698088422068], [-73.89734247111011, 40.556990088947224], [-73.89735688619311, 40.55699842304187], [-73.89738317571349, 40.557008351405194], [-73.89739670068455, 40.557012317697264], [-73.89741076797287, 40.55701628329958], [-73.89742494255967, 40.5570204647744], [-73.8974521682449, 40.5570286757713], [-73.89747886443095, 40.55703569665914], [-73.8974949749068, 40.55703966357232], [-73.89751083763664, 40.557043189993635], [-73.89753033601893, 40.55704771109849], [-73.89754969422717, 40.55705096940199], [-73.89757027835604, 40.55705562482243], [-73.89759035599648, 40.55705926371858], [-73.89760880582952, 40.557061747214036], [-73.89762902628063, 40.55706464675381], [-73.89764889366454, 40.557066889425606], [-73.8976674504286, 40.55706954422247], [-73.89768375286748, 40.557071809511456], [-73.89769954625487, 40.55707415609912], [-73.89771646166686, 40.557076611405705], [-73.89773246916396, 40.55707827365239], [-73.8977478752496, 40.557079800164864], [-73.89776467433823, 40.557081049815494], [-73.89778000860085, 40.5570824320163], [-73.897807315341, 40.557084653806896], [-73.8978294010433, 40.557087716302256], [-73.8978403770865, 40.55709063505544], [-73.89789049233032, 40.55710114337298], [-73.89790819996253, 40.55710321336677], [-73.89792895544443, 40.55710414032963], [-73.89795109363448, 40.55710390854008], [-73.89797204009112, 40.557103188845744], [-73.89799514796813, 40.55710213718018], [-73.89801627293306, 40.55710002160966], [-73.89803275813097, 40.55709855075931], [-73.89804859220533, 40.557097852814124], [-73.8980659371215, 40.557097094724284], [-73.8980844780186, 40.55709461615587], [-73.89810877058365, 40.55709060392503], [-73.89813040714631, 40.557086588830586], [-73.89814966647052, 40.5570845697532], [-73.89817420196248, 40.55708459150909], [-73.89819087384122, 40.55708428295651], [-73.89820767867585, 40.55708191155608], [-73.89823432134004, 40.55707770269115], [-73.89824828256276, 40.557074337825625], [-73.89826081312597, 40.557071169878974], [-73.89828608894082, 40.557065204747936], [-73.89831103530717, 40.557058642578106], [-73.89833278263819, 40.55705182728118], [-73.89833569993478, 40.55705087527831], [-73.89835175395899, 40.55704563910081], [-73.89837620677743, 40.55703704316941], [-73.89839853425389, 40.557029318686105], [-73.89842095425452, 40.55702184574273], [-73.89844375413135, 40.557014013109054], [-73.8984653939166, 40.557006980740866], [-73.89848519167198, 40.55700142401907], [-73.89850283567749, 40.55699765796057], [-73.89850415960983, 40.556997443854726], [-73.89851450413911, 40.556995759436205], [-73.89851946372956, 40.55699495247123], [-73.89854145485809, 40.556990721834175], [-73.89856820836926, 40.55698363147538], [-73.89858221850213, 40.55697927684581], [-73.89859642933534, 40.55697438110259], [-73.89861242394521, 40.55696890175033], [-73.89863179528275, 40.55696323657013], [-73.89865213659446, 40.55695669823901], [-73.89866213109286, 40.55695321020645], [-73.89867377844706, 40.55694914498283], [-73.89869631793299, 40.55694163640726], [-73.89872169283505, 40.55693255370361], [-73.89874579035636, 40.55692456065547], [-73.8987674201223, 40.556916997486745], [-73.89879060968802, 40.55690894867906], [-73.89881272425498, 40.55690063003408], [-73.89883436626594, 40.55689309316581], [-73.89885272293415, 40.556887183801], [-73.89887292268962, 40.55688039334118], [-73.89889380717528, 40.556873108932294], [-73.89891429061703, 40.55686681443596], [-73.89893448970109, 40.55686002396461], [-73.89895716970719, 40.5568528031103], [-73.89898051279202, 40.556845025127714], [-73.899001751476, 40.55683851499893], [-73.89901964668839, 40.55683271392858], [-73.89903742560581, 40.55682668658794], [-73.89905510745946, 40.55682151557216], [-73.89907191645871, 40.556816613844624], [-73.89908534721569, 40.5568121762199], [-73.8990952588455, 40.55680884454961], [-73.89911718426394, 40.55680035379417], [-73.8991319889622, 40.55679355854963], [-73.89913667895024, 40.556785232746535], [-73.8991361980517, 40.55677539714582], [-73.8991316409269, 40.55676089778553], [-73.89912776302342, 40.556747954871945], [-73.899127252222, 40.556744978422046], [-73.89912496638853, 40.55673156195941], [-73.89912301755608, 40.556719997500714], [-73.89912014069037, 40.55670713655059], [-73.899116482255, 40.55669140730439], [-73.89911530531842, 40.556678064858666], [-73.8991148690234, 40.556664876185685], [-73.89911559256481, 40.55666282642276], [-73.89911845657572, 40.55665466872397], [-73.89912302657942, 40.55665146671824], [-73.89912765821087, 40.55665152674128], [-73.89914364231204, 40.55665525676008], [-73.89915940253864, 40.55665960526806], [-73.89916984107614, 40.55666573530582], [-73.89917752317201, 40.55667933978263], [-73.89918229566776, 40.55669318328314], [-73.89918743781125, 40.55670500368147], [-73.89919277230247, 40.556715557722384], [-73.8991936285606, 40.55671725923757], [-73.89919887524741, 40.556729816356224], [-73.89920303747549, 40.556742264968655], [-73.89920745561433, 40.556755866076216], [-73.89921440127033, 40.55676770732204], [-73.89921479106768, 40.556767905349396], [-73.89922957396107, 40.55677540992367], [-73.89924945831343, 40.55677934204017], [-73.89926717190949, 40.55678034131357], [-73.89928589176947, 40.556779848272605], [-73.89930741712652, 40.55677882192768], [-73.89933282135017, 40.556777297226695], [-73.8993620690193, 40.556775311997626], [-73.89937802152436, 40.556773943685776], [-73.89939368875535, 40.55677297970663], [-73.89940979201283, 40.55677185360319], [-73.89942671038659, 40.55677028911546], [-73.89947468412873, 40.55676557209403], [-73.89949229981046, 40.55676332636019], [-73.8995088666736, 40.556761077528186], [-73.89952643464986, 40.55675885804957], [-73.8995428953049, 40.55675639384371], [-73.89955889586747, 40.556754027573284], [-73.89957409399733, 40.556752136551225], [-73.89959008378196, 40.55674977897862], [-73.89960629673782, 40.55674785365834], [-73.8996219310278, 40.55674585595963], [-73.89963666996877, 40.556744044607434], [-73.89965186792827, 40.556742117891204], [-73.89966750247613, 40.556740092711735], [-73.89968428255023, 40.55673729605025], [-73.89969927029304, 40.55673493708442], [-73.89971092077198, 40.556733112638575], [-73.89973007219342, 40.556729420583395], [-73.8997504281423, 40.556725011543264], [-73.899776266027, 40.556720341581084], [-73.89980106402501, 40.556716595135896], [-73.89981738310247, 40.55671434654569], [-73.89983703692896, 40.5567126520351], [-73.8998392362297, 40.556709454634365], [-73.89983906428724, 40.55670796129038], [-73.89983798033158, 40.55669887117745], [-73.89983344253606, 40.55668573873839], [-73.89982902357353, 40.556672165631724], [-73.89982422386589, 40.55665558067514], [-73.89981878639196, 40.556635273652574], [-73.89981540903364, 40.556620218144076], [-73.89981014077418, 40.556602787254555], [-73.89980740755739, 40.556589109738674], [-73.89980742054901, 40.5565842547332], [-73.89980743284025, 40.55657942720206], [-73.89980743937654, 40.55657697272702], [-73.89980794971703, 40.55656255303516], [-73.89980821300148, 40.55655688974863], [-73.89980874945697, 40.55654570776259], [-73.8998153126919, 40.55653798633839], [-73.89982713792294, 40.55653238624398], [-73.89983435952055, 40.55653354749377], [-73.89984389934072, 40.556540530418864], [-73.89985134924063, 40.556549253329294], [-73.89985460397277, 40.55655307032085], [-73.89986043658175, 40.556562689948166], [-73.89986280526094, 40.55656708917102], [-73.89986697964667, 40.556574872229575], [-73.89987355930764, 40.5565874063553], [-73.89987775633037, 40.55659987256426], [-73.89988092664278, 40.55661341610056], [-73.89988436306146, 40.556628058194896], [-73.89989007266728, 40.55664398856507], [-73.89989330450702, 40.556656345205184], [-73.89989600818473, 40.55666805471938], [-73.89990042650166, 40.556681627822904], [-73.89990114258784, 40.55669568018572], [-73.89990465549945, 40.55670002830306], [-73.89990511327498, 40.55670059579077], [-73.89990525489465, 40.5567006315981], [-73.89991054300054, 40.55670205032669], [-73.89992717337739, 40.55670254462433], [-73.89994434628511, 40.55670303939224], [-73.89996296313342, 40.55670076581088], [-73.89998081022584, 40.55669959071996], [-73.89999989596794, 40.5566988201166], [-73.90002112804261, 40.556697216409624], [-73.9000445342698, 40.55669380141463], [-73.9000681069652, 40.556689667353886], [-73.90009143132299, 40.556686062299214], [-73.90012215906638, 40.55668192188677], [-73.9001399862018, 40.55668013292412], [-73.90015874425124, 40.556678166348476], [-73.90017786507306, 40.556676992334125], [-73.90019780082703, 40.55667535178384], [-73.90021788945259, 40.55667400722068], [-73.90023732798454, 40.55667335382035], [-73.90025486708208, 40.55667308525868], [-73.90027092129414, 40.55667285879508], [-73.90028590299302, 40.55667261263472], [-73.90029725478689, 40.55667221740271], [-73.90032064342567, 40.55667085926856], [-73.90034986166143, 40.55666676137776], [-73.9003679049026, 40.55666336411251], [-73.90038436789672, 40.55665988371926], [-73.90039911024817, 40.556657084180756], [-73.90041463921894, 40.556654833647784], [-73.90042800925514, 40.55665332741929], [-73.90044816625621, 40.556650899794256], [-73.90046042736351, 40.55664742563202], [-73.90047944554115, 40.55664038960705], [-73.90049893890898, 40.556632257172254], [-73.90051229863236, 40.556626461824145], [-73.90051999127944, 40.55662312911406], [-73.90054055315451, 40.5566171099823], [-73.90055342537856, 40.556612115167844], [-73.90056152745782, 40.55660626584008], [-73.90056171599474, 40.556601933009674], [-73.90056532353363, 40.556584048797234], [-73.90056628452307, 40.556572812393014], [-73.90056822671836, 40.556560516709055], [-73.90056916442192, 40.556545001738314], [-73.90056656114633, 40.556535414595814], [-73.90056204279021, 40.55651931508015], [-73.90055881847431, 40.556504628626946], [-73.90055216760874, 40.556492482645645], [-73.90054602978998, 40.55647786336152], [-73.90054673933396, 40.55646865954759], [-73.90055243595958, 40.55645858169286], [-73.90055529740901, 40.55645497256074], [-73.90056044243232, 40.55644848003805], [-73.90056883774035, 40.55643863235962], [-73.90058060462704, 40.556432922907675], [-73.90059328462958, 40.55643367989423], [-73.90060403979463, 40.55643994460217], [-73.90061118090355, 40.55644973771218], [-73.90061192070245, 40.5564507539187], [-73.90061827579478, 40.556462990274376], [-73.9006253594575, 40.55647655795008], [-73.90063179349308, 40.55648991866656], [-73.90063752822185, 40.55650546202725], [-73.90063922086148, 40.556520208636705], [-73.90063905316975, 40.556539033492555], [-73.90064338752742, 40.55655731710083], [-73.90064734466947, 40.55657181588195], [-73.90065294273536, 40.5565850309605], [-73.90065660661142, 40.55659881748549], [-73.90065887908541, 40.55660871829085], [-73.90068163438659, 40.556627201777715], [-73.90075764067583, 40.55665491762217], [-73.90087569826764, 40.55668952438501], [-73.90099617912736, 40.556718873414695], [-73.9011186811175, 40.55674286675045], [-73.90124279535341, 40.55676142430721], [-73.90129932513871, 40.55676801494544], [-73.9013790712455, 40.556773489641394], [-73.9015975687611, 40.55678467017266], [-73.90181646619372, 40.55678949224021], [-73.90203544556263, 40.556787948839165], [-73.90211475525074, 40.556785818469656], [-73.90234210194804, 40.55677506579338], [-73.90256871118143, 40.55675744877271], [-73.90279422679731, 40.55673299509556], [-73.90301829436225, 40.556701743194935], [-73.90324056171957, 40.55666374218826], [-73.90346067954327, 40.55661905180018], [-73.90367830188872, 40.556567742268676], [-73.9040586929248, 40.556471486868126], [-73.90443651253365, 40.55636951103587], [-73.90481161283611, 40.55626185468858], [-73.90518384701974, 40.55614855996597], [-73.90555306939628, 40.55602967121492], [-73.90591913545822, 40.55590523497158], [-73.9062540843815, 40.555811201833535], [-73.90658574923901, 40.555710625377294], [-73.90691391033238, 40.55560357223045], [-73.90723835028673, 40.55549011331065], [-73.90755885419611, 40.555370323778796], [-73.90787520976478, 40.555244282989], [-73.90818720744909, 40.55511207443619], [-73.90849464059502, 40.55497378570035], [-73.90879730557717, 40.55482950838891], [-73.91173957602177, 40.553708247752844], [-73.91625684813306, 40.55197106561608], [-73.91701439187267, 40.55162381230808], [-73.91780653110628, 40.55127929509524], [-73.91866643765113, 40.55100383757692], [-73.91948122277843, 40.55062481534694], [-73.9264190939679, 40.54815578031995], [-73.92774080155928, 40.547628052975284], [-73.92887232480227, 40.547093874203505], [-73.93054682243256, 40.54642198013397], [-73.93256825181622, 40.54568875406016], [-73.93408287789984, 40.545164662494585], [-73.93443022334034, 40.54506977725986], [-73.93477468405521, 40.54496895875239], [-73.93511608589348, 40.544862257947116], [-73.9354542562537, 40.54474972879316], [-73.93578902417048, 40.544631428186165], [-73.93612022040107, 40.54450741593956], [-73.9364476775119, 40.5443777547543], [-73.93677122996246, 40.54424251018703], [-73.93709071418934, 40.54410175061698], [-73.93740596868945, 40.54395554721121], [-73.93771683410068, 40.543803973888956], [-73.93802315328331, 40.543647107283576], [-73.93818951458262, 40.54355291606897], [-73.93836059279961, 40.54346377778862], [-73.93853612436767, 40.543379829761406], [-73.93871583886235, 40.543301201310186], [-73.93889945941925, 40.543228013562256], [-73.93908670315919, 40.54316037926307], [-73.93927728162387, 40.543098402602986], [-73.93947090122087, 40.54304217905631], [-73.93966726367522, 40.54299179523458], [-73.93986606648828, 40.54294732875307], [-73.9400670034048, 40.542908848111445], [-73.94026976488325, 40.542876412588114], [-73.9404740385729, 40.542850072149186], [-73.94055905861713, 40.541846776959794], [-73.94073681682697, 40.54182008707755], [-73.94012975795786, 40.54800256129352], [-73.94012808934745, 40.549516289012324], [-73.94034460376172, 40.55046637784821], [-73.94055533794308, 40.55141722291513], [-73.94059494966203, 40.55159562112637], [-73.94062525857242, 40.55177507691533], [-73.94064621694748, 40.551955308142034], [-73.94065779176053, 40.55213603144505], [-73.94065996473707, 40.552316962686625], [-73.94065273238446, 40.55249781740061], [-73.94063610599748, 40.552678311238054], [-73.94061011164119, 40.55285816041557], [-73.94057479011045, 40.553037082161026], [-73.94053019686582, 40.55321479515807], [-73.940476401948, 40.55339101998872], [-73.9404134898667, 40.55356547957265], [-73.94034155946935, 40.55373789960272], [-73.94026072378509, 40.553908008976954], [-73.94017110984856, 40.55407554022405], [-73.9400728584997, 40.55424022992465], [-73.93996612416265, 40.55440181912532], [-73.93985107460422, 40.55456005374592], [-73.93972789066935, 40.55471468497929], [-73.93959676599718, 40.55486546968239], [-73.93945790671738, 40.555012170759014], [-73.93931153112626, 40.55515455753251], [-73.9391578693425, 40.55529240610875], [-73.93899716294716, 40.55542549972815], [-73.93882966460265, 40.555553629107], [-73.93865563765661, 40.55567659276616], [-73.93848163371267, 40.55579451747538], [-73.93830493156165, 40.555910084758686], [-73.93803483987291, 40.55607899748846], [-73.93775898559808, 40.55624240343239], [-73.93747756165946, 40.55640018828606], [-73.9371907648797, 40.556552241676094], [-73.93689879584473, 40.556698457237594], [-73.93665081196751, 40.55681617433575], [-73.93639947503256, 40.55692967968965], [-73.93614490798154, 40.55703891777046], [-73.93588723533774, 40.557143835137], [-73.93579840968133, 40.557186447726444], [-73.9357061256084, 40.55722454588334], [-73.93561077696388, 40.55725796701257], [-73.93551277067246, 40.557286568478986], [-73.93543377715005, 40.55730566730419], [-73.93535359729825, 40.55732163345488], [-73.93524319778129, 40.55733822480907], [-73.93513154622016, 40.55734888056493], [-73.93501919051636, 40.557353548432175], [-73.93490668202666, 40.55735220550456], [-73.93479457285805, 40.55734485837184], [-73.93468341315717, 40.55733154308852], [-73.93457374841152, 40.55731232499572], [-73.9344748677788, 40.557317243925105], [-73.93429592181836, 40.55730697346468], [-73.93411654199369, 40.5573030953179], [-73.9339371206029, 40.55730561796605], [-73.9337580500351, 40.557314535892296], [-73.93357972191251, 40.55732982959333], [-73.93340252623442, 40.55735146562236], [-73.93322685052414, 40.55737939666181], [-73.9330530789821, 40.557413561627094], [-73.93288159164659, 40.55745388580027], [-73.93271276356151, 40.55750028099284], [-73.93254696395779, 40.55755264573926], [-73.93238455544527, 40.5576108655181], [-73.93222589322, 40.557674813002755], [-73.93207132428788, 40.557744348339995], [-73.93192118670595, 40.55781931945503], [-73.93177580884338, 40.557899562384875], [-73.93136701204324, 40.558176479673165], [-73.9287848892434, 40.55992550746975], [-73.92855349315126, 40.560032481434305], [-73.92832629158498, 40.56014456377511], [-73.92810347836831, 40.56026165888648], [-73.92788524358414, 40.56038366688605], [-73.92767177341338, 40.560510483700384], [-73.92746324997567, 40.560642001153326], [-73.92725985117427, 40.56077810705851], [-73.92707319791555, 40.56091033406395], [-73.92689138187725, 40.561046425217704], [-73.9261572327542, 40.56155260972308], [-73.92586523769828, 40.5616517403206], [-73.92568426673157, 40.56170057590932], [-73.92558721708254, 40.56172522840918], [-73.92545002935539, 40.561760074936544], [-73.92529896869686, 40.561807155522494], [-73.92510052720942, 40.56182927793012], [-73.92466271025597, 40.56192245780033], [-73.92421461302578, 40.5620013188248], [-73.92329484208311, 40.562090580451525], [-73.92278702954434, 40.562067196476264], [-73.92208248861199, 40.56207444733086], [-73.92107822881388, 40.56207985288483], [-73.92036810280447, 40.56221883246758], [-73.92020148738833, 40.56253196151498], [-73.92020596714332, 40.56253281768893], [-73.92019698482662, 40.562550749365116], [-73.92024067296587, 40.56254394617107], [-73.9202596317294, 40.56261229356109], [-73.92011735879362, 40.56263952634707], [-73.92009840148403, 40.562571180609986], [-73.92014433287757, 40.56256181714762], [-73.92016901882614, 40.56252254237469], [-73.92017573888783, 40.56252254704519], [-73.92033120142663, 40.562226054886615], [-73.92017877193038, 40.56225588612691], [-73.91962397269194, 40.56236584322539], [-73.91909761878105, 40.562470161207905], [-73.91766081900093, 40.56282614968639], [-73.9172257695885, 40.56298655266716], [-73.91680098049571, 40.563185496799974], [-73.91649706637092, 40.56333720248186], [-73.91610489437377, 40.56355858188918], [-73.91567986551492, 40.56381664714167], [-73.9155244036197, 40.56390384306317], [-73.91519131236099, 40.56409066671078], [-73.91483386530379, 40.56424205554105], [-73.91455336961286, 40.56435343243379], [-73.913867496264, 40.56460809614183], [-73.91324388491566, 40.564656158348534], [-73.9131318012801, 40.56463824394596], [-73.91220200769317, 40.56505456984988], [-73.91215942953983, 40.56497148274081], [-73.91188319663009, 40.564990405238674], [-73.91190258835852, 40.565044479622856], [-73.91192112783689, 40.565096175264955], [-73.9119269207045, 40.56511232806831], [-73.91194211877026, 40.565479631631625], [-73.91195522606857, 40.565731620504174], [-73.91195464366191, 40.56581132267282], [-73.9119815270415, 40.56580963497534], [-73.91214509452946, 40.56577900893883], [-73.91216181893262, 40.56583625156802], [-73.91197024752503, 40.56586941753247], [-73.9117361534006, 40.56587265348571], [-73.91173511047309, 40.565813716219964], [-73.91192551909776, 40.565813862377226], [-73.91192497868256, 40.56573672188277], [-73.91191299706968, 40.56547960930241], [-73.91189556223307, 40.565111450287745], [-73.91189239498307, 40.56510159409269], [-73.91187518743409, 40.565048055970266], [-73.91185809029383, 40.56499543800322], [-73.91148922796503, 40.565114763239244], [-73.91133796691499, 40.56506812591156], [-73.91117000937784, 40.56505851395878], [-73.91107382373276, 40.56505300922295], [-73.91079817683058, 40.5651109958315], [-73.91079826749402, 40.56504309759574], [-73.91056088690934, 40.5650073501188], [-73.91033193473851, 40.56501687075075], [-73.91030675154268, 40.56500493637165], [-73.91031901407925, 40.56496564748373], [-73.91017496486596, 40.56492272352996], [-73.9101477299426, 40.5649431450485], [-73.91011762673844, 40.56495586347528], [-73.91010084549161, 40.56495620165245], [-73.91008149893018, 40.56494246247026], [-73.91007760883511, 40.56491967587889], [-73.91009279361162, 40.564867818035935], [-73.91009922829879, 40.56482442599316], [-73.91003762905791, 40.56479371060192], [-73.91000297018701, 40.56484481755809], [-73.90998387183201, 40.56487988232135], [-73.90976838764617, 40.565275507349334], [-73.90978518025257, 40.56528064510755], [-73.90977170611873, 40.56530540530752], [-73.90971684925964, 40.56528742546118], [-73.909730321818, 40.56526351832803], [-73.90974823471095, 40.5652695117089], [-73.90995051522059, 40.56489994433498], [-73.90996837315812, 40.56487069925949], [-73.91002307642118, 40.56478020953845], [-73.90997230349046, 40.56474460368657], [-73.90987714566829, 40.56448583578525], [-73.9098755490021, 40.564453409409445], [-73.90987653834947, 40.56442096895194], [-73.90981103571444, 40.56432715547762], [-73.90976258436969, 40.56428483692814], [-73.90969073362554, 40.564257701782516], [-73.90966823860984, 40.56424051095107], [-73.90965355123693, 40.564212754457415], [-73.90953155346381, 40.564122832315], [-73.90939134156689, 40.56404609551438], [-73.9092770814906, 40.56399712415304], [-73.90918356455255, 40.563977896402974], [-73.90904934105347, 40.563961275280484], [-73.9089211721263, 40.563947304767474], [-73.9087254513683, 40.56393129153756], [-73.90851583574546, 40.56393904843132], [-73.90838328120986, 40.56396272597695], [-73.90821195105768, 40.56399332454081], [-73.9080904469524, 40.56402656413226], [-73.90807304358626, 40.56408533741216], [-73.90799528456878, 40.56416473737818], [-73.90797175520909, 40.56416153010163], [-73.9079482148663, 40.564158323984365], [-73.90792916599447, 40.56411667823528], [-73.90791411765858, 40.564068937270996], [-73.907951975564, 40.56400805646608], [-73.90797306965376, 40.56397596846283], [-73.90799426100763, 40.563917174658165], [-73.90800859982762, 40.56385718868023], [-73.90801597532176, 40.563796474103945], [-73.90801633051083, 40.56373550013356], [-73.9080096626694, 40.56367473797643], [-73.9079960233449, 40.563614657202166], [-73.90797551796078, 40.56355572211305], [-73.90789096153294, 40.56346261855917], [-73.9077988185825, 40.563373800900266], [-73.90769946157138, 40.563289628128935], [-73.90759329211629, 40.5632104404623], [-73.90748073936408, 40.56313655796707], [-73.9073622582579, 40.56306827926591], [-73.90723832769757, 40.563005880331254], [-73.9071094486046, 40.56294961336929], [-73.90697614189747, 40.56289970580127], [-73.90683894638656, 40.562856359344565], [-73.90669841659687, 40.562819749197196], [-73.90655512052625, 40.562790023330066], [-73.90640963735129, 40.56276730188913], [-73.90626255508623, 40.56275167670948], [-73.90611446820678, 40.56274321094494], [-73.90596597524814, 40.5627419388123], [-73.90581767638682, 40.56274786545329], [-73.90567017101444, 40.56276096691355], [-73.90549533617256, 40.56276575778522], [-73.90512293458974, 40.562828001714436], [-73.9046569315492, 40.56290588894741], [-73.90422374876526, 40.562995158675484], [-73.90391390139247, 40.56308296210139], [-73.9036677346663, 40.56316093955492], [-73.90310623198005, 40.563389272260586], [-73.90298894820239, 40.56344464589919], [-73.90297920728172, 40.563454601328374], [-73.90296735489308, 40.563463121444855], [-73.90295374846806, 40.56346994930782], [-73.90293879833516, 40.5634748790093], [-73.90292295534515, 40.56347776188435], [-73.90290669727534, 40.56347851099414], [-73.90289051442042, 40.56347710374773], [-73.90287489480674, 40.563473582583406], [-73.90286030947519, 40.56346805368901], [-73.9028471982753, 40.563460683799356], [-73.90283595660054, 40.56345169516812], [-73.90282692346567, 40.56344135886543], [-73.90282037128124, 40.56342998660262], [-73.90279436846555, 40.56337471712264], [-73.9027530963235, 40.56331393135542], [-73.90270432981026, 40.563255349841505], [-73.90264896763175, 40.563200283224], [-73.90258744277786, 40.563149162164855], [-73.90252023643049, 40.56310238646769], [-73.90244787420198, 40.563060321951475], [-73.90237092202435, 40.56302329758997], [-73.9022899817226, 40.56299160293852], [-73.90220568630893, 40.56296548587067], [-73.902118695032, 40.562945150638875], [-73.90202968822216, 40.562930756277765], [-73.9019393619705, 40.56292241536045], [-73.9018484226863, 40.56292019311837], [-73.90175758157231, 40.56292410693085], [-73.90166123516516, 40.56293506352471], [-73.90151119979181, 40.56295471672573], [-73.90136239001558, 40.562979197063], [-73.90121507272005, 40.563008460632346], [-73.90108692673103, 40.56303811545822], [-73.90096031793567, 40.563071389888286], [-73.90083542204312, 40.563108237744586], [-73.90067042372952, 40.56316335028136], [-73.90050925255056, 40.56322469732531], [-73.90045365151107, 40.56323897596529], [-73.90006663126383, 40.56338268738672], [-73.89964523066722, 40.56356206489001], [-73.89905579734292, 40.56381294864294], [-73.89863709831356, 40.564013477970896], [-73.89832919744433, 40.56417158181876], [-73.89801645708715, 40.56435914551782], [-73.89796716368488, 40.56437825559053], [-73.89791693883869, 40.564373789015235], [-73.89726738376439, 40.56467302206039], [-73.89718037015771, 40.564726723781384], [-73.89712237369706, 40.5647531921531], [-73.89706323189466, 40.56478017278063], [-73.89704215194091, 40.56478184744614], [-73.89659163481625, 40.565040001527414], [-73.89628126208979, 40.56523272170739], [-73.89621164762941, 40.56528127932281], [-73.89614780767694, 40.56533720689433], [-73.896132624453, 40.56535842756677], [-73.89581663166554, 40.56562431858591], [-73.89566630776342, 40.56583063730831], [-73.89543332392897, 40.56615043598428], [-73.89533725027061, 40.56633210037087], [-73.89533259419743, 40.56634090417397], [-73.89536175546587, 40.56640510268941], [-73.89539935314917, 40.566487840856276], [-73.89542366693382, 40.56658650687262], [-73.89557697279913, 40.566815433211566], [-73.89561861973425, 40.566877634494226], [-73.89564653277833, 40.56693948102173], [-73.89565586855373, 40.567010820495575], [-73.89565980661514, 40.56705946098028], [-73.8955701567844, 40.5670634313859], [-73.89549129592831, 40.5670578919942], [-73.89545231761565, 40.567031818712245], [-73.89543399793207, 40.56701956515993], [-73.89538025688381, 40.566997645046385], [-73.89531210417964, 40.56702218855294], [-73.89522927220416, 40.567074772663595], [-73.89504632135142, 40.567190914696674], [-73.89485403769469, 40.567327029002755], [-73.89479571968951, 40.567368310224516], [-73.8945970952024, 40.56748211622697], [-73.89438257495341, 40.56764180741036], [-73.89436419907221, 40.56764996730549], [-73.89413028093446, 40.567820149652825], [-73.89396328755053, 40.56792814814184], [-73.8936594633678, 40.56810371337828], [-73.89345467847606, 40.56823501855795], [-73.89326255071325, 40.568333312338574], [-73.89314119161189, 40.568395398445716], [-73.89280581743805, 40.56851491061991], [-73.89261013445609, 40.56858724728667], [-73.8925692981124, 40.56855880577257], [-73.89249413164768, 40.56853081091947], [-73.89235367291086, 40.568526113690076], [-73.89228381010733, 40.56851183832509], [-73.89214468849696, 40.56850560986675], [-73.89194768419188, 40.568489179864414], [-73.89184594142127, 40.56847298969817], [-73.89173272278907, 40.568454974278715], [-73.89146452569115, 40.568415124016106], [-73.89120836350693, 40.56834482979347], [-73.8910600526066, 40.568304129775505], [-73.89075729057016, 40.568221047859915], [-73.89050235300425, 40.56818323540314], [-73.89028537165312, 40.56817237514442], [-73.8901649592118, 40.568228524775265], [-73.89008675199516, 40.56816950194033], [-73.89006074216768, 40.568145261794754], [-73.8900201856573, 40.56809098386936], [-73.88993395087572, 40.5680729541433], [-73.88962915033767, 40.56798259952053], [-73.88959492449194, 40.567941890234074], [-73.88944124174323, 40.56793689729114], [-73.88940823061598, 40.567926212965666], [-73.88911103072223, 40.56791623669089], [-73.88891124690583, 40.567925634565825], [-73.88879000780409, 40.567931334052886], [-73.88866128999209, 40.56796810394101], [-73.88840180766067, 40.56807906694457], [-73.88839935499476, 40.56809082598919], [-73.88839431564136, 40.568103436754974], [-73.8883907733278, 40.56812453318512], [-73.88838304501148, 40.56814122068749], [-73.88836803030965, 40.56814492542518], [-73.88835950704319, 40.568143674741066], [-73.88834735692743, 40.568135621167684], [-73.88834467829327, 40.568120507881446], [-73.88834377442494, 40.56809571544478], [-73.88834311927033, 40.568077740036244], [-73.88834097846592, 40.56806602259852], [-73.88818867528397, 40.568033518084455], [-73.8880566098331, 40.568016918935626], [-73.88796642336445, 40.56801876782179], [-73.88786222508219, 40.568038044986416], [-73.88747378409192, 40.56813023872385], [-73.88687638253386, 40.56834983984713], [-73.88686105741907, 40.5683832977044], [-73.88654683936447, 40.568470978109445], [-73.88644241994884, 40.56844599324056], [-73.88629569651762, 40.568465965910924], [-73.88601332217256, 40.56848022914432], [-73.88592976749554, 40.56848445171615], [-73.88572202722436, 40.56852446159176], [-73.88552537443559, 40.56857979268739], [-73.88542104730786, 40.56859090948013], [-73.88534183019685, 40.568599348654644], [-73.88528929704948, 40.568569111026925], [-73.8851467629828, 40.568549031385515], [-73.88506218994054, 40.568527063551635], [-73.88502840887722, 40.568531127188635], [-73.88500002540947, 40.56852173302438], [-73.88489678220195, 40.568499584876896], [-73.88430973105297, 40.56869611953866], [-73.88430466693922, 40.568663571947575], [-73.8848870712472, 40.56847020566865], [-73.88475594832832, 40.56828913918742], [-73.88463979353489, 40.56833718522715], [-73.88470791077437, 40.56842967359395], [-73.88467738972528, 40.56844134290074], [-73.88462764522968, 40.568372968610774], [-73.88433684774819, 40.56848793447798], [-73.88429876082941, 40.56843680205306], [-73.88458722633577, 40.568319457529036], [-73.88459734214325, 40.56832600252621], [-73.88473689799127, 40.568268513240625], [-73.88471750208527, 40.56824534953776], [-73.88445665776179, 40.56789576340595], [-73.88416646626057, 40.568001951564035], [-73.88418090236974, 40.568027725164654], [-73.88418572568219, 40.56802639263762], [-73.88425563707202, 40.56814510250077], [-73.8842560545, 40.56815713340231], [-73.88422623813494, 40.56816646039563], [-73.8841423383738, 40.56802768627495], [-73.88415611937222, 40.56802242566621], [-73.88414806727037, 40.568008977301226], [-73.8839890745021, 40.56806016069418], [-73.88399610047865, 40.568072945956445], [-73.884007526758, 40.56807295749512], [-73.88407171935152, 40.568195014133465], [-73.88403638323946, 40.56820527702404], [-73.88397010168924, 40.56808955490235], [-73.88398465406, 40.56808402453267], [-73.88397412531297, 40.56806497313347], [-73.88379103703075, 40.568123911324484], [-73.88380690303865, 40.56815355417869], [-73.88381937098082, 40.56815198246059], [-73.88402340092412, 40.56850311561082], [-73.88399326079195, 40.56851179890226], [-73.88378819346467, 40.56815987256843], [-73.88379651062661, 40.56815592007595], [-73.88378086756107, 40.568127185959874], [-73.8837601597834, 40.568133851287904], [-73.8837694847298, 40.56814752545786], [-73.88357670672721, 40.56820976975422], [-73.8836919497781, 40.568395359722665], [-73.88347850081561, 40.568473114104954], [-73.88369768110292, 40.56883628546312], [-73.88390845170261, 40.568895911091715], [-73.88390427884407, 40.56890620425921], [-73.88368728096324, 40.56884340483232], [-73.88346534749647, 40.568478019938446], [-73.88336927316276, 40.56833185924823], [-73.88313094774313, 40.568380626014125], [-73.88311385647962, 40.5683831750126], [-73.88311091520163, 40.568383865927984], [-73.88309129854181, 40.568385213510275], [-73.88307333791968, 40.56838478176575], [-73.88305202325523, 40.568384021260954], [-73.88302891606186, 40.56838281849107], [-73.88300768867661, 40.56838026616839], [-73.88299287240591, 40.56837734209021], [-73.88298955629156, 40.568376681660475], [-73.88297087726367, 40.56837501464125], [-73.88296051092841, 40.568374247012414], [-73.88295399278807, 40.568373763916256], [-73.88293863229666, 40.5683723785401], [-73.88292327300584, 40.56837005784429], [-73.88290250762795, 40.56836600289595], [-73.88288811811933, 40.56836313418894], [-73.88287307747646, 40.5683605068968], [-73.88285619540405, 40.568358264718604], [-73.88283837926991, 40.568356247247834], [-73.88282147024864, 40.56835584350892], [-73.88279997403697, 40.56835801856397], [-73.88278550605777, 40.56835918404874], [-73.88276408427694, 40.56835962993066], [-73.88274616121862, 40.5683578281149], [-73.88273126157868, 40.568355975940776], [-73.88271489474496, 40.56835557223702], [-73.88269533764742, 40.568357029544515], [-73.882678706603, 40.568359461509914], [-73.882662866301, 40.56836196665356], [-73.882647356467, 40.56836456276622], [-73.88262970640685, 40.56836892982512], [-73.88261213622364, 40.568374846278374], [-73.88259408211263, 40.56838086191465], [-73.88257939340104, 40.568386366671646], [-73.88256242076477, 40.56839477807907], [-73.88254516419559, 40.56840411697355], [-73.88252920720879, 40.568412502609746], [-73.88251696222004, 40.56841866741233], [-73.88250322795516, 40.568425840392266], [-73.88248871352695, 40.568433542301584], [-73.88247982463253, 40.5684379913428], [-73.88247156263513, 40.56844211601707], [-73.8824508571579, 40.56845183836232], [-73.88243093138236, 40.5684610399817], [-73.8824097890766, 40.56846993343765], [-73.88239008898215, 40.56847859399228], [-73.88237283472289, 40.5684860960667], [-73.88235766000328, 40.568492609329695], [-73.8823432548687, 40.568498610067664], [-73.88232964100602, 40.568504133986934], [-73.8823125284216, 40.56851139244162], [-73.88229819339269, 40.56851799769323], [-73.88228637429398, 40.56852393741253], [-73.8822686225277, 40.56853237545287], [-73.8822549694179, 40.5685397180264], [-73.88224221751369, 40.56854590992248], [-73.88222246686078, 40.568555435017636], [-73.88220943230279, 40.568561077625255], [-73.88219685896784, 40.568566413789725], [-73.88218129409158, 40.568573151119175], [-73.88216455412915, 40.56857725886628], [-73.88214925446407, 40.56858072795744], [-73.88213139393241, 40.56858430314576], [-73.88210953872509, 40.568589052803596], [-73.88209772151953, 40.56859390339428], [-73.88208660430972, 40.5685980904468], [-73.88207621474676, 40.568602004333975], [-73.88209692581147, 40.56862950114764], [-73.88212127827947, 40.5686618181755], [-73.88206955861435, 40.56868029807086], [-73.8820178499041, 40.56869877845679], [-73.88197021885632, 40.568715109414015], [-73.88192258910136, 40.56873144152552], [-73.88192260012147, 40.56873129712619], [-73.88190727884263, 40.56870769636655], [-73.88189261684596, 40.56868509643526], [-73.88187437767158, 40.5686887162597], [-73.8818434971748, 40.568694843229444], [-73.88182559907011, 40.56869926485947], [-73.88180828198186, 40.56870284876547], [-73.88178841023914, 40.5687074945242], [-73.88177058314976, 40.568712004506885], [-73.8817545281002, 40.56871599596354], [-73.88174004464445, 40.568719024064336], [-73.88173357116696, 40.568720350769084], [-73.88172553821391, 40.56872199920032], [-73.88170633904176, 40.56872695087653], [-73.88168919765202, 40.568731300724224], [-73.88167033049703, 40.56873534254707], [-73.88165489126098, 40.56873856767288], [-73.88163393421279, 40.56874310355983], [-73.88161264711533, 40.56874706213031], [-73.88159429211795, 40.568749699061236], [-73.88157319628651, 40.568752090580524], [-73.88155040199122, 40.568754264566195], [-73.88152923752182, 40.56875559319763], [-73.88150835802925, 40.56875652641385], [-73.88148845626256, 40.56875826192994], [-73.8814673627757, 40.56875969985271], [-73.88144694240088, 40.568761515575716], [-73.88142484385884, 40.56876393233508], [-73.88140223669863, 40.56876649465097], [-73.88138189843421, 40.56876833792217], [-73.88135827609673, 40.568770897505814], [-73.88133487873345, 40.56877295389002], [-73.88131201515033, 40.56877405506357], [-73.8812899899062, 40.568774779989496], [-73.88126673733879, 40.56877521549129], [-73.88124592849768, 40.568775698412836], [-73.8812440990778, 40.56877574158702], [-73.88121987918441, 40.56877558101358], [-73.88119962399897, 40.568770481569416], [-73.88118565632857, 40.568768819132416], [-73.88117165492478, 40.56876710170908], [-73.8811485478543, 40.568765403510795], [-73.88113397769796, 40.568764046185976], [-73.88111785003272, 40.568762273951585], [-73.88110252501521, 40.56876006060211], [-73.8810843924999, 40.56875741228492], [-73.8810605336752, 40.56875299260694], [-73.881047513156, 40.568750674584706], [-73.88103254369172, 40.568747668843116], [-73.88101724521775, 40.56874498726411], [-73.88100249970928, 40.56874242369414], [-73.88097532354678, 40.56873766734514], [-73.88095258591643, 40.56873429402476], [-73.8809338020415, 40.56873238398265], [-73.88091670520443, 40.56873069079461], [-73.8808963389228, 40.56872877860321], [-73.88087474315614, 40.56872655972642], [-73.8808512493675, 40.568723679647164], [-73.88083009243813, 40.5687212719085], [-73.88080802486357, 40.5687191327765], [-73.88080640764103, 40.56871894229229], [-73.88078794238275, 40.568716751944535], [-73.88076279486117, 40.56871370930234], [-73.88073904289372, 40.56870950645204], [-73.88071911660474, 40.56870553272956], [-73.88070327264789, 40.56870335447165], [-73.88068567967169, 40.568701715683744], [-73.88066886553068, 40.56870054594793], [-73.8806537174327, 40.56869953876916], [-73.880636017406, 40.56869867385001], [-73.88061442403838, 40.56869549163035], [-73.88059214722345, 40.568692063767905], [-73.88057433278905, 40.56868908365676], [-73.88055347316788, 40.56868529656852], [-73.88054326066376, 40.56868399814717], [-73.88053267017403, 40.5686826455547], [-73.88051295386188, 40.568680427559386], [-73.88049261225292, 40.56867763294268], [-73.8804710279827, 40.56867588124252], [-73.8804558326657, 40.56867437994323], [-73.88043366851842, 40.568673924312975], [-73.88041688627574, 40.56867470009683], [-73.88040176869173, 40.56867567412387], [-73.88038118428805, 40.568677076190674], [-73.88036700049197, 40.56867786288211], [-73.88034903808415, 40.56867839367099], [-73.88032884427633, 40.568678695967364], [-73.88032117991807, 40.56867863420526], [-73.88030893446707, 40.568678540194504], [-73.88028914295056, 40.5686785733476], [-73.88027003542994, 40.56867891412479], [-73.88025376300214, 40.568678752751914], [-73.88023367653386, 40.568678245974596], [-73.88021503170239, 40.56867710173838], [-73.88019740435365, 40.56867540738741], [-73.88017818326908, 40.56867365809682], [-73.88016450973765, 40.568673013585915], [-73.88014654883912, 40.568672580378426], [-73.88012555136847, 40.56867315303115], [-73.88012340249263, 40.56867321394736], [-73.88009809627134, 40.568672692317186], [-73.88007402062833, 40.56867162196599], [-73.8800491891564, 40.56867011993373], [-73.8800250074982, 40.56866789719161], [-73.88000225656683, 40.56866548646285], [-73.87997972913301, 40.56866340164108], [-73.87995753008515, 40.568661954943785], [-73.87993758735001, 40.568660809306266], [-73.87992664178228, 40.56866035592181], [-73.87989303466885, 40.568659465547746], [-73.87987189778727, 40.5686590113857], [-73.87985268664356, 40.568658216133144], [-73.87983861024595, 40.5686577963222], [-73.87982198486952, 40.568656977969745], [-73.87980312654076, 40.56865679573436], [-73.8797862389313, 40.56865691309131], [-73.87976961248047, 40.56865705819396], [-73.87975818201656, 40.568656875014064], [-73.87975305662172, 40.5686567971084], [-73.87973365556653, 40.5686566692434], [-73.87971548137098, 40.56865683901985], [-73.87969751808572, 40.56865736920342], [-73.87968131494341, 40.56865826242203], [-73.87966465098816, 40.56865922652345], [-73.87964545944028, 40.568660359189444], [-73.87962638719442, 40.568660753673534], [-73.87960709085901, 40.568660786725424], [-73.87958856272876, 40.568661209248425], [-73.87956929987946, 40.56866228704478], [-73.87954539852859, 40.56866239751532], [-73.87951793036875, 40.56866272822726], [-73.87949270626528, 40.5686633325175], [-73.87946829663208, 40.568663522685604], [-73.8794413026328, 40.56866283780857], [-73.87941448510396, 40.56866248465178], [-73.87938896798258, 40.56866160312508], [-73.87936348238726, 40.56866272863265], [-73.87933579846116, 40.56866453635203], [-73.87931514238035, 40.56866583880744], [-73.87930192895583, 40.56866551968504], [-73.87928170142983, 40.5686650290109], [-73.8792590306537, 40.568663744354296], [-73.87923833181624, 40.568662327072566], [-73.87921878815278, 40.568662215891074], [-73.87919931538988, 40.568662258573106], [-73.87917440931017, 40.56866266494019], [-73.87915586741869, 40.56866352764723], [-73.87913428067395, 40.568663893604835], [-73.87911696789516, 40.56866417124446], [-73.87909806245413, 40.56866428637676], [-73.87908341950936, 40.56866421718678], [-73.87906848035361, 40.568664066431296], [-73.87905289440748, 40.56866368933004], [-73.87903572381265, 40.56866369872449], [-73.87903327914871, 40.56866368676848], [-73.87901906194138, 40.568663627400305], [-73.87900319058451, 40.56866375559676], [-73.87898677609904, 40.56866367464419], [-73.87896619312761, 40.568663544231505], [-73.87894689779121, 40.5686633888701], [-73.87892860441201, 40.568664224830016], [-73.87890849091838, 40.56866554587747], [-73.87888771809112, 40.56866604600033], [-73.8788706904323, 40.56866585733321], [-73.87885136038072, 40.5686650986443], [-73.87883510134056, 40.56866434252133], [-73.87882058842108, 40.56866404777254], [-73.87880406603716, 40.568664831971326], [-73.87878564227971, 40.56866596580633], [-73.87877015942387, 40.56866676852045], [-73.87875475975711, 40.5686672479876], [-73.87873193236219, 40.568667773388576], [-73.87871454945252, 40.56866783528281], [-73.87869687415929, 40.568666312871535], [-73.878687266263, 40.56866344818506], [-73.87868134846241, 40.568658164746836], [-73.87867973907235, 40.56864652894116], [-73.8786746736171, 40.56863993983796], [-73.87851699995322, 40.568659783454656], [-73.87830322053703, 40.56869386726816], [-73.87819124545663, 40.56871491982494], [-73.87797254160711, 40.568761631557585], [-73.87788701981546, 40.5687822173686], [-73.87780808995733, 40.56880063006316], [-73.8776378530969, 40.5688518421286], [-73.87760577562246, 40.56886089493745], [-73.87745281794943, 40.56891025148544], [-73.87740017966792, 40.56892617883126], [-73.87731299401725, 40.56895836086122], [-73.8770689967848, 40.5690484404372], [-73.87685485878126, 40.56913462528098], [-73.87662125007236, 40.56923183891122], [-73.87656038791111, 40.569257159145344], [-73.87643370124523, 40.56931436855044], [-73.87634075072575, 40.56935313560295], [-73.87611451688053, 40.56945944144884], [-73.87608408414432, 40.569472889410235], [-73.87598454034396, 40.56951979017155], [-73.8758586734124, 40.569580448427644], [-73.87543006202922, 40.569789960054095], [-73.87518736027619, 40.569910348535146], [-73.87510590878087, 40.56995068544372], [-73.87484758565488, 40.57007545069467], [-73.87443047584183, 40.570281828763505], [-73.87392287385016, 40.57052948404442], [-73.87316515806762, 40.570901899886], [-73.87260077649111, 40.57117643076299], [-73.86718075481033, 40.57383579492641], [-73.86663956227181, 40.57409921893446], [-73.86554317468638, 40.57464212861246], [-73.86464221607892, 40.575119214346415], [-73.86348661935081, 40.57574031588714], [-73.86288301945429, 40.576059761127304], [-73.86280361999994, 40.57593303499988], [-73.86278001199997, 40.57589856099986], [-73.86275061599991, 40.575857984999864], [-73.86333068899997, 40.57538325299991], [-73.86341227899989, 40.5753164769999], [-73.86396642099993, 40.57502678699988], [-73.86394267899998, 40.5749881079999], [-73.86340629400003, 40.574114330999855], [-73.86361132999998, 40.573823056999835], [-73.86449302899979, 40.5735427759999], [-73.86662812000002, 40.573057648999914], [-73.86633836399983, 40.572381674999846], [-73.86522555399998, 40.57045847199989]]]}}, {\"id\": \"27\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 28, \"Shape_Leng\": 0.097960782214, \"Shape_Area\": 0.000291203927662, \"zone\": \"Briarwood/Jamaica Hills\", \"LocationID\": 28, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79240413399991, 40.716193040999926], [-73.79268721599992, 40.71568003299994], [-73.79314903799985, 40.714933429999846], [-73.79346012399989, 40.714419191999895], [-73.7935315959999, 40.71427734599991], [-73.7935316439998, 40.71427725599991], [-73.7935316789999, 40.714277165999924], [-73.79358570299998, 40.714128677999874], [-73.79362091299997, 40.71397534699987], [-73.79363634199994, 40.71381958599994], [-73.79363636699993, 40.7138193969999], [-73.79363635499982, 40.71381920899989], [-73.79363166700001, 40.71366304199988], [-73.79360706699994, 40.713508577999924], [-73.79360705599986, 40.713508522999916], [-73.79360704399981, 40.713508469999894], [-73.79356328499998, 40.71335799599989], [-73.79350163599989, 40.71321339799988], [-73.79321635699988, 40.71261327999988], [-73.79313224999994, 40.71240723599992], [-73.79306872799985, 40.71219599599993], [-73.79302654199988, 40.711981363999875], [-73.79300604199986, 40.711765221999904], [-73.79300721999992, 40.7115494699999], [-73.79303800099989, 40.71071107499988], [-73.79389609999994, 40.71035232899985], [-73.79473052399995, 40.71001246199995], [-73.79561668999996, 40.709753078999874], [-73.79639418099987, 40.70953157899987], [-73.79731400799989, 40.70927019899988], [-73.79776810199988, 40.7091431879999], [-73.79807226500003, 40.70963987799992], [-73.79870150699992, 40.71066337699988], [-73.79985783199996, 40.710252562999905], [-73.80106920299994, 40.709818210999856], [-73.80118994599997, 40.70978629699991], [-73.80119015899984, 40.70978624399993], [-73.80119037199992, 40.70978616299991], [-73.80130606099976, 40.709742228999914], [-73.80130615599984, 40.709742192999926], [-73.80130625, 40.709742148999965], [-73.80141399799987, 40.70968695199988], [-73.8014141629999, 40.7096868729999], [-73.80141430599987, 40.7096867729999], [-73.80151137199988, 40.7096217159999], [-73.80151145499995, 40.70962166199995], [-73.80151152599983, 40.70962160799997], [-73.80159590599997, 40.70954842499991], [-73.80193297099994, 40.70891159599995], [-73.80212493399993, 40.7087376039999], [-73.80233119999987, 40.708572470999904], [-73.80255069099992, 40.70841720299988], [-73.80278213499996, 40.70827267099985], [-73.80302414799988, 40.70813965699997], [-73.80278721399985, 40.707715802999864], [-73.80222783399978, 40.70674931299991], [-73.80165725599993, 40.705761474999846], [-73.80265548599988, 40.70548042999997], [-73.80357251599989, 40.705233230999966], [-73.8056477309999, 40.70467172799994], [-73.80613695999992, 40.70569149199989], [-73.80658872800001, 40.70663390999988], [-73.80694856299993, 40.70736709999989], [-73.80934800399993, 40.70668543899991], [-73.81201248599983, 40.70594494499993], [-73.81190436299998, 40.705123939999886], [-73.81180202699996, 40.70436635199991], [-73.81177296699997, 40.7041809509999], [-73.81171288899998, 40.70366220199985], [-73.81163157099994, 40.70305215999991], [-73.81152856899989, 40.70227287499991], [-73.81193002199988, 40.70234083599996], [-73.81301882900003, 40.70249515699991], [-73.81314101299999, 40.702499551999914], [-73.81356470599994, 40.70252849799989], [-73.81398666699995, 40.70254206699991], [-73.81429413399998, 40.70253110799994], [-73.81496204599993, 40.70249848699989], [-73.815315933, 40.702487330999915], [-73.8159402059999, 40.70245890199989], [-73.81628916400003, 40.7024540649999], [-73.81637824499992, 40.70245196399987], [-73.81658976900003, 40.70291543599995], [-73.81669961199981, 40.70315609299994], [-73.8169748309999, 40.70374317299995], [-73.81709738799995, 40.704029800999876], [-73.81735529099986, 40.70455013999986], [-73.81738330599997, 40.70460389799992], [-73.81809905199994, 40.70597371499988], [-73.81822457299995, 40.706177661999895], [-73.81828566399994, 40.706302849999865], [-73.81861275299995, 40.706853399999915], [-73.81892102499985, 40.70734990599987], [-73.81899305699982, 40.70747190299995], [-73.81905016999981, 40.70757022099986], [-73.81910729599991, 40.707668538999904], [-73.81916136299994, 40.70776033199991], [-73.81921543099992, 40.70785211599987], [-73.81955363300001, 40.7084252289999], [-73.81994399099985, 40.70901972299991], [-73.82023145199997, 40.70945750299994], [-73.82045236499991, 40.709774548999896], [-73.82065280899985, 40.70998970699988], [-73.82081877299993, 40.710172776999855], [-73.82098519099998, 40.710323844999934], [-73.82176010399989, 40.710992930999936], [-73.822526781, 40.711654884999845], [-73.82288529599995, 40.71192805799993], [-73.82320247499996, 40.712205947999976], [-73.8234526099998, 40.71245950199993], [-73.82363860699985, 40.71266664399989], [-73.82385072099994, 40.71293320599995], [-73.82398559799991, 40.71321593899988], [-73.82428940599979, 40.71387006599987], [-73.82467827499991, 40.7144485109999], [-73.82471749499993, 40.71450684999993], [-73.82497234899996, 40.71480260499994], [-73.8251664649999, 40.71502786599996], [-73.82537094099992, 40.71526515499991], [-73.82559716399989, 40.715514336999924], [-73.82567283299986, 40.71561510999986], [-73.82570967299992, 40.71566664699994], [-73.82573653299988, 40.715704221999914], [-73.8258906989998, 40.715938525999924], [-73.82591945199992, 40.71598987599992], [-73.82566233799984, 40.716152637999926], [-73.82561290699984, 40.7162098979999], [-73.82524731699984, 40.71645544499987], [-73.825090876, 40.71645279299986], [-73.82346833799978, 40.71673105099993], [-73.82187417699991, 40.716917117999856], [-73.82100705300005, 40.71687995099992], [-73.82051599199986, 40.71681199099988], [-73.82018883899984, 40.71673522399984], [-73.81948747199984, 40.71647859799984], [-73.81931944099983, 40.71640861399995], [-73.81875295799986, 40.71617287499986], [-73.81852868899989, 40.71608211599988], [-73.81828452900005, 40.71600371399992], [-73.81789512299986, 40.715918370999894], [-73.817780212, 40.715901047999864], [-73.81767650099987, 40.71588471999992], [-73.81743591099992, 40.71585731399992], [-73.81720417899986, 40.715840727999925], [-73.81680638099989, 40.71582792999987], [-73.81643940199992, 40.7158318299999], [-73.81636080399981, 40.71583237099993], [-73.8162894489999, 40.71583316699988], [-73.81595932699985, 40.71584062699988], [-73.81524262499985, 40.71585683199992], [-73.81364203999982, 40.71589299199993], [-73.81291344199987, 40.715896708999885], [-73.81151841199983, 40.71590499699989], [-73.81114665699984, 40.71590720599987], [-73.81022434999987, 40.715912667999866], [-73.80780927699995, 40.715853895999906], [-73.80670677799996, 40.71588523999989], [-73.80579237899985, 40.71594264899989], [-73.80494148499987, 40.715996230999956], [-73.80464476399993, 40.716034396999866], [-73.80394838899987, 40.716102911999926], [-73.80319837999987, 40.7162025249999], [-73.80313952600001, 40.71621390699995], [-73.803084842, 40.716220161999935], [-73.80188321, 40.716408810999916], [-73.80092734399994, 40.71660393299992], [-73.8002167269999, 40.71673785899987], [-73.80004693799998, 40.716781906999906], [-73.79959535799996, 40.71689904299997], [-73.79912568499999, 40.717034345999885], [-73.79802996199992, 40.71735242599987], [-73.79741025099996, 40.717532907999946], [-73.79653978199998, 40.717786748999885], [-73.79563775099989, 40.71807616099995], [-73.79480794899995, 40.71837515899988], [-73.7944373599998, 40.7185008579999], [-73.79399335100001, 40.71864514099988], [-73.79315493500002, 40.718917578999864], [-73.79309042899989, 40.71893343799996], [-73.79261566399995, 40.71904520999994], [-73.79249599699979, 40.719067435999904], [-73.79211141299997, 40.7191388639999], [-73.79176986799989, 40.719178606999876], [-73.79155692199994, 40.71920099499993], [-73.791417731, 40.71920231099991], [-73.79073248899982, 40.71920875699987], [-73.79076914099981, 40.71913418899989], [-73.79089607099996, 40.7188759379999], [-73.79123627699988, 40.71868931099988], [-73.79137357500004, 40.71860799199986], [-73.79150283700001, 40.71851882299991], [-73.79162293699987, 40.71842251499991], [-73.79173290199986, 40.7183199289999], [-73.7919010649999, 40.718136366999936], [-73.79201376599997, 40.71796768699994], [-73.79210870199984, 40.71779218799989], [-73.79210873799991, 40.71779211499996], [-73.7921849289999, 40.71761139399988], [-73.79224175599992, 40.71742717899984], [-73.79226766099997, 40.717283926999876], [-73.79234087599984, 40.71646287399992], [-73.79240413399991, 40.716193040999926]]]}}, {\"id\": \"28\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 29, \"Shape_Leng\": 0.0714083127733, \"Shape_Area\": 0.000201673837402, \"zone\": \"Brighton Beach\", \"LocationID\": 29, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96004798699995, 40.58326987199995], [-73.95900739999992, 40.58283812799991], [-73.95733608399979, 40.58302513399989], [-73.95723610100002, 40.5830422709999], [-73.9571329139998, 40.58304572599995], [-73.95703134199992, 40.58303533199994], [-73.95695595999992, 40.5829822629999], [-73.95689047299997, 40.582933978999925], [-73.95682583599991, 40.5828863089999], [-73.95675618299992, 40.58283496999994], [-73.95645729099982, 40.582658850999906], [-73.95604146799998, 40.58273625399989], [-73.95571160099988, 40.58280458499993], [-73.95517441499999, 40.582915870999955], [-73.95436572700001, 40.583081076999875], [-73.95414287899986, 40.58310613999989], [-73.95412132099985, 40.58298298499989], [-73.95411176199985, 40.58291991799995], [-73.95407172899988, 40.58270239799989], [-73.95403970599988, 40.58251639099995], [-73.95392000699991, 40.58186979399993], [-73.9538380169999, 40.581745551999916], [-73.9537406319999, 40.58135589199988], [-73.95334153899987, 40.57926712799989], [-73.95309963899994, 40.577994155999896], [-73.95303970899995, 40.577679586999885], [-73.95300123199996, 40.57747651199985], [-73.95287451099996, 40.57688215899993], [-73.95242628499996, 40.57455910699991], [-73.95235846771321, 40.574274685845246], [-73.95254341027386, 40.57422397950429], [-73.95275899192154, 40.5742136309318], [-73.95295058831675, 40.57425023854544], [-73.95311788871602, 40.57428779386081], [-73.95328291809346, 40.574291889841746], [-73.95340408353178, 40.574294195339526], [-73.95352660081426, 40.57429652370537], [-73.95374257958343, 40.57429723224191], [-73.95377968074372, 40.57428764371035], [-73.9538505137372, 40.5742714772718], [-73.95393490539726, 40.57424290865264], [-73.95399423405075, 40.57426631704181], [-73.95401136297524, 40.57423762256748], [-73.95403199797872, 40.574086297180465], [-73.95408333385912, 40.57407327218331], [-73.95410726751373, 40.57410198308006], [-73.95410029594862, 40.57428723314875], [-73.95413360803454, 40.5743332674952], [-73.954397938218, 40.57437084590723], [-73.95511114426007, 40.57437112680509], [-73.95513705246907, 40.57437113692578], [-73.95569484044366, 40.57432699626322], [-73.95620816595128, 40.574265470675115], [-73.95634506681625, 40.574207221989525], [-73.95636904107674, 40.57417331306188], [-73.95629561203627, 40.573834984654354], [-73.95631111453274, 40.57380017438839], [-73.95634533454607, 40.57379757829185], [-73.95638980040823, 40.57382368774354], [-73.95647169447582, 40.574139487242576], [-73.9565675090591, 40.574170778492906], [-73.95720053498161, 40.574186671380815], [-73.9579441385944, 40.574144812662254], [-73.95843476359003, 40.57408881554891], [-73.95865147139088, 40.57405152377117], [-73.95888419181568, 40.57399159603121], [-73.9589663415273, 40.57394726876846], [-73.95897543388922, 40.57390455247927], [-73.95894268461842, 40.573467167647344], [-73.9589871897255, 40.57343065573189], [-73.9590260777376, 40.573451995118354], [-73.9590724802758, 40.57384554800023], [-73.95909642194573, 40.57386382043326], [-73.95939182223685, 40.57391210937612], [-73.95941778299985, 40.57517798299994], [-73.96103342799984, 40.57507632399994], [-73.96286117999982, 40.57498014099989], [-73.96471128899991, 40.57478261799983], [-73.96675354099992, 40.574832917999906], [-73.96756707299996, 40.574881526999896], [-73.96765743500002, 40.575043681999915], [-73.96780353599996, 40.57502045399985], [-73.96795533399992, 40.57501255399993], [-73.96810857699987, 40.57502089899985], [-73.96825878799997, 40.575045488999905], [-73.96840165999983, 40.57508536799986], [-73.96853345900001, 40.57513875999993], [-73.96883249999995, 40.57524150099993], [-73.9688899589999, 40.57526123899986], [-73.96883335399993, 40.575335858999864], [-73.96880187100001, 40.57537736399989], [-73.96873173299986, 40.57550140999995], [-73.96868119500002, 40.57563088299987], [-73.96866614099994, 40.57569696799986], [-73.96865108899983, 40.57576305399993], [-73.968546354, 40.576242670999875], [-73.96853234199989, 40.57630503099992], [-73.96852475500002, 40.57634038099996], [-73.96834802499988, 40.57711322399996], [-73.96804495099995, 40.578438532999854], [-73.96761555699982, 40.58030516299993], [-73.96759867799993, 40.58037847099992], [-73.9675871379998, 40.58042905899994], [-73.96733403999997, 40.58151752199991], [-73.96708956100004, 40.58259579399989], [-73.96707277600002, 40.582669811999864], [-73.96701115999986, 40.58294762399991], [-73.96691975199984, 40.58335247299993], [-73.96685766099992, 40.58358661799986], [-73.96683458399988, 40.583676861999926], [-73.966813755, 40.58375771499986], [-73.96677678399986, 40.58391498999987], [-73.96676128900002, 40.5839843919999], [-73.96674664199992, 40.584055351999886], [-73.96673011799997, 40.584135972999874], [-73.96670866299985, 40.58423930199995], [-73.96661531499983, 40.58457103699992], [-73.96660079799997, 40.58464174199989], [-73.96653588899984, 40.584957806999974], [-73.96649796199982, 40.58514249299987], [-73.96643740799989, 40.58541469399988], [-73.96636486799986, 40.585751243999916], [-73.96633012800002, 40.58589875099992], [-73.96615022799989, 40.58666882399992], [-73.96597498399979, 40.58743213599996], [-73.96581402699984, 40.58813321799995], [-73.9656408479998, 40.58888742799993], [-73.96546500699992, 40.589643336999856], [-73.96530164399988, 40.59035822999992], [-73.96514385199995, 40.5911019159999], [-73.96483485600002, 40.59113696899989], [-73.96377366299998, 40.59125722899988], [-73.96283785899992, 40.59136011199992], [-73.9606798409999, 40.591597582999945], [-73.9606241039998, 40.59086268099987], [-73.96057230499996, 40.590181426999905], [-73.96051667199988, 40.589452911999885], [-73.96045968399986, 40.58872381999986], [-73.96040710199983, 40.58803435199996], [-73.9603495359999, 40.58730628599994], [-73.96031712099983, 40.58651472399986], [-73.96031766399983, 40.5864364679999], [-73.9603380369999, 40.58628108999996], [-73.96034614999985, 40.586124769999905], [-73.96034190799985, 40.585968365999854], [-73.9603253929998, 40.585812722999954], [-73.96025825899996, 40.58502930799994], [-73.96024315399984, 40.58482708099993], [-73.9602394339998, 40.584791873999954], [-73.96023555999999, 40.584746675999895], [-73.96023124399987, 40.584676004999906], [-73.96022753900004, 40.584608958999915], [-73.96021792199998, 40.584514661999904], [-73.96014924099997, 40.58357589699987], [-73.96010509500002, 40.58340835699995], [-73.96004798699995, 40.58326987199995]]]}}, {\"id\": \"29\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 30, \"Shape_Leng\": 0.0945097669793, \"Shape_Area\": 0.000145862107626, \"zone\": \"Broad Channel\", \"LocationID\": 30, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82075892499992, 40.61523267899991], [-73.81768666599999, 40.61493299799986], [-73.81652681189118, 40.61494701007998], [-73.81655519586245, 40.614881068359615], [-73.81657527753201, 40.61481337831561], [-73.81658687658151, 40.614744545249984], [-73.8165898896374, 40.61467518507705], [-73.81658428973421, 40.61460591806852], [-73.8165701269355, 40.61453736407743], [-73.8165475286072, 40.614470136266654], [-73.81651669585267, 40.614404836864594], [-73.81647790560582, 40.61434204897533], [-73.8164315038978, 40.614282334964926], [-73.81615626616234, 40.61294211575487], [-73.81604375315976, 40.61275962622099], [-73.81602636695548, 40.61236542423196], [-73.81595301908956, 40.61216821893675], [-73.81586106373182, 40.61202025698769], [-73.81588914214508, 40.61188656365408], [-73.81585258478174, 40.61174572809621], [-73.81581602898463, 40.61160489311676], [-73.81573329159819, 40.611463984117556], [-73.81563522523048, 40.61119560007353], [-73.81559405486736, 40.61086335714751], [-73.81559858789666, 40.61067251616471], [-73.81520720849484, 40.60981673379666], [-73.81502615390193, 40.609352013155714], [-73.81491794505943, 40.60905539220143], [-73.81477073796229, 40.60885102889005], [-73.81475287105245, 40.6086327967523], [-73.81476257619768, 40.608463880026925], [-73.81482785097941, 40.60823874224766], [-73.8148653991121, 40.608020598741525], [-73.8149997124251, 40.607825413169444], [-73.81496328852214, 40.60764059421228], [-73.81506129889364, 40.607317030139924], [-73.81520607102756, 40.607092544035126], [-73.81526535891507, 40.60700061106351], [-73.81554657007985, 40.60677293900015], [-73.81568836082222, 40.60661289096192], [-73.81574174081905, 40.60646524511771], [-73.81576751589343, 40.60637159210934], [-73.81580703895482, 40.60636640618301], [-73.81585416784488, 40.60634155617887], [-73.81595962051259, 40.606262483176735], [-73.81600764380624, 40.606194852030484], [-73.81600875994269, 40.606144258981395], [-73.81596093586549, 40.606139717032654], [-73.81597032024116, 40.60607436222665], [-73.8159435787941, 40.60606156480641], [-73.8159191720938, 40.60606152576282], [-73.8159191392178, 40.60602294080149], [-73.81589026342208, 40.606025339220764], [-73.8158882978922, 40.6059290341796], [-73.8159419971478, 40.60592688893786], [-73.81598548513719, 40.60573350521615], [-73.81609284277334, 40.605744093120286], [-73.81642529961537, 40.605224123786925], [-73.81739192103247, 40.60515812414338], [-73.81730244874, 40.60420592104616], [-73.81725370596759, 40.60394032459458], [-73.81755792378435, 40.60278576000433], [-73.8177203266933, 40.60222499987045], [-73.81831121077079, 40.601339712357415], [-73.81842675420909, 40.60008406892017], [-73.81849066110368, 40.59938955923962], [-73.81899346411964, 40.598262463121856], [-73.8192814220081, 40.597529610938246], [-73.81973591782804, 40.59751684313405], [-73.82015695397838, 40.596873907202536], [-73.82022310416497, 40.59680187575785], [-73.82028316165791, 40.59672680602921], [-73.82033688583628, 40.59664899805184], [-73.82038406369512, 40.5965687618151], [-73.82042450677176, 40.596486416145474], [-73.82045805451784, 40.59640229006972], [-73.82048457262269, 40.59631671816084], [-73.82050913385626, 40.59625816595299], [-73.82052562152057, 40.59619799173205], [-73.820533857754, 40.59613684003583], [-73.82053375492012, 40.596075367637795], [-73.8205253145804, 40.59601423203291], [-73.82050862648865, 40.595954089906186], [-73.82048386960406, 40.595895585619374], [-73.82045130963279, 40.59583934575525], [-73.8204507978603, 40.595838573953884], [-73.8204424431122, 40.59580887598263], [-73.82044084709868, 40.5957785271302], [-73.82044605664017, 40.595748414924365], [-73.82045791860484, 40.59571941893889], [-73.8204760866339, 40.595692386931226], [-73.82050002971006, 40.595668108956055], [-73.82059544842939, 40.5956026301113], [-73.82061381211892, 40.59550607512599], [-73.82065843490554, 40.595271452032726], [-73.82071206789055, 40.59517051496212], [-73.82079778694794, 40.59515300679664], [-73.8209309528666, 40.59510806180529], [-73.82108400042337, 40.594928368859804], [-73.82120137599598, 40.594779935044805], [-73.82135758260135, 40.594691009121114], [-73.82166936203787, 40.594750939180386], [-73.82223447124294, 40.59485584588739], [-73.82264377296077, 40.59490106309344], [-73.82301423918268, 40.59488677298898], [-73.82367706984253, 40.59490265315221], [-73.82394993688561, 40.59493279504595], [-73.82480251246255, 40.59508316117646], [-73.82565454783656, 40.594964406762166], [-73.82589974767566, 40.59486145713528], [-73.82625044687596, 40.59495115875487], [-73.8270499051601, 40.59490777901203], [-73.82798591286824, 40.594834876017856], [-73.82835645403685, 40.59479084506242], [-73.82876587213534, 40.59479145596427], [-73.82913626075876, 40.594806869147725], [-73.82937044176717, 40.59471804715625], [-73.82972156186142, 40.59464426120583], [-73.83048206013673, 40.59458593813551], [-73.83135504801284, 40.59463006988999], [-73.8324120968248, 40.594618492810554], [-73.83305498409688, 40.59481262907002], [-73.83381500012786, 40.5949474849947], [-73.83465599507487, 40.59516845882606], [-73.83484682806589, 40.59554343605339], [-73.83468997415088, 40.595899888214426], [-73.83461984485115, 40.59599745410395], [-73.83445520590696, 40.59622650519835], [-73.83439612701622, 40.59646420572123], [-73.83420068078192, 40.59665712408205], [-73.83386890489668, 40.59679039969174], [-73.83351748173193, 40.596983091173726], [-73.83336087711082, 40.59723551119928], [-73.83291189123968, 40.59745778303809], [-73.83273627105066, 40.59751697454668], [-73.832385102244, 40.5976056318385], [-73.83215095202706, 40.597679598145966], [-73.83239364476839, 40.59780335319121], [-73.83154105970068, 40.5979510260275], [-73.83126070906049, 40.5978920460556], [-73.83108000652722, 40.597791464827154], [-73.83095699282347, 40.59764708004299], [-73.8307600942432, 40.59744615815484], [-73.83062852189495, 40.59743342586203], [-73.83049675901624, 40.59749592809601], [-73.83050466559352, 40.59762133483558], [-73.83038922491133, 40.597734019792114], [-73.83026568620224, 40.597796535048566], [-73.8300517969121, 40.597808757865735], [-73.82985422922734, 40.59787116315062], [-73.82975522196986, 40.59799014184838], [-73.82961515193035, 40.598083979149216], [-73.82948341811623, 40.59813394195126], [-73.8293519571253, 40.59807731982705], [-73.82926994412304, 40.59798315098793], [-73.82921276789511, 40.597826322957275], [-73.82909777411142, 40.5977634551103], [-73.82891681736452, 40.5977631859847], [-73.82874408619575, 40.59776292884545], [-73.82857156476862, 40.59768116475648], [-73.82832490242488, 40.59764317810819], [-73.82806159687702, 40.59768040282441], [-73.82794610384788, 40.59781189487175], [-73.82784710901738, 40.597924601112986], [-73.82784691502562, 40.597999838383004], [-73.82763273197193, 40.598124911932295], [-73.8274268880001, 40.59820610885799], [-73.82722092999175, 40.59833119503657], [-73.8271301911274, 40.598431372963816], [-73.82709706289336, 40.598519099830945], [-73.82717084795338, 40.59861325700666], [-73.82710314546546, 40.598688685754354], [-73.82707810089018, 40.59867906322921], [-73.82705133886849, 40.59867267001917], [-73.82702356982604, 40.59866967503489], [-73.82699553156307, 40.59867015805246], [-73.82696796795742, 40.598674106156544], [-73.8269416120754, 40.59868141441143], [-73.82691716394703, 40.59869188925293], [-73.82689527276902, 40.598705252067255], [-73.82687651978097, 40.59872114738559], [-73.82686140304179, 40.5987391540985], [-73.82685032481477, 40.59875879306266], [-73.82684357823473, 40.59877954295343], [-73.82684134389942, 40.59880085219284], [-73.82676287102854, 40.59887852396063], [-73.8266785731343, 40.59895256423353], [-73.82658873898603, 40.59902272078796], [-73.82649367583141, 40.599088752048715], [-73.82639370886822, 40.59915043299935], [-73.8262891801897, 40.59920755206958], [-73.82627138518978, 40.59925406192512], [-73.82624639580337, 40.59929860721097], [-73.82621457216418, 40.59934054492623], [-73.82617637580326, 40.59937926742232], [-73.82613235782433, 40.59941421628603], [-73.82610801087331, 40.599433473202446], [-73.82607971900187, 40.59944929027962], [-73.82604830803486, 40.59946120621747], [-73.82601469481251, 40.59946887314232], [-73.8259798598142, 40.599472067181495], [-73.82594481984405, 40.5994706952275], [-73.8259105971973, 40.599464797799115], [-73.825878191737, 40.59945454595633], [-73.82584854782284, 40.59944023978944], [-73.82587723990645, 40.599388191110776], [-73.82589853794282, 40.59933411909544], [-73.82591220213257, 40.59927863191616], [-73.82591807911876, 40.599222353706445], [-73.82591610285142, 40.59916591704907], [-73.82590629495935, 40.59910995791428], [-73.82588876662335, 40.59905510558804], [-73.82586371395628, 40.59900197663026], [-73.8258314198925, 40.59895116890409], [-73.82579224760113, 40.59890325471297], [-73.82576186668572, 40.598827425613045], [-73.82572400162758, 40.598753578859636], [-73.82567887179285, 40.59868214369582], [-73.82562673962269, 40.59861353386852], [-73.82556790737216, 40.59854814792653], [-73.82550271573429, 40.59848636506479], [-73.82543154384716, 40.59842854402503], [-73.82535480469845, 40.59837502005476], [-73.82527294342637, 40.59832610393457], [-73.82518643452747, 40.59828207907724], [-73.82513360448421, 40.598246143814805], [-73.82507610784262, 40.59821464379464], [-73.82501458361149, 40.5981879287877], [-73.82494971364855, 40.59816629501651], [-73.82488221787078, 40.59814998301003], [-73.82481284576305, 40.59813917377149], [-73.82474236672088, 40.598133986774954], [-73.82465420648664, 40.5981492468515], [-73.8245680498839, 40.598170124067785], [-73.8244845128283, 40.598196469931], [-73.82442324545605, 40.59823055798339], [-73.82436729168752, 40.598269595793205], [-73.8243173368626, 40.59831310502026], [-73.82427399262099, 40.59836055396409], [-73.8242377886617, 40.5984113618214], [-73.82420916849752, 40.59846490623819], [-73.82418848174414, 40.59852053311304], [-73.82403414202359, 40.59875797949145], [-73.82395871079126, 40.59887689683851], [-73.82395853199995, 40.59888785399994], [-73.82406984190722, 40.598922481127985], [-73.82410854765287, 40.59899472680749], [-73.82421870388679, 40.599074487959136], [-73.8241859569957, 40.59921342515717], [-73.8241656989433, 40.59929937983351], [-73.82411388280298, 40.59928535009736], [-73.824071162039, 40.599292259946935], [-73.82401358185359, 40.59942972124062], [-73.8237932359679, 40.599851224209225], [-73.82376612290614, 40.60004007091038], [-73.8236175937432, 40.60003570131408], [-73.82317356168161, 40.59997800515576], [-73.82315293179705, 40.599976662824794], [-73.82303195183424, 40.59993051694214], [-73.82300436542411, 40.59992561919807], [-73.82257638978241, 40.599887738969485], [-73.82255813223716, 40.59994156021139], [-73.82235892273734, 40.59992192205628], [-73.82228834165707, 40.59989695916107], [-73.82215981277453, 40.59986500305416], [-73.82214174920362, 40.59984564479891], [-73.82212726983464, 40.599834563199416], [-73.8221128246864, 40.59982350877661], [-73.82194441779896, 40.59980115598919], [-73.82190636783253, 40.59981162996677], [-73.82180296187538, 40.599861689810076], [-73.82171949113493, 40.599915408915074], [-73.82168859290637, 40.599954022003764], [-73.82158890006589, 40.59997733908516], [-73.82146929866913, 40.59999372263383], [-73.82146921755755, 40.600024098877554], [-73.8214763933384, 40.60005034409147], [-73.82159001545337, 40.60006242989868], [-73.82173154776704, 40.60007648099816], [-73.82172074219885, 40.600125283165575], [-73.82234555578552, 40.60018010003154], [-73.82234362290922, 40.60022566120388], [-73.82248489186438, 40.600235544823065], [-73.82249404104252, 40.60020104008347], [-73.82260631900901, 40.60021364018273], [-73.82261001509886, 40.600186031842206], [-73.82314788294842, 40.60023932910236], [-73.82315326995335, 40.6002572872199], [-73.82330902392228, 40.600269952836015], [-73.82331920489146, 40.600234088936006], [-73.82443643781227, 40.60034498367475], [-73.82440706593387, 40.60059112485041], [-73.82445218054455, 40.60079696284418], [-73.82430388510507, 40.600785199829204], [-73.8242976550786, 40.60085210111633], [-73.82413862984455, 40.60089146682628], [-73.82398900112113, 40.601001986812236], [-73.8239241478234, 40.601106483981475], [-73.82388150021876, 40.60124921618406], [-73.82379033117405, 40.6013431590753], [-73.82370544281271, 40.60139225097998], [-73.82360431215257, 40.60147823411712], [-73.8235113029713, 40.60154577108027], [-73.82346666942101, 40.601634916949756], [-73.82345423589811, 40.6017579531873], [-73.82351857017017, 40.601850342157256], [-73.82358498378176, 40.601940032997845], [-73.8236230300988, 40.602032008883064], [-73.82357799780264, 40.60227189587525], [-73.82323485805279, 40.602295980038114], [-73.82304906345898, 40.602344916077236], [-73.8229963972268, 40.602418667228356], [-73.82294484720244, 40.60258290923388], [-73.82292481177993, 40.60279805612647], [-73.82289824010462, 40.602904582144866], [-73.8228859758601, 40.60296301503914], [-73.82290201513365, 40.60300303204292], [-73.82287967385791, 40.60328278301239], [-73.82280627719284, 40.60397517902289], [-73.82276563488945, 40.60420260478325], [-73.82278089795393, 40.60453487647327], [-73.82281306524087, 40.60458107100474], [-73.82290283001802, 40.60468726877069], [-73.82284474420824, 40.604811847183406], [-73.8228161467707, 40.60494101100073], [-73.82276978379645, 40.60494517875667], [-73.8227152139639, 40.60495008393984], [-73.82267071602584, 40.60498693242402], [-73.82259374778921, 40.605091409601755], [-73.82254505876273, 40.605186701243284], [-73.82253280983204, 40.605238981531194], [-73.82255472623692, 40.60536206595556], [-73.82253633123983, 40.60543279720047], [-73.82246746751217, 40.60552498291131], [-73.82248345709922, 40.605583458011004], [-73.82260024311239, 40.60568515806692], [-73.82269456760869, 40.605738103125375], [-73.82278161094604, 40.6057869581131], [-73.8229046338875, 40.605851277214725], [-73.82266727165995, 40.60611873206085], [-73.82236000888112, 40.606469260830025], [-73.82224712010017, 40.606419864076926], [-73.82223894478628, 40.606423057759585], [-73.82211377790149, 40.606471956085066], [-73.82173772462286, 40.606720559117484], [-73.82167691598228, 40.60681890892526], [-73.82151924671516, 40.6069109548534], [-73.82126470321427, 40.607002850205184], [-73.82119987284851, 40.60709504008171], [-73.8212602880639, 40.60714435605202], [-73.82117120508813, 40.6072488144145], [-73.82119130511715, 40.607279608840344], [-73.82107550220364, 40.60738272511419], [-73.82089163199677, 40.607642153792106], [-73.82088742209929, 40.60770675093409], [-73.82090456163026, 40.607781246109084], [-73.82085990045205, 40.60783996985354], [-73.82029978490344, 40.608576445236174], [-73.82003243531851, 40.608923655879124], [-73.8200281252041, 40.60902516988982], [-73.81998318384011, 40.60922506304476], [-73.81994251894176, 40.609335748089634], [-73.81997458784697, 40.60941885997912], [-73.81998084881896, 40.609534058788874], [-73.81998227923674, 40.60956038585391], [-73.81980831751702, 40.60983495886035], [-73.81961795381072, 40.610479041032136], [-73.81995150270963, 40.61100586586921], [-73.82045405309891, 40.611854557163014], [-73.82051887888557, 40.61194994886219], [-73.82043202517556, 40.611987108137896], [-73.82043403607466, 40.61206678725429], [-73.82050281586973, 40.612133008178944], [-73.82057595418243, 40.61223314094054], [-73.8206893336656, 40.61225874578369], [-73.82076043881737, 40.61228767567876], [-73.82082274277886, 40.61228099237594], [-73.82087841795061, 40.61225734590997], [-73.82095222030904, 40.61210997603296], [-73.82098143849393, 40.611998135984095], [-73.82104384524081, 40.6119524624867], [-73.82108403602687, 40.61189658282583], [-73.82102865614056, 40.61181003904737], [-73.82101391261239, 40.6117666052224], [-73.82098893002488, 40.6116930062198], [-73.82110627924509, 40.611432068048714], [-73.82114995198185, 40.61139335100863], [-73.82120960284034, 40.61136249138747], [-73.82126119794827, 40.61130685698375], [-73.82128305878275, 40.611232605059136], [-73.82128421076361, 40.611096929142995], [-73.82120543662198, 40.61089613499369], [-73.8210622848369, 40.61076384818487], [-73.82078645512787, 40.61063341786839], [-73.82062698209096, 40.61052999510846], [-73.82054872655173, 40.610430824104476], [-73.82048700610763, 40.61022437914668], [-73.82052525668642, 40.610096502374056], [-73.82055813686831, 40.60995210898184], [-73.82063420416088, 40.60985730620809], [-73.82075127944346, 40.60961605984777], [-73.82079777394569, 40.60944279907322], [-73.82085878502924, 40.60923105813554], [-73.82114637042204, 40.6090624110356], [-73.82119903005622, 40.60889339821898], [-73.82133010279306, 40.608684718236645], [-73.82138268112817, 40.608545544973374], [-73.82161460094558, 40.60848031911391], [-73.82176098510635, 40.60861575979552], [-73.82175280507491, 40.60862610303351], [-73.82168241117662, 40.608715106793525], [-73.82177339909592, 40.608854501974754], [-73.82182531319947, 40.60896399709298], [-73.8216943464972, 40.609132890162414], [-73.82166797819498, 40.609232316803684], [-73.82179804199976, 40.6094016131943], [-73.82184945090272, 40.609700096907744], [-73.82186205220738, 40.609869212532935], [-73.8220962121063, 40.61015803186759], [-73.82233079790585, 40.61028770312733], [-73.82253978465067, 40.61022834489589], [-73.82273529622739, 40.610328114080026], [-73.8228923332429, 40.61033702209704], [-73.82319489759729, 40.610937836059286], [-73.82334654206795, 40.61115463895985], [-73.82342464290727, 40.61123433349867], [-73.82352882101249, 40.611324013967064], [-73.8238246859766, 40.61149866495507], [-73.82379847999995, 40.61212942099992], [-73.82234961099986, 40.61426002599995], [-73.82176857999997, 40.61476625499992], [-73.82170208100001, 40.61482419099985], [-73.82163921899993, 40.614870614999845], [-73.82157060299986, 40.614908690999904], [-73.82141601599977, 40.61505476399991], [-73.82075892499992, 40.61523267899991]]]}}, {\"id\": \"30\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 31, \"Shape_Leng\": 0.0964245666516, \"Shape_Area\": 0.000333975927329, \"zone\": \"Bronx Park\", \"LocationID\": 31, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87094462199981, 40.85726608099992], [-73.87094413599989, 40.85726598899989], [-73.87094366199995, 40.85726605199995], [-73.87089011199996, 40.85727278099992], [-73.87088983900003, 40.85727281799995], [-73.87088962499988, 40.85727291599993], [-73.87087754099998, 40.85727822399984], [-73.87084887399999, 40.85729081699992], [-73.86954868099988, 40.85772602999987], [-73.86889622799993, 40.85777277299997], [-73.86841539699994, 40.85778035799994], [-73.86876752999989, 40.85734077999995], [-73.86887858999994, 40.85720213799993], [-73.86907624599999, 40.85694556099986], [-73.86927391299994, 40.856688982999884], [-73.86988842299996, 40.85597106699987], [-73.86990379599995, 40.85587302199991], [-73.86990421600002, 40.85577329299987], [-73.86988939599996, 40.855673861999925], [-73.86985960399987, 40.855576727999924], [-73.86981576299978, 40.855483773999914], [-73.86937675499983, 40.854265722999855], [-73.86916668699996, 40.85368604599991], [-73.86872263199984, 40.85242385199994], [-73.86860515599989, 40.85206864599991], [-73.868530913, 40.8517767559999], [-73.86844433299986, 40.85148667199995], [-73.86834552299989, 40.85119885199987], [-73.86823466099995, 40.85091373699986], [-73.86920944599997, 40.84923837199993], [-73.86941402799984, 40.84878802499991], [-73.86889243299996, 40.848112321999906], [-73.86944230499982, 40.84724417999993], [-73.87109207899995, 40.84432478099992], [-73.87179038499981, 40.8438028139999], [-73.87184610299998, 40.84376077799993], [-73.87190185099998, 40.84372143299993], [-73.87280708199992, 40.843424263999964], [-73.87399295499982, 40.842561505999925], [-73.87456845, 40.84164271999992], [-73.87459175399982, 40.841520647999914], [-73.8760850519999, 40.84206482699989], [-73.876798721, 40.84229978699994], [-73.87686858299995, 40.84231710599993], [-73.87694201699995, 40.84233562799991], [-73.878171756, 40.84262443699987], [-73.8773292409999, 40.84398312499995], [-73.87800756899993, 40.844484295999926], [-73.87919408799995, 40.84532173399988], [-73.88048011699989, 40.84619371599996], [-73.88106059899991, 40.84659008499992], [-73.88199829499979, 40.847235311999874], [-73.88284445599996, 40.847817225999954], [-73.88276104799992, 40.848057845999904], [-73.88278133699995, 40.84807038999989], [-73.88311982600001, 40.848221091999925], [-73.88288269199988, 40.84907404699991], [-73.88284931500006, 40.84919252699994], [-73.88255596199991, 40.85022427099987], [-73.88232256599994, 40.851060791999906], [-73.88200299099991, 40.85218935799987], [-73.8816727419998, 40.8533496789999], [-73.88103358999996, 40.85559995199987], [-73.88073042799992, 40.85667660499989], [-73.88072954099984, 40.856807273999955], [-73.88077132199986, 40.8570397609999], [-73.88079155299991, 40.85712750799994], [-73.88080810099991, 40.857198447999885], [-73.88088225100002, 40.85750954499986], [-73.88094184999994, 40.85769449599996], [-73.88094193200001, 40.85769473899994], [-73.88094199099993, 40.857694990999974], [-73.88098293799995, 40.85788399199989], [-73.88100464399997, 40.85807565699992], [-73.88100678999979, 40.85826802299992], [-73.88098942799995, 40.85845935899989], [-73.880965862, 40.85854520499987], [-73.88075871500001, 40.85949442499993], [-73.88060896899991, 40.860357092999884], [-73.88061726900006, 40.860497010999936], [-73.8806437029999, 40.86063731599991], [-73.88068839499986, 40.86077599299989], [-73.88075099099991, 40.86091098799991], [-73.88083055899997, 40.86104034399993], [-73.88092572600007, 40.86116228799991], [-73.88103469099984, 40.861275294999885], [-73.88200794099986, 40.862046852999946], [-73.8837937109999, 40.86417157099987], [-73.88379386499992, 40.86417182299991], [-73.88386792099986, 40.8642908889999], [-73.88392395799984, 40.86441776099988], [-73.88396002399999, 40.864549718999896], [-73.88397514999986, 40.86468399799992], [-73.88396922299982, 40.864817740999825], [-73.88394303099996, 40.86494818699985], [-73.88390334899988, 40.86507380199995], [-73.88384464399992, 40.86519681299989], [-73.88376749199992, 40.86531475299994], [-73.88367321300004, 40.86542528299993], [-73.883563778, 40.86552635199991], [-73.88344168299979, 40.86561633199988], [-73.88319816099992, 40.86576631399988], [-73.88273658899989, 40.86604857299993], [-73.88267624699999, 40.86608914099984], [-73.88271794699989, 40.86613296599992], [-73.88297537099986, 40.86644973199989], [-73.8830900319999, 40.86659150899993], [-73.88135802599994, 40.86757898699991], [-73.87991019699996, 40.86836304099992], [-73.87981206299987, 40.86841664799988], [-73.87772752999979, 40.86955533799994], [-73.87694351999987, 40.86995468199993], [-73.87619224900003, 40.87047671099988], [-73.87518802799978, 40.87157682799994], [-73.87436750799998, 40.872701008999876], [-73.87255250899983, 40.8751751829999], [-73.87160577400002, 40.87692192799989], [-73.87094367500002, 40.87851076699998], [-73.87033725199989, 40.87835140299995], [-73.8701497639999, 40.87831012099989], [-73.87005983899994, 40.878290316999916], [-73.86996807499997, 40.87826435199985], [-73.86978338299986, 40.87821297799992], [-73.86950315999994, 40.87814566499988], [-73.86943471300005, 40.87812919899994], [-73.86948896199988, 40.87798946799993], [-73.87004672699982, 40.876670530999945], [-73.8701541849999, 40.876416957999886], [-73.8701809419998, 40.87635452999992], [-73.87019944899986, 40.87631135499987], [-73.87023414999992, 40.876217573999924], [-73.87026287199994, 40.87612263599991], [-73.87028554799987, 40.876026758999956], [-73.87030212599987, 40.87593016499994], [-73.87031256699991, 40.87583307799986], [-73.8703168469999, 40.87573572299987], [-73.87031508399987, 40.87561752299991], [-73.8703131089999, 40.87534062099997], [-73.87037667199995, 40.8732843719999], [-73.87056192499989, 40.87151077799995], [-73.87053738699991, 40.8696740309999], [-73.87053494699997, 40.869581702999895], [-73.87053320099994, 40.86951557999986], [-73.87052897299995, 40.86935553299992], [-73.870526217, 40.869251176999924], [-73.87051629799993, 40.868875664999926], [-73.87051473299981, 40.86881643199997], [-73.87049958099983, 40.86824277599994], [-73.87049783499994, 40.86817665499995], [-73.87049187, 40.86795082999989], [-73.87045509099993, 40.866632868999886], [-73.87041916899989, 40.86536282199993], [-73.87046131799995, 40.86327149799997], [-73.87057937699994, 40.86140455299988], [-73.87088409199991, 40.85889518099991], [-73.87097845199983, 40.85819941499993], [-73.87103268899989, 40.857807796999936], [-73.87108262599999, 40.857377039999946], [-73.8710759809998, 40.85734166099993], [-73.87107594599998, 40.85734146299997], [-73.87107580399994, 40.85734126499989], [-73.87104967499978, 40.857305152999885], [-73.87104944999982, 40.857304836999965], [-73.87104905899983, 40.857304602999946], [-73.87100272699996, 40.857277112999924], [-73.87100227699989, 40.857276842999966], [-73.87100173199987, 40.85727674299992], [-73.87094462199981, 40.85726608099992]]]}}, {\"id\": \"31\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 32, \"Shape_Leng\": 0.05426721601, \"Shape_Area\": 0.000150879171971, \"zone\": \"Bronxdale\", \"LocationID\": 32, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85882515999995, 40.85806135699987], [-73.8617906939999, 40.85798290399992], [-73.86287784399997, 40.85795359099992], [-73.8629585349999, 40.857951413999956], [-73.86318907699996, 40.85794519699991], [-73.86326398600002, 40.85794317599989], [-73.86387047899979, 40.85792681599985], [-73.86395713499984, 40.857924478999955], [-73.86455073499997, 40.857908462999944], [-73.86467028199996, 40.85790523699997], [-73.86564157999992, 40.857881545999916], [-73.8665833139999, 40.85785535399993], [-73.86758752499986, 40.85783008999988], [-73.86802029299992, 40.85783059499988], [-73.86807772399996, 40.85784684199991], [-73.86813140299985, 40.857876592999915], [-73.86817334099987, 40.85791854899991], [-73.86819723700003, 40.857967986999896], [-73.86820126399982, 40.85801835599993], [-73.8681883059999, 40.858063833999964], [-73.86841539699994, 40.85778035799994], [-73.86889622699992, 40.85777277299994], [-73.86954868099988, 40.85772602999987], [-73.87084887399999, 40.85729081699992], [-73.87087754099998, 40.85727822399984], [-73.87088962499988, 40.85727291599993], [-73.87088983900003, 40.85727281799995], [-73.87089011199996, 40.85727278099992], [-73.87094366199995, 40.85726605199995], [-73.87094413599989, 40.85726598899989], [-73.87094462199981, 40.85726608099992], [-73.87100173199987, 40.85727674299992], [-73.87100227699989, 40.857276842999966], [-73.87100272699996, 40.857277112999924], [-73.87104905899983, 40.857304602999946], [-73.87104944999982, 40.857304836999965], [-73.87104967499978, 40.857305152999885], [-73.87107580399994, 40.85734126499989], [-73.87107594599998, 40.85734146299997], [-73.8710759809998, 40.85734166099993], [-73.87108262599999, 40.857377039999946], [-73.87103268899989, 40.857807796999936], [-73.87097845199983, 40.85819941499993], [-73.87088409199991, 40.85889518099991], [-73.87057937699994, 40.86140455299988], [-73.87046131799995, 40.86327149799997], [-73.87041916899989, 40.86536282199993], [-73.87045509099993, 40.866632868999886], [-73.87049187, 40.86795082999989], [-73.87049783499994, 40.86817665499995], [-73.87049958099983, 40.86824277599994], [-73.87051057099994, 40.86865883499998], [-73.87051473299981, 40.86881643199997], [-73.87051629799993, 40.868875664999926], [-73.870526217, 40.869251176999924], [-73.87052897299995, 40.86935553299992], [-73.87053320099994, 40.86951557999986], [-73.87053494699997, 40.869581702999895], [-73.87053738699991, 40.8696740309999], [-73.87056192499989, 40.87151077799995], [-73.86935122399996, 40.871483353999906], [-73.86825887799992, 40.87146232799988], [-73.8671586419999, 40.871441912999934], [-73.86614489499986, 40.871422984999924], [-73.865208901, 40.87140764999989], [-73.86426604399986, 40.87138964999989], [-73.86333198300004, 40.8713751569999], [-73.86239212899996, 40.8713567339999], [-73.86137924099984, 40.8713365119999], [-73.8614338389999, 40.869603467999866], [-73.86149826099988, 40.86778221599998], [-73.86150246499992, 40.86755143299991], [-73.86154239799991, 40.86615362299991], [-73.86156274099987, 40.86554946399989], [-73.86055100799992, 40.8655296959999], [-73.85961281199982, 40.865512717999856], [-73.85890956799982, 40.864636925999875], [-73.85871220299993, 40.86425255299985], [-73.85844789699989, 40.86376138999988], [-73.85806098599983, 40.86304235999992], [-73.85780378799993, 40.86303621699989], [-73.85771028399981, 40.862278229999895], [-73.85770552799988, 40.862241284999925], [-73.85769842199993, 40.86218608099991], [-73.85769181699992, 40.8621347729999], [-73.85767885599986, 40.86203407799992], [-73.85767084800001, 40.86197187299994], [-73.8576092669999, 40.861493440999894], [-73.85759960699981, 40.86141838899994], [-73.85756682699994, 40.8611637229999], [-73.85752991799988, 40.86090316799989], [-73.85752224299985, 40.86084898099988], [-73.85747885699989, 40.8605427009999], [-73.85747219599985, 40.860495676999925], [-73.85746244199993, 40.860426811999915], [-73.85739481999993, 40.859984102999896], [-73.85735319699984, 40.859711605999934], [-73.8566377469999, 40.8581191719999], [-73.85882515999995, 40.85806135699987]]]}}, {\"id\": \"32\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 33, \"Shape_Leng\": 0.0532702931967, \"Shape_Area\": 0.000147416802448, \"zone\": \"Brooklyn Heights\", \"LocationID\": 33, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99619227264343, 40.70337714093203], [-73.99513826820316, 40.702974114861405], [-73.99506438495756, 40.70310031332674], [-73.99504592724139, 40.70313183989976], [-73.99490815999987, 40.7030473409999], [-73.99470923399984, 40.702993885999895], [-73.99444009399991, 40.702909269999964], [-73.99417681200005, 40.702824653999855], [-73.99380996599997, 40.70267441199997], [-73.99325388199995, 40.702411970999876], [-73.99286016299997, 40.702202766999896], [-73.99261204999985, 40.702061128999915], [-73.99232767899991, 40.70185536999985], [-73.99221871100002, 40.70177257799986], [-73.9921106779999, 40.701693442999925], [-73.99206681900003, 40.70165885999988], [-73.99202772899994, 40.70162814199991], [-73.99193644899997, 40.701556408999856], [-73.99126294499995, 40.701027101999905], [-73.99093201799984, 40.7007662159999], [-73.99087453800003, 40.70067311699993], [-73.99083516199983, 40.7005697619999], [-73.99081766499997, 40.70046229299989], [-73.99082272099989, 40.7003547539999], [-73.99084157899988, 40.700281256999844], [-73.9910295809998, 40.699847169999956], [-73.99124033199993, 40.69936760199986], [-73.99128347399984, 40.699234293999915], [-73.99132124499982, 40.69910004499998], [-73.99135360999992, 40.698964980999925], [-73.99138053899995, 40.6988292269999], [-73.991402007, 40.698692910999945], [-73.99141799499986, 40.69855615899995], [-73.99142848599986, 40.6984190989999], [-73.99143347099985, 40.69828185899997], [-73.99142289799994, 40.69778099299988], [-73.99136812199991, 40.69701375299993], [-73.99128413699994, 40.69658820999993], [-73.99120620999996, 40.69635779799992], [-73.99118780399986, 40.696287414999894], [-73.99115539099992, 40.696196475999926], [-73.99080803800004, 40.69502713399987], [-73.99066605599978, 40.69456882399993], [-73.99044474099998, 40.69382825099991], [-73.99045286599994, 40.693720583999934], [-73.9905879449999, 40.693391378999905], [-73.99072895999997, 40.693047700999905], [-73.99085041499994, 40.6927982509999], [-73.99097187, 40.692548799999926], [-73.99066626299994, 40.69249717699989], [-73.98947238599989, 40.69229549399994], [-73.98910541699992, 40.69215331399992], [-73.98902944799994, 40.69212386099992], [-73.98930270499983, 40.691060225999884], [-73.98930768499997, 40.690991687999926], [-73.98973690099999, 40.690359972999886], [-73.99016617899996, 40.689729533999916], [-73.99054474599986, 40.689168104999936], [-73.99070669599995, 40.68913674399992], [-73.99236366999985, 40.689690123999924], [-73.99436092799984, 40.69023947999991], [-73.99623278099983, 40.69075777399991], [-73.99776924399993, 40.691194697999954], [-73.99803048599995, 40.69125372299988], [-73.9984330559999, 40.691364914999895], [-73.99910057299985, 40.69153697399986], [-73.99920161899989, 40.69157183399992], [-73.99928819599991, 40.691593959999935], [-73.99936540199982, 40.69161307799989], [-73.99951274099993, 40.6916514499999], [-73.99991517399985, 40.69176722999993], [-74.00021419499998, 40.69185176099989], [-74.00031547799995, 40.69187465199993], [-74.00110519399988, 40.692056594999954], [-74.0016998939999, 40.692382766999906], [-74.00174362072502, 40.69240674985266], [-74.0015417289999, 40.69278596299994], [-74.00127014951309, 40.69329606183691], [-74.0009586845849, 40.694069083791064], [-74.00301393814541, 40.69477784423956], [-74.0026355689508, 40.69536203701875], [-74.00239802583795, 40.695711654864475], [-74.00026872377784, 40.69496580257801], [-73.99988214960374, 40.695605199245655], [-73.9994614344863, 40.69628002660429], [-73.9994197190206, 40.69634693781851], [-73.99941302433201, 40.69635767645331], [-73.99952373903493, 40.69657482604084], [-74.00048933405957, 40.696926195088054], [-74.00043147340423, 40.69705246617853], [-73.99959333515216, 40.69678226147438], [-73.99953135202674, 40.69687661261934], [-73.99958770006698, 40.6968980571681], [-73.99955389083655, 40.69694523239685], [-73.99940738376904, 40.69688518944602], [-73.99950317795836, 40.696726507193716], [-73.99947406820583, 40.69670413446199], [-73.99929468836199, 40.696795125794004], [-73.99918198950836, 40.69688518856733], [-73.99899604139205, 40.696812278373024], [-73.99893968968644, 40.69694094051582], [-73.99887207210413, 40.69692378457409], [-73.99876458027717, 40.697120733093975], [-73.99887117097644, 40.69715757729577], [-74.0010490305752, 40.697908236697415], [-74.00048768562507, 40.69875777763452], [-73.9983776068576, 40.698063296146074], [-73.99802102725101, 40.69876173657377], [-74.00001840433616, 40.699466048073575], [-73.99947463838932, 40.70032277801827], [-73.9974989831, 40.699638869321234], [-73.99736745811681, 40.69963423442719], [-73.99716054243916, 40.69979281264407], [-73.99718148446391, 40.69999448496808], [-73.9971758435588, 40.70010599146384], [-73.99724462871697, 40.70014140989715], [-73.99669768997464, 40.70087697691713], [-73.9968218908586, 40.70099078282786], [-73.99752763273973, 40.701228816958704], [-73.99776011033609, 40.70135941487678], [-73.99780162786318, 40.70131939476617], [-73.99813879899439, 40.701518788248414], [-73.99619227264343, 40.70337714093203]]]}}, {\"id\": \"33\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 34, \"Shape_Leng\": 0.0657059323545, \"Shape_Area\": 0.000173946146651, \"zone\": \"Brooklyn Navy Yard\", \"LocationID\": 34, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.97232032119902, 40.70908288315216], [-73.97066975742996, 40.70725970882147], [-73.97054062182238, 40.707316076353834], [-73.9702621853228, 40.706887504284005], [-73.97048881848585, 40.70540066562183], [-73.97016894509909, 40.70536185340732], [-73.97018075572805, 40.70492927056683], [-73.96762114779945, 40.70334850519434], [-73.96751516691087, 40.703437356878126], [-73.96747674493034, 40.703949870956215], [-73.9683326374704, 40.70448338458415], [-73.96929693837701, 40.705088331229625], [-73.96923632685235, 40.70599431249845], [-73.96929296374243, 40.70709333104775], [-73.96838933699995, 40.70682918699989], [-73.96795139100001, 40.705731216999894], [-73.96761547799979, 40.70496163499988], [-73.96720961999985, 40.70425570899993], [-73.96653747199996, 40.70369661299984], [-73.96607413599993, 40.703320152999936], [-73.96587418600001, 40.70315769499991], [-73.96520438999994, 40.70262274499987], [-73.96451844799995, 40.702074977999864], [-73.96386889399994, 40.70154188899993], [-73.96318252099994, 40.70100009499989], [-73.9625725539999, 40.70051219899991], [-73.96217978199998, 40.70022070999989], [-73.9623657999999, 40.69960558599991], [-73.96283963699993, 40.698038667999946], [-73.96288219599991, 40.69782702599988], [-73.96295416199992, 40.69760374899989], [-73.96305480199987, 40.69739287199992], [-73.96317583599986, 40.69718762099991], [-73.96331636199999, 40.69698969799986], [-73.96347515199992, 40.69680066799991], [-73.96365073599986, 40.69662194399993], [-73.963803372, 40.69649503799992], [-73.963971858, 40.69637865499991], [-73.96415417899982, 40.69627444999987], [-73.96434801500007, 40.69618376599989], [-73.9645507049999, 40.696107591999855], [-73.96546721499983, 40.695932161999956], [-73.96641979599997, 40.69596410599996], [-73.96741363299996, 40.69599862899988], [-73.96840655900004, 40.6960364839999], [-73.9693945009999, 40.696080388999846], [-73.9697278989999, 40.69780401399986], [-73.97076190899998, 40.69783928399989], [-73.97120281199977, 40.69785798999991], [-73.97178445299986, 40.697882663999955], [-73.97279477799988, 40.69792083799993], [-73.97332378599988, 40.697939650999906], [-73.97377513399991, 40.697959390999884], [-73.97478615599996, 40.69800455299992], [-73.97519315199986, 40.69802218699994], [-73.97578275800007, 40.698047729999864], [-73.97678642599985, 40.69808694699989], [-73.97753101699993, 40.69811581199988], [-73.97808892799996, 40.69813832499989], [-73.98054419199988, 40.69824651199988], [-73.98048604199988, 40.69927922799994], [-73.98048077899998, 40.6993727139999], [-73.98046217299986, 40.69970315799986], [-73.98044078599999, 40.7001836299999], [-73.98041744299996, 40.700518152999926], [-73.98040482499992, 40.700699006999855], [-73.98074079499993, 40.70127730999996], [-73.9806490779999, 40.70156087299994], [-73.98022432, 40.703063323999885], [-73.97998854099983, 40.70354105799989], [-73.97960292799985, 40.704176399999916], [-73.9793276249999, 40.70474627599989], [-73.97912179285028, 40.70552703403614], [-73.97896355081058, 40.70550219650426], [-73.97886517755374, 40.7058679562003], [-73.97881909040038, 40.70586246808449], [-73.97892765823725, 40.70549619916906], [-73.9788031024613, 40.70547222712397], [-73.97873829486646, 40.7054630646721], [-73.97863906658567, 40.705830648221756], [-73.97860162154001, 40.70582296852473], [-73.9786969819047, 40.70545626045808], [-73.97868440584477, 40.70545417710785], [-73.97864529228849, 40.705591021595616], [-73.97846557848955, 40.705562221864405], [-73.97843202250104, 40.70567863587691], [-73.97845692665027, 40.70568467134955], [-73.97844899155665, 40.705716545920126], [-73.97842182374887, 40.705713093959766], [-73.97838895552232, 40.705830254587326], [-73.97842178331251, 40.70583801473865], [-73.97841271462556, 40.70587075035045], [-73.97838101898172, 40.70586643677709], [-73.97839230543774, 40.7059706823108], [-73.97840136144173, 40.705972407194324], [-73.97839909439138, 40.70598274520238], [-73.9788564309653, 40.706059505100065], [-73.97885303360711, 40.7060655352846], [-73.97839229957326, 40.70598877473573], [-73.9783877652644, 40.70600600388105], [-73.97834474852871, 40.70599996498762], [-73.97835155258468, 40.705965506274495], [-73.97837419227056, 40.70596809446831], [-73.97836290650389, 40.70586384893295], [-73.97834139805158, 40.70585953726306], [-73.97834706969712, 40.70582507783397], [-73.97837423698817, 40.70583025182303], [-73.97840597500222, 40.70571050704935], [-73.97838559905202, 40.70570533367302], [-73.97839127010316, 40.70567259692237], [-73.97841617508323, 40.705676048465634], [-73.97845405328272, 40.705560375234064], [-73.97842683362268, 40.7055568085091], [-73.97859313620125, 40.70497858414276], [-73.97853198820685, 40.704876523429625], [-73.97844190715928, 40.70487139584311], [-73.97828301996252, 40.7046665516455], [-73.9781619969167, 40.70452842757676], [-73.97817256987123, 40.704485841835506], [-73.97817688845535, 40.70446844361957], [-73.97802067326661, 40.70429096645618], [-73.97735570894386, 40.70417808977397], [-73.97733878404608, 40.7042295507855], [-73.97726204580005, 40.70422009633448], [-73.9772119633924, 40.70421392662421], [-73.97751215494964, 40.70319675266348], [-73.97736083833689, 40.70313952051067], [-73.97729538355674, 40.70318930808753], [-73.97713562625376, 40.703005581402515], [-73.9770551895912, 40.70317876839772], [-73.97683246363418, 40.70361574187124], [-73.97623864768494, 40.70477564419063], [-73.97596096344841, 40.704715506440465], [-73.9769274918834, 40.70279782040038], [-73.97664059239023, 40.702496766094974], [-73.97634835593101, 40.70229910574264], [-73.97615937292022, 40.70217128035684], [-73.9755345648972, 40.703042808805726], [-73.97532987393977, 40.70295192433834], [-73.97594762102841, 40.702011996502776], [-73.97594746888169, 40.70201149422442], [-73.97598092723497, 40.70197503589959], [-73.97600587845436, 40.7019461337818], [-73.97589598081227, 40.7019004713829], [-73.9758141788267, 40.7018951826446], [-73.97581234475751, 40.70189450427652], [-73.97559301790525, 40.70181378876836], [-73.97560071840401, 40.70180646678156], [-73.97561086931884, 40.70179681450733], [-73.97550083176418, 40.70175830896567], [-73.97540348224568, 40.701911147567515], [-73.97455023388345, 40.70159034430477], [-73.97561192264291, 40.70003731645754], [-73.97581713783005, 40.699737121272676], [-73.97581945748546, 40.699706298712556], [-73.97581506305184, 40.69967560732393], [-73.97580407500232, 40.6996458884458], [-73.97578679455877, 40.699617956756555], [-73.9757636954341, 40.69959257794206], [-73.97573541084542, 40.69957044770558], [-73.97570271615527, 40.699552172697445], [-73.97566650761654, 40.69953825388533], [-73.9756277778049, 40.699529072821], [-73.97558758840917, 40.69952488118219], [-73.97554704112723, 40.6995257938729], [-73.97550724746861, 40.699531785873795], [-73.97546929828381, 40.69954269292822], [-73.97543423386269, 40.69955821604439], [-73.97540301541854, 40.69957792969204], [-73.97537649873668, 40.699601293466394], [-73.97535541071757, 40.69962766690227], [-73.9751895311087, 40.69987852225742], [-73.97431887009537, 40.70119515534329], [-73.97411513926478, 40.70130527141927], [-73.973888788485, 40.70108519489564], [-73.97385488106505, 40.70108398602825], [-73.973762109366, 40.70108068054629], [-73.97354417557611, 40.70089229252952], [-73.97355597076056, 40.70086763188122], [-73.97328853602355, 40.700624184552254], [-73.97292642848619, 40.70029455342767], [-73.97257311967198, 40.7000820561209], [-73.97239201397464, 40.700166322752274], [-73.97262103106195, 40.70046599557619], [-73.97298290412095, 40.70080626574634], [-73.973757007132, 40.701534138881854], [-73.97318676670886, 40.70166980367368], [-73.9728840477196, 40.701416795976], [-73.97284574196057, 40.7014347234145], [-73.97156913894533, 40.70030067964449], [-73.97113824583214, 40.69991789325807], [-73.97097896857517, 40.69985645765218], [-73.97078200587988, 40.69997866301123], [-73.97080524432837, 40.70009750034227], [-73.97254221267232, 40.70163644279681], [-73.97250810272976, 40.70166238646294], [-73.9733245610394, 40.70241265601073], [-73.97283512788735, 40.70272874074404], [-73.96982556559514, 40.70024507036794], [-73.96948655218084, 40.70051906125818], [-73.97283889974949, 40.70334642860182], [-73.97266938433755, 40.70350138043111], [-73.97183169654913, 40.702978669579736], [-73.96924539207352, 40.70136474188357], [-73.96893474818414, 40.70161586820729], [-73.96898517428966, 40.70174708878653], [-73.97147483845808, 40.7033165413165], [-73.97231246276117, 40.703844539538565], [-73.97215798824955, 40.70397020812867], [-73.97145657354827, 40.703508715924514], [-73.97110342298826, 40.70345973583976], [-73.97093307886338, 40.703624772048734], [-73.97101963118519, 40.70381561368637], [-73.97340440827125, 40.705304335604026], [-73.97465569608048, 40.70607426299424], [-73.97439083502317, 40.70632577414475], [-73.97347629598461, 40.70568628339133], [-73.97254880057814, 40.70620449000379], [-73.97329876359842, 40.70669771377458], [-73.97462966604417, 40.70768106076819], [-73.9742361696854, 40.70802438612241], [-73.97310627091164, 40.70713556922055], [-73.9722179484904, 40.70650549303291], [-73.97170524941599, 40.70613603268091], [-73.9712249291223, 40.70578990041363], [-73.97109655489108, 40.705850056558255], [-73.97100438685837, 40.70682824094664], [-73.97105692306172, 40.70694643381669], [-73.97275756998387, 40.70883127854767], [-73.97232032119902, 40.70908288315216]]], [[[-73.97892920137538, 40.705931589879214], [-73.97893110285521, 40.70592490623034], [-73.97906084872884, 40.705946028470365], [-73.97905767006142, 40.705952889050316], [-73.97892920137538, 40.705931589879214]]], [[[-73.98237340600002, 40.70554334999991], [-73.98245898620758, 40.705542414486125], [-73.98242287179853, 40.70582205595453], [-73.98102390618274, 40.705898913893535], [-73.98101733900947, 40.70579750646778], [-73.98230205183869, 40.705736979399674], [-73.98232944175258, 40.705552033831964], [-73.98237340600002, 40.70554334999991]]]]}}, {\"id\": \"34\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 35, \"Shape_Leng\": 0.085787918592, \"Shape_Area\": 0.000323825195932, \"zone\": \"Brownsville\", \"LocationID\": 35, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90105329800004, 40.66212482099994], [-73.90066118699986, 40.66058615799992], [-73.900146584, 40.66065327699988], [-73.89988908799995, 40.66063980699986], [-73.89972880499995, 40.66070453499989], [-73.89965233299985, 40.66073541599994], [-73.89921760899985, 40.65911140399994], [-73.89883571499976, 40.657525941999936], [-73.89881834799998, 40.65745381299991], [-73.89880698499995, 40.65740666199988], [-73.89891727499989, 40.65739022499992], [-73.899041804, 40.657371657999846], [-73.89948109299996, 40.65732134299993], [-73.89968548099989, 40.65729793699992], [-73.89988986899985, 40.6572745209999], [-73.90002679299982, 40.65725384799987], [-73.9002942619999, 40.657208882999925], [-73.9003948669999, 40.657184204999936], [-73.90048767199994, 40.65714867899987], [-73.900574915, 40.657099946999885], [-73.90064904799996, 40.65704155799989], [-73.901185599, 40.6565145509999], [-73.90230238299979, 40.65578061099992], [-73.9024538329999, 40.65566436599985], [-73.90261911399998, 40.6555581099999], [-73.90279641599983, 40.655463210999876], [-73.90298366699997, 40.655380780999906], [-73.90317858399993, 40.655311633999915], [-73.90414221299994, 40.655146420999905], [-73.90446144000003, 40.655099535999895], [-73.90489261799986, 40.65503619899993], [-73.90507059799985, 40.655010060999935], [-73.90556272799998, 40.654937909999866], [-73.906067832, 40.6548638509999], [-73.90692334299995, 40.65473841999989], [-73.90784346399994, 40.65459152999988], [-73.90782538299986, 40.65453360399992], [-73.90780156999993, 40.654438948999925], [-73.90740035699999, 40.6528439899999], [-73.90821606899983, 40.65231208699988], [-73.90855790499995, 40.65209593799989], [-73.90933499299997, 40.65280859199989], [-73.9097001379998, 40.65415294699991], [-73.90972827499986, 40.65425186399989], [-73.90974279799993, 40.65431111199997], [-73.90975756199991, 40.654367116999865], [-73.90977962099994, 40.654466254999925], [-73.91012255699995, 40.655851770999945], [-73.91108323599995, 40.65570239299992], [-73.91199538399984, 40.65556155999991], [-73.91236458899995, 40.65551356499985], [-73.91314356299998, 40.65622086399992], [-73.91377943699983, 40.65678367899993], [-73.91402878099997, 40.65700370899984], [-73.91441247700004, 40.65736241899984], [-73.91556629599982, 40.65839257199986], [-73.91572803800001, 40.658550127999874], [-73.91707363499995, 40.65974535399988], [-73.91741819199994, 40.66005119199989], [-73.91818493100003, 40.660726208999954], [-73.91829936299985, 40.66116125699987], [-73.91860168199995, 40.661118048999846], [-73.91893607399996, 40.6614160109999], [-73.91920450999993, 40.661655199999835], [-73.91945727799987, 40.66188078499988], [-73.9195947979998, 40.66199567899985], [-73.9196271919999, 40.661708058999885], [-73.91966280899982, 40.66140503299994], [-73.91974399499986, 40.660770973999895], [-73.91990440499988, 40.66033711699987], [-73.919986348, 40.660115465999894], [-73.92008167099986, 40.659857632999866], [-73.92021313099984, 40.65997544099987], [-73.9209090419998, 40.66059911499987], [-73.92272843000003, 40.66223123199989], [-73.9245494589998, 40.66386259199993], [-73.92637970499992, 40.6655148919999], [-73.92592201499984, 40.66571503599991], [-73.92546270199979, 40.66591675199991], [-73.92439099400002, 40.66638142299991], [-73.92424467299992, 40.66644499999986], [-73.92331039799996, 40.66685345999985], [-73.92323224599997, 40.66688762799987], [-73.92311978499988, 40.666936793999945], [-73.92303439399991, 40.66697412599995], [-73.922811066, 40.66707176099994], [-73.92275257199998, 40.667097332999894], [-73.92221564099988, 40.6673334459999], [-73.92187171199997, 40.667480689999906], [-73.9210198879999, 40.667852697999905], [-73.92014864299998, 40.66822948399992], [-73.92011464299999, 40.668376001999924], [-73.91999873699986, 40.6684094839999], [-73.91923062999997, 40.66863136499991], [-73.91838191199984, 40.66900224699985], [-73.91748266100001, 40.66939223099988], [-73.9171358899999, 40.66954376399992], [-73.91658459999987, 40.6697875489999], [-73.91652296199993, 40.66981426399983], [-73.9164716349998, 40.669836508999886], [-73.91586213199994, 40.670100668999964], [-73.91580361899986, 40.670126027999906], [-73.9157287449999, 40.67015847799994], [-73.91558467299987, 40.67022136299989], [-73.91553422600005, 40.67024338399992], [-73.91487246299994, 40.6705322279999], [-73.91461857800005, 40.67064323799989], [-73.9145448559999, 40.67067547299995], [-73.91403857199998, 40.67089683999989], [-73.91308574699985, 40.67131022599989], [-73.91221776899997, 40.67168716299988], [-73.91133533699998, 40.672076452999896], [-73.9111599549999, 40.67215535099993], [-73.91109867399982, 40.67218291999989], [-73.91104631099985, 40.672204641999954], [-73.910463053, 40.672446601999866], [-73.91037473899996, 40.672483237999884], [-73.91019287099996, 40.67255868499989], [-73.91014265099993, 40.6725795189999], [-73.90995832899995, 40.672655980999906], [-73.90989833499988, 40.67268086899986], [-73.90954171999984, 40.67282880299995], [-73.90925438099987, 40.672947997999856], [-73.90841931599986, 40.673313332999925], [-73.9083295419999, 40.673355302999894], [-73.90827800999982, 40.67337939399993], [-73.90787711599997, 40.6735668089999], [-73.90746843099997, 40.67376161399989], [-73.90695655499991, 40.67398525899991], [-73.90629662099994, 40.6742653329999], [-73.90609438799999, 40.674352931999906], [-73.90582626799984, 40.674469653999886], [-73.90524012899992, 40.674724945999856], [-73.90450912299991, 40.67505211099989], [-73.90438792800002, 40.67520674699993], [-73.90428786799986, 40.67522935599987], [-73.90374374599986, 40.67540683399991], [-73.90347422300006, 40.67550668299985], [-73.90346228899996, 40.675457918999896], [-73.90344272399983, 40.67537791899986], [-73.9032213189999, 40.67447291399992], [-73.90296768999997, 40.67343939499992], [-73.90266008399996, 40.67219717899991], [-73.90228822599984, 40.67093600699995], [-73.90198218099997, 40.669670070999906], [-73.90166767899989, 40.668413528999885], [-73.90126570400001, 40.66688064999991], [-73.9008513569999, 40.665345894999895], [-73.90041654099997, 40.663806526999934], [-73.90050930099981, 40.663793574999914], [-73.90071711999985, 40.66376455299993], [-73.90145114700002, 40.66365836499993], [-73.90105329800004, 40.66212482099994]]]}}, {\"id\": \"35\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 36, \"Shape_Leng\": 0.0870507867063, \"Shape_Area\": 0.000247666139269, \"zone\": \"Bushwick North\", \"LocationID\": 36, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91254913799993, 40.70387341999989], [-73.91180710099985, 40.70343495199993], [-73.912904041, 40.7023618909999], [-73.91216169799988, 40.701923163999915], [-73.91142242699996, 40.70148658799996], [-73.91067882699998, 40.701045968999914], [-73.91180820099989, 40.69993800299986], [-73.91103489599992, 40.6995321139999], [-73.90953273199997, 40.69867510499995], [-73.90877366099991, 40.69825522699988], [-73.9080224919999, 40.69782540899991], [-73.90756985199991, 40.69757174999993], [-73.90727452399994, 40.69739918999991], [-73.9068458349999, 40.69716354899987], [-73.90652350699987, 40.69697614399997], [-73.906116261, 40.6967505569999], [-73.90576535999985, 40.6965485549999], [-73.9054012769999, 40.69635030599995], [-73.90500484799992, 40.696118745999904], [-73.90466384799993, 40.69593224699988], [-73.9042601839999, 40.69570037099989], [-73.90480435399992, 40.695148168999914], [-73.90491424299982, 40.69507357099997], [-73.90579597099993, 40.69412715499987], [-73.90504295399994, 40.69370400599987], [-73.90428792799995, 40.6932780679999], [-73.90386600499991, 40.693035711999904], [-73.90354057099982, 40.69284878199992], [-73.9027906169999, 40.69242104699992], [-73.90207277399993, 40.691999230999905], [-73.90170525399989, 40.69166024399989], [-73.90148893599996, 40.6915568839999], [-73.90123290699994, 40.69144227899997], [-73.90180467199995, 40.6907662979999], [-73.904055772, 40.68854627799986], [-73.90419686399987, 40.68858474399986], [-73.9044739719999, 40.68866816099992], [-73.90482499699988, 40.688808569999956], [-73.90504399299995, 40.688932240999875], [-73.9057928679998, 40.68936474999989], [-73.90654394399994, 40.68979093699993], [-73.90729183099991, 40.690219069999905], [-73.9080425449999, 40.690647001999885], [-73.90879653399988, 40.691074750999896], [-73.90955061499984, 40.69150297299994], [-73.91029860299984, 40.69192995199991], [-73.91105301, 40.692357245999894], [-73.911805654, 40.6927850819999], [-73.91255469999996, 40.69321201999987], [-73.91331306200004, 40.69363224199988], [-73.91405670900002, 40.694068467999884], [-73.914808253, 40.69449685199993], [-73.91502132199989, 40.69461706699989], [-73.91506798799989, 40.694643395999954], [-73.91556201299993, 40.69492212499987], [-73.916313257, 40.69535124599984], [-73.91655004899992, 40.6954859149999], [-73.91661144299985, 40.6955208309999], [-73.91670957700003, 40.695576641999864], [-73.91677235199991, 40.69561234299993], [-73.91686074399993, 40.69566261299991], [-73.91691887399988, 40.69569567199989], [-73.91707162999998, 40.69578254599988], [-73.91728545099988, 40.695904352999946], [-73.917361666, 40.695947768999844], [-73.91743326099993, 40.695988553999875], [-73.91750809599985, 40.6960311839999], [-73.91760570499984, 40.69608678699994], [-73.917698671, 40.69613974499987], [-73.91781824899988, 40.69620786299991], [-73.91800959500003, 40.69631614199996], [-73.91812143899979, 40.69637943199995], [-73.91824631899986, 40.69645009899988], [-73.91835160499986, 40.69650967699986], [-73.91842289799999, 40.69655001999995], [-73.91848765699991, 40.69658666399988], [-73.91856440699996, 40.69663009399993], [-73.91932072899982, 40.69706039699991], [-73.91956944199978, 40.69720230699989], [-73.91960508499987, 40.697222642999876], [-73.92007372399986, 40.69749003399989], [-73.92043053999987, 40.69769341199991], [-73.9204962049999, 40.69773083899991], [-73.92082397399983, 40.697917657999916], [-73.92157402199999, 40.698345508999914], [-73.92175647299993, 40.69845331699989], [-73.92232801999992, 40.69877209799994], [-73.92268944899993, 40.69897762199984], [-73.92308169399985, 40.69920067199989], [-73.92383267099994, 40.6996277979999], [-73.92458241299997, 40.700055251999906], [-73.92532959999984, 40.70048119499994], [-73.92608301599982, 40.70091014499993], [-73.92683805799987, 40.70133420999986], [-73.92758774299998, 40.701762580999855], [-73.92833850299998, 40.70219163999995], [-73.92909071699984, 40.70261967799991], [-73.92984252699986, 40.703046809999876], [-73.93059386199991, 40.70347330799992], [-73.93115533899987, 40.70377233499987], [-73.93172871799996, 40.703545990999906], [-73.93269784899994, 40.70317039099989], [-73.93281494999994, 40.70357711199993], [-73.93293503499983, 40.70399456099992], [-73.93301076499982, 40.70427708599988], [-73.93312548700004, 40.70470086199992], [-73.93320124199991, 40.70497168999993], [-73.93331989699992, 40.70539302699994], [-73.93340041899982, 40.7056682879999], [-73.93351245499996, 40.70609483499993], [-73.93359321499992, 40.70637065399992], [-73.93371088099994, 40.70679153099994], [-73.93379054199995, 40.70706657499993], [-73.93391870799988, 40.70748702599992], [-73.93229708600003, 40.70774944699988], [-73.9320728499999, 40.70778578599989], [-73.92902566099997, 40.70827681499986], [-73.92740683799988, 40.70854132199989], [-73.92576502499996, 40.7088037639999], [-73.92418647499989, 40.70906472899986], [-73.92248241799982, 40.709332487999895], [-73.92189184700005, 40.709396096999896], [-73.92143475999988, 40.7091287659999], [-73.92070065899999, 40.708694629999876], [-73.91995952899983, 40.708257287999885], [-73.91956473400002, 40.708023977999915], [-73.91922162699984, 40.707815133999844], [-73.91884342599998, 40.70759799799987], [-73.91847670899989, 40.70738156299989], [-73.918102996, 40.707159968999896], [-73.91808167700003, 40.70714732999993], [-73.91773662899993, 40.70694273899988], [-73.91736520300005, 40.7067224689999], [-73.9169950679999, 40.706502964999885], [-73.91664948199995, 40.706299387999955], [-73.91660892400002, 40.70627549499992], [-73.91625251999979, 40.70606554299993], [-73.91589738599998, 40.705854816999846], [-73.91551250499982, 40.705626444999965], [-73.91477063799996, 40.70518943699992], [-73.91403097499997, 40.704749112999885], [-73.91329162399998, 40.70431263899994], [-73.91254913799993, 40.70387341999989]]]}}, {\"id\": \"36\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 37, \"Shape_Leng\": 0.142810040466, \"Shape_Area\": 0.000452062644782, \"zone\": \"Bushwick South\", \"LocationID\": 37, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93312548700004, 40.70470086199992], [-73.93301076599988, 40.70427708599989], [-73.93293503499983, 40.70399456099992], [-73.93281494999994, 40.70357711199993], [-73.93269784899994, 40.70317039099989], [-73.93172871799996, 40.703545990999906], [-73.93115533899987, 40.70377233499987], [-73.93059386199991, 40.70347330799992], [-73.92984252699986, 40.703046809999876], [-73.92909071699984, 40.70261967799991], [-73.92833850299998, 40.70219163999995], [-73.92758774299998, 40.701762580999855], [-73.92683805799987, 40.70133420999986], [-73.92608301599982, 40.70091014499993], [-73.92532959999984, 40.70048119499994], [-73.92458241299997, 40.700055251999906], [-73.92383267099994, 40.6996277979999], [-73.92308169399985, 40.69920067199989], [-73.92268944899993, 40.69897762199984], [-73.92232801999992, 40.69877209799994], [-73.92175647299993, 40.69845331699989], [-73.92157402199999, 40.698345508999914], [-73.92082397399983, 40.697917657999916], [-73.9204962049999, 40.69773083899991], [-73.92043053999987, 40.69769341199991], [-73.92007372399986, 40.69749003399989], [-73.91960508499987, 40.697222642999876], [-73.91956944199978, 40.69720230699989], [-73.91932072899982, 40.69706039699991], [-73.91856440699996, 40.69663009399993], [-73.91848765699991, 40.69658666399988], [-73.91842289799999, 40.69655001999995], [-73.91835160499986, 40.69650967699986], [-73.91824631899986, 40.69645009899988], [-73.91812143899979, 40.69637943199995], [-73.91800959500003, 40.69631614199996], [-73.91781824899988, 40.69620786299991], [-73.917698671, 40.69613974499987], [-73.91760570499984, 40.69608678699994], [-73.91750809599985, 40.6960311839999], [-73.91743326099993, 40.695988553999875], [-73.917361666, 40.695947768999844], [-73.91728545099988, 40.695904352999946], [-73.91707162999998, 40.69578254599988], [-73.91691887399988, 40.69569567199989], [-73.91686074399993, 40.69566261299991], [-73.91677235199991, 40.69561234299993], [-73.91670957700003, 40.695576641999864], [-73.91661144299985, 40.6955208309999], [-73.91655004899992, 40.6954859149999], [-73.916313257, 40.69535124599984], [-73.91556201299993, 40.69492212499987], [-73.91506798799989, 40.694643395999954], [-73.91502132199989, 40.69461706699989], [-73.914808253, 40.69449685199993], [-73.91405670900002, 40.694068467999884], [-73.91331306200004, 40.69363224199988], [-73.91255469999996, 40.69321201999987], [-73.911805654, 40.6927850819999], [-73.91105301, 40.692357245999894], [-73.91029860299984, 40.69192995199991], [-73.90955061499984, 40.69150297299994], [-73.90879653399988, 40.691074750999896], [-73.9080425449999, 40.690647001999885], [-73.90729183099991, 40.690219069999905], [-73.90654394399994, 40.68979093699993], [-73.9057928679998, 40.68936474999989], [-73.90504399299995, 40.688932240999875], [-73.90482499699988, 40.688808569999956], [-73.9044739719999, 40.68866816099992], [-73.90419686399987, 40.68858474399986], [-73.904055772, 40.68854627799986], [-73.90180467199995, 40.6907662979999], [-73.90042465099988, 40.688183897999856], [-73.90116154999988, 40.68787793499991], [-73.8965264419999, 40.682403298999894], [-73.89646625099995, 40.68233642199988], [-73.89652633199988, 40.68200291799991], [-73.89658158699996, 40.6815747179999], [-73.89663536799999, 40.68155649499991], [-73.89674965999993, 40.68149202599986], [-73.89687298299991, 40.680895284999934], [-73.89692154999993, 40.680781612999915], [-73.89697886299997, 40.68067032699991], [-73.89704472199993, 40.68056181599996], [-73.89711889399996, 40.68045646099994], [-73.8972011219999, 40.680354629999904], [-73.89729111699982, 40.68025667899985], [-73.89738856299982, 40.68016295199989], [-73.89749312100001, 40.680073775999915], [-73.89767194200002, 40.6799688539999], [-73.89786299899986, 40.67987501099987], [-73.89806444599996, 40.67979343099991], [-73.89827418999997, 40.67972504299988], [-73.89848997800001, 40.67967050899991], [-73.89870944799999, 40.67963016999989], [-73.89889928299989, 40.67956093399993], [-73.89969830299987, 40.67954409199993], [-73.90047001100004, 40.679535243999936], [-73.90075235999984, 40.67967664399992], [-73.90262262299991, 40.6806645029999], [-73.90358874799992, 40.67967480399991], [-73.9040463979999, 40.67922059799985], [-73.90456891299982, 40.67951553699991], [-73.90479113399992, 40.67964096499992], [-73.9054141659999, 40.67999264299987], [-73.90549303499998, 40.68003715799995], [-73.905548368, 40.6800716019999], [-73.90565934199981, 40.68014069099989], [-73.90628287099989, 40.68050675599993], [-73.90671941199996, 40.68076742199985], [-73.90701820299995, 40.68094827999989], [-73.90777158799989, 40.68137784699997], [-73.90851642300002, 40.681803942999906], [-73.90886025599995, 40.681994035999914], [-73.90905526999994, 40.682105574999895], [-73.90927462899988, 40.68223432399993], [-73.91002460999994, 40.68265737099986], [-73.91076954399978, 40.68309270099994], [-73.911511929, 40.683504989999896], [-73.91227301199991, 40.68393496499991], [-73.91303721399989, 40.684367809999884], [-73.91330728099987, 40.6845208579999], [-73.91379700199988, 40.68479987199986], [-73.91453972399987, 40.68522691799994], [-73.91470083399992, 40.68531694499989], [-73.91528963499992, 40.685645535999875], [-73.91550033999997, 40.68576152399992], [-73.91604599299995, 40.68607206899985], [-73.9166103199999, 40.68639135299993], [-73.916798329, 40.68649719899987], [-73.91755226099991, 40.68693022299989], [-73.91768113899987, 40.68700409499997], [-73.91804607, 40.68721324799989], [-73.91829345799984, 40.68735666799993], [-73.91878323299994, 40.687626087999924], [-73.91906109199992, 40.687784368999864], [-73.91980961899992, 40.68821124099985], [-73.92055013100003, 40.68864092199994], [-73.920961039, 40.6888720519999], [-73.92121369999991, 40.68901947699986], [-73.92130587799981, 40.68907164399991], [-73.92207133799985, 40.68948984099992], [-73.922807024, 40.68992392499993], [-73.9231696669999, 40.690123405999884], [-73.9236437979999, 40.690398837999915], [-73.92429213299995, 40.690773356999905], [-73.92439183300002, 40.690831748999926], [-73.92536488899991, 40.69136634399996], [-73.92556064099996, 40.69149083999987], [-73.92631612499989, 40.691928452999896], [-73.92644992099981, 40.692001768999944], [-73.92707069499987, 40.692346223999955], [-73.92750685299994, 40.69260381099993], [-73.92781120699986, 40.692774208999936], [-73.92804330699994, 40.692904171999906], [-73.92850659699988, 40.693165725999904], [-73.92864268999999, 40.6932410129999], [-73.9288283419999, 40.693350098999936], [-73.92882897899982, 40.69335047399987], [-73.9297019239999, 40.69386339699991], [-73.93051146699992, 40.69431652299989], [-73.93114516499988, 40.69467397499987], [-73.93145784300005, 40.6948554709999], [-73.93177404599997, 40.69502821499992], [-73.93192890099984, 40.69511465699991], [-73.93236605699987, 40.69535837699995], [-73.93311862899988, 40.69579115399997], [-73.93382817199998, 40.69618427699988], [-73.93413397299992, 40.69635843099992], [-73.93448784799989, 40.696560267999914], [-73.93457999399985, 40.696612823999956], [-73.93460675199995, 40.69662808499985], [-73.93468892700004, 40.69667495399994], [-73.93479699699994, 40.6967365919999], [-73.93522264399988, 40.69698654499993], [-73.93600679799988, 40.697405748999905], [-73.93638762599981, 40.69761530199985], [-73.93675978999995, 40.69783118899988], [-73.93739064799986, 40.69818579499985], [-73.93777354599992, 40.69840469299989], [-73.93817718599986, 40.69864022299987], [-73.93856854799981, 40.698848135999874], [-73.93896470899988, 40.698801031999935], [-73.93906425899979, 40.69878919499992], [-73.93952025999982, 40.69873497499989], [-73.93960973999992, 40.69872433499995], [-73.94002089599995, 40.69867544399993], [-73.94011148799991, 40.69866467199989], [-73.94051277999984, 40.69861695199994], [-73.9405942079999, 40.69860726799988], [-73.94072013299996, 40.698592293999916], [-73.94081408799997, 40.69858111999986], [-73.94115035299997, 40.69854113199994], [-73.94176673799987, 40.69847070399985], [-73.94185058700003, 40.69846112399994], [-73.94247471499993, 40.698389807999945], [-73.94253735900001, 40.69838264999989], [-73.9429206409998, 40.69833885099996], [-73.94299724499987, 40.69833009799986], [-73.94394947299996, 40.698221278999924], [-73.94409591299996, 40.698972954999924], [-73.94424286099989, 40.69969927999991], [-73.94438788599984, 40.70042452299993], [-73.94193078899983, 40.70072523399989], [-73.94042057899993, 40.701076597999865], [-73.9405445659999, 40.701798044999855], [-73.94060119499999, 40.70214158099988], [-73.9406199669999, 40.702255449999896], [-73.94067555399978, 40.70259265099991], [-73.94076481199986, 40.703134032999884], [-73.94077939399978, 40.70322246899988], [-73.9408928719998, 40.70391415699989], [-73.94103009699987, 40.7046361769999], [-73.94168041699996, 40.70457403599987], [-73.9417696579999, 40.70456550799983], [-73.9421432489999, 40.7045298069999], [-73.9422114949999, 40.7045232859999], [-73.94273448399997, 40.70447330599988], [-73.94285013599988, 40.70518202999993], [-73.94296619399988, 40.70588843999991], [-73.94308587399995, 40.70659666199991], [-73.94320222899984, 40.7073347509999], [-73.94332294399995, 40.708070194999934], [-73.94344009799994, 40.70878089099988], [-73.94354434699991, 40.70941603099986], [-73.94355745099986, 40.709495864999965], [-73.94357094699997, 40.70957809399996], [-73.94358512999999, 40.70966450499993], [-73.94367685499988, 40.71022334799989], [-73.94368828299991, 40.71029296399993], [-73.94378799599994, 40.710900378999945], [-73.94357731299993, 40.710920286999894], [-73.94326978899984, 40.710949344999925], [-73.94283637099988, 40.7109902979999], [-73.94245994699992, 40.7110258639999], [-73.94210580699989, 40.71105932199989], [-73.94196964799987, 40.7110721309999], [-73.94159179599998, 40.711107670999915], [-73.94116852899995, 40.71114748299993], [-73.94075355899997, 40.711186512999916], [-73.94042251999994, 40.711217647999945], [-73.9404771349999, 40.711562642999866], [-73.94068803099992, 40.71196204799987], [-73.93846617699984, 40.71248925599995], [-73.93832001399986, 40.71252368499994], [-73.93724803599986, 40.712782740999884], [-73.93480281900003, 40.713372495999884], [-73.9345332899999, 40.712753303999925], [-73.93425709399997, 40.712123023999936], [-73.93389272699982, 40.711335677999905], [-73.933601465, 40.7106691059999], [-73.93329596399984, 40.709987710999926], [-73.93299690799995, 40.709317159999884], [-73.9326921559999, 40.70863203599986], [-73.93252310799994, 40.70823818799991], [-73.93229708600003, 40.70774944699988], [-73.93391870799988, 40.70748702599992], [-73.93379054199995, 40.70706657499993], [-73.93371087999985, 40.70679153099996], [-73.93359321499992, 40.70637065399992], [-73.93351245499996, 40.70609483499993], [-73.93340041899982, 40.7056682879999], [-73.93331989699992, 40.70539302699994], [-73.93320124099985, 40.704971689999915], [-73.93312548700004, 40.70470086199992]]]}}, {\"id\": \"37\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 38, \"Shape_Leng\": 0.0832175685234, \"Shape_Area\": 0.000327392684821, \"zone\": \"Cambria Heights\", \"LocationID\": 38, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.73624941199995, 40.702536977999884], [-73.73430403699984, 40.702478130999936], [-73.73426739299985, 40.70316624699992], [-73.73426455399982, 40.703219541999935], [-73.7333187329999, 40.70319051999997], [-73.73245633499985, 40.70316405099986], [-73.732314832, 40.70315970599992], [-73.73036969899985, 40.70309853299995], [-73.7277679189999, 40.70302063799987], [-73.7273634669998, 40.70303714099988], [-73.72725657600003, 40.703041498999916], [-73.72713924599991, 40.70304627199988], [-73.72677759516957, 40.7030592911994], [-73.72672841210559, 40.7021457358777], [-73.72678346816939, 40.70198712397415], [-73.72673796384797, 40.7015879518937], [-73.72665409175167, 40.70111265819504], [-73.72654435079282, 40.69898264008306], [-73.72652452210123, 40.698258092187295], [-73.72651147511033, 40.69778119231719], [-73.72650934112171, 40.69751670520512], [-73.72650630194262, 40.697138805009], [-73.72650617482721, 40.69712356793277], [-73.72650576920826, 40.69707238185047], [-73.72650566904106, 40.69705938782327], [-73.72650562109436, 40.69705367791967], [-73.72650548804287, 40.697037243007145], [-73.72650547377476, 40.69703499178466], [-73.72650545484571, 40.697033793849265], [-73.72650544889288, 40.69703238923185], [-73.72650541791589, 40.69702834513061], [-73.72650538383098, 40.69702492404414], [-73.72650531499247, 40.697015963925466], [-73.72650318512274, 40.696880358755614], [-73.72626254324406, 40.69077325308489], [-73.7262583130798, 40.6906108621615], [-73.7262588651076, 40.69026892794887], [-73.72620259721364, 40.68898434090529], [-73.72617495100188, 40.68865003005495], [-73.72616338309388, 40.68845361784905], [-73.72599616320741, 40.686932054144755], [-73.72594098579772, 40.68639910518071], [-73.72593196221484, 40.686311940908524], [-73.725926356196, 40.68622520812825], [-73.72579787921153, 40.68435384991247], [-73.72580147979704, 40.68428378038721], [-73.72586264191175, 40.683241080973865], [-73.72658306499989, 40.68341482999993], [-73.72666460799998, 40.68343688899992], [-73.72675245899993, 40.68346409399994], [-73.72718272499995, 40.68359732999987], [-73.72785251200004, 40.68380044199992], [-73.72872058399992, 40.68406239099987], [-73.72959149799996, 40.68432533799995], [-73.73045844999993, 40.68458679499984], [-73.73132732799995, 40.68485054399987], [-73.73222005799998, 40.68512109599991], [-73.73308454599989, 40.6853825349999], [-73.73350033600005, 40.685500584999865], [-73.73355835299988, 40.68551705299989], [-73.73361575199982, 40.68553440399995], [-73.73396025399995, 40.68563856299993], [-73.73451246099985, 40.685808342999955], [-73.73483039799987, 40.68590609699992], [-73.73569987899988, 40.68617269699991], [-73.73657845699985, 40.68643801399987], [-73.737321908, 40.686664320999924], [-73.73744163699996, 40.68670076699988], [-73.7375999409999, 40.686362011999904], [-73.74125253099989, 40.68745783899989], [-73.74746788899986, 40.68932218099992], [-73.7475517539999, 40.68934733599984], [-73.74728617699986, 40.689737172999834], [-73.74624345599987, 40.6912794439999], [-73.74510485099985, 40.69278293499986], [-73.74487951500002, 40.69309307799993], [-73.74484610600003, 40.69313905699987], [-73.74479652999989, 40.693220371999864], [-73.74467955599985, 40.693412262999885], [-73.7445271819999, 40.69369059899986], [-73.74438939, 40.69397343399991], [-73.74426648799988, 40.69426010099992], [-73.74415874099992, 40.69454989999984], [-73.74386913499988, 40.695400117999945], [-73.7435928609999, 40.69621532499986], [-73.74365178399992, 40.69622415699989], [-73.74458386999987, 40.69636378199985], [-73.74474545399995, 40.69638546499987], [-73.7449099489999, 40.69639459999991], [-73.74507479999994, 40.69639094899984], [-73.74523747599996, 40.696374658999936], [-73.74612956299988, 40.69613573299988], [-73.74618157999997, 40.696121925999904], [-73.74649367299993, 40.696799873999936], [-73.74681909299984, 40.697525451999844], [-73.74712436899999, 40.698198034999926], [-73.74741867999983, 40.69885902799989], [-73.74773012199991, 40.69955313699991], [-73.74767063100002, 40.699568575999876], [-73.74677542799994, 40.69980088799986], [-73.74588751799996, 40.70003041699988], [-73.74500119100004, 40.70026221299988], [-73.74590871000004, 40.70228727399988], [-73.74502395299994, 40.70251829799986], [-73.7441343549998, 40.7027421119999], [-73.74386175999997, 40.70281109599994], [-73.74330723899996, 40.70295979699989], [-73.74287389299987, 40.703074942999876], [-73.74167563399995, 40.70338409999992], [-73.74148805299996, 40.70343161199993], [-73.74129277999985, 40.70346424899991], [-73.74109293999989, 40.7034810629999], [-73.74089188600004, 40.70348164599992], [-73.74069300199989, 40.70346611399988], [-73.74055102499982, 40.70344338099995], [-73.74050336499984, 40.70343355599992], [-73.74051703299997, 40.70338761599989], [-73.74077675800001, 40.702679837999916], [-73.74072128199992, 40.70267801899989], [-73.73819579999989, 40.70259533599989], [-73.73624941199995, 40.702536977999884]]]}}, {\"id\": \"38\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 39, \"Shape_Leng\": 0.132369620757, \"Shape_Area\": 0.00090028293862, \"zone\": \"Canarsie\", \"LocationID\": 39, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.89121853499988, 40.64998769899987], [-73.89056792961276, 40.64920863296715], [-73.89058122439945, 40.64921017109779], [-73.89058918285595, 40.649211440157856], [-73.89059695454304, 40.64921326406146], [-73.8906044702013, 40.649215626827385], [-73.89060767237343, 40.64921690945508], [-73.89061166783519, 40.64921850963346], [-73.89061848326281, 40.64922188561435], [-73.89063192466081, 40.649226539268916], [-73.89064497244473, 40.64923180302112], [-73.89065757649318, 40.64923765973553], [-73.89069206343918, 40.64925829432288], [-73.8907082114337, 40.649268550512616], [-73.89072347249865, 40.649284766350156], [-73.89073969963145, 40.64930042925473], [-73.89075685659586, 40.649315505182834], [-73.89076294103755, 40.649323105694315], [-73.89076981092721, 40.64933030739637], [-73.89077742674992, 40.64933706351059], [-73.89078573315822, 40.649343324563745], [-73.89079467610247, 40.649349054820526], [-73.89080352489603, 40.64935465816821], [-73.89081292865663, 40.64935970986859], [-73.8908228295893, 40.64936417736595], [-73.89083316373392, 40.64936803262233], [-73.89084386800218, 40.6493712519563], [-73.89085292300992, 40.64937409864865], [-73.89086225721337, 40.64937636210385], [-73.89087180795605, 40.64937802400179], [-73.89088150684604, 40.64937907573349], [-73.89089128812886, 40.64937950986542], [-73.8909010812095, 40.6493793204669], [-73.89091635502108, 40.64937809809264], [-73.8909315102687, 40.649376208170885], [-73.89094649855325, 40.64937365601642], [-73.89096126839027, 40.649370450962365], [-73.89097577159515, 40.64936660167447], [-73.89098074859662, 40.64936383966098], [-73.89098963485273, 40.64935891018467], [-73.89100298051534, 40.64935070556575], [-73.89101577841615, 40.64934200789259], [-73.89102799486605, 40.64933283924741], [-73.89103960167664, 40.64932322104731], [-73.89105056977336, 40.649313177891514], [-73.89105967199393, 40.64930394866298], [-73.89106087184408, 40.649302733208444], [-73.89107048189092, 40.64929191361045], [-73.89107937963642, 40.649280744710374], [-73.89108753996057, 40.649269254629544], [-73.89109494390252, 40.64925747099225], [-73.89110204108817, 40.649246311442404], [-73.89110986320975, 40.64923543804672], [-73.89111839306874, 40.649224876420924], [-73.89112760862514, 40.64921465368332], [-73.89113748520066, 40.649204796615074], [-73.8911479974592, 40.64919533048879], [-73.89115673657176, 40.64918821930543], [-73.89115912248874, 40.64918627773139], [-73.89117082990147, 40.64917765975775], [-73.89118308930125, 40.649169502337934], [-73.89118832602998, 40.649162972512464], [-73.89119290665816, 40.64915616095332], [-73.89119680582479, 40.64914910834607], [-73.89119999971275, 40.649141853032205], [-73.89120247461622, 40.6491344373835], [-73.89120421309674, 40.64912689957999], [-73.89120520716251, 40.64911928484688], [-73.89120545080615, 40.649111635395634], [-73.89120504911666, 40.64910559980511], [-73.89120494088279, 40.649103991765074], [-73.89120368479722, 40.6490963990275], [-73.89120168467845, 40.649088900574554], [-73.89119895343714, 40.64908153763062], [-73.89119550640645, 40.64907434907656], [-73.8911913641941, 40.64906737664634], [-73.89118768684253, 40.64904537266772], [-73.89118321862813, 40.64902345237476], [-73.89117796546067, 40.64900163403383], [-73.89117192863348, 40.64897993423124], [-73.8911651109834, 40.64895836737716], [-73.89115752325999, 40.64893695090433], [-73.89114916653827, 40.64891570056145], [-73.89114005025007, 40.64889463277507], [-73.8911301778888, 40.6488737636309], [-73.8911195591107, 40.64885310687541], [-73.89110820334591, 40.648832679772546], [-73.891096114747, 40.648812499079], [-73.89108330605376, 40.648792575696], [-73.8910405899315, 40.648743675910005], [-73.89100218065924, 40.64869899771737], [-73.89099834122673, 40.64869453229449], [-73.89099527837928, 40.64869091329086], [-73.89095656520465, 40.64864515289616], [-73.89091526317843, 40.64859554123491], [-73.8909130586134, 40.6485928488051], [-73.89087444305495, 40.64854570385239], [-73.89085215944324, 40.64852483882152], [-73.89084612619742, 40.648519190906214], [-73.89081721708632, 40.648493050141354], [-73.89078772428525, 40.64846729027708], [-73.89078630206957, 40.64846609059574], [-73.89075765503694, 40.648441921371536], [-73.89072701791493, 40.648416945945236], [-73.8907149265673, 40.64840742219289], [-73.89069582367858, 40.64839237523247], [-73.89066408045666, 40.64836821510403], [-73.8906317979142, 40.64834447327475], [-73.89059898527796, 40.648321156454294], [-73.89056565441582, 40.648298270517], [-73.89054523278674, 40.64828093534612], [-73.89052543387196, 40.64826318244592], [-73.89050627633812, 40.64824502942478], [-73.89048777028857, 40.648226485004336], [-73.89046993285152, 40.64820756578645], [-73.89045277522081, 40.648188286022055], [-73.89043630991277, 40.648168658287844], [-73.89042055075868, 40.64814869801016], [-73.89040550565592, 40.648128417258675], [-73.89039118974873, 40.64810783581657], [-73.89034023735852, 40.648040539493664], [-73.8903212633527, 40.648016379884695], [-73.8903015330314, 40.64799257588406], [-73.89028105671677, 40.647969137217814], [-73.89025984603798, 40.64794608115222], [-73.89023791549239, 40.64792341959505], [-73.89021527583874, 40.64790116411588], [-73.89019194002863, 40.647879330641544], [-73.89016792256294, 40.64785792856739], [-73.89012821103123, 40.64782523431973], [-73.89012215504692, 40.647820247243764], [-73.89011209963773, 40.647811717810754], [-73.89009165255432, 40.64779437482743], [-73.89007712569526, 40.64778205229508], [-73.89003284746892, 40.647743353451155], [-73.88999979852272, 40.64771359091057], [-73.88999517921636, 40.64770942977685], [-73.88998932760883, 40.6477041607715], [-73.88997515369725, 40.6476937708365], [-73.8899614891695, 40.647682992721336], [-73.88994835071871, 40.647671840681795], [-73.88993575921668, 40.64766032864306], [-73.88992372739101, 40.64764847437541], [-73.88991227369016, 40.64763629414708], [-73.88990141283169, 40.64762379936412], [-73.88989115995334, 40.647611013662654], [-73.88988152889169, 40.64759794861563], [-73.88987253171003, 40.647584625008356], [-73.88986418069726, 40.64757105943797], [-73.88985648814152, 40.64755726933937], [-73.88983615944981, 40.64753358329439], [-73.88981510829579, 40.64751026645636], [-73.8897933478588, 40.6474873293918], [-73.8897708897705, 40.647464788026625], [-73.88975468482053, 40.64744928583018], [-73.889747746555, 40.64744265007878], [-73.88972392940431, 40.64742093130631], [-73.88969945435694, 40.647399642278245], [-73.88967433195238, 40.64737879406147], [-73.88964452230006, 40.647348223687004], [-73.88961398550197, 40.64731806975257], [-73.88958273010996, 40.64728834834853], [-73.88955076776878, 40.647259067024244], [-73.88951810770412, 40.647230233494334], [-73.88948476309218, 40.647201860000415], [-73.88945074227674, 40.647173955596465], [-73.88943238166773, 40.64715148611932], [-73.88941469484384, 40.64712870434404], [-73.88939768948204, 40.6471056225077], [-73.88938137787974, 40.64708225251715], [-73.88936576749315, 40.64705860711186], [-73.88935086599885, 40.64703469886416], [-73.88947349934483, 40.646956672140014], [-73.88948944517219, 40.64693396725611], [-73.88948152576064, 40.64691275506822], [-73.88941012796785, 40.646797572277706], [-73.88919383109035, 40.64652321265187], [-73.88908093175131, 40.64638868736344], [-73.88879690848673, 40.64605025805637], [-73.88877900211294, 40.64605604593656], [-73.8887610824867, 40.64606991286121], [-73.88899920564424, 40.64637155947531], [-73.8893087980161, 40.64674597772105], [-73.88941394816807, 40.646893000153845], [-73.88942186350398, 40.64691572797252], [-73.88941387285446, 40.64693843999413], [-73.8893760297505, 40.646974754423105], [-73.88931122378477, 40.647014584778205], [-73.88918799274475, 40.64707951539002], [-73.8891871147141, 40.64707965174989], [-73.88913234608853, 40.64708818630504], [-73.88909473960057, 40.64707132449388], [-73.88906042485348, 40.64704387318984], [-73.8890448745862, 40.64704697354717], [-73.88904239022582, 40.6470656653142], [-73.88904890280085, 40.647084989972164], [-73.88898577577875, 40.64716344527782], [-73.88889569015308, 40.64721881890855], [-73.88886541778209, 40.647220035735565], [-73.88884580264049, 40.64720755383576], [-73.88880164237945, 40.64719691731742], [-73.88878146049778, 40.64719443798171], [-73.88878119331632, 40.64719440589297], [-73.88876076273776, 40.64718067644327], [-73.8887338175948, 40.64714886905943], [-73.88872914685678, 40.647138426680804], [-73.88872322160665, 40.64712517949972], [-73.88871260669775, 40.64711270628508], [-73.88869054474083, 40.64709585938264], [-73.88866601496176, 40.6470877358058], [-73.88864137670687, 40.647079553157944], [-73.88853549296564, 40.646968149350386], [-73.8885490830469, 40.64696755239828], [-73.88855801324306, 40.646965860973445], [-73.88856693949997, 40.64696586962281], [-73.88857944085456, 40.64696384123874], [-73.88859507057262, 40.646958077486524], [-73.88861562143931, 40.64694619935449], [-73.88864048640656, 40.64691067406533], [-73.88865398562774, 40.64688506181712], [-73.88865453631006, 40.64685992504181], [-73.88865520043964, 40.64682954255804], [-73.88864625621946, 40.6467998495324], [-73.88864181592984, 40.646785113451344], [-73.88861498719626, 40.64673469206464], [-73.88859943567105, 40.646712625399175], [-73.88859932552029, 40.646712468820894], [-73.88856459699522, 40.64669108132899], [-73.88849960118229, 40.64666368405489], [-73.88843010109082, 40.646647388505315], [-73.88835945887128, 40.646643048980756], [-73.8883550364737, 40.646643953194506], [-73.88828878739722, 40.6466575013859], [-73.88822784503809, 40.64665321797129], [-73.88818911398977, 40.64664978240827], [-73.88816089191931, 40.6466466088439], [-73.88813886515429, 40.64664448906955], [-73.88810266265712, 40.64664023644676], [-73.88808138892033, 40.646620569814324], [-73.88806574107578, 40.64658895091362], [-73.88806004381225, 40.64655386984077], [-73.88806001214776, 40.646553665216935], [-73.88805686200648, 40.6465342762387], [-73.88805695783927, 40.64653402012595], [-73.88806361998368, 40.6465163451405], [-73.88806627051336, 40.64648805482483], [-73.88806734519848, 40.6463003913232], [-73.88807349504613, 40.646282684692224], [-73.88809065494014, 40.646267785662786], [-73.8880882339975, 40.646251934947095], [-73.88809441935717, 40.6462127865493], [-73.88813708213601, 40.646176461311576], [-73.88814269992719, 40.646169634462595], [-73.88816403043424, 40.64615598790062], [-73.88821007899956, 40.646113324808276], [-73.88821447213182, 40.64611135827374], [-73.88827292415569, 40.646085198546324], [-73.88828977845561, 40.6460655693589], [-73.88828978925125, 40.646059121366456], [-73.8882898142075, 40.64604421554559], [-73.8882894131325, 40.64604441836802], [-73.88827491322176, 40.646051761294835], [-73.88826961075077, 40.64605444547779], [-73.88826877077499, 40.64605511108998], [-73.88826293740479, 40.646059739762364], [-73.88823030919363, 40.646031085985754], [-73.88822400304596, 40.646025545888975], [-73.88822322295583, 40.646024862117216], [-73.88825311510513, 40.64601528273937], [-73.88825622169888, 40.646014287790784], [-73.88825611006853, 40.646014227874694], [-73.8882483105607, 40.646009971439106], [-73.88824053610163, 40.646005730272364], [-73.88822564997575, 40.64600440757536], [-73.88821138355613, 40.6460031399256], [-73.88819178134528, 40.646006853236365], [-73.88817997592882, 40.64600908865002], [-73.88812836780345, 40.64602270476086], [-73.88810538611928, 40.646028151034415], [-73.88807452031926, 40.646035464441496], [-73.88803619812846, 40.64603969911446], [-73.88802853501163, 40.64604054470852], [-73.8880055972574, 40.64603003123822], [-73.88800052027007, 40.646027705354655], [-73.8879817286607, 40.64600448368981], [-73.8879624724094, 40.64598068862193], [-73.88792888504189, 40.64594819926588], [-73.88792863149855, 40.64594803869414], [-73.88787175450808, 40.64591226869686], [-73.88783954471293, 40.645873239341114], [-73.88783930019137, 40.64587294542517], [-73.88783648597834, 40.645868922667376], [-73.88780461952585, 40.645823369910225], [-73.88777108781663, 40.64578495072397], [-73.88776432752009, 40.64577720598652], [-73.88774083832425, 40.64574130817862], [-73.88774534679695, 40.645727646276875], [-73.88774684228964, 40.64572339720428], [-73.88774985339084, 40.64571483860078], [-73.88773925530427, 40.645714047070626], [-73.88771509535594, 40.64571224098125], [-73.88767480358253, 40.64566693024832], [-73.88762889453581, 40.64562503343595], [-73.88760719154664, 40.64559243598081], [-73.88759645566897, 40.645576315279996], [-73.88758479056699, 40.64556083480118], [-73.8875847646705, 40.64556080127017], [-73.88757519707431, 40.64554810716395], [-73.88755462162912, 40.64551611691883], [-73.88754276182515, 40.64549767953528], [-73.88751249922825, 40.64546745510859], [-73.88749910233261, 40.64545407373407], [-73.88749551943745, 40.64545006730356], [-73.88746776206911, 40.64541902291841], [-73.88746169123525, 40.64541317659998], [-73.88744201213828, 40.64539422712546], [-73.88742568126139, 40.645375243566384], [-73.88741627153311, 40.645364306807366], [-73.88741147970725, 40.6453391428567], [-73.88740960156503, 40.64532928012421], [-73.88743649431201, 40.64531702460201], [-73.8874544917705, 40.645308823771735], [-73.88748017095864, 40.64530629844598], [-73.88748028754614, 40.64530628700059], [-73.88748504890106, 40.64530871093783], [-73.8874937323772, 40.64531313364104], [-73.88749425824821, 40.64531326750825], [-73.88750718407012, 40.645316564045956], [-73.88750167617394, 40.64527921806782], [-73.88750164051392, 40.64527897478158], [-73.88751736108665, 40.64526532335296], [-73.88751760842835, 40.64526524669931], [-73.88754765490864, 40.645255956976804], [-73.88760152266909, 40.64523123904407], [-73.88764385071785, 40.64521114683328], [-73.88766437100682, 40.64520140482757], [-73.8876713055163, 40.64519799100466], [-73.88772497502667, 40.64517156888821], [-73.88775225382686, 40.64516012115623], [-73.88776761931854, 40.64515367273634], [-73.88780902508144, 40.64513553060128], [-73.88782597401105, 40.64512810434692], [-73.88784752127042, 40.64511658683801], [-73.88788658219785, 40.645095705810505], [-73.88790367351744, 40.645079836750575], [-73.88791690723532, 40.645067549205415], [-73.88792588075614, 40.64506584899275], [-73.88793370047152, 40.645083793217644], [-73.88795587366926, 40.645114042656836], [-73.88796502879283, 40.645126532113], [-73.88796518926894, 40.645126579512315], [-73.88798856532566, 40.64513338902931], [-73.88800647124246, 40.64515475981581], [-73.88802537505066, 40.64515816799487], [-73.88802553006325, 40.64515819461526], [-73.88802684907814, 40.64515856580252], [-73.88806812125227, 40.645170195600556], [-73.88807196882098, 40.645168524395565], [-73.88807596671336, 40.645166790346074], [-73.88807830661167, 40.64516602735333], [-73.88808383164817, 40.64516423163145], [-73.88808383875663, 40.64515999367044], [-73.88808385457098, 40.64515056535043], [-73.88805587182881, 40.64511978725729], [-73.88804571571661, 40.645110329266984], [-73.88800996589524, 40.645077034694374], [-73.88796406442864, 40.64503257382938], [-73.88794876930156, 40.645021248242095], [-73.88792710474918, 40.645005204707154], [-73.88791252941961, 40.64500348154403], [-73.88788110148197, 40.6450213886855], [-73.88788096371262, 40.64502144065254], [-73.88786807651202, 40.64502633365665], [-73.88784519285471, 40.64503501997499], [-73.88778239005063, 40.64503922968088], [-73.88773528615363, 40.645042600283524], [-73.88767244806121, 40.64506645646278], [-73.88761184035705, 40.64509800015654], [-73.88754899585209, 40.64512612606049], [-73.88749665617937, 40.645152870548], [-73.88748726875768, 40.64515767244712], [-73.88742554053763, 40.64518750498301], [-73.88742502464652, 40.645187844225774], [-73.88736940895883, 40.645224179134345], [-73.88734361665684, 40.645224153871006], [-73.88731487147737, 40.64520457345941], [-73.88730217444433, 40.645195925929514], [-73.88729121814026, 40.64518616402934], [-73.88726858280884, 40.645165996208135], [-73.8872529307313, 40.64513779469932], [-73.88724513428929, 40.64510618297466], [-73.88720940903949, 40.645084221478875], [-73.88719920587415, 40.64507795176963], [-73.88719205915753, 40.64506663842955], [-73.88718354961118, 40.645053165817714], [-73.88718697934982, 40.64501387755388], [-73.88718097101543, 40.64499855034211], [-73.88717357973094, 40.644979697308905], [-73.88716449216427, 40.64497639025955], [-73.88715004600363, 40.64497113294555], [-73.88711196189598, 40.64494547024285], [-73.88709182294312, 40.644918971087044], [-73.8870911770819, 40.6448870693432], [-73.88709078754081, 40.64486772092812], [-73.88707517897967, 40.644813892939005], [-73.88706756030193, 40.644787129314935], [-73.88706180246118, 40.64476690230169], [-73.8870344533698, 40.64474402673167], [-73.88701477169916, 40.64472756447533], [-73.88698457931622, 40.64467713938985], [-73.88695887549864, 40.644625010420356], [-73.88694102650793, 40.64457032547864], [-73.8869365699227, 40.64455409395216], [-73.88692500573482, 40.6445374036129], [-73.88691643330115, 40.644525032083266], [-73.88691203454515, 40.644473777607885], [-73.88688964143898, 40.64445325696575], [-73.88687171566681, 40.64444469705514], [-73.88680331813799, 40.644440358276775], [-73.88678424430306, 40.64444631876019], [-73.88678415535188, 40.64444650529931], [-73.88677895080598, 40.644457363560946], [-73.8867752427269, 40.644465102407096], [-73.88678391809634, 40.644474210426], [-73.88678419647526, 40.644474506052795], [-73.88679424990453, 40.64449843349059], [-73.88679645571422, 40.64451978951413], [-73.88678605503482, 40.64452215466205], [-73.88678523743152, 40.644522340483874], [-73.8867482931071, 40.644487282777895], [-73.88671024836563, 40.644438558843824], [-73.88668366556917, 40.644421641257594], [-73.88668336392223, 40.64442144880515], [-73.88667745680381, 40.64441564817367], [-73.88665201863772, 40.64439066805883], [-73.88663887364095, 40.64438239276388], [-73.88663077735512, 40.644352210219644], [-73.8866304837735, 40.64435206066235], [-73.8866173330251, 40.64434536414791], [-73.88658356007102, 40.64433144708819], [-73.88658091508064, 40.64432978226192], [-73.88657950376457, 40.64433067681233], [-73.88654888955993, 40.6443124903785], [-73.88655111189425, 40.644311013126675], [-73.88657616109464, 40.64429436104308], [-73.88654221594709, 40.64423207966326], [-73.88643332005316, 40.64419306703229], [-73.8864325000971, 40.64419244033611], [-73.88642580656825, 40.64418687445985], [-73.88641689065331, 40.644179458548955], [-73.88640742962818, 40.644170427626754], [-73.8863721085287, 40.64413670710629], [-73.88636185865222, 40.64411318468771], [-73.88635758725518, 40.6441033803843], [-73.88634638759265, 40.64407385514963], [-73.88633749308138, 40.64405040258741], [-73.88630504007673, 40.644005949988816], [-73.88628516767372, 40.64399336251911], [-73.88622602727345, 40.64394905538979], [-73.88623633525125, 40.643894599053915], [-73.88624664233414, 40.643840141710726], [-73.88623646791807, 40.64381678772094], [-73.88616522744307, 40.64380167199119], [-73.88613720679389, 40.64379395720526], [-73.88612910452176, 40.643786530517694], [-73.8861204115461, 40.643778564808024], [-73.88609803773574, 40.64374779279166], [-73.88607233562739, 40.64369566412182], [-73.8860721076197, 40.64369549754005], [-73.88602529679075, 40.6436614504065], [-73.8859908619233, 40.64366101336716], [-73.88595241003655, 40.6436605250908], [-73.88591017758976, 40.6436604832001], [-73.88587952270532, 40.64366045278355], [-73.88587918263688, 40.64366035310168], [-73.88586685006611, 40.64365672542978], [-73.88585038128603, 40.6436518820848], [-73.8858279911263, 40.64363050550078], [-73.88580110949894, 40.64361168680073], [-73.8857885828904, 40.643604758616], [-73.885768621624, 40.64359371675363], [-73.8857275231896, 40.64358289445717], [-73.8857002497673, 40.64357571220986], [-73.8856886383046, 40.64356969343094], [-73.88566888045216, 40.64355945220465], [-73.88564648781826, 40.6435389296429], [-73.88562236163982, 40.64352766632895], [-73.88560615138455, 40.64352009818708], [-73.88558493934023, 40.64351307187971], [-73.88553890885684, 40.64349782319592], [-73.88551316184969, 40.64347217290688], [-73.8854849551704, 40.64346302490487], [-73.88547617787049, 40.64346017806352], [-73.88543581637987, 40.64345586806235], [-73.88537977613835, 40.643439581722255], [-73.8853521641245, 40.64341933719122], [-73.88535177774826, 40.64341905351529], [-73.88532376006884, 40.64340963074776], [-73.88525314667166, 40.64339076796654], [-73.88519039515114, 40.64336508101046], [-73.88512764875115, 40.64333683100629], [-73.88509819735403, 40.64332027740767], [-73.88506827422987, 40.64330345930517], [-73.88502120740556, 40.64328632920507], [-73.8849550854284, 40.643264054489244], [-73.88493829484842, 40.643246955812266], [-73.88492244965363, 40.64322597141716], [-73.88492151268089, 40.64322472926059], [-73.88491396903088, 40.643220741233776], [-73.88491387257027, 40.643220689035786], [-73.88486325515501, 40.64319392276234], [-73.88483222472678, 40.64318019573087], [-73.88479938783794, 40.643165671455975], [-73.88475150350376, 40.643149539566245], [-73.884733266714, 40.64314339661295], [-73.88469181436612, 40.64312114697229], [-73.88468354514835, 40.64311628805958], [-73.88463356426678, 40.64308692194502], [-73.88461734294579, 40.64307417869585], [-73.88438270996289, 40.6429106699506], [-73.88373578781892, 40.642450349094226], [-73.88311261912273, 40.64187446548803], [-73.88251634823919, 40.641350710775676], [-73.8819016777496, 40.640834939061385], [-73.88126533390094, 40.64021877263545], [-73.88078889712776, 40.639757433320376], [-73.88076685499172, 40.63958164757543], [-73.88055123891112, 40.63915881986856], [-73.8797498301519, 40.638422235815135], [-73.8795388800853, 40.63821783916026], [-73.87945522184175, 40.63819224570853], [-73.87935480009125, 40.638179387910526], [-73.87911787849026, 40.638266587220265], [-73.87907715621357, 40.63830298104855], [-73.87905060875187, 40.638435948562076], [-73.87896686085557, 40.63845954384779], [-73.87887362624322, 40.6384394051234], [-73.87876518091636, 40.63836855709631], [-73.87870039087672, 40.63833772487285], [-73.87863863904012, 40.63829360613005], [-73.87859391192085, 40.638217012182864], [-73.87843722479572, 40.637948690637224], [-73.8786115360817, 40.63764882997628], [-73.87863488118161, 40.63755178164712], [-73.87846702122842, 40.637489830663995], [-73.87783675028, 40.63690231029442], [-73.87714434884276, 40.63597649800373], [-73.877493816927, 40.63539042397704], [-73.87793889634258, 40.63501957113924], [-73.8781371779872, 40.63469473189087], [-73.87820590489773, 40.63457294366254], [-73.87833349586315, 40.63448041794522], [-73.87856542917646, 40.63433946507626], [-73.8787277618307, 40.63425138864696], [-73.87890455278942, 40.63411588241607], [-73.87918335624656, 40.63401482552619], [-73.87936895387004, 40.633908015784016], [-73.87971206395956, 40.63369008532766], [-73.87998767503844, 40.63351488348096], [-73.88018468342864, 40.63330963859056], [-73.8802328219404, 40.63322689052148], [-73.88037629489094, 40.63298026236323], [-73.88093320122287, 40.63258706179293], [-73.88127068104548, 40.63236911916103], [-73.88144495909098, 40.63230509611649], [-73.88162197738069, 40.63229710561399], [-73.88163037328907, 40.63229672726628], [-73.88178214769638, 40.63224980097282], [-73.88197891997052, 40.63217295899949], [-73.88208583089636, 40.63207890480774], [-73.88215899502384, 40.63200621538491], [-73.88256945182803, 40.63181830588501], [-73.88272684944738, 40.631767104900234], [-73.88301362103512, 40.6316261495354], [-73.88322181279625, 40.63144231258993], [-73.88345265635674, 40.63115577211293], [-73.88364451728226, 40.630943312138584], [-73.88375501660256, 40.63055643801253], [-73.88376932796366, 40.63022176134234], [-73.88345412923461, 40.62990766993492], [-73.88404726149369, 40.6295601185465], [-73.88263248433404, 40.62827427312689], [-73.88300026358979, 40.62802619438618], [-73.88369231571497, 40.62755937391403], [-73.88512954023768, 40.62882725867137], [-73.88572166971262, 40.62844671572562], [-73.88604117197292, 40.628697246481536], [-73.88621452593078, 40.62868061400437], [-73.88638667007316, 40.62865782666461], [-73.8865572281382, 40.628628934269535], [-73.88672582733146, 40.62859399997052], [-73.8868920991431, 40.62855310012493], [-73.88705568015072, 40.628506324128594], [-73.88721621281638, 40.62845377422153], [-73.887373346267, 40.62839556526336], [-73.88752673706196, 40.628331824482444], [-73.88766290771191, 40.62826907222885], [-73.88779544838252, 40.62820196007808], [-73.88792411875025, 40.628130609723776], [-73.8880486855124, 40.6280551505441], [-73.88816892280943, 40.62797571936736], [-73.8882846126343, 40.62789246022348], [-73.89042299743556, 40.62627859879778], [-73.89048710394742, 40.62622477905808], [-73.89054423031342, 40.626166574144925], [-73.89059386270354, 40.62610450760886], [-73.89063555469917, 40.62603913773376], [-73.89066893130891, 40.62597105251558], [-73.89069369233994, 40.62590086437204], [-73.8907096150978, 40.62582920463516], [-73.89071655638924, 40.62575671787127], [-73.89071445380763, 40.62568405608367], [-73.89070332629487, 40.62561187284788], [-73.8906832739687, 40.62554081743287], [-73.89069934320733, 40.6254490977382], [-73.89072395717571, 40.62535848486902], [-73.89075699047119, 40.62526944034894], [-73.89079827481083, 40.625182417711706], [-73.89084759988879, 40.62509786019118], [-73.89090471444746, 40.62501619846384], [-73.89096932755896, 40.624937848455794], [-73.89104111010586, 40.6248632092237], [-73.89111969645955, 40.62479266092348], [-73.89120468634033, 40.62472656287357], [-73.89129564685899, 40.62466525172516], [-73.89139211472015, 40.62460903974802], [-73.89149359858217, 40.6245582132401], [-73.89160061453713, 40.624429266771706], [-73.89171432619564, 40.62430369149586], [-73.89183455084178, 40.624181689150284], [-73.89196109529959, 40.62406345573154], [-73.8920937562442, 40.62394918118082], [-73.89223232052697, 40.62383904907804], [-73.89237656551936, 40.62373323634773], [-73.8925262594703, 40.623631912974545], [-73.89268116187884, 40.62353524173015], [-73.89284102387963, 40.623443377912686], [-73.8930055886431, 40.623356469095974], [-73.89308340975172, 40.62336167592056], [-73.89313041733622, 40.623409553641906], [-73.89320886543258, 40.62342557109937], [-73.89329777596994, 40.62344159921929], [-73.89351237165282, 40.623378019117084], [-73.89359082527815, 40.62339005080736], [-73.89456448045492, 40.62302023013246], [-73.89480003637225, 40.622932749253586], [-73.89492365447543, 40.62288643173478], [-73.89504313667864, 40.62283419154178], [-73.89515799486826, 40.62277624209296], [-73.89526775982362, 40.62271282013041], [-73.89537198313622, 40.62264418475256], [-73.89547023904039, 40.62257061635524], [-73.8956010701693, 40.62254283280268], [-73.89577374598754, 40.62251907138477], [-73.89594111031164, 40.62254712710863], [-73.89599336881395, 40.62258304936637], [-73.8960194773396, 40.622614963109555], [-73.89597194082525, 40.62265438253494], [-73.89596185641412, 40.62266274499435], [-73.89596702487455, 40.622702612328396], [-73.89602450367005, 40.6227465118264], [-73.89607154396428, 40.622774457969435], [-73.89612385854737, 40.62277450525077], [-73.89619315402952, 40.62277172420758], [-73.89626176678121, 40.62276896998145], [-73.89632454431379, 40.62279224427039], [-73.8963900618076, 40.6228173934646], [-73.89826103011846, 40.622989159921765], [-73.90114685492415, 40.625685387598025], [-73.90192494672036, 40.62584743465486], [-73.90279518238806, 40.62602866456063], [-73.90370369782693, 40.62621785795743], [-73.90412793207628, 40.626306200134245], [-73.9044869797309, 40.626475292106406], [-73.90454477156379, 40.62650251846565], [-73.90460520923169, 40.62653098000287], [-73.90483487851195, 40.62663913392717], [-73.90489270645807, 40.62666636784695], [-73.9049063484711, 40.626672790960335], [-73.9049352622391, 40.62668641368551], [-73.90501149169361, 40.62672230556913], [-73.90504862589897, 40.62673978989416], [-73.90505511040648, 40.62674283907976], [-73.90506354457798, 40.626746817816894], [-73.90529160821045, 40.62685422131578], [-73.90579604091182, 40.62709176652205], [-73.90594908886753, 40.62716383772389], [-73.90663673204166, 40.627487654991654], [-73.90746333160071, 40.62787689951586], [-73.90825201141737, 40.628248276562815], [-73.90826288994353, 40.628253400398386], [-73.9082831586814, 40.62826295364552], [-73.90908063035498, 40.628638461247135], [-73.9099107442635, 40.62902932898019], [-73.91071335177085, 40.629407235283516], [-73.91122829462807, 40.62964968889373], [-73.9115321792163, 40.62979195228684], [-73.91236461548841, 40.63018164237542], [-73.91316650880005, 40.630557023372226], [-73.9139951001166, 40.6309448920763], [-73.91633195427576, 40.63203873255834], [-73.91642761952173, 40.63203648851158], [-73.91652258371712, 40.63202739990856], [-73.9166160050226, 40.63201154731792], [-73.91670705527757, 40.63198907126988], [-73.91679492734241, 40.63196017101], [-73.91687884225443, 40.6319251027331], [-73.91695805613352, 40.63188417731136], [-73.91703186677645, 40.63183775753898], [-73.91709961988298, 40.63178625491546], [-73.91716071492723, 40.63173012599842], [-73.91860649599992, 40.632090414999894], [-73.9186298239999, 40.63234687299989], [-73.91876282599999, 40.6336375289999], [-73.91885164499996, 40.634482718999955], [-73.91886199299982, 40.634577127999876], [-73.91886934499978, 40.63464421599988], [-73.91888284299993, 40.634767351999905], [-73.91889071099986, 40.6348391329999], [-73.91891212099982, 40.63503445999987], [-73.91892161299991, 40.635121060999914], [-73.91893983700002, 40.63528732099991], [-73.91895156599989, 40.63539431799991], [-73.91899154599992, 40.63575906299996], [-73.919006295, 40.63589361799995], [-73.9190277079998, 40.63609754399988], [-73.91905790399989, 40.63638510999988], [-73.91916482699992, 40.63740334199989], [-73.91917180399989, 40.63746984099993], [-73.91928984599986, 40.638596032999914], [-73.91941447399985, 40.63977466499991], [-73.91945472999996, 40.6401731369999], [-73.91946141199999, 40.64023929299983], [-73.91946588499995, 40.64028418699997], [-73.91953493999996, 40.640857740999884], [-73.919555766, 40.641184957999876], [-73.91958298599987, 40.64139576199993], [-73.91965951399997, 40.642078236999915], [-73.91973736999988, 40.64277814799987], [-73.91990782599999, 40.64432958099992], [-73.91995062999999, 40.64470762999989], [-73.91982077699998, 40.644783831999874], [-73.91943014999988, 40.64506079199986], [-73.91925419699997, 40.64517776399986], [-73.91892329099996, 40.64539775299986], [-73.91861676799982, 40.64560209799988], [-73.91830152299998, 40.645812262999925], [-73.91798513199991, 40.646010879999906], [-73.91764732399982, 40.646222935999916], [-73.91730801300002, 40.64644290499994], [-73.91693683599993, 40.64668353499992], [-73.91620934099993, 40.64715114199988], [-73.91550029699984, 40.647604853999916], [-73.91473216799982, 40.64810117499995], [-73.91396129299983, 40.64860065799996], [-73.91325294199991, 40.649061424999935], [-73.91253589199988, 40.64952147899985], [-73.91182280200003, 40.64998342299986], [-73.91110070199989, 40.65044791299992], [-73.91022992899991, 40.65101403099988], [-73.90944846399994, 40.65152159899992], [-73.90868769500001, 40.65072148099995], [-73.90849084299998, 40.650586417999875], [-73.90827570700003, 40.65049857299989], [-73.90761503399983, 40.649895893999876], [-73.907546414, 40.64983425299993], [-73.90637218799996, 40.6487793819999], [-73.90588893999983, 40.64834514399994], [-73.90579983699993, 40.64826507899992], [-73.90509580299975, 40.64760685099991], [-73.90398200599991, 40.64662856799988], [-73.903099438, 40.64719961099992], [-73.90235538599994, 40.64768960199993], [-73.90165833699982, 40.64814221199989], [-73.90089120800002, 40.648641709999886], [-73.90074707899997, 40.6487354449999], [-73.90056999899996, 40.64884760699989], [-73.90027087999994, 40.649037770999946], [-73.90019369799995, 40.64908881799994], [-73.89946894299986, 40.649541768999924], [-73.89872808799994, 40.650024606999885], [-73.89797649399998, 40.65049403099989], [-73.89779457099989, 40.65061310899988], [-73.89771516099994, 40.65066508899988], [-73.89757174800003, 40.65075888699994], [-73.89742477799994, 40.650855011999944], [-73.89736587999998, 40.65089370699995], [-73.89727504499993, 40.650953162999876], [-73.89721054799986, 40.65099537899994], [-73.89680700299976, 40.65126016999992], [-73.89675272299999, 40.6512957869999], [-73.89661390699996, 40.65138687099991], [-73.89655685900003, 40.65142430299988], [-73.89627957599997, 40.65160624199991], [-73.89621590699987, 40.65164801899989], [-73.89586154900002, 40.65188052699987], [-73.89610981499999, 40.65210391199988], [-73.89619634399993, 40.652181767999906], [-73.89654981299998, 40.652499805999916], [-73.896634179, 40.65257571499996], [-73.89676845099996, 40.65269652599995], [-73.89684143499989, 40.65276219199989], [-73.89746257699998, 40.65332105699985], [-73.89761013299987, 40.65345381599992], [-73.89741344699993, 40.653538320999914], [-73.89678213299993, 40.65380955599988], [-73.89667745499989, 40.65385495699997], [-73.8959444279999, 40.654172887999934], [-73.89558884499993, 40.65432447199986], [-73.89546427299999, 40.65437757699991], [-73.89520540399988, 40.654487929999895], [-73.89441113999996, 40.6548329489999], [-73.89354677099988, 40.65520271399988], [-73.89245737399976, 40.653710938999886], [-73.89136543599977, 40.65236471899988], [-73.8900378249999, 40.6511781769999], [-73.89000589199988, 40.651148696999925], [-73.89004263999982, 40.65111858599992], [-73.89125147199995, 40.65002559699993], [-73.89121853499988, 40.64998769899987]]], [[[-73.88833929256269, 40.64671022891759], [-73.88835262237468, 40.64670795760858], [-73.88837395798683, 40.64671039927301], [-73.88840531896192, 40.6467139885153], [-73.88845015102363, 40.646728552537816], [-73.88849720193863, 40.64675678600008], [-73.888536179038, 40.64678824826763], [-73.88853640000424, 40.64678842689961], [-73.88853772941454, 40.646802871820846], [-73.88853970368733, 40.646824304978544], [-73.88853297695874, 40.646861519426885], [-73.88853291212868, 40.646861882230674], [-73.88850950500931, 40.64688116417922], [-73.88846787961647, 40.646899734025276], [-73.88829183121516, 40.64671871903564], [-73.88830775448939, 40.646715602760004], [-73.88833929256269, 40.64671022891759]]]]}}, {\"id\": \"39\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 40, \"Shape_Leng\": 0.0518506648762, \"Shape_Area\": 0.000108949211644, \"zone\": \"Carroll Gardens\", \"LocationID\": 40, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.998192111, 40.68228732999989], [-73.99620846299992, 40.681723424999916], [-73.99588110299992, 40.68239495399995], [-73.99555742799987, 40.683068534999954], [-73.99523240699986, 40.68374079099993], [-73.99490113599977, 40.6844302379999], [-73.99262292599985, 40.68354371199993], [-73.99042374299992, 40.68269393299993], [-73.98827189199999, 40.68186309999987], [-73.98722413899985, 40.68144730199988], [-73.9876680399999, 40.68081780999993], [-73.98808029299991, 40.68017216599984], [-73.98850509899982, 40.6795371569999], [-73.98897369299988, 40.67884987699992], [-73.98931813699981, 40.67834938899991], [-73.98931805499996, 40.67825393999993], [-73.98940896399986, 40.67745451699993], [-73.98986566199996, 40.676826601999934], [-73.99036651399997, 40.67621832099995], [-73.99069861899994, 40.67580741899994], [-73.99256575599999, 40.6761058199999], [-73.99279927099997, 40.67613190599994], [-73.99280910199987, 40.67614874999984], [-73.9931237449998, 40.67617871399997], [-73.99317836299984, 40.676161657999906], [-73.9933342719999, 40.676176012999846], [-73.993491024, 40.67618330199986], [-73.99364806799993, 40.67618349899993], [-73.99380485099994, 40.67617660299988], [-73.99396082100002, 40.67616263899984], [-73.9940579759999, 40.676182783999884], [-73.99420802699994, 40.67615242699986], [-73.99435579199994, 40.67611614799993], [-73.99450491199987, 40.67607277499994], [-73.99465075899997, 40.67602336999993], [-73.99479291199987, 40.67596807599986], [-73.99493096199983, 40.67590705299988], [-73.99506451099984, 40.67584047499984], [-73.99520266299999, 40.67572952499992], [-73.99540989400003, 40.67556309699996], [-73.99572187999998, 40.675222292999905], [-73.9959621319999, 40.67496822399994], [-73.99666043499988, 40.67414278099988], [-73.99696673299994, 40.673784159999876], [-73.99717696100001, 40.67352829399992], [-73.99745115699996, 40.67319456799988], [-73.99805741699998, 40.67246954499988], [-73.99812844699997, 40.672440749999964], [-73.99847134099994, 40.67205910099987], [-73.99855090599986, 40.67192594199986], [-73.9986188799999, 40.67183017199989], [-73.99898242799983, 40.67241358399986], [-73.99903998999996, 40.67250767899989], [-73.99934534299987, 40.67300654899992], [-73.99983349199978, 40.673756049999945], [-73.99991051299993, 40.673872729999836], [-73.99994578200001, 40.67392615699986], [-74.00045011499991, 40.674769424999916], [-74.00077979399994, 40.67529390099993], [-74.00105973699986, 40.675555677999945], [-74.00112481699996, 40.67569755399992], [-74.00090355299992, 40.67563696899988], [-74.00083254999997, 40.67561225999985], [-74.0006740919999, 40.67555578899988], [-74.00102822999993, 40.67641403999991], [-74.00153194399996, 40.67727039699989], [-74.00215698499991, 40.67818268099997], [-74.00221991299993, 40.678260321999836], [-74.00255365299991, 40.678658684999945], [-74.00310560799991, 40.67933188599993], [-74.00329231299999, 40.67964328799992], [-74.00289363599995, 40.680559409999894], [-74.00256464499995, 40.681228463999936], [-74.00225221999996, 40.68189321199991], [-74.002204589, 40.68199132399995], [-74.00193169899997, 40.68256321799992], [-74.00160876999993, 40.683235204999924], [-74.00128182399995, 40.6839096399999], [-74.00095809999992, 40.684579318999916], [-74.00064737999996, 40.685253946999914], [-73.99986579399993, 40.685036468999954], [-73.99908066099994, 40.68481599199985], [-73.99940054699985, 40.684144561999844], [-73.9997255609999, 40.68347314899994], [-74.00004865199998, 40.682803355999965], [-73.998192111, 40.68228732999989]]]}}, {\"id\": \"40\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 41, \"Shape_Leng\": 0.052793109453, \"Shape_Area\": 0.000143093037737, \"zone\": \"Central Harlem\", \"LocationID\": 41, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94773985499985, 40.809599720999884], [-73.94506102999992, 40.80847449999987], [-73.94499360699993, 40.80844618399995], [-73.94491601899982, 40.80841350399989], [-73.94177140199992, 40.807088999999955], [-73.9422631119999, 40.80641573799991], [-73.94275367199994, 40.80574401299988], [-73.94430194599991, 40.806390824999916], [-73.94478208199993, 40.805761255999904], [-73.94523965199991, 40.80513601399989], [-73.9456926129999, 40.804514948999895], [-73.94613131, 40.80387735799992], [-73.94459751299998, 40.803228144999906], [-73.94475976499989, 40.803003040999975], [-73.94505128000004, 40.80259859899995], [-73.94550863900001, 40.8019702309999], [-73.9459678969998, 40.801342394999935], [-73.94646133899998, 40.800665304999875], [-73.94697366799987, 40.79996170299987], [-73.94728018299999, 40.79954178899988], [-73.94732771499984, 40.79947667099991], [-73.94737599799991, 40.799410523999896], [-73.94741996199991, 40.79935029499992], [-73.94771056, 40.79895218099991], [-73.94775716499986, 40.798888331999855], [-73.94781663099997, 40.79880686299992], [-73.94784679099992, 40.798765542999945], [-73.94805130199998, 40.79848535999987], [-73.94810108999994, 40.798417147999885], [-73.94815384299984, 40.79834487599987], [-73.9482064149998, 40.79827285099987], [-73.94831125500004, 40.79812921799992], [-73.94879165299983, 40.797477127999905], [-73.94909695599983, 40.79705785599992], [-73.94922045699984, 40.796909858999896], [-73.94960357199982, 40.79705027099992], [-73.95237419099993, 40.79820977099995], [-73.95245200999996, 40.798240230999944], [-73.95253597599994, 40.798274645999896], [-73.95516503099994, 40.79938223299988], [-73.95521824499984, 40.79940465799989], [-73.95528884999997, 40.79943440199992], [-73.95536651699993, 40.79946642599985], [-73.95786499299992, 40.800515271999885], [-73.95817297099987, 40.800582540999876], [-73.95848111500001, 40.800670477999894], [-73.95964685399987, 40.80115642299993], [-73.95914741099993, 40.80184375999991], [-73.95868839399998, 40.802469793999855], [-73.95824878299989, 40.80310648599987], [-73.958203008, 40.803894270999905], [-73.95825482599987, 40.804713877999916], [-73.95818167299996, 40.80559679099989], [-73.95772176, 40.8062909889999], [-73.95726581599985, 40.806916170999976], [-73.95680543899995, 40.80754570899993], [-73.95634493700001, 40.808173353999905], [-73.95588417599987, 40.80880224899986], [-73.95542974899983, 40.80942915299993], [-73.954966572, 40.81006455499988], [-73.9535757779999, 40.8094766979999], [-73.95327654899991, 40.809879603999875], [-73.95307869999975, 40.81009408499988], [-73.9525912579999, 40.810765332999935], [-73.95210125199992, 40.811442853999935], [-73.95067975599989, 40.81084296399989], [-73.94792306799984, 40.80967692199992], [-73.94783875999985, 40.8096412609999], [-73.94773985499985, 40.809599720999884]]]}}, {\"id\": \"41\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 42, \"Shape_Leng\": 0.0927092274616, \"Shape_Area\": 0.000263896952605, \"zone\": \"Central Harlem North\", \"LocationID\": 42, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93436121591056, 40.83620061961779], [-73.93445239398683, 40.83598096007491], [-73.93456051826952, 40.83586608172801], [-73.934754664506, 40.835589300341795], [-73.93475368707831, 40.83556982111418], [-73.93480514039254, 40.835427472645925], [-73.93481898168233, 40.83539825039474], [-73.93483181137556, 40.83538552549233], [-73.93484813571942, 40.835351432207396], [-73.934869850678, 40.835332715131045], [-73.9348946004199, 40.8352540431223], [-73.93491533578629, 40.835230821938744], [-73.93493620828455, 40.83508021765153], [-73.9349422585534, 40.835026875417256], [-73.93495859566912, 40.8348827790447], [-73.9349794120861, 40.83478912145354], [-73.9349883314069, 40.83474865789361], [-73.9349508717845, 40.83470667336768], [-73.93495775984093, 40.83468276036335], [-73.93496523015544, 40.83465684795365], [-73.93496624164149, 40.83462912364824], [-73.93497316061179, 40.83462163504156], [-73.93497219073525, 40.834608144797535], [-73.93499389736655, 40.834597666183], [-73.93503060323975, 40.834392365341266], [-73.93503612427708, 40.83429832034335], [-73.93503124118601, 40.83424886284948], [-73.93504212490862, 40.8342113992345], [-73.93507644502887, 40.83392404328713], [-73.93506958098223, 40.833887317502175], [-73.93508341363356, 40.833865596929606], [-73.93509687471914, 40.83371535679221], [-73.93509690469148, 40.83368463190985], [-73.9351038425559, 40.83365765691791], [-73.9351124510007, 40.83342573907346], [-73.93511743048032, 40.83337703478394], [-73.93512041704221, 40.83335380357006], [-73.9351322584875, 40.833344822904984], [-73.93512636041858, 40.83332608035632], [-73.93512935500891, 40.833294609479935], [-73.93512443552818, 40.83328111868773], [-73.93513826819033, 40.83326014425817], [-73.93513729127831, 40.833241413188574], [-73.93513040300608, 40.83322866727349], [-73.93512154973675, 40.833201692776605], [-73.93512352949845, 40.833189699006276], [-73.93511565892264, 40.83317620588328], [-73.93513243602646, 40.833162725620355], [-73.93513935847737, 40.83315149100027], [-73.93512752379401, 40.8331424888091], [-73.93512465984584, 40.83305106902925], [-73.93512171853354, 40.833028590347574], [-73.93511580946044, 40.833021842172855], [-73.93511187882297, 40.833003100729265], [-73.93511979365512, 40.83298362817418], [-73.93511783020558, 40.83296713902451], [-73.93512376770067, 40.83294541299675], [-73.93512085547357, 40.83290419576851], [-73.93513170518379, 40.832902697673646], [-73.93514455114395, 40.83288472203645], [-73.93516725294582, 40.83287499277842], [-73.93516480840947, 40.83285288326428], [-73.93515659173339, 40.83268240673981], [-73.93512878131628, 40.83236353447691], [-73.93511398341515, 40.83236577683715], [-73.93507789246162, 40.83194162361134], [-73.93509565184875, 40.8319423812671], [-73.93508783728583, 40.83185919835002], [-73.93506824974095, 40.831704067625786], [-73.93502275465511, 40.83131999954982], [-73.9349850562146, 40.83102735335789], [-73.93493515655027, 40.83060244471324], [-73.9348935333237, 40.83028656383779], [-73.93483087208487, 40.82980843705176], [-73.9347988190574, 40.82979230914965], [-73.93480080676927, 40.829772833254935], [-73.934789962788, 40.82976832407678], [-73.93479293827892, 40.82975634075214], [-73.9347811108578, 40.8297405936699], [-73.93477028229687, 40.82972036169422], [-73.93475944294418, 40.82970012100011], [-73.93476343288322, 40.82965740394861], [-73.93474372591271, 40.82963716812018], [-73.93473930830932, 40.82960906144654], [-73.93474919492763, 40.8295970813634], [-73.9347373734992, 40.8295745894252], [-73.93472260127821, 40.82955060049558], [-73.93472557699917, 40.82952736927088], [-73.9347147602574, 40.829495144079715], [-73.93470100252401, 40.82944043080001], [-73.93469412707354, 40.82941645571865], [-73.93471288560669, 40.829400725944666], [-73.93470504902284, 40.82934077469046], [-73.93469321669625, 40.82932877360132], [-73.93467597101821, 40.82931340221053], [-73.93466228987826, 40.82918075994596], [-73.93468302702506, 40.82916578733352], [-73.93470769402485, 40.82915830979219], [-73.93470080780729, 40.82913207413399], [-73.9346810792479, 40.829133558281335], [-73.93467714880197, 40.829116328547876], [-73.93467025222023, 40.82911257042637], [-73.93465447618146, 40.829097576961054], [-73.93464959243231, 40.82904811940697], [-73.9346619319834, 40.829038760294466], [-73.93466295225556, 40.829014033814005], [-73.93466495971363, 40.828973566826434], [-73.93465611022576, 40.82894283638618], [-73.93465515536495, 40.828912858581106], [-73.93463641852115, 40.828906860149054], [-73.93461640343897, 40.82869365579324], [-73.93461171264791, 40.82868983525945], [-73.93460260323826, 40.82868240123771], [-73.93460883576694, 40.82866118868205], [-73.93461052506869, 40.82865543619221], [-73.93460804993313, 40.82865336301813], [-73.93460065845267, 40.82864718271558], [-73.93459873640637, 40.82860072400894], [-73.93460192118351, 40.8285925492986], [-73.9346115955321, 40.8285677551862], [-73.9346091744548, 40.82852167496818], [-73.93461017716503, 40.82850293629024], [-73.93461219000994, 40.8284572291491], [-73.93460433489878, 40.8284280027947], [-73.93460731363069, 40.82840102556364], [-73.93460043185567, 40.828382282414296], [-73.93460834555331, 40.82836355753902], [-73.93459750565471, 40.82834482035352], [-73.93460048488097, 40.82832833398501], [-73.93459360040826, 40.82830135116009], [-73.9345867414778, 40.828259384992435], [-73.93459562914452, 40.82825114985732], [-73.93458676210527, 40.82823840277648], [-73.9345917047489, 40.82822716755991], [-73.93459763971536, 40.82820843167856], [-73.93458877267241, 40.828195693979055], [-73.93458684240736, 40.828156719571446], [-73.93456513010244, 40.82816270444937], [-73.93455810179871, 40.82811740277773], [-73.93454586266024, 40.82804816146415], [-73.9345423063815, 40.828026870313025], [-73.93453258880513, 40.82796855886424], [-73.93452175638055, 40.827907800091346], [-73.9345389813406, 40.82790347267067], [-73.93453183092761, 40.8278494020435], [-73.934520765281, 40.82776568965099], [-73.93451590672413, 40.82772893075848], [-73.93442396339046, 40.8270428939722], [-73.93448291058755, 40.8270370927787], [-73.93446426549498, 40.826842205193536], [-73.93442844400506, 40.82646785059143], [-73.93436717955333, 40.82582759866107], [-73.93435643452145, 40.82582093714441], [-73.93432574289118, 40.82523259029445], [-73.93429530732783, 40.82464923122279], [-73.9342937167057, 40.824496386917176], [-73.93427285440853, 40.82443895275126], [-73.9342464289508, 40.82416688303727], [-73.9342183480328, 40.823799045790075], [-73.93416594530665, 40.82307858087102], [-73.93414657666992, 40.82302903205036], [-73.93414604382117, 40.823020619180404], [-73.93413373625334, 40.82280065597675], [-73.9341212616508, 40.822647252741746], [-73.93411752612668, 40.82252433268852], [-73.9341023336634, 40.8222012169168], [-73.93409986738209, 40.82213991284362], [-73.93409843799957, 40.82210422734389], [-73.93409112517362, 40.82199603799018], [-73.93407944723481, 40.82180557579632], [-73.9340743435484, 40.82171709911022], [-73.93407734095152, 40.821694561770016], [-73.93407005522698, 40.821559320934185], [-73.93404447476462, 40.82117670357437], [-73.93404599779073, 40.821133876520214], [-73.93403560443168, 40.82095254728241], [-73.93402662334591, 40.82081680975282], [-73.93400389615688, 40.82039784018414], [-73.93400066445355, 40.82033825378207], [-73.9339827694626, 40.82000029618647], [-73.93398644004239, 40.81996843867569], [-73.93397209670252, 40.81973020313448], [-73.93382990021335, 40.81972042397129], [-73.93383051181283, 40.819597453238174], [-73.93383064085586, 40.81957146408635], [-73.93383089287109, 40.819521063761556], [-73.9341167129999, 40.819547052999965], [-73.93431276899999, 40.81962986599991], [-73.93440039999986, 40.81966778299989], [-73.93439036800001, 40.819587434999924], [-73.934381129, 40.819328075999906], [-73.93429487799989, 40.81890498699991], [-73.934235761, 40.81864931899987], [-73.93408481999982, 40.8179308619999], [-73.93407463999996, 40.81782651399992], [-73.93412816799999, 40.81773396299989], [-73.934235195, 40.81755488099992], [-73.93445400100002, 40.81718852899994], [-73.93463790299994, 40.81690804799997], [-73.93487343800004, 40.81652784699992], [-73.93575548199986, 40.81532558399993], [-73.93613554399998, 40.814788126999915], [-73.936209578, 40.81467384999995], [-73.9366881329999, 40.81404324299994], [-73.93714398399993, 40.81342104099985], [-73.93763499299979, 40.81274779999994], [-73.9381355239999, 40.81206561899994], [-73.93820642699986, 40.81196760499992], [-73.93830924999986, 40.81182546399991], [-73.93838748499988, 40.811718443999894], [-73.93863753299986, 40.81137638799993], [-73.93874394499981, 40.8112308209999], [-73.93901753299998, 40.81085655699991], [-73.93740733700004, 40.810175610999856], [-73.93580780199986, 40.80949763799987], [-73.93586263399992, 40.80943020299987], [-73.93625935999997, 40.80885560799991], [-73.93671827500005, 40.80822047899987], [-73.93718011799989, 40.807598999999904], [-73.937634394, 40.8069746069999], [-73.9380866689998, 40.80635301199987], [-73.93854407899988, 40.80572965299991], [-73.93869430499996, 40.80579102899988], [-73.94015793699995, 40.8064090899999], [-73.94177140199992, 40.807088999999955], [-73.94491601899982, 40.80841350399989], [-73.94499360699993, 40.80844618399995], [-73.94506102999992, 40.80847449999987], [-73.94773985499985, 40.809599720999884], [-73.94783875999985, 40.8096412609999], [-73.94792306799984, 40.80967692199992], [-73.95067975599989, 40.81084296399989], [-73.95210125199992, 40.811442853999935], [-73.95150537299989, 40.812007684999905], [-73.95098203399995, 40.812521629999885], [-73.95092724899993, 40.8125754289999], [-73.95035019299995, 40.8131593889999], [-73.94986218999995, 40.81379997999992], [-73.94962668099991, 40.81410004999989], [-73.94962653799999, 40.81410023899991], [-73.94962639600001, 40.81410043599992], [-73.949405505, 40.81440740899996], [-73.94919960099985, 40.81472085399986], [-73.94900927799986, 40.81503994299992], [-73.94883491299987, 40.815363980999955], [-73.94867681799998, 40.8156922859999], [-73.94835601499989, 40.81637430599994], [-73.94800732899995, 40.81711650899995], [-73.94766641299987, 40.817855508999905], [-73.94734953999985, 40.81853499799996], [-73.94703623199985, 40.81920837199988], [-73.94671831999985, 40.81989426799994], [-73.94640289799987, 40.82057204199995], [-73.94607828700005, 40.821263215999906], [-73.94515456999994, 40.82087626499993], [-73.94460364599989, 40.82146198099991], [-73.94404986399995, 40.82205159199989], [-73.9435059449999, 40.82262775299988], [-73.94292366900005, 40.8232605869999], [-73.94242953999998, 40.823909156999896], [-73.94195569699997, 40.82455443999991], [-73.94150312699996, 40.82517136699994], [-73.94104840899992, 40.82579262999989], [-73.94059472699999, 40.826413603999875], [-73.94013953799991, 40.827038689999874], [-73.93968952899993, 40.827656573999896], [-73.93924885599988, 40.82829635099989], [-73.93896034899993, 40.828986329999914], [-73.93864026699997, 40.82973887399988], [-73.93899743399992, 40.82988885299995], [-73.93962569899988, 40.830155836999936], [-73.9401708989999, 40.83038439099996], [-73.94007905799988, 40.8304639579999], [-73.93998939900004, 40.830730965999855], [-73.93996368699995, 40.83082053399987], [-73.93996366299984, 40.8308206059999], [-73.93992395099987, 40.83090805999994], [-73.93992389199987, 40.83090817699982], [-73.93992383299992, 40.830908276999914], [-73.93987063299991, 40.830991796999896], [-73.9398705739998, 40.83099188599994], [-73.93987049099984, 40.83099197699987], [-73.93980498999994, 40.83106978999994], [-73.93980484699993, 40.83106996099988], [-73.9398046819999, 40.8310701129999], [-73.93972831499984, 40.83114089899994], [-73.93960982599997, 40.83125089499988], [-73.93960970799996, 40.83125100199988], [-73.93960960099984, 40.83125111899991], [-73.93950338699985, 40.83136945999995], [-73.939410835, 40.8314949949999], [-73.93941078799995, 40.83149507599993], [-73.93933313199983, 40.83162629999987], [-73.9392711939998, 40.83176165699991], [-73.93927116899984, 40.83176171999991], [-73.93927114599992, 40.831761792999906], [-73.93922536500003, 40.83189966099992], [-73.93913480799989, 40.832097594999915], [-73.93902897299998, 40.832291497999925], [-73.93894859799995, 40.832417335999914], [-73.93890834499989, 40.832480356999945], [-73.9387736029999, 40.83266321199991], [-73.93868316299985, 40.832800788999904], [-73.93862894999997, 40.83285794199987], [-73.93857474799998, 40.832915085999936], [-73.93838508999997, 40.833113502999936], [-73.93795718299988, 40.83345441799985], [-73.93729666499993, 40.833977743999945], [-73.93705905900006, 40.834165988999885], [-73.93700641599989, 40.83419310999991], [-73.93649557199993, 40.83456879599989], [-73.93572624299988, 40.835156346999966], [-73.93530788100004, 40.835477723999915], [-73.93506336299993, 40.83567765899992], [-73.93496274099988, 40.83575993399989], [-73.93485740300004, 40.83598502599991], [-73.9347988639999, 40.83611011599993], [-73.93474569399996, 40.836092093999916], [-73.93467393099985, 40.836064902999915], [-73.93460027799988, 40.83609686799992], [-73.93436121591056, 40.83620061961779]]]}}, {\"id\": \"42\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 43, \"Shape_Leng\": 0.0997386183576, \"Shape_Area\": 0.000379662912054, \"zone\": \"Central Park\", \"LocationID\": 43, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97255352299985, 40.76489773199991], [-73.97301487199995, 40.76427887899991], [-73.9736358469999, 40.7645699129999], [-73.9741922899999, 40.76480652799991], [-73.97623388799995, 40.76567463099986], [-73.97836113000002, 40.766573710999886], [-73.97907658799991, 40.76687608599991], [-73.98136213699995, 40.767836738999904], [-73.98131853299991, 40.76789900199987], [-73.9812909039998, 40.76796936299995], [-73.98129086899992, 40.767969452999914], [-73.98129085699999, 40.76796955199995], [-73.98128253299978, 40.76804427399991], [-73.98129467599998, 40.7681188019999], [-73.98132588999984, 40.76818837999983], [-73.98137263399991, 40.76824931599992], [-73.98145320300002, 40.76832036999991], [-73.98145334499989, 40.768320494999934], [-73.98145349899993, 40.76832060299992], [-73.98154591499996, 40.76838333899988], [-73.98154597399994, 40.768383374999864], [-73.98164804599995, 40.76843632199994], [-73.9810229009999, 40.76927283399994], [-73.980563707, 40.76990532499988], [-73.98011181599992, 40.770528368999884], [-73.97965233399994, 40.771155326999896], [-73.97919214499984, 40.77178126499996], [-73.97872246800002, 40.772418897999934], [-73.97828220099997, 40.773031878999866], [-73.97782639399986, 40.77365619199991], [-73.97737084999984, 40.77428100699992], [-73.97691168299995, 40.774909123999905], [-73.97645763799986, 40.77553235199982], [-73.97595517399989, 40.7762197549999], [-73.97546067199991, 40.77689850299986], [-73.97499744000007, 40.77753254599988], [-73.97453231499993, 40.77816778499994], [-73.97406668299993, 40.778805416999866], [-73.97357117499989, 40.77947786199996], [-73.97172305599986, 40.78202147599997], [-73.97122292299984, 40.782706255999884], [-73.97076013099995, 40.783340137999865], [-73.97030068199992, 40.783975413999904], [-73.96983225599998, 40.784610910999916], [-73.96933573299991, 40.78529327999992], [-73.96884378999985, 40.78596738899994], [-73.96838479399985, 40.78659569699989], [-73.96792696399991, 40.78722157099988], [-73.96744908399992, 40.78786072099995], [-73.96700977099991, 40.78847678999989], [-73.96655226699977, 40.789107152999904], [-73.96609500599993, 40.7897343889999], [-73.96563269799995, 40.790365403999886], [-73.96517705600003, 40.79099034099996], [-73.96468540699995, 40.791664026999904], [-73.96417598599989, 40.79236204499989], [-73.96371096499993, 40.7930129339999], [-73.96280590599986, 40.79423581299993], [-73.96235980199994, 40.79485206099987], [-73.96189985499994, 40.7954792699999], [-73.96144060699996, 40.796110826999964], [-73.96097971799988, 40.796738643999916], [-73.96052271699993, 40.79736846899988], [-73.96004456499999, 40.79804123499991], [-73.95953658899998, 40.798718525999895], [-73.95907669099992, 40.79935223299986], [-73.95861879299989, 40.79997702599994], [-73.95833304999988, 40.80036505399987], [-73.95817297099987, 40.800582540999876], [-73.95786499299992, 40.800515271999885], [-73.95536651699993, 40.79946642599985], [-73.95528884999997, 40.79943440199992], [-73.95521824499984, 40.79940465799989], [-73.95516503099994, 40.79938223299988], [-73.95253597599994, 40.798274645999896], [-73.95245200999996, 40.798240230999944], [-73.95237419099993, 40.79820977099995], [-73.94960357199982, 40.79705027099992], [-73.94922045699984, 40.796909858999896], [-73.94940516399991, 40.79663170399987], [-73.94969538299988, 40.79621687199992], [-73.95015821299991, 40.795587731999895], [-73.95061371799994, 40.7949655739999], [-73.95111102399997, 40.794285034999874], [-73.95160265399991, 40.79360780499995], [-73.95207222299993, 40.792966160999896], [-73.95252222700002, 40.792347621999966], [-73.95298394599986, 40.791721432999964], [-73.95343832999994, 40.791085367999926], [-73.95388917099996, 40.790470498999866], [-73.95439919199988, 40.78977490899994], [-73.95481723799992, 40.789204743999974], [-73.95527481200001, 40.78858337899986], [-73.9557773589999, 40.78791392399995], [-73.95626130899994, 40.78722287599986], [-73.9567186309999, 40.78660079299993], [-73.957168537, 40.785982893999865], [-73.95763599400006, 40.78534345799992], [-73.9580956379999, 40.784717461999946], [-73.9585581369999, 40.78409759899993], [-73.95900363199988, 40.783467289999855], [-73.95946905999989, 40.78283332099994], [-73.95992094300001, 40.782212943999895], [-73.96041638899996, 40.78153163299989], [-73.96091849500006, 40.78085062899991], [-73.961368727, 40.78022974899988], [-73.96183836899999, 40.779584191999874], [-73.96230361500002, 40.77894704099985], [-73.9627671559999, 40.77831304099987], [-73.96322991899993, 40.77767704699989], [-73.96372602699994, 40.77700571799989], [-73.9637612389999, 40.77695096899986], [-73.96422820200003, 40.77631030899995], [-73.96468735799992, 40.77567708199994], [-73.96514745399985, 40.77504423199984], [-73.96560968599992, 40.77440720099987], [-73.96607448999993, 40.77377298899989], [-73.96653616999981, 40.77314036899988], [-73.96703947599987, 40.772461218999844], [-73.96753398399983, 40.77177618399991], [-73.96799327799992, 40.77114722999992], [-73.96844155899997, 40.77051632699994], [-73.96890454699995, 40.769900831999905], [-73.96936484599995, 40.769270656999915], [-73.96982455299987, 40.76864600899992], [-73.97028165499981, 40.768025031999905], [-73.97072777399985, 40.76739372199987], [-73.97118812999996, 40.766766592999915], [-73.97164774299996, 40.76614246799992], [-73.97209852400005, 40.76551725099991], [-73.97255352299985, 40.76489773199991]]]}}, {\"id\": \"43\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 44, \"Shape_Leng\": 0.235688967594, \"Shape_Area\": 0.00194465649192, \"zone\": \"Charleston/Tottenville\", \"LocationID\": 44, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.21220034099994, 40.556443429999874], [-74.213501756, 40.55566761799985], [-74.2143713889999, 40.555588065999885], [-74.21455413999996, 40.55559031799993], [-74.21455422199988, 40.55559031799987], [-74.21455430499994, 40.5555903089999], [-74.21473791099994, 40.55557614399984], [-74.21473802999996, 40.55557613299985], [-74.21473815899992, 40.555576114999965], [-74.21491900899998, 40.555545456999916], [-74.21509339699995, 40.55549893299997], [-74.21525777199992, 40.555437837999925], [-74.21721338099994, 40.55471485899989], [-74.21784397899994, 40.554432717999894], [-74.21787855399997, 40.55423566199991], [-74.21789469299988, 40.554036199999906], [-74.21789204399995, 40.55383585499985], [-74.2178705549999, 40.55363617699989], [-74.21783047799995, 40.553438685999886], [-74.21777232699998, 40.55324488499989], [-74.21778377699997, 40.55304918799994], [-74.21877630399993, 40.552473628999955], [-74.219115764, 40.55223231399988], [-74.21931406599998, 40.55205513499987], [-74.22106803199998, 40.550554733999945], [-74.2211486979999, 40.55048965799989], [-74.22121805099987, 40.55041641999991], [-74.2212741239999, 40.550336848999926], [-74.22131562699998, 40.55025307399992], [-74.22109340299987, 40.54977355599989], [-74.22087752099993, 40.54920399899989], [-74.22083496899992, 40.54909173999988], [-74.22064536199997, 40.54859150899989], [-74.22060624599999, 40.54848651999988], [-74.2205699459999, 40.54839063799991], [-74.22036701999993, 40.54783386999984], [-74.22008210999998, 40.547138270999874], [-74.21997344699999, 40.54683441599984], [-74.21987141799997, 40.54657677199991], [-74.21981449699993, 40.546438396999925], [-74.21954309299988, 40.545699383999846], [-74.21929684599996, 40.54505270699986], [-74.21899304699997, 40.544309916999936], [-74.21888584299994, 40.544070039999895], [-74.21873688699995, 40.54366800399986], [-74.21844432399992, 40.54296409499994], [-74.21817195699992, 40.54225028499995], [-74.21803946399996, 40.54191335999994], [-74.21800676099997, 40.54182852099993], [-74.21776941599994, 40.54112662999992], [-74.21775655499992, 40.54098679299991], [-74.21767785799992, 40.54013154699988], [-74.21770976099997, 40.5383119929999], [-74.21773895399996, 40.53803020699992], [-74.21779984299988, 40.53774949499994], [-74.21781643899992, 40.537655948999934], [-74.21786779799996, 40.53736657899988], [-74.21791053499996, 40.53702886099995], [-74.21806225299997, 40.53649651299984], [-74.21819886099995, 40.53585901999993], [-74.21828207299988, 40.53548704299991], [-74.2183151519999, 40.535312078999915], [-74.21829305299998, 40.53476500299996], [-74.2182685219999, 40.534525247999966], [-74.21800676499996, 40.53229858999995], [-74.21775551799993, 40.531448847999904], [-74.21752355299995, 40.53064379899991], [-74.21728324499995, 40.52987328299983], [-74.21701386699988, 40.5291220849999], [-74.21684516999989, 40.528642086999895], [-74.21672062699999, 40.5284057529999], [-74.21666369900001, 40.52828331599991], [-74.21661081199987, 40.52817658599992], [-74.21648646999992, 40.527925626999924], [-74.2161939549999, 40.52735009899994], [-74.21586638599997, 40.526647941999926], [-74.21574341099993, 40.52646400499992], [-74.21564470499992, 40.525981847999894], [-74.21560783599998, 40.52549473899995], [-74.21559935899988, 40.524373901999894], [-74.215596992, 40.524061069999945], [-74.2156723039999, 40.52395762799989], [-74.21409806799986, 40.52409719599991], [-74.21398779899988, 40.52345242299989], [-74.21390912599999, 40.523014630999945], [-74.21381535699993, 40.52249465099986], [-74.21361475699992, 40.52153163599994], [-74.21354593399995, 40.521235715999886], [-74.21450845999989, 40.52115497999993], [-74.21735501099992, 40.520930048999915], [-74.21760570099997, 40.52087168699991], [-74.21802828299994, 40.520663726999935], [-74.22013810199992, 40.5196218829999], [-74.22269085399991, 40.518512303999934], [-74.22281984999995, 40.518456466999915], [-74.22295200699995, 40.51840511399991], [-74.22308705699992, 40.518358350999904], [-74.22322472599994, 40.51831627099988], [-74.2233647369999, 40.518278959999904], [-74.22350680499993, 40.518246492999864], [-74.22365064300001, 40.51821893699983], [-74.22379595899992, 40.518196345999876], [-74.22398999699992, 40.518212367999894], [-74.22418327999995, 40.51823302699989], [-74.22437561999999, 40.518258301999914], [-74.22456682599996, 40.51828816899989], [-74.22475670999995, 40.51832259699992], [-74.22494508699998, 40.51836155399993], [-74.22513176899994, 40.51840500099992], [-74.22531657499994, 40.51845289399988], [-74.22549932099996, 40.518505187999935], [-74.22567982799994, 40.51856182999992], [-74.22582224199994, 40.51868027699985], [-74.22597849399989, 40.518790184999894], [-74.226147208, 40.51889020799993], [-74.22632671899993, 40.51897915999993], [-74.22651508599992, 40.5190560909999], [-74.226710219, 40.51912029999991], [-74.22690993299994, 40.5191713979999], [-74.22663407199991, 40.51725100999996], [-74.22290251399997, 40.5089365389999], [-74.22321582499997, 40.508658610999866], [-74.22188242299993, 40.502674840999866], [-74.22159300278173, 40.50250043822093], [-74.22193907695905, 40.50257361983842], [-74.22199500258405, 40.50258924110122], [-74.22205322906996, 40.5025987795954], [-74.22211262695768, 40.502602050296346], [-74.22217204406495, 40.50259898975985], [-74.2222303278366, 40.50258965735334], [-74.22228634770266, 40.50257423410402], [-74.22233901700922, 40.50255301918697], [-74.22236766251469, 40.50253421998262], [-74.22239987668148, 40.50251915820311], [-74.22243483089085, 40.50250822126949], [-74.22247162604596, 40.50250169050246], [-74.22250931569815, 40.50249973388721], [-74.2225469303907, 40.50250240175186], [-74.22257824500184, 40.50251123682858], [-74.22261036634667, 40.50251818226514], [-74.2226481858538, 40.50252379471867], [-74.22268651324718, 40.50252678885857], [-74.2227210401853, 40.50252722753436], [-74.22275550111316, 40.50252554054765], [-74.2228336895983, 40.50251862120146], [-74.22291062035362, 40.50250591066576], [-74.2229855590458, 40.50248753026787], [-74.22304657854346, 40.502478179056766], [-74.2231050708459, 40.502461948414016], [-74.2231597130021, 40.5024392054374], [-74.22320926914568, 40.502410464516764], [-74.22325261844665, 40.50237637570049], [-74.22327434185924, 40.5023556759657], [-74.22330007253782, 40.5023378525658], [-74.22332916560721, 40.502323352197806], [-74.22336089192405, 40.50231253827547], [-74.22339445635099, 40.50230568182112], [-74.22342901768424, 40.50230295467355], [-74.22351209883621, 40.50230698498269], [-74.22359532981133, 40.50230567183279], [-74.22367812184865, 40.50229902451277], [-74.22375988929225, 40.50228709004475], [-74.22384005373509, 40.502269952850824], [-74.2239180481101, 40.502247734156306], [-74.22399332070259, 40.50222059113186], [-74.22410098578254, 40.50215106386502], [-74.22421217259708, 40.50208484512531], [-74.22432670677236, 40.502022038757524], [-74.22444440868638, 40.50196274325467], [-74.22475631505407, 40.50185823555003], [-74.2248115505849, 40.501827108477364], [-74.2248716718487, 40.50180180074029], [-74.22493563313823, 40.50178275252197], [-74.22500232195851, 40.501770295132], [-74.22507057837701, 40.50176464524428], [-74.22513921519668, 40.50176590112838], [-74.22520703860451, 40.50177404094029], [-74.22527286893484, 40.50178892310292], [-74.22548789675064, 40.50184439266374], [-74.22570064658579, 40.501904764360354], [-74.22591092670548, 40.501969983786225], [-74.22611854759943, 40.502039992166104], [-74.22623690028469, 40.50209386762523], [-74.226359560403, 40.502141801478466], [-74.22648601674535, 40.502183593949056], [-74.22661574227789, 40.50221907085599], [-74.2267481963388, 40.50224808433943], [-74.2268828268918, 40.502270513477725], [-74.22701907282793, 40.50228626479096], [-74.22715636630399, 40.502295272631336], [-74.22729413510996, 40.50229749945575], [-74.227431805054, 40.5022929359835], [-74.22756880235707, 40.50228160123398], [-74.22769425171127, 40.50226247625823], [-74.22782110258426, 40.50224983829663], [-74.22794877997727, 40.50224374463544], [-74.22807670514588, 40.502244222896394], [-74.22820429822289, 40.50225127091177], [-74.22822138349655, 40.50224965259281], [-74.22823668093693, 40.50224802576499], [-74.22825206733614, 40.50224699935704], [-74.22826750275328, 40.50224657395113], [-74.22828294593444, 40.502246751304384], [-74.22829835803543, 40.502247531996694], [-74.228313695154, 40.50224891343501], [-74.22832892150001, 40.50225089066462], [-74.22834434243411, 40.50225342203293], [-74.2283599010297, 40.502255390388335], [-74.22837556698457, 40.50225679143534], [-74.22839130144582, 40.50225762324074], [-74.22840707060399, 40.50225788268887], [-74.22842283627269, 40.50225757035817], [-74.22843856574725, 40.502256685308716], [-74.2284938457047, 40.50224519955647], [-74.22854942780755, 40.502234599247785], [-74.22860528683556, 40.502224889290545], [-74.22873640620917, 40.50221079463369], [-74.22877770924977, 40.502208772288284], [-74.22881893708691, 40.502206002889466], [-74.22886006514686, 40.502202488831685], [-74.22890106798002, 40.502198233181424], [-74.22895205756527, 40.502189553870046], [-74.2290032537432, 40.50218160422415], [-74.22905463305725, 40.50217439149411], [-74.22907591039723, 40.50217316105877], [-74.22909711797962, 40.50217136416866], [-74.2291182279392, 40.502169003225], [-74.22913921811273, 40.502166079277], [-74.22916006173955, 40.5021625967343], [-74.22918073095957, 40.50215855950567], [-74.22920120186964, 40.50215397316742], [-74.22922144946841, 40.50214884329823], [-74.22924144633903, 40.50214317481132], [-74.2292703849778, 40.50213741759058], [-74.22929952338401, 40.50213225235332], [-74.22932883346786, 40.50212767965797], [-74.22935829308656, 40.50212370608246], [-74.22938787284336, 40.50212033503606], [-74.22941755167608, 40.502117568236], [-74.22942138198772, 40.50211722132797], [-74.2294384115576, 40.50211505779295], [-74.22945528716919, 40.502112286928174], [-74.2294974050212, 40.50210738769351], [-74.22953936378703, 40.502101795518115], [-74.22958114326791, 40.50209550859965], [-74.22961022163645, 40.50209313212578], [-74.22963939213257, 40.50209150867983], [-74.22966861789233, 40.502090640010884], [-74.229684458175, 40.50208957229493], [-74.22970021195388, 40.50208792023397], [-74.22971584237062, 40.50208568758758], [-74.22973131345135, 40.50208287995638], [-74.22974658812029, 40.50207950177041], [-74.229795465952, 40.50206238157272], [-74.22984406719716, 40.50204480572096], [-74.22989238527848, 40.50202677640676], [-74.22991536204442, 40.5020183729186], [-74.22993870875453, 40.50201058570158], [-74.22996239099062, 40.50200342588136], [-74.22998637564558, 40.50199690273843], [-74.23016029131371, 40.50192391709752], [-74.23017873406381, 40.501919548757144], [-74.23019694119272, 40.50191463305972], [-74.2302148848441, 40.50190917491978], [-74.23023253608234, 40.50190318427988], [-74.23024986904592, 40.5018966717464], [-74.23026685720335, 40.501889644409026], [-74.23028347096967, 40.501882114892126], [-74.2302996869051, 40.50187409597532], [-74.23030686452938, 40.50187062423134], [-74.23031441673122, 40.50186764628693], [-74.23032228170659, 40.501865187060595], [-74.23033039937631, 40.50186326258836], [-74.23033265625148, 40.50186016292162], [-74.23033663156602, 40.50185470549131], [-74.23033607342072, 40.50185131829552], [-74.23033285397625, 40.50183180571128], [-74.23032743945022, 40.50181173211535], [-74.23032989415533, 40.501808276383215], [-74.23033358226388, 40.50180481767264], [-74.23033892500186, 40.50180417837197], [-74.23034304643087, 40.501807307956355], [-74.23034718449796, 40.50181545841368], [-74.23035422834207, 40.501831447680786], [-74.2303628392935, 40.501849376157026], [-74.23036582286555, 40.50185558825123], [-74.2303805628857, 40.501878153577], [-74.23038426772325, 40.50188286230653], [-74.2303894844148, 40.50189079984279], [-74.23039390320254, 40.501899016579706], [-74.23040176854838, 40.50190545442037], [-74.23041024635407, 40.501911423283836], [-74.2304192906361, 40.501916889923855], [-74.23042885080227, 40.501921821270706], [-74.23043886991823, 40.501926190633455], [-74.23044929215258, 40.50192997299455], [-74.23046005729444, 40.50193314636069], [-74.23047527187609, 40.501936933837634], [-74.23049071815454, 40.50194012962992], [-74.23050635571904, 40.501942725609766], [-74.23052214460141, 40.501944714653426], [-74.23053804483887, 40.501946091312604], [-74.23055401780135, 40.50194685465975], [-74.2305700202341, 40.50194699925312], [-74.23058600979103, 40.5019465285285], [-74.23061606414724, 40.50194431066832], [-74.23064604933, 40.50194157258755], [-74.23067594493172, 40.501938315332694], [-74.23070573318452, 40.501934541787655], [-74.23073539346244, 40.501930253333974], [-74.23076490778703, 40.50192545536911], [-74.23079425749847, 40.50192014659036], [-74.23082342089151, 40.501914333742846], [-74.23084851211847, 40.50190886380689], [-74.2308523804173, 40.50190801937487], [-74.23088112029114, 40.501901208711786], [-74.23095331081439, 40.5018869544787], [-74.23114494958575, 40.50185121588201], [-74.2311805719308, 40.50183722714865], [-74.23120353968633, 40.501822855627815], [-74.2312232711201, 40.50181685411036], [-74.23122469229227, 40.50181644230333], [-74.23123357692766, 40.50181755087981], [-74.23124443244176, 40.5018268603189], [-74.2315487998192, 40.5017245374848], [-74.23184935718473, 40.50161585082739], [-74.23214587623688, 40.50150088291111], [-74.23243813174554, 40.501379721071565], [-74.2327259017227, 40.5012524573491], [-74.23300896759186, 40.501119188418826], [-74.23304744169111, 40.50110177283039], [-74.23314424618417, 40.50107932001994], [-74.23321049636206, 40.5010956471732], [-74.23359904119728, 40.50091253160122], [-74.23441378278534, 40.50077409423288], [-74.23494577701936, 40.50065915779635], [-74.23501360836578, 40.50065361509914], [-74.23508091792974, 40.50064514096061], [-74.23517395943664, 40.500628383527115], [-74.23526495949417, 40.50060602857461], [-74.23535334118338, 40.50057821782871], [-74.23543854418718, 40.50054512760354], [-74.23552002834258, 40.500506967683684], [-74.2355972770665, 40.500463979993775], [-74.23566269106378, 40.500430831843914], [-74.23573157542359, 40.50040207058007], [-74.23580342653713, 40.500377906471954], [-74.23587771910799, 40.50035851618022], [-74.23595390999242, 40.50034404146448], [-74.23610536564176, 40.500266216723205], [-74.23625245135797, 40.500183652267225], [-74.23635177807365, 40.500123671949574], [-74.2364287410083, 40.500076510868816], [-74.23647263236712, 40.5001123997492], [-74.23652981831931, 40.50009943174849], [-74.23661963667443, 40.50007228189297], [-74.23670606284715, 40.50003933498759], [-74.23678845619574, 40.500000835256294], [-74.2368662059753, 40.499957068083674], [-74.23693873586555, 40.49990835789935], [-74.23700550824262, 40.49985506577299], [-74.23706602816398, 40.49979758673748], [-74.23718407596924, 40.49970224271456], [-74.23729667529008, 40.499603132808204], [-74.23740362019552, 40.499500438300515], [-74.23750471509992, 40.49939434702925], [-74.23759977511978, 40.49928505304476], [-74.23768862641342, 40.49917275625363], [-74.23766770359761, 40.49912325493375], [-74.23774805247368, 40.49910084537648], [-74.23777146009982, 40.49918035138809], [-74.23781982777892, 40.499183845546135], [-74.23786840947196, 40.49918350191914], [-74.23792863579281, 40.49917768581353], [-74.23798737378735, 40.49916598084102], [-74.23804363965198, 40.49914858304881], [-74.23809649099027, 40.49912578383293], [-74.23844104903398, 40.49893585618119], [-74.238513216357, 40.49889607636883], [-74.23895150706927, 40.498443384066505], [-74.23944024302581, 40.49793857961804], [-74.2396684504129, 40.497702924941784], [-74.23971237574322, 40.49766233705473], [-74.23976289569347, 40.49762649844568], [-74.23981913721121, 40.49759602844686], [-74.23984851243979, 40.497583013207], [-74.23988017293566, 40.49757362656005], [-74.23991335528727, 40.4975680948406], [-74.23994725938827, 40.49756655143169], [-74.23998106772962, 40.497569033548785], [-74.24001396511112, 40.49757548134175], [-74.2400451582977, 40.4975857393391], [-74.24007389514547, 40.49759956019564], [-74.24013023081554, 40.49761940728271], [-74.24018972118647, 40.4976328595752], [-74.24025112808033, 40.49763963708909], [-74.24031317342877, 40.49763959876344], [-74.24037456587457, 40.49763274539565], [-74.24046072278645, 40.4976189903655], [-74.24054814968598, 40.49761119759537], [-74.24063614005212, 40.49760943006092], [-74.24072398281064, 40.49761370204587], [-74.24081096808003, 40.4976239790273], [-74.24085905484203, 40.497635316542414], [-74.24090866403729, 40.49764178650346], [-74.24095895398746, 40.497643279140284], [-74.24100907146406, 40.497639769128426], [-74.24105816616476, 40.49763131601914], [-74.24110540514047, 40.497618063229794], [-74.24123510821227, 40.49762608502554], [-74.24136522975829, 40.49762707777318], [-74.24149511694067, 40.49762103649198], [-74.24162411809742, 40.49760799149185], [-74.24175158601285, 40.49758800822129], [-74.24187688116483, 40.49756118693905], [-74.24199937493485, 40.497527662210985], [-74.2420393884239, 40.497512515625424], [-74.24208206686333, 40.49750246612824], [-74.24212630449081, 40.4974977740931], [-74.2421709551481, 40.49749856108657], [-74.24221486197615, 40.49750480671827], [-74.24225688738737, 40.49751634916922], [-74.2422959425381, 40.49753288938479], [-74.24233101554053, 40.497553998821914], [-74.24241668300726, 40.49758356032845], [-74.24250563456835, 40.497606791449726], [-74.24259706962503, 40.49762348309601], [-74.24269016522429, 40.49763348503525], [-74.24278408346595, 40.4976367072458], [-74.24287797904488, 40.49763312072612], [-74.24297100685976, 40.497622757756695], [-74.24311685537805, 40.497602449569634], [-74.24326378176856, 40.49758733310723], [-74.24341146909926, 40.49757744097689], [-74.24355959879793, 40.49757279451684], [-74.24370785133766, 40.49757340374976], [-74.2438559069274, 40.49757926736137], [-74.24397481574329, 40.49758175036924], [-74.24409361501311, 40.49757713079004], [-74.24421157785997, 40.49756543688878], [-74.24432798252589, 40.49754674021474], [-74.2444421167878, 40.49752115516415], [-74.24455328231618, 40.497488838279466], [-74.2446607989485, 40.49744998729183], [-74.24476400885132, 40.4974048399108], [-74.24495162973764, 40.497322388839386], [-74.24513522575204, 40.49723481681643], [-74.24531455716688, 40.49714223819553], [-74.24548938982635, 40.49704477386729], [-74.24565949545183, 40.49694255110184], [-74.2458246519406, 40.49683570338224], [-74.24590703235593, 40.49678525694412], [-74.24603300798422, 40.49666493479786], [-74.24616589257572, 40.4965490249669], [-74.24630542124939, 40.49643775846518], [-74.24645131588674, 40.496331357051396], [-74.24660328568625, 40.49623003278634], [-74.2467610277419, 40.49613398761176], [-74.24694239086584, 40.49612122798881], [-74.24712436215972, 40.49611539517035], [-74.24730648798806, 40.496116503696946], [-74.24748831433008, 40.49612455080525], [-74.24766938791153, 40.49613951643462], [-74.24784925733391, 40.496161363277274], [-74.24802747419955, 40.49619003687112], [-74.24820359422863, 40.49622546573579], [-74.24837717836708, 40.49626756155029], [-74.24854779387996, 40.49631621937367], [-74.24871501543038, 40.49637131790576], [-74.24887842613946, 40.496432719790434], [-74.24903761862493, 40.49650027195716], [-74.24917156868965, 40.49656729705339], [-74.2495597836633, 40.49685685437291], [-74.24968820467004, 40.496990212153534], [-74.2499336255057, 40.49720897724609], [-74.250100267188, 40.49735424453503], [-74.2503270675016, 40.49753852654371], [-74.25050715142565, 40.4976916622374], [-74.25068311794705, 40.49784756275831], [-74.25085489422248, 40.49800616359511], [-74.25086108509564, 40.498029214923285], [-74.25090066516684, 40.49809160134346], [-74.25093333881192, 40.4981562764991], [-74.25095888137142, 40.49822279573272], [-74.25097711721169, 40.49829070170703], [-74.25098792093398, 40.4983595275491], [-74.25099121823624, 40.498428800060296], [-74.25098698642542, 40.49849804296943], [-74.2509881338781, 40.49851023840833], [-74.25100346233354, 40.49861817111827], [-74.2510279779697, 40.49872511105251], [-74.25106157647166, 40.49883060298568], [-74.251104114858, 40.49893419785412], [-74.25116773245702, 40.49899309065367], [-74.25123652201721, 40.49904849188248], [-74.25131015560194, 40.49910013741949], [-74.25138828217872, 40.499147781048116], [-74.25147052929267, 40.49919119562997], [-74.25155650484207, 40.49923017418809], [-74.25172949863416, 40.49929161551243], [-74.25191210542215, 40.49936321333001], [-74.25200034554481, 40.49940041958595], [-74.252074841391, 40.499433209856335], [-74.25208380907416, 40.49943724424436], [-74.25225360988088, 40.499509044716525], [-74.25242564997141, 40.499577668819654], [-74.25259982739979, 40.4996430758863], [-74.2527076122783, 40.49971306866778], [-74.25281011834669, 40.499787539414406], [-74.25290702679237, 40.49986625652177], [-74.25299803620874, 40.499948975178405], [-74.25308286353142, 40.50003543812712], [-74.25316124492005, 40.5001253764648], [-74.25323293657812, 40.50021851047876], [-74.25329771551169, 40.50031455051638], [-74.25335538022304, 40.50041319788582], [-74.25340575133804, 40.50051414578479], [-74.25344867216407, 40.50061708025489], [-74.25357745759506, 40.50095885796737], [-74.25370341736796, 40.50130124889621], [-74.25374291422749, 40.50141464717274], [-74.25379045770413, 40.50152623147129], [-74.25384590619183, 40.50163566934946], [-74.2539090945309, 40.501742634758166], [-74.25397983449942, 40.50184680901308], [-74.25405791537396, 40.501947881744094], [-74.25414310455724, 40.50204555182022], [-74.25423514827004, 40.50213952824683], [-74.25427407656716, 40.50216118050971], [-74.25472027611785, 40.50296520166217], [-74.25523408934775, 40.50393456923247], [-74.25527565808395, 40.50403466831163], [-74.25530969765195, 40.50413641072463], [-74.25533609771456, 40.504239466782785], [-74.25535477268829, 40.504343502538745], [-74.25536566202192, 40.50444818086976], [-74.25536873039188, 40.50455316256854], [-74.25535850504895, 40.50517012777074], [-74.25533004608194, 40.50596510120405], [-74.2553071037761, 40.50653645411903], [-74.25532532868485, 40.506700187309534], [-74.25535252614577, 40.50686319378521], [-74.25538521003818, 40.5070113739394], [-74.2554253090784, 40.50715848929814], [-74.25547276441225, 40.50730432375963], [-74.25552750637893, 40.507448663101925], [-74.25555705464842, 40.507512444405016], [-74.2555776815809, 40.507578238363024], [-74.25558915990914, 40.50764532014473], [-74.25559136315209, 40.50771295072906], [-74.25554269163935, 40.50783774249614], [-74.25553185854751, 40.50786122885186], [-74.25552913160817, 40.50786714123413], [-74.25545272937914, 40.508032776305576], [-74.2552481775318, 40.50824323071299], [-74.25504293715964, 40.50839740737782], [-74.25489888207028, 40.508433685802416], [-74.25462856709899, 40.508674825727006], [-74.25470272483841, 40.508716063138706], [-74.25473104757664, 40.50872401077879], [-74.25490187176642, 40.50876438344174], [-74.25490412260378, 40.5087563679414], [-74.2549813000948, 40.508771172775724], [-74.2549729037097, 40.50880566213322], [-74.25489648202563, 40.50878703523463], [-74.25489963820436, 40.50877727590247], [-74.25471558491711, 40.50873762919246], [-74.25468450878448, 40.50872585538434], [-74.25461876022267, 40.50868612216256], [-74.25455923943342, 40.50874601472694], [-74.25450422784878, 40.50880836852237], [-74.25434064057384, 40.508770803134546], [-74.25412213945084, 40.508984766883515], [-74.25410113834869, 40.508977337855086], [-74.2540475777605, 40.508948357247185], [-74.25403489413355, 40.50894453995541], [-74.25402490190875, 40.50894473413955], [-74.25401623457128, 40.50894787669566], [-74.25400832609722, 40.50895281683775], [-74.25399478574111, 40.508968089878174], [-74.25398428084348, 40.50899055325591], [-74.25397681140788, 40.50902020696289], [-74.25411121318568, 40.509167047180746], [-74.25403804567854, 40.509252671645676], [-74.25430713012449, 40.50937626384592], [-74.25434874440127, 40.50932901714464], [-74.2543887773701, 40.50934418413455], [-74.25431454169312, 40.50941369464892], [-74.25428177080046, 40.509397124218005], [-74.25429083557015, 40.5093915568429], [-74.25402069505334, 40.50927297578743], [-74.25389169939274, 40.509423932519965], [-74.25373355115248, 40.50937689274578], [-74.25368911881061, 40.509425596439094], [-74.25370535508549, 40.50943441372992], [-74.25367604695235, 40.509467390170705], [-74.25369339837492, 40.50947698837386], [-74.25413324931496, 40.50968292755783], [-74.25433256547628, 40.50977376100487], [-74.25434937126049, 40.50978095473888], [-74.25456815512877, 40.509880110297466], [-74.25462336202806, 40.509814602108904], [-74.25464651262905, 40.50982624592433], [-74.25452621732619, 40.50997301588608], [-74.25450038789252, 40.50996319670469], [-74.25455333637498, 40.50989597368409], [-74.25412462467563, 40.50969365976767], [-74.25366810379255, 40.50947991476315], [-74.25366393827429, 40.50947795815997], [-74.25364812518256, 40.509470530436246], [-74.25360711255456, 40.50948090044583], [-74.25350297868137, 40.50960201174755], [-74.25347835886951, 40.50967270210219], [-74.25331579805132, 40.50986150454089], [-74.25328327353373, 40.51000017430845], [-74.25319261503304, 40.51024176573132], [-74.25314984243136, 40.510388496541744], [-74.25312730843176, 40.51071258104537], [-74.25306394553081, 40.51105525736969], [-74.25326008268306, 40.511078643793255], [-74.2532580688566, 40.51108868938079], [-74.25306274811025, 40.51106765345778], [-74.2530377993434, 40.51113607723536], [-74.25300847277559, 40.511193989705326], [-74.25302037923967, 40.51128514610703], [-74.25308195034367, 40.511392839392975], [-74.25318020396979, 40.51145816973165], [-74.25343350448411, 40.511636688346], [-74.2534860296364, 40.51159519786609], [-74.25349300571601, 40.51159899228274], [-74.25345907659539, 40.51163043841861], [-74.25348740701011, 40.51164944524402], [-74.2535239695038, 40.51161898908778], [-74.25353195022859, 40.511622721474204], [-74.25349517785072, 40.51165588724196], [-74.2535268507671, 40.511677766031795], [-74.25350211252186, 40.51170091785187], [-74.25317151924685, 40.51146606218066], [-74.25309286964242, 40.51141076593543], [-74.2530619950522, 40.51141937810189], [-74.25303635381549, 40.511433160458736], [-74.25303207962352, 40.51146822794039], [-74.25300008721311, 40.51183226997468], [-74.25299713902285, 40.51185914541614], [-74.25299940823137, 40.51188605922221], [-74.253006845369, 40.51191242458865], [-74.2530192882889, 40.51193766666826], [-74.25303646570178, 40.51196123510421], [-74.25305800309056, 40.51198261603], [-74.25307641613708, 40.511997777367725], [-74.25309036761324, 40.51201668511298], [-74.25309984531484, 40.51203714387679], [-74.25310456615327, 40.512058542588186], [-74.25310438912015, 40.51208024210097], [-74.25309931949845, 40.51210159428354], [-74.25308950870547, 40.51212196137835], [-74.25307524977121, 40.51214073504986], [-74.25243270667266, 40.51259432885166], [-74.25216570187291, 40.51278281361988], [-74.25195197488479, 40.51299700254723], [-74.2519293126925, 40.51320944549542], [-74.251994679988, 40.5133474300147], [-74.25208955294612, 40.51346052979929], [-74.25214378894988, 40.5135534708155], [-74.2521259139705, 40.51371037979995], [-74.25179186478277, 40.51400480544838], [-74.2517859200255, 40.5140100462964], [-74.25175583198892, 40.51403693838163], [-74.25172346740699, 40.51406224389354], [-74.25168048931724, 40.51409118993782], [-74.2516345376261, 40.51411734028925], [-74.25158592366496, 40.51414051777254], [-74.25153546924818, 40.51417472458121], [-74.25148978164565, 40.514212633084355], [-74.2514493247853, 40.514253858352234], [-74.25141450948402, 40.514297981775876], [-74.25138568927517, 40.51434455531663], [-74.25138383883895, 40.5143556560311], [-74.25137951255292, 40.51436634799982], [-74.25137283954724, 40.514376312087855], [-74.25136401899758, 40.514385250886605], [-74.25135331417916, 40.514392897589765], [-74.25134104461047, 40.51439902395792], [-74.25132757651421, 40.5144034471305], [-74.25131331188783, 40.51440603508408], [-74.25129867650337, 40.51440671057315], [-74.25128410719968, 40.51440545343558], [-74.25127003884319, 40.51440230119468], [-74.25118528349849, 40.51442604654969], [-74.25110334075717, 40.51445497653476], [-74.2510247540467, 40.51448889929356], [-74.25095004454057, 40.51452758985852], [-74.25087970770177, 40.514570791642846], [-74.25075668084278, 40.51462320285307], [-74.25068266586877, 40.514649708649145], [-74.25061193066762, 40.514680999046774], [-74.25054500497247, 40.51471683971555], [-74.25048238999003, 40.51475696224807], [-74.25042455464683, 40.51480106616999], [-74.25037193207766, 40.51484882119001], [-74.25036757485495, 40.51486210326496], [-74.25036065006782, 40.51487473421425], [-74.25035131521172, 40.51488642676049], [-74.25033978259643, 40.51489691496934], [-74.2503263145185, 40.514905960297504], [-74.25031121729425, 40.514913357018564], [-74.25029483429408, 40.514918936901836], [-74.25025503032282, 40.51494653038555], [-74.25022073120984, 40.51497813745146], [-74.25019262447161, 40.51501312455233], [-74.25011262464746, 40.51503074559172], [-74.2500458775657, 40.51510851049426], [-74.25004011503235, 40.51518249067772], [-74.25018131058542, 40.51532776528284], [-74.25017086663703, 40.51533399691467], [-74.25028083612378, 40.51544116717457], [-74.25027416157674, 40.51544556031963], [-74.25028567520496, 40.51545720726751], [-74.25029330638937, 40.51545281320833], [-74.25055526413547, 40.51571634242889], [-74.25045509923827, 40.51577200710897], [-74.25019601571454, 40.51550993109391], [-74.25027518309636, 40.51546306675979], [-74.250261753469, 40.515450693385766], [-74.2502522096781, 40.515455092115594], [-74.25015626516773, 40.51536118819946], [-74.24988824874988, 40.51552154519294], [-74.25047627306125, 40.51611729929947], [-74.25053507045652, 40.51608604354204], [-74.25056007272197, 40.516106741535], [-74.25041535921383, 40.5161883350259], [-74.25039487478166, 40.51616416863538], [-74.25045139325624, 40.51612945938236], [-74.24984374594052, 40.51551805020887], [-74.24935660923894, 40.51500592819369], [-74.24927734323958, 40.51505009620043], [-74.2493572627287, 40.51512887092506], [-74.24933775243697, 40.51514081437928], [-74.2492561503189, 40.515061195085735], [-74.24921528676096, 40.51508486129925], [-74.24952480927368, 40.51540995825117], [-74.24963220109247, 40.515545377951014], [-74.24962270300641, 40.515574730442026], [-74.24950047123725, 40.51564648856688], [-74.24953303060417, 40.515682164214226], [-74.24951393678684, 40.51569398097641], [-74.24994158808036, 40.516137488190985], [-74.24989861903732, 40.516160062404325], [-74.24948229759761, 40.515716531577304], [-74.2494642033825, 40.515725217684285], [-74.24943096899091, 40.51568788559742], [-74.24905586508787, 40.51590409611533], [-74.24903055962491, 40.515881236130525], [-74.24864764632096, 40.51611262849677], [-74.24871583641298, 40.51616954901984], [-74.24870679818525, 40.516176485625], [-74.24884323664601, 40.516305889562055], [-74.24882966609398, 40.516312836002854], [-74.24868189274336, 40.51618172785625], [-74.24862466940564, 40.516215751555514], [-74.2487595831514, 40.51635622113422], [-74.24873695634659, 40.51636491694885], [-74.24854589291671, 40.51616991714839], [-74.24855013363099, 40.51619574727786], [-74.24851245601478, 40.51622571840381], [-74.2485749638381, 40.51628869599919], [-74.24860080823046, 40.5163133319178], [-74.24828422083891, 40.51648694955366], [-74.24894624665603, 40.517194539786644], [-74.24937814038087, 40.51695149632221], [-74.24940770789283, 40.516980831423645], [-74.24851449056031, 40.517475626265046], [-74.24848720326882, 40.51744974618362], [-74.24891910653055, 40.51720843369238], [-74.24822171694589, 40.5164860197475], [-74.24751811835522, 40.51573586477858], [-74.24693048008466, 40.51606645441775], [-74.24687147835442, 40.516036881776515], [-74.24664175789378, 40.516121936076665], [-74.24644039651318, 40.515965028508525], [-74.24612560876693, 40.516116596447006], [-74.24612462197246, 40.516127369298324], [-74.24612857679293, 40.5161373925737], [-74.24613667051254, 40.51614674003258], [-74.24614748095286, 40.51615375951751], [-74.24615993200383, 40.516157604791324], [-74.2461746405067, 40.51615923046234], [-74.24618298246288, 40.51615926634782], [-74.24623549606487, 40.516154056571175], [-74.24625286283556, 40.51615339844281], [-74.24630085106863, 40.51614061103397], [-74.24652992417231, 40.51637770064297], [-74.24659873437136, 40.516358202326835], [-74.24674997194062, 40.516495613240494], [-74.24659339236202, 40.51658682659807], [-74.24651040267673, 40.516669855305985], [-74.24669769426788, 40.51687938448535], [-74.2466932357773, 40.5168818628046], [-74.24670706721054, 40.51689543206918], [-74.24671033904563, 40.51689398125468], [-74.246995569966, 40.51718208519555], [-74.24696892776889, 40.51719824208417], [-74.2466825860256, 40.51690695637092], [-74.24669527621076, 40.51690149656922], [-74.24668162673821, 40.516885006511984], [-74.24666521689264, 40.51689161720225], [-74.24647899705323, 40.51668732721085], [-74.24644391386606, 40.51670254516729], [-74.24642373950073, 40.51671516201796], [-74.24637736435443, 40.51673909335886], [-74.24656439920257, 40.516945792199216], [-74.24653682834258, 40.51695996037955], [-74.24634969801767, 40.51675337193499], [-74.24598176886698, 40.51694324236162], [-74.24595276991943, 40.51693199314788], [-74.24520861528356, 40.517289743624104], [-74.24566989001377, 40.51778970805126], [-74.24559938749574, 40.5178572955057], [-74.24571276694213, 40.517984421621456], [-74.24566427146003, 40.518049469141076], [-74.2456347316994, 40.518075580745766], [-74.24556879842257, 40.51808986210907], [-74.24556587385842, 40.51809049526678], [-74.24552667154194, 40.51808086300408], [-74.24549055670978, 40.518060023004196], [-74.24537078493631, 40.518039913035246], [-74.2453056843134, 40.51802937476492], [-74.24524560494235, 40.51803406401096], [-74.24519792215143, 40.518037784047856], [-74.24508804460535, 40.51806683940414], [-74.24492775667768, 40.51812002114928], [-74.24486088052586, 40.518155324562485], [-74.24477399945269, 40.518208320321065], [-74.24469200084191, 40.51825867000793], [-74.24460054529645, 40.51833956292471], [-74.24451292617638, 40.51841704504164], [-74.24440812395468, 40.51847737885722], [-74.24431751097607, 40.518513227043705], [-74.24429835217606, 40.518515003107694], [-74.2441931864379, 40.518507646784464], [-74.2439995737616, 40.518441159390015], [-74.24382126609419, 40.51832938469257], [-74.24365228511596, 40.51829461462316], [-74.24301946478842, 40.51830792476936], [-74.24282337422682, 40.51830818329581], [-74.24263711942852, 40.518242768834334], [-74.24232477833276, 40.51832652502174], [-74.24209123122782, 40.51848531749823], [-74.24186964584052, 40.51874212138676], [-74.2412596867521, 40.519180469822984], [-74.24109075921592, 40.51934560382892], [-74.24103122918532, 40.519542512687934], [-74.2408696197996, 40.5194940796585], [-74.24075928637673, 40.51931965712868], [-74.24059645807856, 40.51920981244947], [-74.24049069687692, 40.5191916289701], [-74.24039560067511, 40.51927909989127], [-74.24024433791301, 40.51951749846853], [-74.24018802896548, 40.51966380798033], [-74.2400965083332, 40.519868231034316], [-74.23991937852388, 40.52004699652993], [-74.23992701129993, 40.52018125023725], [-74.24003142061724, 40.520313119018425], [-74.24033380813592, 40.52040852891315], [-74.24055584307511, 40.520447528900064], [-74.2407328454876, 40.520569151653746], [-74.2408465963753, 40.520653519664386], [-74.24093977908043, 40.5207536737846], [-74.24092192194361, 40.520806836825486], [-74.24097377890219, 40.52088739946692], [-74.24117293552197, 40.52106013143197], [-74.24121924614649, 40.521032488187025], [-74.2412395676565, 40.520947837847686], [-74.24133219575513, 40.52089451758021], [-74.2413837903956, 40.52090227902593], [-74.2415721732834, 40.52094713835036], [-74.24189065727207, 40.520977549491654], [-74.24293235036517, 40.52122716113276], [-74.2429043393599, 40.52142442574137], [-74.24281484743372, 40.521631214872606], [-74.24259440562305, 40.52205564138241], [-74.24258375639438, 40.52219864900052], [-74.24259082520992, 40.522251637320856], [-74.24264266046022, 40.52241135944537], [-74.24270806228716, 40.5225727261596], [-74.24271469863487, 40.52265675986605], [-74.24273215954128, 40.52270077737195], [-74.24276246783717, 40.52272678912752], [-74.24274849039318, 40.52280007810463], [-74.24269227372879, 40.522840930770386], [-74.24265487812147, 40.52288393777585], [-74.24262345539675, 40.52294713519763], [-74.2426212927105, 40.523025339336115], [-74.24261397838424, 40.523080937524476], [-74.24261666939863, 40.523098937889515], [-74.2426579383821, 40.52313222884206], [-74.24269383331624, 40.52315161267547], [-74.24271065391652, 40.52317754423211], [-74.24272886354078, 40.523229409311334], [-74.24273957613275, 40.52326748653359], [-74.24275355918745, 40.52336703698905], [-74.24275122404825, 40.52351227512357], [-74.24274023652956, 40.52356385012519], [-74.242744234157, 40.52358271771152], [-74.24275345819947, 40.5236062500324], [-74.24276237450053, 40.52362610804578], [-74.24276575861535, 40.52363762634376], [-74.24276517879032, 40.523649952984734], [-74.2427575681127, 40.52368414487768], [-74.24275890301233, 40.52370006921579], [-74.24276471147398, 40.52372130788956], [-74.24279784366254, 40.52376089237593], [-74.24281673693896, 40.52380362991615], [-74.2428183465929, 40.52383245271266], [-74.24281641236698, 40.5238479661431], [-74.24281449076011, 40.52386338790745], [-74.24280882716648, 40.52389195100273], [-74.24280678281418, 40.52392525778574], [-74.24280658251452, 40.52395170899014], [-74.24280252193645, 40.523980193472944], [-74.24280157009575, 40.52400729628661], [-74.24281304883496, 40.52402887377362], [-74.24286227003466, 40.5240873508412], [-74.24290564933676, 40.524133814045314], [-74.24294001416733, 40.52418809487348], [-74.24298985090236, 40.52425392066565], [-74.24302432341914, 40.524338328485776], [-74.24303990307527, 40.524379998795084], [-74.24308072550264, 40.52445364269708], [-74.24311293970743, 40.524511104412284], [-74.24317524593046, 40.52459107967431], [-74.24326436709377, 40.52471217700171], [-74.24330095332317, 40.52477372904665], [-74.24332687887491, 40.524794601716884], [-74.24335792685855, 40.524818913820425], [-74.24338504510442, 40.52483480761203], [-74.24340768923547, 40.524845387088135], [-74.24348882392479, 40.52486125895079], [-74.24360793545556, 40.52484483934403], [-74.24396500758859, 40.52490536633781], [-74.24396761706937, 40.5249509544872], [-74.24396835839076, 40.524998933586524], [-74.24333883883845, 40.52585380517989], [-74.24307389513913, 40.52669417771936], [-74.2430510131873, 40.52681228984682], [-74.2430599177623, 40.526876313027714], [-74.24306019367343, 40.52694069541478], [-74.24305183800163, 40.5270047616636], [-74.24303493837023, 40.52706783974403], [-74.24300967202623, 40.5271292679895], [-74.24297630398283, 40.52718840203788], [-74.24293518423968, 40.527244621590626], [-74.24288674411305, 40.52729733692015], [-74.24285888589208, 40.527335291356586], [-74.2428393395779, 40.52737615650703], [-74.2428286081863, 40.52741888075689], [-74.24282696789425, 40.527462364647704], [-74.24283446093035, 40.5275054891709], [-74.24285089448672, 40.52754714456317], [-74.24287584567887, 40.52758625886605], [-74.24287940175411, 40.527609452185295], [-74.242895150673, 40.52763389612434], [-74.24292862564661, 40.527667928706], [-74.24294563760438, 40.52769784587819], [-74.24294799832856, 40.52772601705562], [-74.24293971177053, 40.527800214062225], [-74.24290983683017, 40.52786685302814], [-74.2428726178986, 40.527922782869666], [-74.2428574567956, 40.527943815603905], [-74.24285076542613, 40.527960130174236], [-74.24284226646175, 40.52800297456181], [-74.24285856207011, 40.528031988594954], [-74.24288723774357, 40.52804910117791], [-74.2428867138121, 40.52820013082601], [-74.24276120556748, 40.52830572728832], [-74.24275675859762, 40.528309693100056], [-74.24270218581978, 40.52834125871439], [-74.24266345955331, 40.528369588623164], [-74.2426421851727, 40.52839460992399], [-74.24262640726322, 40.528408292677646], [-74.2426125770179, 40.52841634680075], [-74.24261267872569, 40.52841757123613], [-74.24259362148119, 40.52842096128823], [-74.24256732309529, 40.5284148646243], [-74.24255062166955, 40.528417523947894], [-74.2425102899773, 40.528445935768495], [-74.24250919866977, 40.52845213755197], [-74.24252416256847, 40.528486458470624], [-74.24253509556485, 40.52851113662018], [-74.24253332154946, 40.52852844211372], [-74.24249931304354, 40.52858421577741], [-74.24249023516836, 40.52863938559633], [-74.24248405744738, 40.52866182499104], [-74.24245926425822, 40.52868332817097], [-74.24242781388115, 40.5287119188902], [-74.24242002837842, 40.52873443622753], [-74.24241293926971, 40.52883255556881], [-74.24241987628413, 40.528867269053514], [-74.24243402108462, 40.52889179048973], [-74.24244956541193, 40.528913784949204], [-74.24245581741143, 40.52893069738365], [-74.24245684392416, 40.52894294590663], [-74.24245541025238, 40.52895469954592], [-74.24244489204953, 40.52896382253608], [-74.24242054279586, 40.52898099845359], [-74.24239489550051, 40.529001927451404], [-74.24237543220742, 40.5290293196263], [-74.24236464310113, 40.529054442708826], [-74.24236058312694, 40.529082928335306], [-74.24233695655916, 40.52913757935194], [-74.24230892358077, 40.52919736476676], [-74.24228365805864, 40.529261320374324], [-74.24225490734621, 40.52931253256033], [-74.2422111306362, 40.529376777503245], [-74.24219764536582, 40.529408180956324], [-74.2421880176231, 40.52943755166016], [-74.24218040993692, 40.529500646074055], [-74.24217260166839, 40.52959019151156], [-74.24217254400455, 40.52966644612745], [-74.24216656926593, 40.52969133454261], [-74.24216769875875, 40.52970480817036], [-74.24217641185439, 40.52972221667961], [-74.24218939305197, 40.5297424900076], [-74.24220131578326, 40.529759741301056], [-74.24222830140566, 40.52978363741721], [-74.2422381043392, 40.529794842469535], [-74.24224565508311, 40.52980800283456], [-74.24224637250255, 40.52981657621738], [-74.24224528137324, 40.52982277950494], [-74.24223421655796, 40.529835003115764], [-74.24202464913051, 40.530062218660646], [-74.24199108185135, 40.53011059776529], [-74.2419404209108, 40.53018728571755], [-74.24190634667363, 40.530244559070034], [-74.24186713793071, 40.530314882318656], [-74.2418432069506, 40.53036233473484], [-74.24182920013061, 40.53039788227688], [-74.24181772165427, 40.530417312887415], [-74.24180708875609, 40.53043141624709], [-74.24178702312778, 40.530453142689694], [-74.2417670305463, 40.530475582488684], [-74.2417475434774, 40.530498351255375], [-74.24172854514237, 40.5305168095949], [-74.24171160461849, 40.530537292564176], [-74.24168936071905, 40.53057403818787], [-74.24168110124019, 40.53059755126657], [-74.24167218669888, 40.530619336638445], [-74.24165729913027, 40.53065101571626], [-74.24164097038334, 40.53068173720384], [-74.24162417534374, 40.53071284398607], [-74.24160715747745, 40.530741837666255], [-74.24159605351099, 40.53076480722383], [-74.24158841240589, 40.53078082617963], [-74.24156916008901, 40.53082873643653], [-74.24154928903585, 40.530874891263835], [-74.24153508696172, 40.53092671714012], [-74.24152427275666, 40.53098012558668], [-74.24150733322897, 40.531041342795135], [-74.24150247553192, 40.531080220323275], [-74.24150032655452, 40.531122852046515], [-74.24149964476034, 40.53114343966333], [-74.24150339439726, 40.53117543685631], [-74.24150594015053, 40.53122310858367], [-74.24150791783289, 40.53124281224186], [-74.24150791145638, 40.531273252933325], [-74.24153092332183, 40.53147007429744], [-74.24171924066692, 40.53160972765347], [-74.24166784213212, 40.53192331944759], [-74.24181061621529, 40.532562675421495], [-74.24217799663231, 40.53257613847663], [-74.2422820033816, 40.53338749651771], [-74.24226378911352, 40.53340449794598], [-74.24224602776077, 40.533417243972124], [-74.24222972576077, 40.53342565004783], [-74.24220533806516, 40.53343668079572], [-74.24218127234485, 40.53344658505237], [-74.24215971270046, 40.53345354737153], [-74.24213691846245, 40.5334577406616], [-74.24211947619509, 40.53345947872177], [-74.24209589720769, 40.533465128998316], [-74.24207044355147, 40.533479400837194], [-74.24205252795593, 40.53349922767831], [-74.24202074376123, 40.533533688413996], [-74.24202006073193, 40.53353443318785], [-74.24199777336811, 40.53355879708958], [-74.24198757151014, 40.53357248725974], [-74.24197674772847, 40.53359367322798], [-74.2419705278025, 40.5336145755794], [-74.24196458870446, 40.53363371843315], [-74.24196285953305, 40.53365293584444], [-74.24196506450933, 40.53366555681744], [-74.24197061665066, 40.53367902330007], [-74.24198641905005, 40.53371102153858], [-74.24199835147876, 40.533728041133706], [-74.24201329191379, 40.53376078284408], [-74.24201538293059, 40.533772005341945], [-74.24201178363337, 40.53379133659616], [-74.24200867165067, 40.53380638523953], [-74.24200316414962, 40.533825142367135], [-74.24200254007258, 40.53384180528584], [-74.24200030699468, 40.533860722371756], [-74.24199689844578, 40.53387294405587], [-74.24198920997155, 40.5339159161509], [-74.24199085844428, 40.53396361772721], [-74.24199307588103, 40.533990045719776], [-74.2419915502286, 40.53401598755378], [-74.24198995216875, 40.53404121636184], [-74.24198749133386, 40.53406721561944], [-74.24199017398918, 40.53410248038075], [-74.24199354944761, 40.53413096455008], [-74.24199960338497, 40.53416317549091], [-74.24201801113493, 40.53422972664229], [-74.24202357895064, 40.53425702543217], [-74.24202759488945, 40.534273431461195], [-74.24203508673898, 40.534305913281365], [-74.24204056270793, 40.53432791577956], [-74.24204570286014, 40.53434637705037], [-74.24205318205567, 40.534365053525036], [-74.2420650025183, 40.53438064665695], [-74.24208466577947, 40.53439611452342], [-74.24209753548908, 40.534403855283955], [-74.24211323214699, 40.53441672238595], [-74.24212310699538, 40.534431716169024], [-74.2421333557727, 40.53445022161224], [-74.24214184326114, 40.53446955468544], [-74.24214917360845, 40.534486832753515], [-74.24215801441201, 40.53451428864594], [-74.2421726750232, 40.53454881562928], [-74.24219707950309, 40.53459235196281], [-74.24220899621434, 40.534604733982555], [-74.2422268606484, 40.534620340649326], [-74.24224858034931, 40.53463780681127], [-74.24226480041055, 40.53464639187509], [-74.24228246325006, 40.534655988960736], [-74.24230272455779, 40.534667885615065], [-74.24232111615434, 40.53467989711757], [-74.2423293469989, 40.53468789408062], [-74.2423437314585, 40.53470581497255], [-74.2423552326689, 40.53472319278325], [-74.24236610201426, 40.53473423256332], [-74.24238253031106, 40.534740345757385], [-74.24240487331872, 40.53474112149672], [-74.24243453514485, 40.53473617417267], [-74.24246309099699, 40.534733779664535], [-74.2424874171602, 40.5347358698341], [-74.24251134992089, 40.53473834447487], [-74.24253258828735, 40.534741675188435], [-74.24254166365212, 40.5347443460579], [-74.24254649044343, 40.53474617534856], [-74.24256127113148, 40.53475448782329], [-74.2425802425302, 40.53476754110567], [-74.24258552957421, 40.5347874024312], [-74.24258647897008, 40.53480117999623], [-74.24258604604269, 40.53481075981608], [-74.24258473066882, 40.53482500723167], [-74.24258240637627, 40.534838600058514], [-74.24257987310271, 40.53485468903216], [-74.24257702716523, 40.53486334113914], [-74.24257356089763, 40.53487948854179], [-74.24257151644906, 40.53489129573698], [-74.24257216705958, 40.53490224668443], [-74.242577236085, 40.53491079880568], [-74.24258619672483, 40.53492123764424], [-74.24259345306332, 40.53492860689938], [-74.24260331267001, 40.534938988384425], [-74.24261556299494, 40.5349541962356], [-74.2426264124883, 40.53496984631141], [-74.24263737229158, 40.53498618286158], [-74.24267503288957, 40.53501404554913], [-74.24268242294441, 40.5350182026167], [-74.24269465731116, 40.5350288255119], [-74.24270423424771, 40.53504099229081], [-74.24271165713903, 40.53505437135167], [-74.24271970002911, 40.5350694783535], [-74.2427319143922, 40.535084685765604], [-74.24274328722593, 40.535096053810946], [-74.24275634772646, 40.53510590648935], [-74.24276764529407, 40.53511656050795], [-74.24278130109536, 40.535132039282225], [-74.24278977364749, 40.5351467622278], [-74.24279405511268, 40.53515677067429], [-74.24279612716543, 40.53517257700573], [-74.24280270365348, 40.53520053285709], [-74.24280937879699, 40.535216054535645], [-74.24282052968051, 40.53522530947483], [-74.24283261641799, 40.53523450597588], [-74.24284499756185, 40.535246528431365], [-74.24285891554203, 40.53526483556585], [-74.24286242837516, 40.53528091276108], [-74.24286096382339, 40.535293733635676], [-74.24285585200917, 40.53530291273569], [-74.24284899763335, 40.53531747476564], [-74.24284699248214, 40.53532999539569], [-74.24284936584534, 40.53533940459772], [-74.24285776172391, 40.5353534403217], [-74.24287082219774, 40.53536326718737], [-74.24288545359322, 40.53537020752436], [-74.24290036605781, 40.53537533676008], [-74.24293260577885, 40.53538652175481], [-74.2429642070392, 40.53540056331801], [-74.24298366574534, 40.53540933358431], [-74.24299170999932, 40.53541524480202], [-74.24299898376137, 40.535427197970556], [-74.24299991637959, 40.53543633513325], [-74.24300083073786, 40.53545011276851], [-74.24300563383345, 40.5354650344494], [-74.24301549236591, 40.535475443376235], [-74.24303042250862, 40.535485183338615], [-74.24305122739914, 40.53549809495174], [-74.24306922463569, 40.53551049081463], [-74.24309258925146, 40.53553495260091], [-74.24310053967928, 40.53554476278485], [-74.24311505623017, 40.53555947123331], [-74.2431294223359, 40.535572780614245], [-74.24314182828353, 40.535580905885354], [-74.24316470983092, 40.53559123344184], [-74.24318048317238, 40.53559559285711], [-74.24319384356232, 40.535599049364784], [-74.24321874959297, 40.535602181198016], [-74.24323948336036, 40.53560518398754], [-74.24324891409962, 40.53560675487901], [-74.24325746299479, 40.53561296756952], [-74.24326300040917, 40.53562184970851], [-74.24326514821085, 40.535629174292325], [-74.24326671547483, 40.53564465131399], [-74.24326787177671, 40.535655930551044], [-74.24327127497509, 40.53567129378235], [-74.24329169749858, 40.53570761941968], [-74.24330785790092, 40.5357293236269], [-74.24332702417657, 40.53575670494126], [-74.24334122131087, 40.5357825035674], [-74.24335015493348, 40.535795769651926], [-74.24336545657813, 40.53580866472732], [-74.24337631463281, 40.535816931634365], [-74.24339700266204, 40.53582707113402], [-74.24342147422972, 40.535839123348126], [-74.24343986400719, 40.535850612146916], [-74.24345994865153, 40.535863223657884], [-74.24347756002385, 40.53587874941399], [-74.24349756433074, 40.535898717363125], [-74.243509230276, 40.53591090762173], [-74.24352601537694, 40.53592649007294], [-74.2435434797697, 40.53594077992129], [-74.24356451801341, 40.53595854917024], [-74.24358815483109, 40.53597861913009], [-74.2435977918341, 40.535986640493775], [-74.24361438955523, 40.53600038367467], [-74.24362902598611, 40.53601852450827], [-74.24363519206028, 40.53603226251381], [-74.24364417849382, 40.536050111601604], [-74.24365012038768, 40.536061381851795], [-74.2436634809121, 40.53607472027231], [-74.24368184908076, 40.53609019011083], [-74.24370362672205, 40.536103318001324], [-74.2437529283142, 40.536127120647656], [-74.24378788666591, 40.53614376297539], [-74.24382561145472, 40.53615995964942], [-74.24384799441027, 40.53617124824739], [-74.24386744479503, 40.53617716315568], [-74.24389476550634, 40.536181414514985], [-74.24395963908158, 40.536184102762945], [-74.24399257458947, 40.53618861775649], [-74.24401106699915, 40.53618877061383], [-74.2440318597577, 40.53618820489821], [-74.24405365117141, 40.53618541375559], [-74.24407177057256, 40.5361819157287], [-74.24409007631027, 40.53618025779554], [-74.24410512126268, 40.53618214772186], [-74.24410657006209, 40.53618280890089], [-74.24411643737697, 40.53618731041337], [-74.24412495988494, 40.53619629572903], [-74.24412917042217, 40.53620647063417], [-74.24412742566432, 40.536221268982885], [-74.24412154393605, 40.536226496050546], [-74.24410856024129, 40.53623736574279], [-74.24405447420418, 40.53628271653438], [-74.24403478981334, 40.536310809887986], [-74.24401667277039, 40.53633481014676], [-74.24399742170681, 40.53636317765834], [-74.24398566896939, 40.536386204268005], [-74.2439810016812, 40.53640820075141], [-74.2439781094152, 40.53644388960205], [-74.24397655106576, 40.53646049855919], [-74.24397853958942, 40.53649255385095], [-74.24397902720065, 40.53651791445383], [-74.24398467190872, 40.53654669629945], [-74.24399781122841, 40.53657807016859], [-74.24401049848484, 40.53660458509813], [-74.24402676361241, 40.536635182181925], [-74.24404541744622, 40.53666964548594], [-74.24406472844545, 40.53670679625804], [-74.24408173920085, 40.53674469357534], [-74.24408936949494, 40.53676507137154], [-74.24410176970596, 40.536801112027725], [-74.2441114292604, 40.53682565819567], [-74.2441234626623, 40.53685006210767], [-74.2441304943149, 40.53686434657369], [-74.24414782538688, 40.53688140991704], [-74.24416391650671, 40.53689367312391], [-74.24420135328856, 40.5369193679386], [-74.24422130941736, 40.536926187550065], [-74.24425651926259, 40.5369333298733], [-74.24428706720578, 40.53693400653993], [-74.24432494105498, 40.53693153796755], [-74.24434400016689, 40.53692921982802], [-74.2443854947254, 40.53692284576036], [-74.24442018769184, 40.53691654060487], [-74.24445168170107, 40.53691046188257], [-74.24448207754243, 40.5369092991662], [-74.24450047479363, 40.53691285687483], [-74.24451809478832, 40.536920394587796], [-74.24455422515392, 40.53693269614596], [-74.24456599437316, 40.53693418237163], [-74.2445875391072, 40.53693292707279], [-74.24467423232045, 40.536900380951046], [-74.2447035632394, 40.53689287991006], [-74.24472875618339, 40.536875809447686], [-74.24475018625944, 40.53685284506122], [-74.24475859495932, 40.53684072136587], [-74.24477092868717, 40.53681917675573], [-74.24478134347447, 40.53679469707654], [-74.24478634953924, 40.53677635100323], [-74.24479242459034, 40.536764974441624], [-74.2448009435216, 40.53675345546139], [-74.24481636013888, 40.53673909519447], [-74.2448285534017, 40.5367283088631], [-74.24483927779454, 40.536719420770524], [-74.24485718066998, 40.53670552126647], [-74.2448864243409, 40.53668484524545], [-74.24490555051072, 40.53667118956239], [-74.24491981038628, 40.53666517544923], [-74.24493693114462, 40.53666387691708], [-74.24495564760973, 40.53666647140767], [-74.24498370121022, 40.536674041738905], [-74.24500424061648, 40.536682973369906], [-74.24506311067748, 40.53670620670757], [-74.24508662785047, 40.53671249704461], [-74.2451085251395, 40.53671890123793], [-74.24513716051963, 40.536728584192375], [-74.2451678287269, 40.53674243397901], [-74.2451719678184, 40.5367446928365], [-74.24518862824804, 40.53675378215509], [-74.24521393085166, 40.53676640751344], [-74.24523301385155, 40.53678061407116], [-74.24524710649037, 40.536797243895386], [-74.24525506330733, 40.536808727716135], [-74.24529835540645, 40.53686103448748], [-74.24532046399311, 40.53688580056195], [-74.24533651176053, 40.53690599443462], [-74.24534237855728, 40.53691668778189], [-74.24534807468928, 40.536929520860824], [-74.24535239770356, 40.53694090203701], [-74.24535342151941, 40.536955228417256], [-74.24535885752975, 40.536985548238725], [-74.24536014307084, 40.5370029190981], [-74.24535957048438, 40.537013350617215], [-74.2453529824317, 40.537031810498874], [-74.24534240123023, 40.53705948171078], [-74.24529455875826, 40.537118337963975], [-74.24526133913146, 40.53715221311302], [-74.24524202938359, 40.537163910477545], [-74.24522190891928, 40.53717249534176], [-74.24520736973874, 40.53717679792458], [-74.24517157497509, 40.53718541625423], [-74.24508434623642, 40.53721208121983], [-74.24499824648248, 40.537241307432495], [-74.24497924473512, 40.53724988819152], [-74.24496024612203, 40.53725932535773], [-74.24494125711304, 40.53727132454766], [-74.24492227413629, 40.53728503152971], [-74.2449044145952, 40.53729958968117], [-74.2448854472566, 40.53731756865711], [-74.24484083543769, 40.53736378835734], [-74.24480069599326, 40.537409145437984], [-74.24478733358518, 40.53742882025493], [-74.24477620246829, 40.537446780327286], [-74.24468497417264, 40.53760414338847], [-74.2445981785753, 40.53774953657848], [-74.24459372136184, 40.53775552541253], [-74.24458926480835, 40.53776151474761], [-74.24457587413374, 40.537773501949964], [-74.24452673493828, 40.53780691892311], [-74.24451222079375, 40.53781805427171], [-74.24450106630034, 40.537829182463874], [-74.24449660594615, 40.53783431655903], [-74.24449326969713, 40.53784030182943], [-74.2444754911651, 40.53787707096217], [-74.24446107385107, 40.53791468432656], [-74.24445001843112, 40.53795314527303], [-74.24444344211138, 40.537991596682474], [-74.2444413433647, 40.538030039061326], [-74.24444259671773, 40.538066765992895], [-74.24444832584695, 40.53810262865369], [-74.24445852742087, 40.53813677448637], [-74.24447097502853, 40.53817262166843], [-74.24448112799658, 40.538193954854066], [-74.24449238967784, 40.538211868218845], [-74.24449913627772, 40.53821954217152], [-74.24450588133934, 40.53822721545689], [-74.24456877240652, 40.53828004014483], [-74.24459346801181, 40.53829792438101], [-74.24465294390939, 40.538335382005464], [-74.24467763731083, 40.538352412659194], [-74.24470122267977, 40.53837286178785], [-74.24470909749752, 40.53838309537064], [-74.24471697231894, 40.538393329288], [-74.24471810829422, 40.53839759839728], [-74.24471924206769, 40.53840186583577], [-74.24472038116701, 40.53840698800471], [-74.24472039993405, 40.53841211306407], [-74.2447170786649, 40.5384223715143], [-74.2447104398981, 40.53844373929485], [-74.24469936577053, 40.538477075707036], [-74.24469938140453, 40.53848134552875], [-74.2446982773697, 40.538485619914695], [-74.24469942831634, 40.53849415767438], [-74.24470736323296, 40.5385206206661], [-74.24471526089539, 40.53853768760958], [-74.24472651957181, 40.538554747388716], [-74.244742256741, 40.53857179594525], [-74.24476246594462, 40.53858712883374], [-74.244781548778, 40.53860075463676], [-74.24480511077407, 40.538614370886506], [-74.24483866257475, 40.53864321162583], [-74.2449002703978, 40.53869616786689], [-74.24494469027066, 40.53874185506519], [-74.24497182440861, 40.538773199632864], [-74.24500274436116, 40.53881641839824], [-74.24504955187172, 40.53887560728864], [-74.24514012678458, 40.53897886632851], [-74.24518150342622, 40.5390579380785], [-74.24523157838199, 40.53914448521872], [-74.24527100769255, 40.53924187018983], [-74.24533131467827, 40.53940088562488], [-74.24530650812298, 40.539444829489995], [-74.24530622799543, 40.53945693367854], [-74.2453161460591, 40.53947349115843], [-74.24533072729713, 40.53948589463464], [-74.24535392271066, 40.539502423238325], [-74.24537245551109, 40.539523134488796], [-74.24538824752979, 40.53954209603954], [-74.2453961907736, 40.53955953559392], [-74.24539945360863, 40.5395761347652], [-74.24540210010908, 40.53959155449746], [-74.24538703564416, 40.539604185504395], [-74.24536803597358, 40.539622891372765], [-74.24536604154534, 40.53963823926825], [-74.245380975995, 40.539658792853935], [-74.24540252861892, 40.539688693405225], [-74.24541723984652, 40.539717042833814], [-74.2454232990036, 40.53975062598475], [-74.24542947177184, 40.53977561925901], [-74.24543254761085, 40.53980001302665], [-74.24542908073475, 40.53984547469635], [-74.24542566851939, 40.53989601527693], [-74.24542188311162, 40.539972301430694], [-74.24541276701893, 40.54021356393066], [-74.2454296977673, 40.54038576486477], [-74.24545378722227, 40.54051850790558], [-74.24547647854257, 40.540633000791615], [-74.24551910321269, 40.54075760692807], [-74.24559250992388, 40.54092675074652], [-74.24560205698324, 40.54094959457726], [-74.2456077070552, 40.540959766346376], [-74.24562047939826, 40.540969538061745], [-74.24563856476077, 40.54097622371919], [-74.24568077627616, 40.540988485109445], [-74.24570589382073, 40.540999053143075], [-74.2457307294372, 40.5410115993066], [-74.24575714989656, 40.54102444346048], [-74.24578384879587, 40.54103531182544], [-74.24580741667182, 40.541045580761235], [-74.24582363793573, 40.54105394518426], [-74.24583169314688, 40.54106282063416], [-74.24584508121524, 40.54108307675139], [-74.24599284790568, 40.54127893215782], [-74.24604749879236, 40.54136651267769], [-74.24606137187132, 40.54140156236368], [-74.24608152745897, 40.54146237133473], [-74.24610313832198, 40.54153712143066], [-74.24614764964227, 40.54168464315558], [-74.24619797483028, 40.54180914959795], [-74.24621414027527, 40.54184134029696], [-74.24626027017561, 40.54193061177822], [-74.24632094551905, 40.542022103147715], [-74.24635676349392, 40.54206635625875], [-74.24640677149628, 40.54212419161222], [-74.24646660694735, 40.5421927392043], [-74.24648614815786, 40.54221339265563], [-74.2465138936479, 40.542234331050516], [-74.24653391966409, 40.54224013504272], [-74.24657486530842, 40.54225014754582], [-74.24659287581972, 40.54226243527179], [-74.24662220136648, 40.54227783118101], [-74.24664837060041, 40.54229751360104], [-74.24676193481058, 40.542366799685], [-74.24677815425639, 40.54242325599929], [-74.2468625313222, 40.54251010144365], [-74.247677780396, 40.54312540330281], [-74.24792244709678, 40.54304911927136], [-74.24803463735688, 40.54309324044144], [-74.24770573967564, 40.54352432645975], [-74.24761962158414, 40.543509361402414], [-74.24618422383882, 40.54563362585427], [-74.24363807144752, 40.547666195235365], [-74.24360202725677, 40.54763781110733], [-74.24336648769979, 40.547808507519036], [-74.2434020856379, 40.54783551137688], [-74.24336346552776, 40.54786513523582], [-74.2432987867762, 40.5478283445317], [-74.24334063308854, 40.547798715030595], [-74.24335033310243, 40.547803617528395], [-74.24358447651343, 40.547623990878584], [-74.24293585929026, 40.547113219245205], [-74.24266546073754, 40.54701079971151], [-74.24231726803761, 40.54696401276844], [-74.24229800775315, 40.54696132400164], [-74.24222961262026, 40.54695380913411], [-74.2422031846343, 40.54694867830012], [-74.24211830776133, 40.546949845386756], [-74.24204967635752, 40.54695597326009], [-74.24196459767579, 40.54697152299089], [-74.24190290276809, 40.54697568895227], [-74.24185443321967, 40.54698495832039], [-74.24179850544289, 40.54700162778557], [-74.24172117606939, 40.54705051065865], [-74.24169335448103, 40.54706835560555], [-74.24160691885201, 40.54709601318317], [-74.24150964965952, 40.54713321742604], [-74.24145107546943, 40.54716476933668], [-74.24140253504106, 40.5471841400858], [-74.24133944836846, 40.54721128113128], [-74.24128029509164, 40.54723163475298], [-74.24122936796708, 40.54725812060806], [-74.24117275175594, 40.547283738350906], [-74.24112113404507, 40.5473083306699], [-74.24107266988072, 40.54733933846189], [-74.24101667065474, 40.54736610928511], [-74.24097422534163, 40.547380086647735], [-74.24093642651823, 40.54739498856361], [-74.24086739059267, 40.5474290593352], [-74.24082703907891, 40.54745456103449], [-74.24076977132052, 40.54748918353885], [-74.24072363579138, 40.54750723132931], [-74.24063341370834, 40.54754326727555], [-74.24057633072829, 40.54756929849615], [-74.24045847559482, 40.5476254843001], [-74.24039182810051, 40.547663310173775], [-74.24035341572257, 40.547687956758296], [-74.2403130059051, 40.54771774081376], [-74.24027644117172, 40.54774575907879], [-74.24023730218572, 40.54777858665622], [-74.2402039240967, 40.547802152137876], [-74.24012539879432, 40.54785965621242], [-74.24003491309954, 40.547909039731046], [-74.23998281879149, 40.54794023641819], [-74.23996167162653, 40.54795780799162], [-74.23994072994779, 40.54797731796285], [-74.23991253323271, 40.54800475706341], [-74.23987753787303, 40.54802838999228], [-74.23984372828757, 40.54804316962737], [-74.23981864192723, 40.54805578178583], [-74.23979131363885, 40.54807536219499], [-74.23975764471815, 40.54810346588541], [-74.23970065441213, 40.54814472315815], [-74.23962496620842, 40.54821055309667], [-74.23959313085258, 40.54823984422175], [-74.2395420884618, 40.54827292758687], [-74.23949951309932, 40.548293453106666], [-74.23945746357062, 40.548314921883666], [-74.2394274566754, 40.54833336824826], [-74.23939195750437, 40.548360284839184], [-74.23934085340967, 40.54841678532668], [-74.23931414637477, 40.5484542133651], [-74.23930273251601, 40.54848711518694], [-74.23928907288798, 40.54851495422356], [-74.23926751220235, 40.54852864853235], [-74.23924448067316, 40.54852454745696], [-74.23922270854456, 40.54852427333402], [-74.23918633439715, 40.54853498193105], [-74.23915003219383, 40.54855837041808], [-74.23912563695845, 40.54858947669096], [-74.23910193115216, 40.54862704334345], [-74.23906950439633, 40.548678810109344], [-74.23902299983374, 40.548730473317214], [-74.2388802047475, 40.5488586741483], [-74.23876882860974, 40.548941675243746], [-74.23870672648498, 40.54896698971901], [-74.23865414487864, 40.54898163177885], [-74.23858723148467, 40.54899781203674], [-74.23852526847641, 40.54902441728221], [-74.23847241578378, 40.54906054025084], [-74.23841922458453, 40.549105462600856], [-74.23837191304226, 40.549141566814335], [-74.2383301218049, 40.5491854570921], [-74.238281079861, 40.54924931059415], [-74.23822562357557, 40.54930900687669], [-74.23814886177914, 40.54939278761239], [-74.23810087098671, 40.54945852675722], [-74.23801045315936, 40.54959421002949], [-74.2379484587301, 40.54968455469139], [-74.23790310005221, 40.54974297609282], [-74.23781194433836, 40.54989605190748], [-74.23780980173328, 40.549899651549325], [-74.23776472013473, 40.54997268902288], [-74.23771157820656, 40.550038095562854], [-74.23766529763601, 40.550083889823725], [-74.23761963467327, 40.55012346857852], [-74.2375608091404, 40.55016353676152], [-74.23751715838253, 40.550189981109085], [-74.2374649628325, 40.55020427319092], [-74.23740672481237, 40.55020983549643], [-74.23736517465862, 40.55021598850419], [-74.23731918162642, 40.55022046599173], [-74.23726283898992, 40.550215828903355], [-74.23722616127561, 40.550207696016955], [-74.23719811959242, 40.55021658899718], [-74.23717117913401, 40.550235820516654], [-74.23713313596454, 40.55026289226511], [-74.23708755767899, 40.55027124596175], [-74.2370364339829, 40.55026758507707], [-74.23698467413543, 40.550277948790786], [-74.23693199895314, 40.55028771863119], [-74.23688278114177, 40.55029792275851], [-74.2368179273547, 40.55032145500105], [-74.23675493076693, 40.55035039947159], [-74.23669255497087, 40.5503851599568], [-74.23662978347146, 40.55039620493579], [-74.2366031550018, 40.55040096003787], [-74.2365461090008, 40.55041114664288], [-74.23650170138596, 40.550430484847816], [-74.2364594704942, 40.55046627256133], [-74.23641623448965, 40.55050862684339], [-74.2363957993608, 40.55054891677277], [-74.23638071914459, 40.55057944389345], [-74.23635240725108, 40.55060981479951], [-74.2363223550692, 40.55064387176757], [-74.23629187216656, 40.550693890746544], [-74.23628213170987, 40.55074652574997], [-74.23628500667785, 40.5508055334556], [-74.23628945987022, 40.550855337809296], [-74.23626812002584, 40.5509493896076], [-74.23624608904953, 40.55105278884636], [-74.23624167918048, 40.55107244438619], [-74.23622070827703, 40.55115875818671], [-74.2362096640445, 40.551202344526395], [-74.23619753075485, 40.55125447330632], [-74.2361865266174, 40.55130916281124], [-74.23617995866755, 40.55135188520049], [-74.2361767365552, 40.55139118392412], [-74.23617462444652, 40.55142706190519], [-74.23617582954002, 40.55145097725124], [-74.2361803753202, 40.55146975946146], [-74.23618942977693, 40.551497073772744], [-74.23622267338287, 40.55158194053545], [-74.23626103691412, 40.55166554111604], [-74.23630443796945, 40.551747695907174], [-74.2363527833242, 40.55182822840692], [-74.23640007645811, 40.55185870042274], [-74.23644130027343, 40.551893929995714], [-74.23647564009453, 40.551933220918755], [-74.23650241728404, 40.551975796723184], [-74.23652110265431, 40.55202081602275], [-74.23653132692615, 40.55206738914062], [-74.23653288802811, 40.55211459569195], [-74.23652575509128, 40.5521615027716], [-74.23651006906039, 40.55220718339062], [-74.23648613990993, 40.55225073479604], [-74.23645444051985, 40.552291296311246], [-74.23641559733107, 40.55232806634515], [-74.236370377967, 40.552360318234236], [-74.23489865800803, 40.55300521667134], [-74.23488455116241, 40.55300565178594], [-74.23486887015231, 40.55300482961923], [-74.23483637407898, 40.55299891624189], [-74.2347870569584, 40.55298620535422], [-74.23477472348716, 40.55298195904458], [-74.23476014439508, 40.55297600901609], [-74.2347107919558, 40.55295304731452], [-74.23469060684636, 40.55294540260436], [-74.23466818577356, 40.55293861435205], [-74.2346581018197, 40.552936927151684], [-74.23464802086423, 40.55293609351202], [-74.23460434130727, 40.55293447394033], [-74.2345685074059, 40.55293454713789], [-74.23456067074976, 40.5529354172145], [-74.23453828333646, 40.55293802548362], [-74.23453044368168, 40.55293804149403], [-74.23452708414345, 40.552938048354854], [-74.23452372226804, 40.552937201652725], [-74.23452035973143, 40.55293635428167], [-74.23451699485459, 40.55293465234248], [-74.23451362931992, 40.5529329507396], [-74.23451026378798, 40.5529312498067], [-74.2344911722997, 40.55291591334802], [-74.23448443889805, 40.55291165674524], [-74.23442608142095, 40.55287590057867], [-74.23437110451499, 40.55284611731526], [-74.23435091567323, 40.55283676189962], [-74.23432737110322, 40.55282912280845], [-74.23430159423013, 40.55282319607221], [-74.23422650945938, 40.55280711922143], [-74.23420521117521, 40.552801183831754], [-74.23418615061145, 40.55279438913461], [-74.2341502629073, 40.55277908804181], [-74.23413904045152, 40.55277227685298], [-74.23413454837392, 40.55276886939518], [-74.23413117753192, 40.55276545998652], [-74.23412668245935, 40.552761197626616], [-74.23412330862575, 40.55275693415352], [-74.23411203164893, 40.552734748803395], [-74.23410865482435, 40.55272963109826], [-74.23410528033038, 40.55272536645317], [-74.23407943768392, 40.552700648245334], [-74.23402778170795, 40.55265975362766], [-74.23398398867991, 40.55262567536108], [-74.23394020898154, 40.55259501500707], [-74.23389532026982, 40.55256777284106], [-74.23384932109046, 40.55254309563204], [-74.23382688696724, 40.55253289163134], [-74.23380333579374, 40.55252268990046], [-74.23377755011934, 40.552514200518914], [-74.23374952550594, 40.55250571568869], [-74.23368676337617, 40.55249132221388], [-74.23360048067302, 40.552475268745255], [-74.23356462611525, 40.55246936234264], [-74.23353549801199, 40.55246600479625], [-74.23350973947808, 40.552465202434355], [-74.23348622564309, 40.55246610498369], [-74.23345711931373, 40.5524687262001], [-74.23342913886752, 40.55247305510378], [-74.23340452091124, 40.552478229729395], [-74.23338102628746, 40.55248425680714], [-74.23335977547116, 40.55249198712617], [-74.23334188482498, 40.55249971061369], [-74.23332400080318, 40.55250914356582], [-74.2333094786134, 40.55251942275053], [-74.23318551828129, 40.55261704975135], [-74.23299007634827, 40.55276863545614], [-74.23295545476876, 40.552795184380805], [-74.23291523611293, 40.552822598884404], [-74.23281579691063, 40.552887718230636], [-74.23281020734505, 40.55289029210781], [-74.23280350017373, 40.55289372131527], [-74.2327889597488, 40.552898876551694], [-74.23273637215114, 40.5529117953626], [-74.23271847305598, 40.55291695672651], [-74.23270282250361, 40.5529246767473], [-74.23269611466445, 40.552928105949704], [-74.23269165371474, 40.55293324007988], [-74.23264143423904, 40.55298288476353], [-74.23255553103091, 40.55307616295529], [-74.23239598706299, 40.55324646599035], [-74.23229112933166, 40.55336369797566], [-74.2322665927849, 40.55339279000899], [-74.23224430607304, 40.553424438860034], [-74.23222314817686, 40.55345864863974], [-74.23220311678335, 40.553494564445494], [-74.23218421497464, 40.55353389474701], [-74.23216532632797, 40.553576640126565], [-74.23214645415379, 40.55362451273828], [-74.23212315508638, 40.55368691449199], [-74.2320998330062, 40.55374333628832], [-74.2320603723538, 40.55380877735871], [-74.23200263530478, 40.553896012849336], [-74.23196648230734, 40.55396419202273], [-74.2319554487189, 40.55402152010054], [-74.23195537304143, 40.554078235835654], [-74.23193331444939, 40.55414983011715], [-74.23188707136137, 40.55421332002535], [-74.2318159763953, 40.554276094802326], [-74.2316830032468, 40.55441163222109], [-74.23156232580645, 40.55453177913723], [-74.23150063118818, 40.554588938073046], [-74.23135620243775, 40.55474173339223], [-74.23121793211067, 40.55486078404755], [-74.23114456149608, 40.554945800970614], [-74.23106943792601, 40.55504383345953], [-74.23098961854346, 40.55514725306872], [-74.23087472791508, 40.55535130375602], [-74.23084007911194, 40.55544809858915], [-74.23078781040344, 40.55550824217585], [-74.23075511246024, 40.55553665854542], [-74.2307103619811, 40.55555759425892], [-74.23065576101537, 40.55557521420864], [-74.23060876282467, 40.55557864294688], [-74.23055514386452, 40.555563741384155], [-74.23050804631487, 40.55553798799348], [-74.23045979656835, 40.55549555823949], [-74.23042469847942, 40.55546227587576], [-74.23034473534743, 40.55541657366522], [-74.23024952153503, 40.55538674616261], [-74.23017519392035, 40.555390231187246], [-74.23004306465734, 40.55543468860076], [-74.2299262674491, 40.55548620505377], [-74.22978862412833, 40.555516498968814], [-74.22968040863911, 40.55552088426538], [-74.22960496893391, 40.555518532877954], [-74.22947149429736, 40.55548961568595], [-74.22937963425443, 40.55548145986459], [-74.22931954185714, 40.55549158696269], [-74.2292737490792, 40.55552753391468], [-74.22924873918484, 40.555566774391956], [-74.22924779391403, 40.555610136159096], [-74.22925226143867, 40.55563764343743], [-74.22926986460222, 40.555670128961175], [-74.22928356407323, 40.55567348226014], [-74.22929656579723, 40.5556781815328], [-74.229308643603, 40.555684145033204], [-74.22931958739098, 40.55569126902325], [-74.22933099923318, 40.55570121788776], [-74.22934020126353, 40.555712419699404], [-74.2293469612076, 40.55572459170768], [-74.22935110843282, 40.555737426672664], [-74.22935253825536, 40.55575060062033], [-74.22933656131292, 40.55578528386582], [-74.22931328089027, 40.555817472830036], [-74.22928333871313, 40.55584628020908], [-74.22924756014524, 40.55587091191282], [-74.22923558767849, 40.55587679894113], [-74.22922255551687, 40.55588118553646], [-74.22920878432424, 40.55588396376424], [-74.2291946129487, 40.55588506526435], [-74.22918039008535, 40.55588446293385], [-74.22916646569585, 40.55588217159345], [-74.22915318239791, 40.55587824762296], [-74.22914086703469, 40.55587278757387], [-74.22910541631074, 40.55591556547958], [-74.22909039712206, 40.55593690078144], [-74.2290729956406, 40.55596361839081], [-74.22907639765481, 40.555999466625074], [-74.2290830609419, 40.55602947176305], [-74.22907443408064, 40.55606451002411], [-74.22905162187638, 40.55610708210507], [-74.22899926815344, 40.5561422082308], [-74.22894905874807, 40.556165655703246], [-74.22890104752673, 40.55619243379969], [-74.2288159888303, 40.5562551428902], [-74.2287242997478, 40.5562970171392], [-74.22862814038024, 40.55631055005351], [-74.22854505831708, 40.55631154871156], [-74.22848274556276, 40.55631250716475], [-74.22846525096287, 40.556311707499106], [-74.2284412209779, 40.55631759239245], [-74.22841287834882, 40.55634183009893], [-74.22839545566364, 40.55636187675416], [-74.22836927678098, 40.55637944009232], [-74.22833214371997, 40.55639035431146], [-74.228267637297, 40.556388814775374], [-74.22821296402216, 40.55638558739673], [-74.22816154992299, 40.55637651692663], [-74.22810792998999, 40.55636161422416], [-74.22807285538782, 40.55633500148823], [-74.22804538468095, 40.55629419722321], [-74.22801360696909, 40.55627258121726], [-74.22797312564695, 40.556263488956944], [-74.2278330723504, 40.55622874465361], [-74.22770173052456, 40.55618397916714], [-74.227632728344, 40.556146591495], [-74.22755928210492, 40.55608836862556], [-74.2274368273054, 40.55603853351635], [-74.22724124339234, 40.5560354117492], [-74.22708256217496, 40.55605853939567], [-74.22692497792441, 40.556101244097164], [-74.2269023090303, 40.556107386824], [-74.22687006826537, 40.55611078705189], [-74.2268383534228, 40.55610709751937], [-74.2267957140918, 40.55610634675829], [-74.22676949190152, 40.556110985686836], [-74.22673731479559, 40.55613314664004], [-74.2266734121504, 40.55614828082246], [-74.2266100107086, 40.556150073704586], [-74.22657942850441, 40.55615847286758], [-74.22657299342214, 40.556195174378374], [-74.22653700730443, 40.5562306798821], [-74.22647661167676, 40.556253862064366], [-74.22640494542956, 40.556255712845854], [-74.22637020813472, 40.5562498014911], [-74.22633201675346, 40.55621570978148], [-74.22618692147473, 40.55622908896703], [-74.22611420492586, 40.55625236709394], [-74.22606854510708, 40.55621187928375], [-74.2239033413449, 40.55589706554456], [-74.22386421498082, 40.55609769408288], [-74.22344265106311, 40.556046569903536], [-74.22347210933358, 40.55594586956392], [-74.22371910444984, 40.55598110029811], [-74.22375741210328, 40.555981025669496], [-74.22377438375146, 40.555964760418526], [-74.22378606709124, 40.555880268009716], [-74.2231947052717, 40.55579556099059], [-74.2231612894273, 40.555787142691706], [-74.22312986736561, 40.55576755796203], [-74.2231129667092, 40.55573684009674], [-74.22310353721308, 40.55568306927582], [-74.22309672573029, 40.55563694213429], [-74.22304228365793, 40.555595792857716], [-74.2230015599747, 40.55557561902706], [-74.22297119575062, 40.55553724065462], [-74.22297219698756, 40.55550337362143], [-74.22288310030724, 40.55547286003058], [-74.22281876589037, 40.55545396330237], [-74.22273960124298, 40.55543418896245], [-74.22265568834065, 40.555415330615595], [-74.22257486801371, 40.555406209779626], [-74.22251833048998, 40.55542682203717], [-74.22247482113684, 40.55545705660065], [-74.22242225993624, 40.555479367739565], [-74.22237077098863, 40.555487154576824], [-74.22232036636152, 40.555483835379874], [-74.22224752905595, 40.555470309396846], [-74.22222452335853, 40.555466121930344], [-74.22221727566942, 40.55546524335242], [-74.22219699760915, 40.55544589044549], [-74.2221224448784, 40.55541688471408], [-74.22201112709658, 40.55540271596946], [-74.22190184151718, 40.55539337577119], [-74.22179863887058, 40.555417070759], [-74.22174872671057, 40.55543934531281], [-74.22169675758535, 40.55550307048973], [-74.22166724221147, 40.55555519773824], [-74.22165396875167, 40.55558920370556], [-74.22160298618641, 40.55563621042137], [-74.22158844541158, 40.555646202680364], [-74.22157006613187, 40.55564898295767], [-74.22154450180608, 40.555634209189904], [-74.22149876696065, 40.555592055800304], [-74.22149377308864, 40.555562037124], [-74.22147280956294, 40.5555418487652], [-74.22145260819535, 40.55552907550155], [-74.22145469372055, 40.5554820906086], [-74.22146243021916, 40.5554513271694], [-74.22145446439751, 40.555412903527994], [-74.2214319532539, 40.555378779452894], [-74.22140054953707, 40.55536432047673], [-74.22134677624109, 40.55535844480241], [-74.22129864604456, 40.555366224685564], [-74.22125504247906, 40.55538766395928], [-74.22121375057472, 40.55543130541423], [-74.22120713936611, 40.55546377614885], [-74.22120284421428, 40.55551930599033], [-74.22119742494193, 40.55557398275408], [-74.22119312650432, 40.55562865735847], [-74.2211832282346, 40.55568334392007], [-74.22115976704038, 40.555700472841316], [-74.22109369280587, 40.55569974582382], [-74.22105338275364, 40.55570153122284], [-74.2210399857696, 40.555714369896684], [-74.22102448950054, 40.55576906654784], [-74.22101010859353, 40.555822907483076], [-74.22099013412594, 40.555878466984055], [-74.22097904527034, 40.55591180005441], [-74.2209432973211, 40.55593834800795], [-74.22086715093401, 40.55594020227767], [-74.22082231656412, 40.555928331013625], [-74.22076962266105, 40.55591049434238], [-74.22069567868726, 40.555901241526286], [-74.22062175949124, 40.55589882100512], [-74.2205534015105, 40.555885284170316], [-74.22052870386395, 40.55586739479367], [-74.22049941794982, 40.55581620030979], [-74.22046901559654, 40.55576586387414], [-74.22045714043932, 40.555748289115115], [-74.22044538227212, 40.555730887524305], [-74.22042290601576, 40.5557070145355], [-74.22041823823064, 40.555649794625886], [-74.22040587158548, 40.55563529615935], [-74.22033301478737, 40.555615790679816], [-74.22029711808946, 40.55559792106273], [-74.22028585305702, 40.55557744382515], [-74.22027782854286, 40.55552108391514], [-74.2202665585134, 40.55549975144269], [-74.22026655569555, 40.55549889670825], [-74.22021943697257, 40.55547336145939], [-74.22018456182457, 40.55542559489575], [-74.22017880896205, 40.55537862790275], [-74.2201831028014, 40.55532224436833], [-74.22018179076905, 40.55526416277169], [-74.22017936854516, 40.555208646349904], [-74.22016238732648, 40.55515315834145], [-74.22014316776007, 40.555097674118095], [-74.22013522631465, 40.555066939144595], [-74.22014369883041, 40.555060135076836], [-74.22014474318152, 40.55500592293347], [-74.22010627165115, 40.55492881342842], [-74.22005143156746, 40.554841907273676], [-74.21999334606815, 40.554764503767444], [-74.21992072431505, 40.55469843787539], [-74.21986518402704, 40.55465078550191], [-74.21980229197143, 40.55462156378432], [-74.21974356930723, 40.55461267207239], [-74.21967214720144, 40.55462683585811], [-74.21960026741088, 40.5546547233884], [-74.21956112618452, 40.5547021463243], [-74.21952987169979, 40.554718646998836], [-74.21948446867218, 40.55472216480092], [-74.21940883619042, 40.55472532972463], [-74.21932321285607, 40.55475453231532], [-74.21925552903423, 40.55474546698497], [-74.21917586317076, 40.55475879414019], [-74.21911014911431, 40.5548012990159], [-74.21908060899351, 40.55484700219797], [-74.21909005126736, 40.55492806552364], [-74.21912835054238, 40.55497470979937], [-74.21917339795966, 40.55507143309305], [-74.2192345849937, 40.555118169685926], [-74.21925824504427, 40.555156195118755], [-74.21923330127086, 40.55526452631137], [-74.21919554653131, 40.555394647826155], [-74.21920978196957, 40.5554555836068], [-74.2192522371676, 40.55547460705378], [-74.21929510878144, 40.555489263849346], [-74.21931015500938, 40.55551218225154], [-74.21932052662625, 40.55561374815316], [-74.21933701278975, 40.55569136698107], [-74.21930539588978, 40.55575085204573], [-74.21928639639636, 40.55579538237191], [-74.21921716370927, 40.55579696987855], [-74.21919443458171, 40.555779445862484], [-74.21918232634566, 40.55575265322801], [-74.2191796802851, 40.55570297648615], [-74.21918136219827, 40.555656201653825], [-74.21918905441791, 40.55556555305959], [-74.21917849469084, 40.55552810538726], [-74.21916939228603, 40.55551736362476], [-74.21913632951241, 40.55553148016613], [-74.21909202087238, 40.55552871065521], [-74.21907546885762, 40.55550714053318], [-74.21907068516887, 40.555442015704514], [-74.21906787202626, 40.55535253892671], [-74.21904407246122, 40.55522799703139], [-74.21899014747787, 40.55504707931044], [-74.2189583263338, 40.55485758391833], [-74.21891390920673, 40.55477765609848], [-74.21886973476656, 40.55473879183282], [-74.21878077485502, 40.55467228928319], [-74.21859325782798, 40.55467421024778], [-74.2185398175649, 40.55470738813924], [-74.21853272333956, 40.55473888410936], [-74.21856501731422, 40.55482004237239], [-74.21855087085385, 40.55487364750933], [-74.21853638146906, 40.55493288147892], [-74.2185676824065, 40.55504025342726], [-74.21863889566342, 40.555270078066606], [-74.21870070984441, 40.55547612471538], [-74.21875918633687, 40.5556624132944], [-74.2187974474673, 40.55581298368631], [-74.21881905965172, 40.555874718986104], [-74.21875399923744, 40.55589346978803], [-74.2188020251887, 40.55595546554497], [-74.2187503688055, 40.556027740537246], [-74.21870767560165, 40.55604101620115], [-74.21867044328168, 40.55605041233876], [-74.21865679372745, 40.55603315036935], [-74.21841519874965, 40.555727626154116], [-74.21826999195912, 40.55569054965283], [-74.21817360193401, 40.55560052324848], [-74.2181215645945, 40.55551041370758], [-74.21812855546021, 40.5554652717783], [-74.21807146383063, 40.555292928825175], [-74.2179149889178, 40.55509504347546], [-74.21775342391726, 40.55500528477618], [-74.21741272772677, 40.55509386541668], [-74.21719072566007, 40.55515908944039], [-74.21697226575563, 40.55523093577839], [-74.2167576874083, 40.55530929281532], [-74.2165473239861, 40.555394038821106], [-74.21624183551103, 40.55548031523552], [-74.21604246060625, 40.55554411878309], [-74.21585156387714, 40.55565864897368], [-74.21556824641291, 40.555813098924425], [-74.21474053198814, 40.556242389132144], [-74.21491217588893, 40.556523927255625], [-74.21497052999892, 40.55665149155003], [-74.21501817201802, 40.55675563837995], [-74.21499256964766, 40.5567949327959], [-74.21462340478293, 40.55691336222088], [-74.21458359755573, 40.5568991652], [-74.21449357509393, 40.55672429854295], [-74.21447533239655, 40.55668886410159], [-74.21443562607354, 40.556700305678405], [-74.21434447631304, 40.55672657135454], [-74.21428212817725, 40.556636752072876], [-74.21422698854253, 40.556557317917004], [-74.21416613986864, 40.55658872941318], [-74.21412277605752, 40.55661973284424], [-74.21405771378673, 40.55663223210392], [-74.21401027808201, 40.55663081139325], [-74.21396934069192, 40.5566583358338], [-74.21395537449884, 40.55665715529948], [-74.21386874413025, 40.55658729585716], [-74.21386359366392, 40.55657448653227], [-74.2139008816071, 40.55654216549755], [-74.2138875548331, 40.55651381386743], [-74.21386858605855, 40.5564961264968], [-74.21384686609589, 40.556487670642085], [-74.2138369925384, 40.556471221300185], [-74.21381634554184, 40.556429263412554], [-74.21379034458955, 40.556401288352674], [-74.21377719790834, 40.556399062054915], [-74.2137646731758, 40.5563969433253], [-74.21373503614481, 40.55641330320351], [-74.213711569401, 40.55643500351552], [-74.21366207475845, 40.55648719216827], [-74.2136290446141, 40.556511819753105], [-74.21359594595765, 40.55652587997942], [-74.21356439624225, 40.55652928774992], [-74.21352072182457, 40.55653389796987], [-74.21347392300072, 40.55653991369645], [-74.21339560620757, 40.55655990370746], [-74.21333332352269, 40.556585957964494], [-74.21327457291389, 40.556624934276535], [-74.21324031404319, 40.55667262386913], [-74.2056960519999, 40.55851600599987], [-74.20699885299997, 40.55756748899989], [-74.20880330699998, 40.557308688999896], [-74.21050789400002, 40.557064181999934], [-74.21220034099994, 40.556443429999874]]]}}, {\"id\": \"44\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 45, \"Shape_Leng\": 0.0459068626235, \"Shape_Area\": 9.13594688556e-05, \"zone\": \"Chinatown\", \"LocationID\": 45, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99750445299988, 40.71406913199995], [-73.99744448599988, 40.71406687299986], [-73.99647830999992, 40.71410726399991], [-73.99484048899991, 40.71420935199992], [-73.99472479499984, 40.71421571299987], [-73.99465684999993, 40.71421944699987], [-73.99450033499993, 40.71423116399996], [-73.99434786700003, 40.71424791599985], [-73.99431855699987, 40.71425088499997], [-73.99414039099992, 40.714268931999854], [-73.99395033499991, 40.71429083199993], [-73.99301588399996, 40.714355624999875], [-73.99282173499992, 40.71436522999989], [-73.99256242199989, 40.71438807699996], [-73.99246827699984, 40.71385682499988], [-73.9924556359999, 40.713762881999834], [-73.99239157000001, 40.71319080999998], [-73.99226747499985, 40.712467556999904], [-73.99225369399991, 40.712354325999854], [-73.99224319399995, 40.71226805499987], [-73.99221855999997, 40.71206565499991], [-73.99220577900002, 40.71196063499989], [-73.99218322399993, 40.71177531099993], [-73.99216806999992, 40.71171189899992], [-73.99215484499997, 40.71165655599985], [-73.99213550299994, 40.71157561499989], [-73.99202215999982, 40.71110129999989], [-73.99200166199995, 40.711015522999894], [-73.99197674899987, 40.71091126599985], [-73.99191690599989, 40.71059750599987], [-73.99177662699996, 40.7098190399999], [-73.99183817499984, 40.709810551999965], [-73.9919070979999, 40.70980199399986], [-73.99204746299989, 40.70977574499992], [-73.99200595899987, 40.70969647899992], [-73.99197509100001, 40.70964553499987], [-73.9919434659999, 40.70958905299992], [-73.99192047499996, 40.70955429399988], [-73.99190824271031, 40.70953292664334], [-73.99203926968342, 40.70951691916274], [-73.99209131635959, 40.70950822240118], [-73.9921486897014, 40.70949860910412], [-73.99236116139895, 40.70946436881142], [-73.99284846748641, 40.70938187763877], [-73.99374012862212, 40.709246120435395], [-73.99400820164024, 40.709237479918954], [-73.99414063200335, 40.709233267399895], [-73.9943782354393, 40.709192639198946], [-73.9945789996373, 40.709130972909996], [-73.99517577369818, 40.70903429381024], [-73.99580385653569, 40.70894601390752], [-73.99601898395763, 40.70891622384743], [-73.99666400996595, 40.70882690451765], [-73.99728287195568, 40.70873161984285], [-73.99808351186982, 40.70850961536707], [-73.99828602320575, 40.70842993146761], [-73.9985275858529, 40.708331527908335], [-73.9985611462062, 40.70831528323353], [-73.99872197955054, 40.708237452896164], [-73.99875396570859, 40.708223651529806], [-73.99880846276638, 40.70820013499265], [-73.99882158420985, 40.70817548117483], [-73.99888130863876, 40.70813663391978], [-73.99898340963244, 40.708092653316136], [-73.99905276738558, 40.70803254536376], [-73.99913434918955, 40.70798239744848], [-73.99919451154429, 40.70794737667571], [-73.99928731099999, 40.708016602999926], [-73.99936195599989, 40.70807228699991], [-73.99939955100004, 40.70810033799996], [-73.99944281399992, 40.70813655699987], [-73.99956273399984, 40.70823648299992], [-73.99981193899987, 40.70844261499991], [-73.9999293639999, 40.70853681799988], [-74.00000765599991, 40.70859961999991], [-74.000123591, 40.708684518999924], [-74.00019103100001, 40.708733912999875], [-74.00026566699995, 40.708792948999864], [-74.00055004999997, 40.70901791699988], [-74.00134439899988, 40.70964629299989], [-74.00147140299997, 40.709746553999864], [-74.00155908199999, 40.70981710799992], [-74.00173688999998, 40.70996016999991], [-74.00223869799997, 40.71036393199988], [-74.0025796699999, 40.71063789699985], [-74.0033244169999, 40.711237482999884], [-74.00336407499992, 40.71126941399996], [-74.00366118299998, 40.71148728299994], [-74.00382358799995, 40.71155186199991], [-74.00418143899992, 40.7116871419999], [-74.00442860399994, 40.7117862259999], [-74.00490674599992, 40.711972278999944], [-74.00499805099996, 40.71201189699987], [-74.00510989999995, 40.71205559399987], [-74.00518548699988, 40.71207123299992], [-74.00522812799994, 40.71207345199985], [-74.00538198399992, 40.712076742999876], [-74.00543986599993, 40.71207853299993], [-74.00573499299995, 40.71197868799989], [-74.0058917509999, 40.71200715999989], [-74.00574044699991, 40.71208875699989], [-74.00558932999998, 40.7121700539999], [-74.00543047899994, 40.712203694999914], [-74.00527561999998, 40.71224943999993], [-74.00512712599996, 40.712306772999966], [-74.00498716699994, 40.712374856999894], [-74.00485768099996, 40.7124525409999], [-74.00470652299995, 40.712574014999866], [-74.00429113899997, 40.71290782599993], [-74.00421405399993, 40.712980742999896], [-74.00405900799997, 40.713134789999884], [-74.00360280399998, 40.71364542999988], [-74.00321983099997, 40.71403623699988], [-74.00272286199994, 40.71438970899991], [-74.002453932, 40.71478981999997], [-74.00211455799992, 40.715158909999936], [-74.00168306199991, 40.71574412299987], [-74.00081842499998, 40.71692970799994], [-74.00027443699997, 40.71761940199987], [-73.99995620900002, 40.71801709499987], [-73.99931241699989, 40.71755024199989], [-73.99858925599978, 40.7170995119999], [-73.99776285999997, 40.71681392799996], [-73.99698375299992, 40.716540542999866], [-73.99605872699998, 40.7162316399999], [-73.99614929199988, 40.716084840999955], [-73.99622493199993, 40.71599579099993], [-73.99653128199989, 40.71549139099988], [-73.99680607099994, 40.7150369559999], [-73.99709223399998, 40.71461839799991], [-73.99750445299988, 40.71406913199995]]]}}, {\"id\": \"45\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 46, \"Shape_Leng\": 0.134475429879, \"Shape_Area\": 0.000926391677672, \"zone\": \"City Island\", \"LocationID\": 46, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.78452431937963, 40.86047706314786], [-73.7844902538973, 40.86041295004045], [-73.78448776047776, 40.860368910954875], [-73.7844800977429, 40.860292836382015], [-73.78447503221116, 40.86022877642979], [-73.78446994161976, 40.86017272439544], [-73.78446215634142, 40.86013467913451], [-73.78442797497668, 40.86010659359294], [-73.78437534588969, 40.86007647132326], [-73.78435963136363, 40.86004441729388], [-73.78436247019211, 40.859982373511805], [-73.78436267137351, 40.859920325971984], [-73.78436807918104, 40.859878303052135], [-73.78438676719428, 40.859806281694055], [-73.78440007895897, 40.859766274445036], [-73.78442391602212, 40.85973229342179], [-73.78449516071953, 40.85971241045506], [-73.7845715637229, 40.85971548228985], [-73.78456904816244, 40.85969053097361], [-73.78455594466404, 40.85966648813517], [-73.78454815872834, 40.859628444550694], [-73.78455623644251, 40.85957641773658], [-73.78456430743651, 40.85952639348691], [-73.78460404682814, 40.85946642116824], [-73.78462528561683, 40.85942042497647], [-73.78460172751247, 40.85936834077295], [-73.78461240880978, 40.85932632718043], [-73.78464412229502, 40.859302366834974], [-73.7846758996747, 40.85925839170871], [-73.78469969172261, 40.85923842071494], [-73.78470241202126, 40.85921240584557], [-73.7847051705325, 40.85917438144233], [-73.78472896902186, 40.85915240837877], [-73.78476590334343, 40.859144471496926], [-73.78480288859096, 40.859120520439255], [-73.78483197791422, 40.85909255283886], [-73.78487654337043, 40.85905832709181], [-73.7849032932319, 40.8590506527833], [-73.78492443458967, 40.85903467961179], [-73.78494557725956, 40.85901870694156], [-73.78497195857572, 40.85901275122567], [-73.78500630915828, 40.858988796867486], [-73.785045897498, 40.85897485945254], [-73.78510392836098, 40.85896496035969], [-73.78513035557019, 40.858944993514605], [-73.78515408763242, 40.85894303630437], [-73.78517519142918, 40.858939072119874], [-73.78518844391954, 40.85891707932429], [-73.78518853442692, 40.858889057638116], [-73.78519126744986, 40.85885903946629], [-73.78519664387113, 40.85882702363741], [-73.78523623919948, 40.858811084764405], [-73.78527841239523, 40.85881316583793], [-73.78529680189469, 40.8588332149537], [-73.78530197795772, 40.858863247846145], [-73.7852676088167, 40.85889320831733], [-73.78525697970875, 40.85891920831323], [-73.78525422152342, 40.858957232731484], [-73.78525939602868, 40.858987266795005], [-73.78527521479502, 40.85898729623645], [-73.78529633019848, 40.858979328737995], [-73.78530432970666, 40.85895132227248], [-73.78533335414066, 40.85894336998699], [-73.78534653595938, 40.858943394512316], [-73.78537016554837, 40.85897346173218], [-73.78535426993068, 40.85899745053398], [-73.78535158219417, 40.85901345811734], [-73.78535416011117, 40.859031477572664], [-73.78538834843641, 40.85905756095048], [-73.7854147107948, 40.85905760998464], [-73.78543056633485, 40.859045630873105], [-73.78545964311272, 40.85902166539953], [-73.78548075336799, 40.859015700938436], [-73.78549921267165, 40.859013732680594], [-73.78552292047279, 40.85901978164125], [-73.78553605067312, 40.85903581812991], [-73.78555172614126, 40.85907988170428], [-73.78555685070432, 40.85912592824412], [-73.78556732396562, 40.85914796467686], [-73.78558046666774, 40.859159997702506], [-73.78559628481331, 40.859160027098696], [-73.78562002983944, 40.859154067508435], [-73.78564115948065, 40.859142096995235], [-73.78566224996798, 40.85914213617787], [-73.78568065813838, 40.85915618155187], [-73.78569113232625, 40.85917821847748], [-73.78569372346445, 40.859192233965686], [-73.78572792401741, 40.85921431495624], [-73.7857726886443, 40.859230410163605], [-73.78582168503188, 40.859254132521045], [-73.78588066129579, 40.85926663873898], [-73.78594123750362, 40.859284765801405], [-73.78601763836582, 40.85930092010696], [-73.78608881774542, 40.8593010520985], [-73.78616528302004, 40.85929718985976], [-73.78622853422603, 40.85930331195699], [-73.78629177967478, 40.859311435413964], [-73.78636034162425, 40.85930555750373], [-73.78642632600138, 40.8592816612963], [-73.78646327947388, 40.859267718518424], [-73.78650814623157, 40.85925178946896], [-73.78657148225405, 40.859231890585875], [-73.78662961453665, 40.85918996507849], [-73.78666931246264, 40.859142000739304], [-73.7867301085431, 40.859092074819856], [-73.78680410917073, 40.85903416597232], [-73.78683848410708, 40.85900220365069], [-73.78687017009233, 40.85898625010733], [-73.7868993290349, 40.85893626464586], [-73.78695485110823, 40.85888632786039], [-73.78700771825798, 40.858842391033136], [-73.78705987145723, 40.85882528948738], [-73.78706841636694, 40.85882248700689], [-73.78710536878255, 40.85880854569622], [-73.78716072383345, 40.858810648729474], [-73.78722398107531, 40.85881476771513], [-73.78726083742153, 40.85883084826379], [-73.78727504379454, 40.858854636519254], [-73.78729498157858, 40.85887094284297], [-73.78733185055222, 40.85888302058116], [-73.78736608944581, 40.85889309138494], [-73.78740560244165, 40.858903172064394], [-73.78743186194176, 40.858935245614425], [-73.7874158916261, 40.85898325360335], [-73.78736038283725, 40.8590291876556], [-73.78732864420107, 40.85906115381899], [-73.78732060778712, 40.85910117062934], [-73.7872941173816, 40.859141152748556], [-73.78726224420458, 40.85921515284668], [-73.78723311169144, 40.85925713098204], [-73.78718675246938, 40.85931159051636], [-73.7871854793933, 40.85931308733737], [-73.78713520321362, 40.85937104019999], [-73.78707963601639, 40.859434987659185], [-73.7870320483265, 40.85947693335828], [-73.78697913779823, 40.859534880786455], [-73.78698691781936, 40.85957492610177], [-73.78702108009922, 40.8596090159042], [-73.78706307596636, 40.85966713902589], [-73.78708149186802, 40.8596791827834], [-73.78710521132136, 40.85968122795866], [-73.78711182162283, 40.85969939972018], [-73.787107726334, 40.85971926164171], [-73.78709446746645, 40.859743257219804], [-73.78707858510543, 40.85976324229673], [-73.78705743121485, 40.859783219317336], [-73.7870415879457, 40.85979119636581], [-73.78702051604037, 40.859785152584585], [-73.78700212067199, 40.85976710514278], [-73.78698106160043, 40.859757058065334], [-73.78694940091926, 40.859765006399016], [-73.78692554523269, 40.85980499329491], [-73.78692012622042, 40.85985101979181], [-73.78691471140883, 40.85989504422061], [-73.7868987212506, 40.859949056309404], [-73.78689858015443, 40.8599930904772], [-73.78685886773299, 40.86004505783956], [-73.78679816291358, 40.86006696264338], [-73.78672426294726, 40.86009284700723], [-73.78663190867239, 40.86011869602478], [-73.7864973984128, 40.86013646185003], [-73.78640247859543, 40.8601402896442], [-73.78627850088282, 40.86016207752269], [-73.7861808787995, 40.860185915016395], [-73.78610698578403, 40.8602097964202], [-73.78604370086556, 40.86021368306415], [-73.78595670855401, 40.8602115203072], [-73.78586179482426, 40.86021334559278], [-73.78576157641793, 40.860225169315676], [-73.78567185124342, 40.860253023349415], [-73.78558478790842, 40.86027287764533], [-73.7854818874012, 40.860298706813445], [-73.78538694703576, 40.860308537949194], [-73.78530786449265, 40.860306389415996], [-73.78522080164107, 40.860326243438216], [-73.7851521214915, 40.86036814741715], [-73.78512562839124, 40.8604081302005], [-73.78510175844113, 40.86045212017063], [-73.78503841008691, 40.86047602052986], [-73.78499095625652, 40.86047593210825], [-73.78494355332282, 40.860459831182176], [-73.78491727455626, 40.86043376292765], [-73.78489093019402, 40.86042770842686], [-73.78483815862118, 40.86044162120279], [-73.7847879909767, 40.86046554650663], [-73.78473518651981, 40.86048946636658], [-73.7846771354921, 40.86050537012106], [-73.78463757790949, 40.860509299770094], [-73.7845914939652, 40.8605097253584], [-73.78455587045978, 40.86050314234216], [-73.78452431937963, 40.86047706314786]]], [[[-73.76783205637173, 40.854442205742075], [-73.7682312652444, 40.85388788880734], [-73.7685566032706, 40.85323832295944], [-73.7690443898714, 40.85329092469354], [-73.76938529139694, 40.85253613399675], [-73.76860817687106, 40.85078839537812], [-73.76959674311465, 40.84759451909819], [-73.76950321562752, 40.8472178342351], [-73.76805636086223, 40.84559821813746], [-73.76789492672636, 40.84528943351961], [-73.76901389181107, 40.84509734272193], [-73.76998042657765, 40.84595141286952], [-73.77118378010564, 40.84694451331219], [-73.77157218833237, 40.8475000585803], [-73.77130772937919, 40.84802877432174], [-73.7708892022117, 40.84975177133807], [-73.7720496366846, 40.85161828975743], [-73.77246892938182, 40.8517091473893], [-73.77246651950136, 40.85180956889385], [-73.7727111749615, 40.85182658093368], [-73.77270912061607, 40.851847663461555], [-73.77246594126663, 40.85183368399969], [-73.77246052295433, 40.8520595421737], [-73.77264417003487, 40.852138980076155], [-73.77267130532901, 40.852098888782244], [-73.77272072829567, 40.85211948523007], [-73.77271845211186, 40.85212716875465], [-73.77291739323162, 40.85217453792541], [-73.77292081162744, 40.85216173175276], [-73.77296356611284, 40.8521592539586], [-73.77296793744162, 40.8521968453855], [-73.772945391628, 40.85221046694682], [-73.77287681617626, 40.85219837376392], [-73.77287683950732, 40.85219154117822], [-73.77270601491698, 40.85214593507688], [-73.77264837746922, 40.85222440262956], [-73.772904294762, 40.852387192161146], [-73.77289075731291, 40.85239826899822], [-73.7726393580297, 40.852230362275975], [-73.77263258309878, 40.85223803743507], [-73.77258429583127, 40.852214025867795], [-73.77261479509698, 40.85217650433531], [-73.77245940163019, 40.85210626708215], [-73.77245149415468, 40.85243589689344], [-73.77173060411904, 40.852476831319095], [-73.7717171593653, 40.853004539281784], [-73.77229680531372, 40.85308637825238], [-73.77227577057599, 40.85331813153632], [-73.77227536313346, 40.853437135363876], [-73.77229293607876, 40.85347977623898], [-73.77231266873179, 40.853527653312796], [-73.77233735401518, 40.85364194680004], [-73.7723620067985, 40.85376576110119], [-73.77234917454822, 40.853851420736646], [-73.77280751808897, 40.85385063127562], [-73.77281122687663, 40.85379708655028], [-73.77296285334867, 40.853800062246464], [-73.77295817517044, 40.854137432923366], [-73.77279597721957, 40.85413175870194], [-73.77280008065789, 40.853963076256626], [-73.77269430639828, 40.85395751299703], [-73.77269448026055, 40.85390663901333], [-73.77233272219733, 40.85390084467517], [-73.77231116956706, 40.853965589494386], [-73.7722877176082, 40.85401044932687], [-73.77222913187389, 40.854122515546315], [-73.77219724701617, 40.85427953840102], [-73.77214036841814, 40.85441271260061], [-73.77212722589735, 40.85458881421388], [-73.77214547937614, 40.85475069709279], [-73.77218226378271, 40.85499353996747], [-73.77220024034331, 40.85523634635437], [-73.77220606901119, 40.85536488384107], [-73.77218644562852, 40.85560285594477], [-73.77217269968631, 40.855955082859246], [-73.77214690490916, 40.85616448177462], [-73.77213970639943, 40.856435799199566], [-73.77216934064688, 40.856935678765595], [-73.77216818289573, 40.85727365139707], [-73.77215380205806, 40.85781152614167], [-73.77216585105373, 40.85795435608545], [-73.77216552491815, 40.858049559538976], [-73.77219618881766, 40.858249549157016], [-73.77220802669532, 40.85845426144408], [-73.77220755388522, 40.85859230561815], [-73.77218824023912, 40.858739834638655], [-73.77218152925143, 40.858868346783424], [-73.77219364526376, 40.85899213574529], [-73.7722368970485, 40.8591778687732], [-73.77235432710658, 40.85967316198051], [-73.77219223055407, 40.85998663859697], [-73.77084562380267, 40.86010101292837], [-73.76947305423998, 40.85933013666858], [-73.76836277290795, 40.858358288851804], [-73.768360859188, 40.857655363705895], [-73.76533243995276, 40.85504359512487], [-73.76783205637173, 40.854442205742075]]], [[[-73.79018745194999, 40.8586099173861], [-73.79013351158892, 40.85856248730968], [-73.79012065319685, 40.85857012058902], [-73.79009141537634, 40.8585387453293], [-73.79009877156537, 40.858531798103954], [-73.7900459983775, 40.85847540491209], [-73.7899222929631, 40.858532289389935], [-73.78972202672374, 40.85856245340872], [-73.78934280432145, 40.85856093104265], [-73.78898385013113, 40.8584708437399], [-73.78897652126375, 40.85844842253513], [-73.7889780616097, 40.85843756780748], [-73.78897964442007, 40.858426403071135], [-73.78897973637031, 40.85839742298204], [-73.78897067954541, 40.858364948535126], [-73.7889661727637, 40.85834175635383], [-73.78895591927375, 40.8583102937374], [-73.78894954009625, 40.85829072099737], [-73.7889328557321, 40.85825591389827], [-73.78892076275636, 40.85821763824252], [-73.78891167199001, 40.85819559643317], [-73.78890874036226, 40.85815733753702], [-73.78891041371965, 40.858110972050746], [-73.78890751432796, 40.858062280004816], [-73.78890090345358, 40.85799406515931], [-73.78887898065909, 40.85791268944159], [-73.78886684361834, 40.85788832386005], [-73.78884710614867, 40.85785351233132], [-73.78882587932831, 40.85780710493002], [-73.78881535014905, 40.85775723897932], [-73.78879564521492, 40.857711994291584], [-73.78879886108632, 40.85766099473813], [-73.78879449753784, 40.85759259352103], [-73.78879167253787, 40.85752071816484], [-73.78878275515255, 40.85744419476535], [-73.78878402854569, 40.857420690336546], [-73.78826498050694, 40.857507000860615], [-73.78826195310619, 40.85749888196358], [-73.78817640998422, 40.857512635102445], [-73.788182450876, 40.85753351143941], [-73.788043429758, 40.85755991768708], [-73.78802830701825, 40.85751468128371], [-73.78816580032998, 40.85748827224901], [-73.7881718624852, 40.85750219354376], [-73.7882558718205, 40.857490755786465], [-73.78825436690734, 40.85748379794236], [-73.78877898544017, 40.85740092519962], [-73.78876389073886, 40.85732121926358], [-73.78875047058206, 40.85727379569839], [-73.78873164778054, 40.85721811889827], [-73.78872093476754, 40.85717276216798], [-73.78870485884443, 40.85710678617594], [-73.78871047688176, 40.85704703271107], [-73.78871879678134, 40.856991406114574], [-73.78875148515341, 40.856954371696006], [-73.78876247260872, 40.85691317561573], [-73.78877463362326, 40.85688983820476], [-73.78879251250657, 40.85685552689661], [-73.7888062667889, 40.8567978495311], [-73.7887955603384, 40.85675043160425], [-73.7887794059048, 40.856709185830454], [-73.78880667250745, 40.85667008025646], [-73.78883933406082, 40.85664128876774], [-73.78886926255444, 40.85661867439667], [-73.78888830770909, 40.85660428310894], [-73.78889654957185, 40.856573386709805], [-73.78888312226606, 40.856528024520195], [-73.78885616237595, 40.85647027146297], [-73.78881837966713, 40.856402195543055], [-73.788782472715, 40.856362941986546], [-73.78877003064156, 40.85634934056999], [-73.78875894260247, 40.856315532906635], [-73.78873183366792, 40.856305178546684], [-73.78872519415646, 40.856304851275965], [-73.78868841289075, 40.85630303881861], [-73.78865312839909, 40.85630297418632], [-73.78862333817503, 40.85628231184338], [-73.78860444893704, 40.856247242657005], [-73.78859643189443, 40.85620807247319], [-73.78858212373594, 40.856162888216524], [-73.78853687735143, 40.85615850479053], [-73.78850992512635, 40.856098690952145], [-73.78852086082745, 40.85607398231431], [-73.78846710159921, 40.855909017262086], [-73.78847402223268, 40.85587324721077], [-73.7884376714846, 40.85577501062688], [-73.78840780812261, 40.855777017072455], [-73.7883594322236, 40.85562648914821], [-73.78842551577243, 40.855329852881944], [-73.7884151118428, 40.855187637104514], [-73.78853250003307, 40.855154676929416], [-73.7885536572479, 40.85514873611422], [-73.78853679526938, 40.85510189085478], [-73.78849696186786, 40.85505383483559], [-73.78844808747019, 40.85505992685202], [-73.78843187532392, 40.85503722914398], [-73.78839114316482, 40.85504333657033], [-73.78835855433422, 40.855049459591385], [-73.78833961444349, 40.855030877113705], [-73.78833159638137, 40.854991706398195], [-73.7883209428071, 40.854927802659496], [-73.78831293199039, 40.85488657074242], [-73.78830763495627, 40.85484534480935], [-73.7882941364666, 40.85482265139003], [-73.78829419550173, 40.8548041044367], [-73.78832138879896, 40.85478766848557], [-73.78835401751454, 40.854769180114204], [-73.78839751595056, 40.854746591925725], [-73.78842472253515, 40.854726032878965], [-73.78842760041378, 40.854674517218335], [-73.78841692123977, 40.85461885577657], [-73.78841706546328, 40.854573518907685], [-73.78842728684886, 40.85455467815888], [-73.78844163730808, 40.85452822516081], [-73.78845805973428, 40.854484978348566], [-73.78848260375628, 40.85444792872273], [-73.7884826692924, 40.85442732090218], [-73.7884773265335, 40.8544005205322], [-73.78843136980228, 40.85434273369264], [-73.78840976856601, 40.854307659487745], [-73.78839632893784, 40.854266418620135], [-73.78834700683626, 40.854153518949865], [-73.78786849474662, 40.85427324147085], [-73.78787151151583, 40.85428484016822], [-73.78765149076932, 40.85434007726066], [-73.78764693594798, 40.85433195385581], [-73.7875567694506, 40.85436076813225], [-73.78756282815222, 40.85437584889325], [-73.787362690749, 40.85441953057157], [-73.78734753146404, 40.85438588545951], [-73.78754919139483, 40.85434336587592], [-73.78755374968587, 40.85435033052877], [-73.78764390876884, 40.854323834435505], [-73.78763935178883, 40.85431687146393], [-73.78834514784103, 40.854132012350114], [-73.7883239944991, 40.85410251564968], [-73.78827221224095, 40.85411296306839], [-73.78819872490023, 40.854127789058104], [-73.78816813692173, 40.85408577890348], [-73.78777873831645, 40.85417338531972], [-73.78769014281566, 40.85418829124551], [-73.78765807500652, 40.85419055109111], [-73.78762757097078, 40.85418122099095], [-73.78756345016592, 40.854181102936444], [-73.787430572486, 40.8541982464804], [-73.78731450066233, 40.854211942725605], [-73.78726555834814, 40.85420762787154], [-73.78722448578456, 40.85419322967826], [-73.7871970601854, 40.85417579079074], [-73.78717271889465, 40.854149084305526], [-73.78714539852163, 40.85409918836224], [-73.7871254596767, 40.854044378654386], [-73.78691339069742, 40.85408369046205], [-73.78690734702697, 40.854063972299144], [-73.7871328723601, 40.854024270878305], [-73.7871545889612, 40.85397968963615], [-73.7872084560664, 40.85395440428848], [-73.78729251210585, 40.85392673777914], [-73.78732350665298, 40.853917164311945], [-73.78741935244048, 40.853887558074995], [-73.78762411569168, 40.85382881638005], [-73.78781206717181, 40.85377583738079], [-73.78787778457543, 40.85375393421109], [-73.78792821707314, 40.85373779738912], [-73.7879679994746, 40.853710050165546], [-73.78799403046978, 40.85368575461534], [-73.78800508108596, 40.8536648684512], [-73.78801104438836, 40.85361623383958], [-73.78801424958195, 40.853568711208176], [-73.78800064949668, 40.853524636327975], [-73.78796264288079, 40.8534747209355], [-73.78760502319017, 40.85311470853485], [-73.78716496421869, 40.85275741228193], [-73.78712672030899, 40.85277997402518], [-73.78712215823363, 40.85277417045418], [-73.78707627655098, 40.85279958743541], [-73.78706106581926, 40.85278217272936], [-73.78710999004264, 40.852760237823894], [-73.78710390562176, 40.85275327033489], [-73.78713667028038, 40.85273387524412], [-73.78693805735816, 40.85257908292886], [-73.78691672214161, 40.85256745204469], [-73.78688470337548, 40.85255464098319], [-73.78684958987257, 40.85255457611149], [-73.78678232558099, 40.85255570723554], [-73.78675959223264, 40.85253122633288], [-73.7867657472106, 40.85251616759754], [-73.78677648933812, 40.852498799145046], [-73.7867765376467, 40.852483729624616], [-73.7867689971708, 40.85245473539846], [-73.78659390163132, 40.8523083524231], [-73.78641733510773, 40.8521445764697], [-73.78637622032105, 40.85211204192054], [-73.7863244234208, 40.852078329311354], [-73.78624762093361, 40.85202281469252], [-73.78588186506632, 40.851788672440925], [-73.78566404122395, 40.8517959958209], [-73.78545277174172, 40.851664225836416], [-73.7854019443249, 40.85164558422445], [-73.78529210719023, 40.85162064997351], [-73.78521881434392, 40.85162515043416], [-73.78510066461135, 40.851652751934395], [-73.78503542905987, 40.8516835424064], [-73.78495591460657, 40.851723580683434], [-73.78489888572953, 40.85173429288914], [-73.78480722795186, 40.85175266902376], [-73.78468307195402, 40.85174934687985], [-73.78433474208568, 40.85177615757233], [-73.78427457330046, 40.85179199351651], [-73.78427798468039, 40.851799172934044], [-73.78413189167978, 40.85183280615726], [-73.78412338226599, 40.85180866377685], [-73.78426432858558, 40.851773717101274], [-73.78426688465254, 40.85177959007606], [-73.78431415165997, 40.85176754570688], [-73.78402637080502, 40.851609931862825], [-73.7839793289859, 40.851532617440434], [-73.78387115358153, 40.85155896167003], [-73.78386093355508, 40.8515328612276], [-73.78397173451735, 40.85150768135716], [-73.78396112488305, 40.851486933922956], [-73.78381306352932, 40.85151888329484], [-73.78357767103407, 40.85156967744472], [-73.78356242001054, 40.85156501244066], [-73.78354267337754, 40.85153797070415], [-73.78347956766932, 40.851560183887145], [-73.78347528477201, 40.85155691566572], [-73.78341001772101, 40.851579018711305], [-73.78338158610377, 40.85158673391434], [-73.78337646578845, 40.85157694324869], [-73.7834133904305, 40.85156341968858], [-73.78353416916762, 40.85152078053071], [-73.78344495936106, 40.85140090922443], [-73.78349609889963, 40.85136877997576], [-73.78360210541598, 40.85131951978262], [-73.78379501276285, 40.85125393535625], [-73.78398791994971, 40.85118835060683], [-73.78414275091885, 40.85114742417621], [-73.78435188706086, 40.85109835504541], [-73.78447953199854, 40.85107180299238], [-73.78460077697598, 40.851095692195194], [-73.78451936300075, 40.850892735021304], [-73.78444750529484, 40.85090481745937], [-73.784434075491, 40.85086151490386], [-73.78450494990614, 40.8508535004636], [-73.78448807935747, 40.850815826824025], [-73.78447441853888, 40.85079145789808], [-73.78445463288537, 40.85077287385447], [-73.78442108103592, 40.85076237961208], [-73.78439207608706, 40.85076232540572], [-73.78434575107664, 40.85077005079134], [-73.78433708793975, 40.85077149590738], [-73.78431265352698, 40.85077376842167], [-73.78426993630782, 40.85076557351952], [-73.78422574727558, 40.8507399892099], [-73.78416329499161, 40.85069698072504], [-73.78411913614823, 40.850662121467856], [-73.78407975214625, 40.850630614908575], [-73.78404133663162, 40.85064458870901], [-73.78404741235457, 40.850653873370376], [-73.78402142627083, 40.850664257945056], [-73.78401687236345, 40.85065613489466], [-73.78385181911865, 40.85071146765854], [-73.78384119307984, 40.850692899498306], [-73.78395428629146, 40.850654857765434], [-73.78395883886576, 40.850662981318294], [-73.78401079664636, 40.85064684972925], [-73.78400319779757, 40.85063640227853], [-73.78403070700506, 40.85062718049514], [-73.78403526024866, 40.85063530404628], [-73.78406580837313, 40.850625233236386], [-73.78401710573823, 40.850584263374934], [-73.78400879049795, 40.8505259320535], [-73.78400362316144, 40.85050541275762], [-73.78401288041832, 40.85047528985302], [-73.78401421301417, 40.85047391345848], [-73.78404198397534, 40.85044520577757], [-73.78412613427056, 40.85038740267569], [-73.78420112637873, 40.85032958288327], [-73.78423943051327, 40.8502867628729], [-73.78425785826201, 40.85025318064228], [-73.78425338375496, 40.850220713839434], [-73.78421247052576, 40.850126741818116], [-73.78420494271475, 40.85009427047384], [-73.78420047134377, 40.8500606442406], [-73.78415333900804, 40.85000143719395], [-73.78409701801216, 40.849951485046276], [-73.78407378345817, 40.849902425389395], [-73.78357825260379, 40.849266386802185], [-73.78353497808732, 40.849220967420266], [-73.78349977784661, 40.84919617197559], [-73.7834536807776, 40.84918372050767], [-73.78339395583185, 40.84918979001323], [-73.78333693054938, 40.84919998765853], [-73.78329074622592, 40.84921432656418], [-73.78323644139, 40.84922246805528], [-73.78322357300496, 40.84922244387418], [-73.78317944898282, 40.84922236094945], [-73.78314691693552, 40.849211995566776], [-73.78311715803491, 40.84918308830608], [-73.78303535537907, 40.84907190812784], [-73.78244952709214, 40.849341154015676], [-73.78221806474305, 40.849166804596116], [-73.78209456304481, 40.849247300009694], [-73.78207020661583, 40.84922638871912], [-73.78219328627868, 40.84914279099544], [-73.78215814152048, 40.849116173923825], [-73.78203367458848, 40.84919386175839], [-73.78201995532704, 40.84918803900289], [-73.78196338004864, 40.849215753630496], [-73.78197402498779, 40.849228525675706], [-73.78179206236673, 40.849319758945455], [-73.7817829373723, 40.849309307955735], [-73.78174317784173, 40.84933009868773], [-73.78175535569149, 40.84934055428367], [-73.7815749238, 40.84943063184538], [-73.78154449088599, 40.84940043464333], [-73.78171883268593, 40.84930570921137], [-73.78173101140975, 40.84931616481152], [-73.78176924362299, 40.84929537236711], [-73.78176163701295, 40.84928724294747], [-73.78193902433554, 40.849194841810736], [-73.78194967598762, 40.84920529450143], [-73.78201082139036, 40.84917990738358], [-73.78214108697831, 40.84910066266766], [-73.78193782713453, 40.8489339814074], [-73.78186608494606, 40.84894484103933], [-73.78184847915905, 40.84894750521786], [-73.7818286909823, 40.84895920536359], [-73.78169581560914, 40.84884836234784], [-73.78161594069067, 40.848896204611144], [-73.7816073854627, 40.84888640853134], [-73.78168604284517, 40.84883788315214], [-73.78161367535198, 40.84877238634991], [-73.78153019763666, 40.8488575718796], [-73.78150620325309, 40.84884252817913], [-73.78159491975414, 40.848752947876605], [-73.78157440440629, 40.84873180821234], [-73.78153956955133, 40.84872637528167], [-73.78145563649926, 40.848813090897714], [-73.78143249522284, 40.84880000581332], [-73.78150756375803, 40.848719289881394], [-73.78149337883968, 40.84871530676672], [-73.78149386587489, 40.84858047582576], [-73.78178514040063, 40.8485098018392], [-73.78203616713365, 40.84843804928343], [-73.78200212772737, 40.84834408892634], [-73.78202189306344, 40.848339562922654], [-73.78206189132574, 40.848449183284664], [-73.78178920235626, 40.848527207728736], [-73.7817970940135, 40.84854714327129], [-73.78207041238296, 40.84846941249452], [-73.78207657695803, 40.8484224763746], [-73.78209640639666, 40.8483983878631], [-73.78220384181529, 40.848367943911796], [-73.78251408510305, 40.8482896312975], [-73.78255877758846, 40.848277325976234], [-73.78261900912084, 40.84823961994345], [-73.78261910501016, 40.84821027812582], [-73.78267467078706, 40.84813967294664], [-73.78189457733075, 40.84832716154012], [-73.78176219386184, 40.84835164156259], [-73.78155442120504, 40.8484007075863], [-73.78152611318389, 40.8483434661371], [-73.78138148782878, 40.848377195829165], [-73.78139151751019, 40.84842203823172], [-73.78133652580439, 40.84843275253867], [-73.78112258967961, 40.84849880844866], [-73.78110827162968, 40.84852042002227], [-73.78108791694191, 40.84852038139701], [-73.78107575079667, 40.848506446947276], [-73.78107580166363, 40.84849099161343], [-73.78109415548866, 40.84848020673815], [-73.78130194972267, 40.84842495894283], [-73.78131826349642, 40.848415716075756], [-73.78131833463324, 40.84839407797139], [-73.78106574674733, 40.848453877688705], [-73.78101890213868, 40.84846306206486], [-73.78099244183939, 40.84846301183228], [-73.78097820938946, 40.8484583479135], [-73.78098438117183, 40.84843826780632], [-73.78101090695077, 40.848418224664776], [-73.78109036010902, 40.84839673722948], [-73.78114129062631, 40.84838292367897], [-73.78130627517389, 40.84834768647257], [-73.78130228002807, 40.84832449541754], [-73.78114557934342, 40.8483164696778], [-73.78112520061113, 40.84832415956639], [-73.78110078567305, 40.84832102158365], [-73.78109878127883, 40.84831174448807], [-73.78110697775321, 40.84829475819656], [-73.78112733303398, 40.84829479681971], [-73.781153783098, 40.84829793765262], [-73.7811538186934, 40.84828711851653], [-73.7812942097643, 40.84830284137517], [-73.78135124158366, 40.84829058454093], [-73.7813714588191, 40.84833235459947], [-73.78172594274021, 40.848234105806], [-73.781831743682, 40.84824667089895], [-73.78263740397888, 40.848072649245836], [-73.78250408633544, 40.84791327376318], [-73.78158418081362, 40.84803705254238], [-73.78148431950568, 40.8478821938486], [-73.78150307163138, 40.847681642963124], [-73.78142920347912, 40.84745634535726], [-73.78152237195346, 40.847313479941405], [-73.781844099006, 40.84713610793703], [-73.78182523912372, 40.84706551767205], [-73.78205261928964, 40.846990837927684], [-73.78209539971068, 40.84691339096235], [-73.78221497817432, 40.846861409589465], [-73.78218546562235, 40.846719715627714], [-73.78216393435586, 40.84663633778076], [-73.78112548290936, 40.846829466907394], [-73.78113626322276, 40.846852155377384], [-73.7810683698102, 40.846866452205624], [-73.7810575836192, 40.84684582392917], [-73.78063668411163, 40.84692333475236], [-73.78067693704058, 40.84706148579347], [-73.7804624159846, 40.84710023328956], [-73.780403269586, 40.846931134166056], [-73.78106043973004, 40.84680255185432], [-73.78115182578563, 40.846246304627094], [-73.78118979824524, 40.84625255880356], [-73.78110675760784, 40.846746997141146], [-73.78168786937833, 40.84663887521194], [-73.78171847815004, 40.84640812132608], [-73.78174288903251, 40.84641229011591], [-73.78171494622761, 40.84665747421467], [-73.78109853456938, 40.846771711438976], [-73.78109301885183, 40.84679849158743], [-73.78215667396833, 40.846607578113655], [-73.78214232783147, 40.84657770660868], [-73.78232827748431, 40.8465486112713], [-73.78229152990997, 40.84645779011659], [-73.7825134511685, 40.846408771551154], [-73.78235702810511, 40.84598158833272], [-73.78257932865696, 40.845935893027736], [-73.78251990755469, 40.84575819426209], [-73.78249597258944, 40.84566156194901], [-73.78309442048638, 40.845518285564644], [-73.78309556892438, 40.84551369086318], [-73.78314644227363, 40.845309942026006], [-73.78311070737087, 40.845308129742314], [-73.78307455830657, 40.84550976429195], [-73.78257509379014, 40.845601850566055], [-73.78257849117324, 40.84561315914988], [-73.7825383977149, 40.84562003866996], [-73.78254177869914, 40.84563656442654], [-73.78242148841352, 40.84566068102307], [-73.78242487155511, 40.845676336667], [-73.78182047648926, 40.845781987498064], [-73.78181646104291, 40.845764978051015], [-73.78238057457457, 40.84566557830359], [-73.78237655393292, 40.845650115280975], [-73.78250690114285, 40.8456240848009], [-73.78250088102259, 40.84559779830172], [-73.78305074940779, 40.845496823175814], [-73.78308181330725, 40.84533304738968], [-73.7830594398957, 40.84532836841475], [-73.78309455887226, 40.84516923547115], [-73.78306023126578, 40.84508570856704], [-73.78190735927909, 40.84535556174675], [-73.78183975772673, 40.84548835758404], [-73.7818153590045, 40.84548058288038], [-73.78188707672663, 40.845333885649346], [-73.78307457446681, 40.845056368733935], [-73.78313318952935, 40.84518167305947], [-73.78310841324824, 40.84528981990339], [-73.78315245505728, 40.84529598332778], [-73.78322207613923, 40.84506344465817], [-73.78309714310711, 40.844991674332086], [-73.78297579244337, 40.84468094802046], [-73.7833653613534, 40.84444472298169], [-73.78335700996655, 40.844402846537584], [-73.7833354347671, 40.84436158971978], [-73.78332100656189, 40.844347824802384], [-73.78328485082511, 40.84433951285005], [-73.7832381891779, 40.844347014854286], [-73.78323416798308, 40.84434766164698], [-73.78315451147776, 40.84436399780923], [-73.78310017474654, 40.844383130600946], [-73.78306040951676, 40.84437206489805], [-73.78304608891058, 40.84432532503511], [-73.78301129829549, 40.844172690347165], [-73.78326987138819, 40.84406939209163], [-73.78309494705954, 40.84386609490481], [-73.78349593173812, 40.84373234293595], [-73.78348428748797, 40.84367854599843], [-73.78298567054594, 40.84379407880406], [-73.78289307603839, 40.8438948842585], [-73.78290650949755, 40.84393612584013], [-73.78057367810457, 40.844469585986374], [-73.78054653995335, 40.844469534359], [-73.78057628971331, 40.84450050300951], [-73.78052736048552, 40.84452514033816], [-73.7799270374445, 40.843883081355834], [-73.77997868590161, 40.84385638925434], [-73.78053302714567, 40.84445302176267], [-73.78056017668013, 40.844448951818684], [-73.78036906217557, 40.8439766611937], [-73.78039623282005, 40.84396640817976], [-73.78058464728879, 40.8444345726982], [-73.78106261364299, 40.844328317876695], [-73.78087690198716, 40.843862220487786], [-73.7809122002328, 40.84385610539353], [-73.78108977797854, 40.84432012707755], [-73.78154602523335, 40.844215889747694], [-73.78135229180006, 40.84371268255372], [-73.78137674098357, 40.84370448654003], [-73.78157590302521, 40.844207703973034], [-73.78200770536681, 40.84410960066972], [-73.78180047475469, 40.843583700688804], [-73.78183306599355, 40.84357551830497], [-73.78203757586272, 40.84410347497348], [-73.78243679895445, 40.844009431099934], [-73.78221611882311, 40.84344641111838], [-73.78224871668732, 40.843436168417554], [-73.78246939706244, 40.84399918783504], [-73.78282243971395, 40.843919481199485], [-73.78257210723775, 40.84329458244578], [-73.78260741176439, 40.84328640495761], [-73.78278498635665, 40.84375454548095], [-73.7830918611237, 40.84368505481119], [-73.78287388243167, 40.843124102362346], [-73.78290103888799, 40.84311797133362], [-73.78313252568665, 40.84369749673174], [-73.78280661506554, 40.84377931612205], [-73.78284151617696, 40.84389478720195], [-73.78286866600375, 40.84389071688291], [-73.78294225812178, 40.84379193640604], [-73.7829830443998, 40.84376728323528], [-73.78347481249244, 40.843654738109784], [-73.7831919451847, 40.842867864988975], [-73.78304539815447, 40.842896332066644], [-73.78324501260472, 40.84347167587895], [-73.78321038850866, 40.84347933886645], [-73.78300874308394, 40.84290244643356], [-73.78291098678886, 40.84292235500552], [-73.78289487635368, 40.842869773693224], [-73.78313925968678, 40.84282231949004], [-73.78311703961026, 40.842771272634415], [-73.78291747085107, 40.8428064456791], [-73.78289925507742, 40.84277549983599], [-73.78312383020314, 40.84273620299365], [-73.78301653344668, 40.842605387297304], [-73.78288778301486, 40.84249482884962], [-73.78273319376788, 40.842521705498456], [-73.78281584526042, 40.842764522596624], [-73.78277919046207, 40.84277063689116], [-73.78270669982565, 40.842532475810295], [-73.78229941270918, 40.84260589641337], [-73.78244452404584, 40.84304203441674], [-73.78240786884528, 40.84304814691726], [-73.78226682874738, 40.84261201822825], [-73.78180048362434, 40.84269614579364], [-73.78196972502653, 40.84322042881595], [-73.7819330659365, 40.843228088076515], [-73.78177196927737, 40.84270227349191], [-73.78124656634495, 40.84279710728944], [-73.78142785188439, 40.84337087428157], [-73.78138916319325, 40.84337698260531], [-73.7812160111927, 40.84280477739565], [-73.78067429925017, 40.84290730453772], [-73.78085958503786, 40.843501171869235], [-73.7808290368131, 40.84350729714562], [-73.78064375993134, 40.842910338123474], [-73.78029552532065, 40.842973044893895], [-73.78029286577419, 40.843778305365376], [-73.78022773872677, 40.84377818129771], [-73.78022662418076, 40.842883268935495], [-73.78029175678296, 40.842881846600335], [-73.78029152196487, 40.84295294475395], [-73.78286499304913, 40.84247810188268], [-73.78271552022753, 40.84244650059041], [-73.78230783665785, 40.84251935826533], [-73.78229572698898, 40.8424884238486], [-73.7824348109359, 40.842466121435066], [-73.78241082488181, 40.84245005032449], [-73.78059735822595, 40.84278659994098], [-73.78058525374094, 40.842754119441445], [-73.78236855953504, 40.84242849314965], [-73.78234000558811, 40.84234092564359], [-73.78215975190268, 40.8423706998425], [-73.78215166139216, 40.84235522911647], [-73.78233051660672, 40.84232245176547], [-73.78231110910542, 40.842224178957146], [-73.78221716280949, 40.84224097717735], [-73.78222115152285, 40.84226571462883], [-73.78216005558019, 40.84227796304772], [-73.78215604531863, 40.842259408368676], [-73.7816286101587, 40.84235424006778], [-73.78162055021907, 40.84232949488805], [-73.78224574731087, 40.84221320956452], [-73.78224979212551, 40.84222094525863], [-73.78230643317583, 40.84221111631256], [-73.78232118543184, 40.84217162017212], [-73.78232536834764, 40.842137624341056], [-73.7823316099391, 40.84209590550887], [-73.78237240991096, 40.84206661567217], [-73.78237057744752, 40.84200478690896], [-73.78233257215871, 40.84192284411137], [-73.78229348197962, 40.8419304528299], [-73.78231163148793, 40.84198149217097], [-73.78227701311965, 40.84198760897666], [-73.78226695851775, 40.84195049461237], [-73.78145418008431, 40.84217152712605], [-73.78144003107849, 40.8421421334934], [-73.7823235128594, 40.84189833045455], [-73.78226771471576, 40.84178617540726], [-73.78207533518163, 40.84183242503751], [-73.78208103076508, 40.84184112996917], [-73.78206499291741, 40.84184457780548], [-73.78206158354132, 40.84183674614081], [-73.78203752533935, 40.84184191671499], [-73.78203410829995, 40.84183669422445], [-73.78226345326249, 40.84177811598246], [-73.78224024540253, 40.8417358661234], [-73.78201593435602, 40.841793189025154], [-73.78201252301916, 40.8417862274759], [-73.78203658908592, 40.84177844890506], [-73.78203546964845, 40.841770621566745], [-73.78205150835818, 40.84176717490872], [-73.78205377835471, 40.8417732641795], [-73.78223592679213, 40.84172788348313], [-73.78221542233346, 40.841704017300245], [-73.78216624418964, 40.84168914411004], [-73.78207235608491, 40.84169331384538], [-73.78202093113624, 40.841704333914905], [-73.78188333127875, 40.84173381939348], [-73.78186601221255, 40.84168736467965], [-73.78140696595466, 40.841772040201484], [-73.7814183019977, 40.84180596942078], [-73.7813667641349, 40.84181282684591], [-73.78135197332533, 40.84178497722712], [-73.7812992811557, 40.84179444162352], [-73.78132082303267, 40.84185794850812], [-73.7812749882927, 40.84187090285884], [-73.78133868571547, 40.8419962184842], [-73.78130775340559, 40.84200311544593], [-73.78124863502482, 40.84187780798532], [-73.78113290101273, 40.84191062605361], [-73.78119772238901, 40.842042029373], [-73.78117137190374, 40.842048065538975], [-73.78109967064516, 40.84192012611332], [-73.78097935660904, 40.84195293533548], [-73.78105448455362, 40.842083489356845], [-73.78102354424689, 40.84209299541667], [-73.7809552932145, 40.8419598454131], [-73.78084643996941, 40.841989198625804], [-73.78092499038787, 40.84212323619836], [-73.78088717448651, 40.84213446705864], [-73.7808143539402, 40.84199870077753], [-73.78069632802139, 40.842032383000685], [-73.78076452835761, 40.842180312882476], [-73.78072098830445, 40.8421915323058], [-73.78065508284249, 40.84204186886624], [-73.78060581105882, 40.84205568586571], [-73.78058402738796, 40.842065207046154], [-73.7805313141506, 40.84208075603331], [-73.7805915762219, 40.84220606653751], [-73.78056292706687, 40.842214706208765], [-73.78050381144551, 40.842089398373815], [-73.78043963564978, 40.84210927259141], [-73.78045552576683, 40.84215103462286], [-73.78039480479654, 40.84216482923703], [-73.78037662608997, 40.84212306401238], [-73.78036980593724, 40.84210827070108], [-73.78034805158231, 40.84210909822444], [-73.7804257718152, 40.842494394206874], [-73.78029863976153, 40.84251154046733], [-73.78019599955293, 40.84204621097616], [-73.78033227924816, 40.842031690267376], [-73.78034466049567, 40.8420960516831], [-73.78037786810293, 40.84209350522443], [-73.78037675812158, 40.842083071546455], [-73.78126135401865, 40.841839578457], [-73.78123754712247, 40.841768242042036], [-73.78185048197766, 40.841653946394324], [-73.78183863598309, 40.8415347846261], [-73.78183884886525, 40.84146986973238], [-73.78183909216312, 40.841395679917284], [-73.78182300321745, 40.84133691700386], [-73.7818048644, 40.841282785939924], [-73.78176234410533, 40.84121624495401], [-73.7817176560301, 40.84118988354887], [-73.78168516996307, 40.84116663855672], [-73.78166051213424, 40.84112417558818], [-73.78084223802306, 40.84127632402818], [-73.78092886504994, 40.84154542526756], [-73.78089220610157, 40.84155308419829], [-73.7807975292347, 40.84125614604844], [-73.7816485529559, 40.84109776352879], [-73.78161356543811, 40.841007272069724], [-73.78139590619043, 40.841047865834945], [-73.78139143568951, 40.84103660961823], [-73.78161069651507, 40.84099624002564], [-73.78159702603972, 40.840948669593274], [-73.78137871885639, 40.8409871920024], [-73.78137488194949, 40.84097887100505], [-73.78159396937245, 40.84093979674803], [-73.78156821700871, 40.84083516109827], [-73.78154308949141, 40.84078478253003], [-73.78151345014929, 40.84077268257038], [-73.7814808892744, 40.840764371324674], [-73.78144833514813, 40.84075441114133], [-73.78142666976504, 40.84073622034478], [-73.78141805071888, 40.84071475571612], [-73.78142028765106, 40.840694961773686], [-73.78144207161212, 40.840676853309866], [-73.7814660242275, 40.840660399064646], [-73.78147694069271, 40.84064392129004], [-73.78148570562806, 40.84062083948602], [-73.78148577066192, 40.84060104025461], [-73.78147718457494, 40.840569675512484], [-73.78146203707614, 40.84055149874544], [-73.78145342800167, 40.84052673391017], [-73.78145349846588, 40.840505284574995], [-73.78146442574841, 40.840485507096744], [-73.78148837696587, 40.8404690528438], [-73.78151347737062, 40.840451271290306], [-73.78152321749889, 40.84044437031867], [-73.7815428414098, 40.84042295969416], [-73.78154296060534, 40.8403866616044], [-73.78153441046068, 40.84034374764454], [-73.78151504322612, 40.84028761446136], [-73.78149172006931, 40.84020622452522], [-73.78125361834401, 40.84025550094135], [-73.78124907233904, 40.840244354805236], [-73.78148878089243, 40.84019732392208], [-73.78146559787315, 40.840175159763376], [-73.78145366531946, 40.84013628677303], [-73.78136267245321, 40.83973018981636], [-73.78118431444939, 40.83976164869876], [-73.78117947802396, 40.839745862255796], [-73.78135932010679, 40.83971170821572], [-73.78134125145772, 40.839478563442896], [-73.78113994352726, 40.839505416893545], [-73.78113875725964, 40.83949427880171], [-73.78133988849423, 40.839469699830296], [-73.78132522577029, 40.83929178135658], [-73.78134240000931, 40.839238374844435], [-73.78156780983515, 40.83853740573627], [-73.78159698419161, 40.83847785014295], [-73.78172154866671, 40.838223566109775], [-73.78200420989037, 40.838015572307114], [-73.7818845572027, 40.83765027660479], [-73.78238142556087, 40.837324106167365], [-73.78272760738487, 40.83748796035137], [-73.78274559046729, 40.83747439359701], [-73.78281452652847, 40.83742239136637], [-73.7829062919088, 40.83732000607281], [-73.7828873348317, 40.837164735646695], [-73.78279680443434, 40.83692052345197], [-73.78270364583639, 40.83684280196558], [-73.78268131435576, 40.83648367715349], [-73.78282260139727, 40.836338155887844], [-73.7830292186274, 40.83634585234474], [-73.78338168059321, 40.83690927168068], [-73.78366393608437, 40.83681161024659], [-73.78410164591317, 40.83672552935798], [-73.78421807493936, 40.83675490217379], [-73.78431347112955, 40.83681282675119], [-73.78448456140005, 40.837024222538496], [-73.78491512256413, 40.836841998575245], [-73.78488487052007, 40.83679244441207], [-73.78498060328094, 40.83674642608051], [-73.78508226642613, 40.83688190821304], [-73.78498435600915, 40.83692957218049], [-73.78494325733283, 40.83687504914361], [-73.7845033396534, 40.83705776517111], [-73.78485009222643, 40.83747048917844], [-73.78512582175478, 40.837446041569414], [-73.78563997845448, 40.83706443458537], [-73.78583067964128, 40.83720998000451], [-73.78579367439913, 40.8372330098709], [-73.78565279572325, 40.83713210472607], [-73.78516584428648, 40.83747133406176], [-73.7852771294841, 40.83775671808692], [-73.78557796850949, 40.8379271132254], [-73.7854986418295, 40.83848036679504], [-73.78554772810212, 40.838725713947404], [-73.78553011511531, 40.83879827733832], [-73.7855603115974, 40.83886597978357], [-73.78568113678003, 40.839101104369114], [-73.7857350929868, 40.83923423177581], [-73.78592998544981, 40.83942928196881], [-73.78607898490304, 40.83953813339062], [-73.78614637767016, 40.839587366253056], [-73.78627272274821, 40.83958830856518], [-73.78652598496416, 40.83972311560703], [-73.78690015821412, 40.83971486189476], [-73.78730353132528, 40.839674724501016], [-73.78732985698998, 40.839703100722524], [-73.78740758745991, 40.839677623196984], [-73.78741374512595, 40.83968663880644], [-73.78733627876298, 40.839710022100135], [-73.78738854482546, 40.83976635600593], [-73.7874710857183, 40.83994515630109], [-73.78749162161397, 40.839989639321495], [-73.78749095280199, 40.8400490260975], [-73.78748134063115, 40.840054545565664], [-73.78745014326732, 40.840072461586495], [-73.78747062422686, 40.840122203428805], [-73.78763931218995, 40.8405318952366], [-73.78739774725157, 40.84058231694795], [-73.7873572424447, 40.84059077177667], [-73.78738672135641, 40.840651925716585], [-73.78749166412584, 40.840842341674076], [-73.78771665246705, 40.8408042706383], [-73.78771346922073, 40.84078075409792], [-73.78791889178505, 40.84074277179641], [-73.78792536909036, 40.84075515855147], [-73.78818132144568, 40.840711080501336], [-73.78816355627822, 40.840661551844406], [-73.78813910778516, 40.840663981607676], [-73.78809400249324, 40.84050179629599], [-73.78815921166512, 40.84049077856704], [-73.78821083495573, 40.84065297632424], [-73.78818800455072, 40.84065912157288], [-73.78820578964854, 40.84070246476651], [-73.78824000945202, 40.84070128975995], [-73.78825453283457, 40.840745863118485], [-73.78796596546518, 40.840798542045825], [-73.78795786196089, 40.84078491614255], [-73.78777526956972, 40.840816753350566], [-73.78778496347275, 40.840842757037976], [-73.78752379733702, 40.84087961218515], [-73.78757937523613, 40.840932710498684], [-73.78778656795107, 40.840892222012805], [-73.78786404049687, 40.84108172583491], [-73.78760640589596, 40.84114188575643], [-73.78761237782592, 40.84119800529587], [-73.78780098073766, 40.84144046390852], [-73.78784487499479, 40.841471479333244], [-73.78789698683862, 40.841480236842195], [-73.78795239075858, 40.841479101375796], [-73.78804369145313, 40.84146194567765], [-73.78812190151476, 40.84146208936972], [-73.78812391608882, 40.84146252561776], [-73.78819680117678, 40.84147831310449], [-73.7882390823508, 40.84150437775736], [-73.78825197587602, 40.84152874537163], [-73.78836617474826, 40.84150461101653], [-73.78837101202566, 40.84152070556352], [-73.7882606446571, 40.841541962860816], [-73.78826006848325, 40.84156628715141], [-73.78826000546677, 40.84158608571318], [-73.78825337746484, 40.84162072123125], [-73.78823370218421, 40.841659045801194], [-73.7882042790782, 40.841688689626494], [-73.78816020303344, 40.841714594533144], [-73.78811941308066, 40.841731843119284], [-73.78805250551622, 40.84176389321272], [-73.78802003550352, 40.84178384145876], [-73.7880484708972, 40.84185072312646], [-73.78829358843848, 40.84178537313663], [-73.78830167638179, 40.841803948821735], [-73.78805599070779, 40.84186703017751], [-73.78823199262384, 40.842196084772084], [-73.78838456613674, 40.842381978370604], [-73.7884390903701, 40.842402498578], [-73.78850993801106, 40.8423910353269], [-73.78852027082415, 40.842411765450514], [-73.78883864895133, 40.84235039596113], [-73.78884273257208, 40.84236345457356], [-73.7885278086737, 40.84242557689478], [-73.78871619613662, 40.842656753314145], [-73.78876384724369, 40.84265962584494], [-73.7887905351074, 40.842721855154856], [-73.78876233407976, 40.84275150135509], [-73.78884308883718, 40.84300024632511], [-73.7888908617258, 40.843070064349575], [-73.78905825501361, 40.84323533177004], [-73.78961660328446, 40.84311265436494], [-73.78965755656859, 40.84323135456166], [-73.78978620461135, 40.843208126160725], [-73.78977897832563, 40.84317470164448], [-73.78983522236199, 40.84316552413694], [-73.78985456556266, 40.843232380061444], [-73.7899242492441, 40.84322415424067], [-73.78991705673337, 40.843179593293684], [-73.78992928728755, 40.843176831317706], [-73.78991962151179, 40.8431415480429], [-73.78990617379498, 40.843143379047184], [-73.78988202767482, 40.84304960029012], [-73.78993093388229, 40.84304226436364], [-73.78995751500784, 40.843138832800726], [-73.78994406956207, 40.84313973556226], [-73.78995006820057, 40.84317501332979], [-73.7899659607601, 40.843173186772184], [-73.78999975309046, 40.843307817243556], [-73.78995207552958, 40.843313298934326], [-73.78993152902541, 40.84324087250168], [-73.78986183858275, 40.843250954146654], [-73.78988953603827, 40.84338093446732], [-73.78973469326743, 40.84341324747867], [-73.78974579972716, 40.843447822588665], [-73.78989919889905, 40.84341714665903], [-73.78990645143352, 40.84344221745637], [-73.78975626839882, 40.84347034066179], [-73.78977421223745, 40.843513574653066], [-73.78980449502046, 40.843586656282376], [-73.78974084707778, 40.84359957047397], [-73.78933708725715, 40.843681498055474], [-73.78931949144742, 40.84374911114966], [-73.78933021337059, 40.84379367790621], [-73.78936047257154, 40.84384323084519], [-73.78942132625365, 40.84383674251027], [-73.78946443345122, 40.843945715050005], [-73.7893470202629, 40.84397519852208], [-73.7893404709631, 40.84398508505802], [-73.78934690531925, 40.844011496597254], [-73.78947033356705, 40.844289624818295], [-73.78947182214094, 40.84429297835204], [-73.78959737629596, 40.84450197311118], [-73.78990804558647, 40.844897610903416], [-73.78992895494343, 40.844924237568044], [-73.79001969152111, 40.84492841715729], [-73.79019920190943, 40.845187777884306], [-73.7904964276791, 40.84549275521502], [-73.79066589001, 40.84574155883438], [-73.79085156066733, 40.84569878164127], [-73.79098858375902, 40.84565283206255], [-73.79101885992935, 40.84569743474458], [-73.79069785364858, 40.84578601255083], [-73.79096069596163, 40.846230246274516], [-73.79090848534602, 40.8462516010898], [-73.790940734436, 40.84630082064029], [-73.79094673676796, 40.84630998184841], [-73.79097333310666, 40.846357311524955], [-73.79106349089012, 40.846346166654506], [-73.7910642712526, 40.84634607040144], [-73.79149076513553, 40.84625100496525], [-73.79149937351545, 40.84627741912916], [-73.79112759246645, 40.84635759094952], [-73.79114478824971, 40.84641701940378], [-73.79112476570172, 40.84642747233158], [-73.79118128785346, 40.84655567592981], [-73.79126057441357, 40.846549170067966], [-73.79131170561902, 40.84653776390074], [-73.79132901030732, 40.84656254319429], [-73.79141379114441, 40.84654784703281], [-73.7914310833159, 40.84657592768415], [-73.79132451946978, 40.846608733001716], [-73.79138070403269, 40.84670617747228], [-73.79141982667437, 40.84670129956033], [-73.7914803472568, 40.846802052713656], [-73.79179343625451, 40.846731672329284], [-73.79180418591311, 40.84676798999678], [-73.79143363007184, 40.846864797968834], [-73.79136709858446, 40.84688764268581], [-73.79137658530533, 40.84692782104442], [-73.79138428282982, 40.846950369625105], [-73.79148637416937, 40.84695880326024], [-73.7914404968833, 40.84730604348358], [-73.79169837795163, 40.84725286942954], [-73.79169408366229, 40.84723636206886], [-73.79177232700852, 40.847228253279155], [-73.79192007100076, 40.84722851987908], [-73.79191997810095, 40.84725821803076], [-73.79174613599426, 40.84726615386965], [-73.79171569758651, 40.847272698856905], [-73.79171562526108, 40.847295797102596], [-73.79142962756488, 40.84735998917434], [-73.79141738478454, 40.8474800465621], [-73.79151739185973, 40.8474604291968], [-73.79154104882262, 40.84753801729824], [-73.79130570048282, 40.84759286747169], [-73.7912670856737, 40.84760186735403], [-73.79127720066779, 40.8476464250921], [-73.79127932953888, 40.84766309155011], [-73.79132975970043, 40.84805792306421], [-73.79160687331743, 40.848026505666404], [-73.79161765431445, 40.84805292440784], [-73.79133588063237, 40.848086238652414], [-73.79136288106794, 40.84823230239035], [-73.79131020699877, 40.84840049699906], [-73.79123086312458, 40.84850103935591], [-73.7912447526424, 40.84855222044715], [-73.79106000608506, 40.84872508882858], [-73.79111195407879, 40.848808127034175], [-73.7910972646854, 40.84881552492793], [-73.79104432321175, 40.84874079644996], [-73.79101783522134, 40.848811669064396], [-73.79091971727904, 40.84886687301587], [-73.7910136233899, 40.84898520971688], [-73.79098057835537, 40.848999999169436], [-73.7908871020705, 40.84888656430166], [-73.79024725580997, 40.849178846989886], [-73.7897571107061, 40.84953146928762], [-73.78983383329343, 40.84985897364479], [-73.78985836952175, 40.84997255936185], [-73.79016063798868, 40.849922539976426], [-73.79015633936459, 40.84990768266867], [-73.79022591071276, 40.849894609320785], [-73.79022586388017, 40.84990945922681], [-73.79064976493049, 40.849844233237555], [-73.79064762821827, 40.849832679590186], [-73.79092590534461, 40.84978203770634], [-73.79091297721088, 40.84974736613829], [-73.7909477685762, 40.84973917964379], [-73.79098877124653, 40.84982834953221], [-73.79067569819729, 40.84988882759024], [-73.79066056595522, 40.84986405098365], [-73.79044102699248, 40.84989170175116], [-73.79044531982218, 40.84990820797589], [-73.79036705822561, 40.84992126516153], [-73.79036275955572, 40.8499064078614], [-73.79023451253238, 40.84992267433129], [-73.79024098424524, 40.849937534414885], [-73.7901714089767, 40.84995225687013], [-73.79016710514148, 40.84993905016657], [-73.78986630313652, 40.84998939070316], [-73.79001758788853, 40.850316332465056], [-73.79016816439426, 40.85029213020269], [-73.7902008337725, 40.850267441648136], [-73.79020729510404, 40.850285601432425], [-73.79024642580437, 40.85027907315151], [-73.790237870093, 40.8502361610853], [-73.79028137292694, 40.850221390189404], [-73.79025767031703, 40.85015865126974], [-73.79035766981725, 40.850142334615924], [-73.7904050605431, 40.85027276271398], [-73.79014198665578, 40.85032508060762], [-73.79015490797059, 40.85036140235727], [-73.79022013429515, 40.85034832110921], [-73.79022659043213, 40.85036813216589], [-73.7900548453779, 40.85039751803697], [-73.79005271916627, 40.85038266468011], [-73.7901244688738, 40.850367946420214], [-73.79011588587774, 40.850333283173384], [-73.7900241182947, 40.85034781958286], [-73.79003888486419, 40.85045128862397], [-73.78991872482864, 40.85061172713771], [-73.79101737069524, 40.85040751477935], [-73.79102381130417, 40.85043227542412], [-73.78992953626674, 40.85062612255504], [-73.79003174223271, 40.8507685478139], [-73.79007093170758, 40.85093202010673], [-73.79024764837524, 40.850903510522876], [-73.79025006400954, 40.8509127959065], [-73.79007405920494, 40.850939715312535], [-73.790086400219, 40.85098447385347], [-73.7902646047139, 40.8509527298865], [-73.79026824507254, 40.85096108807381], [-73.79009094385734, 40.85099476440282], [-73.79012439171836, 40.851189912194556], [-73.78951371516173, 40.85130077171307], [-73.78956272464195, 40.851487336822075], [-73.78966051794421, 40.85147686132127], [-73.78966276217217, 40.851488703320925], [-73.78978510702639, 40.8514622938807], [-73.78977663032417, 40.851437219708586], [-73.78971059211985, 40.85144916458632], [-73.78970213099691, 40.851419451365594], [-73.78968379674485, 40.851419417941834], [-73.78966571913696, 40.85133864383001], [-73.78976843228764, 40.85132398106947], [-73.78977935323188, 40.85134905852415], [-73.7897952542796, 40.85134537500231], [-73.78980855115411, 40.85139180351458], [-73.78979509883112, 40.8513945632504], [-73.78981080478471, 40.85145213196693], [-73.7900578142692, 40.851414530211144], [-73.79001797053547, 40.8512594709595], [-73.79004975343726, 40.851257672809496], [-73.79008961569983, 40.85140716358974], [-73.7903451879796, 40.85136679305499], [-73.79030285187709, 40.851226578857236], [-73.7903370951807, 40.85122014385105], [-73.79037943653302, 40.851358503723944], [-73.79082454198961, 40.8512897063184], [-73.79078463524452, 40.85115413577379], [-73.79081765352956, 40.851148627155794], [-73.79085756325748, 40.85128326944465], [-73.79098717984283, 40.85126401529905], [-73.79099076649098, 40.85128907934996], [-73.7908660390277, 40.85130834236286], [-73.79089018435498, 40.8514039784428], [-73.7908596129088, 40.851408562094896], [-73.79083302265286, 40.851313850998054], [-73.79038545954286, 40.851386356467565], [-73.7904108048791, 40.851488489896646], [-73.79038756834426, 40.85149308842482], [-73.79035488894863, 40.85139094116019], [-73.79009442491898, 40.85143222989302], [-73.79011618216886, 40.851509299305945], [-73.79007338074102, 40.85151664642066], [-73.79005162420025, 40.851439577001], [-73.78966490344513, 40.851500382132095], [-73.78972573712088, 40.85169267056497], [-73.79014778434487, 40.85160740718401], [-73.79017310700394, 40.85166257915488], [-73.79089217930076, 40.85154690354888], [-73.79091633421972, 40.85163882596731], [-73.79019588667279, 40.85175290456291], [-73.79021640272948, 40.85183982678925], [-73.79024838262285, 40.85183343543782], [-73.79023273612084, 40.8517563771663], [-73.79045528741521, 40.85172337250565], [-73.790463705281, 40.85176700615398], [-73.79038421946557, 40.85177985423558], [-73.79038660320028, 40.851799348126626], [-73.79026555282694, 40.85181490584794], [-73.79026673300343, 40.851827900307505], [-73.7902948451226, 40.85182795141295], [-73.79030685108064, 40.85189665031758], [-73.79027873037282, 40.85189938394876], [-73.79029928958964, 40.851969026149405], [-73.79025566394581, 40.85197606492971], [-73.79008396904106, 40.85203452738219], [-73.79015509260996, 40.8522512852358], [-73.79024346325298, 40.852230639986765], [-73.79016095149859, 40.85204023643091], [-73.79019520313294, 40.85203101773807], [-73.79027404235792, 40.85222327109256], [-73.79053212420823, 40.85216341464512], [-73.79045200336085, 40.85198972095023], [-73.7905070423804, 40.851977756847845], [-73.79057493410312, 40.85215328413443], [-73.79080610587764, 40.85209987596774], [-73.79074309499363, 40.85192621249439], [-73.79058037559301, 40.851977889994], [-73.79056092524321, 40.85194444389136], [-73.79079463354844, 40.85186134138583], [-73.79081896780879, 40.851896652302045], [-73.79076267521064, 40.8519188235191], [-73.79079446716632, 40.85191424043077], [-73.79085380717721, 40.85208882496842], [-73.79099692237003, 40.85205288980362], [-73.79098719999038, 40.85203523963261], [-73.79111196150562, 40.85200576731114], [-73.79106594789536, 40.851868329827596], [-73.79102069040403, 40.85187938483921], [-73.79101097118439, 40.85186080525286], [-73.79117366731116, 40.851816553269394], [-73.79118338365927, 40.851836059914085], [-73.79110265321683, 40.85185633106272], [-73.7911669291687, 40.85201607489478], [-73.79118770764545, 40.8520161125067], [-73.79146293338718, 40.85194515019928], [-73.79139621768194, 40.85178354562009], [-73.79132894174195, 40.85180012825631], [-73.7913204511751, 40.85177969609417], [-73.79151738677945, 40.85172993589083], [-73.79152343356945, 40.85175036429258], [-73.79142679645518, 40.85177617591716], [-73.79149596814213, 40.85193407242131], [-73.79176994883002, 40.851870530951366], [-73.79172750188745, 40.851764653959144], [-73.79227187668211, 40.851610648258074], [-73.79226217075832, 40.8515874297298], [-73.79248359357182, 40.851523791406876], [-73.79249450978077, 40.85155072509265], [-73.79229143993996, 40.8516078992365], [-73.79229872412927, 40.85162369008448], [-73.79176780605944, 40.851774007706055], [-73.79180175231375, 40.851862235109934], [-73.79204882760328, 40.85180328466126], [-73.79205731457873, 40.8518255732624], [-73.79016350155166, 40.85227183108285], [-73.79017007126295, 40.852280661153415], [-73.79113795170926, 40.85205585257368], [-73.79120559068458, 40.85219774978446], [-73.79133456753253, 40.85216606767367], [-73.7913418457985, 40.85218348288131], [-73.79121251169283, 40.85221517641534], [-73.79122653385119, 40.8522580105032], [-73.79136187979904, 40.85222597753384], [-73.7913646086994, 40.852232943085355], [-73.79123084007696, 40.85226480653024], [-73.79123655971756, 40.852280289886465], [-73.79318284075846, 40.85180857187128], [-73.79308901965723, 40.851584429758056], [-73.79311837183832, 40.851578295240834], [-73.79322026048357, 40.85182843708702], [-73.79315339612619, 40.85184440453376], [-73.7932525807951, 40.851916352733845], [-73.7932378678764, 40.85193117467445], [-73.7931435368942, 40.8518703721512], [-73.79312894073925, 40.851848072675836], [-73.79124556783773, 40.85230468141072], [-73.79138052695514, 40.8526020383047], [-73.79340960300236, 40.85210987866148], [-73.793383767393, 40.85203283608551], [-73.7934098763117, 40.852021883730366], [-73.79350762831913, 40.852264043924805], [-73.793478629143, 40.85227279103696], [-73.79342400669049, 40.85213630252753], [-73.79139106468679, 40.85262807498589], [-73.79144640027788, 40.85277157192412], [-73.79042053780161, 40.85302297821258], [-73.79049879362735, 40.853188722518674], [-73.79077127595484, 40.853118526990635], [-73.79079150757708, 40.853163399592205], [-73.79100586740225, 40.85311769126193], [-73.79098982724433, 40.853035992826676], [-73.79100777755542, 40.85302860035525], [-73.79103033114585, 40.853111549592796], [-73.7914543630373, 40.853010847713506], [-73.79138972495281, 40.85283625393709], [-73.79143702716479, 40.85282272848614], [-73.79150327251574, 40.85300474895965], [-73.79170223896095, 40.85295808594555], [-73.79163107691939, 40.852784718525385], [-73.79165554436234, 40.8527773382265], [-73.79172995524623, 40.85295442349012], [-73.79196318311136, 40.85289544822876], [-73.7918936538656, 40.852720846583026], [-73.79191649946218, 40.85271098813851], [-73.79198928164561, 40.852888070297965], [-73.79222413063728, 40.852831572094274], [-73.79215297057404, 40.85265696816939], [-73.79217580837809, 40.852649584321576], [-73.79225186097551, 40.85282296038217], [-73.79248669881495, 40.85277017395219], [-73.79241717582832, 40.852593097969546], [-73.79243675473344, 40.85258570820603], [-73.79250954048048, 40.852761552714384], [-73.79283083884894, 40.852680460767814], [-73.79276455802382, 40.85250833946967], [-73.79281511966936, 40.852494819308916], [-73.79290086687061, 40.85269543598237], [-73.79283890218414, 40.85270769837338], [-73.79080758729923, 40.85319906006435], [-73.79086201986841, 40.85331978110344], [-73.7911143142938, 40.85326445192463], [-73.79126573165874, 40.853706864233104], [-73.79252408728894, 40.85340992070495], [-73.79238610589364, 40.85306209388165], [-73.79240931036925, 40.85305333611478], [-73.79255598392581, 40.85340117854083], [-73.79297928311027, 40.85330294490942], [-73.79284710092516, 40.852952929956025], [-73.79287610048584, 40.8529441813284], [-73.7930399117026, 40.85337124928513], [-73.79301381632482, 40.853377802403955], [-73.792999480554, 40.85332937972204], [-73.79127368128135, 40.85373220788165], [-73.79129041006203, 40.85384457425317], [-73.79105721254825, 40.85391841242664], [-73.79128417842017, 40.85386107898408], [-73.79129023495192, 40.853878722994494], [-73.79245816844812, 40.85360070875487], [-73.79248836752521, 40.853671708909346], [-73.79246226638655, 40.853679911499874], [-73.79244069248779, 40.85363037509783], [-73.79107002765942, 40.85394495666465], [-73.79108281745684, 40.853963197035256], [-73.79138477253804, 40.853895478673856], [-73.7915836098969, 40.85389088823204], [-73.79171835293754, 40.85391732878054], [-73.7917861117569, 40.853993539900856], [-73.7918178199531, 40.853989436670055], [-73.79181716188373, 40.85398003873739], [-73.79251418368892, 40.853896308270706], [-73.79250616578281, 40.85385422192266], [-73.79253714604873, 40.85384932781285], [-73.79256758944628, 40.854016434943034], [-73.79253009854688, 40.8540188426912], [-73.79251736175938, 40.85392229994337], [-73.7918261216321, 40.85400148723032], [-73.79182140042258, 40.85399695489827], [-73.79179366809406, 40.85400482990041], [-73.7919181173638, 40.854222354742845], [-73.79133811791583, 40.85443901153831], [-73.79129282160633, 40.854584025976855], [-73.79186843464552, 40.85541675186694], [-73.79207675667554, 40.8556451677071], [-73.79250306685435, 40.855788666137045], [-73.79249297838457, 40.85589406348061], [-73.7924879115736, 40.85594699320981], [-73.79241821541393, 40.855995344340634], [-73.79230720796731, 40.856072355408216], [-73.79211616948511, 40.85601261529166], [-73.79203322933732, 40.85614052347183], [-73.79201712228527, 40.85617694860626], [-73.79257074199624, 40.85620600721377], [-73.79256944485117, 40.85623013583762], [-73.79263787674974, 40.85623582617103], [-73.7926453206665, 40.85620057278915], [-73.79270523985137, 40.85619232890572], [-73.79268768222656, 40.856335218720375], [-73.79311537800541, 40.85637218147505], [-73.79311911983078, 40.85634805890509], [-73.79315700296156, 40.85635091107476], [-73.7931468608596, 40.856467828471146], [-73.79310775533048, 40.8564649741061], [-73.79311527696748, 40.85640466330657], [-73.79261793796836, 40.856358294907594], [-73.79263294381181, 40.856249739029394], [-73.79256817599392, 40.85624498185471], [-73.7925681412599, 40.85625611876997], [-73.79201050009051, 40.85622512771836], [-73.79198719874023, 40.85636551214246], [-73.79193700279743, 40.85640515792216], [-73.79207000927919, 40.85641496516156], [-73.79205892560435, 40.85644139483644], [-73.7920974020609, 40.85644981691218], [-73.79208814819948, 40.85647764200938], [-73.79206981995775, 40.85647552247019], [-73.79205778470703, 40.85651308623457], [-73.79201746383625, 40.85650814112617], [-73.79200911229961, 40.856540144209966], [-73.79190832251761, 40.85652395422946], [-73.7919018334025, 40.85654691136797], [-73.7930513380812, 40.8567259611496], [-73.79306886680175, 40.85667979474308], [-73.79309492152382, 40.85668644140406], [-73.79305685295346, 40.85681616547445], [-73.7930307836123, 40.856813917951534], [-73.7930483589582, 40.856752353857736], [-73.79189422958875, 40.85656888932964], [-73.79185937713562, 40.85666592438058], [-73.7918421311268, 40.85666118333631], [-73.7918284550686, 40.8566681433947], [-73.79196180591408, 40.856716856585116], [-73.79195628211025, 40.85672450329606], [-73.79182299976162, 40.85667610751278], [-73.79179264535283, 40.85671985281914], [-73.79192955714267, 40.85676900166847], [-73.79192219262924, 40.85677873324112], [-73.79178502817685, 40.8567308579451], [-73.79170704257359, 40.85685184285503], [-73.79186032552957, 40.85679280353208], [-73.7918822302671, 40.85682416512516], [-73.79183633462834, 40.85684287530318], [-73.79183269744581, 40.85683312505861], [-73.79173546476552, 40.856866429528196], [-73.79182143771003, 40.85690024255173], [-73.79187172581227, 40.85698428896384], [-73.79220265950248, 40.857146000410324], [-73.79222842636995, 40.857114724238784], [-73.79221469914035, 40.857107042845776], [-73.79237484012394, 40.85690617253139], [-73.7924279413047, 40.856928542536735], [-73.79226687280277, 40.85713289038608], [-73.79224490126386, 40.85712310596905], [-73.79222189186365, 40.85715229842467], [-73.79273719867784, 40.85739366621669], [-73.79282713237977, 40.857435789917886], [-73.79289753888773, 40.85718939649052], [-73.79295400784763, 40.85699178335115], [-73.79301264383005, 40.857003026043145], [-73.792996284781, 40.85706359655042], [-73.79288403888904, 40.85747920249362], [-73.79281074586505, 40.85746422142733], [-73.79271730383812, 40.85741901131974], [-73.79185978715205, 40.857004107259364], [-73.79185432641995, 40.85701379719813], [-73.79268338575463, 40.85742039359269], [-73.79260923020631, 40.857507704502304], [-73.7925766962139, 40.85748784672487], [-73.7926355732806, 40.85742195660253], [-73.79211266783624, 40.85716693187042], [-73.79189451913076, 40.85743712186523], [-73.792465160653, 40.857715331784], [-73.79254150089487, 40.85762472518378], [-73.79255162803088, 40.85763007584013], [-73.79256970440578, 40.857639625362246], [-73.79239521723282, 40.8578455477522], [-73.79238725639992, 40.857839475267994], [-73.79237353813318, 40.85782901033757], [-73.79244336403542, 40.85773674197192], [-73.79174254082349, 40.85739395000424], [-73.7917578139993, 40.85737417945399], [-73.7918728196394, 40.85742718307675], [-73.79209332504095, 40.857157989947204], [-73.79184303217382, 40.85702870201135], [-73.7917397997259, 40.8571783542773], [-73.79160393461318, 40.85737531299246], [-73.79129378852147, 40.85726323869815], [-73.79094202260367, 40.857552748188766], [-73.79123307405831, 40.85768657030246], [-73.79124452095031, 40.857674217379795], [-73.79131613460314, 40.85770528253102], [-73.79130630096441, 40.85772258820141], [-73.7916366458796, 40.85788281257089], [-73.79175600351645, 40.85776052352311], [-73.7917852942325, 40.857775425855564], [-73.79159890915145, 40.85796317755999], [-73.79143695908105, 40.85815468512309], [-73.79140929150854, 40.85814226029774], [-73.79161866075827, 40.8579013420481], [-73.79129321642519, 40.857737414000304], [-73.79127850954808, 40.85774976102998], [-73.79121179649877, 40.85771499223178], [-73.79122161784531, 40.857701399039705], [-73.79093222800743, 40.857567787470536], [-73.79079067047185, 40.85775035081783], [-73.79068061700995, 40.8578027903298], [-73.7904419014552, 40.85791653409865], [-73.79023592738741, 40.85815408481125], [-73.79006133563223, 40.858459548561314], [-73.79012538798082, 40.858522102342256], [-73.79013640468453, 40.85851724995587], [-73.79016016469429, 40.85854165463286], [-73.79014637788677, 40.858552767033466], [-73.79020122975753, 40.858601589710446], [-73.79018745194999, 40.8586099173861]]], [[[-73.78283291447852, 40.85587030844573], [-73.78302371522504, 40.85509276666495], [-73.78367819665118, 40.85517343750966], [-73.78394699722007, 40.85563043752662], [-73.78343487501385, 40.856362865958886], [-73.78283291447852, 40.85587030844573]]], [[[-73.78061730829718, 40.855735175810025], [-73.78090476851291, 40.8549681331623], [-73.7815638969994, 40.855001286433904], [-73.78156303268149, 40.85574479600088], [-73.78122782332017, 40.856087789162764], [-73.78061730829718, 40.855735175810025]]], [[[-73.78833349834521, 40.834667129759346], [-73.78931223606624, 40.83446488655343], [-73.78951019872322, 40.835364042525754], [-73.78845700015209, 40.835309914315395], [-73.78833349834521, 40.834667129759346]]]]}}, {\"id\": \"46\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 47, \"Shape_Leng\": 0.0898275563294, \"Shape_Area\": 0.000163198117339, \"zone\": \"Claremont/Bathgate\", \"LocationID\": 47, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89090203399996, 40.854116885999865], [-73.89216687399981, 40.85276946199989], [-73.89307140900002, 40.85324998699989], [-73.89431774099977, 40.851891310999854], [-73.89478764999994, 40.85108166699989], [-73.89407258999991, 40.85071318999995], [-73.89494627999981, 40.849104852999865], [-73.89557755999984, 40.847854641999895], [-73.89611702199987, 40.846778518999926], [-73.89617712999998, 40.8466583209999], [-73.89535848999982, 40.84651803199993], [-73.89433599299984, 40.846486680999874], [-73.8940930149999, 40.84648822299994], [-73.89384985999999, 40.84647439399987], [-73.89360906599997, 40.84644525599987], [-73.89337309899989, 40.84640119399996], [-73.89290102299996, 40.84626832999992], [-73.89398467599997, 40.84470395899989], [-73.89464252699996, 40.84390413999996], [-73.89479022099985, 40.8437203219999], [-73.89481942600001, 40.84368377099985], [-73.894847588, 40.84364636299994], [-73.89537836199995, 40.842938778999965], [-73.89564979700005, 40.843035539999896], [-73.89591487099987, 40.843142918999895], [-73.8961725309998, 40.8432605479999], [-73.89642177099996, 40.84338795899991], [-73.89666161699988, 40.84352460699992], [-73.89749111599987, 40.84206708899992], [-73.89826966299981, 40.84067994699995], [-73.89903034699984, 40.839330476999876], [-73.89996792899998, 40.8376547629999], [-73.90004236899989, 40.837522574999895], [-73.90014268099992, 40.8373444399999], [-73.90019045299977, 40.837259606999964], [-73.90022284799984, 40.83720207899991], [-73.9002588729999, 40.83713810599989], [-73.90030702699993, 40.837052593999914], [-73.90035730100003, 40.83696331799991], [-73.90038944699995, 40.83690623099993], [-73.90056610899994, 40.83659250799993], [-73.90122534999992, 40.83549991099991], [-73.90134101199988, 40.83544132399986], [-73.90185605899981, 40.83462212499988], [-73.90343501499996, 40.83221638099998], [-73.90367608499999, 40.83185824799992], [-73.90397692599986, 40.83141130499991], [-73.90450621899981, 40.8306249629999], [-73.90553116799988, 40.82910413299989], [-73.90657083599997, 40.829452219999936], [-73.90747341000004, 40.829743308999916], [-73.907527588, 40.82976078199988], [-73.90772165099985, 40.829823367999914], [-73.908959903, 40.83022623299989], [-73.909144659, 40.83028572999996], [-73.90971373799982, 40.83047001399989], [-73.91049010699992, 40.830737372999884], [-73.90962130699984, 40.832281421999895], [-73.9095189569999, 40.83246377699987], [-73.90946039499991, 40.83256811599989], [-73.90942257599983, 40.83263549599993], [-73.9093741529999, 40.832721771999914], [-73.90928354899982, 40.83288319599993], [-73.90923681499991, 40.83296646099988], [-73.909177781, 40.83307163899986], [-73.90910597799993, 40.833199565999905], [-73.90889804099996, 40.833570034999894], [-73.90882575999996, 40.8336988109999], [-73.90869131200002, 40.83393834699988], [-73.90857547999998, 40.834144441999904], [-73.90852151699984, 40.834240455999904], [-73.90837656, 40.83449837099991], [-73.90823980199985, 40.83474169399996], [-73.90810870199985, 40.83497494999988], [-73.90799132699986, 40.8351837829999], [-73.90797135199996, 40.83521932399989], [-73.90783293099987, 40.83546559899987], [-73.9077182169999, 40.835669695999854], [-73.90762665999985, 40.835832588999935], [-73.90757269399987, 40.83592860399989], [-73.90740200499981, 40.836232282999966], [-73.90733306999991, 40.83634242699987], [-73.90727112599996, 40.83644139899997], [-73.90721294199983, 40.83653436399994], [-73.90713954499994, 40.83665163499991], [-73.906975505, 40.836913727999885], [-73.90688497199997, 40.83705837699988], [-73.90679878899991, 40.837196073999976], [-73.90652354, 40.83763584299988], [-73.90646169600002, 40.83773464999995], [-73.90635674699993, 40.837902326999924], [-73.90625475300006, 40.83806527999994], [-73.90597940399996, 40.83850519599985], [-73.90538357999992, 40.839768982999885], [-73.90560739499988, 40.8398436189999], [-73.90548026199987, 40.840052591999935], [-73.90536832399985, 40.84026688999992], [-73.905272123, 40.8404855589999], [-73.90519207999992, 40.840707581999915], [-73.90515655599995, 40.84104021299991], [-73.90503176699985, 40.8422629449999], [-73.9050151499998, 40.842404697999875], [-73.90493735799986, 40.84337489399996], [-73.90484645099987, 40.84458935999987], [-73.90484219199982, 40.844665447999944], [-73.90483494099993, 40.844752618999976], [-73.90483121999989, 40.84480034199987], [-73.9041010739999, 40.8447308219999], [-73.90338651199983, 40.844655295999914], [-73.9026881009999, 40.84458147199991], [-73.901344046, 40.84442875099994], [-73.9013172989999, 40.844473940999926], [-73.90126635699991, 40.844562217999915], [-73.90088731499989, 40.84524135999991], [-73.90020067899994, 40.84647356099991], [-73.89967790699988, 40.84742465799992], [-73.89887482599984, 40.84881118399988], [-73.89808598699996, 40.850213123999914], [-73.89708706499987, 40.85196827699985], [-73.89662446799987, 40.8527875899999], [-73.89560883299981, 40.85457036999986], [-73.89549889799987, 40.85474976599994], [-73.89477596099992, 40.85592952599996], [-73.89441355099994, 40.85643377599997], [-73.8929935429999, 40.85838611999985], [-73.89225707599982, 40.85940997599994], [-73.89174394999986, 40.860177146999945], [-73.89115375599981, 40.86091361599996], [-73.89106059999996, 40.861471814999945], [-73.89106280699994, 40.8615294119999], [-73.89082169799998, 40.86139939599989], [-73.89067455199995, 40.86132004099988], [-73.89036823899993, 40.86115807599985], [-73.89027555599993, 40.86110907299996], [-73.89059489199987, 40.86071569999991], [-73.89112568799987, 40.860061824999924], [-73.89112461399988, 40.8588700059999], [-73.89111943300001, 40.85865896199989], [-73.89111115400003, 40.85838432199987], [-73.89111318899985, 40.857850998999865], [-73.8911055689998, 40.85742738299995], [-73.89109710400002, 40.856779813999886], [-73.89108676599993, 40.85659725699984], [-73.8907742849999, 40.85581193999993], [-73.89044259299995, 40.85504635099993], [-73.89038953999983, 40.85468905799996], [-73.8906903199999, 40.854353261999954], [-73.89090203399996, 40.854116885999865]]]}}, {\"id\": \"47\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 48, \"Shape_Leng\": 0.0437467441431, \"Shape_Area\": 9.42538425377e-05, \"zone\": \"Clinton East\", \"LocationID\": 48, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99117738199989, 40.75532984899992], [-73.99163271299996, 40.75470680499996], [-73.99446824699999, 40.75590886199991], [-73.99500601599988, 40.756136713999865], [-73.99511286399995, 40.75617577399984], [-73.99577488199998, 40.75645451499985], [-73.99730671499998, 40.757110506999894], [-73.997233738, 40.7572095429999], [-73.99692177899989, 40.75763758199988], [-73.99685244899985, 40.75773270899988], [-73.996800897, 40.757789159999945], [-73.99663529299985, 40.75801292199992], [-73.99640941599995, 40.75833915899993], [-73.9959423659999, 40.758980978999965], [-73.99547450199992, 40.759626856999894], [-73.99498358099996, 40.76030016199997], [-73.9945259329999, 40.7609270449999], [-73.99406959, 40.76155258499991], [-73.99360930599991, 40.76218343599989], [-73.99315623899986, 40.7628063609999], [-73.9927014349998, 40.76343080599987], [-73.99224402800002, 40.764056778999894], [-73.99178834099983, 40.76468386799991], [-73.99133264599988, 40.765309269999904], [-73.99087599499987, 40.7659342209999], [-73.99042167999981, 40.76656125799984], [-73.98996382699985, 40.767187474999886], [-73.98950498200003, 40.767815381999874], [-73.98905077199991, 40.768440091999906], [-73.98855898599986, 40.7691158169999], [-73.98806289599996, 40.769790799999925], [-73.98521720299992, 40.76859462999991], [-73.98236545099999, 40.7673921529999], [-73.9828771349999, 40.766715594999866], [-73.98336657899995, 40.766045241999926], [-73.98382185700001, 40.7654184749999], [-73.98428230099987, 40.76479181499994], [-73.98473393799985, 40.76416910499991], [-73.98519152199998, 40.76354367499993], [-73.98564677799999, 40.76291487399988], [-73.98610406199981, 40.762289952999865], [-73.98656138499996, 40.7616601219999], [-73.98701514599995, 40.76103736699989], [-73.987475247, 40.76040478699991], [-73.98792640699999, 40.759790276999865], [-73.98839044599993, 40.75916418599996], [-73.98884253899993, 40.75853349899991], [-73.98929727499991, 40.7579066369999], [-73.98979085399996, 40.7572331319999], [-73.99027621399986, 40.756567564999926], [-73.99072636299992, 40.75595040599995], [-73.99117738199989, 40.75532984899992]]]}}, {\"id\": \"48\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 49, \"Shape_Leng\": 0.0752900183427, \"Shape_Area\": 0.000203185525471, \"zone\": \"Clinton Hill\", \"LocationID\": 49, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96062056499981, 40.69632013999993], [-73.96015854699988, 40.69411730899989], [-73.95931047999983, 40.69421508699989], [-73.95799732999993, 40.694365838999914], [-73.95708701900001, 40.69447043999992], [-73.95614239299985, 40.694579018999896], [-73.95582662799985, 40.692992382999925], [-73.95541057999996, 40.69090829199994], [-73.95635602999991, 40.69079978199989], [-73.95727249099987, 40.69069347799987], [-73.95808563499989, 40.690601463999876], [-73.95818048699985, 40.690590729999876], [-73.95857844799987, 40.69054569399986], [-73.95886713600001, 40.69051224799996], [-73.9589933049999, 40.69049762899987], [-73.95928133699987, 40.69046425699991], [-73.959348624, 40.69045646099992], [-73.95942791399997, 40.69044727499993], [-73.95957591899987, 40.69042998799992], [-73.96008336799996, 40.69037071599989], [-73.96013760799985, 40.69036437999989], [-73.96029281699998, 40.69034624999995], [-73.96023740299987, 40.690062221999874], [-73.96022304499992, 40.6899886269999], [-73.96018691899992, 40.6898034559999], [-73.96017256100002, 40.68972986199988], [-73.96012172900004, 40.689469306999825], [-73.96009714599994, 40.68934520999991], [-73.96000519799986, 40.688881033999955], [-73.959859395, 40.6881474509999], [-73.95971374799981, 40.687415633999926], [-73.95956770199984, 40.686682554999955], [-73.95684165199995, 40.68699607899989], [-73.95468418899998, 40.68724485399985], [-73.95453798699995, 40.686511174999936], [-73.95439296899998, 40.68577971999988], [-73.95424647699987, 40.685046247999914], [-73.95410042600001, 40.68431310799984], [-73.953954531, 40.683580778999946], [-73.95380893500004, 40.68284800799992], [-73.95366256199999, 40.68211490399989], [-73.95351616899988, 40.6813825999999], [-73.9533701749999, 40.680640507999904], [-73.95328187399996, 40.68020003099985], [-73.95436836300001, 40.680075106999915], [-73.95543681599986, 40.67995785499992], [-73.95530907499989, 40.67927313499995], [-73.95529361599996, 40.679203150999825], [-73.9559586239999, 40.679346246999906], [-73.95829216799997, 40.67983113299993], [-73.96134655899995, 40.6804635239999], [-73.96329205499984, 40.680881585999884], [-73.96442184899986, 40.681119287999934], [-73.96447930699999, 40.681131147999956], [-73.965510434, 40.6813440199999], [-73.96657247099982, 40.68156481399995], [-73.96753196799997, 40.68176878399992], [-73.96868961400001, 40.68200525799988], [-73.96867398499984, 40.68208599399989], [-73.96906202000002, 40.68415736099991], [-73.96948939899981, 40.68628830699986], [-73.9697831419999, 40.68775161199994], [-73.97013154199992, 40.68950630799986], [-73.96910362499993, 40.68946752199986], [-73.96952115199994, 40.69156355899989], [-73.96983118499983, 40.69313424399997], [-73.96880989600005, 40.69312038199996], [-73.9693409769999, 40.69576842999993], [-73.96935567900005, 40.69585472899994], [-73.9693719569999, 40.695937759999936], [-73.96937861700003, 40.695991333999906], [-73.9693945009999, 40.696080388999846], [-73.96840655900004, 40.6960364839999], [-73.96741363299996, 40.69599862899988], [-73.96641979599997, 40.69596410599996], [-73.96546721499983, 40.695932161999956], [-73.9645507049999, 40.696107591999855], [-73.96434801500007, 40.69618376599989], [-73.96415417899982, 40.69627444999987], [-73.963971858, 40.69637865499991], [-73.963803372, 40.69649503799992], [-73.96365073599986, 40.69662194399993], [-73.96347515199992, 40.69680066799991], [-73.96331636199999, 40.69698969799986], [-73.96317583599986, 40.69718762099991], [-73.96305480199987, 40.69739287199992], [-73.96295416199992, 40.69760374899989], [-73.96288219599991, 40.69782702599988], [-73.96283963699993, 40.698038667999946], [-73.96248295899996, 40.698095383999956], [-73.9623976909999, 40.69810894499991], [-73.96235903300003, 40.69811509199996], [-73.96233072899996, 40.698119593999884], [-73.96214289299981, 40.69814945699991], [-73.96189837799996, 40.69818833299994], [-73.96105100699992, 40.69832607899989], [-73.96092543799982, 40.697736506999895], [-73.96062056499981, 40.69632013999993]]]}}, {\"id\": \"49\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 50, \"Shape_Leng\": 0.0557479867536, \"Shape_Area\": 0.000173249283326, \"zone\": \"Clinton West\", \"LocationID\": 50, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99667705784736, 40.77351005394221], [-73.99434191991617, 40.77252478194338], [-73.9942497922864, 40.7725264851381], [-73.99416328456307, 40.77248719152385], [-73.99383108201548, 40.77293178742534], [-73.9937873699999, 40.77290384599992], [-73.99372534699988, 40.772864197999915], [-73.9935908639999, 40.77279007999994], [-73.99350262199991, 40.772751632999885], [-73.99387445099994, 40.77223816799994], [-73.99378189799992, 40.77219909099988], [-73.99365606199987, 40.772145961999854], [-73.990913076, 40.770987758999894], [-73.99059644699987, 40.77085480999995], [-73.98806289599996, 40.769790799999925], [-73.98855898599986, 40.7691158169999], [-73.98905077199991, 40.768440091999906], [-73.98950498199986, 40.76781538299985], [-73.98996382699985, 40.767187474999886], [-73.99042167999981, 40.76656125799984], [-73.99087599499987, 40.7659342209999], [-73.99133264599988, 40.765309269999904], [-73.9917883399999, 40.76468386799991], [-73.99224402800002, 40.764056778999894], [-73.9927014349998, 40.76343080599987], [-73.99315623899986, 40.7628063609999], [-73.99360930599991, 40.76218343599989], [-73.99406959, 40.76155258499991], [-73.9945259329999, 40.7609270449999], [-73.99498358099996, 40.76030016199997], [-73.99547450199992, 40.759626856999894], [-73.99623223099987, 40.759946031999924], [-73.99831474500002, 40.76082320899991], [-74.00103587499999, 40.76196835199993], [-74.00115054199988, 40.762025811999884], [-74.00127796299991, 40.76207582499987], [-74.00133748499992, 40.76220852199988], [-74.00153309083484, 40.762644598063744], [-74.00375735018837, 40.763582241820224], [-74.00361214797076, 40.76377704395487], [-74.00138603698596, 40.76285009500187], [-74.0009730802201, 40.763420271725614], [-74.00314976988997, 40.764332888184576], [-74.00294809515721, 40.76461683643388], [-74.00306161234062, 40.764665782653374], [-74.00313458336218, 40.76464959820679], [-74.00316194577034, 40.76461261045457], [-74.00322275585351, 40.7646380376702], [-74.00315587058185, 40.76473050988001], [-74.00314652967363, 40.76473628139182], [-74.00313606019984, 40.76474079644633], [-74.00312475155106, 40.76474393024101], [-74.00311291631381, 40.76474559615368], [-74.00310088163067, 40.7647457481358], [-74.0030889801568, 40.764744381986716], [-74.00106364220507, 40.763888208262884], [-74.00073105253733, 40.76375528636147], [-73.99992712367724, 40.76340652847929], [-73.99963853555366, 40.76356907099078], [-73.99962263458193, 40.76355908121048], [-73.99951767096813, 40.76370627918037], [-73.99944642217294, 40.76380619457846], [-73.99943330724624, 40.76382391019702], [-73.99915098087908, 40.76420525958265], [-74.00227436642368, 40.76552518109951], [-74.00228802371483, 40.76556154429785], [-74.00205713831559, 40.76587739041855], [-74.0020245433034, 40.76589326657037], [-73.99919997246725, 40.764701914982496], [-73.9991083256454, 40.76466249985751], [-73.99888473174974, 40.76456937704858], [-73.99857638530527, 40.76499860727083], [-73.99859978707626, 40.76507334028592], [-73.99863487876338, 40.765085794737736], [-73.99866530473436, 40.7651044807472], [-73.99868637477196, 40.76513739426769], [-73.99869573037279, 40.76517476567959], [-73.99868870694561, 40.765208570449786], [-73.99866764711366, 40.765237035342864], [-73.99863371556073, 40.76526016874122], [-73.99858924105148, 40.765276178806765], [-73.99848276468678, 40.76529753698438], [-73.9984383037147, 40.765316213833856], [-73.99840202518257, 40.7653509090876], [-73.99827090308345, 40.76555223341328], [-73.99828160709528, 40.765567775437326], [-74.00154217871193, 40.76694470253266], [-74.00157335385273, 40.766977192210646], [-74.0015618057547, 40.766996499368275], [-74.00139437159504, 40.76723004773297], [-74.0013643461298, 40.76724936447367], [-74.00131469635994, 40.76725112004992], [-74.00129391014984, 40.76724145824158], [-73.99815678438965, 40.76589403537412], [-73.99814818647361, 40.76590560707495], [-73.9980667590535, 40.76587188113572], [-73.99772134825699, 40.76634912055583], [-73.99737808447954, 40.76682338637958], [-74.00062837395299, 40.76820311105873], [-74.00065016786208, 40.768220374289214], [-74.00064950530414, 40.76824523610559], [-74.00045489336712, 40.76850419517739], [-74.000417653932, 40.768512479964485], [-74.00038586489072, 40.76850419542923], [-73.99715588949206, 40.76712869617278], [-73.99681598463503, 40.76760062185919], [-73.9964571540504, 40.76809881579563], [-73.999728191901, 40.769473506633716], [-73.99973590247859, 40.76950617733152], [-73.99955518930786, 40.769750807740046], [-73.99951552161599, 40.7697675658378], [-73.99947253734007, 40.76976421496914], [-73.99707176039315, 40.768732643576804], [-73.99702628821062, 40.76879551562384], [-73.99715035477142, 40.76884738771338], [-73.99701071413831, 40.76902879065363], [-73.99689311312365, 40.76897962881938], [-73.9965712643956, 40.76942456827389], [-73.9990268283291, 40.7704632729821], [-73.9987620691431, 40.7708250567092], [-73.99631327022087, 40.76979883633644], [-73.99588004351683, 40.77040057901223], [-73.99601693196188, 40.77045484888618], [-73.99590077425023, 40.77061312350089], [-73.99576733283615, 40.77055459645645], [-73.99542474436292, 40.77103161516795], [-73.99575648212907, 40.77116066846575], [-73.99570859069954, 40.77122581253298], [-73.99579774020465, 40.77128256276402], [-73.99581687579034, 40.77125651641553], [-73.9959128360384, 40.771297855623835], [-73.99592803670667, 40.771275102837656], [-73.99600175749052, 40.77130746533058], [-73.99589990100937, 40.77145029858209], [-73.99589260894868, 40.771447087527264], [-73.99585437252816, 40.77149918074849], [-73.99579117748759, 40.77147551882039], [-73.99582805886249, 40.771421667222434], [-73.99572993314403, 40.77137881879378], [-73.99575123412814, 40.77135428277978], [-73.99562982240303, 40.77132529262805], [-73.99562306904355, 40.77133695751285], [-73.99530566365368, 40.77119600490955], [-73.99512395556876, 40.77145073826708], [-73.99499014482059, 40.77144070980647], [-73.99495733353699, 40.771449190697716], [-73.99493501658995, 40.77146814657798], [-73.99492385794417, 40.77148560593743], [-73.99492582120644, 40.771516042405466], [-73.99500915438713, 40.771607339944175], [-73.99497830771192, 40.771645249473266], [-73.99497688494725, 40.77165877492327], [-73.99711218538603, 40.77255323669952], [-73.99690946902788, 40.77283294565996], [-73.99477407930827, 40.77192954753019], [-73.99444501293378, 40.77237712658826], [-73.99545514995584, 40.772802675029084], [-73.9955449252083, 40.77267481382616], [-73.99644894667618, 40.77305684199676], [-73.99635558854986, 40.77317889215508], [-73.9962297200193, 40.77312897504538], [-73.99614329133088, 40.77324754156344], [-73.99656159477641, 40.773410941950964], [-73.99668544431172, 40.77350368804627], [-73.99667705784736, 40.77351005394221]]]}}, {\"id\": \"50\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 51, \"Shape_Leng\": 0.0953613442277, \"Shape_Area\": 0.000395756553505, \"zone\": \"Co-Op City\", \"LocationID\": 51, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82091617199997, 40.86891693399989], [-73.81940759199996, 40.867999270999874], [-73.81938761399987, 40.867986901999856], [-73.81578764099999, 40.8657580609999], [-73.81641455899984, 40.8651514979999], [-73.81699760299986, 40.864459030999946], [-73.8168449589999, 40.864306792999926], [-73.8176165129999, 40.86362634299998], [-73.81884300999991, 40.862568209999885], [-73.82037552999996, 40.86103300499993], [-73.8245057509999, 40.85976402099993], [-73.82444012099991, 40.85950138399989], [-73.82632571099997, 40.85929799699995], [-73.82699653199998, 40.85908778599986], [-73.82714556299992, 40.85923239499992], [-73.82720794099991, 40.85941373899988], [-73.82732348799999, 40.859686886999896], [-73.82741458099993, 40.85995987299994], [-73.82759107999998, 40.86007855099994], [-73.82787032399982, 40.860678212999915], [-73.82799444999982, 40.860840185999905], [-73.82826796599981, 40.860837332999935], [-73.82834347399992, 40.86089202599988], [-73.82840758999997, 40.860948300999894], [-73.82954731199996, 40.862164086999925], [-73.83022388499991, 40.863429527999884], [-73.83054182799988, 40.86373143499994], [-73.83175316499981, 40.8659067679999], [-73.83255828799996, 40.86728777299991], [-73.83312947400005, 40.86826747399991], [-73.83315960399985, 40.86831975599994], [-73.8334796669999, 40.86821612199987], [-73.83354911599997, 40.868193637999916], [-73.83391246099991, 40.868075988999884], [-73.8348179389999, 40.86778172199993], [-73.83565766499987, 40.86751275599992], [-73.83655628099997, 40.86935217599989], [-73.83740181399989, 40.87110892899994], [-73.83829221499984, 40.87295133899995], [-73.83746027699993, 40.87322264999988], [-73.8383540939998, 40.87492195799993], [-73.83924788999988, 40.876618737999905], [-73.83841016399981, 40.87686761499991], [-73.8381484219999, 40.876953298999915], [-73.83855197999986, 40.87711952799991], [-73.83925549199988, 40.87741982299989], [-73.8395648579999, 40.87754981599993], [-73.839711891, 40.8776115949999], [-73.8401480179999, 40.8777884259999], [-73.84114053199991, 40.878215735999945], [-73.84228241899993, 40.878701972999906], [-73.84239559299994, 40.87875039299991], [-73.84324113799987, 40.879108912999904], [-73.8427378519999, 40.87942490299989], [-73.84201454899988, 40.87987225999992], [-73.84127784699999, 40.8803175479999], [-73.84046213299987, 40.880622867999875], [-73.83965170899998, 40.88092595799992], [-73.83874273699996, 40.881265967999916], [-73.83782792499997, 40.88160813999987], [-73.8369874379998, 40.88193908299993], [-73.83664134299984, 40.88212002499992], [-73.83621837399996, 40.8823930119999], [-73.83560954999997, 40.88294527899995], [-73.83475583399996, 40.8835562939999], [-73.83434542099987, 40.88385001999993], [-73.83404698799987, 40.883990319999945], [-73.83361799799997, 40.88418563099992], [-73.8334224639999, 40.88427528899988], [-73.83342222600004, 40.8842753959999], [-73.8334219759999, 40.88427549499986], [-73.83321720299985, 40.88435431699989], [-73.83321711999992, 40.88435435299993], [-73.83321703599982, 40.88435437999986], [-73.83300454299992, 40.884421634999974], [-73.83300425899994, 40.88442172499991], [-73.83300396199995, 40.88442179699992], [-73.83278561399996, 40.88447683399994], [-73.83278538799992, 40.8844768879999], [-73.83278516199996, 40.88447693199989], [-73.83256256300001, 40.884519400999864], [-73.83256233799996, 40.884519444999945], [-73.83256211199998, 40.88451947199998], [-73.83233699299986, 40.88454919399987], [-73.83130254299985, 40.884711422999956], [-73.83065359099987, 40.88485293899989], [-73.83033698199986, 40.88496949199991], [-73.82953327299994, 40.885303333999936], [-73.82791367899995, 40.88599054299988], [-73.82777310699997, 40.886047017999886], [-73.82700490399988, 40.88634614699992], [-73.82672826599993, 40.88644534299988], [-73.82611278899988, 40.886666035999895], [-73.82529579999982, 40.88697637999996], [-73.82448935999992, 40.88742251999991], [-73.82403093399992, 40.88769579499991], [-73.82387140599998, 40.88778362799996], [-73.82385099099994, 40.88776831599994], [-73.82382288900006, 40.8877472369999], [-73.82077408699998, 40.886710602999884], [-73.82031167699992, 40.8856097399999], [-73.82027432999985, 40.885550699999925], [-73.82023958299997, 40.88549105099987], [-73.81992586499996, 40.88211933299987], [-73.82110268599995, 40.878969244999915], [-73.82329599399986, 40.87309741799988], [-73.821266938, 40.869071605999885], [-73.82111615299983, 40.869008136999895], [-73.82103881099985, 40.868976079999875], [-73.82097601199978, 40.868949786999956], [-73.82091617199997, 40.86891693399989]]]}}, {\"id\": \"51\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 52, \"Shape_Leng\": 0.0289076710093, \"Shape_Area\": 4.52027509603e-05, \"zone\": \"Cobble Hill\", \"LocationID\": 52, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99588110299992, 40.68239495399995], [-73.99620846299992, 40.681723424999916], [-73.998192111, 40.68228732999989], [-74.00004865199998, 40.682803355999965], [-73.9997255609999, 40.68347314899994], [-73.99940054699985, 40.684144561999844], [-73.99908066099994, 40.68481599199985], [-73.99986579399993, 40.685036468999954], [-74.00064737999996, 40.685253946999914], [-73.99969626699995, 40.68719803399989], [-73.99963768399985, 40.68731476799996], [-73.99937882999994, 40.68785319299995], [-73.99906961899994, 40.68849807499985], [-73.99874207699997, 40.689177013999924], [-73.99841229299983, 40.68985608799994], [-73.99808645299991, 40.69053507799987], [-73.99776924399993, 40.691194697999954], [-73.99623278099983, 40.69075777399991], [-73.99436092799984, 40.69023947999991], [-73.99236366999985, 40.689690123999924], [-73.99262286099999, 40.68916054899988], [-73.99268552799995, 40.68903324699992], [-73.99293238499986, 40.68851457599991], [-73.99300451699985, 40.68836463599994], [-73.99326829999998, 40.68782445099991], [-73.99333822999985, 40.687678796999876], [-73.99360423199991, 40.6871272299999], [-73.99378736999994, 40.68675596699995], [-73.9939292959999, 40.686452870999865], [-73.99409147700004, 40.68611343899993], [-73.99425367999989, 40.685777996999924], [-73.99440802399982, 40.685452495999925], [-73.99457806799994, 40.68510014199988], [-73.99490113599977, 40.6844302379999], [-73.99523240699986, 40.68374079099993], [-73.99555742799987, 40.683068534999954], [-73.99588110299992, 40.68239495399995]]]}}, {\"id\": \"52\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 53, \"Shape_Leng\": 0.161500913385, \"Shape_Area\": 0.000947530980821, \"zone\": \"College Point\", \"LocationID\": 53, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.83839013354552, 40.798304872075995], [-73.83919004727183, 40.7977772664477], [-73.84022425799824, 40.79761115131156], [-73.84025860809128, 40.79758885843212], [-73.83997099590027, 40.796905641892224], [-73.83903676806118, 40.79705247704045], [-73.83843368293606, 40.796786758946396], [-73.83840083088981, 40.79675525865344], [-73.83736225294896, 40.79426352412381], [-73.83730248787447, 40.79421958581836], [-73.83722791411225, 40.79411138424652], [-73.83717497419711, 40.794023700013284], [-73.8370840678496, 40.79390346198528], [-73.83706498890116, 40.793851098213075], [-73.83707146788825, 40.7937989222257], [-73.83712604698766, 40.79369555002053], [-73.83686308614463, 40.793066798056], [-73.83663288911686, 40.79251637576927], [-73.83818069199988, 40.792748501999945], [-73.83887362400004, 40.792949805999854], [-73.83905064799991, 40.79298782099989], [-73.83905076599997, 40.79298784899991], [-73.83905088399997, 40.79298788499988], [-73.83922303099997, 40.79304304599991], [-73.83938616999993, 40.79311469299992], [-73.83953619099991, 40.79320127799991], [-73.83953626099989, 40.79320132299989], [-73.83966981899991, 40.793300636999966], [-73.83966987799987, 40.793300689999946], [-73.83978455899992, 40.79341008099993], [-73.83990622699986, 40.793516588999935], [-73.84004094899993, 40.793615254999885], [-73.84018725699998, 40.79370467999992], [-73.84018731600003, 40.7937047159999], [-73.84018738599987, 40.793704751999925], [-73.84034364899992, 40.79378386199987], [-73.84050794799997, 40.793851761999875], [-73.84050805399978, 40.79385179899986], [-73.84067828300006, 40.79390789199993], [-73.84076080099992, 40.79250035099991], [-73.84081532599994, 40.7915048849999], [-73.83987372999988, 40.7914705419999], [-73.83895676099984, 40.79143843099996], [-73.83900524099994, 40.79061732199994], [-73.83992306199995, 40.790649901999906], [-73.83999984499997, 40.789251996999916], [-73.84008157499997, 40.78785153099992], [-73.83916328899986, 40.78782010599988], [-73.83824429099994, 40.78778591599995], [-73.83834963799995, 40.78585030599993], [-73.83845605499975, 40.78582446799988], [-73.83853062500003, 40.78470030299992], [-73.83862028899992, 40.7832271429999], [-73.83869921299997, 40.781676956999874], [-73.83777640299981, 40.78165237399991], [-73.83669385900005, 40.78164957499991], [-73.83218249399988, 40.78165887099986], [-73.82770726099993, 40.781667916999865], [-73.82678476000001, 40.78166766899994], [-73.82569679399981, 40.78169053099991], [-73.82528484199993, 40.78169729199987], [-73.82513740299993, 40.78169970599994], [-73.82557907999986, 40.78083309199997], [-73.82576928399986, 40.78046316399985], [-73.82616437200001, 40.77968464499987], [-73.82642473899999, 40.779228734999954], [-73.82665895999983, 40.778818600999884], [-73.82789985099988, 40.7766822329999], [-73.82830621299996, 40.776000787999905], [-73.828793895, 40.77524292899992], [-73.82908256199991, 40.7747943209999], [-73.83000011099983, 40.77373240699996], [-73.83090463599999, 40.7729830689999], [-73.83150226899997, 40.77248795599991], [-73.83161455499989, 40.77241020899988], [-73.83209910199999, 40.77210429399989], [-73.83338983199998, 40.77134538399988], [-73.83433960199982, 40.77078781999989], [-73.83511636199991, 40.77028363399989], [-73.83536623599981, 40.77012144099994], [-73.83614052499993, 40.76952737399995], [-73.83678033699985, 40.76896614799991], [-73.83711212099999, 40.76864287199989], [-73.83746794599992, 40.76830671799994], [-73.83805766700006, 40.76759634399986], [-73.83866554599997, 40.76674036599996], [-73.83871102200001, 40.76666838599987], [-73.83913189281712, 40.765991105784146], [-73.83925621702686, 40.76603420963875], [-73.8395267864783, 40.76613954972556], [-73.8395437155588, 40.766211721721604], [-73.83969772549081, 40.76639267704533], [-73.83973778802071, 40.76665087171896], [-73.83973386806854, 40.76673441465323], [-73.83969375990986, 40.76682480595994], [-73.8395605054185, 40.76691684117707], [-73.83951000594936, 40.766974160215085], [-73.83949805795821, 40.76701701698471], [-73.83959072609817, 40.76711082471904], [-73.83968175000263, 40.76710489116111], [-73.83975387982034, 40.767120445440874], [-73.83978750472959, 40.767051485410185], [-73.83986957818284, 40.76708653570916], [-73.83993922177474, 40.76706427714744], [-73.8399402231328, 40.76700188164961], [-73.8398843082266, 40.766952204208515], [-73.83990190913454, 40.76682415841403], [-73.84000758970934, 40.76662342046593], [-73.840113137722, 40.76656666461221], [-73.84022374446803, 40.7665292403013], [-73.84027020920483, 40.76646466650519], [-73.84027438049095, 40.76637967295619], [-73.84031170315292, 40.76629479789843], [-73.84057222218038, 40.76627040438149], [-73.84086515071182, 40.76621486974827], [-73.84084759497334, 40.766162020200035], [-73.8408401364553, 40.76616295457574], [-73.84083644357402, 40.76614783092416], [-73.84086007103146, 40.76614219398064], [-73.84087122017897, 40.766157327427656], [-73.84086500211306, 40.766159208867364], [-73.84088883538395, 40.7662103792582], [-73.84102191922433, 40.766185149281355], [-73.84100802893788, 40.76611688421589], [-73.8410788870798, 40.76611036771657], [-73.84108497073595, 40.766165182066395], [-73.84105761922727, 40.766168924392886], [-73.84105643008813, 40.766178605084185], [-73.84120530561162, 40.76615038087748], [-73.84120337400219, 40.766097476524635], [-73.84119574495959, 40.76609729965605], [-73.84120075497815, 40.76608124266235], [-73.84123679749322, 40.766081292334725], [-73.84123676821112, 40.76609357613729], [-73.84122309104404, 40.76609544731743], [-73.84122723929276, 40.76614622161617], [-73.84144364387917, 40.766105192987204], [-73.84172286075056, 40.7660432805295], [-73.84170125822992, 40.765988326283264], [-73.84167323806332, 40.76599495749284], [-73.84166211638194, 40.76596848399208], [-73.8417130964574, 40.765958160228685], [-73.84174481256957, 40.76603841406753], [-73.84196444921889, 40.76598971084912], [-73.84198401916458, 40.76596514588186], [-73.84196546583135, 40.76592732305348], [-73.8419157397382, 40.7659329244163], [-73.84191572852082, 40.76593765030904], [-73.84187344475318, 40.76594893027433], [-73.84186231494718, 40.76592529148888], [-73.8419033560003, 40.76591400814698], [-73.84190583063506, 40.76591873694828], [-73.84231868231662, 40.76582102929955], [-73.842317444969, 40.76581913757782], [-73.8423572306718, 40.765812576589134], [-73.84236463453715, 40.765835266052655], [-73.84232856912517, 40.765844665167286], [-73.84232609735666, 40.76583899337525], [-73.84227261595433, 40.76585592824656], [-73.84228991176651, 40.76589941989143], [-73.84231828009945, 40.76591125238223], [-73.84251347571919, 40.7658679674676], [-73.84249272360755, 40.765802367584925], [-73.84248152879428, 40.76580613233626], [-73.84247287859779, 40.76578533237701], [-73.84252261658963, 40.76577500488192], [-73.84252754583214, 40.765792965547035], [-73.84251137775344, 40.76579766885277], [-73.84253623494851, 40.765862920695355], [-73.8426819600242, 40.765830607164354], [-73.84295915637364, 40.7658547224249], [-73.84336969753039, 40.76572529284769], [-73.84346820695714, 40.76565307141732], [-73.84359974376382, 40.7656434804411], [-73.8437063352798, 40.76567996834533], [-73.84394748335806, 40.765494881485765], [-73.84407906365587, 40.76549676136358], [-73.84426568006505, 40.76546921500887], [-73.84448895884195, 40.765459602583896], [-73.84491332846012, 40.76552838817853], [-73.84528163541583, 40.76566611948253], [-73.84593023292821, 40.76601257632095], [-73.84602715981245, 40.76604487155383], [-73.84614072149937, 40.76611570431363], [-73.84620024037936, 40.76619634316636], [-73.84652170418624, 40.76629580063177], [-73.84666768544619, 40.76628375673098], [-73.84684964636887, 40.76626203475838], [-73.84701309197986, 40.76626796152773], [-73.84712188593454, 40.766290563967594], [-73.84723067976141, 40.76631315809567], [-73.84738263097066, 40.76641941262396], [-73.8476935357496, 40.76670591213375], [-73.84780575104425, 40.76677294817163], [-73.84786167715994, 40.76685023340278], [-73.84792234568383, 40.767012496032976], [-73.84799048271634, 40.767117161337204], [-73.84804248817566, 40.76723778076487], [-73.84809411442576, 40.76737876897698], [-73.84821324018463, 40.767634597847994], [-73.84828222856984, 40.76774537878623], [-73.8483741186844, 40.76786934596774], [-73.84842038279727, 40.76791076681383], [-73.84849386891709, 40.767976573164425], [-73.84850405385016, 40.76799249900586], [-73.8485178978279, 40.76801414798668], [-73.84856697139021, 40.7680909071991], [-73.84852060615141, 40.768135412162714], [-73.84848118613256, 40.76817324547555], [-73.84842604843061, 40.768223374851715], [-73.84837586137671, 40.7682764253777], [-73.84833089177016, 40.76833211504436], [-73.84829137867685, 40.76839014781252], [-73.84829430420871, 40.76841059214461], [-73.8483013819906, 40.7684304406085], [-73.84831243929806, 40.768449208817756], [-73.84832720628944, 40.76846643874888], [-73.8483453225899, 40.76848170991812], [-73.84841573602714, 40.76850098668506], [-73.84848358226597, 40.76852500253367], [-73.8485483108146, 40.7685535626044], [-73.84860939647602, 40.76858643516744], [-73.84866634360749, 40.768623353501596], [-73.84883522381716, 40.76875230209401], [-73.84901287079057, 40.76921530435127], [-73.8489225578029, 40.769324534965435], [-73.84892276167722, 40.76940717390534], [-73.84884133186712, 40.76946410514129], [-73.8487178873391, 40.76954777089411], [-73.84865863960181, 40.76961433219833], [-73.84860729245335, 40.76970346891756], [-73.84860101425126, 40.76981782588793], [-73.84872999189092, 40.76998954177914], [-73.84892724343374, 40.76991478836153], [-73.848817810295, 40.76973805463504], [-73.84950427178478, 40.76951022388226], [-73.84953434085783, 40.76951712501355], [-73.84955532911304, 40.76954917482335], [-73.84955529811104, 40.76956289850219], [-73.84949207284846, 40.7695834015085], [-73.84955506042947, 40.769668112981556], [-73.84960321683452, 40.76965902709566], [-73.84963028831295, 40.769661349325055], [-73.84970521295284, 40.76978953551773], [-73.84969012358812, 40.76981010191443], [-73.84964193972178, 40.7698306246238], [-73.84969586733, 40.769931335979635], [-73.84975004147579, 40.769922257882556], [-73.84977407793218, 40.769936012942956], [-73.8497619875757, 40.769961156547325], [-73.84907551195148, 40.77019356377016], [-73.84899274432657, 40.770028680072755], [-73.84872671824314, 40.77010618908874], [-73.8486255096204, 40.770387757427166], [-73.84861273687663, 40.77042603035469], [-73.84858664304565, 40.77050424990497], [-73.84858418501082, 40.770543302646296], [-73.84858696848256, 40.770605638316596], [-73.84859368539874, 40.77062392002509], [-73.8485979041902, 40.7706354051098], [-73.84861027612754, 40.77065908656987], [-73.84862191287661, 40.77068804462914], [-73.84864527048879, 40.7707192413199], [-73.8486715286884, 40.770761572902536], [-73.84874746796304, 40.77084403300933], [-73.84882636338835, 40.770913141615985], [-73.84890527065537, 40.770982250685776], [-73.84912991114923, 40.771063788377575], [-73.84933814775343, 40.771187619866005], [-73.84949402711861, 40.77132861681541], [-73.8495760947736, 40.771438916609654], [-73.84960979016878, 40.771484204461004], [-73.84973230426058, 40.77173009452529], [-73.84973804497628, 40.77174241985013], [-73.8497730365678, 40.77180479915258], [-73.84980501475297, 40.771842589265155], [-73.84982268798642, 40.77185829087455], [-73.84993252885828, 40.77189377858084], [-73.84994046677184, 40.77192059790689], [-73.84994241882967, 40.77192719325984], [-73.85020188336162, 40.771937392784736], [-73.85020240095857, 40.7719455348769], [-73.85030397450447, 40.77193819958954], [-73.85031075024499, 40.77192028681626], [-73.85028379823378, 40.771925372392815], [-73.8502906085098, 40.771892099056686], [-73.85031415825226, 40.77190237084045], [-73.85033808818525, 40.771743668538754], [-73.8503784815698, 40.77175140188951], [-73.85036823526043, 40.77181539433311], [-73.85126052461455, 40.771872872816765], [-73.85126742588308, 40.77179863646516], [-73.85130108374476, 40.77180636079771], [-73.85127037620717, 40.771985534743514], [-73.85124007806625, 40.771980375036954], [-73.85125374913325, 40.77189078614855], [-73.85036146530206, 40.77183074564787], [-73.85033410569142, 40.772017606260675], [-73.85029707789941, 40.77200987726779], [-73.85029717573791, 40.77196635402256], [-73.85019734544956, 40.771961326490576], [-73.85019486668942, 40.77197227426243], [-73.84995162517303, 40.77195829887714], [-73.849962928372, 40.77199648468658], [-73.84994620808664, 40.772057372616196], [-73.84986902411926, 40.77218306200828], [-73.85003738572694, 40.772181073933574], [-73.85004065546363, 40.772224600554914], [-73.84983958377667, 40.77223100286741], [-73.84972769544879, 40.77241320129405], [-73.84957160955894, 40.772561103830746], [-73.84918580437129, 40.77310472058777], [-73.84897619867189, 40.77356166795698], [-73.84896112400045, 40.7737426847533], [-73.84912365290268, 40.77426022845488], [-73.84921197208924, 40.77454763447593], [-73.84921678107065, 40.77463958212611], [-73.8492142740838, 40.77473158268647], [-73.8492044602571, 40.77482330005166], [-73.84918737541369, 40.77491439914948], [-73.84916308193993, 40.77500454716513], [-73.84913166855965, 40.775093414757016], [-73.84909325000882, 40.775180677260124], [-73.84906220320683, 40.77524019861195], [-73.84903349588859, 40.77530039479502], [-73.84901442828632, 40.77534379717124], [-73.8489728769192, 40.77545132951898], [-73.84893873474758, 40.77556035848066], [-73.84900348408753, 40.77568014817956], [-73.84911614425796, 40.77578626632446], [-73.84911636597812, 40.77578751886946], [-73.84903813840937, 40.777004753468105], [-73.84903491312333, 40.777288443429185], [-73.84901842052128, 40.77740463203713], [-73.84899941431672, 40.77752205117339], [-73.84897771676305, 40.77763491828796], [-73.8489740492564, 40.777674654087846], [-73.84897165987938, 40.77777261678307], [-73.84895680884702, 40.77789056185411], [-73.84895450913376, 40.77792770447221], [-73.84895559325399, 40.77796167373135], [-73.84896008358999, 40.777993097546876], [-73.84896802544691, 40.77802221269157], [-73.84897383480374, 40.778035349458925], [-73.84898207469953, 40.77804830060991], [-73.84899254202304, 40.77806074087802], [-73.84899380740195, 40.77806192275443], [-73.84900491920531, 40.778072283364835], [-73.84901882664829, 40.77808258615508], [-73.84903388635963, 40.778091376356365], [-73.84904969603315, 40.7780984127068], [-73.8490659137323, 40.77810355721967], [-73.84911406661877, 40.778114696407165], [-73.84917605388847, 40.7781268708982], [-73.84920561102659, 40.778131385698444], [-73.84923662799216, 40.778135009824275], [-73.84928764656799, 40.778140966261525], [-73.8493120634367, 40.77814545571613], [-73.84933338575311, 40.77815101361854], [-73.84934534071354, 40.77815546800863], [-73.84935671111596, 40.778161336890044], [-73.84936761626928, 40.778168680728115], [-73.84937825751246, 40.77817763581692], [-73.84939585408341, 40.77819636205012], [-73.84943535665232, 40.77824556318454], [-73.84947844499678, 40.77829606613999], [-73.84949436325334, 40.77831779842351], [-73.84950757331885, 40.7783385362538], [-73.84951731632304, 40.778357153370855], [-73.84952520807873, 40.77837657889388], [-73.84953114505714, 40.77839659523964], [-73.84953505641919, 40.77841695203263], [-73.8495358659634, 40.778425885557084], [-73.8496919692504, 40.77842802269303], [-73.84969626130186, 40.778415067669734], [-73.84974741420471, 40.778414054006106], [-73.84974445808719, 40.77846481410773], [-73.84953664690558, 40.778457620289885], [-73.8495343034803, 40.77847747370973], [-73.84952990130999, 40.77849672067872], [-73.84952858977923, 40.77849996041985], [-73.84952657007254, 40.77850304678793], [-73.84952383818543, 40.778505996530164], [-73.84952033715504, 40.7785088558093], [-73.84951133367666, 40.77851424792634], [-73.84948131861695, 40.77852894025979], [-73.84947722420407, 40.77853175319544], [-73.84947388931542, 40.77853462207157], [-73.84947124090881, 40.77853758951187], [-73.84946927977728, 40.778540694550685], [-73.84945846130667, 40.778562822784885], [-73.84945106324766, 40.77858090421403], [-73.8494461945274, 40.778597215353415], [-73.8494438267176, 40.778611854335985], [-73.84944397412765, 40.778625218719164], [-73.84944670731139, 40.77863709985818], [-73.84944905360408, 40.77864250512285], [-73.84945203895039, 40.7786475428336], [-73.84945566507523, 40.778652230080084], [-73.84945995577684, 40.77865656806611], [-73.84946904846156, 40.778663783211684], [-73.84947957696491, 40.77867039747487], [-73.84950513864881, 40.77868183198594], [-73.8495368533371, 40.778691039733545], [-73.84957720816625, 40.77869873929359], [-73.84960429240051, 40.7787021144363], [-73.849686879789, 40.77870938977827], [-73.84969513318684, 40.778710697192274], [-73.84973917397001, 40.778717689519304], [-73.84975786860362, 40.77872230577249], [-73.84977311348453, 40.77872756702233], [-73.84978540855205, 40.778733714823765], [-73.84979030575784, 40.77873708076647], [-73.84979436044361, 40.77874064379382], [-73.84979758647495, 40.77874441213234], [-73.84979999376891, 40.778748385795204], [-73.84980157152407, 40.7787525659411], [-73.84980132097789, 40.77876730703297], [-73.84993353296176, 40.778769123381046], [-73.84993080856658, 40.77879720785134], [-73.84980088112201, 40.778793284827636], [-73.84979610479625, 40.77907534402301], [-73.8497922389795, 40.779092971223974], [-73.84979044990959, 40.77909795713514], [-73.8497879996364, 40.779102772648265], [-73.84978489696313, 40.779107423135834], [-73.84978114316354, 40.779111930042795], [-73.84977161159078, 40.779120544852006], [-73.84975923714768, 40.779128740992924], [-73.84973108396828, 40.779142220586884], [-73.84966179483854, 40.779168695923204], [-73.84965811870529, 40.77917013100993], [-73.84965398205614, 40.77917143164276], [-73.84964409727885, 40.7791736610763], [-73.84960160472559, 40.77917910716696], [-73.84959577388824, 40.779180162339316], [-73.84959302348445, 40.77918066383929], [-73.84958612583421, 40.77918247349432], [-73.84958246208258, 40.77918379216453], [-73.84957931896626, 40.77918526530271], [-73.8495766621079, 40.779186891691445], [-73.84957448948224, 40.77918868992333], [-73.84957279225395, 40.779190668865944], [-73.84957155453695, 40.77919283670706], [-73.84957754850187, 40.7793974127518], [-73.84957921443754, 40.77941026414162], [-73.8495828564969, 40.779423236718856], [-73.84958833734376, 40.77943585972485], [-73.84959547717891, 40.779447728517624], [-73.84960400544777, 40.7794584194067], [-73.84961378096662, 40.77946779835428], [-73.84962460342139, 40.77947566406813], [-73.84963631633588, 40.77948192035157], [-73.8496396893305, 40.779483275176695], [-73.84964324176299, 40.77948441446292], [-73.84965096095709, 40.77948605439123], [-73.84965952458613, 40.77948684891368], [-73.84966912057455, 40.77948682574971], [-73.84968773448676, 40.77948502148875], [-73.84973187730172, 40.77947808268722], [-73.84974262512041, 40.779477132759546], [-73.84974869047639, 40.77947705120936], [-73.849751900693, 40.77947700161922], [-73.8497540443524, 40.779477201424385], [-73.8497385127623, 40.77940144669318], [-73.85007007124759, 40.77934793010841], [-73.85009114089188, 40.77935138794181], [-73.85009398131915, 40.779373355585975], [-73.85012866957865, 40.779366811085104], [-73.8501114812998, 40.7793008963792], [-73.85067227797757, 40.77919399711241], [-73.85065788414983, 40.77916981875145], [-73.85096718956476, 40.77911091411253], [-73.85094705724252, 40.77906915543026], [-73.85118700387086, 40.77901455483171], [-73.85119275794374, 40.77902554507494], [-73.85147606060222, 40.77896440973063], [-73.85148755480326, 40.77899297861757], [-73.85120426185316, 40.779049719637875], [-73.85120998157538, 40.779076085337444], [-73.85101339619412, 40.77912195667439], [-73.8510276713222, 40.779198850460226], [-73.8510507916471, 40.77919668391865], [-73.85107946016522, 40.779297758688315], [-73.85105055581148, 40.779302114020446], [-73.85103904766675, 40.77928013352065], [-73.85094655574714, 40.77929319249282], [-73.85094937054778, 40.77932614289468], [-73.85074415423085, 40.77935443134818], [-73.85073555596264, 40.779323670381125], [-73.85015741651672, 40.77943274199089], [-73.85014017315828, 40.7793909869257], [-73.85011125881357, 40.779399735201125], [-73.85012275250426, 40.77942830422235], [-73.85003696878474, 40.77944943201958], [-73.84979304347623, 40.77949768469487], [-73.84981720066014, 40.77963034805418], [-73.8498785983377, 40.779625736165144], [-73.84987537319611, 40.77961461194829], [-73.84998643275294, 40.77960582076213], [-73.85000542456552, 40.779603661787675], [-73.85000701949747, 40.77961725471423], [-73.852352678813, 40.779480672833344], [-73.85232086242219, 40.77916928470371], [-73.85235661936242, 40.77916933052226], [-73.85238832778414, 40.7795289023869], [-73.85236232095829, 40.77953010473945], [-73.85235751917273, 40.779496739045214], [-73.84997493056258, 40.779641290896265], [-73.84982102259761, 40.779651339303875], [-73.84984362844325, 40.77977547491927], [-73.8498424903832, 40.779776059105814], [-73.84984112663376, 40.779776552534464], [-73.84984040336778, 40.77977673218471], [-73.84983952619352, 40.7797769473172], [-73.8498363029474, 40.77977739325839], [-73.84983225116706, 40.779777594370024], [-73.84981375969693, 40.7797772551562], [-73.84980586942713, 40.77977757808099], [-73.84980207764751, 40.77977809581887], [-73.84979887674888, 40.779778893089244], [-73.84979622113285, 40.77977996061912], [-73.84979509438978, 40.779780607977145], [-73.84979408627251, 40.779781326855726], [-73.84977014020235, 40.779804078118424], [-73.84975449350075, 40.779818952421415], [-73.8497381878465, 40.77983187150904], [-73.8497217813201, 40.77984202528526], [-73.84970553216176, 40.77985031570939], [-73.84968940703446, 40.77985676954223], [-73.8496734158966, 40.77986136920672], [-73.84966424252009, 40.77986296833341], [-73.84965393555235, 40.779863756496745], [-73.84961851526822, 40.77986309814073], [-73.84961020050447, 40.77986294288282], [-73.84960016522162, 40.77986336233683], [-73.84959129058164, 40.779864421745124], [-73.84958633626894, 40.77986552362804], [-73.84958143974038, 40.77986719148755], [-73.84957661227024, 40.77986941009375], [-73.84957193877196, 40.779872150408075], [-73.84956744174248, 40.77987540475347], [-73.8495632520746, 40.77987908216709], [-73.84955945235377, 40.77988311139089], [-73.84955614907031, 40.77988737512835], [-73.84955259386662, 40.77989340341914], [-73.84955001985372, 40.779899902397595], [-73.84954767585884, 40.77991462266048], [-73.84954849514146, 40.7799297889858], [-73.84954857704788, 40.77993031060005], [-73.84955331323854, 40.77996244668725], [-73.84955456302885, 40.77997084118439], [-73.84955451960579, 40.779979232987905], [-73.84955450077001, 40.77998279003447], [-73.84955315550077, 40.779990415384006], [-73.84973465207516, 40.77999639195549], [-73.84974018040859, 40.78004045002199], [-73.85031476156011, 40.780031757060556], [-73.85031473957734, 40.78004153600408], [-73.85037426478351, 40.780035150505384], [-73.85037642446964, 40.78000959208551], [-73.85041635235571, 40.78001124077503], [-73.85041840010895, 40.78003520780309], [-73.85044992795434, 40.780033650696375], [-73.85045656031697, 40.7798882799137], [-73.85048178019531, 40.77988831263897], [-73.85048371588393, 40.77996180377535], [-73.85162286781565, 40.779939312964586], [-73.85163563328629, 40.77986903549033], [-73.85169028085423, 40.779867508478176], [-73.85167099162219, 40.780035228878106], [-73.85162266134178, 40.78003197177817], [-73.85162492646164, 40.77995848657489], [-73.85048578225796, 40.77997778253564], [-73.85047510549936, 40.78005285432789], [-73.85042045570427, 40.78005597878778], [-73.85041830321092, 40.78007834233692], [-73.85036786060351, 40.780079873703535], [-73.85036580861214, 40.780057505868655], [-73.84999800412726, 40.78006341795454], [-73.8497371105003, 40.78006376491308], [-73.84949381208321, 40.780065828114246], [-73.84948977929692, 40.78007331542979], [-73.8494874774256, 40.7800805064753], [-73.84948686941622, 40.78008751126687], [-73.84948719303776, 40.78009097879954], [-73.84948794336292, 40.78009442996937], [-73.84949102749154, 40.78010288834467], [-73.84949574651277, 40.78011190001433], [-73.84951236065773, 40.7801355242267], [-73.84952637351847, 40.780152066412974], [-73.8495653283726, 40.78019228873934], [-73.84957495568457, 40.78020439464494], [-73.8495821916791, 40.78021587708098], [-73.84958596479899, 40.780224021907294], [-73.84958872905418, 40.780232725119845], [-73.84959131415296, 40.78025225075464], [-73.8495907270451, 40.78027117820336], [-73.84958612528546, 40.78031502771827], [-73.84958618687178, 40.780335081952984], [-73.84959129206666, 40.780388083496284], [-73.84959707959024, 40.780448108602435], [-73.8496031561403, 40.780490530998115], [-73.84961037731607, 40.780529082030384], [-73.8496130358922, 40.780537549730376], [-73.84961740029898, 40.780546002066416], [-73.84962343481696, 40.78055434903004], [-73.84963103378945, 40.78056243619226], [-73.84964012579178, 40.78057017450349], [-73.84965044093985, 40.78057732924293], [-73.84966171544059, 40.78058370071067], [-73.84967357421606, 40.780589084370966], [-73.84967914925255, 40.780590981667125], [-73.84968532984298, 40.78059242190543], [-73.8496921628949, 40.78059342055946], [-73.84969977836884, 40.78059400745033], [-73.84970842754518, 40.78059377498105], [-73.84975146751532, 40.78059264267863], [-73.84976094548882, 40.78059283562826], [-73.84976904639166, 40.78059355767057], [-73.84977733487823, 40.78059514388641], [-73.8497810051181, 40.78059625702005], [-73.849784389933, 40.78059757617311], [-73.84978747936445, 40.78059911942538], [-73.84979031790787, 40.780600896886384], [-73.84979287208388, 40.78060289996859], [-73.84979518947944, 40.78060513610534], [-73.84980162766686, 40.78061304215576], [-73.84980707998362, 40.78062176545094], [-73.8498082942797, 40.78062481097365], [-73.84981503152129, 40.78064171303461], [-73.84981746562705, 40.780655619993134], [-73.84981911467304, 40.780665050314816], [-73.84981943123964, 40.78069217479532], [-73.84981943042993, 40.78069253380164], [-73.84981214796522, 40.78077040025503], [-73.84981067388031, 40.780798305751105], [-73.84981008898778, 40.78085272165433], [-73.84981173168418, 40.780870733633066], [-73.8498148830908, 40.78088682136889], [-73.84982676964914, 40.78092633202067], [-73.84984980805791, 40.780990486812705], [-73.84985673695408, 40.7810069582442], [-73.84988263236191, 40.78105985182873], [-73.84988560815704, 40.78106957972739], [-73.84988800830871, 40.78107744549927], [-73.84988932081794, 40.78108913479838], [-73.84991219996311, 40.78108750458773], [-73.85093970945037, 40.78108116289207], [-73.85093646396534, 40.781119575156005], [-73.84991855289967, 40.781120744575325], [-73.84988728063747, 40.781121511316904], [-73.84984470805739, 40.78159250358273], [-73.84984092044256, 40.781596308346295], [-73.84983660014267, 40.78159962491627], [-73.84983172551624, 40.781602472529784], [-73.84982629433667, 40.7816048610679], [-73.84981313500577, 40.78160927446724], [-73.84979896935607, 40.781612912585345], [-73.84976687720906, 40.781618083645604], [-73.84973620036537, 40.781620440270125], [-73.84965318361573, 40.78162387984903], [-73.8495843534843, 40.781629426594684], [-73.8495464630235, 40.781631393972475], [-73.84949053395457, 40.78163144675125], [-73.84945107629284, 40.7816301974027], [-73.84936169911538, 40.78162737087983], [-73.84928815389694, 40.781628156909555], [-73.84920125952722, 40.781632293191265], [-73.84919977948272, 40.781632444372924], [-73.84919835691278, 40.78163271373529], [-73.84919696998568, 40.781633107113116], [-73.84919562929343, 40.781633618656954], [-73.84919434894228, 40.78163424771528], [-73.84919312672957, 40.781634993782426], [-73.84919084924894, 40.781636827049724], [-73.84918882990345, 40.781639121182664], [-73.84918709418461, 40.78164181154949], [-73.84918567739616, 40.781644879936366], [-73.84918460308101, 40.78164824629691], [-73.84918361402742, 40.78165580961947], [-73.84918433205542, 40.78166340130996], [-73.84918531851869, 40.78166690656572], [-73.84918667463242, 40.78167009551429], [-73.8491883858092, 40.78167288923217], [-73.84919040545033, 40.781675233044986], [-73.8492029285044, 40.78168554220345], [-73.84921772841386, 40.78169429651935], [-73.84923492578879, 40.78170156835328], [-73.84925485088841, 40.781707502208036], [-73.84927462461532, 40.78171166930421], [-73.84928876450584, 40.781713786390114], [-73.8492976002965, 40.78171510447715], [-73.84938705771674, 40.781723721340434], [-73.84942345201131, 40.781728766522896], [-73.84944211373582, 40.781732608813165], [-73.84945880658724, 40.78173718798607], [-73.8494738402559, 40.78174258351931], [-73.84948737902812, 40.78174885091145], [-73.84950124349734, 40.78175747765476], [-73.8495137890027, 40.78176812001698], [-73.84952476609249, 40.78178056960574], [-73.84953389183205, 40.781794512779435], [-73.84954094279978, 40.781809632288464], [-73.84954561024456, 40.781825335024706], [-73.84954774265991, 40.781841078342794], [-73.84954725821561, 40.78185630494597], [-73.84954069379472, 40.781888227584595], [-73.84952913459368, 40.78191821799483], [-73.84952161585517, 40.7819321025794], [-73.84951298514834, 40.78194514908879], [-73.84950332753195, 40.781957265494775], [-73.84949266315807, 40.7819684089368], [-73.84947294903945, 40.78198472698115], [-73.84945388296065, 40.78199710281962], [-73.84944812889454, 40.78200084134151], [-73.8494227221584, 40.78201476570605], [-73.84936186667622, 40.782045555102805], [-73.84933420378349, 40.78206141376419], [-73.84931176686678, 40.782077151262975], [-73.84929392410037, 40.78209333669785], [-73.84926950224772, 40.782121661807885], [-73.84925186203198, 40.78214824899512], [-73.84924064542695, 40.78217363739409], [-73.84923742986847, 40.782185916998515], [-73.84923580368731, 40.782197945048345], [-73.84923644337586, 40.78222417737107], [-73.84923866768678, 40.78223600406681], [-73.84924212637716, 40.78224677780399], [-73.84924675980236, 40.78225636532168], [-73.8492525818285, 40.782264775014035], [-73.84925954248891, 40.782271979173956], [-73.84926345684903, 40.78227510948317], [-73.8492676449515, 40.78227794145226], [-73.84929600721517, 40.78229326072907], [-73.84932904044368, 40.782307045206025], [-73.84936730007063, 40.782319567834136], [-73.84941366539037, 40.78233176733061], [-73.84947755621798, 40.78234686175886], [-73.84952319204757, 40.78235653832871], [-73.84956490053888, 40.782363850994194], [-73.84960336728695, 40.782368818244514], [-73.84964463464058, 40.78237219479521], [-73.84974246546523, 40.7823772129673], [-73.84978655217171, 40.782380556957676], [-73.84982764974079, 40.78238524745948], [-73.84993441724671, 40.78239948850579], [-73.84999442110552, 40.78240473663232], [-73.85017560390794, 40.78241117596823], [-73.85020961297208, 40.78241285003427], [-73.8502303385557, 40.782415578320965], [-73.85023230404349, 40.7824160584907], [-73.85023422072196, 40.782416682333846], [-73.85023606851233, 40.782417459875795], [-73.8502378544331, 40.782418406370724], [-73.85024128462477, 40.782420761379306], [-73.8502445600841, 40.78242380957484], [-73.85024983944788, 40.78243044391331], [-73.85026066567171, 40.78244717133412], [-73.85026365512806, 40.78245087669524], [-73.85026662248302, 40.782453825649796], [-73.85026987468241, 40.78245624190818], [-73.85027156691282, 40.78245719933736], [-73.85027330557236, 40.782457993657154], [-73.85027510454267, 40.782458626225456], [-73.85027696382406, 40.782459096875044], [-73.85027889487543, 40.782459406290684], [-73.85028089657797, 40.782459562009656], [-73.85028520882398, 40.78245945017648], [-73.85028961793519, 40.782458798199265], [-73.85029412126644, 40.7824576062418], [-73.85029868703998, 40.782455892355834], [-73.85030798571955, 40.782450862920975], [-73.85031711167846, 40.782443939887486], [-73.85032112499816, 40.78244003656178], [-73.85032509149033, 40.78243539489009], [-73.85034040945192, 40.78241432532851], [-73.85034477980167, 40.782409639116274], [-73.85034933834744, 40.78240574587901], [-73.85035396591742, 40.78240261849052], [-73.85035918415372, 40.7823997795102], [-73.85037136170378, 40.782394995882846], [-73.85038574038333, 40.78239145798388], [-73.85040198724082, 40.7823892365787], [-73.85044211739618, 40.78236554175274], [-73.85046237692144, 40.78235969761345], [-73.85048366289533, 40.78235568166829], [-73.85050121243168, 40.78235387589481], [-73.85051869908605, 40.78235337707155], [-73.85052785662293, 40.78235357775081], [-73.85057294395939, 40.78235455459715], [-73.85064452756707, 40.78235741190032], [-73.85080255677516, 40.78236568563748], [-73.85085191062834, 40.78236610903307], [-73.85095974363315, 40.78236442938333], [-73.85105865211817, 40.782364882258165], [-73.85115888224064, 40.7823669396393], [-73.85116661764656, 40.78236750732233], [-73.85117451639574, 40.78236875101386], [-73.8511824502199, 40.78237067054823], [-73.85119028608574, 40.78237322839524], [-73.85120513920205, 40.782380055476935], [-73.85121184750511, 40.78238420654224], [-73.85121791474403, 40.78238872617335], [-73.85122501973319, 40.7823954522826], [-73.85123047734257, 40.782402547333724], [-73.85168735535527, 40.78236274850467], [-73.85192054727274, 40.78234735174143], [-73.85201582564918, 40.78234443136425], [-73.8520964483179, 40.78234201293649], [-73.85216039867626, 40.78234159057518], [-73.85228097618968, 40.782346302562715], [-73.85234426450937, 40.78234537466375], [-73.85238157190422, 40.78234440474127], [-73.8525345815865, 40.78233528001999], [-73.8526005387797, 40.78233259168669], [-73.85277635625364, 40.78233360799108], [-73.85285529539563, 40.782331943929556], [-73.85297395843433, 40.782329447809246], [-73.85314993069179, 40.782323936276164], [-73.85322330950031, 40.782060533865014], [-73.85324407205734, 40.78205239243221], [-73.85326692782857, 40.78204537960012], [-73.85329196150529, 40.78203946800102], [-73.85331933850043, 40.78203460406847], [-73.85346812122447, 40.78201330752581], [-73.85364506982407, 40.78198934521578], [-73.8536797409504, 40.781985742044256], [-73.85371711005834, 40.781983465738875], [-73.85375796075665, 40.78198246418576], [-73.85380512238734, 40.781982659050755], [-73.85395650777669, 40.781989847872026], [-73.85405570899185, 40.781991036328684], [-73.85413969536411, 40.7819891532955], [-73.85417157081446, 40.78198584295777], [-73.8542402730513, 40.78197377388685], [-73.85426817724941, 40.781971169812834], [-73.85429401467086, 40.781971157586874], [-73.85435233041154, 40.78197423068179], [-73.85437833322167, 40.78197457865837], [-73.85449826596835, 40.78197074035898], [-73.85459641854935, 40.78196425435653], [-73.85463146933225, 40.781960354355036], [-73.85470961107612, 40.781949485505095], [-73.85474428264793, 40.78194594517099], [-73.8548062259648, 40.781941772103394], [-73.8548542920655, 40.78194004146991], [-73.8549001856964, 40.78194016242124], [-73.85496441517024, 40.7819419356182], [-73.85505842171236, 40.78194635879554], [-73.85516739460483, 40.781954131617], [-73.85519577265883, 40.78195717553486], [-73.85522450332166, 40.78196174922935], [-73.85525344295303, 40.78196782839602], [-73.85528242786589, 40.78197538434999], [-73.85530228006235, 40.78198218987457], [-73.85532050466512, 40.781990722048356], [-73.85533642950311, 40.782000665917934], [-73.85534961480477, 40.78201175858745], [-73.85536631186146, 40.78203115839585], [-73.8553794260627, 40.78205276939831], [-73.8553884517444, 40.78207571631062], [-73.85545062834929, 40.783143079138306], [-73.85561617015411, 40.78314629374278], [-73.85573475263911, 40.78315657201521], [-73.85573467582252, 40.78319202567684], [-73.8558186269766, 40.783194156960796], [-73.85587567996764, 40.78330970432567], [-73.85587458930368, 40.78319827873433], [-73.85592656549973, 40.783196317504526], [-73.85593296680116, 40.78331787949883], [-73.8561272379995, 40.78334818699362], [-73.85613060095005, 40.78334903769837], [-73.85614261963912, 40.78335207787735], [-73.85621394085632, 40.78337804706227], [-73.85623171362438, 40.783382941714855], [-73.8562481864888, 40.78338579863812], [-73.85626839522845, 40.783386949783264], [-73.85628930749792, 40.78338574120036], [-73.85631107802644, 40.78338216671607], [-73.85633400316311, 40.78337616186691], [-73.85635641010697, 40.783368437555716], [-73.85638100310933, 40.783358282489075], [-73.85641761444457, 40.78334009262385], [-73.85643317343843, 40.78333236797427], [-73.85645206661752, 40.783322990816075], [-73.85643993662929, 40.78315751797215], [-73.85667690893332, 40.78315008816212], [-73.85667159264416, 40.78309526904827], [-73.8566533206465, 40.78297943929009], [-73.85662133635174, 40.78297939950117], [-73.85661875947618, 40.78293887790554], [-73.85660942954789, 40.78293987965894], [-73.85660286912328, 40.782892261248804], [-73.85664951090587, 40.78289231927643], [-73.85665341005878, 40.782937907649995], [-73.8566813904766, 40.78293996917436], [-73.85670256831024, 40.78309712187021], [-73.85670377431897, 40.78314924541216], [-73.85671074167976, 40.78318160961588], [-73.8567166980192, 40.78319946435095], [-73.85671872229025, 40.783205536651764], [-73.85672676824508, 40.78322101763507], [-73.85674405554418, 40.783247802497335], [-73.856783191625, 40.78330561795276], [-73.85680365242803, 40.78333241445574], [-73.85682476802232, 40.78335788881941], [-73.85683209884915, 40.783364957953886], [-73.85684129014666, 40.78337136348521], [-73.85685209384219, 40.78337695969253], [-73.85686419228801, 40.78338156776652], [-73.85687720455233, 40.78338502305834], [-73.85689057554147, 40.78338718331959], [-73.85690380601619, 40.783387965005055], [-73.85690977849755, 40.78338767539815], [-73.85691574879351, 40.783387377076934], [-73.856916424825, 40.7833873417309], [-73.85693056185812, 40.783385630084105], [-73.85698714612556, 40.78337878469752], [-73.8570807328061, 40.783369842024115], [-73.8571287329935, 40.783366002386124], [-73.85717784525652, 40.783363208125216], [-73.85727946259598, 40.78336078528364], [-73.85729827323259, 40.78336189883376], [-73.85731850031053, 40.78336539919497], [-73.85734052316816, 40.78337127058618], [-73.85736824871026, 40.78338057174476], [-73.85739520262219, 40.78339091596058], [-73.85741998550928, 40.78340204083335], [-73.85744261237805, 40.78341393716906], [-73.85746304507938, 40.78342661447072], [-73.85749351368149, 40.783449524574486], [-73.8575173357796, 40.78347265210974], [-73.8575267060602, 40.783484235517875], [-73.85753435811877, 40.783495825009766], [-73.85757561010641, 40.78355295044696], [-73.85761223352816, 40.78361187050412], [-73.85769080619146, 40.78363660261761], [-73.85777187429395, 40.78365613500528], [-73.8578548434883, 40.783670324466094], [-73.85791465221827, 40.78367601652251], [-73.85797317515247, 40.78368698496617], [-73.85802960947828, 40.78370307933287], [-73.85808318103317, 40.783724078841665], [-73.85809293159618, 40.78373619674262], [-73.8580938512611, 40.783737359496776], [-73.85810281002682, 40.783753056106434], [-73.8581093889967, 40.78376903809657], [-73.85811509504617, 40.78378777899396], [-73.85813601750543, 40.78385775769632], [-73.85815907631199, 40.78393343388455], [-73.85816966828361, 40.78399506346119], [-73.85817248535963, 40.784006781482184], [-73.858173788655, 40.78401740135328], [-73.85817325802043, 40.784032261215046], [-73.85817002215366, 40.78404870153351], [-73.85814972842107, 40.78410864682302], [-73.85813635063349, 40.784155208635866], [-73.85813380512944, 40.78416260811786], [-73.85812502709221, 40.78418815181475], [-73.85812230968615, 40.78419420116462], [-73.85811884686251, 40.78420016013823], [-73.85810975652801, 40.784211732830144], [-73.85809779066977, 40.78422277611823], [-73.8580831293917, 40.78423316826499], [-73.85806577086481, 40.78424283639321], [-73.85804623764804, 40.784251473568894], [-73.85802503979681, 40.78425884889897], [-73.8580030190051, 40.78426465768563], [-73.85799502501645, 40.78426594883851], [-73.85798583738342, 40.784266498559816], [-73.85795493700509, 40.78426555248949], [-73.8579439287294, 40.78426575687127], [-73.85793355654516, 40.78426708847736], [-73.857928959443, 40.784268231699905], [-73.85792468298948, 40.784269699814395], [-73.85790564851207, 40.7842775098377], [-73.85789394771578, 40.784283714960836], [-73.8578892976976, 40.78428703527752], [-73.85788555791413, 40.784290465440584], [-73.85788269478857, 40.78429404109152], [-73.857880693497, 40.78429778901622], [-73.857856785958, 40.784362595740525], [-73.85785479002129, 40.7843685828167], [-73.85784993964482, 40.78438315908842], [-73.85783635535137, 40.78442391535011], [-73.85783272297559, 40.784434841570814], [-73.8578319424332, 40.78443819463532], [-73.85783105450373, 40.78444208833942], [-73.85783033335551, 40.78444972661102], [-73.85783174883879, 40.784466309860214], [-73.85783684133779, 40.784484815644426], [-73.8578466600017, 40.7845090440646], [-73.85793550046684, 40.78450208755773], [-73.85793785344377, 40.784507155590646], [-73.85794201573734, 40.784516124506425], [-73.85800470789609, 40.78465118792153], [-73.85799433965465, 40.784650953342414], [-73.85798864693247, 40.78465074546464], [-73.85798309391498, 40.784651067976895], [-73.85797817966105, 40.78465188698374], [-73.85797615343813, 40.78465244167644], [-73.85797435379807, 40.78465309431558], [-73.8579728131553, 40.78465383790521], [-73.85797153240988, 40.784654663567544], [-73.85797051068255, 40.784655570296344], [-73.85797010587352, 40.7846560556213], [-73.85796977329701, 40.78465656850943], [-73.85796951145142, 40.78465708969348], [-73.85796930885482, 40.78465762904325], [-73.85796917696919, 40.78465818607045], [-73.85796911579062, 40.78465876245011], [-73.85796943206935, 40.7846635133594], [-73.85797081502555, 40.78466829305747], [-73.85797327315859, 40.784673146618985], [-73.85797685542525, 40.78467816456815], [-73.85798576369062, 40.78468741105699], [-73.85800821599177, 40.78470690216521], [-73.85804860814078, 40.78474288931853], [-73.85813170195247, 40.78481040622703], [-73.85816525317925, 40.784839057018765], [-73.85827576991517, 40.78493830103915], [-73.85831062791249, 40.784972703035855], [-73.85834010817545, 40.78500481152161], [-73.85841272034449, 40.78508992460954], [-73.85846495883229, 40.78514968445889], [-73.85847639475126, 40.78516071698645], [-73.8585039039529, 40.78518305124955], [-73.85851353347384, 40.785192524741674], [-73.85851907436073, 40.78519990165849], [-73.85852327431452, 40.785208007674804], [-73.85852665393895, 40.7852176661474], [-73.85852698593592, 40.78522893917458], [-73.85899156050911, 40.7852041691424], [-73.85905644466132, 40.78543181356221], [-73.85909453895707, 40.78543918586412], [-73.85909193046099, 40.78542107669319], [-73.8590811990642, 40.78542106357173], [-73.8589906592535, 40.785104901340546], [-73.85903628973827, 40.785092722779524], [-73.85912145674305, 40.78541295765197], [-73.85910536075212, 40.7854129379749], [-73.8591104979856, 40.785442585200954], [-73.85921738129012, 40.78546535001113], [-73.85921602303392, 40.785477503662385], [-73.85923734418844, 40.785478543067555], [-73.85924002857145, 40.78546942944455], [-73.85952909197228, 40.78553056053404], [-73.8595236950597, 40.785561954252664], [-73.85941579536208, 40.7855395376104], [-73.85941715122297, 40.78552839731539], [-73.85939317223568, 40.785524316794614], [-73.85939048227354, 40.7855354549592], [-73.85906638595809, 40.785466688265586], [-73.85908214610843, 40.785521985402546], [-73.85899048805017, 40.7854915994347], [-73.85894034299622, 40.785567647386635], [-73.85895721464995, 40.78557826770512], [-73.85896755907882, 40.78558367485105], [-73.85897274386161, 40.785585751810096], [-73.85897777442442, 40.785587279096866], [-73.85898271378166, 40.7855882666723], [-73.85898760509338, 40.78558873268225], [-73.8589912421843, 40.78558873713243], [-73.85899522435685, 40.785588462572214], [-73.85901335620464, 40.78558580150029], [-73.85901770625782, 40.78558540995414], [-73.85902162583585, 40.78558534338306], [-73.85903817556573, 40.78558635386962], [-73.8590444406207, 40.78558745999671], [-73.85904993483408, 40.785589025205574], [-73.85905354569631, 40.785590487595506], [-73.85905705909373, 40.785592284246945], [-73.85906403880534, 40.785596984164904], [-73.8590848266754, 40.78561632526704], [-73.8591048538104, 40.78563645883563], [-73.85912438142353, 40.78566226403373], [-73.85912918731835, 40.785670707159134], [-73.8591326727759, 40.78567987891349], [-73.85913472666259, 40.785689490180026], [-73.85913527518866, 40.78569920665536], [-73.85913505673093, 40.78570141202809], [-73.85913431982947, 40.785708643032685], [-73.85913188308693, 40.78571758122148], [-73.85912804012771, 40.785725808053535], [-73.85912285858095, 40.78573313179497], [-73.85911539050561, 40.785740794157654], [-73.85910597960674, 40.78574843722465], [-73.85907851498793, 40.78576585577383], [-73.859073094663, 40.785768631743245], [-73.85906659710821, 40.7857712532768], [-73.85904595729141, 40.78577773857549], [-73.85904279128076, 40.785778734327444], [-73.85903403038795, 40.78578198265143], [-73.85902600066773, 40.78578612410957], [-73.85902273779875, 40.78578841622181], [-73.85901993539109, 40.78579089903854], [-73.85901500687321, 40.7857967552268], [-73.859012474675, 40.785801218356866], [-73.85912461223712, 40.78585015966324], [-73.85910322798064, 40.78587950876995], [-73.8590075486969, 40.785833144017104], [-73.85900765755157, 40.78583794358627], [-73.85901262561131, 40.78586400104041], [-73.85901586237468, 40.785873623137356], [-73.85902672912445, 40.78589531103742], [-73.85903045064073, 40.785905311999144], [-73.85903169431462, 40.7859113826315], [-73.85903226093974, 40.78591774945819], [-73.85903137893663, 40.78593146385327], [-73.85902784022804, 40.785946714222355], [-73.85902122503792, 40.785965328088835], [-73.8590158599168, 40.785975749313124], [-73.85900818594033, 40.78598580703045], [-73.85899816623991, 40.785995529841884], [-73.85898579987948, 40.78600494287451], [-73.85897116148841, 40.78601398322948], [-73.8589540346854, 40.786022832070934], [-73.8589107413936, 40.78604072691422], [-73.85885571609528, 40.78606040676397], [-73.8588048897857, 40.786077345990286], [-73.85875663984856, 40.78609206512187], [-73.85871030047439, 40.78610475868033], [-73.85865102736396, 40.78611605142519], [-73.8586507788746, 40.786116095012126], [-73.85863807102099, 40.786119285861986], [-73.85862705727051, 40.786122963439624], [-73.85861177077352, 40.786130050277556], [-73.85859860229465, 40.786138625159964], [-73.85859310695238, 40.78614328282774], [-73.85858840508548, 40.786148140153834], [-73.85858453141702, 40.78615314122782], [-73.85858152390757, 40.786158261972375], [-73.8585764279673, 40.78617016895421], [-73.85857233568788, 40.786183455064155], [-73.85856623298426, 40.7862177485718], [-73.85856492361425, 40.78624832822732], [-73.85856645703221, 40.786262620864846], [-73.85856942617816, 40.78627618569098], [-73.85857844553281, 40.7863022756081], [-73.85859127699717, 40.7863296299931], [-73.8586081570016, 40.786358800963264], [-73.85863031307774, 40.78639192965726], [-73.8586413032543, 40.78640518987496], [-73.85866900182882, 40.786433274970705], [-73.85867919947988, 40.78644630788587], [-73.85868879765647, 40.78646233960763], [-73.85869445579392, 40.786475593784566], [-73.8586968468518, 40.786487599409384], [-73.85869681117975, 40.78649319153127], [-73.85869595921667, 40.786498531196784], [-73.85869073576778, 40.786514642914504], [-73.85868283350074, 40.78653116647522], [-73.85867223053742, 40.78654812798552], [-73.85865887626714, 40.786565591209666], [-73.8586094910695, 40.786623693779205], [-73.8585672513092, 40.78666969367949], [-73.85854493188523, 40.78669090901091], [-73.85848303366855, 40.78674469176868], [-73.85847124228764, 40.78675704183654], [-73.85844506383575, 40.78678813992626], [-73.85842953643257, 40.786802816684634], [-73.85842561959295, 40.78680560385147], [-73.85842092224304, 40.78680830931207], [-73.85839718127706, 40.78681907631928], [-73.8583918434028, 40.78682199726703], [-73.85838745384751, 40.786824962769], [-73.85832309838246, 40.78687465396305], [-73.85826743102713, 40.78691671061672], [-73.85826152049127, 40.786922025964856], [-73.85825574781519, 40.786928430564906], [-73.85823412406135, 40.786957292885525], [-73.85820537379047, 40.786988657304825], [-73.85816956419355, 40.78702511987387], [-73.85815754110043, 40.78703537974478], [-73.85814354176146, 40.7870443229452], [-73.85812817158494, 40.78705156105829], [-73.85811203509817, 40.787056809029245], [-73.85809131675386, 40.78706102459309], [-73.8580690623594, 40.78706349214436], [-73.85804551953424, 40.7870641652476], [-73.8580358282388, 40.78706372075975], [-73.85802109164295, 40.787063036692686], [-73.85799961604485, 40.7870607052523], [-73.85797998052438, 40.787057394041554], [-73.85796202844763, 40.787053067520425], [-73.85794568998246, 40.78704770834859], [-73.85792172009276, 40.78703739424951], [-73.85786314840233, 40.78700855171836], [-73.85780599688668, 40.78698547261065], [-73.85772986169724, 40.78695886725275], [-73.85770909350568, 40.786953304396256], [-73.85768482601809, 40.78694943843553], [-73.85760255605203, 40.786943869791116], [-73.85757378493405, 40.78694053834411], [-73.85754734103337, 40.78693456336776], [-73.8575358112561, 40.78693055112214], [-73.85752521760811, 40.78692579236686], [-73.85750542076829, 40.78691415988882], [-73.85746576865716, 40.786885870763506], [-73.85744739357587, 40.786874394827315], [-73.85737743400954, 40.786840601825745], [-73.85736136582724, 40.78683114439631], [-73.85734834299558, 40.78682149372409], [-73.85729374398954, 40.78677411293459], [-73.85728559591055, 40.78676704282163], [-73.85724349816154, 40.78672979066156], [-73.85723836454146, 40.78672603943953], [-73.85723247119613, 40.78672269067732], [-73.85721807696163, 40.78671708017688], [-73.8572031450027, 40.78671328816679], [-73.85716818589594, 40.786706338610955], [-73.85715356205525, 40.78670234879371], [-73.85712345763554, 40.78669109213223], [-73.85705112356236, 40.786660367900836], [-73.85703042955764, 40.78665296658985], [-73.85701118969901, 40.786647358763524], [-73.85700889280255, 40.786646861378806], [-73.8570065822965, 40.78664654339674], [-73.85700423681998, 40.78664639607964], [-73.85700184401013, 40.786646428793645], [-73.85699693804672, 40.786647007872645], [-73.85699174536678, 40.78664828986718], [-73.85698165442462, 40.7866523115247], [-73.8569625005109, 40.78666189600631], [-73.8569521826025, 40.7866669170036], [-73.85694857751135, 40.786668199794114], [-73.85694418001007, 40.786669401523774], [-73.85693993619978, 40.786670161512006], [-73.85693580067614, 40.78667048020512], [-73.85691545961245, 40.786670230468765], [-73.85686648756545, 40.786667458929365], [-73.85684155844486, 40.78666805819638], [-73.85681133765485, 40.78667261369714], [-73.8567614699094, 40.78668437433237], [-73.85670911174445, 40.78669716832916], [-73.85667405292824, 40.786708715817774], [-73.85665906408536, 40.7867152877934], [-73.85664594534656, 40.7867223946566], [-73.85663891840045, 40.78672716776114], [-73.85663457884486, 40.78673011516583], [-73.85662488072505, 40.78673849595679], [-73.85660829894911, 40.7867579799755], [-73.85659430490371, 40.78678148497367], [-73.85658367206896, 40.78680570446048], [-73.85656012485876, 40.78686911535254], [-73.85655178203112, 40.78688678978821], [-73.8565428156595, 40.78690211407084], [-73.85652107510892, 40.7869405473222], [-73.85651686958872, 40.78694562095422], [-73.85651235931141, 40.786949947043226], [-73.85650746918343, 40.78695359753186], [-73.85650214336668, 40.786956607363194], [-73.85650037645205, 40.786957380303186], [-73.85649853926243, 40.78695803454765], [-73.8564946154943, 40.78695898371911], [-73.8564903374551, 40.7869594565097], [-73.85648559735232, 40.786959458815865], [-73.85647634701824, 40.786958447672376], [-73.8564516069265, 40.786954023171276], [-73.85644453537552, 40.78695350106511], [-73.85644134734402, 40.78695352339555], [-73.85643833805504, 40.786953754182555], [-73.85640981200986, 40.78695812221798], [-73.8563852262836, 40.78696449390209], [-73.85636465148082, 40.786972834815174], [-73.85635589754584, 40.78697773038881], [-73.85634817318284, 40.78698311524666], [-73.85633435604609, 40.78699556074094], [-73.85632138571486, 40.7870107630821], [-73.85628495146457, 40.787066558227224], [-73.85626918661896, 40.78708693478534], [-73.85625338291675, 40.78710253866144], [-73.85621071201784, 40.78713918136811], [-73.85620020315073, 40.78715009995904], [-73.8561920416777, 40.78716059010005], [-73.85617856602606, 40.78718439975185], [-73.85616904420905, 40.787210385465734], [-73.85616371236826, 40.787237898041326], [-73.85616275026324, 40.78726606556619], [-73.85617045836483, 40.787356485124846], [-73.8561760427645, 40.78743673608788], [-73.85617604394368, 40.787463228890125], [-73.85617436960048, 40.787487053267014], [-73.8561737828743, 40.78749021390658], [-73.85617275786664, 40.787493219875685], [-73.85617128128915, 40.78749610047487], [-73.85616935713371, 40.78749884448483], [-73.85616696948826, 40.787501471486266], [-73.85616413290576, 40.78750397881662], [-73.85615699040497, 40.78750872544567], [-73.85614910324811, 40.78751260487092], [-73.85613932084577, 40.78751645613109], [-73.85609772992429, 40.787529507038656], [-73.85608104623329, 40.78753581665557], [-73.85607509299993, 40.78753875381504], [-73.8560664806232, 40.787543885586466], [-73.8560569353297, 40.78755305807727], [-73.85603321264523, 40.78758738271038], [-73.85601603404221, 40.78760921644483], [-73.85597608420362, 40.78766353806566], [-73.85597404083201, 40.787665958599305], [-73.85597165404891, 40.78766818052124], [-73.85596893750498, 40.787670211052124], [-73.85596586474206, 40.787672054346814], [-73.8559586891305, 40.78767517877375], [-73.85595002350544, 40.78767760744772], [-73.85592982941742, 40.78768039227652], [-73.85589212531865, 40.787682128110426], [-73.85580406250807, 40.78768710554236], [-73.85573914362435, 40.78768956432036], [-73.85560310030938, 40.78769279689538], [-73.85556149088127, 40.78769294344053], [-73.85542196652793, 40.78768971584818], [-73.85539747579985, 40.787690540339725], [-73.85537111936169, 40.78769366813679], [-73.85534234055956, 40.78769915164486], [-73.85530780984412, 40.78770760987653], [-73.85529901270421, 40.787710399018664], [-73.8552896094324, 40.78771430445829], [-73.85525882990035, 40.78773015073166], [-73.85524681590036, 40.78773555760208], [-73.855234071033, 40.78773963892992], [-73.8552279423134, 40.78774084545549], [-73.85522186387641, 40.78774153171004], [-73.85512749399335, 40.78774797743824], [-73.8549497141386, 40.78776024401784], [-73.85491542031168, 40.78776363177171], [-73.85483685607232, 40.787774707791776], [-73.85480173164524, 40.78777853589499], [-73.85476457332538, 40.78778001121077], [-73.85473138125592, 40.787777970130236], [-73.8547070531812, 40.78777443717722], [-73.85468022861875, 40.7877693420801], [-73.85457032389033, 40.787744394278064], [-73.85453886969691, 40.787738357341624], [-73.85451097917964, 40.78773329704505], [-73.85449877839754, 40.787732263925705], [-73.8544845470797, 40.78773329935781], [-73.8544674801146, 40.78773617850981], [-73.85444998385327, 40.78774013681833], [-73.85442754479743, 40.78774524112081], [-73.85441587100252, 40.78774703682185], [-73.85440532437036, 40.78774783264652], [-73.85439211484702, 40.787747770894], [-73.85437741367019, 40.78774676257946], [-73.85431409235855, 40.78773918177169], [-73.85429883286434, 40.78773858552389], [-73.85428498197076, 40.7877391174959], [-73.85416877913472, 40.787749622955054], [-73.85401805617188, 40.78776425411682], [-73.85393152201209, 40.78777359140016], [-73.85371524447409, 40.78779889084302], [-73.85354068080385, 40.78781480652712], [-73.8534518446134, 40.78782611229885], [-73.85336457538442, 40.78784148047623], [-73.85332033752428, 40.78785160001044], [-73.85328433106342, 40.78786264787821], [-73.85325391367482, 40.78787544243505], [-73.85322865679291, 40.7878902230359], [-73.85313945328275, 40.787955269406766], [-73.85304318488666, 40.78802164841081], [-73.85301986377856, 40.788040223035985], [-73.85299455222662, 40.78806364011936], [-73.85293742640218, 40.78812567459657], [-73.85287292575686, 40.78817980298654], [-73.85285256044187, 40.78820079505672], [-73.85284230133867, 40.78821137173992], [-73.85284201138356, 40.788342555973536], [-73.8528617928357, 40.788344319807976], [-73.85288990261189, 40.788346670054686], [-73.85297296851711, 40.788351837329195], [-73.85306663884121, 40.78835654815511], [-73.8531259333357, 40.78835847974775], [-73.8531642037784, 40.788358203513255], [-73.8532509376304, 40.78835436105497], [-73.85328967042591, 40.78835371647664], [-73.85346961745066, 40.78835850276173], [-73.85353771951809, 40.78835829180134], [-73.85359051889232, 40.78835591820986], [-73.85361442297669, 40.78835306713739], [-73.85367261904089, 40.78834211887525], [-73.85369958159862, 40.78833865014911], [-73.85386496770758, 40.78832686549008], [-73.85401770764051, 40.78831756723785], [-73.85415345566268, 40.78831452445786], [-73.85424119113252, 40.7883134195514], [-73.85432601017287, 40.78831359045082], [-73.85435363507843, 40.7883153364687], [-73.85438364788577, 40.78832012876301], [-73.85441611713868, 40.788327770073565], [-73.85446440284554, 40.78834142892451], [-73.85450027715643, 40.78835285606134], [-73.85453503358329, 40.78836618285395], [-73.85456828189716, 40.788381253347474], [-73.85459973691678, 40.78839795242853], [-73.85463110180122, 40.78841798363433], [-73.85466212641388, 40.78844194035955], [-73.85469341458857, 40.7884702736738], [-73.85472744729312, 40.78850521093849], [-73.85476549772548, 40.788545807907276], [-73.85478518134681, 40.788571254517144], [-73.85479249924812, 40.78858380770184], [-73.85479768615775, 40.788595871875124], [-73.85480083624125, 40.78860762657569], [-73.85480194733185, 40.788619154725914], [-73.85480065341628, 40.788642440126985], [-73.85479536031538, 40.78867916507016], [-73.8547935555014, 40.7886916255108], [-73.85479289872487, 40.788699684328606], [-73.85479170822225, 40.788713045824494], [-73.85479015014025, 40.78880257276635], [-73.85478828752537, 40.788841633480914], [-73.85477488753459, 40.788955872242866], [-73.85477486760239, 40.78901401842672], [-73.8547725437133, 40.789036374605345], [-73.85476700102882, 40.789056944927935], [-73.85475092459623, 40.78910019378737], [-73.85474513677647, 40.78911925858459], [-73.8547271694004, 40.78919784905344], [-73.8547134155788, 40.78925082651861], [-73.8547006703475, 40.78929330543442], [-73.85469330882114, 40.789311216005146], [-73.8546740230833, 40.78934767083617], [-73.85466678497424, 40.78936366406562], [-73.85464575832671, 40.78943264241254], [-73.85463754730239, 40.78944903077786], [-73.85461486312076, 40.789484472306896], [-73.85460743543419, 40.7895006629729], [-73.85459364262289, 40.78954447405569], [-73.8545792407035, 40.78959649495239], [-73.85456120188782, 40.78966970120305], [-73.85454851361663, 40.789761392247286], [-73.85453150689433, 40.789839561726886], [-73.8545254129648, 40.789863084644026], [-73.8545234573319, 40.7898742202651], [-73.85452124363728, 40.789905609038165], [-73.85451938179, 40.789971568112975], [-73.8545147646747, 40.79006760170975], [-73.85450773724963, 40.79014387398984], [-73.85450297303252, 40.79018264389067], [-73.85449618458735, 40.790209263136454], [-73.85448648377765, 40.790235058500784], [-73.8544773471365, 40.79025192329882], [-73.8544719407123, 40.79025897531954], [-73.85446599251512, 40.79026501061485], [-73.85445942840997, 40.79027006309863], [-73.85445222779767, 40.790274178814265], [-73.85444728241936, 40.79027652261423], [-73.85443480521755, 40.79028244093846], [-73.85442371474154, 40.790287722071255], [-73.85415926148362, 40.790412719467504], [-73.8540697206796, 40.790452327389076], [-73.85389890575122, 40.790521431455424], [-73.85383120135553, 40.790550035830094], [-73.85362955320053, 40.790642954822104], [-73.85355197936214, 40.79067681416448], [-73.85347749436237, 40.79070629240607], [-73.85340954040993, 40.79073004233696], [-73.85337851301512, 40.79073919665927], [-73.8533068131879, 40.79075803360744], [-73.85327445626994, 40.79076805080772], [-73.85316791455793, 40.790806232186505], [-73.85312868602469, 40.790822120897474], [-73.85309376971468, 40.79083784371317], [-73.85300596376405, 40.7908801362715], [-73.85283622037684, 40.79096246926646], [-73.85279912994584, 40.79098046739349], [-73.8527712656627, 40.790994984391645], [-73.85274514158607, 40.791010602069825], [-73.85273595696937, 40.791016920781324], [-73.852720981704, 40.79102720261027], [-73.85269895180701, 40.79104467163848], [-73.85268853906085, 40.79105464484897], [-73.8526792733986, 40.79106580778186], [-73.85267114137163, 40.79107816192768], [-73.85266414518988, 40.79109170477684], [-73.85265830693045, 40.79110642061034], [-73.85265359064718, 40.79112231759108], [-73.85264759969778, 40.791157717578535], [-73.85264742419872, 40.79118940644141], [-73.85265299090051, 40.79122229927846], [-73.85266418431857, 40.791255795028036], [-73.85268071109523, 40.79128899001694], [-73.85270703591517, 40.791332437174276], [-73.85273431336955, 40.79137402132106], [-73.85276257920901, 40.79141382308164], [-73.85279187961731, 40.7914518894217], [-73.85283502179291, 40.79150469555183], [-73.852858796491, 40.791533785514275], [-73.85288469081824, 40.79156296763563], [-73.85290948021424, 40.79158884423955], [-73.85297910559555, 40.79165682135949], [-73.85306080953109, 40.79173345914617], [-73.85314657162884, 40.791813010953646], [-73.85317478813853, 40.79183739653493], [-73.85320119873275, 40.79185858275661], [-73.85335450891063, 40.79197048428281], [-73.85342445523895, 40.79202232566936], [-73.85346170233008, 40.79205180973618], [-73.85357001852789, 40.79214460965114], [-73.8536655370685, 40.79222770321815], [-73.85371695117264, 40.79226911070586], [-73.85374337894176, 40.79229377280514], [-73.85375336392234, 40.79230565363566], [-73.85376137210825, 40.7923174426662], [-73.85376299952767, 40.792320948019615], [-73.853769898906, 40.79233577890684], [-73.85377222418454, 40.792345210011746], [-73.8537752195276, 40.79235706387968], [-73.85377760787473, 40.79238053384989], [-73.85377854910695, 40.79242127400702], [-73.8537779734007, 40.79246239989308], [-73.85377595819327, 40.79247919124222], [-73.8537726775182, 40.79249457678897], [-73.85373126537303, 40.79264103597808], [-73.85369739407176, 40.792752095911716], [-73.85369038565968, 40.79277081768012], [-73.85367235728862, 40.792811759410796], [-73.85366602731114, 40.79282949112686], [-73.85365784517334, 40.79286462703543], [-73.85364967670063, 40.79292081669766], [-73.85363607087064, 40.793024249617865], [-73.85363080166015, 40.79309818271064], [-73.8536315280857, 40.793145252533556], [-73.8536365935801, 40.7932072949071], [-73.85365042177162, 40.793335930646315], [-73.85368396814027, 40.793453722878766], [-73.85371089772221, 40.79359540618731], [-73.85372635874646, 40.79371998350508], [-73.85374051410696, 40.793840496841945], [-73.85373351593732, 40.793935375091095], [-73.85371825553295, 40.793962036128015], [-73.85368561848587, 40.79401839408178], [-73.85358451638858, 40.79410782853745], [-73.85353237176378, 40.794138856811834], [-73.85351764622185, 40.794143377362026], [-73.85338820350945, 40.79418311461311], [-73.85321701027608, 40.79422058136176], [-73.85304034260227, 40.794263553963894], [-73.85290227105678, 40.79431522857123], [-73.85282872033454, 40.79435526049111], [-73.852766271984, 40.794389247829294], [-73.85268508272915, 40.79447497027753], [-73.85263725293453, 40.79454229280629], [-73.85262416811536, 40.7946134426322], [-73.85262310783055, 40.79475015592458], [-73.85265711034053, 40.794875089776454], [-73.85265879996285, 40.794919712443615], [-73.8526309825624, 40.79494485424474], [-73.85259261251832, 40.79493563836851], [-73.85258042693633, 40.79492158400388], [-73.85255611510601, 40.794893564972604], [-73.85253153207259, 40.794790057555886], [-73.85249076582929, 40.794674974705615], [-73.85242651222973, 40.79460975882105], [-73.85233601016024, 40.7945453656644], [-73.85225478631885, 40.79451254658353], [-73.85214062585973, 40.79444649258847], [-73.85200288947318, 40.79436318108858], [-73.85193160677206, 40.794309789545224], [-73.85189730713526, 40.794286656815935], [-73.85181651299672, 40.79423690787666], [-73.85168072178335, 40.79414255807322], [-73.85159077157229, 40.79408653158045], [-73.85143723745655, 40.79402591832599], [-73.8512917816512, 40.79398321871114], [-73.85112011660962, 40.7939241220335], [-73.85097034152753, 40.793857948323975], [-73.85079807258425, 40.79380961136817], [-73.85062808294123, 40.79377521705116], [-73.85049082789963, 40.79373727270515], [-73.8503263813649, 40.79367766269842], [-73.85016769642245, 40.79364306610521], [-73.85000721695765, 40.79360091123408], [-73.8498408564086, 40.79354448520947], [-73.84973467938572, 40.79349569482057], [-73.84962251839083, 40.79346569570807], [-73.8494823389847, 40.793418600803406], [-73.8493365487603, 40.79337149956624], [-73.8492203438544, 40.79336616002198], [-73.84913455725516, 40.79336270580534], [-73.84906160835088, 40.79336261029852], [-73.84898342755147, 40.79339361180352], [-73.84893051832798, 40.79341069705515], [-73.84890131445043, 40.793435387178214], [-73.84886198397842, 40.793468635780464], [-73.84884894723092, 40.79350168565831], [-73.84885848370783, 40.793591099022926], [-73.84887965495557, 40.79365488253947], [-73.84890147425774, 40.79366828351349], [-73.84896194839715, 40.7938049149536], [-73.849017211755, 40.79392484379938], [-73.84907814795721, 40.79414538448089], [-73.84903163032307, 40.79433808463523], [-73.84896271217512, 40.794408477406584], [-73.84878370153791, 40.794542102225336], [-73.84877745470259, 40.79457974348267], [-73.84877122000401, 40.79461737604402], [-73.84878457368544, 40.794700915276955], [-73.84884690961489, 40.79490523039205], [-73.84887162206815, 40.795027488024886], [-73.8488765456334, 40.79513404169125], [-73.84884566489967, 40.795229166577556], [-73.84881376432953, 40.79528731529211], [-73.84870999361335, 40.795416481289855], [-73.84855988738983, 40.79549506021838], [-73.84840067725122, 40.79556462149242], [-73.84823442276506, 40.79563184132044], [-73.84808903411276, 40.79568851740229], [-73.84792364797373, 40.795753514196456], [-73.84781967180619, 40.79578469546817], [-73.84763762072186, 40.79582581476201], [-73.84745840378464, 40.79586126478076], [-73.84729839674371, 40.795900350307335], [-73.84712646747435, 40.795935061799575], [-73.84685831176914, 40.79597954211446], [-73.84677146173631, 40.795992890189915], [-73.84673086658063, 40.796001479344746], [-73.84660914258137, 40.79602723440848], [-73.84651631916125, 40.79606183379271], [-73.84644727612637, 40.79610167934103], [-73.84637823258807, 40.79614151613582], [-73.84632264317928, 40.79616408876088], [-73.84624528406336, 40.796161950667056], [-73.84620432632532, 40.79613943809977], [-73.84612011376143, 40.79609316530997], [-73.84604010531895, 40.79603792173839], [-73.84597412890419, 40.79601730187317], [-73.84588460454623, 40.79601358879331], [-73.84570427007124, 40.79602888164454], [-73.84567461137927, 40.79602909389694], [-73.8456210638299, 40.79602948171253], [-73.84559167122411, 40.79601181902573], [-73.84559498485325, 40.79598311567544], [-73.84562722402833, 40.795891811886165], [-73.84561606324512, 40.795845718962255], [-73.84557627918223, 40.79580748365859], [-73.84551678745504, 40.79576485622225], [-73.8454367018999, 40.79564636850851], [-73.84537463759747, 40.7955483553866], [-73.84528519601372, 40.79544339037916], [-73.84520216815449, 40.795398019954966], [-73.84506393886154, 40.79532545177626], [-73.84498447620877, 40.7952652094112], [-73.84495114910605, 40.79520249792143], [-73.84488892176357, 40.79510909661496], [-73.8448730254321, 40.79509683700132], [-73.84480116912152, 40.7950576760676], [-73.84466734418736, 40.79495665817199], [-73.8445991504444, 40.79491154987514], [-73.84451391906175, 40.79488179016884], [-73.84438471988366, 40.794870341529595], [-73.84422311865038, 40.794876985073586], [-73.84407887373855, 40.794875060841726], [-73.84390523936045, 40.794869306206444], [-73.84382499651181, 40.79487536227074], [-73.84379857938126, 40.79487735637718], [-73.84373233041337, 40.79488235365055], [-73.84354668938991, 40.79491394401049], [-73.8434326202129, 40.79494844908789], [-73.84338511407093, 40.79498491861969], [-73.84333760876113, 40.79502138696035], [-73.84332414153785, 40.79506905019075], [-73.84335588803987, 40.795234353799444], [-73.84336415641455, 40.79534609866879], [-73.84335118053633, 40.79547252924368], [-73.8433063238691, 40.79550093348892], [-73.84323462158838, 40.79547660430573], [-73.84317961314706, 40.795433873032174], [-73.84314830001613, 40.79538678733097], [-73.84311059001811, 40.79526369123992], [-73.84304363919044, 40.795214740016874], [-73.84293426865992, 40.795191149897065], [-73.84289545847199, 40.79518278540822], [-73.84286053658465, 40.79519887517494], [-73.84284865322397, 40.795213401671674], [-73.84289254581853, 40.79539277833716], [-73.8432849719419, 40.79617439268503], [-73.84320653636648, 40.796192903595745], [-73.8428087524122, 40.79541822241309], [-73.84225959368331, 40.79558496893736], [-73.84292069025288, 40.796903686184], [-73.8428667525936, 40.79692222907191], [-73.8421674734115, 40.7956129401636], [-73.84208804901434, 40.7956370563647], [-73.84203334901652, 40.79563868313395], [-73.84197864967462, 40.79564031105082], [-73.84190480251937, 40.7956217942407], [-73.84183741226606, 40.79558859863672], [-73.8417820928015, 40.79553692460987], [-73.8416869879848, 40.79540867089491], [-73.84165397120607, 40.79539590970684], [-73.84158721174394, 40.795401069221235], [-73.84153823334024, 40.79541881301255], [-73.84148925424489, 40.795436558457354], [-73.8414273671699, 40.795490430363095], [-73.84136287271946, 40.795619950695176], [-73.84135034620307, 40.79573990748322], [-73.84136133643683, 40.795856600577515], [-73.84149409342267, 40.79598188920635], [-73.84148780606058, 40.79601448800372], [-73.84144315649658, 40.79604513413277], [-73.84134129989056, 40.796095358756304], [-73.84130710746976, 40.796138076853374], [-73.8412920482614, 40.79616745712962], [-73.8413057246617, 40.79620095620876], [-73.84129943545976, 40.79621966932996], [-73.84059982294626, 40.796806803210494], [-73.84013930364449, 40.796879187836076], [-73.84043977575945, 40.79763751324531], [-73.84040055225613, 40.79764862952207], [-73.84041517975831, 40.7976784372587], [-73.8392338815539, 40.79788530644824], [-73.83920948324085, 40.79784431435393], [-73.83841944083812, 40.798342147138], [-73.8384291582917, 40.79837567115333], [-73.83789967503593, 40.79850897118662], [-73.83787044050948, 40.79844190861319], [-73.83839013354552, 40.798304872075995]]], [[[-73.8678840747076, 40.78736103669457], [-73.86803304873462, 40.7871820299527], [-73.86813032000018, 40.78723277784419], [-73.86918113468879, 40.78594348978492], [-73.8692349173292, 40.78596991495815], [-73.86817127125168, 40.78725619565448], [-73.86823784768373, 40.787279643084794], [-73.8680991225789, 40.78745866181839], [-73.8678840747076, 40.78736103669457]]]]}}, {\"id\": \"53\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 54, \"Shape_Leng\": 0.0762192448554, \"Shape_Area\": 0.000132452130536, \"zone\": \"Columbia Street\", \"LocationID\": 54, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-74.00174362072502, 40.69240674985266], [-74.00400795971088, 40.693208199632345], [-74.00374561906423, 40.69361620298103], [-74.0034621809253, 40.69405701507471], [-74.00127014951309, 40.69329606183691], [-74.0015417289999, 40.69278596299994], [-74.00174362072502, 40.69240674985266]]], [[[-74.00174362072502, 40.69240674985266], [-74.0016998939999, 40.692382766999906], [-74.00110519399988, 40.692056594999954], [-74.00031547799995, 40.69187465199993], [-74.00021419499998, 40.69185176099989], [-73.99991517399985, 40.69176722999993], [-73.99951274099993, 40.6916514499999], [-73.99936540199982, 40.69161307799989], [-73.99928819599991, 40.691593959999935], [-73.99920161899989, 40.69157183399992], [-73.99910057299985, 40.69153697399986], [-73.9984330559999, 40.691364914999895], [-73.99803048599995, 40.69125372299988], [-73.99776924399993, 40.691194697999954], [-73.99808645399993, 40.690535077999876], [-73.99841229299983, 40.68985608799994], [-73.99874207699997, 40.689177013999924], [-73.99906961899994, 40.68849807499985], [-73.99937882999994, 40.68785319299995], [-73.99963768500004, 40.68731476799996], [-73.99969626699995, 40.68719803399989], [-74.0006473809999, 40.68525394699991], [-74.0009581009999, 40.68457931899991], [-74.00128182399995, 40.6839096399999], [-74.00160876999993, 40.683235204999924], [-74.00193169899997, 40.68256321799992], [-74.002204589, 40.68199132399995], [-74.00225222099995, 40.681893211999906], [-74.00256464499995, 40.681228463999936], [-74.00289363599995, 40.680559409999894], [-74.00329231299999, 40.67964328799992], [-74.0033891689999, 40.679721305999905], [-74.00345372099991, 40.679789886999885], [-74.00371762999994, 40.68002103199993], [-74.00378241799994, 40.6800860019999], [-74.00447389899988, 40.680993011999846], [-74.00456896599997, 40.68114997599986], [-74.00503641499995, 40.681910287999884], [-74.00560068499993, 40.68281599899994], [-74.00567492299993, 40.68292992799995], [-74.00596041099988, 40.683362870999936], [-74.00581925399995, 40.683656554999914], [-74.0054980969999, 40.68432595799987], [-74.0051790759999, 40.68499722499988], [-74.00485241699997, 40.68566813099993], [-74.00549714699999, 40.68584772699993], [-74.0071710969999, 40.68635700999991], [-74.00753083199993, 40.68607448699991], [-74.00771707547891, 40.68626442900919], [-74.00754474403942, 40.686318244593565], [-74.00701988682644, 40.686526417194024], [-74.0064414683233, 40.68628849023208], [-74.00588224183223, 40.6861957688936], [-74.00566680246698, 40.68669807883377], [-74.00562986330395, 40.686784206043626], [-74.00783293766679, 40.68738505516274], [-74.00742012154092, 40.68820629043599], [-74.00500373315064, 40.687605985400886], [-74.00459270521766, 40.688215222986614], [-74.00678438439806, 40.68882687995826], [-74.00636461225261, 40.68966966863697], [-74.00382038919723, 40.688929644699236], [-74.00340921624516, 40.689613261160105], [-74.00575900670344, 40.69023640001614], [-74.0053420962491, 40.6910916029778], [-74.00297565436571, 40.69042657592073], [-74.00196321992853, 40.69013637946073], [-74.00185304524615, 40.69038798674182], [-74.0009615685494, 40.69012876835003], [-74.00069905841875, 40.690611858419075], [-74.00479415294552, 40.69176162037443], [-74.00440941259976, 40.692528800389354], [-74.00196938526075, 40.69184320831752], [-74.00181765040728, 40.692221962481405], [-74.00174362072502, 40.69240674985266]]], [[[-74.01092841300002, 40.684491472999824], [-74.01193259955522, 40.683887749077535], [-74.01217596614636, 40.68409518562848], [-74.00878590130924, 40.686146602298834], [-74.00869559889543, 40.6861933180126], [-74.00859803329703, 40.686252564084974], [-74.00835328548824, 40.6864001294722], [-74.00816320571415, 40.68617364485845], [-74.00842516199995, 40.68601553199992], [-74.00851294299997, 40.68596254899994], [-74.0086007239999, 40.68590956499989], [-74.01092841300002, 40.684491472999824]]]]}}, {\"id\": \"54\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 55, \"Shape_Leng\": 0.149956807524, \"Shape_Area\": 0.000736274713586, \"zone\": \"Coney Island\", \"LocationID\": 55, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98671582299981, 40.58456849999991], [-73.98627573699999, 40.584479980999895], [-73.9858605929999, 40.5845257129999], [-73.98548157999991, 40.58451967699994], [-73.98492730999999, 40.5845959529999], [-73.98482192899988, 40.584168522999924], [-73.983875933, 40.58397574499992], [-73.98281157699994, 40.58379238799989], [-73.98175452099981, 40.58358376399984], [-73.98044790899985, 40.58341064299993], [-73.98006567999988, 40.583317863999845], [-73.97742525299988, 40.58290659299994], [-73.97468194399984, 40.58312411799988], [-73.97461091899993, 40.5831298759999], [-73.97449712999993, 40.583153389999886], [-73.97308506499995, 40.5834298259999], [-73.97174499599997, 40.58366274899991], [-73.970846422, 40.58382021499993], [-73.97045373499985, 40.583892278999876], [-73.970055463, 40.58394772999985], [-73.9698544559998, 40.58396698199992], [-73.96899979299997, 40.584063502999896], [-73.96781584999998, 40.5845390509999], [-73.9668442979999, 40.584569328999876], [-73.96670868800001, 40.584569082999906], [-73.96661531499983, 40.58457103699992], [-73.96670866299985, 40.58423930199995], [-73.96673011799997, 40.584135972999874], [-73.96674664199992, 40.584055351999886], [-73.96676128900002, 40.5839843919999], [-73.96677678399986, 40.58391498999987], [-73.966813755, 40.58375771499986], [-73.96683458399988, 40.583676861999926], [-73.96685766099992, 40.58358661799986], [-73.96691975199984, 40.58335247299993], [-73.96701115999986, 40.58294762399991], [-73.96707277600002, 40.582669811999864], [-73.96708956100004, 40.58259579399989], [-73.96733403999997, 40.58151752199991], [-73.9675871379998, 40.58042905899994], [-73.96759867799989, 40.58037846999991], [-73.96761555699982, 40.58030516299993], [-73.96804495099995, 40.578438532999854], [-73.96834802499998, 40.577113222999856], [-73.96852475500002, 40.57634038099996], [-73.96853234099981, 40.57630503099995], [-73.96854635499986, 40.57624267099993], [-73.96865108899983, 40.57576305399993], [-73.96866614199996, 40.57569696799993], [-73.96868119599986, 40.57563088299987], [-73.96873173200001, 40.57550140999989], [-73.96880187199986, 40.57537736399993], [-73.96883335399993, 40.575335858999864], [-73.9688899589999, 40.57526123899986], [-73.96883249999995, 40.57524150099993], [-73.96853345900001, 40.57513875999993], [-73.96840165999983, 40.57508536799986], [-73.9682587889998, 40.575045488999876], [-73.96810857699987, 40.57502089899985], [-73.96795533399992, 40.57501255399993], [-73.96780353599996, 40.57502045399985], [-73.96765743500002, 40.575043681999915], [-73.96756707299996, 40.574881526999896], [-73.96675354099992, 40.574832917999906], [-73.96471128899991, 40.57478261799983], [-73.96286117999982, 40.57498014099989], [-73.96103342799984, 40.57507632399994], [-73.95941778299985, 40.57517798299994], [-73.95939182223685, 40.57391210937612], [-73.95982864911235, 40.573929309635766], [-73.96021532299316, 40.573911179120955], [-73.9604890840351, 40.57388518154406], [-73.96068755363642, 40.57387742198219], [-73.96081416941384, 40.57386180993874], [-73.96090885893602, 40.57386502894637], [-73.96210769550765, 40.57371352001862], [-73.96267607127344, 40.57366693903902], [-73.96305899786316, 40.573635553517924], [-73.96398292380015, 40.57355757223531], [-73.96453618454449, 40.57351319942493], [-73.96508478576706, 40.5734691986379], [-73.96602239542973, 40.57338859483251], [-73.96720294103956, 40.573326317397424], [-73.96738975478867, 40.573258999904446], [-73.96743567793125, 40.57320896967127], [-73.96736390080565, 40.57304456895217], [-73.96739472238792, 40.5729950026884], [-73.96742628474536, 40.57298510896918], [-73.96744946374342, 40.573008064912955], [-73.96753149753678, 40.573190732887284], [-73.96762388104095, 40.57319858543997], [-73.96779155223932, 40.57318819631816], [-73.96818506098592, 40.573177868546125], [-73.96839037828869, 40.57315444294007], [-73.96856335985457, 40.57312594655183], [-73.96884893944646, 40.573058027445015], [-73.96897214113847, 40.57302153230467], [-73.96906454391251, 40.57298763701027], [-73.96920229198774, 40.572957418516594], [-73.96945126202378, 40.57285728036321], [-73.96946838633161, 40.5728233657055], [-73.96921197952125, 40.57233537642345], [-73.96924621215574, 40.572304075654515], [-73.96928385228033, 40.57230147645185], [-73.96955895151744, 40.57280858046076], [-73.96975580336873, 40.572846924690374], [-73.96996110201732, 40.57286002458749], [-73.97021089387842, 40.57285226198176], [-73.97085363074368, 40.57277051527984], [-73.97108691689401, 40.57272724421559], [-73.9713298962632, 40.572643810384115], [-73.97135044587482, 40.572599460045254], [-73.97137785791334, 40.572510753098854], [-73.97114248794517, 40.57205063151452], [-73.97113515153814, 40.571960152135816], [-73.97118306355702, 40.571941899084166], [-73.97122069416346, 40.57196278220161], [-73.97145657076729, 40.572479462745875], [-73.9715179092025, 40.57258168240442], [-73.97160706452378, 40.572628223919835], [-73.97183632056803, 40.57262828035426], [-73.97249673747078, 40.5725762567474], [-73.97293476162697, 40.57247460264789], [-73.97314693344347, 40.57241463999529], [-73.9733450222981, 40.57232121400607], [-73.97336256224074, 40.57226335704354], [-73.97323268078627, 40.571905865967], [-73.97321224564611, 40.57166842470265], [-73.97324990115135, 40.57162668661163], [-73.97329095478177, 40.57164235029656], [-73.97332515756229, 40.571676279132454], [-73.97334306976626, 40.57174057042443], [-73.97334904033113, 40.571851099622315], [-73.97336611476469, 40.57193720809319], [-73.97339000577036, 40.57208854678842], [-73.97342758666748, 40.57223467076782], [-73.97344805325683, 40.5722721069693], [-73.9737320997119, 40.57228692509402], [-73.97400678258738, 40.57227436254645], [-73.97440961940833, 40.57223750300214], [-73.97484762696547, 40.57216454176584], [-73.97528564967088, 40.572049832668604], [-73.9754708190279, 40.57196900381691], [-73.97540561019994, 40.57150453486901], [-73.97540564094209, 40.57142104064054], [-73.97545696559193, 40.5714210516176], [-73.97548432668431, 40.571454978349074], [-73.9755088303412, 40.57152449244048], [-73.97551718820384, 40.57169431528035], [-73.97560269948812, 40.57185928310358], [-73.97567918029272, 40.57195598449735], [-73.97592552091535, 40.57201865743752], [-73.97621978857089, 40.57201611012744], [-73.97684256247973, 40.571948398935305], [-73.97730110158642, 40.571862385946396], [-73.97766384588132, 40.571729387799046], [-73.97756820091203, 40.57124666665904], [-73.97759877204504, 40.57119103051838], [-73.97767087523813, 40.57117102007902], [-73.978047249339, 40.57119979395958], [-73.97807599798215, 40.571229570128686], [-73.97771533072489, 40.571251913842254], [-73.97769819728057, 40.5713249683295], [-73.97770502384324, 40.5713771534471], [-73.97773918874084, 40.57153371211849], [-73.97775127949755, 40.57168028978257], [-73.97776503791167, 40.57170769410005], [-73.9777840795185, 40.57173319377245], [-73.97780796518646, 40.571756200726284], [-73.97783614406792, 40.57177618437464], [-73.97788258903813, 40.57179554497028], [-73.97793080193595, 40.57181220403646], [-73.97799924492543, 40.571830476810476], [-73.97806978098814, 40.57184331224861], [-73.97819660524176, 40.57186619222339], [-73.97832516078613, 40.57188251422633], [-73.97845486511183, 40.5718922042992], [-73.97858513050289, 40.571895218534415], [-73.9787153667011, 40.57189154327378], [-73.97884498357945, 40.57188119517073], [-73.97897339381907, 40.57186422111464], [-73.97910001556873, 40.57184069801798], [-73.97945681046644, 40.571742768460936], [-73.9795218304509, 40.57171407873419], [-73.97976481057256, 40.57158627158868], [-73.97984694469402, 40.571539319610174], [-73.97986064824686, 40.5714819205131], [-73.97980592543583, 40.57140363454865], [-73.97972382225471, 40.57134621741238], [-73.97966565809048, 40.5713331610994], [-73.9794672106511, 40.57129920658622], [-73.97945877680687, 40.57130181734474], [-73.97944987919773, 40.571303280990314], [-73.97944078394436, 40.57130355374672], [-73.97943176307878, 40.571302627455914], [-73.97942308640852, 40.5713005298225], [-73.97941501344556, 40.571297323585156], [-73.97940778564593, 40.57129310463992], [-73.97940161918756, 40.57128799917216], [-73.97939669850376, 40.571282159882365], [-73.97939317076805, 40.57127576141902], [-73.97939114149196, 40.57126899515512], [-73.97939067136882, 40.57126206346414], [-73.97944328367615, 40.57122092594798], [-73.9795835711477, 40.57122616884359], [-73.97975464745674, 40.57124968258197], [-73.97995988926644, 40.57144019050048], [-73.97998743713075, 40.57158360582356], [-73.98000771572241, 40.571698509185964], [-73.98010351551893, 40.57172722648448], [-73.9803259222601, 40.57173770221856], [-73.9806489086185, 40.571737732454054], [-73.9810444940945, 40.57168042036584], [-73.98133877923644, 40.57161002034813], [-73.98163648890623, 40.57152657358946], [-73.98184180651329, 40.57146920355463], [-73.9820407585251, 40.57148037052242], [-73.98203685076774, 40.5714379242801], [-73.98201633411738, 40.57138834604919], [-73.9819923913309, 40.5713518133093], [-73.98192054501716, 40.57132049258673], [-73.98169656909477, 40.571311371820634], [-73.98159891057547, 40.571307395484105], [-73.98155785470236, 40.571294342868484], [-73.98154417427375, 40.57127346698696], [-73.98157155423429, 40.5712499884248], [-73.9819479404238, 40.57123700215743], [-73.98202662918752, 40.57127615208919], [-73.98209504346248, 40.5713466114785], [-73.98214632133059, 40.57152143608206], [-73.98217710691104, 40.57156057845449], [-73.98225580253015, 40.57156841828967], [-73.98249190051142, 40.57156584591296], [-73.98282723201832, 40.57154241357462], [-73.98308044496548, 40.571508531433324], [-73.98367270153267, 40.57140705768737], [-73.98340440737672, 40.57016554451071], [-73.98305809245075, 40.57020965626845], [-73.98304328883707, 40.570145860988035], [-73.98339168436635, 40.57009468501941], [-73.98326996675307, 40.569541894356284], [-73.98336058039273, 40.56952999448666], [-73.98348521866141, 40.57008901506274], [-73.98382125394879, 40.57004578998432], [-73.9838394948926, 40.570112467157976], [-73.98348842146662, 40.57015603054908], [-73.98375790448641, 40.571396227250865], [-73.98572931754806, 40.571195889981496], [-73.98593538175733, 40.57117818892235], [-73.9864508758885, 40.57114890891035], [-73.98646799852705, 40.571078462725126], [-73.98650905821334, 40.57108107733054], [-73.98654668594091, 40.57113326486579], [-73.98686148425026, 40.5711072087355], [-73.98726866835315, 40.57107594447696], [-73.98752187630423, 40.571047270923586], [-73.98777508425758, 40.5710185968119], [-73.98803716678532, 40.57098510876868], [-73.9886202548146, 40.570896051314094], [-73.98857772038879, 40.57069790950493], [-73.9886339755543, 40.57069514448042], [-73.98866476412249, 40.570729067240016], [-73.98869554520905, 40.57081517428702], [-73.98872145848834, 40.57088369208261], [-73.98883582208126, 40.570877808557896], [-73.98892820617891, 40.5708752082258], [-73.98914719716011, 40.570843919183126], [-73.98935934175859, 40.57083089249508], [-73.9896465753043, 40.57079215540961], [-73.98993761163376, 40.570757887126035], [-73.99014629531219, 40.5707260797864], [-73.99026267527424, 40.57070834011726], [-73.99052614849931, 40.57065878712539], [-73.9907731032393, 40.570599421837834], [-73.99074174672937, 40.570403103606914], [-73.99071781592744, 40.57025959565118], [-73.99079264766938, 40.57024864875552], [-73.99082728598098, 40.57042137437878], [-73.99086374684879, 40.570602016242326], [-73.99119337523612, 40.5706118749683], [-73.99156291512742, 40.57057798348913], [-73.99189139756308, 40.57053364999877], [-73.9922061931928, 40.57049975207031], [-73.99229545219589, 40.57049106967158], [-73.99252782858784, 40.57046846400287], [-73.9927285489706, 40.5704337880374], [-73.99296923014857, 40.57036673204229], [-73.99294529319609, 40.57023627108529], [-73.99298098072161, 40.57019677330923], [-73.99301372686757, 40.570212792475736], [-73.99302740713195, 40.57027802296811], [-73.99305289944259, 40.57033902555883], [-73.99315741886727, 40.570379789438825], [-73.9932977052498, 40.57038240748573], [-73.99362618034408, 40.570392863610316], [-73.9936750974706, 40.570395135670054], [-73.99401966538295, 40.570411148678886], [-73.99440631046578, 40.57039029447153], [-73.99468360322813, 40.57036133741584], [-73.99480664237873, 40.57036422070033], [-73.99493763229952, 40.570354674512366], [-73.99513854436809, 40.57031466120666], [-73.99515224064955, 40.570178982743485], [-73.9951351400427, 40.570066787611914], [-73.99518726085735, 40.570021877633096], [-73.99524805213261, 40.57009810177863], [-73.9952617292692, 40.570228562336936], [-73.99529301203255, 40.57032128271386], [-73.99551492045956, 40.57035381411382], [-73.99576469677095, 40.570392962362455], [-73.99600078794127, 40.57039297082444], [-73.99640796358749, 40.570374718693564], [-73.99665089870697, 40.57037211675966], [-73.99684901717619, 40.570366965468814], [-73.99699640776534, 40.57036269138141], [-73.99704438661118, 40.57035647342443], [-73.99719493825835, 40.570348649043794], [-73.997318117723, 40.57032256042304], [-73.99735575750893, 40.570278204147925], [-73.99735576071078, 40.57019731968318], [-73.99736944882595, 40.57013730754285], [-73.99741966158959, 40.570140854445846], [-73.99744814048576, 40.57028603439299], [-73.99751999355462, 40.57032517356224], [-73.9976671213519, 40.57037475116615], [-73.99784504412378, 40.57041389391206], [-73.997995595213, 40.570447815853086], [-73.99818720620968, 40.57046086506931], [-73.99829669781617, 40.57047391227484], [-73.99846777900149, 40.570497396973], [-73.9986324281592, 40.570508006690254], [-73.99899128811825, 40.57048957527737], [-73.99922738028934, 40.57048957710792], [-73.9996140257524, 40.57046609660146], [-73.9996413990163, 40.570445223004704], [-73.99965850686432, 40.570382602502804], [-73.99970321419985, 40.5703864975226], [-73.9997329117931, 40.57041151211029], [-73.99972950792677, 40.57045044112285], [-73.99971929823451, 40.570473797740014], [-73.9997485215654, 40.57049579695692], [-73.99995444080645, 40.57049153081292], [-74.00015998281083, 40.57048111670441], [-74.00036483202332, 40.57046457061954], [-74.00056867395234, 40.57044191796051], [-74.00077119565373, 40.57041319350455], [-74.00091800593184, 40.57041051221619], [-74.0010643863451, 40.57040154143544], [-74.00120987388024, 40.57038630953778], [-74.00135400834894, 40.57036486470276], [-74.0016656216866, 40.57030728168833], [-74.0018778651766, 40.57024746431582], [-74.002011670539, 40.570208758524196], [-74.00205780931616, 40.570152462901795], [-74.00198396888365, 40.5695965418238], [-74.00208547639728, 40.569585983985014], [-74.00220084969976, 40.570303754391674], [-74.00222854320391, 40.57060986276879], [-74.00234852717965, 40.57110596794083], [-74.00248707737698, 40.57173838887243], [-74.00254067002723, 40.57179380137707], [-74.00259949630396, 40.57184603577699], [-74.00266323324675, 40.571894805291336], [-74.00273153093207, 40.57193984216144], [-74.00277776235734, 40.571997860624464], [-74.00283150615377, 40.57205198482916], [-74.0028922028448, 40.57210165132107], [-74.00295922056691, 40.57214634305124], [-74.00303186164808, 40.57218559475904], [-74.00310936987007, 40.57221899781597], [-74.00319093834194, 40.57224620448044], [-74.00327571790024, 40.57226693151758], [-74.00336282594905, 40.572280963148565], [-74.00345135565061, 40.5722881532969], [-74.00348277706472, 40.57227667938334], [-74.00351575170326, 40.572268106676184], [-74.00354982074944, 40.572262554458256], [-74.00359727599627, 40.572259984064814], [-74.00364465678643, 40.57226325710292], [-74.00369073370504, 40.5722722886454], [-74.0037343111684, 40.572286844345854], [-74.00377425844584, 40.572306546519286], [-74.00380953899872, 40.57233088394228], [-74.00383923737562, 40.57235922511667], [-74.00386258296655, 40.57239083465626], [-74.00387896999834, 40.57242489236713], [-74.00388734496535, 40.57245649855172], [-74.00388971484409, 40.57248869279238], [-74.00396101384514, 40.57255240674605], [-74.00403859723095, 40.57261169787529], [-74.00412199472655, 40.572666206772126], [-74.0042107008097, 40.57271560301656], [-74.0043041777745, 40.5727595871793], [-74.00440185899215, 40.57279789263783], [-74.00450315234521, 40.57283028719197], [-74.00460744381672, 40.57285657447203], [-74.0047141012138, 40.572876595129344], [-74.0048224779998, 40.57289022780212], [-74.00666915632603, 40.57339364837534], [-74.00870865730187, 40.57375981858765], [-74.00942893690326, 40.57379726841015], [-74.01007410785421, 40.57383080951765], [-74.01071306846103, 40.57402909731149], [-74.01115668729254, 40.57416676185885], [-74.01190196490126, 40.574790166507505], [-74.01201540191126, 40.57488505247416], [-74.01232183520665, 40.57565268236508], [-74.01236788614587, 40.57585681441278], [-74.01240471798019, 40.57606203174], [-74.01243228719036, 40.57626809234315], [-74.01245056117921, 40.576474753222335], [-74.01245951831018, 40.57668177066747], [-74.01279721934527, 40.577489237133136], [-74.01286422267526, 40.57756332933402], [-74.0129242497425, 40.57764082044216], [-74.0129770052135, 40.57772132923269], [-74.01302222952468, 40.577804459633505], [-74.01303113457197, 40.57785701811856], [-74.0130314800179, 40.57791001271724], [-74.0129633543014, 40.57803631343701], [-74.01290105351399, 40.5781643507723], [-74.01284465365903, 40.578293968594366], [-74.01279422354547, 40.57842500884618], [-74.01274982470373, 40.57855731173525], [-74.01235035929233, 40.57922182165903], [-74.0122466287476, 40.57935557381623], [-74.01213565146023, 40.57948590084882], [-74.01201762162991, 40.57961257464747], [-74.01189274580409, 40.57973537349643], [-74.01176124251562, 40.57985408246062], [-74.01162334190214, 40.5799684937623], [-74.0114792853013, 40.58007840714522], [-74.01132932483026, 40.58018363022479], [-74.0111737229432, 40.58028397882522], [-74.01101275197246, 40.58037927730177], [-74.01086035571247, 40.58046224526471], [-74.01070388953003, 40.580540686107405], [-74.01054358463756, 40.58061448390999], [-74.01037871958451, 40.58063917725279], [-74.01021580270462, 40.580670498035765], [-74.01005529194393, 40.58070835821916], [-74.00989763848804, 40.58075265138161], [-74.00974328549282, 40.58080325301871], [-74.00959266683995, 40.58086002089339], [-74.00944620591711, 40.5809227954353], [-74.00930431442846, 40.58099140018941], [-74.00919506149998, 40.58104995924493], [-74.00908914150955, 40.58111197592423], [-74.00898674257299, 40.58117734009006], [-74.00888804655423, 40.58124593565999], [-74.00886572727528, 40.581260159293336], [-74.0087265097835, 40.581343762669775], [-74.00858199198898, 40.58142196326759], [-74.00843253422542, 40.5814945660956], [-74.00827850914793, 40.58156139011996], [-74.00812030080365, 40.581622268715506], [-74.00795830367427, 40.58167705008141], [-74.00779292169176, 40.581725597620476], [-74.00762456723098, 40.58176779027861], [-74.00745366008165, 40.58180352284798], [-74.00647984834727, 40.58192825746383], [-74.00604734235807, 40.58198365303581], [-74.0048362757677, 40.58185484358028], [-74.0039802361642, 40.58172129566388], [-74.00294374157795, 40.581432835521014], [-74.00220582876804, 40.581365886020336], [-74.0018599131742, 40.58138751901811], [-74.0012403263368, 40.58142626537587], [-74.00083687503617, 40.58134456441586], [-74.00073378767112, 40.58131818388558], [-74.00062901792148, 40.58129598823696], [-74.00050742414106, 40.58127580080334], [-74.00038445327529, 40.5812612491398], [-74.00026055424323, 40.58125238636873], [-74.00013617935197, 40.581249244844756], [-74.00001178264505, 40.581251836036216], [-73.99988781824608, 40.58126015048382], [-73.99976473870005, 40.58127415783461], [-73.99964299332305, 40.581293806953454], [-73.99952302656166, 40.58131902610889], [-73.99936820504236, 40.581352942365946], [-73.99918928335447, 40.58139024745273], [-73.99900970569459, 40.58142567324118], [-73.99891497675803, 40.58145143567648], [-73.99881798626606, 40.58147172512276], [-73.99871927734135, 40.58148642796408], [-73.99861940273051, 40.58149546186772], [-73.99851892170828, 40.58149877624593], [-73.9984183969461, 40.58149635253897], [-73.99831839136024, 40.58148820431907], [-73.9982194649592, 40.58147437721429], [-73.99812217170776, 40.58145494865329], [-73.99807180872422, 40.58141850756346], [-73.99802693464608, 40.58137811167257], [-73.99798808035105, 40.58133423888848], [-73.99795570549585, 40.5812874082524], [-73.99793019307933, 40.5812381737973], [-73.9979118449124, 40.581187117993416], [-73.99790087804635, 40.58113484485748], [-73.99789742220709, 40.581081972806], [-73.9979196930635, 40.5810072072905], [-73.99794615384133, 40.580933239811536], [-73.99797675572093, 40.580860206799585], [-73.99798815806982, 40.58082943193382], [-73.99799310469567, 40.58079767587739], [-73.99799147376537, 40.58076572082806], [-73.99798330546042, 40.58073435388409], [-73.99796880098621, 40.58070434765646], [-73.99794831761601, 40.58067644123903], [-73.99792055699606, 40.58064984875662], [-73.99788738519247, 40.580627144377196], [-73.99784972512147, 40.58060895978559], [-73.99780862456939, 40.58059580091587], [-73.99776522704225, 40.580588033875834], [-73.9921546480924, 40.579320625935416], [-73.9918291279026, 40.57924708092878], [-73.99174714246814, 40.579235614184704], [-73.9916642362242, 40.579229040197816], [-73.9915809106584, 40.579227398733515], [-73.99149224635235, 40.57923108680759], [-73.99140428678798, 40.57924035855055], [-73.99131763577279, 40.57925515031558], [-73.99123288813318, 40.579275360563095], [-73.99115062563098, 40.57930085055747], [-73.99107141296983, 40.57933144531963], [-73.99069770945255, 40.579373051599745], [-73.99036206466565, 40.579424580120424], [-73.98969501304418, 40.57921326907598], [-73.98957956138615, 40.57919294169361], [-73.98944797186188, 40.579055815691504], [-73.98899093556328, 40.57883509996882], [-73.9884283376944, 40.57881584261956], [-73.98814208290418, 40.57886596371379], [-73.98798324056253, 40.57894533574087], [-73.98760837934643, 40.57903313614564], [-73.98761631521326, 40.57906416865276], [-73.98785456901827, 40.5790036437428], [-73.98786647687928, 40.579026351107814], [-73.9875128581447, 40.57911569169754], [-73.9874515182491, 40.579131513198426], [-73.98744159488433, 40.579107291373404], [-73.98746186967524, 40.57910287834868], [-73.98759745387363, 40.57906643764763], [-73.98758653958217, 40.57904070176424], [-73.98748785269989, 40.57906401752726], [-73.98739064388411, 40.57908698306361], [-73.98738302206559, 40.5791375607141], [-73.98669539997061, 40.5798149081325], [-73.9867512931689, 40.57986429349332], [-73.98681225500573, 40.57991006023784], [-73.98687788391226, 40.57995190688341], [-73.98694774757386, 40.57998955776978], [-73.987021385777, 40.580022764875224], [-73.98678109583868, 40.58029393767769], [-73.98616526744581, 40.58095022879234], [-73.98588567394853, 40.581144424564776], [-73.98591295341917, 40.58122700200037], [-73.98592418776619, 40.58126743761216], [-73.98604509916208, 40.58172068515157], [-73.98684478996456, 40.58123210200515], [-73.98694506928851, 40.581136421805546], [-73.98703902188056, 40.581037078806155], [-73.98712641800256, 40.58093431595641], [-73.98720704395298, 40.58082838456841], [-73.98728070259058, 40.580719543701626], [-73.98734721381531, 40.580608059529524], [-73.98740641500886, 40.58049420468864], [-73.98745816143226, 40.58037825761143], [-73.98757400228179, 40.58003145122741], [-73.98774679739302, 40.57985965559591], [-73.98800973376738, 40.579670687155776], [-73.98811730541621, 40.57967968595251], [-73.98890353923805, 40.58000867868471], [-73.9891363620069, 40.580214877714866], [-73.98954195208073, 40.58041536536176], [-73.9896996939193, 40.580421107402984], [-73.98995504960881, 40.58066166976562], [-73.98971762499994, 40.5808382269999], [-73.98928724999996, 40.58195758299993], [-73.98888884999998, 40.58256733499994], [-73.98874850099989, 40.58282032299996], [-73.98852399299994, 40.58314353499986], [-73.98829948299982, 40.58346674699986], [-73.98730893199986, 40.58359445299993], [-73.98662350499995, 40.58367393699996], [-73.98654457099988, 40.583658590999946], [-73.98660338299986, 40.58376512099988], [-73.98679577199978, 40.58411362799992], [-73.98683590499999, 40.58418633099989], [-73.98687540199994, 40.58425158799985], [-73.98710120299997, 40.58464459199988], [-73.98701022999991, 40.58462626699987], [-73.98671582299981, 40.58456849999991]]]}}, {\"id\": \"55\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 56, \"Shape_Leng\": 0.0568478126677, \"Shape_Area\": 0.000180907844436, \"zone\": \"Corona\", \"LocationID\": 56, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85360568399982, 40.74749682799991], [-73.853254932, 40.74682354999987], [-73.8529235349999, 40.74618194799996], [-73.85257733899984, 40.7455056469999], [-73.85224097499999, 40.74484096599985], [-73.85189537499991, 40.74417973899993], [-73.85155107299987, 40.74350722699996], [-73.85135102799988, 40.743119056999895], [-73.85120929199985, 40.74284402099991], [-73.85086917500001, 40.742180411999875], [-73.85051160899988, 40.741518856999946], [-73.85020444599994, 40.74090203399989], [-73.85011556599989, 40.74072356299995], [-73.84996425999991, 40.74070177199997], [-73.84984431299986, 40.740665549999896], [-73.84965207399982, 40.740614663999935], [-73.8493530509999, 40.74052324999996], [-73.84892344899993, 40.740391914999854], [-73.84795885999989, 40.74009701599994], [-73.84780164699998, 40.740049676999874], [-73.84794174099986, 40.73981640499991], [-73.84823627999982, 40.739290400999955], [-73.84777844199982, 40.7393448349999], [-73.84775740199994, 40.739299088999864], [-73.84767033700003, 40.7391952349999], [-73.8475557979999, 40.7390639149999], [-73.84750820299992, 40.73900780699995], [-73.84801295299997, 40.73876334399993], [-73.84859447999989, 40.73850117099989], [-73.84956453799992, 40.738106091999875], [-73.8499544029998, 40.737960303999884], [-73.85079216499992, 40.73768175599991], [-73.8508760629999, 40.73765385999991], [-73.85130352699998, 40.73751172799992], [-73.85187067499986, 40.73734292899995], [-73.85244443499992, 40.737202311999916], [-73.85340136499995, 40.73700275799991], [-73.85415418099988, 40.73687781299987], [-73.85505083399991, 40.736720220999885], [-73.85567427699986, 40.73658665799995], [-73.85628567699989, 40.73642604299987], [-73.8566154109999, 40.73633083699988], [-73.85801767699988, 40.73590708399994], [-73.85862016399997, 40.73573990699991], [-73.85872465799989, 40.73570864399988], [-73.85877141199997, 40.7357852899999], [-73.85886555799988, 40.73593913899987], [-73.85942834899984, 40.73578873199989], [-73.86106394499986, 40.73531449099991], [-73.862840003, 40.73479949499992], [-73.86462193299997, 40.73428275699993], [-73.86481505400003, 40.73466021599994], [-73.86515865399998, 40.73532501399992], [-73.86550061599992, 40.73598933199985], [-73.86582955999995, 40.73664322399992], [-73.86609463699996, 40.73725142999991], [-73.8664801159999, 40.73821096399996], [-73.86676012199987, 40.73889036999984], [-73.86703973799995, 40.73957218999992], [-73.867146698, 40.73983722399991], [-73.86731779499996, 40.740261344999915], [-73.86756253399983, 40.740941888999956], [-73.8675572959999, 40.74167769699988], [-73.86747169199991, 40.74209572299991], [-73.86749135099993, 40.74330204699993], [-73.86768504899995, 40.744071717999915], [-73.86782691099991, 40.74462945699988], [-73.86785097499994, 40.74471654499991], [-73.86787595099985, 40.74481530599989], [-73.86790963899986, 40.74493371599989], [-73.8660885059999, 40.745473952999944], [-73.86407236699985, 40.7460736519999], [-73.86253100999993, 40.74653380199991], [-73.86032027699984, 40.747195698999896], [-73.86027336099988, 40.74710172799993], [-73.85734765199993, 40.74797143499989], [-73.85442575299999, 40.748851725999884], [-73.85436928099995, 40.748738929999874], [-73.85405453299994, 40.748158316999906], [-73.85382450799997, 40.74785403999988], [-73.8537828699998, 40.74780299099994], [-73.85360568399982, 40.74749682799991]]]}}, {\"id\": \"56\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 57, \"Shape_Leng\": 0.0192705048557, \"Shape_Area\": 1.80259807917e-05, \"zone\": \"Corona\", \"LocationID\": 56, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8513110499998, 40.74984337599988], [-73.85442575299999, 40.748851725999884], [-73.8544701179998, 40.748939347999936], [-73.85479286699997, 40.74956323099989], [-73.85513928899988, 40.75022639499993], [-73.85547890700003, 40.75089123399986], [-73.855834956, 40.7515772719999], [-73.85455552299993, 40.7519619189999], [-73.85497762399996, 40.75278221699993], [-73.85538540499991, 40.75357105999993], [-73.85579909099987, 40.75440952799994], [-73.85478630699988, 40.75470744899992], [-73.85383081399983, 40.75495833699992], [-73.85323754899999, 40.75509560099989], [-73.85312635499996, 40.75510934299989], [-73.85302119999993, 40.754618001999866], [-73.85300762299991, 40.75455456499989], [-73.8526172539999, 40.75340506499992], [-73.85246329699996, 40.75312740899989], [-73.85220450599998, 40.752660681999934], [-73.85185614699996, 40.75196793299992], [-73.85171508699987, 40.751687413999875], [-73.85151883499987, 40.751297136999916], [-73.85118150899994, 40.75062629599989], [-73.85098691399993, 40.75023929799988], [-73.85086439699992, 40.74999564199986], [-73.8513110499998, 40.74984337599988]]]}}, {\"id\": \"57\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 58, \"Shape_Leng\": 0.0598554094851, \"Shape_Area\": 0.000204980931361, \"zone\": \"Country Club\", \"LocationID\": 58, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.81561382067721, 40.84818955758243], [-73.81568607715445, 40.84816530989142], [-73.81575017833134, 40.848189775457044], [-73.81579073434887, 40.84816422113697], [-73.8157983885727, 40.84815939778304], [-73.81586274203565, 40.84809250102881], [-73.81590281420543, 40.848013121984266], [-73.81589711649299, 40.84789897954558], [-73.81589732510677, 40.8478236051903], [-73.81586136315966, 40.847775581851245], [-73.81577576231183, 40.84773090610563], [-73.8156945975249, 40.847713645630954], [-73.81563145161934, 40.847706692829895], [-73.81560442844047, 40.84768951881167], [-73.81560451391982, 40.84765868316219], [-73.81564516681594, 40.84764161781635], [-73.81569476839677, 40.84765197550431], [-73.81575340325438, 40.84765892106598], [-73.81580298434562, 40.84767613107147], [-73.81587968574792, 40.8476762535213], [-73.81595734753985, 40.84765673360196], [-73.81597449927862, 40.84765242209665], [-73.8160558923281, 40.8475874556508], [-73.81609670609068, 40.847512146535394], [-73.81612862993478, 40.84738885611984], [-73.81614705574096, 40.84725184039079], [-73.81617026811975, 40.84701547486435], [-73.81616622859919, 40.84684416216906], [-73.81614872235644, 40.84664884528276], [-73.81613006048687, 40.84652798288451], [-73.81611797359875, 40.84648732650394], [-73.81610537477145, 40.8464545444374], [-73.81607246774338, 40.846423657352936], [-73.81602685114903, 40.846400458770034], [-73.81584432887755, 40.8463269331039], [-73.81577586619423, 40.84630562530891], [-73.81571903834562, 40.8462934434988], [-73.81568488630352, 40.84634080490548], [-73.81568770320885, 40.846341840348664], [-73.81545867615472, 40.84660302877153], [-73.81533644376374, 40.84669088909078], [-73.81503963145569, 40.84687195469736], [-73.8149346545324, 40.84693500918082], [-73.81486160038685, 40.84698411349446], [-73.8146272778874, 40.84714279476088], [-73.81462363819276, 40.84716768999988], [-73.81466741414532, 40.84720400648179], [-73.81464594697542, 40.84721961352901], [-73.81451106769597, 40.84710607802672], [-73.81453370023934, 40.84709089939391], [-73.81457540884595, 40.847124816995624], [-73.81460847368038, 40.847127137241564], [-73.8148473509641, 40.846963571625594], [-73.81496489547806, 40.84688983746205], [-73.81515846332752, 40.84677136764705], [-73.8153287782355, 40.84666566380006], [-73.81543413390234, 40.84659072608568], [-73.8155040927001, 40.84651260985806], [-73.81547456779656, 40.84649683598922], [-73.81555508085557, 40.84640893914352], [-73.81558313973727, 40.846423627606036], [-73.81568288414337, 40.846307825124846], [-73.8156910465945, 40.84631082662619], [-73.81570353189448, 40.84629134042384], [-73.81566171153669, 40.84628809801731], [-73.81561559185766, 40.846293276014435], [-73.81561094027825, 40.846293798289835], [-73.81557285082476, 40.84630144567658], [-73.81551442131553, 40.84632255239668], [-73.81516370934223, 40.84649929275548], [-73.81511795800365, 40.84652427263893], [-73.81507222380488, 40.846543471616144], [-73.81503922076097, 40.84654727225552], [-73.81499863613446, 40.84653949889562], [-73.81496569829835, 40.84652017377916], [-73.81492265739294, 40.84648348869217], [-73.81484683784961, 40.84652123412636], [-73.8147021413662, 40.846603315666634], [-73.81458976583284, 40.846669227806274], [-73.81456579118257, 40.846683016022794], [-73.81451301364092, 40.8467133689247], [-73.81447890592031, 40.84673298372796], [-73.81443423469617, 40.84675867437311], [-73.8144095316957, 40.84677288069775], [-73.81435726288593, 40.84680294067716], [-73.81433272712044, 40.846817050760656], [-73.8142787652347, 40.84684808397358], [-73.8142426258526, 40.8468688674331], [-73.81420111199441, 40.84689274108202], [-73.81416157394456, 40.84691547990903], [-73.81412394940415, 40.846937116818296], [-73.81409776493562, 40.846952176568365], [-73.81404710378914, 40.84698131125024], [-73.81402010393954, 40.846996837897606], [-73.8139672085705, 40.847027257368424], [-73.81392771742684, 40.847049968716775], [-73.81389036650836, 40.84707144784833], [-73.81386532718723, 40.84708584851121], [-73.81381218552613, 40.84711640856845], [-73.81378699193706, 40.8471308984231], [-73.81373354969317, 40.84716163134604], [-73.81370805578399, 40.84717629257799], [-73.81365740839362, 40.84720527717644], [-73.81365654726848, 40.847215705153666], [-73.81370542127051, 40.84726491500232], [-73.81368634738199, 40.84727432550905], [-73.81361194639693, 40.847202595716325], [-73.81356372861671, 40.847151411278524], [-73.81358305958231, 40.8471407102681], [-73.8136274532093, 40.84718958339062], [-73.8136447278293, 40.84719024319919], [-73.81369588380107, 40.8471605188045], [-73.81372091416893, 40.8471461873514], [-73.8137731809535, 40.84711626217957], [-73.81380002008125, 40.84710089498112], [-73.81385116806617, 40.847071610254524], [-73.8138789424857, 40.84705570830289], [-73.8139064166056, 40.84703997824203], [-73.81395483717435, 40.84701225458044], [-73.81400644417954, 40.846982707009985], [-73.81403390211042, 40.84696698560396], [-73.81408421730943, 40.84693817754444], [-73.8141126142839, 40.84692191921119], [-73.81412248680294, 40.846916266110604], [-73.81428325967708, 40.84682421385871], [-73.81431749077636, 40.84680461522695], [-73.81442363291605, 40.846743841837906], [-73.81443094799361, 40.84673965478643], [-73.81458182304954, 40.846653268772705], [-73.81484315852319, 40.846505089301154], [-73.814908582032, 40.846468764878985], [-73.81465695793413, 40.846205545301444], [-73.81446907674172, 40.846234151701964], [-73.81442352579822, 40.84618782507239], [-73.81465507970388, 40.84596849742507], [-73.81460449034037, 40.84590867307145], [-73.81468170770279, 40.84586721536335], [-73.81475299202867, 40.84581753085871], [-73.81480534287805, 40.84577794558428], [-73.81486296240931, 40.84574356282199], [-73.81508593855773, 40.84559500341856], [-73.8150262760428, 40.845499995058404], [-73.81420573129112, 40.84601018088845], [-73.81399042132385, 40.84582691919379], [-73.81390002991992, 40.84586580865546], [-73.81318865072497, 40.8462295364752], [-73.81314460200237, 40.84616856213872], [-73.8138188057647, 40.84580248822438], [-73.81400589013697, 40.84566869163601], [-73.81401812695707, 40.84563145688775], [-73.81397964395012, 40.845649113005585], [-73.8134832683883, 40.84596517094709], [-73.81346409683638, 40.84594950610961], [-73.81397419150323, 40.845635554708096], [-73.81402255974176, 40.8456179666301], [-73.81403198219364, 40.845589297867896], [-73.81395424463112, 40.845521567553575], [-73.81368589093518, 40.8456590622548], [-73.81367220100483, 40.84564653246912], [-73.81394279128922, 40.8455115891112], [-73.81390206940699, 40.84547610835489], [-73.813856677988, 40.84541756288903], [-73.81372892648774, 40.845485068172216], [-73.81371114891479, 40.84546210844934], [-73.8138425886934, 40.84539939043908], [-73.8138054459058, 40.84535148296823], [-73.81363258232699, 40.84528076903137], [-73.81361921535247, 40.845306143731605], [-73.81361456155729, 40.845315742706], [-73.8136099439493, 40.84532525948444], [-73.81360627346879, 40.84533285629322], [-73.8135984047367, 40.84534902639254], [-73.81351835900998, 40.84551779014537], [-73.8135169818933, 40.84552058590417], [-73.8134852598153, 40.84558498593946], [-73.81345513687894, 40.84564614160247], [-73.81345137381052, 40.84565378114233], [-73.81342525440728, 40.84571018508857], [-73.8134056557891, 40.84575250478186], [-73.8134017466251, 40.845760946691], [-73.8133590558502, 40.84584549933101], [-73.81335311605325, 40.84585424368401], [-73.81333092964161, 40.84588690276115], [-73.81331925050839, 40.845904095438364], [-73.81328932692855, 40.84595102273163], [-73.81321295244251, 40.84592293984184], [-73.81321888080062, 40.84591364264857], [-73.81328359296052, 40.84593743814766], [-73.81330787049271, 40.84589936323206], [-73.81331213394002, 40.845892188380766], [-73.81333800338913, 40.84584866229296], [-73.81334354640917, 40.84583933649325], [-73.81336084994268, 40.84581022336595], [-73.81336591663198, 40.84580169873154], [-73.81343964459093, 40.8456501583979], [-73.81346215335952, 40.845601698458296], [-73.8134714600873, 40.845581657372854], [-73.81348768163403, 40.845546732410114], [-73.81349880426079, 40.84552278743233], [-73.81350354028108, 40.84551259137301], [-73.81352545669, 40.84546545122117], [-73.81352888367317, 40.84545807950861], [-73.8135513216049, 40.84540981761643], [-73.81355519045734, 40.84540149809662], [-73.81357718780484, 40.84535418233245], [-73.81358150186045, 40.8453449056292], [-73.81358334364413, 40.8453454641151], [-73.81359057586107, 40.84533097660404], [-73.81359235780202, 40.84532738391684], [-73.81360307935834, 40.84530593757009], [-73.81360529679493, 40.84530146843104], [-73.81360874968259, 40.845294529304766], [-73.81361858267053, 40.845275042388515], [-73.81346166413532, 40.84521084953066], [-73.81325997724689, 40.84523399045822], [-73.8132385841432, 40.84529364896686], [-73.81323306389241, 40.84530870462134], [-73.81320532266803, 40.845376859208855], [-73.81318496869365, 40.845426867328605], [-73.8131813263456, 40.84543581742801], [-73.81315836336442, 40.84549223296162], [-73.81312894063261, 40.84556452164169], [-73.81310414849683, 40.84562543004051], [-73.81310021214391, 40.84563482259397], [-73.81307003010966, 40.84570682624162], [-73.81306571530193, 40.84571712180477], [-73.81305592038038, 40.845740487583925], [-73.81303214849963, 40.845797201479186], [-73.81297222057104, 40.84593197685412], [-73.81296525153213, 40.84594764931379], [-73.81296109021649, 40.84595700916516], [-73.81293267079396, 40.84601706732122], [-73.81292791787256, 40.846027110714765], [-73.8128995185849, 40.84608712617606], [-73.81289475100886, 40.846097199698704], [-73.81286634522233, 40.84615722988279], [-73.8128616248963, 40.846167206986784], [-73.81283471482801, 40.846224073218984], [-73.81282910435463, 40.84623593117618], [-73.81280034773182, 40.84629500504941], [-73.81285506329952, 40.84631005254602], [-73.81284939958525, 40.84631879949822], [-73.81272457676761, 40.84628661994713], [-73.81273118509972, 40.84627639345535], [-73.81278637726436, 40.84628888969871], [-73.81281302458174, 40.846232678405585], [-73.81291272697044, 40.84602236357809], [-73.81291695576553, 40.846013094970935], [-73.81294340748332, 40.84595513052506], [-73.81294861826635, 40.845943712166594], [-73.81298050490136, 40.845873834682926], [-73.81298556073538, 40.845862755977556], [-73.81301735339505, 40.84579308572515], [-73.81302130291374, 40.845784431976575], [-73.81304250986348, 40.84573352064155], [-73.81308968175215, 40.845620268975544], [-73.81311515692455, 40.84555911090309], [-73.8131199180934, 40.84554768024854], [-73.81314023855164, 40.845496754893226], [-73.81315189449639, 40.845467545869504], [-73.8131663297347, 40.84543136866384], [-73.81316934230622, 40.84542381787774], [-73.81319008790805, 40.84537182875385], [-73.81320083160338, 40.84534490411173], [-73.81321337703977, 40.84531346728017], [-73.81321733224652, 40.84530355425585], [-73.81321883271909, 40.84530004019003], [-73.81324326350918, 40.84523590835256], [-73.8132073362552, 40.84524002990253], [-73.81318603211815, 40.845218681683086], [-73.8131607253336, 40.84520002274004], [-73.81313199687555, 40.84518448142888], [-73.81310050626846, 40.84517241453328], [-73.81306697644588, 40.84516409907407], [-73.81303217715568, 40.84515972594982], [-73.8129969072876, 40.84515939555485], [-73.81293647420665, 40.84514842232433], [-73.8128770359758, 40.8451346679459], [-73.81286150958843, 40.845174987722444], [-73.81299499392401, 40.84521129724105], [-73.81314914269447, 40.845199582814956], [-73.81315597044244, 40.84521210037631], [-73.8129934380874, 40.84522477118414], [-73.81285536957633, 40.845186285461466], [-73.81283121824967, 40.84525297023396], [-73.81275096195384, 40.84546508915683], [-73.81269357395503, 40.845603247141305], [-73.81263653799114, 40.84574901292879], [-73.81260846487879, 40.84582075804066], [-73.81258924696179, 40.845869871979886], [-73.81258526774707, 40.845880039579306], [-73.81251579437537, 40.84606170959747], [-73.81248886322167, 40.84613512255843], [-73.812469820348, 40.84618703139432], [-73.81246665727525, 40.846195654572554], [-73.81244657871599, 40.8462515223213], [-73.81236589325664, 40.84647602663034], [-73.81234003852444, 40.84654452112497], [-73.81241445328128, 40.846559380748744], [-73.8124095770743, 40.846570544319746], [-73.81227790900483, 40.846543458279285], [-73.81228170265241, 40.84653122414932], [-73.81232960133006, 40.84654151183249], [-73.81239225664594, 40.846359429128874], [-73.81240162310021, 40.84633220748286], [-73.81240460541035, 40.846323542299295], [-73.8124274491956, 40.8462571530233], [-73.8124437463115, 40.846213126925214], [-73.81245152428279, 40.84619211782991], [-73.81245518041634, 40.846182240029286], [-73.81246537830961, 40.846154694230144], [-73.8124794586283, 40.846116656183504], [-73.81249769032014, 40.846067405637086], [-73.81250124152137, 40.84605780927202], [-73.8125193042311, 40.8460090167925], [-73.81252247454867, 40.84600032225913], [-73.81254490240872, 40.845938814598576], [-73.8125481027327, 40.845930037858956], [-73.81256795450659, 40.84587559467993], [-73.81257091004653, 40.845867489982936], [-73.8125890919327, 40.84581762502888], [-73.81259307324952, 40.845806709773306], [-73.8126158658126, 40.84574419972368], [-73.81262066867791, 40.84573288173383], [-73.81264753175402, 40.845669588414026], [-73.81265211763225, 40.8456587843688], [-73.81267740391031, 40.84559920533424], [-73.81268166510716, 40.84558916500479], [-73.81270305301986, 40.84553876949518], [-73.81270737483655, 40.84552858603019], [-73.81273003634698, 40.8454696920823], [-73.81273369555522, 40.84546018215934], [-73.81275913732, 40.84539405886649], [-73.8127626091814, 40.84538503596562], [-73.81278779244019, 40.84531958368211], [-73.81279160895893, 40.84530966826943], [-73.81281480700318, 40.845249377688724], [-73.81281873708731, 40.84523916158629], [-73.81284143551395, 40.84518017067741], [-73.81260675421473, 40.845129560282885], [-73.81261378787293, 40.845109507865814], [-73.81284896543414, 40.84516302844035], [-73.8128577433794, 40.845129544589135], [-73.81278677056558, 40.845107764018586], [-73.81271815525626, 40.84508199258821], [-73.81265228592719, 40.8450523762078], [-73.8126418704152, 40.845039728473985], [-73.81263431789743, 40.845025957807046], [-73.81262983160204, 40.84501143476023], [-73.8126285322481, 40.84499655013244], [-73.81254457815619, 40.84497959800826], [-73.8124400843122, 40.84523707432019], [-73.81254866464359, 40.845260690103196], [-73.81253844344073, 40.845283730151216], [-73.81245544053235, 40.84526432796373], [-73.81218860345625, 40.8459521030058], [-73.81217481496283, 40.84594604165193], [-73.81195504551972, 40.84647582483539], [-73.81196127339025, 40.846484425279904], [-73.81203383713486, 40.84650169799506], [-73.81202172362966, 40.84652942842734], [-73.81203501629656, 40.84653260540141], [-73.8120266503986, 40.8465509818762], [-73.81191568266975, 40.84652738919134], [-73.8118845884477, 40.84659496706121], [-73.81230236497042, 40.846689407061106], [-73.81243625060671, 40.84673771207555], [-73.81242358654634, 40.84675536693241], [-73.81230255202037, 40.84671279219771], [-73.81187766058915, 40.84661655663379], [-73.81184149221674, 40.84671412757344], [-73.81169971823665, 40.84668170081364], [-73.81223386948285, 40.845396783522574], [-73.81224107628775, 40.84537890034358], [-73.81231816792578, 40.84518730651574], [-73.81231936981116, 40.845184317833315], [-73.81232956124433, 40.845159027573864], [-73.81234277106647, 40.84512618337472], [-73.81243223218368, 40.84486828248771], [-73.81243398195343, 40.844863208183774], [-73.81244786509014, 40.84482326774509], [-73.81251553354934, 40.84462820008201], [-73.81251753679987, 40.84462241253633], [-73.81252704775312, 40.844594927089545], [-73.81260112783652, 40.844381397924316], [-73.81260320565703, 40.844375445822116], [-73.81262135209091, 40.84432313360275], [-73.8126482067927, 40.84424569285329], [-73.81266141349049, 40.84420764950083], [-73.81277306108588, 40.84422651790778], [-73.81278395303981, 40.84422834819608], [-73.81278360795028, 40.844229471889484], [-73.81278360787154, 40.84422949986578], [-73.81271381182702, 40.844461863794514], [-73.81271242301725, 40.84446653629642], [-73.81271596620674, 40.84446710962054], [-73.8127238416698, 40.844468412509286], [-73.81272387652409, 40.84446841256587], [-73.81272698328182, 40.8444689386095], [-73.81272816252338, 40.84446912379517], [-73.81277618716038, 40.84428468345949], [-73.8127769894225, 40.8442823511525], [-73.81277790078785, 40.84427958161732], [-73.81277841251041, 40.84427733176299], [-73.81277881686847, 40.84427513668232], [-73.81278096800884, 40.84425543198108], [-73.81282121937906, 40.84425961554686], [-73.81279995233872, 40.84435487689028], [-73.81276123529807, 40.844473304438125], [-73.81276862682175, 40.84447858573546], [-73.81277607099767, 40.84448005192654], [-73.81280551188065, 40.844485864208735], [-73.81280554651427, 40.84448586426493], [-73.81299120304948, 40.84452250755805], [-73.81298325218074, 40.84454982704335], [-73.81297181218781, 40.84454745193529], [-73.81275955824992, 40.84450334094461], [-73.81276135037977, 40.844496606708994], [-73.81276183239135, 40.84449480309046], [-73.81274408650032, 40.84449191263888], [-73.81271493393892, 40.844487198761406], [-73.81271240439833, 40.84448678288029], [-73.81270337280483, 40.844485312432425], [-73.81270267989628, 40.84448750871942], [-73.8126978656709, 40.84450289819947], [-73.8126488933667, 40.84465946766736], [-73.81289731370842, 40.84471323145966], [-73.81291074426947, 40.84468373602105], [-73.81290225785118, 40.8446814440978], [-73.8129117569329, 40.84465804769341], [-73.81296220789088, 40.84467253925254], [-73.8129721379385, 40.84465018537361], [-73.8131131815271, 40.844683954956196], [-73.8130928806672, 40.84473104823433], [-73.81304510205834, 40.84471754979041], [-73.8130361971365, 40.844743284463085], [-73.81307041826864, 40.84475069087712], [-73.8130647312539, 40.844772889142], [-73.81311795023566, 40.84478760356964], [-73.81309338317085, 40.84484827629709], [-73.81265744462624, 40.84475144920646], [-73.81268023984991, 40.844690116161225], [-73.81267984609725, 40.84469003159224], [-73.81264095809888, 40.844681662260335], [-73.81254909120709, 40.84495227514607], [-73.81263397114111, 40.844970866820425], [-73.81264025469774, 40.84495917724791], [-73.81264904026584, 40.84494845795869], [-73.81266007700297, 40.84493901500449], [-73.81267304979318, 40.84493111799548], [-73.81268758824365, 40.844924992402994], [-73.81270327725996, 40.84492081312166], [-73.8127196688978, 40.844918699475926], [-73.8128536716289, 40.8449199046705], [-73.8129874745271, 40.84492558937607], [-73.81312081806666, 40.84493574256625], [-73.81325344361194, 40.84495034454784], [-73.8133517176982, 40.844953369980544], [-73.81344958137997, 40.844960824851476], [-73.81354668877825, 40.84497268281276], [-73.8135476194773, 40.84504186920363], [-73.81359095648517, 40.84504822935521], [-73.81363296498053, 40.84505851711379], [-73.8136558066588, 40.8450591224657], [-73.81367848758785, 40.84505698086103], [-73.81370043852486, 40.845052146049476], [-73.81372110854787, 40.84504473937447], [-73.81373997888441, 40.84503494672788], [-73.81375657593014, 40.84502301388406], [-73.81377048313631, 40.84500924033168], [-73.81378135146424, 40.844993971756715], [-73.81378890814463, 40.84497759136734], [-73.81383957780267, 40.84496631314284], [-73.81389185092073, 40.84496065565071], [-73.81394377728165, 40.84494924196862], [-73.81399303795402, 40.844932335656196], [-73.81403863128376, 40.84491028048288], [-73.814079630191, 40.84488352491331], [-73.81411520102003, 40.844852612988504], [-73.81414462049185, 40.84481817326274], [-73.81416729040971, 40.844780906022834], [-73.81418274982286, 40.84474156904799], [-73.81411591388145, 40.84476412450858], [-73.8138620492567, 40.8448183815554], [-73.81386694883041, 40.84484233215543], [-73.81374940735262, 40.844867067648444], [-73.81373976204523, 40.8448413061524], [-73.81369979287803, 40.84484845686337], [-73.81369162177985, 40.84482910503367], [-73.81389591464291, 40.84478890255925], [-73.8138938118485, 40.84477314504453], [-73.81395108050033, 40.844760197864176], [-73.8139610494715, 40.84477705546289], [-73.81403795661886, 40.844760857781395], [-73.81418653212441, 40.8447254159458], [-73.81418334597303, 40.84468219953711], [-73.81417097564207, 40.84463994715471], [-73.81414974846557, 40.844599776790766], [-73.81412022612626, 40.84456275134531], [-73.8140831897909, 40.84452985050305], [-73.81403961944139, 40.8445019448113], [-73.81399066794314, 40.84447977264607], [-73.81393763054012, 40.844463920675146], [-73.81388191058579, 40.84445480833596], [-73.81383041777484, 40.84446023443392], [-73.81381176562526, 40.844461915921144], [-73.81372273148042, 40.844451988439126], [-73.81365179646245, 40.84443683825208], [-73.81358891945234, 40.844397618968216], [-73.81355528835343, 40.84437910481038], [-73.81353932708109, 40.84434009802115], [-73.81353615806098, 40.84433235060963], [-73.81349521226797, 40.84428217748505], [-73.81345592741259, 40.844242626879904], [-73.81343074980109, 40.844187440444145], [-73.81342078460055, 40.84412370037605], [-73.81341603573895, 40.84406308117255], [-73.81339678166465, 40.84400775336993], [-73.81337679294293, 40.8439574595646], [-73.81336555156658, 40.84390562317463], [-73.81336324694246, 40.84385311769116], [-73.81336991788451, 40.84380082787789], [-73.8133854519607, 40.84374963486259], [-73.81340958738964, 40.843700401288686], [-73.81344191745198, 40.84365395677988], [-73.81348189734598, 40.843611083960184], [-73.81354351238565, 40.84357829593178], [-73.813596874996, 40.843556684842824], [-73.81365345267933, 40.84354046171452], [-73.81371230684839, 40.84352989567722], [-73.8137724611528, 40.84352516201438], [-73.81383291767408, 40.843526339254254], [-73.81389267348071, 40.8435334078671], [-73.81395479579975, 40.84352943040399], [-73.81401565719491, 40.843519171144855], [-73.81407416129895, 40.84350281490183], [-73.81412925421093, 40.843480656319045], [-73.81417994348193, 40.843453094564694], [-73.8142253159947, 40.843420626139995], [-73.81426455441185, 40.84338383593459], [-73.81429695190002, 40.843343386690535], [-73.8143219248619, 40.84330000706269], [-73.81433902344891, 40.84325447849229], [-73.81434793966353, 40.84320762113033], [-73.81433904017473, 40.84319282466564], [-73.81433357893498, 40.84317709535262], [-73.81433171460951, 40.84316089018348], [-73.81433350136119, 40.84314467997544], [-73.81433888727611, 40.84312893569123], [-73.81434771587207, 40.84311411475723], [-73.81435973064592, 40.84310064777258], [-73.81437458252545, 40.8430889259994], [-73.81443311693067, 40.8430982852661], [-73.81447435420355, 40.84318544130673], [-73.8145302823009, 40.84325409145745], [-73.81458365727987, 40.843288897497416], [-73.81464226888777, 40.84331844841381], [-73.81470522534168, 40.843342294582804], [-73.81476748814121, 40.8433566433511], [-73.81482697475403, 40.843376663783964], [-73.81488279316403, 40.843402055671135], [-73.81493410635808, 40.84343243825602], [-73.81498014487664, 40.843467355945016], [-73.81502021835243, 40.843506285138574], [-73.81505372586119, 40.84354864208284], [-73.81508016493551, 40.84359379162279], [-73.8150935717148, 40.843606944986675], [-73.81511033055865, 40.843677638455965], [-73.81513413714164, 40.8437471559981], [-73.81516485293174, 40.8438150930236], [-73.81520229918517, 40.84388105414047], [-73.81524625798409, 40.84394465545534], [-73.81529647350618, 40.84400552680751], [-73.81535265351195, 40.84406331392409], [-73.81539614036446, 40.84410241829302], [-73.81544124858463, 40.844136302679125], [-73.81548639348827, 40.84416496237132], [-73.81553442761394, 40.84419077171556], [-73.8155538550995, 40.84420973962538], [-73.81557050239195, 40.84423017542359], [-73.81558475951772, 40.844252913713], [-73.81559557426921, 40.84427672133807], [-73.8155948819365, 40.84428990065019], [-73.81559649706655, 40.844303033292164], [-73.81560039113369, 40.84431588730953], [-73.8156064953604, 40.84432823566863], [-73.81561361207982, 40.844338512605155], [-73.81562226066892, 40.8443480879866], [-73.81563232274601, 40.844356830744815], [-73.81564400690353, 40.84436483141949], [-73.81565687237841, 40.84437170790415], [-73.81567073272595, 40.84437736054545], [-73.81569792948075, 40.8443864281129], [-73.81572663977093, 40.84439220931232], [-73.81575617959831, 40.84439456641176], [-73.81581729631598, 40.844404255607856], [-73.81587930562611, 40.84440977839467], [-73.81594171601608, 40.84441109099639], [-73.81595261849685, 40.84441232782473], [-73.81596363860406, 40.84441212134858], [-73.81597444750204, 40.844410477728914], [-73.81598472265817, 40.844407446010834], [-73.81599415746581, 40.84440311665946], [-73.81600247039512, 40.84439761886071], [-73.81600941339211, 40.84439111666641], [-73.81601477928108, 40.844383804099316], [-73.81601840794703, 40.84437589936285], [-73.81602019111288, 40.844367638330915], [-73.81602007557044, 40.844359267508544], [-73.81601806476836, 40.84435103667714], [-73.81600028898751, 40.84433276285884], [-73.81598630210995, 40.84431267964334], [-73.81597583544776, 40.84428954898817], [-73.81597039431108, 40.84426544231978], [-73.81597011983966, 40.844240984973766], [-73.81597501914739, 40.84421681138169], [-73.81598496513935, 40.84419354861375], [-73.81599969980863, 40.84417180011267], [-73.81601970133299, 40.8441345510654], [-73.81604649513972, 40.844099849652835], [-73.8160795332132, 40.84406840561157], [-73.81611813982926, 40.84404086205575], [-73.81616152537532, 40.84401778232297], [-73.81620880249945, 40.8439996384538], [-73.81625900426044, 40.84398680153721], [-73.81631110390217, 40.84397953412081], [-73.81634628559065, 40.84398173672404], [-73.81638078456292, 40.843987418855995], [-73.81641400672652, 40.84399648266722], [-73.81644537997586, 40.844008772073586], [-73.81646142591855, 40.844020929929], [-73.81647463438595, 40.84403491867298], [-73.81648464895692, 40.844050360831815], [-73.8164911993952, 40.84406683971259], [-73.8164941089401, 40.84408391064677], [-73.81649329907742, 40.8441011129896], [-73.81649226951575, 40.844104836284345], [-73.81649206747677, 40.84410863769033], [-73.81649269855802, 40.844112411885185], [-73.81649414527524, 40.844116054301104], [-73.81649636754568, 40.8441194640211], [-73.8164993037987, 40.84412254657506], [-73.81650287268272, 40.844125216557565], [-73.8165069753176, 40.84412739999382], [-73.81651115785635, 40.844128937003745], [-73.81651560208203, 40.84412996680365], [-73.81652020207204, 40.8441304648495], [-73.81652484819097, 40.84413041927109], [-73.8165294297045, 40.84412983115472], [-73.8165338374184, 40.84412871451719], [-73.81653796627933, 40.84412709597242], [-73.81654171788186, 40.84412501409629], [-73.81654803579622, 40.84411713039859], [-73.81655579562262, 40.84411002784444], [-73.81656489614979, 40.84410381933773], [-73.81657509495035, 40.84409868886183], [-73.81658617249596, 40.84409474684975], [-73.8165978903427, 40.84409207815315], [-73.81660999626493, 40.84409074021557], [-73.8166311886855, 40.84410697792727], [-73.81665561354282, 40.8441203430208], [-73.8166826036293, 40.84413047040477], [-73.81671142166353, 40.84413708343122], [-73.81674128042832, 40.84414000145353], [-73.81677136427729, 40.84413914476039], [-73.81680085141552, 40.8441345367542], [-73.8168289363478, 40.84412630331068], [-73.81684719792797, 40.84411229365962], [-73.81686826993233, 40.8441007819621], [-73.81689156699254, 40.84409208800587], [-73.81691644193094, 40.84408645330339], [-73.8169422037372, 40.84408403438295], [-73.81696813676572, 40.84408489844079], [-73.8169935206138, 40.844089021473685], [-73.81701765013457, 40.844096288946766], [-73.81703985502557, 40.84410649897402], [-73.81705951844759, 40.84411936792766], [-73.81707609416169, 40.84413453831601], [-73.81708912170255, 40.84415158871505], [-73.81709823916935, 40.84417004547454], [-73.81709507751613, 40.844187660670585], [-73.81708835296621, 40.8442046895192], [-73.81707822858992, 40.84422071906344], [-73.81707489073452, 40.844225516511436], [-73.81707264035197, 40.844230666283316], [-73.81707153906245, 40.84423602736941], [-73.81707161702109, 40.84424145297411], [-73.81707287209358, 40.8442467945344], [-73.81707526991408, 40.84425190578928], [-73.81707874482646, 40.84425664678347], [-73.81708320168126, 40.84426088770004], [-73.8170885184427, 40.84426451241531], [-73.81709454952792, 40.84426742167806], [-73.81710112979529, 40.84426953582733], [-73.81710807906538, 40.8442707969741], [-73.81711520705454, 40.84427117058587], [-73.81715021150436, 40.84427945628585], [-73.81718636615088, 40.84428408789739], [-73.81722301274397, 40.84428498109465], [-73.81731841759297, 40.84430674088586], [-73.81741119253728, 40.84433428898618], [-73.81750072974424, 40.844367444910375], [-73.81783277820627, 40.844370746825334], [-73.81823722746202, 40.844355041712625], [-73.81824065893247, 40.84434212790991], [-73.81824124459443, 40.84432896131916], [-73.81823896847384, 40.84431590110469], [-73.81823389266117, 40.84430330352895], [-73.81822615561835, 40.84429151223401], [-73.81821596840095, 40.8442808488678], [-73.81820566350187, 40.84427409514651], [-73.81819696203605, 40.84426614606328], [-73.8181901011048, 40.84425721821861], [-73.81818526765763, 40.84424755488286], [-73.81818259339703, 40.844237419367], [-73.81818215119185, 40.844227087847976], [-73.81818395309016, 40.844216841843895], [-73.81818794999268, 40.84420696054203], [-73.81819403298859, 40.84419771319239], [-73.81819985147386, 40.84417669362955], [-73.81820127234383, 40.84415524178031], [-73.81819826067787, 40.844133884922904], [-73.818190890506, 40.844113147999956], [-73.81816887844329, 40.844111065173344], [-73.81814670636902, 40.84411157594584], [-73.81812490102688, 40.844114668182925], [-73.81810398044887, 40.84412026842222], [-73.81808444164687, 40.84412824361817], [-73.81806674880681, 40.84413840430326], [-73.81805132226046, 40.84415050908909], [-73.81801695603708, 40.84418849707869], [-73.81797529226338, 40.84422200999295], [-73.81792732610218, 40.84425024734845], [-73.81786296480249, 40.84426306619356], [-73.81780564298111, 40.84426942426333], [-73.81774776234208, 40.84427124056503], [-73.8176899451903, 40.84426849557082], [-73.81763281314862, 40.84426121879356], [-73.81759643820345, 40.84425411569277], [-73.8175607738279, 40.84424517284428], [-73.81751331632168, 40.84422661675826], [-73.81746907981335, 40.84420393077191], [-73.81742868454596, 40.84417743296862], [-73.8173926969017, 40.844147494877205], [-73.81736232764281, 40.844115384571715], [-73.81733703579171, 40.84408082946184], [-73.81731715864584, 40.84404429039278], [-73.81728029492882, 40.84399785628158], [-73.81723673532649, 40.843954896219806], [-73.81718704179603, 40.84391596441751], [-73.81716878406534, 40.8439113982863], [-73.81714976960329, 40.84390925737281], [-73.81713055270801, 40.84390960408763], [-73.81711169357992, 40.84391242832392], [-73.81669068165955, 40.84397496582519], [-73.81667877696744, 40.84397102198651], [-73.81665600190942, 40.84396347568799], [-73.81662677325096, 40.843944898560224], [-73.81660732931886, 40.84391707357562], [-73.81661234792885, 40.84386705152037], [-73.81661236142156, 40.84386215716094], [-73.81661244498649, 40.84383184466543], [-73.81657668436155, 40.84381076823534], [-73.81655885162657, 40.84380025890586], [-73.81651503368255, 40.843763129555185], [-73.81648830094262, 40.84372417472246], [-73.8164445195421, 40.84367407385203], [-73.816437307343, 40.84363515057113], [-73.81642277916197, 40.84359436132307], [-73.81639116165071, 40.84355725134012], [-73.81636686950459, 40.8435183008651], [-73.81635809445422, 40.843495916409964], [-73.81635233182159, 40.843481218060724], [-73.81635489948353, 40.84343489740633], [-73.81636235714114, 40.84338487806814], [-73.8163503053, 40.843331122875284], [-73.81630900985746, 40.843264350232594], [-73.81627260471049, 40.843193878224916], [-73.81623130431188, 40.84312895869531], [-73.8162021785278, 40.84307332313071], [-73.81615112347605, 40.84300653420623], [-73.81610484559108, 40.84297681371047], [-73.81602683823573, 40.842950746214136], [-73.81596837571968, 40.84291544711158], [-73.8159387775396, 40.84289193312023], [-73.81590994535827, 40.84286902912583], [-73.81587589832016, 40.84282820931973], [-73.81586704543801, 40.84278860976364], [-73.81572300131998, 40.8425365659869], [-73.81517113664873, 40.842577161623836], [-73.81516936817916, 40.84255492332517], [-73.81570207846407, 40.84251972168959], [-73.8156179809096, 40.84245835926559], [-73.81550698344904, 40.84242060831805], [-73.81544243533584, 40.84241082719334], [-73.81541503896409, 40.8423913277834], [-73.81540599274848, 40.84235379026757], [-73.81540790011593, 40.84232599873667], [-73.81543361658642, 40.84229129674323], [-73.81546847090182, 40.842260778031296], [-73.81546968074404, 40.84225954263417], [-73.8155051742541, 40.84222331387149], [-73.81552354975133, 40.84219554851018], [-73.81554743972904, 40.842159453120026], [-73.81556406036422, 40.842105280978515], [-73.81556419521851, 40.84205664041994], [-73.81555879401647, 40.84202466853028], [-73.81557540624677, 40.84197327392124], [-73.81559206830936, 40.84190381376299], [-73.81558492163327, 40.84184126460356], [-73.81556869858584, 40.8417522956215], [-73.81552506814646, 40.841647995900175], [-73.81547217396636, 40.84158537238329], [-73.81542649444837, 40.84156028460357], [-73.81537535736693, 40.84152406966567], [-73.81532969699884, 40.841492032142156], [-73.81530238614381, 40.84144195885927], [-73.8152915382384, 40.84139469015294], [-73.81528985873143, 40.841340487564345], [-73.81528083654403, 40.84129461172706], [-73.81526263397656, 40.841259839422975], [-73.81523211232641, 40.84123377337664], [-73.8147974828056, 40.84138417014289], [-73.81480293388553, 40.84139807535181], [-73.81476263162953, 40.8414132987143], [-73.81473345817534, 40.8413757283108], [-73.81522026179319, 40.84120267649966], [-73.81521713938885, 40.84116804427172], [-73.81522089631062, 40.84113330713769], [-73.81523378640593, 40.84110414274757], [-73.81524671254658, 40.84106247173637], [-73.81523404887245, 40.841009641496534], [-73.81522135566156, 40.840967928942966], [-73.81520503680377, 40.840913702896174], [-73.8151831980253, 40.84087058573759], [-73.81514858030347, 40.84081633153237], [-73.81512854729566, 40.8407815562704], [-73.81510852634034, 40.84074261067954], [-73.81509998662794, 40.84072432982889], [-73.81509033516018, 40.8407036696935], [-73.81507398877265, 40.840659170867454], [-73.81506678156754, 40.84061885703699], [-73.8150663842936, 40.84059068688925], [-73.81505399109437, 40.840538085896554], [-73.81497890551917, 40.840551141246124], [-73.81498025770709, 40.84055843942445], [-73.8149157299578, 40.84056667373986], [-73.81491163259714, 40.840559371155685], [-73.81490477086118, 40.84055936014459], [-73.81490069938222, 40.840542676755], [-73.8149089431792, 40.84053956264442], [-73.81490758787837, 40.84053330696266], [-73.8149707522553, 40.840521943627536], [-73.81497347937898, 40.840528201509166], [-73.815049126282, 40.840517619551235], [-73.81500569765576, 40.84029373486581], [-73.81497336213258, 40.84007688469778], [-73.81480646696015, 40.839804367389014], [-73.81436596502486, 40.83986536376494], [-73.81436188587786, 40.83985180818457], [-73.81479697880752, 40.83978795740684], [-73.81478892933622, 40.83977327891799], [-73.81477935155439, 40.83976284018757], [-73.81476701970095, 40.83975552437362], [-73.81475878782422, 40.83975446932075], [-73.81474506385649, 40.83975444727889], [-73.81473269673526, 40.8397596390901], [-73.8147203367438, 40.839762745403654], [-73.81470935814201, 40.83976272776754], [-73.81469838960713, 40.839758541476456], [-73.81469567415563, 40.83974811376349], [-73.81469983843603, 40.83973144359535], [-73.81470537675676, 40.839713733132], [-73.81471914179329, 40.83969916439648], [-73.81473018446566, 40.839676251098346], [-73.8147302657413, 40.83964706668479], [-73.81472761705194, 40.83961266621052], [-73.81472319684482, 40.83958888660002], [-73.81447505522941, 40.83962580976234], [-73.81447095193839, 40.83962059265944], [-73.81438444888595, 40.839635044333015], [-73.81438717009922, 40.83964338772789], [-73.81431578800081, 40.83964952650949], [-73.81430900133682, 40.839622416379285], [-73.81438450995489, 40.839613156567296], [-73.81438722825912, 40.839622542292545], [-73.81447097519408, 40.839612253681175], [-73.81446962061506, 40.83960599799443], [-73.81471999083577, 40.83957187959457], [-73.81469650157487, 40.8394510589777], [-73.81478277740266, 40.83907648795766], [-73.81538843489142, 40.8388523153941], [-73.81540002409541, 40.83881569859166], [-73.81542613964203, 40.83878279776938], [-73.8154521927371, 40.83877295737675], [-73.8154769186815, 40.838776128130306], [-73.8154782077358, 40.838776292690945], [-73.8154868552591, 40.83878618910912], [-73.81552154585935, 40.838789538794444], [-73.81554757935396, 40.838786286235745], [-73.81559976613778, 40.83873695709624], [-73.81566927530216, 40.83869753887238], [-73.81573871986129, 40.838681177688514], [-73.81579944386067, 40.838681274682536], [-73.81584721151336, 40.83866158629328], [-73.8158863476013, 40.838625412462164], [-73.81596456850167, 40.83857283051692], [-73.81606022905474, 40.838487333713715], [-73.81612981830266, 40.83841826741801], [-73.81619078912547, 40.83832942150598], [-73.81626470640903, 40.838263655717135], [-73.81629965234724, 40.838174768285], [-73.81633894250302, 40.83808259306099], [-73.81634147251413, 40.83803170124037], [-73.81630880926807, 40.838000190365356], [-73.81628721293391, 40.83796721452012], [-73.81628730390062, 40.837934272546605], [-73.81630907303996, 40.8379046587926], [-73.81632225026149, 40.837894692322834], [-73.81635253674234, 40.83787178705994], [-73.81640906870993, 40.83781916909941], [-73.81645257929428, 40.83776982592475], [-73.81647878477862, 40.8377039838885], [-73.8164789029114, 40.837661159370924], [-73.81644869632308, 40.83760511031885], [-73.81644033122897, 40.8374930950356], [-73.81642765539806, 40.83737118912981], [-73.81642365412327, 40.837249298199865], [-73.81641525209483, 40.83715045912872], [-73.81639818381184, 40.83704831256697], [-73.81638980002965, 40.83694288512829], [-73.81636407644693, 40.83683413638706], [-73.8163425982448, 40.83675833484838], [-73.8162993988223, 40.83669567616344], [-73.81623447512567, 40.83664616063875], [-73.81619549444879, 40.83662633267886], [-73.81616090485221, 40.836586747032165], [-73.81616966141667, 40.83655711423162], [-73.81620014053696, 40.8365143380932], [-73.81620466896722, 40.83644516702766], [-73.81618326520262, 40.83634301401463], [-73.81614023718535, 40.836217765478104], [-73.81609280117425, 40.83611886413313], [-73.81606068217324, 40.83603019622843], [-73.81720434599991, 40.83574058099987], [-73.8180458349999, 40.8355423629999], [-73.81890004099992, 40.83534343599992], [-73.81977578099998, 40.835132839999915], [-73.82062699699992, 40.83492251499993], [-73.82150930799996, 40.83473357299988], [-73.82223130099996, 40.8345560129999], [-73.822580515, 40.83447870199988], [-73.82265928699988, 40.834469871999886], [-73.82309210499989, 40.834421378999934], [-73.82319485499993, 40.83440178799987], [-73.8232749899999, 40.8343881879999], [-73.82368336699984, 40.83431888899992], [-73.82379383199985, 40.8343001469999], [-73.82463346500003, 40.833855807999925], [-73.82489104199995, 40.83422775199989], [-73.82515883900005, 40.83461447199994], [-73.82601252600003, 40.83402077799985], [-73.82612415599984, 40.834214164999864], [-73.82622087800003, 40.834354707999864], [-73.82604118599988, 40.83450929499991], [-73.8258777559998, 40.83467404699993], [-73.82573143499985, 40.8348481889999], [-73.82560350399983, 40.83503014699991], [-73.8254949649998, 40.83521822399992], [-73.82544607399977, 40.83536112899984], [-73.82541568999991, 40.83550885499988], [-73.82540482599977, 40.83565927499994], [-73.82541397499979, 40.835810148999876], [-73.82544296399993, 40.83595917099997], [-73.82549108799998, 40.83610415299995], [-73.82555705999988, 40.836243043999914], [-73.82565389199993, 40.836442047999874], [-73.82573525099993, 40.83664539499994], [-73.82580066799997, 40.83685202199996], [-73.8258498599999, 40.83706086699989], [-73.8258866379999, 40.8381214889999], [-73.82590798500001, 40.83874481799989], [-73.82596226799994, 40.83997641699986], [-73.82596550399984, 40.840326137999924], [-73.825986869, 40.84067575199988], [-73.82602634299997, 40.84102446399987], [-73.82608382099987, 40.841371456999894], [-73.82624132999996, 40.84299359399991], [-73.82580705599993, 40.84288069299988], [-73.82567803799994, 40.8428471519999], [-73.82558155599993, 40.842818658999946], [-73.82517999, 40.842700058999874], [-73.82513836199985, 40.843343597999905], [-73.82507950099988, 40.84430400299992], [-73.82508489299991, 40.8446976979999], [-73.82495697699986, 40.846105444999864], [-73.82443760499989, 40.84637787399988], [-73.82395955500006, 40.84664124199988], [-73.82301297099994, 40.847161737999905], [-73.82203826799994, 40.84769036899989], [-73.82110355699993, 40.84819897999991], [-73.82048405599983, 40.84850102599995], [-73.82014043700003, 40.848020598999895], [-73.81915009199983, 40.84817017999989], [-73.81820226899997, 40.848311031999884], [-73.81726322699988, 40.848426566999954], [-73.816301192, 40.84857217199994], [-73.81507165904237, 40.84913361979451], [-73.81517828216496, 40.84905376734075], [-73.81522652669221, 40.84901120779714], [-73.81529101730445, 40.8488955840948], [-73.81530585173346, 40.84881790341616], [-73.81530729613678, 40.84881033800619], [-73.81526739378965, 40.84873718412659], [-73.8152675460326, 40.84868236537186], [-73.81527578737355, 40.84860319691036], [-73.81541233179878, 40.84853641599008], [-73.81548477457869, 40.84844516872847], [-73.81550118820577, 40.84831119549049], [-73.81561382067721, 40.84818955758243]]]}}, {\"id\": \"58\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 59, \"Shape_Leng\": 0.0377948070893, \"Shape_Area\": 6.28765230648e-05, \"zone\": \"Crotona Park\", \"LocationID\": 59, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88941724899998, 40.839422732999836], [-73.88869040799987, 40.83915802599988], [-73.88813752199994, 40.83986200199991], [-73.88754251799996, 40.839640657999944], [-73.88659802299989, 40.839279049999895], [-73.88730373200002, 40.83812719099994], [-73.88841187199996, 40.838553390999905], [-73.88874067899994, 40.8379684589999], [-73.89057363299985, 40.836901047999916], [-73.8907397549999, 40.83682930899993], [-73.89091660699987, 40.83677103399994], [-73.89110124799988, 40.83672748199993], [-73.89129045299998, 40.83669941999994], [-73.8914808639999, 40.83668706399992], [-73.89337676299994, 40.836540007999886], [-73.89361903299995, 40.8364761079999], [-73.89385624299995, 40.8363951019999], [-73.89408527499982, 40.8362973469999], [-73.89430313999998, 40.83618364399991], [-73.89450698899992, 40.83605519399989], [-73.89469434299991, 40.83591357399992], [-73.89486311099988, 40.83576065299992], [-73.89501173599986, 40.835598554999876], [-73.89513914999998, 40.83542950399993], [-73.89526368299991, 40.835315097999946], [-73.89539409500006, 40.83520453499989], [-73.89553017999992, 40.83509798999991], [-73.89567172399995, 40.83499562899987], [-73.89581850399996, 40.83489761499993], [-73.89597028899992, 40.834804100999925], [-73.89612683899982, 40.834715234999926], [-73.89628790799986, 40.8346311569999], [-73.89645324200002, 40.834551998999835], [-73.89662258200005, 40.83447788599992], [-73.89679566, 40.83440893499992], [-73.89697220299995, 40.8343452539999], [-73.89809341399994, 40.834628802999916], [-73.89845304899994, 40.834720240999935], [-73.89963414999981, 40.835016222999904], [-73.9012157839999, 40.8354101409999], [-73.90122534999992, 40.83549991099991], [-73.90056610899994, 40.83659250799993], [-73.9003894469999, 40.83690622999995], [-73.90035730100003, 40.83696331799991], [-73.90030702699993, 40.837052593999914], [-73.9002588729999, 40.83713810599989], [-73.90022284799984, 40.83720207899991], [-73.90019045299977, 40.837259606999964], [-73.90014268099999, 40.83734444099984], [-73.90004236899996, 40.83752257599983], [-73.89996792899998, 40.8376547629999], [-73.89903034699984, 40.839330476999876], [-73.89826966299981, 40.84067994699995], [-73.89749111599987, 40.84206708899992], [-73.89666161699988, 40.84352460699992], [-73.89642177099996, 40.84338795899991], [-73.8961725309998, 40.8432605479999], [-73.89591487099987, 40.843142918999895], [-73.89564979700005, 40.843035539999896], [-73.89537836199995, 40.842938778999965], [-73.89527784899985, 40.84287986099988], [-73.89509807299986, 40.84277448099986], [-73.89501415799995, 40.84272529099996], [-73.89493579999981, 40.84267935899995], [-73.89484862799985, 40.84262825999993], [-73.89410150499992, 40.84219030299994], [-73.89317098900001, 40.841637343999906], [-73.89216815599981, 40.84104963999991], [-73.89017468599991, 40.83987366699987], [-73.88941724899998, 40.839422732999836]]]}}, {\"id\": \"59\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 60, \"Shape_Leng\": 0.0786482878226, \"Shape_Area\": 0.000161833284912, \"zone\": \"Crotona Park East\", \"LocationID\": 60, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87823856799989, 40.839985917999854], [-73.87791380399995, 40.8399703279999], [-73.87747229399999, 40.839983546999925], [-73.87561285199983, 40.839902520999885], [-73.87359713599987, 40.83979855499992], [-73.87283453899995, 40.839754862999925], [-73.87277204699981, 40.83975128199995], [-73.87298042899992, 40.83895248499995], [-73.87312728200003, 40.83833509599989], [-73.87314714699997, 40.838263898999905], [-73.87322928299983, 40.83815590199992], [-73.87347510499988, 40.837746339999896], [-73.87353901799992, 40.83766841799995], [-73.87367306200004, 40.837541539999926], [-73.87379590199993, 40.837445132999896], [-73.87401051299989, 40.837310004999914], [-73.87407495699992, 40.83728961699988], [-73.87419768599986, 40.837250794999896], [-73.87430028599992, 40.8372424419999], [-73.87437386800003, 40.837242521999904], [-73.87454332199997, 40.83726133799991], [-73.87471801499991, 40.83731259599988], [-73.874908511, 40.83735960999999], [-73.87472739999993, 40.83723397399996], [-73.87470145099985, 40.83716231099987], [-73.87469588499998, 40.83708331399992], [-73.8747145209998, 40.83700225399988], [-73.87475779999978, 40.83692560499988], [-73.87482228199987, 40.836859479999866], [-73.87490147299994, 40.83680809399989], [-73.87498772399994, 40.836772977999864], [-73.87525928600002, 40.83670942699994], [-73.87545635000001, 40.836691981999934], [-73.87565111799996, 40.83665954899988], [-73.87584064899983, 40.836612485999915], [-73.87609408699988, 40.83652621199991], [-73.87615142899986, 40.83649857399996], [-73.87628029499993, 40.83646153199987], [-73.8764548759999, 40.83638732799986], [-73.87655099199999, 40.83634482899989], [-73.87673323499986, 40.836205617999944], [-73.87698921499998, 40.83610249599995], [-73.87728333599982, 40.83596804299988], [-73.87734872299994, 40.83593341499993], [-73.87746425099982, 40.835872232999876], [-73.87747843299982, 40.83586472899987], [-73.87756524599996, 40.8358187519999], [-73.87783304799999, 40.835655621999926], [-73.8779460599999, 40.83557678399992], [-73.87801301899998, 40.835530071999905], [-73.87808502000001, 40.83547984299993], [-73.87838370399996, 40.83525903199987], [-73.87866481499992, 40.83502565899989], [-73.87892705599994, 40.83478096399987], [-73.87906181199985, 40.83463927099988], [-73.87916928999984, 40.83452626099988], [-73.87925239499992, 40.83442478999991], [-73.87940077099982, 40.83447236499995], [-73.88005188499987, 40.83470205899989], [-73.88007189299988, 40.834671876999934], [-73.88057338999994, 40.83403109299997], [-73.88251168699992, 40.83155430699988], [-73.88384720899988, 40.829872518999906], [-73.88390094800003, 40.82980281599986], [-73.88357411800004, 40.829478645999934], [-73.8835201879999, 40.82926018099994], [-73.88378414099992, 40.828274291999946], [-73.88438705399992, 40.826817283999965], [-73.88460938899989, 40.82530353599988], [-73.88461335999997, 40.825276502999905], [-73.8846216589999, 40.824956205999904], [-73.88465102199984, 40.823822897999925], [-73.8843985489999, 40.822965575999866], [-73.88456358400003, 40.82293706699989], [-73.88508162399985, 40.822868444999834], [-73.88658010099988, 40.822668116999864], [-73.88698003699992, 40.82261725799987], [-73.88709505499989, 40.82260272499986], [-73.88719405099982, 40.82259352899993], [-73.88753429599987, 40.82250933899995], [-73.88706248999988, 40.823190274999945], [-73.88710697199997, 40.823305870999874], [-73.88647552899988, 40.825285423999915], [-73.88635032500002, 40.826012622999926], [-73.88615037899993, 40.82730567799994], [-73.88609624199987, 40.82765577699989], [-73.88601262699991, 40.828238530999926], [-73.88755325099989, 40.82929634299989], [-73.88761307899992, 40.8293374189999], [-73.88805361099989, 40.82950305199991], [-73.88898205299992, 40.829814800999905], [-73.8899110649999, 40.830124191999865], [-73.89087299799995, 40.83013600299989], [-73.89187594399993, 40.830137505999936], [-73.8928855849999, 40.8301475709999], [-73.893360418, 40.83015253099996], [-73.894692052, 40.83061909399988], [-73.89485843399979, 40.830601857999895], [-73.89560633499985, 40.830524382999876], [-73.89669632499995, 40.8304145039999], [-73.89717220999982, 40.830368832999966], [-73.89690017899987, 40.8310333359999], [-73.89669216099999, 40.83196614399987], [-73.89647296199996, 40.83304756799986], [-73.89630199999992, 40.833543217999896], [-73.89678082699999, 40.834126789999914], [-73.89697220299995, 40.8343452539999], [-73.89679566, 40.83440893499992], [-73.89662258200005, 40.83447788599992], [-73.89645324200002, 40.834551998999835], [-73.89628790799986, 40.8346311569999], [-73.89612683899982, 40.834715234999926], [-73.89597028899992, 40.834804100999925], [-73.89581850399996, 40.83489761499993], [-73.89567172399995, 40.83499562899987], [-73.89553017999992, 40.83509798999991], [-73.89539409500006, 40.83520453499989], [-73.89526368299991, 40.835315097999946], [-73.89513914999998, 40.83542950399993], [-73.89501173599986, 40.835598554999876], [-73.89486311099988, 40.83576065299992], [-73.89469434299991, 40.83591357399992], [-73.89450698899992, 40.83605519399989], [-73.89430313999998, 40.83618364399991], [-73.89408527499982, 40.8362973469999], [-73.89385624299995, 40.8363951019999], [-73.89361903299995, 40.8364761079999], [-73.89337676299994, 40.836540007999886], [-73.8914808639999, 40.83668706399992], [-73.89129045299998, 40.83669941999994], [-73.89110124799988, 40.83672748199993], [-73.89091660700001, 40.83677103499991], [-73.8907397549999, 40.83682930899993], [-73.89057363299985, 40.836901047999916], [-73.88874067899994, 40.8379684589999], [-73.88841187199996, 40.838553390999905], [-73.88730373200002, 40.83812719099994], [-73.88659802299989, 40.839279049999895], [-73.88618432099992, 40.840105043999884], [-73.88583496499986, 40.83989692699992], [-73.88469619299984, 40.83920338699989], [-73.88464898099988, 40.83917463299988], [-73.88435324499997, 40.83927764999993], [-73.88418143199976, 40.839335306999914], [-73.88412724499999, 40.83935591899995], [-73.88403342700002, 40.83930464099993], [-73.88351820399988, 40.8390223659999], [-73.8831387399999, 40.838810671999944], [-73.88274769899994, 40.8385923409999], [-73.88279558600006, 40.83854994599993], [-73.88285365099989, 40.83849533499994], [-73.88246404800005, 40.83824724099992], [-73.88235442499995, 40.83814382399988], [-73.88225831200006, 40.83803121799985], [-73.88217759099985, 40.837911036999905], [-73.88211370499987, 40.83778515499997], [-73.88206752699985, 40.837655624999876], [-73.88203937499988, 40.8375245929999], [-73.88200384100001, 40.837443781999895], [-73.88191969599985, 40.837616932999914], [-73.88182513299996, 40.837787064999915], [-73.88172047499994, 40.83795364899991], [-73.88169674899986, 40.83798711299989], [-73.88166303799993, 40.838028618999914], [-73.88160324799993, 40.83809985899994], [-73.88154924699987, 40.838182756999885], [-73.88138310599982, 40.83837771399988], [-73.88135107299988, 40.83840917199989], [-73.88113550299995, 40.838622141999906], [-73.880965322, 40.838748079999895], [-73.88080919100001, 40.83888681999984], [-73.88080895400005, 40.838887035999875], [-73.88080875199994, 40.83888725199996], [-73.88066897399986, 40.83903757099989], [-73.88066892699996, 40.839037624999904], [-73.88066887899986, 40.83903768799993], [-73.88054735399987, 40.839198110999895], [-73.88044599500002, 40.839366353999885], [-73.88044592400004, 40.839366479999924], [-73.88044586500001, 40.83936659699989], [-73.88036587299996, 40.83954043499988], [-73.88036583800002, 40.83954050699991], [-73.88036581299994, 40.839540578999895], [-73.88030771499984, 40.83971779899986], [-73.88030769199985, 40.8397178709999], [-73.88030767999986, 40.839717943999894], [-73.88027152699999, 40.839896239999874], [-73.88010086899989, 40.84006661599993], [-73.880061996, 40.840162234999845], [-73.87992249999996, 40.840154029999916], [-73.87978300299987, 40.840145825999855], [-73.8786598319999, 40.840019903999824], [-73.87850461299986, 40.840005593999905], [-73.87841924999984, 40.83999458999992], [-73.87831009099979, 40.8399893509999], [-73.87823856799989, 40.839985917999854]]]}}, {\"id\": \"60\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 61, \"Shape_Leng\": 0.117310808097, \"Shape_Area\": 0.000510772092295, \"zone\": \"Crown Heights North\", \"LocationID\": 61, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92916130699993, 40.68343632399992], [-73.92900883999985, 40.68270290499991], [-73.92609461699993, 40.683038709999884], [-73.92281374299995, 40.68341442999991], [-73.92267012199994, 40.68268151199991], [-73.9225447959999, 40.68193561299989], [-73.92250799299985, 40.68171006999996], [-73.92249266199998, 40.681616108999926], [-73.92242652099985, 40.68121074499992], [-73.92240197099996, 40.68108955299994], [-73.92238256000003, 40.68099373699993], [-73.92236962399983, 40.68092987999994], [-73.92235458599998, 40.68085564499991], [-73.92233849599998, 40.680776217999885], [-73.92231261899987, 40.680648472999955], [-73.92227774199988, 40.6804762929999], [-73.92225617599988, 40.68036776399987], [-73.92223681799994, 40.680270342999904], [-73.92221626999996, 40.680166934999924], [-73.92217715699984, 40.679970084999894], [-73.92215222999988, 40.67984463499988], [-73.92213242899994, 40.67974497799986], [-73.92211149999984, 40.67965755699986], [-73.92208787199984, 40.67955885799993], [-73.92205453499997, 40.6794196039999], [-73.92203214100006, 40.67932605899988], [-73.92201534599985, 40.679255899999916], [-73.9219999079998, 40.6791914139999], [-73.92197287199998, 40.67907846999987], [-73.92183500099998, 40.67894555899991], [-73.92164666399992, 40.67886992299994], [-73.92171159899995, 40.67811571399989], [-73.92181538999996, 40.67701838999991], [-73.92182360999988, 40.67693835799988], [-73.92183232600003, 40.67684904199989], [-73.92190090500002, 40.67613162099988], [-73.92197495299999, 40.675348546999935], [-73.92204843299987, 40.67457439699991], [-73.92213153399979, 40.673683888999946], [-73.92221444299993, 40.67279163299992], [-73.92229950400001, 40.67190026199992], [-73.92238401799992, 40.67100876299986], [-73.92245355399994, 40.67026114899993], [-73.92246696799995, 40.6701169299999], [-73.92254432199987, 40.66927581099986], [-73.92261354299985, 40.668564106999874], [-73.92263210799995, 40.66837321099994], [-73.92265529999987, 40.668127114999905], [-73.9227271419999, 40.667365055999866], [-73.92275257199998, 40.667097332999894], [-73.922811066, 40.66707176099994], [-73.92303439399991, 40.66697412599995], [-73.92311978499988, 40.666936793999945], [-73.92323224599997, 40.66688762799987], [-73.92331039799996, 40.66685345999985], [-73.92424467199989, 40.666444999999875], [-73.92439099400002, 40.66638142299991], [-73.92546270199979, 40.66591675199991], [-73.92592201499984, 40.66571503599991], [-73.92637970499992, 40.6655148919999], [-73.92739349999997, 40.665074715999936], [-73.92771188899987, 40.66493432799991], [-73.928228724, 40.66470986499991], [-73.92872204599985, 40.66449556899988], [-73.92869200600002, 40.66496229699995], [-73.92868404399992, 40.66508602299993], [-73.92930683899975, 40.66511700699991], [-73.9314554999999, 40.665233817999884], [-73.93422426299998, 40.665377742999915], [-73.93698362399992, 40.665535507999955], [-73.93694519299984, 40.66598129699992], [-73.93690681099987, 40.66642658999988], [-73.93686802199987, 40.666849828999894], [-73.9368252179999, 40.66731699399983], [-73.93674946999998, 40.66812737099991], [-73.93951520799993, 40.66829561599992], [-73.94228420399995, 40.6684614269999], [-73.94504645700006, 40.66861461599994], [-73.94781857499984, 40.66878375399989], [-73.94775110299997, 40.669495265999934], [-73.9477299629999, 40.66971344199991], [-73.9505004649999, 40.66986833099988], [-73.95160495999987, 40.669923718999904], [-73.95327019699985, 40.67000720499991], [-73.95526963699997, 40.6701249049999], [-73.957973172, 40.67065727399988], [-73.95904725399988, 40.67088226799995], [-73.96131877999991, 40.67140667599988], [-73.96126442099991, 40.67156359099989], [-73.96102898299978, 40.67224318399995], [-73.96070974499996, 40.673121393999985], [-73.96039301899985, 40.67399860199986], [-73.96007428300001, 40.67487807099994], [-73.96318238899983, 40.675529503999876], [-73.9628644419999, 40.67640534099992], [-73.96254635399981, 40.67728661699993], [-73.96222841199993, 40.67816620799992], [-73.96195644399998, 40.67893384299989], [-73.96167998599988, 40.67970220599991], [-73.96153961999995, 40.68008625499992], [-73.9614104629999, 40.680413152999925], [-73.96134655899995, 40.6804635239999], [-73.95829216799997, 40.67983113299993], [-73.9559586239999, 40.679346246999906], [-73.95529361599996, 40.679203150999825], [-73.95530907499989, 40.67927313499995], [-73.95543681599986, 40.67995785499992], [-73.95436836300001, 40.680075106999915], [-73.95328187399996, 40.68020003099985], [-73.9533701749999, 40.680640507999904], [-73.95155682700002, 40.680498847999935], [-73.95115828499993, 40.68047861499993], [-73.94955688699993, 40.68039040299995], [-73.94773023599987, 40.6802918459999], [-73.94674916000001, 40.68023966099991], [-73.94627471000001, 40.68021332599986], [-73.94397347799995, 40.680088127999916], [-73.94326176899995, 40.68005060699992], [-73.9412086429999, 40.67993835399988], [-73.94032794, 40.679889974999874], [-73.93843613299988, 40.679785851999924], [-73.93739763699983, 40.679729899999934], [-73.93487210899987, 40.679593087999855], [-73.93435066499993, 40.67956340799991], [-73.93439530999997, 40.67983537999992], [-73.93450838999993, 40.6802216929999], [-73.93457098299989, 40.68056476199988], [-73.93471441099992, 40.6812973839999], [-73.93486257200001, 40.68203056899991], [-73.93500821699992, 40.68276376799985], [-73.93208669799994, 40.68310034999988], [-73.92916130699993, 40.68343632399992]]]}}, {\"id\": \"61\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 62, \"Shape_Leng\": 0.0822593359838, \"Shape_Area\": 0.000158238169004, \"zone\": \"Crown Heights South\", \"LocationID\": 62, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92916752199997, 40.664299531999895], [-73.9294035289999, 40.66419984599983], [-73.92987364799986, 40.66399077299992], [-73.93068465499985, 40.66362047699992], [-73.93160743400001, 40.66355672299988], [-73.93437459299999, 40.663372810999896], [-73.93455236699992, 40.6633622859999], [-73.93726377600001, 40.66319183299991], [-73.94002976999997, 40.66302074299991], [-73.94282892199992, 40.6628474859999], [-73.94269628999989, 40.664055128999934], [-73.94408198699985, 40.664128572999914], [-73.94489227700002, 40.66417414399993], [-73.94546293699983, 40.66422610399989], [-73.94816604199978, 40.66408182199995], [-73.9494063899999, 40.66400530799991], [-73.9509741009999, 40.6639048679999], [-73.95384851099975, 40.66372780899991], [-73.95720935499996, 40.663517396999914], [-73.95729172699991, 40.66430263299986], [-73.95787061899992, 40.66427335799993], [-73.95934422499985, 40.66417564699991], [-73.96033682899981, 40.66410869799995], [-73.96061544799986, 40.663302749999936], [-73.96095595499996, 40.66328505499992], [-73.96110886899996, 40.664062323999914], [-73.96125791699997, 40.66480182299993], [-73.96161627599987, 40.66669521599988], [-73.96180658699991, 40.66768122699987], [-73.96199657599993, 40.66866286999989], [-73.96211021099997, 40.66919569699996], [-73.96219447199988, 40.66964102999985], [-73.96252326599993, 40.67137973899986], [-73.96254108199987, 40.67146011899995], [-73.96258784899996, 40.671711596999884], [-73.96131877999991, 40.67140667599988], [-73.95904725399988, 40.67088226799995], [-73.957973172, 40.67065727399988], [-73.95526963699997, 40.6701249049999], [-73.95327019699985, 40.67000720499991], [-73.95160495999987, 40.669923718999904], [-73.9505004649999, 40.66986833099988], [-73.9477299629999, 40.66971344199991], [-73.94775110299997, 40.669495265999934], [-73.94781857499984, 40.66878375399989], [-73.94504645700006, 40.66861461599994], [-73.94228420399995, 40.6684614269999], [-73.93951520799993, 40.66829561599992], [-73.93674946999998, 40.66812737099991], [-73.9368252179999, 40.66731699399983], [-73.93686802199987, 40.666849828999894], [-73.93690681099987, 40.66642658999988], [-73.93694519299984, 40.66598129699992], [-73.93698362399992, 40.665535507999955], [-73.93422426299998, 40.665377742999915], [-73.9314554999999, 40.665233817999884], [-73.92930683899975, 40.66511700699991], [-73.92868404499993, 40.66508602299991], [-73.92869200600002, 40.66496229699995], [-73.92872204599985, 40.66449556899988], [-73.92905235499991, 40.66435100699988], [-73.92916752199997, 40.664299531999895]]]}}, {\"id\": \"62\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 63, \"Shape_Leng\": 0.119100721883, \"Shape_Area\": 0.000353190910142, \"zone\": \"Cypress Hills\", \"LocationID\": 63, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86650407899994, 40.684474051999885], [-73.86631496899999, 40.683248942999924], [-73.86630263900003, 40.68318669399984], [-73.86629002999987, 40.68311500699992], [-73.86602666799992, 40.681918051999936], [-73.8649940449999, 40.68212896999991], [-73.86410096700003, 40.68237284999988], [-73.86399008800005, 40.68192857599995], [-73.86378674500001, 40.68111374699991], [-73.86334943599995, 40.67935163999989], [-73.86330964200002, 40.67916796999987], [-73.86328640800004, 40.67907719699985], [-73.86296470699986, 40.67911281599989], [-73.86234580499983, 40.67916478599994], [-73.86225943099994, 40.67881678599989], [-73.86202843099984, 40.67788611399987], [-73.86171238699991, 40.67661059599987], [-73.86139775899983, 40.67533658999988], [-73.86131863599985, 40.6750212689999], [-73.86106294999989, 40.67434293499984], [-73.86176409699986, 40.6742322599999], [-73.86310051, 40.67464721299993], [-73.86413333499988, 40.67499725199989], [-73.86455313199994, 40.675151121999896], [-73.86521514199991, 40.67538657399996], [-73.86627349499993, 40.67576404499992], [-73.86725717699991, 40.676112683999875], [-73.86838105899999, 40.67653059899985], [-73.86942312899993, 40.67691222799995], [-73.86977606799984, 40.67702720899994], [-73.870458215, 40.67722236799989], [-73.87148141800003, 40.677500662999925], [-73.87176417499987, 40.677572227999875], [-73.87204099299996, 40.677657418999935], [-73.87231018000001, 40.67775574699987], [-73.87257015500002, 40.67786660999989], [-73.87304845699995, 40.678194106999904], [-73.87318521799988, 40.678330001999946], [-73.87331183100002, 40.678471800999894], [-73.87342767099986, 40.67861884699991], [-73.87353219499991, 40.67877042099986], [-73.87363895999985, 40.678944030999894], [-73.87376075799997, 40.679112496999885], [-73.87389692799991, 40.67927472199993], [-73.87404657399993, 40.67942964799988], [-73.87420866699985, 40.67957633999987], [-73.8744016199999, 40.67971160899987], [-73.87461126400001, 40.679834253999964], [-73.87483538999999, 40.679942569999874], [-73.87507136399991, 40.68003514599991], [-73.87531627599998, 40.68011098399988], [-73.8755670759999, 40.68016946499992], [-73.87657495299996, 40.68016700199996], [-73.87688955199985, 40.68016499699994], [-73.877426862, 40.6800144369999], [-73.87838855899999, 40.67973905399991], [-73.87862924200003, 40.67967653299995], [-73.87926692499995, 40.679491084999896], [-73.87974063299995, 40.67936096099988], [-73.88015110700003, 40.67924206299993], [-73.88061377799994, 40.679110031999876], [-73.88063900599983, 40.679176984999856], [-73.88109834200004, 40.68107447699986], [-73.88116730300005, 40.68135366299988], [-73.88210725299986, 40.68121749099986], [-73.88303083299995, 40.68108517499993], [-73.88397193799985, 40.68094711499989], [-73.88489254699988, 40.68081111599983], [-73.88467046600006, 40.679940284999894], [-73.88555502399997, 40.67965659699994], [-73.88643790499991, 40.67937347599991], [-73.88730460899997, 40.67909753699995], [-73.8881563449999, 40.67882783499992], [-73.88900569799982, 40.678557043999916], [-73.88987439199998, 40.67830214999987], [-73.89077319200003, 40.678164588999934], [-73.89167121099989, 40.67803172099987], [-73.89257905399984, 40.67790012399996], [-73.8934840539999, 40.677762060999896], [-73.8943888149999, 40.677629888999896], [-73.89531770399991, 40.67749603799994], [-73.89621773699986, 40.67735818399991], [-73.8965367649999, 40.67859358099985], [-73.89695521899982, 40.67840924299985], [-73.89747545199997, 40.67816325499989], [-73.89761600299984, 40.67811198799991], [-73.89775655500003, 40.67806072099993], [-73.89783505699992, 40.67799357599992], [-73.898199615, 40.677761731999944], [-73.89836229199989, 40.677658275999875], [-73.89883201499994, 40.677446439999855], [-73.89912562799992, 40.67731263799992], [-73.89992296499994, 40.67694927299984], [-73.9000462609999, 40.676893089999936], [-73.90014864399993, 40.67694931599989], [-73.9005649959999, 40.67717798499989], [-73.90099766899989, 40.67745246799996], [-73.90136992499991, 40.67768861999989], [-73.90140356899998, 40.6777099629999], [-73.90142169599987, 40.6777214619999], [-73.90144574099996, 40.67773656199995], [-73.90187352500006, 40.678005191999866], [-73.90198538099985, 40.67807543199993], [-73.90199214699999, 40.678079679999925], [-73.90236198199986, 40.67831191499993], [-73.90239451899987, 40.67833234699987], [-73.9023946, 40.67833239799992], [-73.90287079599995, 40.6786314149999], [-73.9031631289999, 40.6788149779999], [-73.90324287399993, 40.6788650509999], [-73.90333666199992, 40.6789239429999], [-73.90333723699995, 40.678924302999874], [-73.903630796, 40.67904695599991], [-73.90400588299991, 40.67920367099988], [-73.90402950499993, 40.67921353999991], [-73.9040463979999, 40.67922059799985], [-73.90358874799992, 40.67967480399991], [-73.90262262299991, 40.6806645029999], [-73.90075235999984, 40.67967664399992], [-73.90047001100004, 40.679535243999936], [-73.89969830299987, 40.67954409199993], [-73.89889928299989, 40.67956093399993], [-73.89870944799999, 40.67963016999989], [-73.89848997800001, 40.67967050899991], [-73.89827418999997, 40.67972504299988], [-73.89806444599996, 40.67979343099991], [-73.89786299899986, 40.67987501099987], [-73.89767194200002, 40.6799688539999], [-73.89749312100001, 40.680073775999915], [-73.89738856299982, 40.68016295199989], [-73.89729111699982, 40.68025667899985], [-73.8972011219999, 40.680354629999904], [-73.89711889399996, 40.68045646099994], [-73.89704472199993, 40.68056181599996], [-73.89697886299997, 40.68067032699991], [-73.89692154999993, 40.680781612999915], [-73.89687298299991, 40.680895284999934], [-73.89674965999993, 40.68149202599986], [-73.89663536799999, 40.68155649499991], [-73.89658158699996, 40.6815747179999], [-73.89652633199988, 40.68200291799991], [-73.89646625099995, 40.68233642199988], [-73.89643010399996, 40.68244635899987], [-73.89641909099993, 40.68247987599991], [-73.896367991, 40.68259975699988], [-73.89635898799989, 40.68262086699985], [-73.89628634699977, 40.68275838299988], [-73.89620169599988, 40.68289148199994], [-73.89599727299998, 40.68319687899986], [-73.89577945499994, 40.6834971829999], [-73.89561476499993, 40.683707421999955], [-73.89554859399983, 40.6837918919999], [-73.89530504899982, 40.684080519999966], [-73.8950492689999, 40.68436259899988], [-73.89417463299992, 40.6852832479999], [-73.89414204200001, 40.68524656699989], [-73.89404051299984, 40.68513229799984], [-73.89274142399994, 40.683670185999866], [-73.89252316799995, 40.683424532999865], [-73.8916601069998, 40.6840441779999], [-73.89015418699982, 40.68500398999987], [-73.88962787599998, 40.684236453999894], [-73.8890889529999, 40.68464055299995], [-73.88808341799984, 40.68529363199983], [-73.88728392899995, 40.68562247499987], [-73.88452250999984, 40.68668474899988], [-73.88377698600002, 40.68786341099992], [-73.88226665399998, 40.689038550999896], [-73.88118296199987, 40.689866269999904], [-73.87956139399984, 40.691104780999915], [-73.87950641199996, 40.69114676899987], [-73.87939315099985, 40.691209631999875], [-73.87402053199992, 40.69419129499995], [-73.870042929, 40.69494148499996], [-73.86891704399994, 40.69515042299987], [-73.8684248969999, 40.694718118999916], [-73.86868454299989, 40.6940346919999], [-73.86859478099983, 40.693884374999854], [-73.8680588529999, 40.69118395799991], [-73.867909485, 40.69047626099993], [-73.86776832199985, 40.68980328899986], [-73.86762340399984, 40.6891453419999], [-73.867460889, 40.688415017999915], [-73.86728057399986, 40.68774928499989], [-73.86701802999987, 40.686830348999884], [-73.86687058599995, 40.686330590999944], [-73.86659848299999, 40.68526955899992], [-73.86650407899994, 40.684474051999885]]]}}, {\"id\": \"63\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 64, \"Shape_Leng\": 0.18445188474, \"Shape_Area\": 0.00105790284614, \"zone\": \"Douglaston\", \"LocationID\": 64, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.74412314584049, 40.77863865777706], [-73.74153532911315, 40.777110631086295], [-73.74075991577695, 40.77669471074994], [-73.74036629409532, 40.77638970422471], [-73.74007973985933, 40.7762087951981], [-73.73937598893414, 40.77601366983492], [-73.73912362001862, 40.775704143036336], [-73.73886227381915, 40.775573157950646], [-73.73770283611138, 40.7747935858661], [-73.73688222993059, 40.77433096117149], [-73.73389261595659, 40.77252717887968], [-73.73304215822819, 40.77195443418381], [-73.73191230278991, 40.7713532077885], [-73.73113636491937, 40.77086955609971], [-73.73030460922762, 40.77041060576091], [-73.72970531807846, 40.770053446074], [-73.72870722109356, 40.76946148404628], [-73.7277187531248, 40.76879613592288], [-73.72658720816285, 40.76811459879865], [-73.72554599082781, 40.76749535591531], [-73.72356002293202, 40.76612876183406], [-73.72305924186259, 40.76614440615705], [-73.72248438405902, 40.76552721017787], [-73.72114272775374, 40.764872606934006], [-73.7209758841712, 40.76476015914438], [-73.72080904877663, 40.764647715919004], [-73.72075222623495, 40.76458735257297], [-73.72071269900161, 40.76453219025292], [-73.72063083099366, 40.76447640375274], [-73.72021435708592, 40.76419260579632], [-73.71657585309137, 40.76178181176188], [-73.71413538009266, 40.76024270987026], [-73.71386982697975, 40.76006061549925], [-73.71365557006699, 40.75993557094455], [-73.71351828186042, 40.75983773091619], [-73.71426287799991, 40.7598267469999], [-73.71455685899994, 40.759808807999896], [-73.71498213999979, 40.759782864999885], [-73.71612528199985, 40.75966071399991], [-73.71613842699988, 40.759658278999964], [-73.71685440399995, 40.75952551499989], [-73.71767237099996, 40.75931121199987], [-73.71831971899985, 40.7590820759999], [-73.71877787700004, 40.758871691999886], [-73.7189705039999, 40.75878323799989], [-73.71917909599986, 40.758666436999924], [-73.71956460799996, 40.7584505599999], [-73.7196773229999, 40.75837249899988], [-73.71998297099998, 40.75816082099986], [-73.72031595299998, 40.75789725799992], [-73.7206331939999, 40.75761683499996], [-73.72096511799994, 40.75729378899992], [-73.72111001499998, 40.75711152599996], [-73.72121176099995, 40.7569792089999], [-73.72145870299994, 40.75665808099994], [-73.72159631499989, 40.7564275039999], [-73.72161357099998, 40.756398594999894], [-73.72176339599984, 40.75613846699984], [-73.72180289199983, 40.75606989799992], [-73.72207071399988, 40.75549670299993], [-73.72241094799993, 40.754920188999925], [-73.72264543599995, 40.75460052399995], [-73.72279703199989, 40.75443353799988], [-73.72299301299984, 40.754217670999914], [-73.72326673499998, 40.75394759899994], [-73.72344135499989, 40.75379084499986], [-73.72351867299997, 40.753721436999825], [-73.72382698899992, 40.75346908899995], [-73.72384451899983, 40.75345681199988], [-73.72475823099984, 40.7529081479999], [-73.72506848099987, 40.752721843999886], [-73.72509445399984, 40.75270759199993], [-73.72551247899986, 40.75247819799989], [-73.72582988899985, 40.752291720999935], [-73.72602966399984, 40.752174354999916], [-73.7264147219999, 40.75191587799986], [-73.72666529899992, 40.75171358799987], [-73.726777395, 40.75162309199987], [-73.72709389599997, 40.75134160399987], [-73.72737889099987, 40.751029566999854], [-73.72771356199995, 40.750626894999904], [-73.72788401199996, 40.750387482999884], [-73.72805355599995, 40.750082150999894], [-73.7282538099999, 40.74969372899995], [-73.72837798399998, 40.74938116699989], [-73.728491785, 40.74904470699989], [-73.72862207399999, 40.74851209199989], [-73.72871952799996, 40.74819708099994], [-73.728962516, 40.74763288799986], [-73.72906670999978, 40.74739095899988], [-73.729145939, 40.74727379999993], [-73.72953448999986, 40.746716395999925], [-73.72964619299997, 40.74656191399992], [-73.72993169099983, 40.74618812499993], [-73.73043134700004, 40.74558483599989], [-73.7306280189999, 40.745364157999866], [-73.7310646189999, 40.744888375999885], [-73.73226918099995, 40.743818962999946], [-73.73232731699994, 40.74377143299988], [-73.73241694699988, 40.74369457599995], [-73.7336646159999, 40.74281318399988], [-73.73370171499998, 40.74278515699989], [-73.73375484899985, 40.742747627999975], [-73.7343696589998, 40.74235159899991], [-73.73539272699988, 40.74181104999989], [-73.73560786699984, 40.74169621999989], [-73.73685894199984, 40.741050400999875], [-73.73751038099981, 40.74071544799988], [-73.73816047999985, 40.74037643499986], [-73.738571759, 40.74009212299992], [-73.73901210399985, 40.73974647899997], [-73.73933151399987, 40.739474474999916], [-73.73972244199989, 40.739036089999914], [-73.73986995499992, 40.73886690099992], [-73.74016951799993, 40.73848189799986], [-73.74029081599988, 40.738516669999974], [-73.74055585499997, 40.73858718299989], [-73.740773448, 40.738635272999886], [-73.7408821769999, 40.73869623999991], [-73.7410061249999, 40.73874389499987], [-73.74100625499993, 40.73874393099987], [-73.74100638499996, 40.738743967999895], [-73.74114115799996, 40.73877533799986], [-73.74128113899988, 40.73878869199996], [-73.74142000699983, 40.73878380799991], [-73.74155202599987, 40.73876204099987], [-73.74222678199988, 40.738615671999874], [-73.74306820799991, 40.7384408179999], [-73.743469568, 40.73887010399992], [-73.74360280299993, 40.73901260899992], [-73.74371941299982, 40.73938314999984], [-73.74385264199996, 40.740093948999835], [-73.74396193599988, 40.740789125999925], [-73.74657796099996, 40.74034659199985], [-73.74996804299987, 40.73974575299995], [-73.7500915089999, 40.73981690899991], [-73.75009170899982, 40.73981702699994], [-73.75009192199988, 40.73981711699985], [-73.75022786299992, 40.73987649399992], [-73.75022795699996, 40.739876530999865], [-73.75037344499992, 40.73992242099994], [-73.75037358699991, 40.73992246699986], [-73.75037372900003, 40.7399224929999], [-73.75052495299994, 40.739953708999934], [-73.75052516599999, 40.73995375399992], [-73.7505254029999, 40.739953780999905], [-73.75067885099993, 40.73996996299991], [-73.75083158999993, 40.73998889799992], [-73.75083176700001, 40.73998891699989], [-73.75083193299997, 40.73998895299995], [-73.75098218299992, 40.74002143499992], [-73.75098230100001, 40.740021462999884], [-73.75098241899988, 40.74002149899998], [-73.75112706799992, 40.74006711599992], [-73.7511273519999, 40.74006720699986], [-73.75112759899997, 40.740067314999884], [-73.75126376099992, 40.74012514299993], [-73.75126399699991, 40.74012524299995], [-73.75126422199989, 40.74012536999987], [-73.75138967499991, 40.74019415999988], [-73.75347299999999, 40.74240484399992], [-73.75253331799986, 40.742611332999914], [-73.75163327499985, 40.742806759999915], [-73.75072790300003, 40.74299784699989], [-73.74979834499989, 40.74314244299993], [-73.74887697299994, 40.74328837499995], [-73.74784861300004, 40.743296077999936], [-73.74671975299987, 40.74312364799993], [-73.74567621100002, 40.7430272989999], [-73.74477587099986, 40.74328109699994], [-73.74593060399998, 40.74571565899988], [-73.74625984399985, 40.74671147499993], [-73.74384322099985, 40.74769009599988], [-73.74409668799983, 40.748206641999936], [-73.74507911699986, 40.75027389699992], [-73.74577948799998, 40.75156080499994], [-73.746216657, 40.75168585099987], [-73.74637489399997, 40.75176036499987], [-73.74652681399978, 40.7518426459999], [-73.74667146099986, 40.75193218699994], [-73.746807946, 40.752028417999924], [-73.74684219699994, 40.75205807499989], [-73.74687468099984, 40.75208864699991], [-73.7470988539999, 40.75234682899989], [-73.74737514999993, 40.752959618999974], [-73.74760569999985, 40.75350787799988], [-73.74774116499985, 40.75384994499991], [-73.74711113899991, 40.75417328599991], [-73.74809587899985, 40.75589954999988], [-73.75004764499985, 40.75840465799996], [-73.75057710699984, 40.75801489499991], [-73.75115582999983, 40.7590697689999], [-73.75230259399993, 40.759455198999916], [-73.752771716, 40.75887302799991], [-73.75320997199995, 40.7582448729999], [-73.75335958499983, 40.75832324899992], [-73.75352058499996, 40.758390329999905], [-73.75369069100002, 40.75844478799994], [-73.7538672559999, 40.75848565299992], [-73.75404744899984, 40.75851238599993], [-73.75422838200002, 40.75852489199992], [-73.75497607999992, 40.75850155999997], [-73.75511648399994, 40.758479497999915], [-73.75525632799999, 40.75844214599991], [-73.75539137399988, 40.758389247999915], [-73.75551718999986, 40.75832147399991], [-73.7555734369999, 40.75828095199988], [-73.75583713299987, 40.757742061999934], [-73.75625606399994, 40.75843046999994], [-73.75682062199998, 40.75973480599985], [-73.75800131799996, 40.76037986699988], [-73.75848075099995, 40.7619947369999], [-73.75811452099987, 40.76205389699988], [-73.75802364199994, 40.76211096099997], [-73.75794391899996, 40.76218407699988], [-73.75792673399994, 40.7622083249999], [-73.75788220699995, 40.76227115299988], [-73.75784418199986, 40.76236794099986], [-73.75783301499992, 40.76246859399991], [-73.75784470399994, 40.76254231099988], [-73.75784858699993, 40.76256679999993], [-73.75788761399987, 40.76265702199986], [-73.75794474699994, 40.7627354049999], [-73.75809857499982, 40.76298453899995], [-73.75823811499988, 40.763238765999944], [-73.75836296599996, 40.76349738299991], [-73.75844568799998, 40.76368152999987], [-73.75843838499995, 40.763928487999856], [-73.75844672099987, 40.76411514599987], [-73.75851382899985, 40.76410483099993], [-73.75859320499987, 40.764088229999906], [-73.7588643139999, 40.76327454099993], [-73.759097561, 40.76405669599989], [-73.75921689799992, 40.76432485599988], [-73.75862488400004, 40.76438314999996], [-73.75856122399995, 40.76438654599987], [-73.75850831399994, 40.76439435899995], [-73.7583941619999, 40.7644052469999], [-73.75635669699987, 40.764599557999915], [-73.75601361099996, 40.76463784199992], [-73.75567226899989, 40.76469114899985], [-73.75533439, 40.764759409999954], [-73.75500165499989, 40.76484240599992], [-73.75467572399998, 40.76493975999991], [-73.75435817399999, 40.76505094399988], [-73.75405049899983, 40.7651753319999], [-73.75377036482259, 40.76530463174143], [-73.75380733759789, 40.76511533246209], [-73.75380441445209, 40.765068227844544], [-73.75375190107101, 40.76503043842622], [-73.75367229455681, 40.764971622785595], [-73.75355704488845, 40.76486880710364], [-73.75346745383098, 40.76477581534768], [-73.75340994726432, 40.76469265965659], [-73.75330736474879, 40.764638713155435], [-73.75321775669782, 40.76455060405654], [-73.75311540991821, 40.764433163533354], [-73.75297443068116, 40.76434006075026], [-73.75285898250387, 40.76429096965374], [-73.75264732366146, 40.764202599394764], [-73.7526925441715, 40.76413431754799], [-73.75264128126541, 40.76410001719451], [-73.7524937694181, 40.76403620602584], [-73.75236834738628, 40.763949379623185], [-73.75226304070037, 40.763894066011986], [-73.75219261536506, 40.76383041851506], [-73.75212233558226, 40.763727698090904], [-73.75204563186178, 40.76362496379624], [-73.7520040194552, 40.76350287108586], [-73.75195275558251, 40.76336126369825], [-73.75189013131875, 40.76320438155103], [-73.75181884878003, 40.76299772753364], [-73.75178042777567, 40.76289650166449], [-73.75160082700452, 40.76257386182551], [-73.75150345347899, 40.762362502304455], [-73.75139722117518, 40.76223244243897], [-73.75128533569809, 40.76211176666061], [-73.75117340232892, 40.762003900915104], [-73.75107037762622, 40.761917410250106], [-73.75105245896142, 40.761902850669735], [-73.75104033492728, 40.76184047245043], [-73.7510842820423, 40.761803838736846], [-73.7510743225255, 40.76176367133622], [-73.7509735123674, 40.76168572586612], [-73.75090170424691, 40.76166250926074], [-73.75082187737034, 40.76168112774134], [-73.75072728692017, 40.76174242208793], [-73.75057468214737, 40.7616788158593], [-73.75041128338685, 40.7615121726565], [-73.75019289977786, 40.7614285535738], [-73.74998548162718, 40.76133664570978], [-73.74949376072769, 40.76124411676728], [-73.74914665740043, 40.761192791089265], [-73.74868090837376, 40.76096434231705], [-73.74861225072561, 40.76092780307599], [-73.74853166386495, 40.760882140561826], [-73.7484570766551, 40.76083194257801], [-73.7484004868574, 40.76076813730873], [-73.7482281259856, 40.76050136976691], [-73.74796573496857, 40.7602960998587], [-73.7478976470288, 40.76023808603646], [-73.74778567833137, 40.76014217625904], [-73.74773327425393, 40.76003870885871], [-73.7476583327047, 40.75995398486197], [-73.74753513475062, 40.75985719618731], [-73.74746117500982, 40.75981005494173], [-73.74728967123532, 40.759714014147775], [-73.7471698249882, 40.759622357401994], [-73.74695558696763, 40.75953903440671], [-73.74673016390221, 40.75944372971427], [-73.7465598186793, 40.75933914768842], [-73.7464254631052, 40.75922354214357], [-73.74628904491358, 40.75905924481752], [-73.74628582855765, 40.75901909233278], [-73.74628600098946, 40.75897382351047], [-73.74629189268191, 40.75890123331028], [-73.74632597925213, 40.75879966345884], [-73.74634096546987, 40.7586989079362], [-73.74632333540838, 40.75860918308185], [-73.74630325887233, 40.75857155662324], [-73.74635621902131, 40.758528110630884], [-73.74641486447456, 40.758466741898815], [-73.74638144167064, 40.75839406564263], [-73.74640309403442, 40.758312968449296], [-73.74641337628168, 40.7582677218775], [-73.7464134955851, 40.758267113348396], [-73.74642482955997, 40.758209665279864], [-73.74641167584906, 40.758124220706435], [-73.74637157591067, 40.758035301609446], [-73.74633023584065, 40.75797712659557], [-73.74631454146692, 40.757968550858195], [-73.74626743586923, 40.757949655989066], [-73.74624838205933, 40.75793850985342], [-73.74624390337235, 40.757935084455255], [-73.74624054497137, 40.757931660352156], [-73.7462371874519, 40.7579282362508], [-73.7462360784208, 40.75792481827516], [-73.74623496178856, 40.75792310637381], [-73.74623496830108, 40.75792139694663], [-73.74623497480721, 40.757919689194615], [-73.74623498131783, 40.757917980269944], [-73.74623499433008, 40.757914564766], [-73.74623725253437, 40.75791115303519], [-73.74625308746172, 40.75788300039028], [-73.74627682644942, 40.75784376316326], [-73.74628135718679, 40.7578335225235], [-73.74628363711001, 40.75782498586535], [-73.74628478821236, 40.75781730093923], [-73.74628480122651, 40.757813884262504], [-73.74628369242, 40.75781046511508], [-73.74628034510107, 40.757804479723], [-73.74627700539382, 40.75779678540849], [-73.74626136313661, 40.7577745414491], [-73.74623789033056, 40.75774374023382], [-73.7462188906345, 40.757718928060626], [-73.7462099395368, 40.75770951361565], [-73.74620770191625, 40.757706946198816], [-73.7462054625862, 40.75770523232625], [-73.7461953829455, 40.757697523160154], [-73.74618193736195, 40.75768895185608], [-73.7461752084484, 40.75768552149715], [-73.7461684791486, 40.75768294418255], [-73.74615725922665, 40.75767950107878], [-73.74615501338599, 40.75767949612995], [-73.74614828083048, 40.75767777352784], [-73.74613929851625, 40.757676900688345], [-73.74613143454941, 40.757676883358265], [-73.74611008616621, 40.75767854407576], [-73.74610334252881, 40.757679382258424], [-73.74609771771479, 40.757681933521056], [-73.74607183962367, 40.757692980648315], [-73.74605833716052, 40.75769892890423], [-73.74600094895115, 40.75772528164094], [-73.74598069133656, 40.75773548641946], [-73.7459491686193, 40.7577542080311], [-73.74589061547654, 40.75779166231073], [-73.74588385596999, 40.75779677187182], [-73.74587709276122, 40.75780273614509], [-73.74584552065286, 40.75783427072537], [-73.74583763539503, 40.75784023252097], [-73.74582750623101, 40.75784533414149], [-73.74581738010093, 40.75784958255485], [-73.74580726225128, 40.75785212271662], [-73.74579714457572, 40.75785380882725], [-73.74579265266253, 40.757853798915036], [-73.7457892814692, 40.757853791475746], [-73.74578479215516, 40.75785292735082], [-73.74578142620533, 40.75785206570453], [-73.74577918555481, 40.757850352993536], [-73.74577582220617, 40.75784863663218], [-73.74576014295548, 40.75783664481693], [-73.74575006788852, 40.757827226180424], [-73.7457444715135, 40.757822089357006], [-73.7456974812008, 40.75777329804245], [-73.74567957544278, 40.75775532210831], [-73.74563031826347, 40.75771165241087], [-73.74554859793469, 40.75763886874296], [-73.74553069357489, 40.75762089278826], [-73.74552062382129, 40.75761061942213], [-73.74548705882032, 40.757575525503874], [-73.74546131211113, 40.757552406566525], [-73.74544674827708, 40.75754212493889], [-73.74542993623295, 40.75753269138673], [-73.74538396629083, 40.75751038195785], [-73.74537387044663, 40.75750694411011], [-73.74532563749915, 40.75748889990954], [-73.7453043316149, 40.757479456404035], [-73.74528752286861, 40.757469169792586], [-73.74527295907737, 40.757458888645495], [-73.74526176625159, 40.75744861277299], [-73.74525169762303, 40.75743748684236], [-73.74523383674908, 40.75740840676913], [-73.74522713431325, 40.75739899554475], [-73.74519135509507, 40.75735535546363], [-73.74515554196519, 40.75732110951952], [-73.74512866796508, 40.757298842230995], [-73.74509394882874, 40.75727228732276], [-73.74508946871022, 40.75726886020077], [-73.74508610540423, 40.7572671449918], [-73.74507601527569, 40.7572619970184], [-73.7450390130659, 40.75724483344262], [-73.74502892382331, 40.75723968613712], [-73.74502331642866, 40.757237110065674], [-73.74500848856957, 40.75722838462264], [-73.74499417705718, 40.75721996387875], [-73.74498184854157, 40.75721310217035], [-73.74498072779814, 40.7572122454709], [-73.74495943751793, 40.75719853218402], [-73.74494264524719, 40.75718397530416], [-73.74493593369215, 40.75717627180867], [-73.7449303481282, 40.7571685724813], [-73.7449236542483, 40.75715659927284], [-73.74491360993711, 40.75713949484265], [-73.74490803614141, 40.75712837883434], [-73.74489352243867, 40.75710528413908], [-73.74488682530038, 40.757094165139506], [-73.74488348163936, 40.75708732432283], [-73.7448812604461, 40.757080486662126], [-73.74487575929356, 40.757050579684545], [-73.74487577237984, 40.757047163007705], [-73.74487355184948, 40.7570403253482], [-73.74486908179121, 40.75703433625488], [-73.74486796826072, 40.757031770128414], [-73.74486572898314, 40.7570300574023], [-73.7448646115155, 40.7570283466582], [-73.74486237136128, 40.757026632757416], [-73.74485676618775, 40.757024058357956], [-73.74485003174925, 40.757022335675316], [-73.74483543956013, 40.75701974086647], [-73.74481186189642, 40.757017126151005], [-73.74479277587083, 40.757014521381635], [-73.74478716195661, 40.75701365422342], [-73.74478155285772, 40.75701193285728], [-73.74477594221253, 40.75701021266016], [-73.74477145951205, 40.757007640743375], [-73.7447681003836, 40.757005070813264], [-73.74474344954679, 40.75698878704991], [-73.74472217336258, 40.756971657043756], [-73.74467627739422, 40.75693055636426], [-73.74464494262489, 40.75689973739264], [-73.7446359905721, 40.75689117586707], [-73.74463376024596, 40.75688690117124], [-73.74463152687576, 40.7568834780066], [-73.74462817997272, 40.75687749239859], [-73.74462819307169, 40.7568740757217], [-73.74462820289574, 40.75687151325582], [-73.74462822254384, 40.75686638832415], [-73.74462936531678, 40.75686126638481], [-73.7446406799881, 40.75683993785017], [-73.74464180595058, 40.75683908562516], [-73.74464294894216, 40.756833963686155], [-73.74464409259474, 40.75682884174865], [-73.7446441089695, 40.756824570357985], [-73.7446452569718, 40.756817738988104], [-73.7446453028102, 40.756805781875045], [-73.74464533555502, 40.75679724026632], [-73.74464542068948, 40.756775032619544], [-73.7446421825129, 40.75674085921853], [-73.74463894040585, 40.756707539524264], [-73.74464011025233, 40.75669558440013], [-73.74464352524502, 40.756683634085206], [-73.74464465907351, 40.756681074110936], [-73.74464918475132, 40.75667254296403], [-73.74465596716065, 40.75666145382896], [-73.74469086913759, 40.756640177067666], [-73.74469001514127, 40.75661324074209], [-73.74468986758575, 40.75660857161472], [-73.74468877720241, 40.756600027516434], [-73.74465748646857, 40.75655810497731], [-73.74465411927862, 40.75655724279255], [-73.74462653500422, 40.75655130464249], [-73.74462606107197, 40.75655120257331], [-73.74461483813911, 40.756548615205176], [-73.74460585993965, 40.75654688636107], [-73.74460137007021, 40.75654602168568], [-73.74455864030779, 40.756557885832535], [-73.74455292838134, 40.75658264466303], [-73.74454039038096, 40.75663044806491], [-73.7445502656967, 40.75669111478052], [-73.74455137593789, 40.75669453394865], [-73.74455908496117, 40.756734695287655], [-73.744560185375, 40.756740677424126], [-73.74456566802672, 40.75677571002285], [-73.74456887603034, 40.756817569649364], [-73.7445710218452, 40.75684405366969], [-73.74457308423663, 40.756891891121754], [-73.74457639642843, 40.756907272386385], [-73.7445819524476, 40.75692266030129], [-73.7445897571554, 40.75693805152646], [-73.7446009329801, 40.75695259717943], [-73.74461548062342, 40.7569671491338], [-73.74462443683731, 40.75697485712175], [-73.74462490978154, 40.75697521768243], [-73.74463451750677, 40.75698256693157], [-73.74464235954294, 40.756987708788095], [-73.74465468364045, 40.75699627778754], [-73.74469950605639, 40.7570245641607], [-73.74471071138478, 40.75703142173538], [-73.74471631220506, 40.75703570557416], [-73.74473870826283, 40.75705369278365], [-73.7447745258561, 40.757087082975666], [-73.74480809130256, 40.75712132355832], [-73.7448215019215, 40.75713929084582], [-73.74485052430722, 40.75718718683517], [-73.74486058801497, 40.75719916751141], [-73.74486618338649, 40.75720430487898], [-73.74487290409643, 40.75720944423588], [-73.74491660449456, 40.75723772854086], [-73.74491884704905, 40.75723858772495], [-73.74495919877508, 40.75726088540533], [-73.74497388924861, 40.75726953047292], [-73.7449883390321, 40.75727803277557], [-73.74499506563798, 40.75728146386926], [-73.74504887346743, 40.75730976932465], [-73.74506008019742, 40.75731662804068], [-73.74509033710218, 40.75733548612527], [-73.74510712943892, 40.757350044156034], [-73.74512056306564, 40.75736203176307], [-73.745125039926, 40.75736631242475], [-73.745148540614, 40.757389426971976], [-73.74516307822353, 40.757406542493584], [-73.74519549607847, 40.757447613163535], [-73.74521003022959, 40.75746558171702], [-73.7452234567032, 40.75747927756497], [-73.74524025695877, 40.75749212782685], [-73.74525817665554, 40.75750583360742], [-73.74527946383118, 40.757520400379555], [-73.74528282626937, 40.75752211675349], [-73.74528731383276, 40.75752383444244], [-73.74529292298591, 40.75752555628655], [-73.74529853606852, 40.75752642341801], [-73.74533783768734, 40.75752992584022], [-73.74535130665119, 40.75753251759941], [-73.7453569203959, 40.757533384729555], [-73.74536140249367, 40.75753595662189], [-73.74536588983982, 40.75753767430737], [-73.74536924923648, 40.75754024472283], [-73.74542635286744, 40.757588203300145], [-73.74547897707495, 40.7576327357487], [-73.74550361788225, 40.75765158081212], [-73.74555064800546, 40.757690121675985], [-73.7455663154504, 40.757705531372245], [-73.74558197157725, 40.75772350235585], [-73.7456110278219, 40.75776285771527], [-73.7456266711447, 40.75778424587238], [-73.7456423188386, 40.757804780488705], [-73.7456792232361, 40.75784756869152], [-73.74570608133453, 40.757874107240035], [-73.74573295575978, 40.75789637439206], [-73.74575984954205, 40.75791351710864], [-73.74576433167874, 40.75791608948764], [-73.74576881906094, 40.75791780715734], [-73.74578003900936, 40.7579212486236], [-73.74579238861907, 40.7579229853177], [-73.74580474105058, 40.75792386729598], [-73.74581822381136, 40.757923042325366], [-73.74583058928901, 40.7579205054465], [-73.74584183531493, 40.75791711355347], [-73.74585196056832, 40.7579128661413], [-73.74589923219948, 40.7578882001035], [-73.74592738556296, 40.75786947107443], [-73.74597807807304, 40.75783199944327], [-73.74599384573716, 40.757820931719834], [-73.74600622658913, 40.757814125105035], [-73.74601297697997, 40.75781157750125], [-73.74601859854921, 40.75780988095649], [-73.746024222978, 40.75780818559046], [-73.74602983803098, 40.75780819796979], [-73.74603433126282, 40.75780820787569], [-73.74603882515608, 40.757808217782866], [-73.7460421911072, 40.75780907942148], [-73.74604668174588, 40.75780994286928], [-73.74605453182136, 40.757813376879874], [-73.74606237384283, 40.75781851914108], [-73.74607581187087, 40.75782965309685], [-73.74610490620208, 40.757858758298426], [-73.7461463202189, 40.75789728603619], [-73.74616086056976, 40.75791354721772], [-73.74617203822314, 40.75792809271902], [-73.7461753788079, 40.75793578653575], [-73.74617871872883, 40.75794348135603], [-73.74618532398014, 40.757979371566336], [-73.74618866302863, 40.757987064876666], [-73.74619089673813, 40.757990487508565], [-73.74619312957277, 40.75799390846309], [-73.74619648775423, 40.75799733256716], [-73.74619984721016, 40.75799990195313], [-73.74620320407233, 40.75800332555147], [-73.74620768688935, 40.75800589891986], [-73.74621778416946, 40.75800933786992], [-73.74622339361433, 40.75801105799371], [-73.74622788360375, 40.758011922605704], [-73.74623013011656, 40.75801192755456], [-73.74625592955528, 40.75802137911002], [-73.74631196705487, 40.75805481402274], [-73.74630710944605, 40.75815046956326], [-73.74630110309532, 40.75825295380464], [-73.74629290506739, 40.75834091284919], [-73.74627011032777, 40.75842713197698], [-73.74619442433067, 40.758538859618106], [-73.74609973183355, 40.758626628049555], [-73.7460621494217, 40.7587615004492], [-73.74610201562086, 40.75891191914624], [-73.74616224371887, 40.759024799276894], [-73.74625836714088, 40.75915057154628], [-73.74635893029055, 40.759290872701186], [-73.74655378098826, 40.75945017403582], [-73.74672422976337, 40.75952742182279], [-73.74691154237048, 40.75960129137661], [-73.74708983971603, 40.75968282611382], [-73.74736655364313, 40.759879034440424], [-73.74749424153497, 40.75997668685211], [-73.7474971489094, 40.759981616662685], [-73.74754670154863, 40.76006563412159], [-73.74760477048451, 40.76015630091755], [-73.74765057016263, 40.76022387943766], [-73.74774348406547, 40.760308643293214], [-73.74790688416107, 40.76046872647574], [-73.7480344027219, 40.76061164876987], [-73.74815865904617, 40.76072637728021], [-73.74824810096803, 40.760838465418395], [-73.7482454950954, 40.76085038845037], [-73.7482389574376, 40.76088029909271], [-73.7482084978379, 40.760914398721546], [-73.74821281435773, 40.760961385107564], [-73.74824062871396, 40.761032340776104], [-73.74823023192417, 40.76110833840965], [-73.74821974793956, 40.76120739621181], [-73.74823056733841, 40.76131675001058], [-73.74825832786304, 40.761402225746664], [-73.74837456439464, 40.76155878963078], [-73.74847532850517, 40.761647841442766], [-73.74858298910182, 40.761695909285656], [-73.74872212469546, 40.76173806599188], [-73.7488690635061, 40.761796468669914], [-73.74902175447068, 40.761819009230464], [-73.74913505044145, 40.761861962688094], [-73.7492900714412, 40.76186230015056], [-73.74936203349068, 40.76184452034025], [-73.74938249132595, 40.76178135642719], [-73.74933672650815, 40.76170352992407], [-73.7493392273061, 40.761636058201745], [-73.74933054315699, 40.76155574870995], [-73.7492847562694, 40.76148390115628], [-73.74921977977574, 40.7614367811098], [-73.74917158863074, 40.76140678142459], [-73.74917394462378, 40.76137774482792], [-73.74947027461442, 40.76143988860852], [-73.74966102560238, 40.7614975308269], [-73.74979792102468, 40.76153797410411], [-73.75004042130583, 40.7615760827396], [-73.75018846125836, 40.7616404648094], [-73.75032282701994, 40.76175692013944], [-73.75046183009748, 40.76183494941578], [-73.75057049365302, 40.76191547482149], [-73.75063847963764, 40.76205911882312], [-73.75084802828285, 40.76219794405175], [-73.75098356953296, 40.76230073499874], [-73.75110899242155, 40.76240606640948], [-73.75123100279916, 40.76252249564248], [-73.75132609064228, 40.76262946906892], [-73.75142048789601, 40.76276169436681], [-73.7515099562405, 40.762949040571364], [-73.75153545083482, 40.76303855921119], [-73.75156943251743, 40.76316082612466], [-73.75156024509342, 40.76331573008709], [-73.75160045302414, 40.76343919180898], [-73.7516212179657, 40.763492759268495], [-73.75163503200241, 40.763528699851285], [-73.75163833199954, 40.76353629608098], [-73.75163834488852, 40.76353632576103], [-73.75163837349352, 40.76353639501102], [-73.75164899665232, 40.76355787419641], [-73.75165063764818, 40.76356119174191], [-73.75165082833455, 40.763561578636434], [-73.7516952345571, 40.763651827368705], [-73.7518283443928, 40.76380586056595], [-73.75197615773142, 40.76393259332168], [-73.75207014373261, 40.76403443876415], [-73.7521574815898, 40.76411150121125], [-73.75219219823434, 40.76414061642908], [-73.75222243034136, 40.76416715900465], [-73.75221773780626, 40.76422096091368], [-73.75225349976849, 40.76427057840176], [-73.75234326679305, 40.764298958896], [-73.75240147997198, 40.76435374973257], [-73.7524484210661, 40.764418765585695], [-73.7524785175615, 40.764482037465825], [-73.75251092750104, 40.76452737691871], [-73.75256704187835, 40.76454201880534], [-73.75266249865089, 40.76455076377027], [-73.75274102673858, 40.76457997404058], [-73.75281605035713, 40.76464590422408], [-73.75289327855934, 40.764722944411886], [-73.75298298856781, 40.764766697677715], [-73.75304571089994, 40.76481722743511], [-73.75307921070038, 40.76487196427665], [-73.75311373500155, 40.7649531830293], [-73.75314599997928, 40.76503781195535], [-73.75319227856545, 40.765101952761405], [-73.75321270194533, 40.765132346676744], [-73.75323764786424, 40.76516072229634], [-73.75326677589696, 40.76518669238562], [-73.75332826529235, 40.765267114094485], [-73.75338853072753, 40.7653339455862], [-73.75339682847972, 40.76535743094704], [-73.75341078980225, 40.765387042137355], [-73.75342969954757, 40.76541716951095], [-73.75345277791402, 40.76544728653655], [-73.75346684422311, 40.76546301168187], [-73.75346796389704, 40.7654638671222], [-73.75347120459557, 40.76546708872052], [-73.75347915705996, 40.76547499522476], [-73.75348713687973, 40.765483131298055], [-73.75349594446142, 40.76549211397729], [-73.75350218209317, 40.76549655404032], [-73.7535027027036, 40.765496924047376], [-73.75350496885727, 40.76549853681403], [-73.7535052704541, 40.765498787241235], [-73.75351274922001, 40.765504961846815], [-73.75355758134413, 40.76553324473846], [-73.75357662436602, 40.765547806322985], [-73.75359676377177, 40.765570057699804], [-73.75361689198797, 40.76559658817567], [-73.75361887109622, 40.76559925976317], [-73.75363589047707, 40.76562224392221], [-73.75365491524836, 40.76564192991895], [-73.7536593957098, 40.765645356702194], [-73.75366115798266, 40.765646367303766], [-73.75366387999607, 40.7656479282712], [-73.75367357800295, 40.76565417244392], [-73.75368517388588, 40.765661640433045], [-73.75369077667018, 40.76566592383144], [-73.75369215758916, 40.76566698018958], [-73.75369525780206, 40.765669348939326], [-73.75370754683831, 40.76568731326578], [-73.75370946721739, 40.76569067426458], [-73.75372317471823, 40.76571467893723], [-73.75372540254051, 40.765719808666475], [-73.75372586129843, 40.76572266530188], [-73.75372650076784, 40.76572664491773], [-73.753728648975, 40.765753127054154], [-73.75372941146722, 40.76575488201507], [-73.7537308774602, 40.76575825678467], [-73.7537314575656, 40.76576095067618], [-73.75373198134588, 40.76576338427885], [-73.7537546208494, 40.76579963528729], [-73.75375655148282, 40.76580272725988], [-73.75378449274987, 40.765840364315075], [-73.75381582683036, 40.765874602969355], [-73.75383791548944, 40.76589543549867], [-73.75384941236669, 40.7659062782456], [-73.753852845085, 40.76590852859029], [-73.75385725748721, 40.765911419470896], [-73.75386677763164, 40.76591733439259], [-73.75387519046984, 40.76592256193281], [-73.75387692703067, 40.76592355706362], [-73.7538796747829, 40.76592513349339], [-73.75388209534087, 40.76592760180443], [-73.75388303331701, 40.76592855787087], [-73.75393115954603, 40.765977347183544], [-73.7539457200029, 40.76598933597816], [-73.75397933654801, 40.76601247028503], [-73.75399278071055, 40.766022748423524], [-73.75400845052455, 40.76603815745188], [-73.75402298892796, 40.76605612487066], [-73.75403639591704, 40.76607665185295], [-73.75405983454807, 40.766117702460434], [-73.7540687588374, 40.766134803839336], [-73.75406990996136, 40.76613770902588], [-73.7540765503842, 40.76615446561802], [-73.75409428749936, 40.76621856488986], [-73.75409438750937, 40.76621886983374], [-73.75410206777033, 40.76624164351026], [-73.75410302014075, 40.76624383528004], [-73.75411097902656, 40.766262163065655], [-73.75412214720791, 40.766280123278335], [-73.75413696709579, 40.766297128508526], [-73.7541400449881, 40.766300661506705], [-73.75415172628783, 40.76631207437325], [-73.75416019515129, 40.76632034981886], [-73.75416058406073, 40.76632064599768], [-73.75418483942393, 40.766339193499206], [-73.75420482305684, 40.76635314434022], [-73.75421061235566, 40.766357184862485], [-73.75421846070479, 40.766361473025135], [-73.75422631000248, 40.76636490747495], [-73.75426781969234, 40.7663812241278], [-73.75427395377056, 40.76638474336995], [-73.75427678686661, 40.766386367717004], [-73.75428575309054, 40.766392366525565], [-73.7542880240036, 40.76639425537017], [-73.75431151258437, 40.766413774537455], [-73.75433389504231, 40.76643688478174], [-73.75435199344946, 40.76645969053854], [-73.75435290758911, 40.76646084037817], [-73.75436854227797, 40.76648564449461], [-73.754427699876, 40.76658997735722], [-73.75445786021571, 40.76664274408993], [-73.75450248682485, 40.76672082036144], [-73.75453245940645, 40.76677266996907], [-73.75453709317966, 40.766780685758185], [-73.75454235546341, 40.76678927381596], [-73.75458846135214, 40.76686450022992], [-73.75460968591979, 40.76689700433747], [-73.75463428056362, 40.766930368159684], [-73.75466224335983, 40.766963739651175], [-73.75469245300037, 40.766997115919565], [-73.75470700774542, 40.76701081303696], [-73.75472380440347, 40.767025369644536], [-73.75473151276402, 40.76703126304617], [-73.75480222531259, 40.767085326550436], [-73.75481634567822, 40.76709756139691], [-73.75481902377297, 40.767099883147836], [-73.75483357366191, 40.76711443378519], [-73.75487271796067, 40.76716234936266], [-73.75488500979354, 40.76718031356404], [-73.75488893674114, 40.7671882191101], [-73.7548939308534, 40.76719826891797], [-73.75491614552455, 40.76724764045222], [-73.75492624631949, 40.767270087181146], [-73.754945267647, 40.767303935292325], [-73.75496084794494, 40.7673316585225], [-73.75497199993241, 40.767353891708765], [-73.75498535207142, 40.76738893930633], [-73.75499531479272, 40.76742910516272], [-73.75499536929894, 40.76742938253471], [-73.75507210849031, 40.76753310678742], [-73.75507603383382, 40.767663748308834], [-73.75499605461913, 40.76784780377672], [-73.75495617473564, 40.76799105146965], [-73.75492911896265, 40.76801576419739], [-73.75492457714904, 40.768021104656974], [-73.75491896445862, 40.76802770096123], [-73.7549042883991, 40.768047315050715], [-73.75489184864901, 40.768069496881004], [-73.75488164924215, 40.76809339023432], [-73.75487256100014, 40.76812070533027], [-73.75487060374897, 40.7681292428303], [-73.75486571084737, 40.76815058431918], [-73.7548627188635, 40.7681803689903], [-73.75485987826801, 40.76820865361768], [-73.75485653531162, 40.76822609437051], [-73.75485529629613, 40.76823256109786], [-73.75485129960616, 40.768247537320136], [-73.75484960110137, 40.76825390205139], [-73.75484682303404, 40.768261426615666], [-73.75483258063379, 40.768299990854324], [-73.75482801752048, 40.76831877123044], [-73.75482801691055, 40.768318936912685], [-73.7548279986502, 40.7683238971604], [-73.75482798602786, 40.76832732589454], [-73.75482798293194, 40.76832816687006], [-73.75482908672076, 40.768333295356385], [-73.75483123007663, 40.768336576564344], [-73.75483243914007, 40.768338426952276], [-73.7548347461836, 40.76834231512126], [-73.75483802115825, 40.7683478340406], [-73.75485479639369, 40.76836836976145], [-73.75485605431479, 40.76837029647758], [-73.75486150013704, 40.768378633451896], [-73.7548624413177, 40.7683822640742], [-73.75486260862132, 40.76838290622344], [-73.7548632480079, 40.7683853772567], [-73.75486371578062, 40.76838717982988], [-73.75486371140651, 40.76838758992498], [-73.75486370005846, 40.76839145121471], [-73.75486256146147, 40.768395718535245], [-73.75486142307676, 40.768399988034055], [-73.75486084218902, 40.7684010797398], [-73.75485915829324, 40.76840425295842], [-73.75485577282537, 40.76840851717352], [-73.75485126381926, 40.76841277782529], [-73.75485002766149, 40.76841361835497], [-73.75484000164228, 40.76842044080652], [-73.75480509138609, 40.768442574811026], [-73.75480345733583, 40.76844411827146], [-73.75480058171519, 40.7684468349567], [-73.75479607269864, 40.768451096779025], [-73.75479533576191, 40.768451840702745], [-73.75479269036948, 40.76845450678226], [-73.75479042712017, 40.76845877221111], [-73.75478135898942, 40.768480960701346], [-73.75477459400017, 40.768506719555276], [-73.75476768394783, 40.768533035794846], [-73.75476068535333, 40.76854827014798], [-73.7547574915997, 40.768555220214914], [-73.75475315314952, 40.7685625861583], [-73.75474392479494, 40.76857825384548], [-73.75473150442745, 40.768595310733495], [-73.75472181615622, 40.76860488685348], [-73.75471684573888, 40.768609800372744], [-73.75470219963017, 40.76862087283543], [-73.7546923613104, 40.768626915000965], [-73.75468418375833, 40.768631938623905], [-73.75466855538552, 40.76863815022539], [-73.75466280690638, 40.768640433597966], [-73.75464656395906, 40.76864533466979], [-73.75464031279178, 40.768647219095804], [-73.75463798636613, 40.76864763513515], [-73.75461670478812, 40.76865143973457], [-73.75460658949804, 40.768652273915315], [-73.75459423181064, 40.768652247594744], [-73.75458524725447, 40.768651372733466], [-73.75455307361956, 40.768649145119234], [-73.75454705986645, 40.76864872990984], [-73.75454548351453, 40.768648724876286], [-73.754538071276, 40.76864871076057], [-73.75453734347198, 40.76864870921004], [-73.75453694839094, 40.76864870836834], [-73.75452908086366, 40.768649544650586], [-73.75452766896971, 40.76864987703017], [-73.7545110901395, 40.76865377824045], [-73.75450838321264, 40.76865418307988], [-73.75450546904293, 40.76865461930755], [-73.75449196913841, 40.76865971500431], [-73.7544891892807, 40.76866107492285], [-73.75447283530931, 40.76866907061462], [-73.75445332042013, 40.76867994001487], [-73.75445144581661, 40.76868098339416], [-73.75444506541723, 40.76868540471995], [-73.7544379299309, 40.76869035097133], [-73.75442666048079, 40.76869972165972], [-73.75441753972149, 40.768707995776516], [-73.75441538875721, 40.76870994706131], [-73.75440748905719, 40.76871932610307], [-73.75440071107668, 40.768729561082004], [-73.75439618245464, 40.768738947811386], [-73.75439547239391, 40.76874471037513], [-73.75439502293871, 40.76874834004711], [-73.75439438120904, 40.76875122834032], [-73.7543938826934, 40.76875346274863], [-73.75439432480357, 40.768755170784054], [-73.75439498828825, 40.768757735853434], [-73.75439497883637, 40.76876029881814], [-73.75439497838663, 40.76876042077632], [-73.7543949779937, 40.76876052732225], [-73.75439497568617, 40.768761153028116], [-73.75439609110637, 40.76876371788809], [-73.75439716604436, 40.76876495149782], [-73.75439832787339, 40.76876628463585], [-73.75440056618083, 40.76876885188936], [-73.7544039271558, 40.76877142153601], [-73.75440729062186, 40.76877313646868], [-73.75441292701873, 40.76877621605693], [-73.75441513992966, 40.76877742461781], [-73.7544263612727, 40.76878086523446], [-73.7544387095356, 40.768783454035166], [-73.75444246049993, 40.768783462029525], [-73.75444320350958, 40.76878346361305], [-73.75445218896665, 40.76878433748228], [-73.75446197302739, 40.768784358333434], [-73.75446451187187, 40.76878436156604], [-73.75446454713334, 40.76878435996597], [-73.75447054668635, 40.76878396817526], [-73.75460276521305, 40.76877526190596], [-73.75467018424933, 40.76877284350026], [-73.75467355401318, 40.768772850675404], [-73.75467639031449, 40.7687733958143], [-73.75467804330147, 40.76877371327814], [-73.7546800519135, 40.76877415513362], [-73.75468590272664, 40.768775438948126], [-73.75469052084651, 40.76877745792352], [-73.75469375453986, 40.76877887236429], [-73.75470047988182, 40.768783156928265], [-73.7547051792433, 40.76878615158182], [-73.75470720411681, 40.76878744316481], [-73.75471167921705, 40.768792577153974], [-73.75471391687849, 40.76879514389752], [-73.7547150263036, 40.76879856228822], [-73.7547150243261, 40.76879909920572], [-73.75471501686158, 40.768801125923034], [-73.75471586026373, 40.76880306549521], [-73.75471613163991, 40.768803689103244], [-73.75471580642598, 40.76880769195444], [-73.75471495394928, 40.768818207440624], [-73.75471296984013, 40.76884132502884], [-73.75471143383741, 40.768859199997806], [-73.75471115070579, 40.76893607268946], [-73.75471113632253, 40.768939977864314], [-73.754711122394, 40.768943759573354], [-73.75471342579756, 40.768949656879414], [-73.75471446004603, 40.768952308342094], [-73.75473118097804, 40.768988218353776], [-73.75473175595867, 40.7689922626574], [-73.75473227688181, 40.76899590763075], [-73.75473311585844, 40.769001801986654], [-73.7547333705779, 40.76900359807563], [-73.75472987183133, 40.76903861032414], [-73.75472073687521, 40.76907873671094], [-73.75469682582235, 40.76916495330184], [-73.75468757418824, 40.76919361460781], [-73.75467864238544, 40.76922128852382], [-73.75467753501032, 40.76922867776093], [-73.75467633797275, 40.76923665968261], [-73.75467516247359, 40.7692511768507], [-73.75467563707237, 40.76926171442707], [-73.75467619953538, 40.76927424106258], [-73.7546820222256, 40.76930279223348], [-73.75468613309415, 40.769322948032354], [-73.75469736216039, 40.76940669722155], [-73.7546981632071, 40.769412660014815], [-73.7546992140291, 40.769431452837644], [-73.75469914166756, 40.769451098490784], [-73.75469444943913, 40.76950490045573], [-73.75468301130525, 40.76956039582006], [-73.75468068508523, 40.769581743934395], [-73.75468172347436, 40.76960480865059], [-73.75468418709654, 40.76962739493064], [-73.75468498470264, 40.769634710826296], [-73.7546863132379, 40.76964510966561], [-73.75469044476876, 40.76967742908769], [-73.75469594529733, 40.769709044463255], [-73.7547019742048, 40.76972864329839], [-73.75470593650574, 40.769741522945246], [-73.75472038781855, 40.76978340730226], [-73.75472483218735, 40.76979708338937], [-73.75472685555812, 40.76980291978894], [-73.75474706538549, 40.76986119159111], [-73.75475122821963, 40.769872157507734], [-73.75475485442077, 40.76988170819281], [-73.75476377580713, 40.769899663549594], [-73.75478053634816, 40.76992446948909], [-73.75481298260995, 40.76996126716663], [-73.75487228166313, 40.77002801719697], [-73.75490125042909, 40.77008055843747], [-73.75520665378217, 40.77067505360193], [-73.75533279459194, 40.77099815465322], [-73.75540415008778, 40.7709818761834], [-73.75539975859014, 40.77097006430209], [-73.75551295786343, 40.770945013956315], [-73.75553498728895, 40.77098552649437], [-73.75540180261751, 40.77101727887619], [-73.75539741111736, 40.77100546716245], [-73.75534216255899, 40.77102215324632], [-73.75540541388642, 40.77115189018926], [-73.75542237803558, 40.771186687391356], [-73.75550243961897, 40.77128250909249], [-73.7555614028682, 40.77144869926292], [-73.7555733264932, 40.77152912592269], [-73.75550146510625, 40.77159968041305], [-73.75542423703571, 40.77164213534985], [-73.75525555666596, 40.7716574387015], [-73.75508267713634, 40.77171322899481], [-73.75510753193235, 40.77183111826108], [-73.7551141490279, 40.771830713509175], [-73.75511523562321, 40.77183490648697], [-73.75507386869062, 40.771840266849175], [-73.75507498743053, 40.77183607855887], [-73.7550805011998, 40.77183567146361], [-73.75505920386043, 40.771719174693374], [-73.75501054255479, 40.771737511519824], [-73.75485465862842, 40.77179791526066], [-73.75470604943379, 40.771870737751996], [-73.75438074757537, 40.772041271642195], [-73.75429600282433, 40.7721303370364], [-73.75419859120392, 40.77220176666602], [-73.75403331561613, 40.77233837754101], [-73.75397521229509, 40.772383121369145], [-73.75392578468146, 40.772419465504065], [-73.75387756050716, 40.772454887276105], [-73.7538086391264, 40.772504431493566], [-73.75372221291418, 40.772570217951014], [-73.75363974518132, 40.77263163450949], [-73.75357000906136, 40.77269461612923], [-73.75350609831754, 40.77276955713253], [-73.75345192598736, 40.77286268480925], [-73.75340985812166, 40.772959174816684], [-73.75338994029593, 40.773056448326535], [-73.75336691999294, 40.77320784776336], [-73.7533923720637, 40.773272201808666], [-73.75347847944079, 40.77348991346416], [-73.75356167498403, 40.77366488473272], [-73.7536224771741, 40.77382223841322], [-73.75363972748997, 40.77386687964439], [-73.75367969121629, 40.77397297882968], [-73.75375277248573, 40.774055560926136], [-73.75380768158755, 40.77414573010844], [-73.75383615718921, 40.77422433743274], [-73.75391367176762, 40.77427246809169], [-73.75395573939913, 40.77430620907517], [-73.75398027625992, 40.77439672509203], [-73.75400596724323, 40.77446940655343], [-73.75402061841987, 40.77451085549081], [-73.75410406104969, 40.77471047414804], [-73.7541211549045, 40.774917483321616], [-73.75414415458538, 40.775044679046815], [-73.75418344182958, 40.77511851809306], [-73.7541895302817, 40.77512995803968], [-73.75427404182139, 40.775184235667176], [-73.75424149999385, 40.77526683036528], [-73.75433349266498, 40.77541954835593], [-73.75442248917767, 40.77554919354539], [-73.75448302120184, 40.775629023427626], [-73.75454560558667, 40.77568395070161], [-73.754571582022, 40.77570674904423], [-73.75457996749098, 40.775718976040814], [-73.7546604244237, 40.77583628448718], [-73.754709121297, 40.77585519732232], [-73.75475233563776, 40.77587791923645], [-73.75480097120251, 40.77589859528199], [-73.75480082727547, 40.77593768284009], [-73.75481957405265, 40.77599121013415], [-73.75483825982116, 40.776061197070916], [-73.75485431476889, 40.77611060485996], [-73.75485691445691, 40.776139412150236], [-73.75488688128335, 40.77618932212071], [-73.7549295876917, 40.77624654271092], [-73.75501053125387, 40.77631254606015], [-73.75509684051448, 40.776377323742835], [-73.75513664233078, 40.77640719547497], [-73.75516154144688, 40.77645893085605], [-73.75518575394315, 40.7764980708302], [-73.75517697141768, 40.776543673716674], [-73.75510694138278, 40.77658842240476], [-73.75506630223659, 40.77660067868264], [-73.75502286717963, 40.77663761615085], [-73.75497549376992, 40.7766984471819], [-73.75496848490822, 40.7767074476952], [-73.7549385058147, 40.77676498578115], [-73.75489578230594, 40.776829846623684], [-73.75488054179407, 40.77684806875527], [-73.75486883106046, 40.77686774160918], [-73.75486088379957, 40.77688847260641], [-73.75485685860569, 40.77690984805278], [-73.75485683580645, 40.77693144139307], [-73.75485996015284, 40.7769477275573], [-73.75486616959907, 40.77696349456987], [-73.75487533188723, 40.77697840659675], [-73.75488725186452, 40.77699214601451], [-73.75490167563905, 40.777004420176], [-73.75491970109897, 40.777015725276094], [-73.75493983826522, 40.777024738326176], [-73.75498193747218, 40.77704380336285], [-73.75497732886075, 40.777063064882405], [-73.75496863967113, 40.777081494448325], [-73.75495610373531, 40.77709859610114], [-73.75494005840592, 40.777113909617135], [-73.75492093547855, 40.77712702289251], [-73.75491087095614, 40.77713571502858], [-73.7549026611194, 40.77714546896862], [-73.75489649670558, 40.777156058102484], [-73.7548925209316, 40.77716723641653], [-73.75489082616622, 40.777178744208726], [-73.75489145178453, 40.777190314122066], [-73.75489438325314, 40.77720167735625], [-73.75483577692417, 40.777466986201695], [-73.75484180870602, 40.77748504846197], [-73.754849296987, 40.77750747469179], [-73.75481517188274, 40.7776130205783], [-73.75481322893425, 40.77761903230558], [-73.75541831287357, 40.77763574482788], [-73.75542434578475, 40.777583945671296], [-73.75551781604716, 40.777584144019926], [-73.75551737581792, 40.77770404836418], [-73.75542002110872, 40.77770088125016], [-73.75542214211723, 40.77765351641497], [-73.75481217058518, 40.77763599592199], [-73.75478633943226, 40.77764784602946], [-73.754710055367, 40.77767238421957], [-73.75468724014247, 40.77768181762751], [-73.75466391587001, 40.77769477151285], [-73.75464068611555, 40.77771048072842], [-73.75460152243082, 40.77773986165084], [-73.75457883060362, 40.77775777932316], [-73.75455791105837, 40.77777659066429], [-73.75448452828304, 40.777904467981585], [-73.75464981317504, 40.777992106294526], [-73.7544419628204, 40.77812515653839], [-73.75437336549648, 40.77826269398068], [-73.75430183540871, 40.778406111489865], [-73.75430006363402, 40.778417615914684], [-73.75428865261564, 40.778517935169106], [-73.75428214069518, 40.778558516029946], [-73.75427379768243, 40.77859495944767], [-73.75426250967827, 40.778632963288175], [-73.754248170366, 40.77867296925754], [-73.75423069482916, 40.77871529279238], [-73.75412244438803, 40.77897174125878], [-73.75411171878478, 40.77902456898705], [-73.75413143029364, 40.77914672865191], [-73.75412964083071, 40.77938414323761], [-73.75402624009541, 40.779603879394756], [-73.75393275467012, 40.77976261058041], [-73.75390659784934, 40.77977871026392], [-73.75353040031737, 40.78023400238808], [-73.75300812549725, 40.780866075188506], [-73.75299839296423, 40.781289489522116], [-73.75274786895811, 40.7815180842781], [-73.75245962687079, 40.7817810905684], [-73.75080593606877, 40.782893378833954], [-73.75029101900083, 40.7824829202005], [-73.74706767699601, 40.780545532125686], [-73.7452105881078, 40.7793265500935], [-73.74412314584049, 40.77863865777706]]]}}, {\"id\": \"64\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 65, \"Shape_Leng\": 0.0446070683658, \"Shape_Area\": 8.1803882541e-05, \"zone\": \"Downtown Brooklyn/MetroTech\", \"LocationID\": 65, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98712491499988, 40.70063447999997], [-73.98705765100006, 40.70063138099992], [-73.9869230459999, 40.70063356299989], [-73.98679382899994, 40.7006332229999], [-73.986798143, 40.700405223999915], [-73.98679868199991, 40.70019229799991], [-73.98680917799996, 40.70007095599988], [-73.98681513199986, 40.69993433099993], [-73.98686354799999, 40.69922012199989], [-73.98694749699983, 40.69873467399986], [-73.98696892299995, 40.698503613999954], [-73.98615985099988, 40.698478503999915], [-73.98584009799997, 40.69845759999986], [-73.98571222099984, 40.6984492459999], [-73.98557305399999, 40.69844703999995], [-73.98537892999997, 40.69844056799989], [-73.98496067199994, 40.69844496199991], [-73.98399562199991, 40.698413970999916], [-73.98314891699987, 40.69838165699989], [-73.98305147099988, 40.69837793399991], [-73.98305797599978, 40.698296076999945], [-73.98311793999989, 40.697542042999935], [-73.98227354299986, 40.69750629999991], [-73.98220436299987, 40.69750337199986], [-73.98210986499997, 40.69749936799996], [-73.9820190099999, 40.697496579999886], [-73.98050926199988, 40.69744627499991], [-73.9804096149999, 40.696559343999915], [-73.98040269799995, 40.6965021049999], [-73.98039561699981, 40.69644348999992], [-73.98038723500002, 40.696382911999926], [-73.98037063399991, 40.69626304999989], [-73.9803568289999, 40.69617751699992], [-73.98031019799991, 40.695905994999876], [-73.9802635679999, 40.69563447299992], [-73.98024270299999, 40.69545652899992], [-73.98021990699989, 40.6952871029999], [-73.9801557449999, 40.694810117999964], [-73.98013518099992, 40.694698016999936], [-73.98009475099997, 40.694587003999835], [-73.9800347509999, 40.69448012199993], [-73.97995651799985, 40.69438029499986], [-73.97986236999992, 40.694290091999946], [-73.97975539199989, 40.69421147699993], [-73.97961647699987, 40.694115582999935], [-73.97961635799989, 40.6941155019999], [-73.97961626399996, 40.69411542099993], [-73.9794907449999, 40.69400713899989], [-73.97949061399991, 40.69400703099989], [-73.97949049699993, 40.69400690499993], [-73.97938094200002, 40.69388775699988], [-73.9792896699999, 40.69375999399987], [-73.97928957399992, 40.69375986699991], [-73.97928950399996, 40.69375974199993], [-73.97921815699976, 40.69362600299992], [-73.97921806199986, 40.69362583099995], [-73.97921799199985, 40.69362565099988], [-73.97916718399999, 40.69348832299992], [-73.98011299200002, 40.6935238809999], [-73.98098333199992, 40.69355661699993], [-73.98147150299985, 40.693574934999965], [-73.98154756999999, 40.69357778999993], [-73.9817531689998, 40.69358338499997], [-73.9822668969999, 40.693604775999866], [-73.98234987399988, 40.69360788799985], [-73.98242892099991, 40.693610852999925], [-73.98251122999989, 40.69361433399992], [-73.98253983699992, 40.693002615999944], [-73.98175888999991, 40.692955172999895], [-73.98179025899985, 40.69202541599988], [-73.98178907299996, 40.691644867999905], [-73.98206215699987, 40.69131209099991], [-73.98213180199994, 40.69122723699989], [-73.98150118599992, 40.68998815499986], [-73.98147568199991, 40.689930943999876], [-73.98157420299987, 40.68993518299988], [-73.98276720099986, 40.68998648099994], [-73.98313467299995, 40.690002276999884], [-73.9835048569999, 40.69000570599997], [-73.98372011099993, 40.69000671899992], [-73.9840703399998, 40.69014307899988], [-73.98443216299995, 40.690292171999936], [-73.98510411999996, 40.69056114499992], [-73.98536391899995, 40.69066661299989], [-73.98633134999982, 40.69105050999989], [-73.98646436299995, 40.69110361099993], [-73.98734787400004, 40.69145766799992], [-73.9882070139999, 40.69180695999995], [-73.98893356299986, 40.69208664099989], [-73.98902944799994, 40.69212386099992], [-73.98910541699992, 40.69215331399992], [-73.98947238599989, 40.69229549399994], [-73.99066626299994, 40.69249717699989], [-73.99097186899995, 40.69254879999995], [-73.99085041499994, 40.6927982509999], [-73.99072895999997, 40.693047700999905], [-73.9905879449999, 40.693391378999905], [-73.99045286499987, 40.693720583999934], [-73.99044474099998, 40.69382825099991], [-73.99066605599978, 40.69456882399993], [-73.99080803800004, 40.69502713399987], [-73.99115539099992, 40.696196475999926], [-73.99118780399986, 40.696287414999894], [-73.99120620999996, 40.69635779799992], [-73.99128413800005, 40.69658820999993], [-73.99136812199991, 40.69701375299993], [-73.99142289799994, 40.69778099299988], [-73.99143347099985, 40.69828185899997], [-73.99142848599986, 40.6984190989999], [-73.99141799499986, 40.69855615899995], [-73.991402007, 40.698692910999945], [-73.99138053899995, 40.6988292269999], [-73.99135360999992, 40.698964980999925], [-73.99132124499982, 40.69910004499998], [-73.99128347399984, 40.699234293999915], [-73.99124033199993, 40.69936760199986], [-73.9910295809998, 40.699847169999956], [-73.99084157899988, 40.700281256999844], [-73.99082272099989, 40.7003547539999], [-73.99081766499997, 40.70046229299989], [-73.99083516199983, 40.7005697619999], [-73.99087453800003, 40.70067311699993], [-73.99093201799984, 40.7007662159999], [-73.99087002799989, 40.70076967799992], [-73.99038407800006, 40.70074858499989], [-73.99025489399995, 40.70074725899985], [-73.99013277299997, 40.7007393779999], [-73.98997709599989, 40.700738283999925], [-73.98972975399985, 40.70073504399989], [-73.98881846999987, 40.70071191899989], [-73.98866705299986, 40.70070499699995], [-73.98774282899993, 40.700662955999924], [-73.98717933299986, 40.700636986999925], [-73.98712491499988, 40.70063447999997]]]}}, {\"id\": \"65\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 66, \"Shape_Leng\": 0.0546334593634, \"Shape_Area\": 0.000108378855948, \"zone\": \"DUMBO/Vinegar Hill\", \"LocationID\": 66, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97906084869959, 40.7059460290057], [-73.97924508744678, 40.705548389137], [-73.97912179285028, 40.70552703403614], [-73.9793276249999, 40.70474627599989], [-73.97960292799985, 40.704176399999916], [-73.97998854, 40.703541057999935], [-73.98022432, 40.703063323999885], [-73.9806490779999, 40.70156087299994], [-73.98074079499993, 40.70127730999996], [-73.98040482499984, 40.700699005999915], [-73.98041744299996, 40.700518152999926], [-73.98044078599986, 40.700183630999895], [-73.98046217199987, 40.69970315799986], [-73.98048077899998, 40.6993727139999], [-73.98048604199988, 40.69927922799994], [-73.98054419199988, 40.69824651199988], [-73.98050926199988, 40.69744627499991], [-73.98201901099986, 40.69749657999987], [-73.98210986499997, 40.69749936799996], [-73.98220436399998, 40.69750337199986], [-73.98227354299996, 40.697506300999926], [-73.983117941, 40.69754204299991], [-73.98305797499987, 40.69829607699988], [-73.98305147099988, 40.69837793399991], [-73.98314891699982, 40.6983816579999], [-73.98399562199991, 40.698413970999916], [-73.98496067199994, 40.69844496199991], [-73.98537892999997, 40.69844056799989], [-73.98557305399999, 40.69844703999995], [-73.98571222099984, 40.6984492459999], [-73.98584009799997, 40.69845759999986], [-73.98615984999995, 40.698478503999915], [-73.98696892299995, 40.698503613999954], [-73.98694749699983, 40.69873467399986], [-73.98686354799999, 40.69922012199989], [-73.98681513199986, 40.69993433099993], [-73.98680917799996, 40.70007095599988], [-73.98679868199991, 40.70019229799991], [-73.98679814199983, 40.70040522399987], [-73.98679382799996, 40.700633222999954], [-73.9869230459999, 40.70063356299989], [-73.98705765100006, 40.70063138099992], [-73.98712491499988, 40.70063447999997], [-73.98717933299986, 40.700636986999925], [-73.98774282899993, 40.700662955999924], [-73.98866705299986, 40.70070499699995], [-73.98881846899985, 40.70071191899997], [-73.98972975399985, 40.70073504399989], [-73.98997709599989, 40.700738283999925], [-73.99013277299997, 40.7007393779999], [-73.99025489399995, 40.70074725899985], [-73.99038407800006, 40.70074858499989], [-73.99087002799989, 40.70076967799992], [-73.99093201799984, 40.7007662159999], [-73.99126294499995, 40.701027101999905], [-73.99193644899997, 40.701556408999856], [-73.99202772899994, 40.70162814199991], [-73.99206681900003, 40.70165885999988], [-73.9921106779999, 40.701693442999925], [-73.99221871100002, 40.70177257799986], [-73.99232767899991, 40.70185536999985], [-73.99261204999985, 40.702061128999915], [-73.99286016199979, 40.70220276699991], [-73.99325388199995, 40.702411970999876], [-73.99380996599997, 40.70267441199997], [-73.99417681200005, 40.702824653999855], [-73.99444009399991, 40.702909269999964], [-73.99470923300002, 40.702993885999916], [-73.99490815999987, 40.7030473409999], [-73.99504592724139, 40.70313183989976], [-73.99544564312069, 40.70327530547179], [-73.99529346700928, 40.70355406589793], [-73.9949820331283, 40.7033817155352], [-73.99480038274874, 40.703291958141975], [-73.99466290343602, 40.703430608056536], [-73.99485765933736, 40.70356613852609], [-73.99499542533941, 40.70365056321164], [-73.99488239902435, 40.703788025837355], [-73.99491043069509, 40.703797315958745], [-73.99492964609256, 40.70377131508382], [-73.99502925916443, 40.703808200868025], [-73.99488204726403, 40.70400865389511], [-73.99483637435264, 40.70398633908107], [-73.99476934191567, 40.70395289495363], [-73.99471424235146, 40.70399668906091], [-73.99477158125163, 40.70404936037442], [-73.99479231460045, 40.704068405633734], [-73.99479231299986, 40.70408889470715], [-73.99477286162535, 40.70410454169924], [-73.99469807470796, 40.704164700797286], [-73.99477614353454, 40.70424586105244], [-73.99470229068396, 40.704292531913545], [-73.99463603408383, 40.70433440342858], [-73.99456978950866, 40.70437626401652], [-73.99386341031877, 40.704556007433], [-73.99380687248414, 40.704539613214074], [-73.9937815542649, 40.70446894454372], [-73.993774572078, 40.70444945740879], [-73.9936933612927, 40.70447612883778], [-73.99368792224479, 40.70447791421154], [-73.99358606975355, 40.704509118218105], [-73.99362643916744, 40.704595244657405], [-73.99350881403605, 40.70462350527397], [-73.99280104228988, 40.704634023927845], [-73.99269135574991, 40.70462984994532], [-73.99251210682041, 40.70469411255762], [-73.99230833787269, 40.7046900017427], [-73.99212703535072, 40.70468964203349], [-73.99212100639093, 40.70439620157788], [-73.99195715431195, 40.704200677853294], [-73.99186159189317, 40.70408664210433], [-73.99141794782705, 40.70400408902072], [-73.99094755350794, 40.704032442139194], [-73.99071473114778, 40.70435137954312], [-73.99054617998553, 40.704416167501705], [-73.9903076849381, 40.704839880601966], [-73.99028496035659, 40.70486642939217], [-73.99022721196006, 40.70493404448709], [-73.99009111393421, 40.7049434632891], [-73.98949358971352, 40.70480903812361], [-73.98934734717507, 40.70472969170266], [-73.98920046436149, 40.704723896988796], [-73.98906989652578, 40.704705914698266], [-73.98898897757786, 40.70469035678166], [-73.9886972330476, 40.704564578776804], [-73.9884074480715, 40.704554500840814], [-73.98830332208532, 40.70454046895503], [-73.98828999111284, 40.70507065950018], [-73.98795676491503, 40.70513421741247], [-73.98726969301138, 40.70517572069416], [-73.98686320660372, 40.70515985694144], [-73.98681551756646, 40.705147146367295], [-73.9867708111665, 40.70512922547279], [-73.98673006395875, 40.70510648571843], [-73.98669416601165, 40.705079423826106], [-73.98666390146528, 40.70504863092951], [-73.98663993140148, 40.705014779659855], [-73.9866227794053, 40.704978609454166], [-73.98652973012089, 40.70497909917376], [-73.98642911608661, 40.70506917993066], [-73.98643429826662, 40.70534860741953], [-73.98474329943133, 40.70554290726692], [-73.9840214979145, 40.70559863478224], [-73.98343806260941, 40.705644094923855], [-73.98342627521598, 40.70557722117468], [-73.98340650315147, 40.70546637083022], [-73.98336819889047, 40.70546106503822], [-73.98326398719114, 40.70548506633028], [-73.98308639129395, 40.705477090851126], [-73.98308986582964, 40.70551154191292], [-73.98283552150646, 40.705555807198955], [-73.98274161892988, 40.70557244126629], [-73.98274508079308, 40.70555270993459], [-73.98274859333847, 40.705532691119664], [-73.98245956886106, 40.705508797112664], [-73.98245898620758, 40.705542414486125], [-73.98237340600002, 40.70554334999991], [-73.98232944175258, 40.705552033831964], [-73.98092510305972, 40.705369255100244], [-73.98090634668807, 40.7059060592102], [-73.98073921101802, 40.70585303034286], [-73.98022034622862, 40.70584764123357], [-73.98023778333459, 40.705760192228695], [-73.97956222047334, 40.70573887396689], [-73.97955873010817, 40.705765374298394], [-73.97933238122667, 40.705760033226255], [-73.97928362082712, 40.70578387584555], [-73.97927661710943, 40.705911077051084], [-73.97921742108453, 40.70590046607558], [-73.97918606171815, 40.70595876205], [-73.97905429658508, 40.70596017019387], [-73.9790576702781, 40.70595288914233], [-73.97906084869959, 40.7059460290057]]]}}, {\"id\": \"66\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 67, \"Shape_Leng\": 0.0997470781551, \"Shape_Area\": 0.000394282272487, \"zone\": \"Dyker Heights\", \"LocationID\": 67, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.0109702639999, 40.63068186899988], [-74.01000340799995, 40.630113088999906], [-74.00922358699998, 40.63063393399994], [-74.0084560119999, 40.631150269999914], [-74.00773968299988, 40.63163008999991], [-74.00702302499997, 40.63210999499996], [-74.00633870699993, 40.63169362599995], [-74.00413727999998, 40.630363965999905], [-74.00368597199991, 40.63009178099986], [-74.00306153999993, 40.6297151749999], [-74.00193011399998, 40.629032766999885], [-74.00222793499994, 40.62875116999988], [-74.00251454799998, 40.62847420999986], [-74.00031369499995, 40.62714458299991], [-73.99811614300002, 40.625815706999944], [-73.99724067099987, 40.62528868699994], [-73.99748403999989, 40.624519845999934], [-73.99771358999998, 40.62368026999996], [-73.99932090299998, 40.62465524699996], [-73.99994665000001, 40.624050621999864], [-74.00052987499996, 40.623489938999946], [-74.00120222399994, 40.62284271199989], [-74.00169380199993, 40.622372426999966], [-74.00227586299998, 40.621814273999874], [-74.00285701599996, 40.621253642999896], [-74.0034355699999, 40.62069540399994], [-74.00401906899995, 40.62013618899991], [-74.00459907, 40.619576852999955], [-74.00522665999992, 40.618973153999875], [-74.00585319899992, 40.61837047699991], [-74.00643394999992, 40.617814716999874], [-74.00701531599991, 40.6172539459999], [-74.00759806899993, 40.61669446099994], [-74.00817687599992, 40.61613318099995], [-74.00876039999996, 40.61557582299988], [-74.00934164499988, 40.615015095999865], [-74.0099237199999, 40.61445664299991], [-74.01050403599997, 40.613896610999866], [-74.01108648099991, 40.61333904399986], [-74.01171201499994, 40.61273502899992], [-74.01348255899993, 40.61103739799988], [-74.01522379399994, 40.6093563839999], [-74.01701910199995, 40.60765441699994], [-74.0162363669999, 40.60723326599993], [-74.01549854999998, 40.606841523999925], [-74.01640553099998, 40.605975924999875], [-74.01671283899998, 40.60568263499987], [-74.0172836679999, 40.60513782499993], [-74.01746711399991, 40.604965104999906], [-74.01750868599993, 40.604850496999866], [-74.01804702599995, 40.604365734999874], [-74.01811904999991, 40.60431522299989], [-74.01817646699998, 40.604252608999936], [-74.01821458599989, 40.60418195799988], [-74.01823145099998, 40.604108472999926], [-74.018660218, 40.60364106799989], [-74.0186913569999, 40.60360792399995], [-74.01909859999992, 40.60373226399985], [-74.01912316599994, 40.60373976999991], [-74.019189715, 40.60376183999993], [-74.01924239499986, 40.60377850099991], [-74.0205358359999, 40.60419641799992], [-74.02129949199998, 40.60442832299987], [-74.02139467199993, 40.604493997999924], [-74.02122508399991, 40.604738061999974], [-74.02078540899996, 40.60567368599989], [-74.02247769499994, 40.60667421599987], [-74.02150602899994, 40.60782209899992], [-74.02430437599996, 40.60945696999991], [-74.02439534399993, 40.60954853399993], [-74.02447207599994, 40.60965008299994], [-74.02453195999996, 40.60975959499989], [-74.02457307699996, 40.6098745279999], [-74.02459441699996, 40.60999204099989], [-74.02459593799989, 40.61010922499989], [-74.0245785549999, 40.610223301999916], [-74.02478376299999, 40.60969962899984], [-74.02497932499995, 40.609741974999906], [-74.02710660599995, 40.61024395599991], [-74.02637609099996, 40.61245060299993], [-74.02653228199998, 40.61242537099991], [-74.02754297299998, 40.61293718299988], [-74.02736880500001, 40.61323451599991], [-74.02651463099991, 40.61467032699992], [-74.02624732399988, 40.61510478099994], [-74.02607013199996, 40.615486666999885], [-74.02574989099996, 40.616058222999904], [-74.02525327199992, 40.616678457999946], [-74.02462461299996, 40.61738902399985], [-74.02387367099986, 40.618259319999865], [-74.02363350499998, 40.61853764799987], [-74.02287598699992, 40.61946614799993], [-74.02226919899991, 40.620109624999934], [-74.02172925499997, 40.6206794519999], [-74.02167353899998, 40.62073479999989], [-74.02117792099988, 40.621252195999865], [-74.02090368299993, 40.62152616599993], [-74.02062330999992, 40.62180625199994], [-74.02008667300001, 40.62242095199986], [-74.01970491599991, 40.6221849649999], [-74.01938702899992, 40.62289181899993], [-74.01907578299992, 40.62361467399988], [-74.0189099719999, 40.62408698799987], [-74.0187518069999, 40.62431751099996], [-74.01784193399993, 40.626590693999944], [-74.01722288099991, 40.62810068899994], [-74.01696955099997, 40.628728785999925], [-74.0169264479999, 40.62883563899991], [-74.01664336499988, 40.62957607899994], [-74.01634322399995, 40.630285595999915], [-74.01604820399999, 40.631032888999876], [-74.01574804399996, 40.63176426899986], [-74.01542409899997, 40.632513138999904], [-74.01514630699995, 40.63322193799993], [-74.01317645999991, 40.63201376599993], [-74.0109702639999, 40.63068186899988]]]}}, {\"id\": \"67\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 68, \"Shape_Leng\": 0.0493373795569, \"Shape_Area\": 0.000111073378655, \"zone\": \"East Chelsea\", \"LocationID\": 68, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00201937799989, 40.74044408899989], [-74.00252381399994, 40.739752643999864], [-74.00310733899988, 40.73999752099994], [-74.00515808199998, 40.74085808099991], [-74.00536591799992, 40.74094529499997], [-74.00697077699994, 40.7416271269999], [-74.0077911609999, 40.74197084499982], [-74.00820401899988, 40.74214751799986], [-74.00789568799996, 40.74257005099984], [-74.00770461499994, 40.74283381299992], [-74.00723774599994, 40.743470231999865], [-74.00696604399992, 40.74384701399987], [-74.00681149999996, 40.74406132399991], [-74.00638477199996, 40.74464590299987], [-74.00595977599997, 40.74523098499991], [-74.00553403999999, 40.74581556899986], [-74.0051071349999, 40.746402772999865], [-74.0046527429999, 40.74702295299989], [-74.004168307, 40.74768929299991], [-74.0039029229999, 40.74805366099985], [-74.00368344599993, 40.74835499999993], [-74.003235611, 40.74897425499992], [-74.00316116799988, 40.749076422999906], [-74.00312806199997, 40.749121859999946], [-74.00278526999996, 40.74959231999985], [-74.002458683, 40.75003580599991], [-74.002431475, 40.75007275199989], [-74.00233168599986, 40.75020825799991], [-74.00188266699993, 40.75082793099988], [-74.00142970699987, 40.75144572099988], [-74.000974548, 40.752054664999896], [-74.00086139300001, 40.75221483899992], [-74.00052667499992, 40.75268862399987], [-74.00001529900001, 40.75339158699995], [-73.99992459199981, 40.75351627999993], [-73.99963228999997, 40.75391808599989], [-73.99914292099987, 40.754584933999865], [-73.99866067199993, 40.755251761999894], [-73.99820870599997, 40.75586865599991], [-73.99775836599989, 40.75648752899987], [-73.99730671499998, 40.757110506999894], [-73.99577488199998, 40.75645451499985], [-73.99511286399995, 40.75617577399984], [-73.99500601599988, 40.756136713999865], [-73.99446824699999, 40.75590886199991], [-73.99163271299996, 40.75470680499996], [-73.99207900099996, 40.75409111599986], [-73.99252855999994, 40.75347391299987], [-73.99297937799993, 40.75285340299989], [-73.99346417699986, 40.75219005499987], [-73.99394825699984, 40.751522221999934], [-73.99437946899984, 40.75093038999993], [-73.99484363799989, 40.75029330899991], [-73.99529821499979, 40.749671682999924], [-73.99574855499995, 40.74905219599985], [-73.99619655499993, 40.74843932799997], [-73.99664704399989, 40.74782272899995], [-73.99709902899994, 40.74720510199991], [-73.99755164599995, 40.746580899999834], [-73.99800822000003, 40.74595130199988], [-73.99848652399989, 40.745294163999894], [-73.99897237399989, 40.74463005499994], [-73.99942208999991, 40.74400957299992], [-73.9998482359998, 40.74342501899993], [-74.0002751799999, 40.74283879799987], [-74.00070275499995, 40.74225361099988], [-74.00112917399991, 40.741668097999884], [-74.00155531499993, 40.741080025999935], [-74.00201937799989, 40.74044408899989]]]}}, {\"id\": \"68\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 69, \"Shape_Leng\": 0.0789896450483, \"Shape_Area\": 0.00019905160381, \"zone\": \"East Concourse/Concourse Village\", \"LocationID\": 69, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91179361999997, 40.8264281559999], [-73.91130974499993, 40.824848557999886], [-73.91192435199989, 40.82504975899994], [-73.91316674499997, 40.82545645499989], [-73.91351191699981, 40.824804943999915], [-73.91475121099991, 40.82520047299993], [-73.91527590199993, 40.82565539499992], [-73.91541108199998, 40.82577390899993], [-73.91545061499994, 40.82574777899985], [-73.91592284999992, 40.8254400559999], [-73.91613011899993, 40.82530572099995], [-73.91618778499992, 40.82526834599989], [-73.91624748099997, 40.82522965499987], [-73.91630388, 40.825193101999915], [-73.9163666169999, 40.82515243999989], [-73.91642451899982, 40.82511491299993], [-73.91661368499985, 40.82499230799991], [-73.91630917499997, 40.82490286699987], [-73.91608782899984, 40.82483785199988], [-73.91677400599986, 40.82438708699987], [-73.9168919579999, 40.8243119259999], [-73.917126938, 40.82416263099988], [-73.91719007399993, 40.824122516999914], [-73.91725010299993, 40.82408437699994], [-73.9173178239999, 40.82404134999992], [-73.9174045909999, 40.82398622199988], [-73.91746772799986, 40.82394610699989], [-73.9175659939999, 40.823883671999944], [-73.91782151699984, 40.82371837099987], [-73.91788272099983, 40.823678776999884], [-73.91833693999992, 40.82338493199989], [-73.91875570599991, 40.8231152179999], [-73.91888836899983, 40.82302977399992], [-73.91929142000006, 40.82277017699986], [-73.91938041799983, 40.82271235199997], [-73.91944303599995, 40.82267166799987], [-73.91980424599993, 40.822436974999896], [-73.91993471699983, 40.82235220199986], [-73.92038676299994, 40.822058483999875], [-73.92048603299985, 40.82197953699992], [-73.92054346999996, 40.821933856999856], [-73.92066539199996, 40.82183689399995], [-73.92114408999996, 40.82156300199989], [-73.92194764099997, 40.82103154499987], [-73.92265261199995, 40.82049431499993], [-73.92402557199983, 40.8192853299999], [-73.92454726099986, 40.81871262399987], [-73.92518486499998, 40.818012668999884], [-73.92538538499986, 40.818065414999865], [-73.92582070699982, 40.81818033699992], [-73.92635859099993, 40.818320691999936], [-73.92724662700006, 40.81854298499994], [-73.92731653099999, 40.81855963399991], [-73.92687781699988, 40.81947887199991], [-73.92662466400002, 40.820026393999946], [-73.92634345699999, 40.82057855199989], [-73.92628127099981, 40.82053716099994], [-73.92610404099997, 40.820745970999944], [-73.92565452999989, 40.82112523299984], [-73.92573567299983, 40.8211742359999], [-73.92537221999999, 40.82160252099991], [-73.92517593199985, 40.821899196999944], [-73.92499654899987, 40.82220270099992], [-73.9248346779998, 40.82251216799988], [-73.92469079500003, 40.82282671599988], [-73.92456530199985, 40.823145454999896], [-73.92445847599993, 40.823467428999955], [-73.92442266399998, 40.823644865999924], [-73.92437413499995, 40.8238206829999], [-73.92431310200004, 40.8239940979999], [-73.92347379199992, 40.82541321999991], [-73.92306184099985, 40.82609561099991], [-73.9229413879998, 40.82629934999992], [-73.9228330809999, 40.82650710299994], [-73.92273725099989, 40.826718249999935], [-73.92267670999986, 40.82684999599992], [-73.92264951599996, 40.826972409999904], [-73.922233014, 40.82772080999991], [-73.92211960699997, 40.82792587599993], [-73.92183738200004, 40.82843621099994], [-73.92174660200004, 40.82860240899991], [-73.9213893, 40.82923701399989], [-73.92117904700004, 40.82962895599993], [-73.9204611219999, 40.830914820999936], [-73.91986089399987, 40.83194722699992], [-73.91949947100001, 40.8324794919999], [-73.91903136700003, 40.83308734899987], [-73.91839346699996, 40.833809411999866], [-73.91778665899993, 40.83442951199992], [-73.91773115000002, 40.83448623099987], [-73.91770299099986, 40.83451500899989], [-73.91767974800003, 40.83453875599988], [-73.9176438849999, 40.83457540699993], [-73.91720735700002, 40.83502151399988], [-73.91667316899982, 40.83556740499994], [-73.91647433399987, 40.835779030999895], [-73.91587067499995, 40.836393593999915], [-73.9153159669999, 40.83694742399996], [-73.91477300600005, 40.8375141009999], [-73.91423754099989, 40.83810879599992], [-73.91383125899985, 40.83863904499986], [-73.91377104999991, 40.838721647999925], [-73.91371372699993, 40.83880893499991], [-73.91313156299994, 40.83981053999988], [-73.912960009, 40.84019622899994], [-73.9127782319999, 40.840691064999945], [-73.91269769199985, 40.840930128999894], [-73.9126225619999, 40.84115715899991], [-73.91246930299995, 40.841620256999924], [-73.91240381699996, 40.84181649699988], [-73.91222180499983, 40.84235659099988], [-73.91197045599988, 40.84310439399992], [-73.91193751499992, 40.843202387999916], [-73.91175379899997, 40.84318035699991], [-73.91096872799983, 40.84313621399987], [-73.91043026399994, 40.8431077299999], [-73.91046008499987, 40.842791642999885], [-73.90958727299986, 40.84275637499993], [-73.90865715699992, 40.84271936999993], [-73.90774413799994, 40.84270215599993], [-73.90684546599987, 40.842667510999966], [-73.90596061799985, 40.84262870399992], [-73.9058672189999, 40.84257998699996], [-73.90514893999989, 40.84254719499995], [-73.9050151499998, 40.842404697999875], [-73.90503176699985, 40.8422629449999], [-73.90515655599995, 40.84104021299991], [-73.90519207999992, 40.840707581999915], [-73.905272123, 40.8404855589999], [-73.90536832399985, 40.84026688999992], [-73.90548026199987, 40.840052591999935], [-73.90560739499988, 40.8398436189999], [-73.90538358099987, 40.8397689829999], [-73.90597940399996, 40.83850519599985], [-73.90625475300006, 40.83806527999994], [-73.90635674699993, 40.837902326999924], [-73.90646169600002, 40.83773464999995], [-73.90652354, 40.83763584299988], [-73.90679878899991, 40.837196073999976], [-73.90688497199997, 40.83705837699988], [-73.906975505, 40.836913727999885], [-73.90713954499994, 40.83665163499991], [-73.90721294199983, 40.83653436399994], [-73.90727112599996, 40.83644139899997], [-73.90733306999991, 40.83634242699987], [-73.90740200499981, 40.836232282999966], [-73.90757269399987, 40.83592860399989], [-73.90762665999985, 40.835832588999935], [-73.9077182169999, 40.835669695999854], [-73.90783293099987, 40.83546559899987], [-73.90797135199996, 40.83521932399989], [-73.90799132699986, 40.8351837829999], [-73.90810870199985, 40.83497494999988], [-73.90823980199985, 40.83474169399996], [-73.90837656, 40.83449837099991], [-73.90852151699984, 40.834240455999904], [-73.90857547999998, 40.834144441999904], [-73.90869131200002, 40.83393834699988], [-73.90882575999996, 40.8336988109999], [-73.90889804099996, 40.833570034999894], [-73.90910597799993, 40.833199565999905], [-73.909177781, 40.83307163899986], [-73.90923681499991, 40.83296646099988], [-73.90928354899982, 40.83288319599993], [-73.9093741529999, 40.832721771999914], [-73.90942257599983, 40.83263549599993], [-73.90946039499991, 40.83256811599989], [-73.9095189569999, 40.83246377699987], [-73.90962130699984, 40.832281421999895], [-73.91049010699992, 40.830737372999884], [-73.90971373799982, 40.83047001399989], [-73.909144659, 40.83028572999996], [-73.908959903, 40.83022623299989], [-73.90948178399991, 40.82959211899987], [-73.91030770399989, 40.828772484999895], [-73.91169430299985, 40.82775871599993], [-73.91179361999997, 40.8264281559999]]]}}, {\"id\": \"69\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 70, \"Shape_Leng\": 0.0638403183367, \"Shape_Area\": 0.000195458476728, \"zone\": \"East Elmhurst\", \"LocationID\": 70, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85912375899987, 40.76127549899994], [-73.85896628099991, 40.761158795999904], [-73.85880387699986, 40.76104606399992], [-73.858636721, 40.76093742299987], [-73.8584649899998, 40.760832988999894], [-73.85828886699981, 40.760732872999895], [-73.85810853999996, 40.7606371819999], [-73.857924202, 40.760546017999935], [-73.85773604799981, 40.760459477999845], [-73.85754427999996, 40.76037765399996], [-73.85734910199996, 40.76030063399988], [-73.856705981, 40.76004386599993], [-73.85655895700003, 40.75998156099994], [-73.85638216199997, 40.7599066399999], [-73.85603980899994, 40.75972578699984], [-73.85576997499993, 40.75953834299993], [-73.85553311699984, 40.75934259099992], [-73.85525140099983, 40.7591475299999], [-73.855067026, 40.758943277999926], [-73.85500920899997, 40.758879225999884], [-73.85463995799994, 40.75846615799992], [-73.8545797199998, 40.7583876199999], [-73.85449593299987, 40.75827863899985], [-73.85462918699993, 40.75830695499996], [-73.85476880299996, 40.758328129999896], [-73.85490872699997, 40.75833644899987], [-73.85505441, 40.758330370999914], [-73.85530200199995, 40.75829284199989], [-73.85540366099997, 40.75834030899989], [-73.85560819899995, 40.758305410999846], [-73.85695186799988, 40.7581331269999], [-73.85775351199999, 40.75802727499996], [-73.85856138999993, 40.75791243699992], [-73.85888932400003, 40.75789947599986], [-73.85951332399988, 40.75787858999995], [-73.85992746199989, 40.75786410199987], [-73.86046162199992, 40.75784373199991], [-73.86099882399996, 40.75782499299988], [-73.8619061719999, 40.757794863999926], [-73.86280704199999, 40.75775898399997], [-73.86371499900002, 40.757725670999875], [-73.86461956599985, 40.75769371599994], [-73.86567436999994, 40.75762958299988], [-73.86656997600001, 40.75753552099992], [-73.86746336500002, 40.757441539999895], [-73.86848374899984, 40.75733493699991], [-73.86950314799998, 40.75722801899992], [-73.87041371399981, 40.75713241199987], [-73.87132395899997, 40.75703633999996], [-73.87221856399995, 40.75694322399988], [-73.87311054799989, 40.756850881999895], [-73.87365721799983, 40.756803485999946], [-73.87420803099994, 40.75673156299993], [-73.87513874300004, 40.756633815999905], [-73.87551080399989, 40.758679635999975], [-73.875806546, 40.76030783599991], [-73.87601371899997, 40.76144373199991], [-73.87549091099996, 40.76135159499989], [-73.87499575099986, 40.761264321999924], [-73.87531552900002, 40.762944974999876], [-73.87532089499993, 40.763008007999886], [-73.87532637000001, 40.76305660499986], [-73.875535095, 40.76423051299991], [-73.87586503899996, 40.76605165399989], [-73.87613975699989, 40.76769550599995], [-73.87623300399999, 40.768085937999885], [-73.87624365599994, 40.76812831899984], [-73.87625062099983, 40.76816607499988], [-73.87638792400003, 40.76899955799995], [-73.87601415599997, 40.769816975999944], [-73.87602197499997, 40.770312000999915], [-73.87600937699986, 40.77048155299986], [-73.87597756999998, 40.77090960899993], [-73.8759736269999, 40.77104276199989], [-73.87598378999998, 40.771176146999935], [-73.87600801500002, 40.7713084929999], [-73.87604778699992, 40.77144359299993], [-73.87608949699984, 40.77153690299987], [-73.87601656500001, 40.77155104399987], [-73.87517573299982, 40.77167671199991], [-73.87475943399997, 40.77171998699991], [-73.87426330499983, 40.771753387999894], [-73.87380524599989, 40.77176797799988], [-73.8732794149999, 40.77176740099993], [-73.87272561699997, 40.7717366429999], [-73.87223577699984, 40.77169565899993], [-73.8717353399999, 40.7716225859999], [-73.87143357299982, 40.77157646699987], [-73.87071447499993, 40.771390689999926], [-73.87016128299994, 40.77120898499992], [-73.86962970499988, 40.77100636599992], [-73.869437198, 40.77092448199991], [-73.8688938269999, 40.770649021999915], [-73.868098735, 40.77017200899994], [-73.86728193799998, 40.76965000299989], [-73.86606875999985, 40.768796181999946], [-73.8655058869999, 40.768345767999946], [-73.86509294099989, 40.76801531499993], [-73.86458486999996, 40.76756668999989], [-73.86404975699998, 40.76706394799991], [-73.86353063400003, 40.76657622399992], [-73.86340214900002, 40.76645551199989], [-73.86321805300003, 40.76625081599989], [-73.86286118599996, 40.765846235999874], [-73.86180888799991, 40.76454470799988], [-73.8610972439999, 40.76366447699992], [-73.85992937999985, 40.76220030099993], [-73.859666663, 40.761901912999896], [-73.85913545299991, 40.76141844099987], [-73.85912375899987, 40.76127549899994]]]}}, {\"id\": \"70\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 71, \"Shape_Leng\": 0.0931694551828, \"Shape_Area\": 0.000382572587364, \"zone\": \"East Flatbush/Farragut\", \"LocationID\": 71, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93177153199984, 40.65527364199989], [-73.93160868699992, 40.653741638999875], [-73.9313911669999, 40.6516896079999], [-73.93038144399998, 40.65173405399985], [-73.93020593200004, 40.65010866699993], [-73.92999915499996, 40.64818106399986], [-73.9298024389999, 40.64632007699988], [-73.92974213599983, 40.645743383999914], [-73.92959986299991, 40.64443271899991], [-73.9293545129999, 40.64213431899986], [-73.92910623099985, 40.63979510599992], [-73.92809818199994, 40.639856615999875], [-73.92761438900001, 40.639887640999845], [-73.92716497399995, 40.639916461999874], [-73.92622007399989, 40.63997609499989], [-73.92594627399988, 40.63999816299988], [-73.92574389399985, 40.64001446599985], [-73.92582711299985, 40.63962626999988], [-73.925834756, 40.63941858799992], [-73.92586068599987, 40.63921107199991], [-73.92590483099994, 40.63900510599991], [-73.92596691799999, 40.63880208899997], [-73.92604638799985, 40.638603341999904], [-73.92606338099995, 40.63852406099986], [-73.9260813459999, 40.63816294599993], [-73.9261268489999, 40.63774352299986], [-73.92616377799996, 40.63757296099988], [-73.9262739919998, 40.63721532799985], [-73.92652490899994, 40.63649272299989], [-73.92668615699995, 40.636257087999915], [-73.92674443099993, 40.636173105999916], [-73.92682085699984, 40.63606066199992], [-73.9269145769998, 40.6359394399999], [-73.92712769599991, 40.63566745599993], [-73.92744822299987, 40.635258409999864], [-73.92766858599995, 40.63524502299991], [-73.92861283799982, 40.63518764599997], [-73.92962453299997, 40.63512131599987], [-73.93055333099983, 40.63506291499989], [-73.9315273509998, 40.63500796799989], [-73.93223554799995, 40.634963338999924], [-73.93249995699995, 40.63494706899993], [-73.93342687599988, 40.634884050999915], [-73.93439440299989, 40.63482680299991], [-73.93535985000001, 40.6347673119999], [-73.93629596800004, 40.63470952599985], [-73.9372670019998, 40.634650624999885], [-73.93706492900004, 40.6327711669999], [-73.93703669099992, 40.63250924999988], [-73.93701746200003, 40.63231097899989], [-73.9374083309999, 40.63228068499994], [-73.93759273599986, 40.63226785699986], [-73.93799799799994, 40.63224680599991], [-73.93892456999993, 40.63219308999986], [-73.93985458399987, 40.63213497499991], [-73.94032359799989, 40.63210526799983], [-73.94078821499988, 40.632075845999886], [-73.94175680799997, 40.63201662699987], [-73.94272877999995, 40.631957293999896], [-73.94365994499996, 40.63189942699991], [-73.94462885499995, 40.63183773499993], [-73.94488042799988, 40.634176953999926], [-73.9451233509999, 40.636448278999865], [-73.94537272499998, 40.638786954999865], [-73.94638150299986, 40.638726249999905], [-73.94715150299994, 40.63867640599994], [-73.94826499699982, 40.63860718999994], [-73.94841343400003, 40.639987463999944], [-73.94744296799982, 40.6400476649999], [-73.94754496799999, 40.64100547599989], [-73.94778994699976, 40.643305735999924], [-73.94685570599987, 40.64336214099992], [-73.94586879199989, 40.64342374399988], [-73.94598813699993, 40.64453379699989], [-73.94607216799986, 40.64531181199988], [-73.94625972599981, 40.647006214999905], [-73.94647882399988, 40.64910075099988], [-73.94667414499985, 40.65094811899995], [-73.94570606199984, 40.65100678399988], [-73.94590468799987, 40.65285464999991], [-73.94606690599983, 40.65441026199991], [-73.94703257599998, 40.65433871399988], [-73.94715896299984, 40.65559378699993], [-73.94507909299995, 40.65572204699989], [-73.94487284399997, 40.655734761999966], [-73.94446083699984, 40.65576016599992], [-73.94335488899993, 40.655828334999896], [-73.94240147899986, 40.655884810999964], [-73.94146260999986, 40.655944248999944], [-73.94053214699989, 40.65600153899992], [-73.93955925399985, 40.656062752999915], [-73.93859348999989, 40.65612237899993], [-73.93765923499997, 40.656183875999886], [-73.93712063899989, 40.65621590299989], [-73.936684695, 40.65624182299995], [-73.936179278, 40.6562739719999], [-73.93571813299981, 40.656303296999894], [-73.93478437399983, 40.65636039499992], [-73.93381012099998, 40.65642299299988], [-73.93284657399983, 40.65647848199988], [-73.931907438, 40.656537652999916], [-73.93177153199984, 40.65527364199989]]]}}, {\"id\": \"71\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 72, \"Shape_Leng\": 0.0952020695166, \"Shape_Area\": 0.000323532461947, \"zone\": \"East Flatbush/Remsen Village\", \"LocationID\": 72, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92021313099984, 40.65997544099987], [-73.92008167099986, 40.659857632999866], [-73.919986348, 40.660115465999894], [-73.91990440499988, 40.66033711699987], [-73.91974399499988, 40.660770972999885], [-73.91966280899982, 40.66140503299994], [-73.9196271919999, 40.661708058999885], [-73.9195947979998, 40.66199567899985], [-73.91945727799987, 40.66188078499988], [-73.91920450999993, 40.661655199999835], [-73.91893607399996, 40.6614160109999], [-73.91860168199995, 40.661118048999846], [-73.91829936299985, 40.66116125699987], [-73.91818493100003, 40.660726208999954], [-73.91741819199994, 40.66005119199989], [-73.91707363499995, 40.65974535399988], [-73.91572803800001, 40.658550127999874], [-73.91556629599982, 40.65839257199986], [-73.91441247700004, 40.65736241899984], [-73.91402878099997, 40.65700370899984], [-73.91377943699983, 40.65678367899993], [-73.91314356299998, 40.65622086399992], [-73.91236458899995, 40.65551356499985], [-73.91199538399984, 40.65556155999991], [-73.91108323599995, 40.65570239299992], [-73.91012255699995, 40.655851770999945], [-73.90977962099994, 40.654466254999925], [-73.90975756199991, 40.654367116999865], [-73.90974279799993, 40.65431111199997], [-73.90972827499986, 40.65425186399989], [-73.9097001379998, 40.65415294699991], [-73.90933499299997, 40.65280859199989], [-73.90855790499995, 40.65209593799989], [-73.90944846399994, 40.65152159899992], [-73.91022992899991, 40.65101403099988], [-73.91110070199989, 40.65044791299992], [-73.91182280200003, 40.64998342299986], [-73.91253589199988, 40.64952147899985], [-73.91325294199991, 40.649061424999935], [-73.91396129299983, 40.64860065799996], [-73.91473216799982, 40.64810117499995], [-73.91550029699984, 40.647604853999916], [-73.91620934099993, 40.64715114199988], [-73.91693683599993, 40.64668353499992], [-73.91730801300002, 40.64644290499994], [-73.91764732399982, 40.646222935999916], [-73.91798513199991, 40.646010879999906], [-73.91830152299998, 40.645812262999925], [-73.91861676799982, 40.64560209799988], [-73.91892329099996, 40.64539775299986], [-73.91925419699997, 40.64517776399986], [-73.91943014999988, 40.64506079199986], [-73.91982077699998, 40.644783831999874], [-73.91995062999999, 40.64470762999989], [-73.91990782599999, 40.64432958099992], [-73.91973736999988, 40.64277814799987], [-73.92025802399995, 40.64236629899986], [-73.92154948099989, 40.64134469199992], [-73.92332654599986, 40.63990021599991], [-73.92335409199994, 40.64017632399992], [-73.92337418899983, 40.640495543999904], [-73.92341500199993, 40.640923605999916], [-73.92356525899983, 40.642499297999905], [-73.92454228399995, 40.64243917699988], [-73.92518618999996, 40.642397290999945], [-73.92538233099987, 40.64238453199987], [-73.92547887299986, 40.64158704199992], [-73.9255828079999, 40.64107003099984], [-73.92562125699988, 40.64081121999991], [-73.92570388299987, 40.6403609499999], [-73.92574389499987, 40.64001446599983], [-73.92594627400003, 40.63999816199989], [-73.92622007399989, 40.63997609499989], [-73.92716497399995, 40.639916461999874], [-73.92761438900001, 40.639887640999845], [-73.92809818199994, 40.639856615999875], [-73.92910623199991, 40.6397951059999], [-73.9293545129999, 40.64213431899986], [-73.92959986299991, 40.64443271899991], [-73.92974213599983, 40.645743383999914], [-73.9298024389999, 40.64632007699988], [-73.92999915499996, 40.64818106399986], [-73.93020593200004, 40.65010866699993], [-73.93038144299994, 40.65173405399986], [-73.9313911669999, 40.6516896079999], [-73.93160868699992, 40.653741638999875], [-73.93177153199984, 40.65527364199989], [-73.93190743899989, 40.656537652999944], [-73.93090354899984, 40.65661035599996], [-73.92989802399997, 40.65668443599994], [-73.92896419499985, 40.6567426129999], [-73.92799465199988, 40.65680210099988], [-73.92817215400002, 40.658477344999945], [-73.92834017599988, 40.660060035999884], [-73.92819257899993, 40.660177173999905], [-73.92725505399994, 40.65933756399987], [-73.92646424299983, 40.659814361999956], [-73.92828249199995, 40.6614470669999], [-73.92854068399994, 40.66168068599989], [-73.93068465499985, 40.66362047699992], [-73.92987364799986, 40.66399077299992], [-73.9294035289999, 40.66419984599983], [-73.92916752199997, 40.664299531999895], [-73.92905235499991, 40.66435100699988], [-73.92872204599985, 40.66449556899988], [-73.928228724, 40.66470986499991], [-73.92771188899987, 40.66493432799991], [-73.92739349999984, 40.66507471699986], [-73.92637970499992, 40.6655148919999], [-73.9245494589998, 40.66386259199993], [-73.92272843000003, 40.66223123199989], [-73.9209090419998, 40.66059911499987], [-73.92021313099984, 40.65997544099987]]]}}, {\"id\": \"72\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 73, \"Shape_Leng\": 0.0853020209129, \"Shape_Area\": 0.000291799754395, \"zone\": \"East Flushing\", \"LocationID\": 73, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79566545000003, 40.75023619999991], [-73.79644617199999, 40.74985921699992], [-73.79729524799988, 40.74943505299992], [-73.79704716099987, 40.7493323179999], [-73.796581036, 40.74897739799995], [-73.79598269399985, 40.74853189299996], [-73.79472264899997, 40.74759932999989], [-73.7946756879998, 40.74721192799994], [-73.79826235399989, 40.746965655999894], [-73.79921673599986, 40.74685628199987], [-73.80017472200002, 40.746755577999906], [-73.80327809299982, 40.746439510999934], [-73.8033521419999, 40.74643196499987], [-73.80302898599999, 40.747194800999935], [-73.80294034199993, 40.747408916999916], [-73.80286300399993, 40.74764296999995], [-73.80280481399987, 40.74788116399995], [-73.80276629299996, 40.74812214499988], [-73.80274770699985, 40.74836447599992], [-73.802749109, 40.74860675099995], [-73.80277033699984, 40.748847537999936], [-73.80280784099993, 40.74906666099989], [-73.80287666499987, 40.749765148999856], [-73.80295670699991, 40.74977052699987], [-73.80353070499987, 40.7498254129999], [-73.80446953099978, 40.74989927499992], [-73.80539953699991, 40.7499727289999], [-73.80633253899988, 40.750047665999936], [-73.8072707609999, 40.75011897499993], [-73.80750599299991, 40.750136586999936], [-73.80976035399992, 40.748779221999904], [-73.81161754599998, 40.74766484299989], [-73.81235411899996, 40.74722328299991], [-73.81316297299992, 40.74673791899988], [-73.81341871899997, 40.746584445999915], [-73.81416172099998, 40.74613857999986], [-73.81477152699999, 40.74668237099996], [-73.8153731739999, 40.747218337999946], [-73.81598751399984, 40.74776993599989], [-73.81654271199997, 40.748260041999856], [-73.81710925599998, 40.74876054399992], [-73.8176865099999, 40.749288211999925], [-73.81779541099995, 40.749392014999906], [-73.81826477299987, 40.749826989999875], [-73.81883698299988, 40.75035215799989], [-73.81911566899988, 40.750621072999856], [-73.81941130799996, 40.75088775499989], [-73.81998860999985, 40.751428620999896], [-73.820849288, 40.75224154499988], [-73.81977819399994, 40.752936894999884], [-73.819092321, 40.7533848199999], [-73.8184078289999, 40.75383042199998], [-73.82030067699999, 40.75551497999992], [-73.81961754399985, 40.75596557899992], [-73.81926771999996, 40.75619984799989], [-73.818936954, 40.75640957799994], [-73.81789245699991, 40.75708633799991], [-73.81856914499988, 40.7575188489999], [-73.819232864, 40.75794681199992], [-73.8200450059999, 40.75848322799985], [-73.82063593299985, 40.75887226799989], [-73.8172882139999, 40.75977554699992], [-73.8150873259998, 40.76037161099985], [-73.81399882899986, 40.760665033999906], [-73.8131322989999, 40.76089834899995], [-73.81226778099978, 40.761130021999854], [-73.81104963199985, 40.761417020999964], [-73.81044348399986, 40.761377438999894], [-73.80951372599986, 40.761299910999874], [-73.80859484099997, 40.76113246199993], [-73.80768023699983, 40.76095995899987], [-73.80676048599993, 40.7607877729999], [-73.80584005399999, 40.760615390999874], [-73.804916685, 40.760440411999895], [-73.80399971299993, 40.76027048799994], [-73.80307328899983, 40.76012624799988], [-73.80215291900004, 40.75998721799985], [-73.80125017499996, 40.75982823699995], [-73.80030038999979, 40.759702298999926], [-73.79940088899988, 40.759397218999936], [-73.798501602, 40.75907103599994], [-73.79760259799993, 40.758744178999905], [-73.79670467899993, 40.758417711999876], [-73.79621926899995, 40.75824167999989], [-73.79580437699985, 40.758091216999915], [-73.79493246199993, 40.75779802999994], [-73.79474720599988, 40.755628187999925], [-73.79457726999996, 40.7535630109999], [-73.794627692, 40.75270267499999], [-73.79465583899984, 40.75198814499991], [-73.79468324699988, 40.75127504499995], [-73.79471901099991, 40.75064761699991], [-73.79566545000003, 40.75023619999991]]]}}, {\"id\": \"73\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 74, \"Shape_Leng\": 0.11029093625, \"Shape_Area\": 0.000295038418204, \"zone\": \"East Harlem North\", \"LocationID\": 74, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.93380589881859, 40.81651249324708], [-73.93382900884613, 40.81558536018751], [-73.93394608825352, 40.815587624739145], [-73.93394609933729, 40.81557645756461], [-73.93394170146216, 40.81556529841819], [-73.93394905857512, 40.8155563695585], [-73.93395054151384, 40.81554298000987], [-73.93394173470415, 40.81553180845374], [-73.93394175021196, 40.81551618496343], [-73.93395205657892, 40.81549721872188], [-73.93395353730935, 40.81548605238904], [-73.93393445709587, 40.81547153450898], [-73.93394623227614, 40.81544251797633], [-73.93393892276251, 40.81541572428048], [-73.93394921255528, 40.81540122322664], [-73.93394628434712, 40.815390054870534], [-73.93395659090885, 40.81537220417373], [-73.93394853112638, 40.81534820251395], [-73.93394855439136, 40.81532476149812], [-73.9339485843048, 40.815294621684686], [-73.93395302092101, 40.81526672687829], [-73.93395452158198, 40.81523547202671], [-73.93394426378258, 40.815205325816855], [-73.93394870283136, 40.815175197910264], [-73.93394725313905, 40.815155097995664], [-73.93396344530967, 40.81512944302358], [-73.93396788302533, 40.8151004229544], [-73.9339635051222, 40.81506916406496], [-73.93396941358041, 40.81503902711455], [-73.93398565957851, 40.81494751042225], [-73.93398936930922, 40.814912357030906], [-73.93398203685818, 40.814895612781136], [-73.93398352753833, 40.81487439749633], [-73.93397473519484, 40.81484872821333], [-73.93398210661385, 40.81482529307837], [-73.93397332647235, 40.814787331860224], [-73.93399097511455, 40.81477506811652], [-73.9339777641614, 40.81475831111913], [-73.93398954202252, 40.81473822765621], [-73.93399543936422, 40.81471925787887], [-73.93398075656609, 40.81470474252187], [-73.93398224833994, 40.81468242057032], [-73.93398523963945, 40.814629959140966], [-73.9339764506598, 40.81460093082527], [-73.93396620950318, 40.814565211251484], [-73.93399121978341, 40.81452838643417], [-73.93401476726973, 40.814483752624774], [-73.93385652638806, 40.81448147441923], [-73.9338623097275, 40.814249296823014], [-73.93386383534269, 40.81418806900028], [-73.93386510261887, 40.814153157955616], [-73.93386927846709, 40.814101174816585], [-73.93387232548957, 40.814063264260696], [-73.933929846002, 40.813347690633734], [-73.93398178986803, 40.812814635545315], [-73.93400770204929, 40.81260849869726], [-73.93403843917875, 40.8123408800709], [-73.93409306691906, 40.81196840253071], [-73.93413711416075, 40.81171237092448], [-73.93419558749154, 40.811393177692835], [-73.93428227801395, 40.81082305050867], [-73.93430417398088, 40.810481665612656], [-73.93431440507302, 40.81032226491122], [-73.93434080971474, 40.80992758274956], [-73.93433864584743, 40.8095622138313], [-73.93421391016982, 40.809059173792676], [-73.93419639195767, 40.809031851582375], [-73.9341353262405, 40.80873639051419], [-73.93412073484292, 40.80865645197743], [-73.93356119311476, 40.80760067738665], [-73.93332852361837, 40.80731894646299], [-73.93325708526282, 40.80723665117304], [-73.93318395639754, 40.80715619829966], [-73.93317413539488, 40.80714539094985], [-73.93308471749718, 40.807026914702085], [-73.93293288329282, 40.806871022133876], [-73.93263088267565, 40.806552933906765], [-73.932390171191, 40.806299141471094], [-73.93154417434337, 40.805397783326654], [-73.9314959803404, 40.805351225035174], [-73.93147540101658, 40.805328196076914], [-73.93146693969662, 40.80530609218759], [-73.93144877545446, 40.805297790588014], [-73.93143182674825, 40.80528212682836], [-73.93140883156653, 40.80524712392972], [-73.93139067303517, 40.80523330439408], [-73.93130474200962, 40.80513197562958], [-73.93126904139163, 40.805090979973535], [-73.93115889451225, 40.80496753829942], [-73.93112621780075, 40.804923329268384], [-73.93111339209408, 40.80490502354579], [-73.93109717576777, 40.80488187751258], [-73.93107297157022, 40.804847791690065], [-73.93103304094329, 40.80479528916923], [-73.93102699955034, 40.80478331306253], [-73.93097556926416, 40.80471653438344], [-73.93097435882132, 40.804706407289295], [-73.93095741258308, 40.804687979468646], [-73.93093322065235, 40.804653894125565], [-73.9309082463483, 40.80462041055983], [-73.93082673180044, 40.80451111776966], [-73.93080494630988, 40.804475197603566], [-73.93071479334765, 40.804356834780585], [-73.93070875470728, 40.80434209583947], [-73.93069422131265, 40.80432735999997], [-73.93068575499123, 40.80431170072459], [-73.93063735551432, 40.8042472209788], [-73.93063009780597, 40.80423156175632], [-73.93060831208301, 40.80420853141013], [-73.93057080064615, 40.80415418355387], [-73.93057080892787, 40.804144975559375], [-73.93054055322638, 40.8041090497368], [-73.93044072678737, 40.80397594555043], [-73.93040321487011, 40.803922524216716], [-73.93035118741409, 40.80384975152175], [-73.93033061473649, 40.803821194039045], [-73.93030641012538, 40.803788035633886], [-73.93027617112581, 40.80373645606578], [-73.9302592352763, 40.803707901774544], [-73.93018908730323, 40.803585400031146], [-73.93018061288154, 40.80357619510721], [-73.93005719833698, 40.80336412151401], [-73.93001261302975, 40.80328749842868], [-73.92996159942555, 40.803183595030795], [-73.92996684404639, 40.80317620551488], [-73.93003768593684, 40.8030762926764], [-73.92993459708444, 40.80290522425536], [-73.92991008609907, 40.80286455074737], [-73.92988948082964, 40.802830357459], [-73.92978668000165, 40.802660748717486], [-73.92969251214573, 40.80244028594792], [-73.9295769383949, 40.80218002521572], [-73.92952043928469, 40.802053479615516], [-73.92945366333494, 40.80190736883596], [-73.92943654118756, 40.801861711857406], [-73.92935435998545, 40.80167125043787], [-73.92933126971816, 40.80159885399085], [-73.92930220396845, 40.801495801348956], [-73.92927487688024, 40.801373190232695], [-73.92920817595964, 40.80114752061126], [-73.92911838814219, 40.8011130032263], [-73.92903490240305, 40.80108090385195], [-73.92902671406671, 40.80096986031424], [-73.92902123584639, 40.80089557256171], [-73.92901644642038, 40.80083071554217], [-73.92900788120616, 40.80077234912531], [-73.92899994866313, 40.800710097692665], [-73.92898618762995, 40.80060212311935], [-73.92895348657567, 40.80030936848467], [-73.92893763668584, 40.800170843777614], [-73.92890152344123, 40.79985521516526], [-73.92891942515779, 40.799458186491485], [-73.92894396946947, 40.79891158893545], [-73.92897121002336, 40.798258860579715], [-73.9290364017962, 40.79676259410132], [-73.92927936741167, 40.795853139520645], [-73.92930773275864, 40.79581025745309], [-73.92953058056904, 40.79550779834928], [-73.92968294251123, 40.79529623376317], [-73.92974594691611, 40.79520874823406], [-73.92987485778272, 40.795035156225694], [-73.92984244753445, 40.79501503785587], [-73.92985144395278, 40.79500140093118], [-73.92980955349977, 40.79498316629674], [-73.9298325404253, 40.7949490611408], [-73.9298874065193, 40.7949672926039], [-73.92990139788401, 40.79494682436246], [-73.9299163612059, 40.79494911213701], [-73.92993735334552, 40.7949187862844], [-73.92992039501047, 40.79490967283261], [-73.93006881035544, 40.79470082860543], [-73.9301077102813, 40.79471299106365], [-73.93019664392844, 40.794593224854886], [-73.93041980479966, 40.79447087234853], [-73.93111408045347, 40.794096501343894], [-73.93142758557968, 40.79392798838291], [-73.93165821105735, 40.79380174043914], [-73.93179292520253, 40.793727962293545], [-73.93187500171737, 40.7936819218437], [-73.93201001457182, 40.793605486169795], [-73.93207886021688, 40.79356650933645], [-73.93235128869077, 40.7934138451512], [-73.93250920949691, 40.79333190154454], [-73.9325385265489, 40.79331668213541], [-73.93267239130893, 40.79324823277293], [-73.93287712655086, 40.793130251367806], [-73.93310683207811, 40.79299532820945], [-73.93324587394584, 40.792914296531684], [-73.93327130499016, 40.79289947490208], [-73.93348475349612, 40.792774284831985], [-73.93372762154044, 40.79260172681524], [-73.93388747605493, 40.792489813861515], [-73.93406949343559, 40.79235873425466], [-73.93427161550181, 40.79221786711484], [-73.93433968650805, 40.79216855838269], [-73.93450968573121, 40.79205197655582], [-73.9347379410771, 40.79190156964996], [-73.93505300002766, 40.79168720295447], [-73.93508265899982, 40.79171004399993], [-73.93513822199999, 40.791752834999876], [-73.93519360499981, 40.79179799899988], [-73.93524060799987, 40.79183387499989], [-73.93529038099987, 40.791873136999875], [-73.93542842699999, 40.791996879999886], [-73.93555312799995, 40.79184222599987], [-73.93679209299992, 40.792403928999924], [-73.93773612099982, 40.792815803999936], [-73.93819346899994, 40.79218819499989], [-73.93864883299987, 40.79155775499987], [-73.94103496899987, 40.79254666899993], [-73.94327251199996, 40.79348810299993], [-73.94281682199988, 40.79413407699997], [-73.942346766, 40.79476079799989], [-73.94187604699994, 40.79540586899984], [-73.94232710899988, 40.795593102999895], [-73.9424792319999, 40.79565624799992], [-73.94275347999998, 40.79577008499989], [-73.94287776099993, 40.795821670999935], [-73.94348323999992, 40.796072993999935], [-73.94415121699987, 40.79635931699993], [-73.94421313799992, 40.79638585799993], [-73.9449594289999, 40.79670574099988], [-73.94509614599993, 40.79676348299992], [-73.94522698899986, 40.7968201329999], [-73.94556506799988, 40.79696479499997], [-73.9456668469999, 40.79700834399995], [-73.94670030999991, 40.797450543999894], [-73.94722520899997, 40.797671685999944], [-73.94831125500004, 40.79812921799992], [-73.9482064149998, 40.79827285099987], [-73.94815384299984, 40.79834487599987], [-73.94810108999994, 40.798417147999885], [-73.94805130199998, 40.79848535999987], [-73.94784679099992, 40.798765542999945], [-73.94781663099997, 40.79880686299992], [-73.94775716499986, 40.798888331999855], [-73.94771056, 40.79895218099991], [-73.94741996199991, 40.79935029499992], [-73.94737599799991, 40.799410523999896], [-73.94732771499984, 40.79947667099991], [-73.94728018299999, 40.79954178899988], [-73.94697366799987, 40.79996170299987], [-73.94646133899998, 40.800665304999875], [-73.9459678969998, 40.801342394999935], [-73.94550863900001, 40.8019702309999], [-73.94505128000004, 40.80259859899995], [-73.94459751299998, 40.803228144999906], [-73.94613131, 40.80387735799992], [-73.9456926129999, 40.804514948999895], [-73.94523965199991, 40.80513601399989], [-73.94478208199993, 40.805761255999904], [-73.94430194599991, 40.806390824999916], [-73.94275367199994, 40.80574401299988], [-73.9422631119999, 40.80641573799991], [-73.94177140199992, 40.807088999999955], [-73.94015793699995, 40.8064090899999], [-73.93869430499996, 40.80579102899988], [-73.93854407899988, 40.80572965299991], [-73.9380866689998, 40.80635301199987], [-73.937634394, 40.8069746069999], [-73.93718011799989, 40.807598999999904], [-73.93671827500005, 40.80822047899987], [-73.93625935999997, 40.80885560799991], [-73.93586263399992, 40.80943020299987], [-73.93580780199986, 40.80949763799987], [-73.93740733700004, 40.810175610999856], [-73.93901753299998, 40.81085655699991], [-73.93874394499981, 40.8112308209999], [-73.93863753299989, 40.8113763889999], [-73.93838748499988, 40.811718443999894], [-73.93830924999986, 40.81182546399991], [-73.93820642699986, 40.81196760499992], [-73.9381355239999, 40.81206561899994], [-73.93763499299979, 40.81274779999994], [-73.93714398399993, 40.81342104099985], [-73.9366881329999, 40.81404324299994], [-73.936209578, 40.81467384999995], [-73.93613554399998, 40.814788126999915], [-73.93575548199986, 40.81532558399993], [-73.93487343800004, 40.81652784699992], [-73.93463790299994, 40.81690804799997], [-73.93445400100002, 40.81718852899994], [-73.934235195, 40.81755488099992], [-73.93412816799999, 40.81773396299989], [-73.93407463999996, 40.81782651399992], [-73.93408481999982, 40.8179308619999], [-73.934235761, 40.81864931899987], [-73.93429487799989, 40.81890498699991], [-73.934381129, 40.819328075999906], [-73.93439036800001, 40.819587434999924], [-73.93440039999986, 40.81966778299989], [-73.93431276899999, 40.81962986599991], [-73.9341167129999, 40.819547052999965], [-73.93383089287109, 40.819521063761556], [-73.9338294812938, 40.81947330955996], [-73.93382777421705, 40.819415494918886], [-73.93382614137711, 40.819356429750336], [-73.93380019473052, 40.818349758874035], [-73.93379334625767, 40.81777660857971], [-73.93378997399186, 40.817151373701705], [-73.93380589881859, 40.81651249324708]]], [[[-73.92688606022001, 40.800840648368855], [-73.92650954883642, 40.80065703126224], [-73.9265524180424, 40.80060482504614], [-73.92684528429592, 40.80073964671281], [-73.92683162740748, 40.80076552901467], [-73.92686908679453, 40.80078367679929], [-73.92699531303118, 40.80063617610429], [-73.92704640297914, 40.800651743031324], [-73.92688606022001, 40.800840648368855]]], [[[-73.92666565490181, 40.80018271401784], [-73.92669572614771, 40.80014697707624], [-73.9272477078927, 40.800392957368906], [-73.92711464230626, 40.80056375622957], [-73.9270669662848, 40.800543012125786], [-73.92716933022471, 40.80040585369612], [-73.92666565490181, 40.80018271401784]]], [[[-73.92672091168967, 40.800414210943586], [-73.92653846445016, 40.80033389815394], [-73.92655351676811, 40.80031517296066], [-73.92674263892977, 40.800403498972756], [-73.92673683953086, 40.80041147148191], [-73.92672526216785, 40.800407339325346], [-73.92672091168967, 40.800414210943586]]], [[[-73.92718514789182, 40.797349896890104], [-73.9273548544178, 40.797316580786315], [-73.92734692252739, 40.79729390212041], [-73.92767122620857, 40.79721701426623], [-73.92770694127064, 40.7972986638821], [-73.92738263624496, 40.797377063580456], [-73.92736675660653, 40.79734682034475], [-73.9272029978516, 40.797386747899886], [-73.92718514789182, 40.797349896890104]]]]}}, {\"id\": \"74\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 75, \"Shape_Leng\": 0.0876638997685, \"Shape_Area\": 0.00024056392715, \"zone\": \"East Harlem South\", \"LocationID\": 75, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94103496899987, 40.79254666899993], [-73.93864883299987, 40.79155775499987], [-73.93819346899994, 40.79218819499989], [-73.93773612099982, 40.792815803999936], [-73.93679209299991, 40.792403927999956], [-73.93555312799995, 40.79184222599987], [-73.93542842699999, 40.791996879999886], [-73.93529038199985, 40.79187313699991], [-73.93524060899985, 40.791833874999874], [-73.93519360499981, 40.79179799899988], [-73.93513822199999, 40.791752834999876], [-73.93508265899982, 40.79171004399993], [-73.93505300002766, 40.79168720295447], [-73.93510685331493, 40.79164962938518], [-73.93517135169378, 40.791605207031644], [-73.93526809778392, 40.79152802580044], [-73.93526500788931, 40.79152576348113], [-73.93519039296176, 40.791471160129014], [-73.93546762088405, 40.79126365205755], [-73.93553896695929, 40.79131869365975], [-73.93583106788897, 40.79109470487024], [-73.93602391823781, 40.79082912812599], [-73.93632010870607, 40.79042595764855], [-73.93669302210014, 40.789913049159075], [-73.9370616874706, 40.78941359021347], [-73.93705304578876, 40.789406416757465], [-73.93708104382307, 40.78936639571], [-73.93707246262223, 40.78934584188616], [-73.93701479686747, 40.78932128118494], [-73.93614073314119, 40.78894907341048], [-73.93612836235276, 40.788938530059575], [-73.93612689279682, 40.78892686798536], [-73.93621859869943, 40.78880049757431], [-73.93622751295341, 40.78879561294985], [-73.93624303325042, 40.78879577415203], [-73.93714965461099, 40.78916844905941], [-73.93722890167288, 40.78899630803707], [-73.93739788609298, 40.78862171984343], [-73.93741675677381, 40.788579762018934], [-73.93754989162562, 40.7882837445288], [-73.93763482912277, 40.788094244814715], [-73.93768669385378, 40.787973560579], [-73.9376956302226, 40.78795729352046], [-73.93774122176191, 40.78785390435085], [-73.937773632551, 40.787783358062626], [-73.9378172269292, 40.787688468680855], [-73.93794768976979, 40.78747729942922], [-73.93794590851209, 40.78746915925781], [-73.93795930775185, 40.787456279929934], [-73.93797181801379, 40.787431873382516], [-73.93811570089733, 40.787184762756446], [-73.93813356192969, 40.787165790398426], [-73.9381942962531, 40.78709733029969], [-73.93824519951416, 40.78704106810817], [-73.93853937331639, 40.78671077700018], [-73.93926642644561, 40.78588875256448], [-73.93982619696462, 40.785256552415454], [-73.93983453997872, 40.7852539728617], [-73.93984282390869, 40.78525141086653], [-73.93985267603377, 40.78524393165018], [-73.93988045560609, 40.78523237601269], [-73.93988852061734, 40.785223527347014], [-73.9399100328247, 40.78521196675792], [-73.93992974399465, 40.78520176553124], [-73.93995483679325, 40.7851895227944], [-73.93998978019029, 40.7851704772902], [-73.94001308386723, 40.78515687380644], [-73.94002473252543, 40.78515347642582], [-73.94004982707635, 40.785138513728796], [-73.94006415729082, 40.78513102962013], [-73.94011434323865, 40.78510382464465], [-73.94014947276271, 40.78508725537949], [-73.94018512931001, 40.7850662025632], [-73.94020230181276, 40.78505918714035], [-73.94022475284191, 40.785045150788385], [-73.94023795768307, 40.785037134684046], [-73.94026700995506, 40.785022101453436], [-73.94036209351322, 40.784971984082354], [-73.94045320748533, 40.78492287258088], [-73.94053377198209, 40.78487877160597], [-73.94055093807705, 40.784866742096156], [-73.94058130200887, 40.784853708709214], [-73.94059054968368, 40.78484668914218], [-73.9406235705765, 40.78482965041163], [-73.94067374402164, 40.78480359792665], [-73.94071732676325, 40.78478054106708], [-73.94072128361502, 40.78478054310243], [-73.94074901720532, 40.784767518202216], [-73.94076750825941, 40.78475749512128], [-73.94078868006106, 40.78474744878865], [-73.94083088142574, 40.78472742455638], [-73.94102499731892, 40.78463522248629], [-73.94108573710648, 40.78460916651668], [-73.94111082838812, 40.78459714918312], [-73.94113723613098, 40.78458311400423], [-73.94117288854788, 40.784566076962534], [-73.94144612018933, 40.78443147385159], [-73.94152576810184, 40.78439090096986], [-73.94167454195741, 40.78431383063195], [-73.9418544859221, 40.78422074667744], [-73.94201431753096, 40.78413758489741], [-73.94217918595555, 40.78405136339483], [-73.94231388874256, 40.78398116626903], [-73.9423912852098, 40.78394072618266], [-73.94243451137048, 40.7839178303223], [-73.94246970444752, 40.783896460947055], [-73.94257225524868, 40.78382472415543], [-73.9426044272356, 40.78380183093073], [-73.94267379818272, 40.78375374314713], [-73.94278137533362, 40.78367819009659], [-73.94286783792664, 40.78361865580254], [-73.94293520531122, 40.78357210608961], [-73.94301663136942, 40.78351486545848], [-73.94311516211297, 40.78344617062962], [-73.94316141910545, 40.78341182999217], [-73.94320565536934, 40.783369842987604], [-73.94323985739757, 40.78332785169434], [-73.94327003035636, 40.78328815517934], [-73.94331027382039, 40.783233180946446], [-73.94334749366392, 40.783182032998575], [-73.94339679077277, 40.783109502846365], [-73.94344105526409, 40.78304918212819], [-73.94355143839495, 40.78290350798582], [-73.94354420706223, 40.78288052416259], [-73.94360760400002, 40.782865369999904], [-73.9437600529999, 40.78282893199988], [-73.94383256699986, 40.78285908899991], [-73.94389324599985, 40.78288772699988], [-73.94395460199989, 40.78291551999991], [-73.94405439399999, 40.782964357999894], [-73.94464666000002, 40.78321787299997], [-73.94472264499986, 40.78324757199987], [-73.94706938199998, 40.78423622699993], [-73.94933170599992, 40.78519312699994], [-73.95033804099997, 40.785617149999936], [-73.95092733999991, 40.78586544699993], [-73.95244615099996, 40.78650248599994], [-73.95255145499999, 40.78654858099986], [-73.95266136699996, 40.78659501999989], [-73.95416383099995, 40.787226860999944], [-73.9557773589999, 40.78791392399995], [-73.95527481199989, 40.78858337999986], [-73.95481723799992, 40.789204743999974], [-73.95439919199988, 40.78977490899994], [-73.95388917099996, 40.790470498999866], [-73.95343832999994, 40.791085367999926], [-73.95298394599986, 40.791721432999964], [-73.95252222700002, 40.792347621999966], [-73.95207222299993, 40.792966160999896], [-73.95160265399991, 40.79360780499995], [-73.95111102399997, 40.794285034999874], [-73.95061371799994, 40.7949655739999], [-73.95015821299991, 40.795587731999895], [-73.94969538299988, 40.79621687199992], [-73.94940516399991, 40.79663170399987], [-73.94922045699984, 40.796909858999896], [-73.94909695599983, 40.79705785599992], [-73.94879165299983, 40.797477127999905], [-73.94831125500004, 40.79812921799992], [-73.94722520899997, 40.797671685999944], [-73.94670030999991, 40.797450543999894], [-73.9456668469999, 40.79700834399995], [-73.94556506799988, 40.79696479499997], [-73.94522698899986, 40.7968201329999], [-73.94509614699999, 40.79676348299994], [-73.9449594289999, 40.79670574099988], [-73.94421313799992, 40.79638585799993], [-73.94415121699987, 40.79635931699993], [-73.94348323999992, 40.796072993999935], [-73.94287776199995, 40.7958216709999], [-73.94275347999998, 40.79577008499989], [-73.9424792319999, 40.79565624799992], [-73.94232710899988, 40.795593102999895], [-73.94187604699994, 40.79540586899984], [-73.942346766, 40.79476079799989], [-73.94281682199988, 40.79413407699997], [-73.94327251199996, 40.79348810299993], [-73.94103496899987, 40.79254666899993]]]}}, {\"id\": \"75\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 76, \"Shape_Leng\": 0.241203016269, \"Shape_Area\": 0.00126660143241, \"zone\": \"East New York\", \"LocationID\": 76, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88063900599983, 40.679176984999856], [-73.88061377799994, 40.679110031999876], [-73.88015110700003, 40.67924206299993], [-73.87974063299995, 40.67936096099988], [-73.87926692499995, 40.679491084999896], [-73.87862924200003, 40.67967653299995], [-73.87838855899999, 40.67973905399991], [-73.877426862, 40.6800144369999], [-73.87688955199985, 40.68016499699994], [-73.87657495299996, 40.68016700199996], [-73.8755670759999, 40.68016946499992], [-73.87531627599998, 40.68011098399988], [-73.87507136399991, 40.68003514599991], [-73.87483538999999, 40.679942569999874], [-73.87461126400001, 40.679834253999964], [-73.8744016199999, 40.67971160899987], [-73.87420866699985, 40.67957633999987], [-73.87404657399993, 40.67942964799988], [-73.87389692799991, 40.67927472199993], [-73.87376075799997, 40.679112496999885], [-73.87363895999985, 40.678944030999894], [-73.87353219499991, 40.67877042099986], [-73.87342767099986, 40.67861884699991], [-73.87331183100002, 40.678471800999894], [-73.87318521799988, 40.678330001999946], [-73.87304845699995, 40.678194106999904], [-73.87257015500002, 40.67786660999989], [-73.87231018000001, 40.67775574699987], [-73.87204099299996, 40.677657418999935], [-73.87176417499987, 40.677572227999875], [-73.87148141800003, 40.6775006639999], [-73.870458215, 40.67722236799989], [-73.86977606799984, 40.67702720899994], [-73.86942312899993, 40.67691222799995], [-73.86838105899999, 40.67653059899985], [-73.86725717699987, 40.676112684999936], [-73.86627349499987, 40.6757640459999], [-73.86521514199991, 40.67538657399996], [-73.86455313199994, 40.675151121999896], [-73.86413333499988, 40.67499725199989], [-73.86310051, 40.67464721299993], [-73.86176409699988, 40.67423226099992], [-73.86106294999989, 40.67434293499984], [-73.86107192499996, 40.67401981399992], [-73.86077512799977, 40.67280562199989], [-73.86038937900003, 40.67126877499985], [-73.85947416499987, 40.671380531999944], [-73.85855480399997, 40.67153293399986], [-73.85763323199998, 40.67165619399985], [-73.85730947699993, 40.67041623299989], [-73.8572848669999, 40.6703219889999], [-73.85726744599997, 40.670255293999965], [-73.85724675799997, 40.670174050999925], [-73.85722693699994, 40.670096211999876], [-73.85684676199999, 40.668602934999925], [-73.85646311899993, 40.66704176299991], [-73.85618809699999, 40.66584655499992], [-73.85610973799986, 40.66550601099989], [-73.85568461199988, 40.66386749199993], [-73.85611710199991, 40.663808401999844], [-73.85659075099987, 40.66374715999991], [-73.85753563299991, 40.66359786299995], [-73.85842950899992, 40.66345335999987], [-73.85811779299999, 40.66203657099987], [-73.8580250429998, 40.66172104999989], [-73.85792219800001, 40.6613188559999], [-73.85761052799985, 40.660108355999924], [-73.85851436399994, 40.65989894199987], [-73.859386918, 40.6597727149999], [-73.8603597819999, 40.65964582899991], [-73.86122244999984, 40.659220936999894], [-73.86174758999981, 40.65893835699988], [-73.86317083299996, 40.65827651199994], [-73.86269778299766, 40.65761830900614], [-73.86300202201295, 40.657488116627256], [-73.86300994210443, 40.65747609303757], [-73.86301532164248, 40.65746328042212], [-73.86301803082985, 40.657449987927706], [-73.86301800429979, 40.65743653627954], [-73.86301524269408, 40.65742325004282], [-73.86300981264718, 40.65741044979137], [-73.86300184517782, 40.6573984443731], [-73.86301297536053, 40.657349024043626], [-73.86300267139957, 40.65733403625386], [-73.86299489115147, 40.65731819000542], [-73.86298975506234, 40.65730173061768], [-73.86299451835025, 40.65728255653711], [-73.86300321589033, 40.6572642008645], [-73.86301562734518, 40.65724712860224], [-73.8630314382944, 40.657231772240266], [-73.86305024819939, 40.657218520799205], [-73.86307158054906, 40.65720770997593], [-73.86307775804119, 40.65721242301864], [-73.86308485453102, 40.65721631374498], [-73.86309268342231, 40.657219279851915], [-73.86310103886122, 40.657221243347905], [-73.86310970114857, 40.65722215260464], [-73.86311844251696, 40.657221983714], [-73.86312343186994, 40.65721921414201], [-73.86312777533718, 40.65721587249529], [-73.86313136108828, 40.65721204481046], [-73.86313409680184, 40.65720782963826], [-73.86313591204231, 40.65720333550579], [-73.86313676007325, 40.65719867812249], [-73.86313661906057, 40.65719397740099], [-73.86313549263544, 40.657189354369905], [-73.86313340979943, 40.657184928057], [-73.86313042417935, 40.65718081242592], [-73.86312661264536, 40.657177113440476], [-73.86312207333215, 40.65717392633775], [-73.86312086315435, 40.6571678659817], [-73.86312096522205, 40.657161736440834], [-73.86312237680909, 40.65715570140494], [-73.86312506021913, 40.65714992204003], [-73.8631289437907, 40.657144552684535], [-73.86313392381267, 40.657139736727196], [-73.86313986729341, 40.65713560277828], [-73.8631466155115, 40.65713226123533], [-73.86315398825552, 40.65712980133423], [-73.86315810974206, 40.65708778410358], [-73.86321868203983, 40.65704187934266], [-73.86321606220991, 40.65703782728763], [-73.86321439496035, 40.6570334928127], [-73.86321372982172, 40.65702900468712], [-73.86321408655384, 40.65702449624463], [-73.86321545455876, 40.657020101422546], [-73.86321779319539, 40.65701595078287], [-73.86322103298691, 40.65701216763333], [-73.8632250776853, 40.65700886436401], [-73.86322980713008, 40.6570061391089], [-73.86323508081834, 40.6570040728301], [-73.8632407420788, 40.6570027269128], [-73.86324662272614, 40.65700214134173], [-73.86326056957371, 40.65698307345786], [-73.86327051713553, 40.65696260378152], [-73.86327623068586, 40.656941215328104], [-73.86324988488909, 40.656900408905685], [-73.86321767415188, 40.656862120798515], [-73.86318000805555, 40.65682683785396], [-73.86313736554554, 40.65679500870645], [-73.86309028884169, 40.656767038074264], [-73.86303937654401, 40.65674328161242], [-73.86300171715261, 40.65665375737019], [-73.86303117341409, 40.656508181683144], [-73.86303820106212, 40.65650375080435], [-73.863044194545, 40.656498512212096], [-73.86304899586064, 40.65649260400802], [-73.86305247843529, 40.65648618194618], [-73.86305455046099, 40.65647941532709], [-73.86305515731418, 40.656472482534596], [-73.86305428299774, 40.65646556633301], [-73.86305195056094, 40.6564588490495], [-73.86304822149259, 40.65645250776703], [-73.86304319409992, 40.65644670965629], [-73.86303700091668, 40.65644160756877], [-73.86302980520972, 40.656437336007194], [-73.8630130787124, 40.65641760960361], [-73.86299992602065, 40.656396371337], [-73.86299057835193, 40.65637399457031], [-73.86299398147607, 40.65636539001103], [-73.86299920962628, 40.65635733382082], [-73.86300611826856, 40.656350048715005], [-73.86301451641084, 40.65634373609222], [-73.86302417188485, 40.65633857046629], [-73.8630348177622, 40.65633469464218], [-73.86304615973482, 40.6563322157678], [-73.86305788425207, 40.65633120237212], [-73.86306966718652, 40.6563316824709], [-73.86312378925449, 40.6563899390293], [-73.86313059624987, 40.65639524283843], [-73.8631384969637, 40.656399578800716], [-73.86314725719119, 40.65640281838299], [-73.8631566172485, 40.656404865552574], [-73.86316629966998, 40.6564056596242], [-73.86317601743468, 40.65640517705863], [-73.86318487953123, 40.65639753532538], [-73.8631920616611, 40.65638893130981], [-73.86319738674325, 40.65637957715094], [-73.86320072348435, 40.65636970348318], [-73.86320198961536, 40.65635955374959], [-73.86320115391976, 40.656349378199856], [-73.86319823700312, 40.65633942771994], [-73.86319331078552, 40.65632994764667], [-73.86318649672684, 40.6563211717182], [-73.86312264260036, 40.65621227417521], [-73.86305132215273, 40.6561060983717], [-73.86297273468902, 40.656002940964], [-73.8628870998149, 40.65590309017312], [-73.86279465682358, 40.65580682498031], [-73.86269566402707, 40.655714414347514], [-73.8626411734316, 40.65566753700771], [-73.86173355547587, 40.655295221774004], [-73.86158297119158, 40.65528829429209], [-73.86143294284753, 40.65527625603836], [-73.86128376936315, 40.65525913099786], [-73.86113574795394, 40.655236953291066], [-73.86098917376947, 40.65520976705256], [-73.86059601899994, 40.65477969699992], [-73.86056716699993, 40.654747451999874], [-73.86052767799987, 40.6547059619999], [-73.86048308199992, 40.65465892899989], [-73.86042178299996, 40.65461220199988], [-73.85975533686633, 40.65410417609493], [-73.85898002989568, 40.65267545791771], [-73.85695870295893, 40.65058737348713], [-73.85701278090167, 40.65054096134554], [-73.85706090518094, 40.65049089031753], [-73.85710265151224, 40.650437602309765], [-73.8571376518816, 40.65038156710168], [-73.85716559774956, 40.650323278690045], [-73.85718624276892, 40.6502632509345], [-73.85720730991866, 40.65023104914557], [-73.85723212115558, 40.65020043363679], [-73.85726047236975, 40.65017165626019], [-73.8575242937435, 40.64999619023134], [-73.85759335427862, 40.649397285892846], [-73.85770542077616, 40.648745971467285], [-73.85783469416334, 40.648302338287635], [-73.85837605251689, 40.64727441531264], [-73.85839397177297, 40.647139412197646], [-73.85840315341014, 40.64700390123735], [-73.85840357526584, 40.64686821031541], [-73.85839523637277, 40.646732667748836], [-73.85837815696057, 40.64659760149385], [-73.8583523784073, 40.6464633383521], [-73.85831796313752, 40.646330203180305], [-73.8582749944715, 40.646198518103844], [-73.85822357642378, 40.64606860173777], [-73.85816383344905, 40.645940768415805], [-73.85809591014267, 40.645815327430086], [-73.85801997088917, 40.64569258228269], [-73.85793619946534, 40.6455728299514], [-73.85784479859338, 40.64545636017166], [-73.85774598945184, 40.6453434547353], [-73.85772560911354, 40.64532159083552], [-73.85763299887402, 40.6451968240604], [-73.85754830556488, 40.645068835220016], [-73.85747172186305, 40.644937915557776], [-73.85740342198902, 40.64480436298478], [-73.85734356130943, 40.64466848140127], [-73.85729227598536, 40.64453058000568], [-73.85724968266203, 40.64439097259035], [-73.85721587820477, 40.6442499768278], [-73.85719093947807, 40.644107913547664], [-73.85717492317225, 40.64396510600649], [-73.85716786567292, 40.643821879152576], [-73.85716978298207, 40.64367855888597], [-73.8574134314329, 40.64359349657502], [-73.85766067130159, 40.64351467655158], [-73.85791122705675, 40.64344218665107], [-73.85816481947492, 40.64337610765482], [-73.85842116595126, 40.64331651319946], [-73.85867998081395, 40.64326346969551], [-73.85894097564288, 40.64321703625277], [-73.85920385959095, 40.6431772646152], [-73.85946833970695, 40.6431441991029], [-73.85973412126268, 40.64311787656272], [-73.86000090808096, 40.64309832632753], [-73.86026840286512, 40.64308557018329], [-73.8605284136282, 40.64307059470142], [-73.86078784559764, 40.64305062748121], [-73.86104653361731, 40.64302568123402], [-73.86130431300528, 40.64299577184074], [-73.86156101965909, 40.64296091834184], [-73.86178755735516, 40.642925904886596], [-73.86201301027748, 40.64288704102003], [-73.86289463224384, 40.64255067968245], [-73.86331138939109, 40.642299129028494], [-73.86379865777398, 40.641799578171785], [-73.86439393961581, 40.64141360621634], [-73.86509362394393, 40.64093967569411], [-73.86531173665126, 40.64075119144387], [-73.86670411973783, 40.6400217634955], [-73.86609210225917, 40.63943387946263], [-73.86598797060661, 40.63950441118262], [-73.86535389969112, 40.638951293675774], [-73.86534275703448, 40.63890203859465], [-73.86536256486231, 40.63883569112215], [-73.86548637000524, 40.638765183481624], [-73.86557348743777, 40.6387695662165], [-73.86563247391335, 40.63878676362291], [-73.86625535025003, 40.63931631406574], [-73.86617093011522, 40.639367600569045], [-73.86681528201245, 40.6399635276668], [-73.86702994403059, 40.639851069021354], [-73.86732291072707, 40.63987198395282], [-73.86867282895084, 40.640925079203356], [-73.8699123933944, 40.64327820299355], [-73.87001708273587, 40.64407137240543], [-73.87061492461066, 40.64485674188673], [-73.87119172949637, 40.64530535631248], [-73.87157174183598, 40.64541496283752], [-73.8719847673393, 40.64544587181563], [-73.87212202857904, 40.645423703864466], [-73.87257043102892, 40.64557561235967], [-73.87337828287791, 40.64671520526748], [-73.87342508856524, 40.6467812305158], [-73.87346558258602, 40.64681663296923], [-73.87352036861594, 40.6468762094049], [-73.87623339658143, 40.650554274824266], [-73.87629452461884, 40.65063714033261], [-73.87921658230903, 40.65459108377298], [-73.87926863785954, 40.65456913073254], [-73.87933211680556, 40.65462201684979], [-73.8796386550231, 40.65449909180424], [-73.87971979034293, 40.65437592977636], [-73.87924124622316, 40.653737186432], [-73.87910837392907, 40.65370623553959], [-73.87676193956291, 40.65056095449518], [-73.87672243382207, 40.65037540309043], [-73.87672192491898, 40.6503232219678], [-73.87395425313315, 40.64644443461837], [-73.87390127736091, 40.64636836786121], [-73.87386034009012, 40.64631440660161], [-73.87382160378151, 40.64625653142808], [-73.87375709027744, 40.64616014330094], [-73.8736728495642, 40.646067979143226], [-73.87360265286914, 40.64596662919093], [-73.87352462463141, 40.64587122906811], [-73.87345352818083, 40.645752814416575], [-73.87342337082691, 40.6456507035949], [-73.8733574021273, 40.645594475914194], [-73.87330678000762, 40.64558267959164], [-73.87318593330937, 40.64544501336156], [-73.87331378312935, 40.64522208871266], [-73.87335638175688, 40.645114714533555], [-73.87339074846517, 40.64500564205859], [-73.8734167680292, 40.64489523711673], [-73.87343435322148, 40.64478387000366], [-73.87344344510424, 40.644671914240746], [-73.87344401322652, 40.64455974532177], [-73.87343605572586, 40.64444773945352], [-73.87341959933369, 40.644336272293806], [-73.8733946992858, 40.644225717692095], [-73.87336143913527, 40.64411644643527], [-73.87331993047317, 40.64400882500423], [-73.87327031255236, 40.6439032143449], [-73.87309815600656, 40.643934684043515], [-73.87297320732566, 40.64383923106529], [-73.87275635704977, 40.6432489646548], [-73.87194880116543, 40.64239408453872], [-73.87189439830122, 40.642248236727454], [-73.87088107173645, 40.64077187075254], [-73.87084503422264, 40.640556620183226], [-73.87061752711526, 40.64038627866385], [-73.87043578304258, 40.64029080737933], [-73.87008544224433, 40.63984418136626], [-73.87007958407006, 40.639211694017035], [-73.86989583386247, 40.63893452891954], [-73.86977753759143, 40.63884414565845], [-73.86952435189357, 40.638645701918655], [-73.86859780095368, 40.638349566578306], [-73.86808857108026, 40.63818561428028], [-73.86799709668125, 40.63811526520958], [-73.8686067398553, 40.63774336259496], [-73.86926691693779, 40.637311481309595], [-73.86960972136559, 40.63708553460871], [-73.86961569339263, 40.63709056489554], [-73.86962489466698, 40.637083191836915], [-73.86969549818932, 40.63714738961505], [-73.8692084611063, 40.637465809261], [-73.86955902490304, 40.6379474843275], [-73.86976333670746, 40.63786528033555], [-73.86997059067167, 40.63778746815808], [-73.870180624162, 40.637714108851334], [-73.87025308570794, 40.63769004523161], [-73.87042969307484, 40.63763400156946], [-73.87060795904935, 40.63758109327911], [-73.87077065591667, 40.6375237119634], [-73.87093022302477, 40.63746143911107], [-73.87108640707612, 40.63739437357764], [-73.87123896014546, 40.63732262182698], [-73.87138764007423, 40.637246297761536], [-73.87153221085434, 40.63716552254151], [-73.8716724430032, 40.63708042439297], [-73.87243460834033, 40.63654851920012], [-73.87255337118395, 40.636493365056346], [-73.872676601456, 40.636444226820885], [-73.87280377480171, 40.63640131357759], [-73.87293435009087, 40.636364807922334], [-73.87306777172168, 40.636334865186626], [-73.87320347198329, 40.6363116127764], [-73.87334087347163, 40.63629514963032], [-73.87347939154502, 40.63628554579879], [-73.87368012187775, 40.636349316203116], [-73.87387767294561, 40.63641860954281], [-73.87407178301434, 40.63649333401509], [-73.87426219490607, 40.636573390621486], [-73.87444865634032, 40.636658673298705], [-73.87463092026667, 40.63674906905968], [-73.87480874519362, 40.63684445814244], [-73.8749818955075, 40.63694471416899], [-73.87515014178462, 40.637049704312986], [-73.87531326109587, 40.63715928947522], [-73.8754710373013, 40.637273324467614], [-73.87562736530136, 40.6374406598268], [-73.87579100112605, 40.63760389050788], [-73.87596175925121, 40.6377628313934], [-73.87613944607081, 40.63791730222954], [-73.87632386011742, 40.63806712783068], [-73.87664550484685, 40.638294638557646], [-73.87696550124826, 40.638523495048375], [-73.87702388943292, 40.63857717609552], [-73.87708844516781, 40.63862658898945], [-73.87715863327793, 40.63867132408132], [-73.87723387188987, 40.63871101050158], [-73.87731353725628, 40.63874531923534], [-73.87739696892777, 40.63877396585033], [-73.87748347522731, 40.63879671285495], [-73.87757233898627, 40.63881337166764], [-73.8776628234893, 40.63882380417986], [-73.87778782070119, 40.63882050261284], [-73.87791286639488, 40.638822468673865], [-73.87803757833375, 40.638829696353355], [-73.87816157529974, 40.63884216355752], [-73.87828447826044, 40.638859832176976], [-73.87835076562644, 40.63887161374519], [-73.87841589406925, 40.638912189353306], [-73.87797341300002, 40.63980711799987], [-73.87752894599991, 40.64071679999995], [-73.8773424829999, 40.641082679999904], [-73.87709627099984, 40.641602954999875], [-73.87683348999987, 40.64211858899987], [-73.87654859999992, 40.64263892799989], [-73.8761993309999, 40.643257172999895], [-73.87617767099991, 40.643289658999905], [-73.87616087400002, 40.64332052799993], [-73.87596309099979, 40.643644483999935], [-73.87545861899982, 40.64442693099989], [-73.87530646399993, 40.64465566899992], [-73.874953578, 40.64514158399989], [-73.8745808109999, 40.64561010599992], [-73.87468212999995, 40.6456544679999], [-73.87610398000004, 40.6462147889999], [-73.87679877399998, 40.64714224999992], [-73.87798475599999, 40.64872529299991], [-73.87843582199996, 40.649327343999914], [-73.87908517399995, 40.65020211699988], [-73.87992598699992, 40.651334759999905], [-73.88141998000002, 40.653347175999926], [-73.88172188099986, 40.653753826999846], [-73.88220052999989, 40.65444157599989], [-73.88221364899994, 40.65451863799996], [-73.88278243099995, 40.65426976799992], [-73.88465215799992, 40.65345165399985], [-73.8855122199999, 40.65309253199994], [-73.88621116599981, 40.65280058399992], [-73.88628975299989, 40.65276890099989], [-73.88637373700001, 40.65273440399987], [-73.88713099000002, 40.65239391599988], [-73.88794902799988, 40.652040973999924], [-73.88876540999999, 40.65169059099987], [-73.88959044299983, 40.651337545999866], [-73.89000589199988, 40.651148696999925], [-73.8900378249999, 40.6511781769999], [-73.89136543599977, 40.65236471899988], [-73.89245737399976, 40.653710938999886], [-73.89354677099988, 40.65520271399988], [-73.89441113999996, 40.6548329489999], [-73.89520540399988, 40.654487929999895], [-73.89546427299999, 40.65437757699991], [-73.89558884499993, 40.65432447199986], [-73.8959444279999, 40.654172887999934], [-73.89667745599995, 40.65385495699996], [-73.89678213299993, 40.65380955599988], [-73.89741344699992, 40.653538319999896], [-73.89761013299987, 40.65345381599992], [-73.89746257800005, 40.653321056999886], [-73.89684143499989, 40.65276219199989], [-73.89676845099996, 40.65269652599995], [-73.896634179, 40.65257571499996], [-73.89654981299998, 40.652499805999916], [-73.89619634499996, 40.652181767999906], [-73.89610981499999, 40.65210391199988], [-73.89586154900002, 40.65188052699987], [-73.89621590699987, 40.65164801899989], [-73.89627957599997, 40.65160624199991], [-73.89655685999999, 40.65142430299996], [-73.89661390699996, 40.65138687099991], [-73.89675272299998, 40.65129578599992], [-73.896807004, 40.65126016999988], [-73.89721054799986, 40.65099537899994], [-73.89727504499993, 40.650953162999876], [-73.897365881, 40.65089370699991], [-73.89742477899996, 40.6508550119999], [-73.89757174800003, 40.65075888699994], [-73.89771516099994, 40.65066508899988], [-73.8977945719999, 40.65061310899986], [-73.89797649499998, 40.650494030999894], [-73.89872808799994, 40.650024606999885], [-73.89946894299986, 40.649541768999924], [-73.90019369799995, 40.64908881799994], [-73.90027087999994, 40.649037770999946], [-73.90056999899996, 40.64884760699989], [-73.90074707899997, 40.6487354449999], [-73.90089120800002, 40.648641709999886], [-73.90165833699982, 40.64814221199989], [-73.90235538599994, 40.64768960199993], [-73.903099438, 40.64719961099992], [-73.90398200599991, 40.64662856799988], [-73.9050958029999, 40.64760685199993], [-73.90579983699993, 40.64826507899992], [-73.90588893999983, 40.64834514399994], [-73.90637218799996, 40.6487793819999], [-73.90754641399995, 40.64983425399987], [-73.90761503399983, 40.649895893999876], [-73.90827570699999, 40.65049857399994], [-73.90849084299998, 40.650586417999875], [-73.90868769500001, 40.65072148099995], [-73.90944846399994, 40.65152159899992], [-73.90855790499995, 40.65209593799989], [-73.90821606899983, 40.65231208699988], [-73.90740035699999, 40.6528439899999], [-73.90780156999993, 40.654438948999925], [-73.90782538299986, 40.65453360399992], [-73.90784346399994, 40.65459152999988], [-73.90692334299995, 40.65473841999989], [-73.906067832, 40.6548638509999], [-73.90556272799998, 40.654937909999866], [-73.90507059799985, 40.655010060999935], [-73.90489261799983, 40.65503619999986], [-73.90446144000003, 40.655099535999895], [-73.90414221400002, 40.65514642099993], [-73.90317858399993, 40.655311633999915], [-73.90298366699997, 40.655380780999906], [-73.90279641599983, 40.655463210999876], [-73.90261911399985, 40.65555811099995], [-73.90245383299998, 40.6556643669999], [-73.90230238299979, 40.65578061099992], [-73.901185599, 40.6565145509999], [-73.90064904799996, 40.65704155799989], [-73.900574915, 40.657099946999885], [-73.90048767199994, 40.65714867899987], [-73.9003948669999, 40.657184204999936], [-73.9002942619999, 40.657208882999925], [-73.90002679299982, 40.65725384799987], [-73.89988986899985, 40.6572745209999], [-73.89968548099993, 40.65729793799992], [-73.89948109299996, 40.65732134299993], [-73.899041804, 40.657371657999846], [-73.89891727499989, 40.65739022499992], [-73.89880698499995, 40.65740666199988], [-73.89796594499987, 40.65753203599994], [-73.89704215399999, 40.65766358899991], [-73.89612587299993, 40.657814663999936], [-73.89613759299986, 40.6578822489999], [-73.8961675609999, 40.657978451999924], [-73.89571848099996, 40.65815995999984], [-73.89491382099995, 40.65852644899994], [-73.8940984499999, 40.65887791999994], [-73.89327894299984, 40.659230173999916], [-73.89246408899987, 40.659580274999925], [-73.89164919999989, 40.65993257599996], [-73.89083184199988, 40.660283967999895], [-73.89001492699988, 40.66063583299995], [-73.88919997600003, 40.6609880809999], [-73.88838556399988, 40.66133773899992], [-73.88756863499995, 40.66168908099994], [-73.88681373099988, 40.66201382299991], [-73.88593573999994, 40.66239371299988], [-73.88511751799982, 40.662745639999876], [-73.88430189299984, 40.66309545699992], [-73.88348677599993, 40.663447816999906], [-73.88309163, 40.66361709199986], [-73.88266944200001, 40.66379795399993], [-73.88185557299985, 40.66414998799994], [-73.88208825799985, 40.66446414699991], [-73.88330883399975, 40.66611200499991], [-73.88391264599986, 40.66585179899992], [-73.88412170000004, 40.665760741999854], [-73.88474247400003, 40.66549489399995], [-73.88493743699995, 40.665407108999936], [-73.88556364499998, 40.66514048699991], [-73.88641344699995, 40.66478080699991], [-73.88657767900006, 40.6647080369999], [-73.88726131299987, 40.66441357399985], [-73.88739217299995, 40.66435345699988], [-73.88810817399994, 40.664047411999896], [-73.88898470699996, 40.663916614999884], [-73.889906696, 40.66377828999986], [-73.89029917799994, 40.66531175899996], [-73.89069385399985, 40.66685223399992], [-73.88976988599985, 40.66698991499991], [-73.89016521899985, 40.668524376999876], [-73.89055869, 40.67006023099991], [-73.89087805100002, 40.671318086999875], [-73.8917995219999, 40.67118041299995], [-73.89271090700004, 40.67104582799992], [-73.89302801299988, 40.67231449199992], [-73.89335128299989, 40.67358164599988], [-73.89367036399987, 40.67482177099988], [-73.89459383999989, 40.6746841039999], [-73.895499435, 40.67455074399987], [-73.89650978, 40.67439704799988], [-73.89748422799983, 40.674255484999954], [-73.89838655899997, 40.67412124399994], [-73.89930675399987, 40.67398204199991], [-73.8997089949999, 40.67549424999994], [-73.89971755899985, 40.6755642639999], [-73.89972239699993, 40.675603736999896], [-73.89973501099999, 40.675654870999914], [-73.89974568900007, 40.67571196399987], [-73.9000274839998, 40.67681911399995], [-73.90004626199993, 40.67689308999995], [-73.89992296499994, 40.67694927299984], [-73.89912562799992, 40.67731263799992], [-73.89883201599999, 40.677446439999855], [-73.89836229199989, 40.677658275999875], [-73.89783505699992, 40.67799357599992], [-73.89775655599985, 40.67806072099989], [-73.89761600399984, 40.67811198799995], [-73.89747545199997, 40.67816325499989], [-73.89695521899982, 40.67840924299985], [-73.8965367649999, 40.67859358099985], [-73.89621773699986, 40.67735818399991], [-73.89531770399991, 40.67749603799994], [-73.8943888149999, 40.677629888999896], [-73.8934840539999, 40.677762060999896], [-73.89257905399984, 40.67790012399996], [-73.89167121099989, 40.67803172099987], [-73.89077319200003, 40.678164588999934], [-73.88987439199998, 40.67830214999987], [-73.88900569799982, 40.678557043999916], [-73.8881563449999, 40.67882783499992], [-73.88730460899997, 40.67909753699995], [-73.88643790499991, 40.67937347599991], [-73.88555502399997, 40.67965659699994], [-73.88467046600006, 40.679940284999894], [-73.88489254699988, 40.68081111599983], [-73.88397193799985, 40.68094711499989], [-73.88303083299995, 40.68108517499993], [-73.88210725299986, 40.68121749099986], [-73.88116730300005, 40.68135366299988], [-73.88109834200004, 40.68107447699986], [-73.88063900599983, 40.679176984999856]]]}}, {\"id\": \"76\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 77, \"Shape_Leng\": 0.075461081444, \"Shape_Area\": 0.000191917981315, \"zone\": \"East New York/Pennsylvania Avenue\", \"LocationID\": 77, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8997089949999, 40.67549425099991], [-73.89930675399987, 40.67398204199991], [-73.89838655899997, 40.67412124399994], [-73.89748422799983, 40.674255484999954], [-73.89650978, 40.67439704799988], [-73.895499435, 40.67455074399987], [-73.89459383999989, 40.6746841039999], [-73.89367036399987, 40.67482177099988], [-73.89335128299989, 40.67358164599988], [-73.89302801299988, 40.67231449199992], [-73.89271090699994, 40.671045828999915], [-73.8917995219999, 40.67118041299995], [-73.89087805100002, 40.671318086999875], [-73.89055869, 40.67006023099991], [-73.89016521899985, 40.668524376999876], [-73.88976988599985, 40.66698991499991], [-73.89069385399985, 40.66685223399992], [-73.89029917799994, 40.66531175899996], [-73.889906696, 40.66377828999986], [-73.88898470699996, 40.663916614999884], [-73.88810817399994, 40.664047411999896], [-73.88739217299995, 40.66435345699988], [-73.88726131299987, 40.66441357399985], [-73.88657767900006, 40.6647080369999], [-73.88641344699995, 40.66478080699991], [-73.88556364499998, 40.66514048699991], [-73.88493743699995, 40.665407108999936], [-73.88474247400003, 40.66549489399995], [-73.88412170000004, 40.665760741999854], [-73.88391264599986, 40.66585179899992], [-73.88330883399975, 40.66611200499991], [-73.88208825799985, 40.66446414699991], [-73.88185557299985, 40.66414998799994], [-73.88266944200001, 40.66379795399993], [-73.88309163100001, 40.66361709099988], [-73.88348677599993, 40.663447816999906], [-73.88430189399988, 40.66309545699991], [-73.88511751799982, 40.662745639999876], [-73.88593573999994, 40.66239371299988], [-73.88681373099988, 40.66201382299991], [-73.88756863499995, 40.66168908099994], [-73.88838556499994, 40.66133773799988], [-73.88919997600003, 40.6609880809999], [-73.89001492699988, 40.66063583299995], [-73.89083184299982, 40.660283967999895], [-73.89164919999989, 40.65993257599996], [-73.89246408799995, 40.659580274999925], [-73.89327894299984, 40.659230173999916], [-73.8940984499999, 40.65887791999994], [-73.89491381999983, 40.658526448999886], [-73.89571848199988, 40.65815995999985], [-73.8961675609999, 40.657978451999924], [-73.89613759299986, 40.6578822489999], [-73.89612587399984, 40.657814663999915], [-73.89704215399999, 40.65766358899991], [-73.89796594499987, 40.65753203599994], [-73.89880698499995, 40.65740666199988], [-73.89881834799998, 40.65745381299991], [-73.89883571499976, 40.657525941999936], [-73.89921760899985, 40.65911140399994], [-73.89965233299985, 40.66073541599994], [-73.89972880499995, 40.66070453499989], [-73.89988908799995, 40.66063980699986], [-73.900146584, 40.66065327699988], [-73.90066118699986, 40.66058615799992], [-73.90105329800004, 40.66212482099994], [-73.90145114700002, 40.66365836499993], [-73.90071711999985, 40.66376455299993], [-73.90050930099981, 40.663793574999914], [-73.90041654099997, 40.663806526999934], [-73.9008513569999, 40.665345894999895], [-73.90126570400001, 40.66688064999991], [-73.90166767899989, 40.668413528999885], [-73.90198218099997, 40.669670070999906], [-73.90228822599984, 40.67093600699995], [-73.90266008399996, 40.67219717899991], [-73.90296768999997, 40.67343939499992], [-73.9032213189999, 40.67447291399992], [-73.90344272399983, 40.67537791899986], [-73.90346228899996, 40.675457918999896], [-73.90347422300006, 40.67550668299985], [-73.90313803899994, 40.67562221499988], [-73.90302076399996, 40.675656480999905], [-73.90291887299995, 40.675684580999864], [-73.90279745999989, 40.67575749299986], [-73.90267831799984, 40.6757897289999], [-73.90257716399994, 40.67583170699991], [-73.90245583099997, 40.6758820509999], [-73.90096234799994, 40.67650221199989], [-73.90079183699987, 40.67657245799992], [-73.90004626199993, 40.67689308999995], [-73.9000274839998, 40.67681911399995], [-73.89974568900007, 40.67571196399987], [-73.89973501099999, 40.675654870999914], [-73.89972239699999, 40.67560373799994], [-73.89971755899985, 40.6755642639999], [-73.8997089949999, 40.67549425099991]]]}}, {\"id\": \"77\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 78, \"Shape_Leng\": 0.0935944495806, \"Shape_Area\": 0.000191114419551, \"zone\": \"East Tremont\", \"LocationID\": 78, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88963328299991, 40.85317381799993], [-73.88993961599995, 40.85259076299992], [-73.88987871399985, 40.85207022099993], [-73.88963044999998, 40.85188593199995], [-73.88951837699989, 40.85180274499989], [-73.88895256299988, 40.85144078099994], [-73.8887824569999, 40.851341184999924], [-73.88823974099995, 40.85103536599995], [-73.88800539899992, 40.85097610299994], [-73.88751770499984, 40.85094946799988], [-73.88643555899993, 40.850309427999875], [-73.88581418499989, 40.84993079699989], [-73.88480193499986, 40.84930945899987], [-73.88377755799993, 40.84868218399991], [-73.88322877699981, 40.84834587799986], [-73.88311982600001, 40.848221091999925], [-73.88278133699995, 40.84807038999989], [-73.88276104799992, 40.848057845999904], [-73.88284445599996, 40.847817225999954], [-73.88199829499979, 40.847235311999874], [-73.88106059899991, 40.84659008499992], [-73.88048011699989, 40.84619371599996], [-73.87919408799995, 40.84532173399988], [-73.87800756899993, 40.844484295999926], [-73.8773292409999, 40.84398312499995], [-73.878171756, 40.84262443699987], [-73.87757279999985, 40.84248377099988], [-73.87694201699995, 40.84233562799991], [-73.87686858199996, 40.84231710599989], [-73.876798721, 40.84229978699994], [-73.87608505099992, 40.84206482699991], [-73.87459175399982, 40.841520647999914], [-73.87456845, 40.84164271999992], [-73.87399295499982, 40.842561505999925], [-73.87280708199992, 40.843424263999964], [-73.87190184999997, 40.843721432999914], [-73.87184610299998, 40.84376077799993], [-73.87179038499981, 40.8438028139999], [-73.87109207899995, 40.84432478099992], [-73.87047298799992, 40.84542033999984], [-73.86944230399996, 40.84724417999992], [-73.86889243299996, 40.848112321999906], [-73.86941402699996, 40.848788024999884], [-73.86920944599997, 40.84923837199993], [-73.86837388899988, 40.8489695979999], [-73.86785992999985, 40.84886493099987], [-73.86800652899981, 40.84830196599988], [-73.86835465899985, 40.84708023299993], [-73.86843808099991, 40.84705127899987], [-73.86892652699994, 40.84688174099993], [-73.86883109200002, 40.84673752699992], [-73.86875399499986, 40.84658447099989], [-73.86875397099986, 40.8465844179999], [-73.8687539479999, 40.846584363999966], [-73.86869718899982, 40.8464246689999], [-73.86869712999989, 40.846424505999934], [-73.86869709399987, 40.84642434499988], [-73.86866197399985, 40.84626032399993], [-73.86866193799993, 40.84626015299994], [-73.86866192800001, 40.846259982999946], [-73.86864915099987, 40.84609409699992], [-73.86864914000002, 40.84609392599993], [-73.86864915200002, 40.846093754999906], [-73.8686587279999, 40.84592863299986], [-73.86865873999996, 40.84592845299991], [-73.86865877499986, 40.845928281999925], [-73.86869009299983, 40.84576627499991], [-73.86874186599977, 40.845609728999904], [-73.86863721699989, 40.845309842999896], [-73.86804200399997, 40.8448138389999], [-73.8688925879999, 40.844409658999936], [-73.86959555499995, 40.84398144199994], [-73.87030135699996, 40.8435486959999], [-73.87099518100001, 40.843111391999926], [-73.87153346199983, 40.84251428099987], [-73.87223024699988, 40.841643169999934], [-73.87229921099988, 40.84157999599991], [-73.87236365199989, 40.84147695999991], [-73.87244734699995, 40.84134426699992], [-73.87280724899995, 40.84086864499993], [-73.87326716999992, 40.84021994899994], [-73.87359713599987, 40.83979855499992], [-73.87561285199983, 40.839902520999885], [-73.87747229399999, 40.839983546999925], [-73.87791380399995, 40.8399703279999], [-73.87823856699993, 40.839985917999904], [-73.87831009099979, 40.8399893509999], [-73.87841924999984, 40.83999458999992], [-73.87850461299986, 40.840005593999905], [-73.8786598319999, 40.840019903999824], [-73.87978300299987, 40.840145825999855], [-73.87992249999996, 40.840154029999916], [-73.880061996, 40.840162234999845], [-73.88010086899989, 40.84006661599993], [-73.88027152599983, 40.83989623999985], [-73.88030767899997, 40.839717943999894], [-73.88030769199985, 40.8397178709999], [-73.88030771499984, 40.83971779899986], [-73.88036581299994, 40.839540578999895], [-73.88036583800002, 40.83954050699991], [-73.88036587199986, 40.839540434999904], [-73.88044586500001, 40.83936659699989], [-73.88044592299988, 40.839366479999924], [-73.88044599399986, 40.83936635399988], [-73.88054735399987, 40.839198110999895], [-73.88066887899986, 40.83903768799993], [-73.88066892599986, 40.83903762499989], [-73.88066897399986, 40.83903757099989], [-73.88080875199994, 40.83888725199996], [-73.88080895400005, 40.838887035999875], [-73.88080919100001, 40.83888681999984], [-73.880965322, 40.838748079999895], [-73.88113550199986, 40.83862214199998], [-73.88135107299988, 40.83840917199989], [-73.88138310599982, 40.83837771399988], [-73.88154924699987, 40.838182756999885], [-73.88160324799993, 40.83809985899994], [-73.88166303799993, 40.838028618999914], [-73.88169674899986, 40.83798711299989], [-73.88172047499994, 40.83795364899991], [-73.88182513299996, 40.837787064999915], [-73.88191969599985, 40.837616932999914], [-73.88200384100001, 40.837443781999895], [-73.88203937499988, 40.8375245929999], [-73.88206752699985, 40.837655624999876], [-73.88211370499987, 40.83778515499997], [-73.88217759099985, 40.837911036999905], [-73.88225831200006, 40.83803121799985], [-73.88235442499995, 40.83814382399988], [-73.88246404800005, 40.83824724099992], [-73.88285365099989, 40.83849533499994], [-73.88279558600006, 40.83854994599993], [-73.88274769899994, 40.8385923409999], [-73.8831387399999, 40.838810671999944], [-73.88351820399988, 40.8390223659999], [-73.88403342700002, 40.83930464099993], [-73.88412724499999, 40.83935591899995], [-73.88418143199976, 40.839335306999914], [-73.88435324499997, 40.83927764999993], [-73.88464898099988, 40.83917463299988], [-73.88469619299984, 40.83920338699989], [-73.88583496499986, 40.83989692699992], [-73.88618432099993, 40.8401050449999], [-73.88659802299989, 40.839279049999895], [-73.88754251799996, 40.839640657999944], [-73.88813752199994, 40.83986200199991], [-73.88869040799987, 40.83915802599988], [-73.88941724899988, 40.83942273399985], [-73.89017468599997, 40.839873667999896], [-73.89216815599981, 40.84104963999991], [-73.89317098899986, 40.84163734299991], [-73.89410150499995, 40.84219030199994], [-73.89484862699986, 40.84262825999986], [-73.89493579899987, 40.84267935899992], [-73.89501415699996, 40.84272529099994], [-73.89509807299986, 40.84277448099986], [-73.89527784899985, 40.84287986099988], [-73.89537836099996, 40.842938778999944], [-73.89484758699984, 40.84364636299994], [-73.89481942600001, 40.84368377099985], [-73.89479022099985, 40.8437203219999], [-73.89464252699996, 40.84390413999996], [-73.89398467499994, 40.844703958999936], [-73.89290102299996, 40.84626832999992], [-73.89337309899989, 40.84640119399996], [-73.89360906599997, 40.84644525599987], [-73.89384985999999, 40.84647439399987], [-73.8940930149999, 40.84648822299994], [-73.89433599299984, 40.846486680999874], [-73.89535848999982, 40.84651803199993], [-73.89617712999998, 40.8466583209999], [-73.89611702199987, 40.846778518999926], [-73.89557755999984, 40.847854641999895], [-73.89494627999981, 40.849104852999865], [-73.89407258999995, 40.85071318899992], [-73.89478764999994, 40.85108166699989], [-73.89431774099977, 40.851891310999854], [-73.89307140900002, 40.85324998699989], [-73.89216687399981, 40.85276946199989], [-73.89090203399996, 40.854116885999865], [-73.8906903199999, 40.854353261999954], [-73.89038953999983, 40.85468905799996], [-73.89003039600001, 40.854220991999945], [-73.88960978099988, 40.85362187999995], [-73.88963328299991, 40.85317381799993]]]}}, {\"id\": \"78\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 79, \"Shape_Leng\": 0.0426249113144, \"Shape_Area\": 0.000107893068218, \"zone\": \"East Village\", \"LocationID\": 79, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98377990999991, 40.72153595399992], [-73.98382387299982, 40.72147287199987], [-73.98467780299995, 40.72173060499991], [-73.98551686199984, 40.72198522499986], [-73.98628574000003, 40.722241919999874], [-73.98713584799998, 40.7224698109999], [-73.98786491500006, 40.72269708899991], [-73.98854361399988, 40.722903617999904], [-73.98863862799996, 40.7229337199999], [-73.9887568319999, 40.72297036599986], [-73.98939218099996, 40.723174057999884], [-73.99027361600001, 40.72342601799986], [-73.99102410399989, 40.723649431999895], [-73.99127307099994, 40.7237298129999], [-73.99135472799999, 40.72375461899987], [-73.99250385799984, 40.72410592499995], [-73.99260322199994, 40.72413644999987], [-73.99257987399992, 40.724201554999915], [-73.99232642199998, 40.72490823199993], [-73.99220971099992, 40.72523128999992], [-73.9920616679999, 40.72564244299991], [-73.99179379299984, 40.72639145899992], [-73.99154832699986, 40.72708947099994], [-73.99136388099988, 40.72755978299991], [-73.99135869199992, 40.72778246999995], [-73.99112256699988, 40.72856796599996], [-73.99093053499996, 40.7291422959999], [-73.99075728499993, 40.729740830999916], [-73.99072530399991, 40.729878686999896], [-73.99050558599988, 40.73056495699994], [-73.99039009999987, 40.731288523999865], [-73.99021190999987, 40.73199633399989], [-73.99002687699983, 40.73275628399994], [-73.98986852099996, 40.73352711499991], [-73.98990295999991, 40.7344347899999], [-73.98877931699997, 40.733965399999924], [-73.98718049699994, 40.733292147999876], [-73.98494102799992, 40.73234760399984], [-73.98255629299986, 40.731350133999875], [-73.98246876199994, 40.731314883999914], [-73.98150053399984, 40.730918076999934], [-73.98082000799998, 40.73063915899997], [-73.98034007599992, 40.73044246099994], [-73.97926849199996, 40.729981427999846], [-73.97802697999985, 40.729433059999984], [-73.97805542899995, 40.7293830859999], [-73.97853582999998, 40.72874608499992], [-73.97899862099985, 40.72810574699991], [-73.979469131, 40.727467676999865], [-73.97990650199998, 40.72686577299986], [-73.9803329819999, 40.72628033599988], [-73.98076713900005, 40.7256855949999], [-73.9812112959998, 40.72507279999992], [-73.981636639, 40.72449348099985], [-73.9820824239999, 40.72388204399991], [-73.98252449499985, 40.723277976999945], [-73.9829641439999, 40.72267132499997], [-73.98344363599992, 40.72201850699989], [-73.98377990999991, 40.72153595399992]]]}}, {\"id\": \"79\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 80, \"Shape_Leng\": 0.117212621448, \"Shape_Area\": 0.00040732245622, \"zone\": \"East Williamsburg\", \"LocationID\": 80, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93174841322003, 40.728053727696675], [-73.93168317826739, 40.72803172609697], [-73.93166858749639, 40.72803828280952], [-73.93157356186828, 40.7280182950609], [-73.93120770151333, 40.72794133862442], [-73.9308047913174, 40.72784883926168], [-73.93072254234549, 40.72784107268961], [-73.93032635559958, 40.72779514224887], [-73.92991812728, 40.72763889835636], [-73.92983531275382, 40.72760788932665], [-73.92966308042121, 40.727543397711244], [-73.92965484905174, 40.727547652551394], [-73.92963224605802, 40.72756459428362], [-73.92962400635008, 40.72757757457619], [-73.92962734303951, 40.727600054005656], [-73.92961163522367, 40.72761121943942], [-73.92958020690887, 40.72761282070866], [-73.9294975342307, 40.72755906458196], [-73.92942476304782, 40.727511752515085], [-73.92937588603782, 40.72747996953887], [-73.92924333948385, 40.72739794231863], [-73.92926898893549, 40.72736521532118], [-73.92886953254997, 40.72700263162835], [-73.92849418540158, 40.72661460330786], [-73.92833738013012, 40.72641758269856], [-73.92815870725637, 40.726205230757685], [-73.92813074743837, 40.7261726868244], [-73.9280542953431, 40.726072340162595], [-73.92795514073464, 40.725916517513646], [-73.92793912625204, 40.725883729045734], [-73.92792754362857, 40.72587905684713], [-73.92790171856065, 40.725875780441555], [-73.9278825766526, 40.7258664841229], [-73.9278717240502, 40.725854051197615], [-73.92786431997051, 40.72582667905824], [-73.92785895385158, 40.72575459949976], [-73.92780584783978, 40.72504980016611], [-73.92779650104073, 40.7249256692162], [-73.92720275963465, 40.72365294718613], [-73.92561470516611, 40.72006166221244], [-73.92481798943687, 40.72026513747534], [-73.92481537158021, 40.72022555855079], [-73.92470891655738, 40.71999887556637], [-73.92471098441669, 40.71999119597938], [-73.92474503108099, 40.71998524724633], [-73.92467622369254, 40.719885695480855], [-73.92460885647313, 40.7197896928349], [-73.92453432562169, 40.71966615777251], [-73.92451999415132, 40.71961273042429], [-73.92452018596838, 40.719568370774205], [-73.92452083048202, 40.71955356651187], [-73.92453854985803, 40.719342580204575], [-73.92455355876741, 40.71927913119411], [-73.92457948088894, 40.7191625233347], [-73.92462954199868, 40.71898697487623], [-73.9246900485712, 40.71872233555582], [-73.92470227807098, 40.71868830410057], [-73.92477183253615, 40.71856461085536], [-73.9248155567726, 40.718486835905985], [-73.9248641350909, 40.71844044541349], [-73.92493181378975, 40.718405828749404], [-73.92499204426545, 40.718384788263776], [-73.92508677520757, 40.71838421920426], [-73.92615790234161, 40.718099468135776], [-73.92638366600745, 40.71808480802871], [-73.92837930101507, 40.717578367290116], [-73.92839153834066, 40.71757875368058], [-73.92840375075583, 40.71758019296097], [-73.92842127419078, 40.7175842105983], [-73.92842419720883, 40.71758530268078], [-73.92842688160763, 40.71758675412747], [-73.92842937757847, 40.71758857435136], [-73.92843171853902, 40.71759080894033], [-73.92843568965912, 40.717595953982475], [-73.92844561386065, 40.71761186331317], [-73.92844953863248, 40.717616088268166], [-73.92845165582267, 40.71761777322944], [-73.92845389020685, 40.717619198094184], [-73.92846194563565, 40.717623120381695], [-73.92847083084905, 40.7176262868002], [-73.92848064026589, 40.717628732421325], [-73.9284915378775, 40.71763049638031], [-73.92851353839693, 40.71763209434512], [-73.92857425221365, 40.71763313166123], [-73.9285917078938, 40.71763506050507], [-73.92860667565203, 40.71763833823792], [-73.9286372006141, 40.71764795729271], [-73.92866774684488, 40.71765931310398], [-73.92869826813597, 40.71767238068159], [-73.92872866939294, 40.7176871410362], [-73.9287353864659, 40.71769118761855], [-73.92874207976142, 40.71769646885917], [-73.92876845782247, 40.717722042250244], [-73.92877608561962, 40.717727270793475], [-73.92877998940898, 40.71772931636664], [-73.92878398825955, 40.71773100248575], [-73.92878963057198, 40.71773285212232], [-73.9287956187773, 40.71773429739506], [-73.92880855316629, 40.717735971449116], [-73.92882252966149, 40.71773599768121], [-73.92883713697681, 40.71773436746044], [-73.92885202896086, 40.717731107711856], [-73.92886642560443, 40.717726344101365], [-73.92887965179995, 40.717720318623556], [-73.92889108105969, 40.71771331116421], [-73.92896850706042, 40.71764754015081], [-73.92898221166264, 40.71763657123715], [-73.92907419357621, 40.71757133281935], [-73.92909346674548, 40.717556161224586], [-73.92911003942831, 40.71754160111232], [-73.9291243277067, 40.71752729272452], [-73.92913648610188, 40.717513071310634], [-73.92913856054393, 40.717510038005024], [-73.92914020897827, 40.71750676236066], [-73.92914231209284, 40.71749933299893], [-73.92914297144002, 40.71749071598783], [-73.92914245456932, 40.71746451965398], [-73.92914397867989, 40.717456613991715], [-73.92914530784584, 40.717453139463885], [-73.92914701545092, 40.7174499159568], [-73.92915700695039, 40.717436414577925], [-73.92916989772768, 40.71742355510231], [-73.92918539042974, 40.71741165011861], [-73.92920294211369, 40.717401098006235], [-73.92956856905529, 40.71722153556257], [-73.92960578835353, 40.71719018455505], [-73.92963940071803, 40.71715649885372], [-73.92964876146104, 40.71714594216955], [-73.92966723720504, 40.71712254843958], [-73.92968295619124, 40.71709844304385], [-73.92969572853036, 40.717073929091114], [-73.92970539915328, 40.71704927938918], [-73.92970762282195, 40.71704020380317], [-73.92970848430281, 40.7170306673565], [-73.92970798609383, 40.71702080692028], [-73.92970612734644, 40.71701080158015], [-73.92969847812826, 40.71699103024035], [-73.92969285538712, 40.716981725376904], [-73.9296861770915, 40.71697307523373], [-73.92950714859491, 40.71659078087593], [-73.92950234927692, 40.71658551842918], [-73.92949846168197, 40.71657965377863], [-73.92949561546526, 40.71657342087128], [-73.92949390595588, 40.71656703436778], [-73.92949347200953, 40.7165632699393], [-73.92949464975428, 40.716535769271715], [-73.92949639794345, 40.71652681971331], [-73.92949731486091, 40.71652213772954], [-73.9295016940007, 40.71649978980536], [-73.92931999766078, 40.71611200883265], [-73.92931572636294, 40.71607810126976], [-73.92931651965557, 40.7160438378059], [-73.92932187977752, 40.71601207044922], [-73.9293317763477, 40.71597656097953], [-73.92934483421232, 40.71593891798302], [-73.92936131668621, 40.71589831554491], [-73.92938250209323, 40.71585162805965], [-73.92940052893961, 40.715816518766324], [-73.92941981817356, 40.71578500369684], [-73.92944067813087, 40.71575657861639], [-73.92946325081381, 40.71573104475621], [-73.92947356921455, 40.71572145262114], [-73.92948531923271, 40.71571242793815], [-73.92949834532271, 40.71570406928501], [-73.92951247231095, 40.71569650521456], [-73.92952747418936, 40.7156898412977], [-73.92954313880661, 40.71568419668339], [-73.9295592038559, 40.71567964911099], [-73.92957543323405, 40.71567627365539], [-73.93029962426964, 40.71550787636393], [-73.93149304610854, 40.71520848107516], [-73.93151298432782, 40.71521190462473], [-73.93153325506185, 40.71521398848785], [-73.93153596540606, 40.71521416080974], [-73.93155892431574, 40.71521485076743], [-73.9315843812832, 40.71521425275001], [-73.93165926717428, 40.71520855239426], [-73.93170534538301, 40.715203167612366], [-73.93175008954245, 40.7151945849367], [-73.93179267229117, 40.71518297509124], [-73.93183242829471, 40.71516852832981], [-73.93190482995988, 40.715137583883475], [-73.93198814917388, 40.71510013641414], [-73.9320446165215, 40.71507417192053], [-73.93208047053639, 40.715056497540985], [-73.93210478113058, 40.715042599269246], [-73.93212999514387, 40.71502560378398], [-73.9321557326611, 40.7150060591762], [-73.93219753146339, 40.71497210749033], [-73.93220607307306, 40.71496398092936], [-73.93221363282309, 40.71495510661935], [-73.93222529811959, 40.71493579746084], [-73.93223037717549, 40.71492154524626], [-73.93223414916336, 40.71490201583286], [-73.93223469045104, 40.71489346121944], [-73.9322353635064, 40.714882853137915], [-73.93223397469464, 40.71486442837482], [-73.93223001508791, 40.7148469655458], [-73.93221519306172, 40.71480561350932], [-73.9321969843879, 40.71476644888617], [-73.93217367921345, 40.7147221395194], [-73.93215665337218, 40.71469483448544], [-73.93215361559824, 40.71469104222988], [-73.932149986012, 40.714687464730225], [-73.93214128153326, 40.71468129159166], [-73.93213686921216, 40.71467904647472], [-73.93175357182275, 40.714741675505685], [-73.93159048356533, 40.71439949248125], [-73.93155662007531, 40.714393384423], [-73.93154203825726, 40.714359606502086], [-73.93148254530635, 40.714221748840004], [-73.93124067059496, 40.71372481088467], [-73.93145301733048, 40.713438844463575], [-73.9315560427113, 40.71339947275473], [-73.93159144262145, 40.713384815917905], [-73.93164527391656, 40.713342793758144], [-73.93197184127631, 40.7132532609102], [-73.9324516315665, 40.71312171743927], [-73.93255041730532, 40.71309537126197], [-73.93257765480163, 40.71308911115324], [-73.9325956361949, 40.713083700676535], [-73.93261051935492, 40.71307728824735], [-73.93261675980561, 40.71307367214419], [-73.9326222660997, 40.713069749371755], [-73.93263159981665, 40.7130613711127], [-73.9326390524247, 40.71305258616936], [-73.93264437548959, 40.713043701808665], [-73.93264621458815, 40.71303927229798], [-73.93264747285893, 40.7130348786332], [-73.93264823497748, 40.7130310430457], [-73.9326491737301, 40.713015815055016], [-73.93264693956343, 40.71300073028623], [-73.93203262980515, 40.71159022307427], [-73.93338484615303, 40.711233727745665], [-73.93328349320237, 40.71097870745788], [-73.9319010192085, 40.71133847142828], [-73.93166398860762, 40.71081024398526], [-73.93151092590584, 40.71046980839423], [-73.93126572703261, 40.70992444355293], [-73.93102978770145, 40.70935851220726], [-73.93093227547743, 40.709165311254075], [-73.93079490181064, 40.70911470106856], [-73.9307918037048, 40.70911293481054], [-73.93078745172906, 40.709110005160596], [-73.93078694295174, 40.709109590392615], [-73.93078352647346, 40.70910679800705], [-73.93078002706979, 40.70910330128765], [-73.93077693109801, 40.709099480980846], [-73.9307719933313, 40.70909101303475], [-73.93076857210804, 40.709081132574475], [-73.93076600066632, 40.70906065402331], [-73.93076683651668, 40.7090191674455], [-73.93076602623842, 40.70900254307664], [-73.93076270533346, 40.70898637705963], [-73.93075999154775, 40.70897937849044], [-73.93075653104769, 40.708972982737215], [-73.93073903156917, 40.708947316942], [-73.93071882196693, 40.70892173947502], [-73.93069593784345, 40.708896313011245], [-73.93067043926465, 40.708871073938994], [-73.93064180374176, 40.70884694166527], [-73.93057213306344, 40.70879486826071], [-73.93054296635056, 40.70876887089745], [-73.93052794395754, 40.708751183869595], [-73.93052572285508, 40.708747608338385], [-73.93051585863094, 40.70873176236181], [-73.93050712316408, 40.70871122513413], [-73.93050203180053, 40.708690294909395], [-73.93050100083194, 40.70868122503959], [-73.9303609456579, 40.7087177385352], [-73.93037347956368, 40.70873930350495], [-73.93037720999881, 40.70874751861006], [-73.9303818492876, 40.70875776871385], [-73.93038563133163, 40.70877453878897], [-73.93038574076616, 40.70878231914898], [-73.93038466900703, 40.708789765583106], [-73.9303802173509, 40.70880284776285], [-73.93037287805068, 40.70881587324679], [-73.93036285425627, 40.70882846505405], [-73.93035053503768, 40.70884013775799], [-73.93028879813227, 40.70888572873882], [-73.93026628094589, 40.708904924092934], [-73.93026509729741, 40.70890609372025], [-73.93025651835315, 40.70891654288407], [-73.93025038849073, 40.708927912919975], [-73.93037671406199, 40.70922233953024], [-73.93060170783275, 40.7097467525867], [-73.93062904674879, 40.709791038263994], [-73.93063814682539, 40.709801768663844], [-73.93065009956263, 40.7098127901007], [-73.93070567535537, 40.70985517357679], [-73.93071575895901, 40.70986497746762], [-73.93072360705276, 40.70987464532034], [-73.93072912419363, 40.70988365305337], [-73.93073348083954, 40.70989336336667], [-73.93073890171183, 40.709915357740414], [-73.93074036930459, 40.70993797930222], [-73.93073947816461, 40.70999853812925], [-73.93074145900518, 40.71001600921204], [-73.93074537263652, 40.7100313020978], [-73.93076957459168, 40.7100982520984], [-73.930791487333, 40.710146640057424], [-73.93080914645333, 40.71017853827857], [-73.93085266627386, 40.71025176672247], [-73.93087163508754, 40.71028754614978], [-73.93090318414211, 40.710355886650596], [-73.93093021730881, 40.71042117175436], [-73.9309979955702, 40.71059932449671], [-73.93103036450829, 40.71068854870446], [-73.93115836965862, 40.71100104854482], [-73.93154613746852, 40.71185535175101], [-73.93088881912188, 40.71205600685815], [-73.93085966173909, 40.71206494115675], [-73.9308357102634, 40.712074931985754], [-73.93081637309481, 40.71208622106957], [-73.93080841450526, 40.71209238429651], [-73.93080159026496, 40.71209889079763], [-73.93078780933578, 40.71211647921726], [-73.93078286503948, 40.71212531932618], [-73.9307793659171, 40.71213390717069], [-73.93077736917618, 40.712142064536344], [-73.93077687548653, 40.71214978254474], [-73.9307778853388, 40.712157013953636], [-73.93077894737037, 40.71216042795544], [-73.93078037522965, 40.71216370664792], [-73.930790204141, 40.71218082226048], [-73.93080080544334, 40.71219469685591], [-73.93081258043564, 40.71220580594958], [-73.93081894298919, 40.712210349085794], [-73.9308256373487, 40.71221423370464], [-73.93083369272601, 40.71221792246413], [-73.93084682544475, 40.71222140637212], [-73.93085528765457, 40.712220663782], [-73.93086409444768, 40.71221819268688], [-73.93161032734847, 40.71203802318303], [-73.93182042077258, 40.712478983611106], [-73.93183334985444, 40.71250752798771], [-73.93184122321138, 40.71252826277801], [-73.93184635118098, 40.71254724913816], [-73.93184868877432, 40.71256477418327], [-73.93184794872995, 40.712593491059096], [-73.93184249493441, 40.71262723799059], [-73.93183415746144, 40.712655374577345], [-73.93182187286679, 40.712679465226174], [-73.93181633870033, 40.71268716098756], [-73.93180970336468, 40.71269477484574], [-73.93179285963701, 40.71271003780262], [-73.93177226424528, 40.71272495661109], [-73.93173436851761, 40.71274929223796], [-73.93066788646628, 40.71302126689367], [-73.9305711431028, 40.7130510969146], [-73.93054642776369, 40.71305574646063], [-73.9305271320743, 40.71306156090452], [-73.9305190693687, 40.71306503188885], [-73.9305120595292, 40.71306888245433], [-73.93050607875219, 40.713073137548186], [-73.93050113783234, 40.713077790811234], [-73.93049150439325, 40.71309041971448], [-73.93047361522203, 40.71311996285386], [-73.93046418241934, 40.71313167232028], [-73.93043603849989, 40.713156086450766], [-73.93043058814568, 40.713161549060715], [-73.93042631105753, 40.71316678789339], [-73.93042448583894, 40.71316958700425], [-73.93042294313065, 40.71317255867104], [-73.93042263899005, 40.71319163126945], [-73.93077166547451, 40.71392946758245], [-73.93095073591746, 40.714363819478436], [-73.93105231901723, 40.71461022392334], [-73.93085190134036, 40.71490298559276], [-73.92991665528311, 40.715149816999386], [-73.92990279901291, 40.71512496347697], [-73.92931412285716, 40.71527983468949], [-73.92927167066954, 40.715314216978754], [-73.92925366567535, 40.71532880253978], [-73.92843881814578, 40.71553094957575], [-73.92890242854061, 40.7165728123179], [-73.92890598986219, 40.716596354731784], [-73.92890601265002, 40.71661895731207], [-73.9289025104377, 40.71664010793748], [-73.92889551897623, 40.71665951948844], [-73.92888441854586, 40.71668042166825], [-73.92887072694849, 40.716701449064075], [-73.92885446710963, 40.71672257421461], [-73.92883566172434, 40.7167437780346], [-73.9287905809, 40.716787938570135], [-73.92876805027394, 40.71680733061646], [-73.92874531987793, 40.716825128348454], [-73.92871111320488, 40.716849745859825], [-73.92867728796146, 40.71687240872566], [-73.92864365422389, 40.716893243646275], [-73.92861009191238, 40.71691233129565], [-73.92857581108359, 40.71692983368455], [-73.92848910920299, 40.716971041911755], [-73.92843622401524, 40.716998834267095], [-73.92832241942101, 40.717062456200466], [-73.92827145107029, 40.71708972883212], [-73.92825263489712, 40.71709859586681], [-73.9282325304494, 40.717106453750795], [-73.92821135004563, 40.71711323979312], [-73.9281892718893, 40.71711888022515], [-73.9281665195066, 40.7171233328014], [-73.92814336621645, 40.717126524817886], [-73.92812006106152, 40.71712843649421], [-73.92809688744708, 40.717129034669306], [-73.92595163943949, 40.7176910705699], [-73.92594091108849, 40.71767415165964], [-73.92514606059815, 40.71785561237674], [-73.9249974068021, 40.717889555087254], [-73.9249747370717, 40.717893899534175], [-73.92495489653578, 40.71789814597476], [-73.92494252442464, 40.71790088484829], [-73.92492621081297, 40.71790485414829], [-73.924902393685, 40.717909422318826], [-73.92488911247905, 40.71791154744498], [-73.92486299711757, 40.71791650059256], [-73.92483730903177, 40.71792192342344], [-73.92482214344714, 40.71792509251212], [-73.92479461917726, 40.71793177393681], [-73.92477173579029, 40.71793694712741], [-73.92475077183728, 40.717940443953374], [-73.92472906134888, 40.71794405068778], [-73.92471227577259, 40.717946155912], [-73.92469362102345, 40.71794812488463], [-73.92467591434051, 40.71794992259175], [-73.92466004074598, 40.71795144323361], [-73.92464513863976, 40.71795310891734], [-73.9246255503998, 40.717953843264645], [-73.92460339476072, 40.717953306073376], [-73.92458693486758, 40.717950702646235], [-73.92458407107215, 40.71795024074147], [-73.92456824137071, 40.71794488190683], [-73.92455290971758, 40.71793913473411], [-73.9245380167141, 40.717932371796], [-73.92452081662728, 40.7179244813951], [-73.92450755701465, 40.717917637268215], [-73.92449266537841, 40.71790967583763], [-73.92447781046084, 40.71790098551972], [-73.92446061312032, 40.7178908795681], [-73.92445062958915, 40.71788537964227], [-73.92442924671535, 40.71787229884444], [-73.9244087753187, 40.717858894811435], [-73.92439020556472, 40.717848256552855], [-73.92436927002785, 40.7178364200924], [-73.92435070091092, 40.717825231837054], [-73.92433303151248, 40.71781399072887], [-73.92431306786708, 40.717802154896944], [-73.92429305607753, 40.717790065394034], [-73.92427585871782, 40.717780607576316], [-73.92426006927353, 40.717770880796486], [-73.9242456522157, 40.717761730707366], [-73.9242335769337, 40.71775456383365], [-73.9242219623098, 40.71774774806305], [-73.92420571235455, 40.71773697627477], [-73.92419082355931, 40.71772697114595], [-73.92418084168771, 40.71772024707839], [-73.92416598865648, 40.717709971916534], [-73.92415062601867, 40.717699992771976], [-73.92413531121534, 40.717689582275256], [-73.9241199951009, 40.717679360578266], [-73.92410280094333, 40.7176671372024], [-73.9240842361937, 40.71765219513093], [-73.92406646435994, 40.7176385671579], [-73.92404826607324, 40.71762433680912], [-73.92403042353494, 40.71760939353005], [-73.92401456906461, 40.71759477706518], [-73.92400123172588, 40.71758340440814], [-73.92398912454873, 40.71757201446884], [-73.92397727997053, 40.71755992092113], [-73.92396563780986, 40.717546415588856], [-73.92395450324652, 40.71753361051829], [-73.92394037469352, 40.717520813855145], [-73.9239313285829, 40.71751413595133], [-73.92390367881583, 40.717495620696546], [-73.92389417064558, 40.7174890594181], [-73.92387326148531, 40.717474546641604], [-73.92385189177755, 40.717460755430274], [-73.92383238966089, 40.717448477883465], [-73.92381472136069, 40.71743663242616], [-73.92380657161958, 40.717431494845535], [-73.92379521885265, 40.717423553926565], [-73.92378242354737, 40.71741459365532], [-73.92377013710404, 40.717405994405105], [-73.92374875470075, 40.7173921758591], [-73.9237283193837, 40.71737827651839], [-73.92370971501525, 40.71736731362656], [-73.9236873979865, 40.71735310679241], [-73.92367370384754, 40.71734403836096], [-73.92366000842217, 40.71733551941521], [-73.92363989262059, 40.71732065749333], [-73.92362699414979, 40.71730857309419], [-73.92361499494635, 40.71729682250124], [-73.92360155242616, 40.71728473773811], [-73.92358786233967, 40.71727158063685], [-73.92357604244324, 40.717258425947975], [-73.9235660685184, 40.71724527147838], [-73.92355875481299, 40.71723477608635], [-73.92354950298376, 40.71722055092756], [-73.92353534366401, 40.71720456649131], [-73.92352727087955, 40.7171951504717], [-73.92351902168116, 40.71718501413588], [-73.92351052402459, 40.71717381467611], [-73.92349611939407, 40.71715461539783], [-73.92348498545175, 40.71714217213949], [-73.92347481994251, 40.71713080085855], [-73.92346278696195, 40.71711764601619], [-73.92345233852708, 40.71710589643689], [-73.92344002043036, 40.71709292970286], [-73.92342891999935, 40.717081999229585], [-73.92341861434718, 40.717069015738694], [-73.923410699233, 40.71705692524807], [-73.92340522164903, 40.71704449445611], [-73.92340130597819, 40.717032083298584], [-73.92339299433871, 40.71701539074552], [-73.92338539779375, 40.717002949663616], [-73.92337834167141, 40.71699424629923], [-73.92337052250669, 40.71698144505282], [-73.92336526384003, 40.71697452587305], [-73.92335355684713, 40.716965899489054], [-73.92328653237342, 40.71683400310813], [-73.92323157113599, 40.71669943915755], [-73.92316776085262, 40.716478915386126], [-73.92313445722395, 40.716479468945344], [-73.923117181684, 40.71647618845653], [-73.9231114960376, 40.71646918823154], [-73.92310361124972, 40.71645153249399], [-73.92307579911487, 40.716389234766446], [-73.92305500014334, 40.71634266479336], [-73.92326039798908, 40.716249734135324], [-73.92331244916073, 40.71622884989675], [-73.92333477379717, 40.71622532676395], [-73.92336627583238, 40.71621730560632], [-73.92338579746807, 40.71621278917319], [-73.92340491085959, 40.716192404479855], [-73.9234316927592, 40.71617252109859], [-73.92347144070784, 40.71614666662387], [-73.92361145155542, 40.716103508171265], [-73.92359001003088, 40.71602814919749], [-73.92431329961565, 40.715847056576735], [-73.92459408281036, 40.715534375855434], [-73.92483287559034, 40.715282912041374], [-73.92488553507611, 40.71526718765259], [-73.92485230936, 40.71519860069832], [-73.92478745137134, 40.715075593546025], [-73.92476629284467, 40.715032049009196], [-73.92476128782185, 40.71501947417909], [-73.92476257206151, 40.71500407627395], [-73.92476493340077, 40.71498865277085], [-73.92476155049336, 40.71497597144838], [-73.92472981629032, 40.71493796647526], [-73.92469009338394, 40.714880957032975], [-73.92466013078389, 40.71486688040726], [-73.92461501505854, 40.71482708365424], [-73.92455568744005, 40.71475365373278], [-73.9245258166989, 40.71469041784693], [-73.92451226161313, 40.71462077218059], [-73.92448221734742, 40.71452345230061], [-73.92443153174062, 40.714364776176346], [-73.92441910967021, 40.71434961252492], [-73.92441453117704, 40.714348538518855], [-73.92440910864588, 40.71434991286799], [-73.92439028050491, 40.71434917913236], [-73.92436718832793, 40.7143308294748], [-73.92434751117983, 40.7143070530094], [-73.92433779606827, 40.714253708710935], [-73.92433699107761, 40.71420158735123], [-73.92433637463596, 40.71414008175541], [-73.92433589757124, 40.71412259346647], [-73.9240590971852, 40.71411155982241], [-73.92404011299993, 40.714008312999916], [-73.92313936299986, 40.713394727999905], [-73.92223294699991, 40.712854424999904], [-73.92168663599989, 40.71189431699987], [-73.92154601699998, 40.71104328399984], [-73.92122073500002, 40.71079055899986], [-73.92109442099995, 40.71072126499991], [-73.92074519699985, 40.71052968599989], [-73.92189184699987, 40.709396095999836], [-73.92248241799982, 40.709332487999895], [-73.92418647499989, 40.70906472899986], [-73.92576502499996, 40.7088037639999], [-73.92740683799988, 40.70854132199989], [-73.92902566099997, 40.70827681499986], [-73.9320728499999, 40.70778578599989], [-73.93229708600003, 40.70774944699988], [-73.93252310899986, 40.70823818799996], [-73.93269215700003, 40.708632035999905], [-73.93299690799995, 40.709317159999884], [-73.93329596399984, 40.709987710999926], [-73.933601465, 40.7106691059999], [-73.93389272699982, 40.711335677999905], [-73.93425709399997, 40.712123023999936], [-73.93453329099987, 40.71275330399992], [-73.93480281900003, 40.713372495999884], [-73.93724803599986, 40.712782740999884], [-73.93832001399986, 40.71252368499994], [-73.93846617699984, 40.71248925599995], [-73.94068803099992, 40.71196204799987], [-73.9404771349999, 40.711562642999866], [-73.94042251999994, 40.711217647999945], [-73.94075355899997, 40.711186512999916], [-73.94116852899995, 40.71114748299993], [-73.94159179599998, 40.711107670999915], [-73.94196964799987, 40.7110721309999], [-73.94210580699989, 40.71105932199989], [-73.94245994699992, 40.7110258639999], [-73.94283637099988, 40.7109902979999], [-73.94326978899984, 40.710949344999925], [-73.94357731299993, 40.710920286999894], [-73.94378799599994, 40.710900378999945], [-73.94368828299991, 40.71029296399993], [-73.94367685499988, 40.71022334799989], [-73.94358512999999, 40.70966450499993], [-73.94357094699997, 40.70957809399996], [-73.94355745099986, 40.709495864999965], [-73.94354434699991, 40.70941603099986], [-73.94344009799994, 40.70878089099988], [-73.94332294399995, 40.708070194999934], [-73.94320222899984, 40.7073347509999], [-73.94308587399995, 40.70659666199991], [-73.94296619399988, 40.70588843999991], [-73.94285013599988, 40.70518202999993], [-73.94273448399997, 40.70447330599988], [-73.9422114949999, 40.7045232859999], [-73.94214324999996, 40.70452980699987], [-73.9417696579999, 40.70456550799983], [-73.94168041699996, 40.70457403599987], [-73.94103009799993, 40.70463617699996], [-73.9408928719998, 40.70391415699989], [-73.94077939399978, 40.70322246899988], [-73.94076481300007, 40.70313403299987], [-73.94067555499998, 40.7025926509999], [-73.9406199669999, 40.702255449999896], [-73.94060119499999, 40.70214158099988], [-73.9405445659999, 40.701798044999855], [-73.94042057899993, 40.701076597999865], [-73.94193078899983, 40.70072523399989], [-73.94220058700002, 40.700890666999896], [-73.94306406899986, 40.701424434999886], [-73.94322685999988, 40.70152070899991], [-73.94382473799989, 40.70186200799988], [-73.94400503999998, 40.70196179199989], [-73.94431460199992, 40.70213334499988], [-73.94463910199993, 40.702313693999926], [-73.94544988199996, 40.70276063599993], [-73.945854991, 40.70298720699987], [-73.94625107799986, 40.703208746999934], [-73.94705205299991, 40.70366394899985], [-73.94785367299988, 40.704111382999876], [-73.948140661, 40.704271253999885], [-73.94854962699999, 40.704500430999936], [-73.94866498899984, 40.704565073999945], [-73.949253357, 40.70489593099994], [-73.94947068299989, 40.7050190189999], [-73.94983788699977, 40.70522902199988], [-73.95023693799996, 40.705473245999876], [-73.95028918399986, 40.70591343899985], [-73.95040588999979, 40.706652745999875], [-73.95053010000007, 40.70739098499994], [-73.95065919499987, 40.70816998299994], [-73.95076027199997, 40.708802859999906], [-73.95082963899989, 40.70914359799985], [-73.95087492699996, 40.70951311499991], [-73.94865533499987, 40.70972498299988], [-73.94877149699997, 40.71043225899994], [-73.94889233899988, 40.71116753499993], [-73.94901358399997, 40.71190375599989], [-73.94913082999982, 40.71261242099995], [-73.94924526199999, 40.7133203089999], [-73.94937231899985, 40.71406687799986], [-73.95162366299994, 40.7140663609999], [-73.9529104979999, 40.714060986999876], [-73.9530314939999, 40.714062846999894], [-73.95316497599985, 40.714065097999914], [-73.95325272999993, 40.71406562999987], [-73.95348472199983, 40.71407020799988], [-73.95335294599992, 40.71418365499992], [-73.95275381300002, 40.71473599099994], [-73.95210878499995, 40.715258893999874], [-73.95182429600005, 40.715475968999925], [-73.95114433699997, 40.71603452099991], [-73.95004619199985, 40.71680522799992], [-73.9498379229999, 40.716938665999855], [-73.94954034299991, 40.717121995999946], [-73.94904281699998, 40.717428518999924], [-73.94866397500004, 40.7176827709999], [-73.94832167599995, 40.71786451199997], [-73.94721341799986, 40.718560985999865], [-73.94679211599993, 40.7188216079999], [-73.94560358899994, 40.71956728599989], [-73.94440536699992, 40.72024041199988], [-73.94367547999985, 40.7206599749999], [-73.94281683699987, 40.72114362099989], [-73.94195633599989, 40.721652318999894], [-73.94150380799985, 40.7219909789999], [-73.9410941879999, 40.72230924499994], [-73.94022123699996, 40.722891504999936], [-73.93931199899998, 40.72334789399991], [-73.93891183799992, 40.72357882499994], [-73.9384885709999, 40.723808905999924], [-73.93765314599989, 40.72426555499987], [-73.93676382399988, 40.724838918999836], [-73.93539740399997, 40.725906866999956], [-73.93412337399995, 40.726855470999894], [-73.93255438599991, 40.727862889999905], [-73.93226852175046, 40.72817465801565], [-73.93200420453815, 40.72814508289344], [-73.93196085293658, 40.728125624359585], [-73.93188125492881, 40.72809538279412], [-73.93174841322003, 40.728053727696675]]]}}, {\"id\": \"80\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 81, \"Shape_Leng\": 0.197592771731, \"Shape_Area\": 0.00039956655019, \"zone\": \"Eastchester\", \"LocationID\": 81, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8236684811033, 40.889903740147645], [-73.82267708790054, 40.88961946903566], [-73.82248620406577, 40.890098054202106], [-73.823264898565, 40.88998702102998], [-73.8233660128753, 40.891075135853136], [-73.82285335053221, 40.8912068290476], [-73.82061223573477, 40.89061328894856], [-73.81957464105017, 40.890315096230374], [-73.8194872442203, 40.89028998018013], [-73.8194065171428, 40.890260434137176], [-73.81827738067017, 40.88984713076362], [-73.8176062682385, 40.88967702389679], [-73.81661400923757, 40.88943342775268], [-73.81601699496034, 40.889323833921495], [-73.81574261419216, 40.88919733695206], [-73.81533047581053, 40.88908250586605], [-73.81524197088714, 40.889060148794826], [-73.81516854879645, 40.88903776212969], [-73.81365108415028, 40.88867845028846], [-73.8124290411048, 40.88837921189019], [-73.8112683481647, 40.88809890623766], [-73.80989822392435, 40.8877405420035], [-73.80943273224275, 40.88763840646659], [-73.80872106090143, 40.887480598188404], [-73.80755944196362, 40.887108891800416], [-73.8063796861458, 40.886829379091225], [-73.80483870751024, 40.886473964049], [-73.80452614974675, 40.88640924817111], [-73.80423761712582, 40.88634168194171], [-73.80290946460593, 40.886030680167714], [-73.80192449924954, 40.88571296801209], [-73.79930206256022, 40.88506905492873], [-73.79772783123815, 40.88464812978279], [-73.79732759309121, 40.88454106108392], [-73.79628181490725, 40.884290884012586], [-73.7948690107813, 40.88391531791826], [-73.79405517398024, 40.88369996689047], [-73.79385527171416, 40.883643311166146], [-73.79300652624502, 40.88335873744268], [-73.79322870891143, 40.88282259495222], [-73.79416555799988, 40.88299321599987], [-73.79449254399985, 40.88306825099994], [-73.79744825599991, 40.88384538399988], [-73.79937520399986, 40.88435196999995], [-73.80220388299996, 40.8851211149999], [-73.803286821, 40.885415552999916], [-73.80520113699998, 40.88599995599985], [-73.80550576700001, 40.886092948999874], [-73.80571277199984, 40.8860393679999], [-73.80592498899998, 40.88599671799993], [-73.80614087499991, 40.88596540999986], [-73.80635884999995, 40.88594565999989], [-73.80657730199997, 40.88593752499988], [-73.80693113299988, 40.88592660899994], [-73.80728647099995, 40.885931813999875], [-73.80764139499996, 40.885953251999915], [-73.80799398199987, 40.88599083099993], [-73.80834232399997, 40.88604431399994], [-73.8086845939999, 40.88611329099997], [-73.80915068099985, 40.88621447699993], [-73.8097107929999, 40.88633607499994], [-73.81027053199992, 40.88645124299992], [-73.81083611999995, 40.88655115199987], [-73.81140645199994, 40.88663558699991], [-73.811980437, 40.88670433899995], [-73.81209659199986, 40.88671500999993], [-73.813567347, 40.8867787089999], [-73.81399748399998, 40.8867821319999], [-73.81442762200005, 40.88677196499989], [-73.81485665899983, 40.88674823199989], [-73.81518316800005, 40.88671965999993], [-73.81528345499991, 40.886711023999936], [-73.81536464199993, 40.886701220999875], [-73.8154787309999, 40.88668744599995], [-73.81670778599998, 40.88648095099994], [-73.81742638899989, 40.88629430599987], [-73.81841703399995, 40.886036993999916], [-73.81983275399992, 40.885669253999886], [-73.82027432999985, 40.885550699999925], [-73.82031167699992, 40.8856097399999], [-73.82077408699998, 40.886710602999884], [-73.82382288900006, 40.8877472369999], [-73.82385099099994, 40.88776831599994], [-73.82387140599998, 40.88778362799996], [-73.82403093399992, 40.88769579499991], [-73.82448935900001, 40.88742251999986], [-73.82529579999982, 40.88697637999996], [-73.82611278899988, 40.886666035999895], [-73.82672826599993, 40.88644534299988], [-73.82700490399988, 40.88634614699992], [-73.82777310599991, 40.88604701799995], [-73.82791367899995, 40.88599054299988], [-73.82953327299994, 40.885303333999936], [-73.83033698199986, 40.88496949199991], [-73.83065358999984, 40.884852938999906], [-73.83130254299985, 40.884711422999956], [-73.83233699199998, 40.88454919399989], [-73.83256211099994, 40.88451947199993], [-73.83256233799996, 40.884519444999945], [-73.83256256300001, 40.884519400999864], [-73.83278516199996, 40.88447693199989], [-73.83278538799992, 40.8844768879999], [-73.83278561299993, 40.88447683399987], [-73.83300396199995, 40.88442179699992], [-73.83300425799992, 40.88442172499994], [-73.83300454299992, 40.884421634999974], [-73.83321703599982, 40.88435437999986], [-73.83321711899991, 40.88435435299995], [-73.83321720199997, 40.88435431699985], [-73.8334219759999, 40.88427549499986], [-73.83342222600004, 40.8842753959999], [-73.8334224639999, 40.88427528899988], [-73.83361799799997, 40.88418563099992], [-73.83404698799987, 40.883990319999945], [-73.83434541999985, 40.88385001999989], [-73.83475583399996, 40.8835562939999], [-73.83560954999997, 40.88294527899995], [-73.83621837399981, 40.88239301299993], [-73.83664134299984, 40.88212002499992], [-73.8369874379998, 40.88193908299993], [-73.83782792499997, 40.88160813999987], [-73.83874273699996, 40.881265967999916], [-73.83965170899998, 40.88092595799992], [-73.84046213299987, 40.880622867999875], [-73.84127784699999, 40.8803175479999], [-73.84201454899988, 40.87987225999992], [-73.8427378519999, 40.87942490299989], [-73.84324113799987, 40.879108912999904], [-73.84239559299994, 40.87875039299991], [-73.84228241899993, 40.878701972999906], [-73.84114053199991, 40.878215735999945], [-73.8401480179999, 40.8777884259999], [-73.839711891, 40.8776115949999], [-73.8395648579999, 40.87754981599993], [-73.83925549199988, 40.87741982299989], [-73.83855197999986, 40.87711952799991], [-73.8381484219999, 40.876953298999915], [-73.83841016399981, 40.87686761499991], [-73.83924788999988, 40.876618737999905], [-73.8383540939998, 40.87492195799993], [-73.83746027699993, 40.87322264999988], [-73.83829221499984, 40.87295133899995], [-73.83740181300006, 40.87110892899991], [-73.83655628099997, 40.86935217599989], [-73.83565766499987, 40.86751275599992], [-73.83481793799997, 40.8677817219999], [-73.83391246099991, 40.868075988999884], [-73.83354911599997, 40.868193637999916], [-73.8334796669999, 40.86821612199987], [-73.83315960399985, 40.86831975599994], [-73.83312947400005, 40.86826747399991], [-73.83175316499981, 40.8659067679999], [-73.83054182799988, 40.86373143499994], [-73.83022388499991, 40.863429527999884], [-73.82954731199996, 40.862164086999925], [-73.82840758999997, 40.860948300999894], [-73.82834347399992, 40.86089202599988], [-73.82841963500002, 40.86093193999992], [-73.82911929099998, 40.86150997699985], [-73.82924089599999, 40.8616318679999], [-73.82950892699994, 40.86184121299988], [-73.82972614799992, 40.86195274499995], [-73.83012015999996, 40.862049281999894], [-73.83089329699999, 40.86221284899995], [-73.83162007899985, 40.86265937499991], [-73.83578436699997, 40.86501601099999], [-73.8361860139999, 40.86522409899994], [-73.83678824299999, 40.86554590599999], [-73.83697021099982, 40.8656431359999], [-73.83784380099986, 40.86609918799993], [-73.83875223499997, 40.866572776999945], [-73.83922085399998, 40.866820839999924], [-73.83954475699979, 40.86698673799988], [-73.83974233299988, 40.867087930999894], [-73.84064364799995, 40.86755417499991], [-73.84122362499986, 40.86785815099992], [-73.84185186599998, 40.868191924999934], [-73.842406294, 40.86847474399989], [-73.84280443499989, 40.868680752999886], [-73.84315715499991, 40.868863265999934], [-73.84358581499981, 40.86908763699989], [-73.84423146999998, 40.86942034999993], [-73.84442133700004, 40.86952229899989], [-73.84487737499977, 40.86976050699997], [-73.845172567, 40.86991469799994], [-73.84578147499997, 40.870227695999894], [-73.84608957099984, 40.870389771999946], [-73.84640766399981, 40.870554300999885], [-73.84699273699992, 40.87085288799987], [-73.84725144800001, 40.870987051999926], [-73.84792614099985, 40.87134223399991], [-73.84859676099995, 40.871670784999864], [-73.84987645199996, 40.8721018809999], [-73.85078983299995, 40.87239629199989], [-73.85108463399987, 40.87249279099995], [-73.85170987299989, 40.872697458999944], [-73.85265751899979, 40.872992186999966], [-73.85357628499997, 40.87327602099987], [-73.85363638799987, 40.87330059899996], [-73.85362131899994, 40.87331208699986], [-73.85278034299986, 40.87395322899988], [-73.85215435899995, 40.87443419899993], [-73.85206424599991, 40.874503646999905], [-73.85143247999996, 40.87498727099993], [-73.8506895569999, 40.87548667799993], [-73.85093186799989, 40.875830013999874], [-73.85172236099986, 40.87684337099995], [-73.85254854099988, 40.877091586999924], [-73.8521121739999, 40.87729787999995], [-73.85091896099982, 40.877893440999856], [-73.85138374099995, 40.878476222999886], [-73.85186563799999, 40.87905886499989], [-73.85207011699983, 40.87929765699988], [-73.85248305800003, 40.87946370499989], [-73.85203280500001, 40.88013997199988], [-73.85156600599989, 40.88082536399993], [-73.851106036, 40.88150897099993], [-73.8506512309999, 40.882191789999865], [-73.85020149399993, 40.88286986999993], [-73.84973488199986, 40.8835558029999], [-73.84927742099983, 40.88424150399993], [-73.84882044099987, 40.88491966699994], [-73.84876385000003, 40.88500531699991], [-73.84846274099996, 40.88546103199993], [-73.84836417199985, 40.885610210999886], [-73.84832190799986, 40.8856731779999], [-73.84828534100001, 40.885727657999915], [-73.84824941299993, 40.88578118699991], [-73.84820835300003, 40.88584236099989], [-73.84817178599984, 40.88589684199993], [-73.84813585799989, 40.88595036999992], [-73.84809415299982, 40.886012505999865], [-73.84804918100001, 40.88607950799991], [-73.84802165699985, 40.886120514999924], [-73.84798702099991, 40.886172117999884], [-73.84790388199997, 40.88629598399993], [-73.84785777299996, 40.88636441199987], [-73.84748708800002, 40.886914537999864], [-73.84744396900003, 40.88697853099991], [-73.84704957999995, 40.8876718159999], [-73.84572953199992, 40.88787170599993], [-73.84525554699991, 40.887943469999875], [-73.84465398699979, 40.888033088999855], [-73.84437965699982, 40.888074788999845], [-73.84432835199986, 40.88808258799989], [-73.84407903899991, 40.88812048499992], [-73.844021975, 40.88812915899988], [-73.84376573899986, 40.88816810699991], [-73.84367707000004, 40.888181673999924], [-73.84362775199982, 40.88818922099994], [-73.8431922509999, 40.88825585499997], [-73.84313196799998, 40.88826507899989], [-73.84285880200002, 40.88830687399991], [-73.84196149599987, 40.88845596699989], [-73.84105278599993, 40.888553466999895], [-73.84017616799996, 40.88868679899988], [-73.83998563399987, 40.88871867199988], [-73.83979050699996, 40.8887364029999], [-73.839593563, 40.8887395329999], [-73.83939762699993, 40.88872801299994], [-73.83920549799991, 40.88870221699993], [-73.83822700999995, 40.8886445249999], [-73.83732165099995, 40.8887419639999], [-73.83643359099989, 40.888935663999916], [-73.83554853999995, 40.88913350399993], [-73.83645878199995, 40.89151612599989], [-73.83737956161106, 40.893924741304346], [-73.83630541779426, 40.89359916544134], [-73.83525790584999, 40.89339164402785], [-73.83356849751318, 40.89280642617215], [-73.83267808688781, 40.892601826167834], [-73.83184501081413, 40.89235427900909], [-73.83095773989488, 40.89203015895004], [-73.83007965306355, 40.89178823291273], [-73.82890564213675, 40.89143945016886], [-73.82866341175486, 40.891367484944055], [-73.82796933603909, 40.89114527505283], [-73.82722429759038, 40.89091023377534], [-73.82520822804308, 40.890344349924874], [-73.82391571097277, 40.88997462522143], [-73.8236684811033, 40.889903740147645]]]}}, {\"id\": \"81\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 82, \"Shape_Leng\": 0.119875649697, \"Shape_Area\": 0.000323601079994, \"zone\": \"Elmhurst\", \"LocationID\": 82, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86768504899995, 40.744071717999915], [-73.86749135099993, 40.74330204699993], [-73.86747169199991, 40.74209572299991], [-73.8675572959999, 40.74167769699988], [-73.86756253399983, 40.740941888999956], [-73.86731779499996, 40.740261344999915], [-73.867146698, 40.73983722399991], [-73.86703973799995, 40.73957218999992], [-73.86676012199987, 40.73889036999984], [-73.8664801159999, 40.73821096399996], [-73.86609463699996, 40.73725142999991], [-73.86582955999995, 40.73664322399992], [-73.86550061599992, 40.73598933199985], [-73.86515865399998, 40.73532501399992], [-73.8648150539999, 40.73466021499993], [-73.86462193199988, 40.73428275699994], [-73.864572486, 40.73418839799996], [-73.86450230199999, 40.73407539899988], [-73.86447086199989, 40.73400727399998], [-73.86491748899991, 40.733866509999864], [-73.86524772599986, 40.73376224899993], [-73.8661043939999, 40.73349176299991], [-73.86682801299989, 40.733245779999926], [-73.86792586499989, 40.73287256899993], [-73.86810582299985, 40.73282625499992], [-73.86827761499993, 40.732767071999874], [-73.8683180679999, 40.73275361999993], [-73.86835853299982, 40.732740158999874], [-73.86845203499989, 40.732713375999886], [-73.8686094929999, 40.732653176999854], [-73.8690483869999, 40.73251391799992], [-73.86955585299982, 40.73235284299989], [-73.87054725300005, 40.73203701799992], [-73.87112231399985, 40.73187152899993], [-73.87128940500001, 40.7318162619999], [-73.87134309799984, 40.7317993569999], [-73.87149800699996, 40.73175058699994], [-73.87192058399992, 40.731613928999884], [-73.87315527199992, 40.731243969999866], [-73.87415476600002, 40.73095008599991], [-73.87481299, 40.73075339399985], [-73.87509154199992, 40.73067136299992], [-73.8753864729999, 40.73059067299993], [-73.87628302399997, 40.73031309399994], [-73.87699724199997, 40.73004835299989], [-73.87789402699987, 40.729715930999944], [-73.87836629199992, 40.72951588699991], [-73.87847478499992, 40.72946892199991], [-73.87969354099992, 40.729010158999884], [-73.88008877499988, 40.728906274999936], [-73.88049068199993, 40.72881654199995], [-73.88088484699995, 40.728743729999906], [-73.88089793699997, 40.728741311999876], [-73.88130917699985, 40.72868082699991], [-73.88172301799989, 40.72863525399986], [-73.88208519900004, 40.728610374999924], [-73.88297543399985, 40.728540950999914], [-73.8835332119999, 40.72849744799988], [-73.88387642499987, 40.7284675829999], [-73.88391980299994, 40.72846391299988], [-73.88444917100003, 40.72841912699995], [-73.88500729499987, 40.7283718949999], [-73.88559330999979, 40.728274440999954], [-73.88590069999995, 40.728211322999904], [-73.88623765899992, 40.72812701699997], [-73.8865683549999, 40.72802889099993], [-73.88661323299988, 40.72801339299988], [-73.8867298489999, 40.72797312799988], [-73.88689134399985, 40.727917365999915], [-73.88720524999994, 40.727792996999874], [-73.88720482599993, 40.72786882899991], [-73.88720430099983, 40.72801082999992], [-73.88720051699987, 40.72809239399991], [-73.88719765199994, 40.72818371199987], [-73.88720114099984, 40.728612069999926], [-73.88723573099988, 40.728932964999956], [-73.88728902599995, 40.72925284999991], [-73.887360922, 40.729570737999936], [-73.88736093299984, 40.72957078299989], [-73.88736094499983, 40.72957082699987], [-73.88745118199992, 40.72988564399985], [-73.88745122999997, 40.729885822999904], [-73.8874512889999, 40.729885994999904], [-73.88755962199991, 40.73019705599993], [-73.888059752, 40.730087229999896], [-73.88811223499987, 40.73059071099994], [-73.88817834099999, 40.73119478199992], [-73.88825349299994, 40.73189716199991], [-73.88828525999989, 40.732260539999864], [-73.88830747899983, 40.73251463099991], [-73.8883315609999, 40.73276054799993], [-73.8883492629999, 40.73294143199996], [-73.88837510099995, 40.73320535899991], [-73.88840065599999, 40.73344799199987], [-73.88844349199991, 40.73385475599986], [-73.88847908999993, 40.73415815399987], [-73.8885228599999, 40.734531159999904], [-73.88770199299982, 40.73429132299987], [-73.88751426599988, 40.734225348999864], [-73.88718676699979, 40.7341589399999], [-73.88689197799985, 40.7340991629999], [-73.88639651899999, 40.734067462999846], [-73.88598467599994, 40.734041121999894], [-73.88548975099985, 40.73400945499987], [-73.88532612599988, 40.734008121999885], [-73.88532603099999, 40.73400812199985], [-73.88532593599987, 40.73400811199986], [-73.885162107, 40.73399498199991], [-73.88500028499989, 40.73397007399991], [-73.88500017899989, 40.73397005699992], [-73.88500007199993, 40.733970027999895], [-73.88484260399989, 40.73393370699992], [-73.88484243799992, 40.73393366999992], [-73.88484227299996, 40.73393361599995], [-73.88469091999995, 40.73388646899984], [-73.88381086999975, 40.73355953599985], [-73.883529104, 40.733848278999844], [-73.88268121899998, 40.733511239999885], [-73.88182222699987, 40.73320039799993], [-73.8810869159999, 40.73293562999988], [-73.8802939189999, 40.732561644999926], [-73.87932144399987, 40.732207540999916], [-73.87844215099989, 40.73192119199992], [-73.87752195799992, 40.731593168999915], [-73.87725122499988, 40.7317941529999], [-73.87822455999998, 40.73210226899988], [-73.87708276299998, 40.73290931699991], [-73.87771964099996, 40.73341722599989], [-73.87834921099993, 40.73390580499987], [-73.87895373100004, 40.73438004699995], [-73.87931157699985, 40.734662119999875], [-73.87955435499998, 40.734853488999924], [-73.88015913099984, 40.73532715699987], [-73.88056928399986, 40.735644932999904], [-73.88066188999987, 40.73570787399994], [-73.88051536899988, 40.73575217199992], [-73.88035425599993, 40.73713112399985], [-73.88033854099989, 40.737265696999906], [-73.88033059399996, 40.73733367699989], [-73.88051819099995, 40.73736701099992], [-73.881509822, 40.73753989799992], [-73.88203267999987, 40.73764290299995], [-73.88230252700001, 40.737707700999934], [-73.88213493800001, 40.73773983999991], [-73.88178451799989, 40.73780702699984], [-73.88128967299998, 40.73789514699991], [-73.87964850399992, 40.7381904399999], [-73.87955038800004, 40.73818743799991], [-73.87945154899991, 40.73819811399992], [-73.8793568999999, 40.7382222559999], [-73.87927094299985, 40.73825836599989], [-73.87823490600005, 40.73844453599987], [-73.87797778399988, 40.73849579099991], [-73.87743058400002, 40.73860051599991], [-73.87791831799989, 40.739394956999874], [-73.87848682299996, 40.739933908999916], [-73.87863239399987, 40.74005996299993], [-73.87910739799986, 40.74044507799988], [-73.879319796, 40.740602079999896], [-73.87999120499994, 40.741070261999894], [-73.88011277199999, 40.74116823899993], [-73.88027765499996, 40.74129746199992], [-73.88100387999987, 40.74184066199995], [-73.88186399099997, 40.74173456799989], [-73.882942868, 40.741609543999836], [-73.88282912199988, 40.74099228299989], [-73.88325318299987, 40.7408594559999], [-73.88333435099989, 40.740834026999906], [-73.88416972299986, 40.74057237299994], [-73.88461489999989, 40.74043293499989], [-73.88540705199999, 40.740196431999934], [-73.88551225099985, 40.74016501899988], [-73.88559037099996, 40.74057655799991], [-73.88572588199987, 40.74128870599986], [-73.88661927199986, 40.741181715999865], [-73.88666826999996, 40.741178144999914], [-73.887511321, 40.74108252699992], [-73.88798650799986, 40.743548191999935], [-73.88886955199995, 40.74339931999985], [-73.88974416799982, 40.74326829099992], [-73.88979159800003, 40.743261185999884], [-73.88983831799999, 40.74325444899993], [-73.89071647499993, 40.74312778399992], [-73.8911102629999, 40.74520271899995], [-73.89139145499985, 40.74664100399984], [-73.89175048799991, 40.7468117269999], [-73.89143128599983, 40.74684473599992], [-73.8905012919998, 40.74694193899997], [-73.88957128399994, 40.7470390539999], [-73.88867766199992, 40.74713427699987], [-73.88778326899983, 40.74722892599993], [-73.88764663699996, 40.747244838999926], [-73.88671656299988, 40.7473414629999], [-73.88578740499986, 40.74743975399987], [-73.88485538499994, 40.747534811999856], [-73.88393579499991, 40.747630242999904], [-73.8829991829999, 40.747734401999914], [-73.88256876999999, 40.74777784499986], [-73.88207112100002, 40.74782992699986], [-73.88113747599988, 40.74792729299991], [-73.88020647999986, 40.74802497999991], [-73.87983894900005, 40.74806172499992], [-73.87927779999994, 40.74812409299993], [-73.87845779299981, 40.74821481099991], [-73.87834312599989, 40.7482226599999], [-73.87741612499993, 40.748319373999955], [-73.87692458299989, 40.74836898899989], [-73.87647952999997, 40.74841640199987], [-73.87625180199986, 40.748438849999914], [-73.87555643099991, 40.74851360799994], [-73.87543236199986, 40.74852735999987], [-73.87462946399991, 40.74861392799995], [-73.87398293699992, 40.74868566899991], [-73.8737009829999, 40.748717318999866], [-73.87330310099995, 40.748758124999895], [-73.87277356899986, 40.74881295799992], [-73.87237256399987, 40.74885155099986], [-73.87184095900001, 40.7489055769999], [-73.871356485, 40.74895695099991], [-73.87037881299995, 40.74906407799989], [-73.86942458599987, 40.74915685199991], [-73.86919188099995, 40.74876289399987], [-73.8689139579999, 40.748368876999955], [-73.86852925199995, 40.747588301999954], [-73.86843750999991, 40.747031196999934], [-73.86826502599992, 40.74631422299992], [-73.86808574299982, 40.745600582999884], [-73.86790963899986, 40.74493371599989], [-73.86787595099985, 40.74481530599989], [-73.86785097499994, 40.74471654499991], [-73.86782691099991, 40.74462945699988], [-73.86768504899995, 40.744071717999915]]]}}, {\"id\": \"82\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 83, \"Shape_Leng\": 0.105984933269, \"Shape_Area\": 0.000217463718718, \"zone\": \"Elmhurst/Maspeth\", \"LocationID\": 83, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89111026199994, 40.745202717999874], [-73.89071647499993, 40.74312778399992], [-73.88983831799999, 40.74325444899993], [-73.88979159800003, 40.743261185999884], [-73.8897441679999, 40.74326829199989], [-73.88886955199995, 40.74339931999985], [-73.88798650799986, 40.743548191999935], [-73.887511321, 40.74108252699992], [-73.88666826999996, 40.741178144999914], [-73.88661927199986, 40.741181715999865], [-73.88572588200003, 40.741288704999874], [-73.88559037099996, 40.74057655699986], [-73.88551225099985, 40.74016501899988], [-73.88540705199999, 40.740196431999934], [-73.88461489999985, 40.74043293399991], [-73.88416972299986, 40.74057237299994], [-73.88333435099989, 40.740834026999906], [-73.88325318299987, 40.7408594559999], [-73.88282912199988, 40.74099228299989], [-73.882942868, 40.741609543999836], [-73.88186399099997, 40.74173456699986], [-73.88100387999987, 40.74184066199995], [-73.88027765499996, 40.74129746199992], [-73.88011277199999, 40.74116823899993], [-73.87999120499994, 40.741070261999894], [-73.879319796, 40.740602079999896], [-73.87910739799986, 40.74044507799988], [-73.87863239399987, 40.74005996299993], [-73.87848682299996, 40.739933908999916], [-73.87791831799989, 40.739394956999874], [-73.87743058399992, 40.738600514999916], [-73.87797778399988, 40.73849579099991], [-73.87823490600005, 40.73844453599987], [-73.87927094299985, 40.73825836599989], [-73.8793568999999, 40.7382222559999], [-73.87945154899991, 40.73819811399992], [-73.87955038799986, 40.738187436999986], [-73.87964850399992, 40.7381904399999], [-73.88128967299998, 40.73789514699991], [-73.88178451799989, 40.73780702699984], [-73.88213493800001, 40.73773983999991], [-73.88230252700001, 40.737707700999934], [-73.88203268000002, 40.73764290399986], [-73.881509822, 40.73753989799992], [-73.88051819099995, 40.73736701099992], [-73.88033059399996, 40.73733367699989], [-73.88033854099989, 40.737265696999906], [-73.88035425599993, 40.73713112399985], [-73.88051536899988, 40.73575217199992], [-73.88066188999987, 40.73570787399994], [-73.88056928399986, 40.735644932999904], [-73.88015913099984, 40.73532715699987], [-73.87955435499998, 40.734853488999924], [-73.87931157699985, 40.734662119999875], [-73.87895373100004, 40.73438004699995], [-73.87834921099993, 40.73390580499987], [-73.87771964099996, 40.73341722599989], [-73.87708276299998, 40.73290931699991], [-73.87822455999998, 40.73210226899988], [-73.87725122499988, 40.7317941529999], [-73.87752195799992, 40.731593168999915], [-73.87844215099989, 40.73192119199992], [-73.87932144399987, 40.732207540999916], [-73.8802939189999, 40.732561644999926], [-73.8810869159999, 40.73293562999988], [-73.88182222699987, 40.73320039799993], [-73.88268121899998, 40.733511239999885], [-73.883529104, 40.733848278999844], [-73.88381086999975, 40.73355953599985], [-73.88469091999995, 40.73388646899984], [-73.88484227299996, 40.73393361599995], [-73.88484243799992, 40.73393366999992], [-73.88484260399989, 40.73393370699992], [-73.88500007199993, 40.733970027999895], [-73.88500017899989, 40.73397005699992], [-73.88500028499989, 40.73397007399991], [-73.885162107, 40.73399498199991], [-73.88532593599987, 40.73400811199986], [-73.88532603099999, 40.73400812199985], [-73.88532612599988, 40.734008121999885], [-73.88548975099985, 40.73400945499987], [-73.88598467599994, 40.734041121999894], [-73.88639651899999, 40.734067462999846], [-73.88689197799985, 40.7340991629999], [-73.88718676699979, 40.7341589399999], [-73.88751426599988, 40.734225348999864], [-73.88770199299982, 40.73429132299987], [-73.8885228599999, 40.734531159999904], [-73.88910318999984, 40.73459239599987], [-73.88993450699996, 40.734559955999906], [-73.89113852799989, 40.73451296999988], [-73.89138497899995, 40.734504988999895], [-73.8917501639998, 40.73448480999984], [-73.89232644100004, 40.73447014099991], [-73.89255554699984, 40.73447812599994], [-73.89278405699996, 40.73449596299988], [-73.89301088099992, 40.734523575999944], [-73.89323494499989, 40.7345608119999], [-73.89347408299997, 40.734602177999875], [-73.8937090569999, 40.7346574519999], [-73.89393780599978, 40.73472620999992], [-73.89415835499986, 40.734807791999884], [-73.89424644900001, 40.73484181299991], [-73.89440759699976, 40.73489932399987], [-73.89457531899991, 40.73494649199996], [-73.89457544899996, 40.7349465289999], [-73.89457557899993, 40.734946554999915], [-73.8947480899998, 40.73498275099996], [-73.89474827899986, 40.734982786999915], [-73.89474848099997, 40.73498281499997], [-73.89492421799993, 40.73500772199991], [-73.89510133499994, 40.73502118299989], [-73.89602931599993, 40.735017891999846], [-73.89651105199997, 40.73509674199992], [-73.89691187999993, 40.73508660199993], [-73.8978323689999, 40.73516035399992], [-73.89806882999987, 40.73518359899987], [-73.89806895999997, 40.73518360799988], [-73.89806910099988, 40.735183616999954], [-73.89830786699996, 40.73519417399996], [-73.89830799699995, 40.735194183999965], [-73.89830812699986, 40.73519418399992], [-73.8985475869999, 40.735191909999934], [-73.89854772799987, 40.735191909999855], [-73.89854787099992, 40.7351919009999], [-73.89878621299994, 40.73517681999992], [-73.89878634299994, 40.73517681099992], [-73.89878646099986, 40.73517679299992], [-73.89902209999995, 40.73514908399993], [-73.89922944799997, 40.73511159899988], [-73.89943348500005, 40.73506258999993], [-73.89963257999993, 40.73500235799992], [-73.89982513099983, 40.734931394999926], [-73.90000969399993, 40.73485031999996], [-73.90015343399983, 40.73479640499991], [-73.90028989999986, 40.73472960699994], [-73.90029003099994, 40.734729542999915], [-73.90029014899983, 40.734729471999906], [-73.90041629699991, 40.734650803999884], [-73.90041646300001, 40.73465070499992], [-73.90041661699985, 40.73465058799992], [-73.90053009099984, 40.73456135699991], [-73.90053014999994, 40.73456130199992], [-73.90062887299989, 40.73446333999988], [-73.90062893299998, 40.73446328599989], [-73.90062897999994, 40.734463231999946], [-73.9007113889999, 40.73435872699986], [-73.90091094899987, 40.7341260369999], [-73.90107970900003, 40.73391463399987], [-73.90194324799982, 40.73434002499993], [-73.9024461159998, 40.734572838999945], [-73.9028664949999, 40.73476746099994], [-73.90333476000002, 40.734996402999876], [-73.903367991, 40.73501264999993], [-73.90378660899988, 40.735217315999876], [-73.90455512499977, 40.73560527299987], [-73.90470297599984, 40.73567990299986], [-73.90516948899983, 40.73585964299986], [-73.90573589799986, 40.73463343599988], [-73.9060183039999, 40.73401073899992], [-73.90568645899998, 40.73391582499989], [-73.9059835329999, 40.73326568399991], [-73.90628903499987, 40.732400751999876], [-73.91068876399997, 40.733471081999916], [-73.91023874099996, 40.73433921699993], [-73.90991066500006, 40.73499734099988], [-73.90959716499998, 40.7356564569999], [-73.90932941199996, 40.73623244499992], [-73.90930551799993, 40.73628728499989], [-73.90928542399993, 40.73633405899988], [-73.90926002999988, 40.736403179999854], [-73.90890220399997, 40.73644071099989], [-73.90789283699988, 40.736597387999936], [-73.907619066, 40.73665029899992], [-73.90739546899985, 40.736709318999935], [-73.90672950499994, 40.73691348499992], [-73.90646283399997, 40.73702598499988], [-73.90598452499982, 40.7372186479999], [-73.90556671799989, 40.737414398999896], [-73.90486204399993, 40.73772361199986], [-73.90463288899997, 40.73782416299991], [-73.90369365299995, 40.73819895599996], [-73.90364752599992, 40.73821736899992], [-73.903575259, 40.73824323399991], [-73.90316328499986, 40.738390687999924], [-73.90268254099986, 40.73856750299995], [-73.90174321599993, 40.73890039799996], [-73.90102306499992, 40.73926666799989], [-73.90057825399985, 40.73949289599988], [-73.89974871399977, 40.74013220399991], [-73.89920950799993, 40.7405454809999], [-73.89916350300004, 40.74059856999987], [-73.89908954799988, 40.74065715499993], [-73.89896997799984, 40.74075331499988], [-73.89889421999989, 40.740814239999935], [-73.8988332499998, 40.74085897699991], [-73.89868042099991, 40.74097933999994], [-73.89818354199996, 40.74139090199992], [-73.89762150399993, 40.741841761999865], [-73.89732904499989, 40.74207636599991], [-73.897207419, 40.74217393599989], [-73.89706891399995, 40.74228451999994], [-73.89638129299982, 40.74284491999988], [-73.89576162799989, 40.74335656899989], [-73.89557809299986, 40.74350275399994], [-73.89540662199998, 40.74365743999987], [-73.89524819599988, 40.74381972999992], [-73.89499026399977, 40.744134212999896], [-73.8949120899999, 40.74428123099996], [-73.89484503999992, 40.744431661999876], [-73.89478949299988, 40.74458478799991], [-73.89474567599989, 40.74473983299986], [-73.894693961, 40.74495583499987], [-73.89466630799988, 40.74517683999988], [-73.89466369299987, 40.74540049699994], [-73.8946865809999, 40.74562431999986], [-73.89473484699982, 40.74584579899992], [-73.89480782, 40.74606246499995], [-73.89490427299994, 40.746271992999944], [-73.89496338799982, 40.74637214699989], [-73.89502251499994, 40.7464722929999], [-73.89535589999988, 40.74710841299986], [-73.89566359199999, 40.747695496999896], [-73.89583033399997, 40.74801363999995], [-73.89610755799994, 40.7485008479999], [-73.89624245599988, 40.74871954499987], [-73.89614389199978, 40.74867878599984], [-73.89583955299999, 40.74854685699991], [-73.89560209500003, 40.748449719999904], [-73.89521297399997, 40.748290552999926], [-73.89379708799999, 40.747691499999874], [-73.89240906700005, 40.747135327999885], [-73.89175048799991, 40.7468117269999], [-73.89139145499985, 40.74664100399984], [-73.89111026199994, 40.745202717999874]]]}}, {\"id\": \"83\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 84, \"Shape_Leng\": 0.233623987032, \"Shape_Area\": 0.00207375572052, \"zone\": \"Eltingville/Annadale/Prince's Bay\", \"LocationID\": 84, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.16005732699996, 40.527635041999886], [-74.15988277075512, 40.52736935827443], [-74.15994367873692, 40.52736460798024], [-74.15999341625403, 40.52734733915582], [-74.16001146160204, 40.527342810785534], [-74.16004022395717, 40.52734323910654], [-74.16005554683676, 40.52734493811437], [-74.16008419227262, 40.52736090101808], [-74.16013065929955, 40.52740191736376], [-74.16016839366702, 40.527448332314734], [-74.16019359405244, 40.52748925315915], [-74.16020751740506, 40.52750820806293], [-74.16022582487547, 40.52752444888366], [-74.16023231856387, 40.52753020940202], [-74.16025716713632, 40.527547438061426], [-74.16026450758767, 40.527552533640204], [-74.1603042631396, 40.527555747824614], [-74.1603409737616, 40.527549828131136], [-74.16040893562746, 40.52752986340302], [-74.16047493730298, 40.52750451007662], [-74.16054717701898, 40.52748726819039], [-74.16061775898544, 40.52746724757211], [-74.16152666515212, 40.527029381829415], [-74.16154625377834, 40.52702377298726], [-74.16158217423518, 40.52701347777668], [-74.16164743469216, 40.52701465879783], [-74.16169141315441, 40.52702390351627], [-74.16175881551293, 40.52704801621294], [-74.16182749073316, 40.52707161543773], [-74.161897175874, 40.5270880273121], [-74.1619669783145, 40.52710417763109], [-74.16203117512099, 40.52713034777669], [-74.16206636173392, 40.52714437519105], [-74.16209796862164, 40.52715697375597], [-74.16214390483917, 40.52718045998372], [-74.16220244797076, 40.52719607656212], [-74.16222577962044, 40.527202274889866], [-74.16227196108895, 40.52719426281089], [-74.16233039191978, 40.52716606171715], [-74.16238512563739, 40.52712627773963], [-74.1624403983033, 40.52709544379206], [-74.16250698728896, 40.52706896148613], [-74.16256536120572, 40.52703685337438], [-74.16263198944374, 40.527007327916294], [-74.16269855315247, 40.52697494795893], [-74.16276280751576, 40.52694471328185], [-74.16282757854665, 40.52691804388985], [-74.16289520141652, 40.52689026370809], [-74.1629575060745, 40.526853845457396], [-74.16301732011821, 40.52681663075126], [-74.1630784953535, 40.52678672410565], [-74.16312608537125, 40.52676093369092], [-74.16317181206539, 40.52671474203093], [-74.16322654513108, 40.5266750471119], [-74.16328589798596, 40.526638443401794], [-74.1633490917526, 40.5266030787796], [-74.16340865825354, 40.526566807919906], [-74.16346849035503, 40.52652829693557], [-74.16351595922495, 40.52649539134296], [-74.16356404014883, 40.526452166655076], [-74.16361738290826, 40.52641368125647], [-74.16367305827846, 40.526378857404445], [-74.1637200159487, 40.526350032996056], [-74.16378904863853, 40.52633450479553], [-74.16382940261882, 40.526328513449315], [-74.16386460199075, 40.52633280710802], [-74.16389790294477, 40.52634212167503], [-74.16393757258862, 40.52633604180663], [-74.16400488402428, 40.52631544294323], [-74.16407189088119, 40.5262921512552], [-74.1643995358268, 40.526209094151916], [-74.164602248092, 40.52615279838773], [-74.16474657068605, 40.52608034550147], [-74.1650341131794, 40.52582754961078], [-74.16530607172459, 40.52564918360973], [-74.16542588856903, 40.525592387035516], [-74.16592084252488, 40.52541220547592], [-74.16613400493428, 40.5253150796155], [-74.16652810000556, 40.525081427226304], [-74.16690462540775, 40.52483129006192], [-74.16755867477953, 40.52440902876843], [-74.16829560779412, 40.523949868110265], [-74.16839091003757, 40.52389048803941], [-74.16884431820021, 40.52359068215665], [-74.16898557136982, 40.52352916214104], [-74.16920066800394, 40.52343810320529], [-74.16935136134755, 40.52336054452499], [-74.16944771743327, 40.523327955346645], [-74.16955299276312, 40.52327628994602], [-74.16963714493947, 40.523229437825805], [-74.16967960342768, 40.52320579866472], [-74.16983248964985, 40.52313389981412], [-74.1699971116146, 40.523045772452306], [-74.17007269489395, 40.52302655162228], [-74.17014586058583, 40.52300793760332], [-74.17022498733132, 40.523008972567176], [-74.17032706385987, 40.523026957841964], [-74.17039614222017, 40.523053682661704], [-74.17048859810274, 40.523084172649895], [-74.17057009188476, 40.52314788073061], [-74.1705931469583, 40.52318574255418], [-74.1706100526443, 40.52321348729538], [-74.17076227805249, 40.523355701798174], [-74.17080648143494, 40.52338975028164], [-74.17089812897197, 40.52346034360391], [-74.17103027207942, 40.52352912247979], [-74.17110263696176, 40.523549706673656], [-74.1712653606599, 40.52355912147656], [-74.17134981850703, 40.523553392209635], [-74.1720632017547, 40.52332056936472], [-74.17269667263014, 40.52296873552358], [-74.17278271090665, 40.52292094996797], [-74.17342635928968, 40.522569159917396], [-74.17506810745108, 40.521494273787326], [-74.17601245479067, 40.52087596282906], [-74.17760133813562, 40.51931649499758], [-74.17758110816588, 40.51931360137952], [-74.17758082600331, 40.51929588947166], [-74.17756158077196, 40.51928751791759], [-74.17755135508193, 40.51928306954647], [-74.17753078094204, 40.51925861375083], [-74.1775491391979, 40.519235298764094], [-74.17756306042942, 40.519217618159864], [-74.17756663736948, 40.51921662739506], [-74.177570375152, 40.51921608295464], [-74.17757417880544, 40.51921599867233], [-74.17757795168454, 40.51921637668948], [-74.1775815979258, 40.51921720740124], [-74.17758502488357, 40.51921846970048], [-74.17758814548375, 40.51922013151399], [-74.17770261696565, 40.51932100548376], [-74.17777234503342, 40.519352398801956], [-74.1782708127379, 40.519923675664856], [-74.17831876690433, 40.51994578563496], [-74.17837082900233, 40.519961583533544], [-74.17842558448912, 40.51997064012644], [-74.1784936298451, 40.51997981156714], [-74.17856035946404, 40.519993500585656], [-74.17862525939358, 40.52001160174881], [-74.17870240960569, 40.52002807719417], [-74.17878128209175, 40.52003879855972], [-74.17886114175502, 40.52004366592176], [-74.17894124429743, 40.52004263391609], [-74.17902084315703, 40.52003571216092], [-74.17909919646684, 40.520022965167755], [-74.17917557396926, 40.52000451173923], [-74.1791799692992, 40.51999851587877], [-74.17918564647486, 40.51999318380906], [-74.17919243740532, 40.51998867340321], [-74.17920014102289, 40.519985118206094], [-74.17920852923737, 40.519982623480864], [-74.17921735368851, 40.519981263091864], [-74.1792263531002, 40.51998107731764], [-74.17923526101619, 40.51998207165874], [-74.17924381368908, 40.51998421667446], [-74.17925175788962, 40.51998744885473], [-74.17925885840404, 40.519991672500694], [-74.17926490499875, 40.519996762557405], [-74.17926971864468, 40.52000256831761], [-74.17927315681786, 40.520008917883], [-74.17929773247596, 40.520158179785646], [-74.17932303420199, 40.52020078552198], [-74.17935437773546, 40.52024098423037], [-74.17939137556665, 40.52027827890804], [-74.1794335702753, 40.520312208455735], [-74.17943716601998, 40.52031477241966], [-74.17946516743717, 40.52033326607575], [-74.17970770292054, 40.520493668952895], [-74.17981016948934, 40.520552337706], [-74.17998905151346, 40.520588878574365], [-74.18009755932276, 40.52059826147795], [-74.1802352477878, 40.52059804536449], [-74.18033122156834, 40.520601078045196], [-74.18049398813153, 40.520616743326855], [-74.18064008999073, 40.5206419860285], [-74.18087275843453, 40.520682306633134], [-74.18104081208887, 40.520705036287154], [-74.18115760502826, 40.52069211651063], [-74.18147441265003, 40.520583356443936], [-74.18182006847873, 40.520477649292964], [-74.18189434441686, 40.52043417820423], [-74.18192155369601, 40.52041825301177], [-74.18195667134623, 40.52041960762215], [-74.18204081284071, 40.52043420132781], [-74.18213362064886, 40.52040839326869], [-74.18233555621087, 40.52035223806129], [-74.18263745466407, 40.52022874621202], [-74.18283208173136, 40.52008772084574], [-74.18314810962117, 40.519870832266164], [-74.18325813603732, 40.51985265666168], [-74.18342370583406, 40.51988972456803], [-74.1838130349663, 40.51975173601095], [-74.18403696242659, 40.519618704645445], [-74.18417322756973, 40.51960411413418], [-74.18428548317002, 40.519578353874394], [-74.1848311647976, 40.519341358600826], [-74.1851629037099, 40.519129282625485], [-74.18546140887109, 40.51892234394305], [-74.18588730164078, 40.5186361994914], [-74.18656733191354, 40.51811585040488], [-74.18759896367169, 40.51717107849326], [-74.18779042528426, 40.517012970083776], [-74.18797576814546, 40.516850670927234], [-74.18815483487393, 40.51668431887701], [-74.18832747342314, 40.51651405522783], [-74.18849353720901, 40.51634002459576], [-74.18865288523577, 40.51616237479525], [-74.18880538221464, 40.51598125671339], [-74.1889508986793, 40.515796824182246], [-74.18906140992775, 40.51564809361495], [-74.18916738113884, 40.51549745094445], [-74.18926875557315, 40.515344976872356], [-74.18884614386798, 40.51510484487468], [-74.18884404213027, 40.515103352411614], [-74.18884230472543, 40.51510160823291], [-74.18884098256879, 40.51509966345229], [-74.1888401144064, 40.51509757506203], [-74.18883972568021, 40.51509540426325], [-74.18883982778155, 40.51509321467174], [-74.18884041771852, 40.51509107045404], [-74.1888414782028, 40.51508903444723], [-74.18884297815646, 40.51508716631711], [-74.18884487362294, 40.51508552080951], [-74.18884710905482, 40.51508414614676], [-74.18884961894243, 40.51508308261369], [-74.18885232973261, 40.51508236137739], [-74.18885516198492, 40.515082003573944], [-74.18885803269946, 40.51508201968887], [-74.18940933082028, 40.51533369213587], [-74.18945073974398, 40.51534873688931], [-74.18949485290003, 40.51535830964661], [-74.18954043873414, 40.51536214315457], [-74.18958622457698, 40.51536013038885], [-74.18963093217458, 40.515352327542146], [-74.18967331337697, 40.5153389524551], [-74.18971218498393, 40.51532037853429], [-74.19158090301478, 40.51334072190166], [-74.19162047377624, 40.513263554592754], [-74.19166863525211, 40.51318928635761], [-74.19172502696235, 40.513118473037345], [-74.1917892268341, 40.513051644613725], [-74.1918607543608, 40.51298930124345], [-74.1919390741988, 40.512931909514684], [-74.19202360017422, 40.5128798989557], [-74.19211369967013, 40.51283365882027], [-74.19220869836079, 40.512793535174744], [-74.19279996381384, 40.51220932809702], [-74.19281567384283, 40.51219380443744], [-74.19316595564403, 40.51184769858796], [-74.19330807318319, 40.51170952195666], [-74.19344318346744, 40.51156730959325], [-74.19357108999736, 40.51142126837362], [-74.19369160675681, 40.51127161074228], [-74.19380455848221, 40.51111855440297], [-74.19390978091913, 40.51096232200203], [-74.19400712105934, 40.51080314080423], [-74.19409643736367, 40.510641242362226], [-74.19521339999068, 40.5099776529556], [-74.19593485609805, 40.509856203578536], [-74.19603087689242, 40.50985296643164], [-74.19612693068868, 40.509855571490036], [-74.19622240829706, 40.50986400223228], [-74.19631670418124, 40.50987820518893], [-74.19640922029949, 40.50989809028247], [-74.19649936989559, 40.50992353139785], [-74.19658658122064, 40.50995436718267], [-74.19670276823794, 40.510006581247204], [-74.19681433917133, 40.51006436353172], [-74.19692083659992, 40.51012747714548], [-74.19702182389827, 40.51019566334], [-74.19725817895724, 40.51034094940742], [-74.19748914407793, 40.51049120095594], [-74.19771454022825, 40.510646301541996], [-74.19779692577328, 40.51072398261458], [-74.19788563701174, 40.510797493614284], [-74.19798031248911, 40.510866535003416], [-74.19808056644499, 40.51093082545619], [-74.19818144044018, 40.51098770518905], [-74.19828668015873, 40.51103977364202], [-74.19839589221958, 40.51108683618085], [-74.1984996773149, 40.511144655614984], [-74.19860831704759, 40.51119702314984], [-74.19872132060244, 40.51124370219286], [-74.19883817744555, 40.51128448185153], [-74.19895835963109, 40.51131917788564], [-74.19908132418689, 40.5113476335404], [-74.19920651556788, 40.51136972025444], [-74.19933336816597, 40.51138533824086], [-74.19946130886673, 40.51139441693824], [-74.19949239977682, 40.51138775948159], [-74.19952447399193, 40.51138490638349], [-74.1995567375377, 40.51138592827046], [-74.199588391753, 40.5113907998464], [-74.19961865305945, 40.511399400518776], [-74.19964677235937, 40.511411517384055], [-74.19967205357709, 40.511426850497486], [-74.1996938708912, 40.5114450202982], [-74.19971168422506, 40.51146557700449], [-74.2000371231315, 40.51267759982975], [-74.19992960944485, 40.512712946186575], [-74.19990247925942, 40.5127218647023], [-74.19892572793859, 40.51277730599833], [-74.19896278699422, 40.51282012058021], [-74.19900552600113, 40.51285973204682], [-74.19905347339986, 40.5128957033372], [-74.19910610016022, 40.512927637554526], [-74.19916282561876, 40.5129551823451], [-74.19922302388592, 40.512978033786645], [-74.19928603075158, 40.512995939741934], [-74.19936127407836, 40.51301019443217], [-74.19943822355076, 40.513017401531776], [-74.19951575024025, 40.513017455305196], [-74.1995927167492, 40.51301035496338], [-74.19966799389906, 40.512996204675694], [-74.19970753197478, 40.51298169765322], [-74.19974595151908, 40.512965534528036], [-74.19981408314813, 40.512947170035005], [-74.19988465622781, 40.51293533816418], [-74.19995659988416, 40.51293021845216], [-74.20002882244746, 40.51293188858514], [-74.20010022801556, 40.51294032322073], [-74.20018520017844, 40.51297068123254], [-74.20027304849728, 40.51299582229696], [-74.20036322411362, 40.51301558933668], [-74.20045516362643, 40.51302985885054], [-74.20054829261352, 40.51303854168506], [-74.2006420292192, 40.513041583590976], [-74.20073578779198, 40.513038965563155], [-74.20082898254232, 40.513030703958414], [-74.20097738056025, 40.51300730199303], [-74.20268081102346, 40.512738661754625], [-74.20323003312072, 40.51262093396144], [-74.20391668552791, 40.512473744077646], [-74.20419746366701, 40.512416025219316], [-74.20447746264747, 40.512358417799], [-74.20464466309228, 40.51232810346069], [-74.20467010671484, 40.5123234901946], [-74.20470203423241, 40.51231643884202], [-74.20474262938437, 40.51230534323708], [-74.20479737906909, 40.512293569791595], [-74.20485317338206, 40.512285123141105], [-74.20490966586989, 40.51228005573488], [-74.20498315433055, 40.512285655353494], [-74.20505574742559, 40.512296025872296], [-74.2051269144439, 40.512311091474636], [-74.20519613509917, 40.512330742019735], [-74.2053586899099, 40.51231661021616], [-74.206238443888, 40.51214565451205], [-74.20759701728615, 40.51183515615181], [-74.20780809056909, 40.51176873971944], [-74.20801578972448, 40.51169639198572], [-74.20821982874101, 40.51161821258057], [-74.20841992665042, 40.51153430916454], [-74.20861580791524, 40.5114447972802], [-74.2088072028073, 40.51134980019293], [-74.20899384778049, 40.51124944872136], [-74.20917548583245, 40.51114388105671], [-74.20935186685985, 40.511033242572715], [-74.20952274800162, 40.51091768562526], [-74.20968789397448, 40.510797369342356], [-74.20927776652347, 40.51048338085088], [-74.20922122417662, 40.51052353035443], [-74.2091899682503, 40.51049905695518], [-74.2093220046419, 40.510398628895395], [-74.20935837735608, 40.51042236182036], [-74.2093018345809, 40.510461711572255], [-74.20971374205877, 40.510777703242994], [-74.20979622955178, 40.510712935588195], [-74.2098765501006, 40.5106466003708], [-74.20999477704768, 40.5105424951943], [-74.21013598908237, 40.51040899868249], [-74.21027071719323, 40.51027165401236], [-74.21039878156121, 40.51013064453609], [-74.21052001126658, 40.50998615849728], [-74.21063424451708, 40.5098383887794], [-74.21074132886355, 40.50968753264821], [-74.21084112140274, 40.50953379148834], [-74.21093348896892, 40.50937737053439], [-74.21101830831059, 40.50921847859689], [-74.21109546625489, 40.50905732778337], [-74.21110649763585, 40.509040446116686], [-74.2111198592797, 40.509024575519604], [-74.21114086689144, 40.50900543275003], [-74.21116524905771, 40.508988783056054], [-74.2111925033283, 40.50897496954096], [-74.2112220680678, 40.50896427686242], [-74.21124713782135, 40.50895562031976], [-74.2112702602015, 40.50894423571413], [-74.21129092925099, 40.508930372161124], [-74.21130869269622, 40.50891433301951], [-74.2113231618442, 40.508896469253536], [-74.2113394556123, 40.50886265597218], [-74.21135666295963, 40.50883803057888], [-74.21137155741015, 40.50882118680084], [-74.21138832089028, 40.50880539940495], [-74.21149650880552, 40.50868783698416], [-74.2115985998613, 40.50856713987462], [-74.21169443837452, 40.5084434921724], [-74.21178387820076, 40.50831708247287], [-74.21178832276219, 40.5083133753558], [-74.2117948024027, 40.508306124722154], [-74.21180347990948, 40.50829641620653], [-74.21181466920432, 40.50827491340112], [-74.2118193208539, 40.50826237417325], [-74.21181694045957, 40.50825128014592], [-74.21181314633507, 40.50823803897107], [-74.21179478802503, 40.5082180224169], [-74.21178440618708, 40.508198349656574], [-74.21178477846054, 40.5081939209598], [-74.21179183197995, 40.50817327349501], [-74.21180447511732, 40.50816644740152], [-74.21182935119818, 40.50816562414525], [-74.21184952499443, 40.50817209263543], [-74.21186206100487, 40.50817773153637], [-74.21187243262713, 40.508178200590365], [-74.21188275583197, 40.50817730406347], [-74.2118927242892, 40.50817506855932], [-74.21190204219565, 40.50817156041406], [-74.21191043305252, 40.508166883728144], [-74.21198875142503, 40.50810220205466], [-74.21206129133526, 40.508033706992975], [-74.21212773490433, 40.50796169871397], [-74.21218779097335, 40.50788649278341], [-74.21224119637824, 40.507808418779284], [-74.21228771710221, 40.50772781884722], [-74.21228088629543, 40.50772201316972], [-74.21227541636684, 40.50771542086717], [-74.21227145982967, 40.5077082257477], [-74.21226912700006, 40.507700628426775], [-74.21226848292227, 40.50769284073455], [-74.21226954555374, 40.507685079808375], [-74.21227228526553, 40.50767756204003], [-74.21227662566783, 40.507670497040685], [-74.21228244574074, 40.50766408179794], [-74.21228958320754, 40.50765849518247], [-74.21229783906037, 40.507653892960654], [-74.21230698310823, 40.50765040345257], [-74.21231676039584, 40.50764812395292], [-74.21232689831143, 40.507647118018966], [-74.21233711418861, 40.50764741369819], [-74.21234712318729, 40.50764900274668], [-74.21235935187873, 40.507649118464606], [-74.21237144723337, 40.50764773899954], [-74.21238309749239, 40.507644899907085], [-74.21239400236918, 40.50764067436548], [-74.21240388078938, 40.50763517128831], [-74.21241247813637, 40.507628532518076], [-74.21241957281302, 40.50762092916938], [-74.21242498195419, 40.50761255721916], [-74.212469109696, 40.507543027976546], [-74.21252145049147, 40.507476901447916], [-74.21258155816881, 40.50741474129019], [-74.21260429253782, 40.50738539835731], [-74.21263191375205, 40.50735860954538], [-74.21266393606682, 40.50733484595486], [-74.21281403368066, 40.507230146332546], [-74.21296033917497, 40.50712236499311], [-74.21299767227997, 40.50708181270402], [-74.21303970639433, 40.5070440355638], [-74.21308608941341, 40.50700935001108], [-74.21313643280568, 40.50697804658764], [-74.21322298566018, 40.5069107548499], [-74.21330481575855, 40.50684010443856], [-74.21338169973609, 40.50676628821535], [-74.21345342773242, 40.50668950768308], [-74.21351980396362, 40.506609972435704], [-74.21352692087261, 40.50660119549384], [-74.21353206366952, 40.50659164901709], [-74.21353509417533, 40.50658158950686], [-74.21353593096521, 40.50657128724867], [-74.21353455155688, 40.50656101905034], [-74.21353099301393, 40.50655106080436], [-74.21351794029142, 40.50651322910223], [-74.21351359963718, 40.506474248040966], [-74.21351810019696, 40.506435277546395], [-74.21353130803675, 40.50639747722785], [-74.2135586930013, 40.506403728058004], [-74.21358454603869, 40.50641303341087], [-74.21360828614134, 40.506425184162815], [-74.21362937978549, 40.506439907244435], [-74.21364735292187, 40.50645687177696], [-74.21375171236187, 40.50654706072898], [-74.21380778551456, 40.506595519978646], [-74.21383094316958, 40.5066027625566], [-74.21385533847373, 40.5066070246834], [-74.2138803178839, 40.50660819217817], [-74.21390521220889, 40.506606233763875], [-74.21392935453652, 40.506601201906], [-74.21434250129728, 40.506518911287216], [-74.2145052736106, 40.506436853488815], [-74.21466348884147, 40.50634975767173], [-74.21481687952192, 40.506257771084194], [-74.21496518634406, 40.50616104924237], [-74.21510815859857, 40.50605975566708], [-74.2152455545974, 40.50595406160774], [-74.21537714208378, 40.50584414575232], [-74.21552102597181, 40.505732714782305], [-74.21565837406435, 40.50561657803736], [-74.21578892250353, 40.505495958655736], [-74.21591242050023, 40.505371088386966], [-74.2160286308154, 40.505242207146686], [-74.21613733021583, 40.50510956255549], [-74.21623830990282, 40.50497340946303], [-74.21633137591344, 40.50483400945808], [-74.2164163494922, 40.50469163036647], [-74.21735453442754, 40.503642896458096], [-74.21740975778067, 40.503561971545125], [-74.21751853650296, 40.50339134449908], [-74.21752648446406, 40.503380022023215], [-74.21753620569285, 40.50336953669987], [-74.21754755095681, 40.50336004949062], [-74.2175640128441, 40.50334967242369], [-74.21758240164539, 40.503341392663486], [-74.21760226300273, 40.50333541478962], [-74.2176231061741, 40.503331886505485], [-74.2176447374599, 40.503330899516875], [-74.21766629906656, 40.5033325511201], [-74.21768724213341, 40.50333679927285], [-74.21770703354522, 40.50334353583657], [-74.21772516950195, 40.503352589328784], [-74.21774118834347, 40.503363729288715], [-74.21779200715531, 40.503371648880396], [-74.21784380521872, 40.50337369060309], [-74.21789542238488, 40.50336980872706], [-74.21794570255717, 40.503360090196935], [-74.21799351958559, 40.503344752683596], [-74.21803780248969, 40.50332413970897], [-74.2180775594475, 40.50329871295202], [-74.21811190000946, 40.50326904190765], [-74.21814317872035, 40.50324771545581], [-74.21817648437909, 40.50322825936179], [-74.21822997486109, 40.50320275948046], [-74.21828692035882, 40.50318207330557], [-74.21834658055987, 40.503166469764125], [-74.218408179861, 40.50315615170689], [-74.21861332128304, 40.50310799064943], [-74.21881609351551, 40.50305430112445], [-74.21901624093961, 40.50299515081598], [-74.21921351124648, 40.50293061429154], [-74.21940765575711, 40.50286077290877], [-74.21959842973497, 40.50278571471262], [-74.21968321146456, 40.502761677014604], [-74.21977016417237, 40.50274268846472], [-74.21985877264986, 40.50272886157314], [-74.2199485118786, 40.502720278265876], [-74.22003885014124, 40.50271698940033], [-74.22012925217143, 40.502719014463445], [-74.22021918232528, 40.502726341456416], [-74.22032276952281, 40.502741577459105], [-74.2204276250922, 40.5027504140985], [-74.22053307166941, 40.502752794290394], [-74.22063842807191, 40.50274870265868], [-74.2207430136998, 40.50273816563523], [-74.22084615293313, 40.502721251288875], [-74.22094717949659, 40.502698068885834], [-74.22104544076478, 40.50266876818359], [-74.22111032227026, 40.502659437877796], [-74.22117326247195, 40.50264421809158], [-74.22123334742355, 40.5026233298297], [-74.22128970464198, 40.502597076407945], [-74.22134151577676, 40.50256583904863], [-74.22134174271483, 40.502565707936654], [-74.22138680334184, 40.50254306201155], [-74.22143526134924, 40.5025249593466], [-74.22148633479932, 40.50251169205292], [-74.22153919955196, 40.50250347421612], [-74.22159300278173, 40.50250043822093], [-74.22188242299993, 40.502674840999866], [-74.22321582499997, 40.508658610999866], [-74.22290251399993, 40.50893653999989], [-74.22663407199991, 40.51725100999996], [-74.22690993399992, 40.51917139799985], [-74.226710219, 40.51912029999991], [-74.2265150859999, 40.51905608999989], [-74.22632671899993, 40.51897915999993], [-74.226147208, 40.51889020799993], [-74.22597849399989, 40.518790184999894], [-74.22582224299991, 40.51868027699988], [-74.2256798289999, 40.51856182999988], [-74.22549932099996, 40.518505187999935], [-74.22531657499994, 40.51845289399988], [-74.22513176899994, 40.51840500099992], [-74.22494508599989, 40.518361553999895], [-74.22475670999995, 40.51832259699992], [-74.22456682499998, 40.51828816899994], [-74.22437561899991, 40.518258301999865], [-74.22418327999995, 40.51823302699989], [-74.22398999699992, 40.518212367999894], [-74.22379595899992, 40.518196345999876], [-74.22365064300001, 40.51821893699983], [-74.22350680499993, 40.518246492999864], [-74.2233647369999, 40.518278959999904], [-74.22322472599994, 40.51831627099988], [-74.22308705699992, 40.518358350999904], [-74.22295200699995, 40.51840511399991], [-74.22281984999994, 40.51845646599994], [-74.22269085399991, 40.518512303999934], [-74.22013810199992, 40.5196218829999], [-74.21802828299994, 40.520663726999935], [-74.21760570099995, 40.520871687999865], [-74.21735501199993, 40.5209300489999], [-74.21450845999989, 40.52115497999993], [-74.21354593499994, 40.521235714999925], [-74.2136147579999, 40.52153163699993], [-74.21381535699993, 40.52249465099986], [-74.21390912599999, 40.523014630999945], [-74.21398779899988, 40.52345242299989], [-74.21409806799993, 40.52409719499991], [-74.2135190009999, 40.524148838999885], [-74.21229294499989, 40.524276852999876], [-74.21201514099994, 40.52431623199986], [-74.21142518399989, 40.52441410199989], [-74.21028201899998, 40.52472106699988], [-74.2101209019999, 40.52479737599988], [-74.20997053199989, 40.52488777599995], [-74.20983404399989, 40.52499076799997], [-74.20983398499993, 40.524990811999864], [-74.20983392599999, 40.5249908669999], [-74.20971385199991, 40.5251045449999], [-74.20961205799995, 40.52522663499995], [-74.20953726399988, 40.52526850299993], [-74.20945757499997, 40.52530473499994], [-74.20937372099999, 40.525334998999874], [-74.20928646399996, 40.52535902099987], [-74.20919660199999, 40.525376579999865], [-74.209104955, 40.52538751599993], [-74.20901235799994, 40.525391729999946], [-74.2089196559999, 40.525389182999895], [-74.20882769699993, 40.52537989899983], [-74.20873731700001, 40.52536396199985], [-74.20864934299995, 40.525341516999916], [-74.20856457599997, 40.525312769999886], [-74.208441382, 40.52528672099987], [-74.20831655199999, 40.52526569499987], [-74.20819043499995, 40.52524975299988], [-74.20806338799994, 40.525238938999905], [-74.20793576699992, 40.52523328199992], [-74.20780793299994, 40.52523279999989], [-74.20768024599988, 40.52523749299995], [-74.20755306399994, 40.52524734899988], [-74.20742674699993, 40.525262338999916], [-74.20730164999993, 40.52528242199985], [-74.20674433099997, 40.52534553599984], [-74.20666645799994, 40.52535435799996], [-74.20546090199988, 40.525544336999936], [-74.20525952799987, 40.525576179999874], [-74.203687632, 40.525782940999896], [-74.20276973299998, 40.5259072179999], [-74.20217525999989, 40.52606646299992], [-74.20152661599995, 40.526265205999934], [-74.20142565199997, 40.526382032999905], [-74.20133258999991, 40.526502620999935], [-74.20124767199998, 40.52662665999985], [-74.20117111699992, 40.52675382699987], [-74.20103324199994, 40.5269194149999], [-74.200879328, 40.52707723199989], [-74.20071057499999, 40.52722595199986], [-74.2005284199999, 40.52736440399985], [-74.19984360499993, 40.52785744099988], [-74.19927554899992, 40.52819690099989], [-74.19909783799994, 40.52829964499994], [-74.19779445399999, 40.52903208399988], [-74.19608390199986, 40.52991174699987], [-74.19478532600002, 40.53051789799983], [-74.19453974799998, 40.530619741999885], [-74.1939845689999, 40.53087104699991], [-74.19313382999995, 40.53119491599988], [-74.19250928499991, 40.53149740799989], [-74.19174368799999, 40.53191669699986], [-74.19237403399998, 40.533040610999905], [-74.19267129299996, 40.533633488999904], [-74.19300900999994, 40.53432605099987], [-74.19329057000002, 40.53487778199998], [-74.19335436399993, 40.53499987599987], [-74.19341548699992, 40.53512049899993], [-74.19356025199987, 40.53540617199993], [-74.19377169299999, 40.53583384699993], [-74.19135323799996, 40.53666485899984], [-74.19030442599998, 40.53724645399989], [-74.189510226, 40.537788579999905], [-74.18903145199995, 40.53815934199989], [-74.18826921099989, 40.53886614199993], [-74.18770381799999, 40.5395269479999], [-74.18738120699997, 40.53989224299986], [-74.1870889679999, 40.54024712299989], [-74.18657647099997, 40.54085328399988], [-74.18559859999995, 40.54195759399991], [-74.18369101699987, 40.54417358399989], [-74.18285329599989, 40.545145824999885], [-74.18206740999993, 40.54604483999988], [-74.18128532999992, 40.54695757299989], [-74.18018524699995, 40.54837960899992], [-74.17991198599996, 40.548834452999905], [-74.1793348139999, 40.54988835199988], [-74.17874485199995, 40.5509958689999], [-74.17815548399993, 40.552112789999924], [-74.17746276899993, 40.553411145999846], [-74.17661991899996, 40.55499100699991], [-74.17611659899998, 40.555937540999935], [-74.17579913099998, 40.55651243799986], [-74.17559957599995, 40.55685071199989], [-74.1754729799999, 40.557065311999914], [-74.17512402999996, 40.557620037999925], [-74.1750629909999, 40.55771126399991], [-74.17475740499992, 40.55816798199986], [-74.17437344399991, 40.55870866599984], [-74.17358785199997, 40.55969842799995], [-74.17321541599998, 40.560077661999884], [-74.17245326199999, 40.56088825699991], [-74.17222993499993, 40.561135857999915], [-74.17185166499996, 40.56152092299991], [-74.17177723899994, 40.56149241599994], [-74.17146296099997, 40.56137542899988], [-74.17126370299995, 40.561323959999875], [-74.17106256099997, 40.561276946999875], [-74.17085970499996, 40.561234430999875], [-74.17065530999997, 40.561196445999926], [-74.17044954799991, 40.56116302699989], [-74.17024259599992, 40.561134199999856], [-74.17003462899996, 40.561109990999896], [-74.16982582400001, 40.56109042099991], [-74.16983437299987, 40.560949543999875], [-74.16983351999993, 40.56080851799988], [-74.16982326899996, 40.56066770699989], [-74.16980364499997, 40.560527475999876], [-74.16977469899997, 40.56038818799986], [-74.16973650599986, 40.560250200999896], [-74.16947719799997, 40.559126671999856], [-74.16931680999987, 40.55840978199988], [-74.1691669539999, 40.55770084699984], [-74.16900167899998, 40.55697216699986], [-74.16884272899996, 40.55626910699991], [-74.16868441299991, 40.555567584999864], [-74.16852396199995, 40.55485193799988], [-74.16836706799991, 40.554131592999944], [-74.16820548500002, 40.55343190299991], [-74.16810265799997, 40.55294619199988], [-74.16805222599993, 40.55271771199992], [-74.16789463099991, 40.55200100599988], [-74.16786573399993, 40.55188942699987], [-74.16776724799992, 40.551438072999865], [-74.16773525199996, 40.551291459999916], [-74.16757223299994, 40.55057473399989], [-74.16741722299999, 40.54988134599994], [-74.1672586319999, 40.54916666599995], [-74.16709947799988, 40.54844736699987], [-74.167103286, 40.54837591999994], [-74.16710008899994, 40.548304454999865], [-74.16708990499995, 40.54823337199991], [-74.16707279099998, 40.548163066999926], [-74.16704884299999, 40.54809393299987], [-74.16701819400001, 40.548026356999856], [-74.16698101499993, 40.547960714999924], [-74.16693751499993, 40.547897373999916], [-74.1668879359999, 40.54783668799997], [-74.16683255399994, 40.547778995999906], [-74.16623327299993, 40.54734247899993], [-74.16619699799993, 40.54711041099993], [-74.16574238499992, 40.545397704999914], [-74.16548821099998, 40.544652541999874], [-74.16540080799992, 40.54443272299989], [-74.16526055199988, 40.54423960899985], [-74.16478748999995, 40.543802109999916], [-74.16423140999989, 40.54330115899992], [-74.16359221499997, 40.54264214599987], [-74.16252750299994, 40.5415027069999], [-74.16102399199994, 40.539897758999885], [-74.15955749199995, 40.53833428299992], [-74.15914106399993, 40.53789864499992], [-74.15853613800002, 40.537252883999905], [-74.1584155779999, 40.537106678999876], [-74.1582881369999, 40.53696391199987], [-74.15815398399998, 40.536824769999896], [-74.15801329399991, 40.53668943899991], [-74.15786625499994, 40.53655809499985], [-74.15771305999992, 40.53643091299995], [-74.15755391099991, 40.53630806099992], [-74.15738901899991, 40.53618969999996], [-74.15721859999992, 40.53607598699992], [-74.15704288099992, 40.535967071999885], [-74.15593326499996, 40.53535375799987], [-74.15404693799991, 40.534308456999916], [-74.15397419799994, 40.53426814799993], [-74.15415844299997, 40.53407479199993], [-74.15438619499999, 40.533837150999865], [-74.15447193999992, 40.53374918899989], [-74.15502905599998, 40.53316020399993], [-74.15553157399994, 40.53263083299993], [-74.15602014099989, 40.53211509399988], [-74.1566887699999, 40.53142883599987], [-74.15679305099997, 40.531341828999906], [-74.15690813799995, 40.531262027999915], [-74.15703269699993, 40.53119055399988], [-74.15716510199995, 40.53112831799989], [-74.15730358599997, 40.53107601399994], [-74.15851066999988, 40.53056568999989], [-74.16037397400001, 40.529783172999906], [-74.16122606799999, 40.5294244229999], [-74.16134961699994, 40.5293734239999], [-74.16085825899998, 40.52871854699992], [-74.16042481199997, 40.528118481999854], [-74.16005732699996, 40.527635041999886]]]}}, {\"id\": \"84\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 85, \"Shape_Leng\": 0.0574751473562, \"Shape_Area\": 0.000143637804933, \"zone\": \"Erasmus\", \"LocationID\": 85, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94715896299984, 40.65559378699993], [-73.94703257599998, 40.65433871399988], [-73.94606690599983, 40.65441026199991], [-73.94590468799987, 40.65285464999991], [-73.94570606199984, 40.65100678399988], [-73.94667414499985, 40.65094811899995], [-73.94647882399988, 40.64910075099988], [-73.94625972599981, 40.647006214999905], [-73.94607216799986, 40.64531181199988], [-73.94598813699993, 40.64453379699989], [-73.94586879199989, 40.64342374399988], [-73.94685570699994, 40.64336214099992], [-73.94778994699976, 40.643305735999924], [-73.94754496799999, 40.64100547599989], [-73.94744296799982, 40.6400476649999], [-73.94841343400003, 40.639987463999944], [-73.94826499600005, 40.63860718999994], [-73.94923295699995, 40.63854767299993], [-73.95017361500003, 40.63851059699987], [-73.95116311199983, 40.63844818299984], [-73.95213852299987, 40.63844349599993], [-73.95307699899985, 40.638446260999864], [-73.9536000469999, 40.63842234699991], [-73.95374853799996, 40.63858633799993], [-73.95404358599993, 40.6388775489999], [-73.95427358399985, 40.63911553899988], [-73.95476763299989, 40.63962444699996], [-73.95563820699984, 40.64050392599988], [-73.95571599999984, 40.64058252599991], [-73.95707031, 40.64196067099995], [-73.95746359899987, 40.64237166999987], [-73.95763644599988, 40.64268949899989], [-73.95789394399986, 40.64401556199989], [-73.95791873699986, 40.64414326499991], [-73.95797764699991, 40.64457625599989], [-73.95802176499988, 40.6450699619999], [-73.95803664499991, 40.64530249999989], [-73.95810672200003, 40.64651800899988], [-73.95814068499989, 40.647503403999934], [-73.95815003599986, 40.64776767399996], [-73.95825517200004, 40.64853244899996], [-73.9587078989999, 40.65038727299989], [-73.9558251839998, 40.6505298079999], [-73.95480071999985, 40.65056930299989], [-73.95295084000004, 40.650655559999954], [-73.95246901399994, 40.650669454999914], [-73.95137845899984, 40.65071574499988], [-73.94957113100006, 40.65080789999991], [-73.94966851199989, 40.651697519999914], [-73.94976875000005, 40.65264232299987], [-73.94993008899999, 40.65415032699993], [-73.9500666489999, 40.65542299699989], [-73.94715896299984, 40.65559378699993]]]}}, {\"id\": \"85\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 86, \"Shape_Leng\": 0.134245282582, \"Shape_Area\": 0.000623278815249, \"zone\": \"Far Rockaway\", \"LocationID\": 86, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76669071688973, 40.61422662157113], [-73.76663280474156, 40.614188090292245], [-73.76659603795996, 40.61415620580289], [-73.76655752621322, 40.61412554090954], [-73.76648154954675, 40.61407197232676], [-73.76640026141413, 40.614023141025854], [-73.76631416907505, 40.613979351720964], [-73.76622123270072, 40.613948266132155], [-73.76612561446998, 40.613922330741474], [-73.766027805102, 40.613901678650315], [-73.76600277923055, 40.61389096579561], [-73.76597356286003, 40.61388206310417], [-73.76594390100304, 40.61387557276066], [-73.76590929681872, 40.613870072495615], [-73.76577937364263, 40.61385568693899], [-73.765713840849, 40.61384767414089], [-73.76554735771236, 40.61382929695943], [-73.7654573313066, 40.61381699220672], [-73.7653331716059, 40.61379659421426], [-73.7653079415949, 40.61379371497338], [-73.76524354206231, 40.61378942287313], [-73.7652255416092, 40.61378698222855], [-73.7652096838288, 40.613783626388305], [-73.76520308506144, 40.61378166738938], [-73.76519677155784, 40.613779232518894], [-73.76519073170368, 40.613776311198635], [-73.7651849533732, 40.61377291345589], [-73.76517409928843, 40.613764625041625], [-73.76516400829198, 40.61375420268657], [-73.76514898195599, 40.613732785002355], [-73.7651244912455, 40.61368755610694], [-73.76511369195244, 40.61367026208242], [-73.76508378272462, 40.61362660576178], [-73.76506031696873, 40.61358912366839], [-73.76504104839242, 40.61355425272034], [-73.76502554335005, 40.6135211527105], [-73.7650205004746, 40.61350668008154], [-73.76501724522896, 40.61349135519837], [-73.76501580220098, 40.61347518749322], [-73.76501614607245, 40.61345818696599], [-73.7650182993051, 40.61344034243834], [-73.76502224207296, 40.613421664926456], [-73.76502798219816, 40.613402117757175], [-73.76503554665682, 40.613381655919895], [-73.76503813998191, 40.61337674305695], [-73.76504185479139, 40.61337205044227], [-73.765046739582, 40.61336754031307], [-73.76505287604441, 40.613363112821354], [-73.76506594816784, 40.613355791192376], [-73.76509874740034, 40.613340234254515], [-73.76511217652426, 40.61333197618983], [-73.76511719002606, 40.61332774575997], [-73.76512108018139, 40.613323386717155], [-73.76512389686395, 40.613318844883956], [-73.76512568643251, 40.6133140654055], [-73.76513733345502, 40.61326178699131], [-73.76514592812889, 40.6132066746086], [-73.76515155831194, 40.613148314136886], [-73.76515223873422, 40.613132682677545], [-73.76515429478773, 40.61308589621872], [-73.76515707653228, 40.6129871130944], [-73.76515561199474, 40.612956619226765], [-73.76515206853225, 40.61292936213759], [-73.76514573378662, 40.61289949610398], [-73.76512924250447, 40.6128342651111], [-73.7651237245843, 40.6128072378575], [-73.76511218412732, 40.612738782987726], [-73.76510837815503, 40.61269529861784], [-73.76510843477006, 40.612645409277974], [-73.76511130585179, 40.61257854171328], [-73.7651175619944, 40.612523271428145], [-73.7651187061248, 40.61250393114174], [-73.76511737315961, 40.61248297248403], [-73.76511291010738, 40.61246510722905], [-73.76511124503968, 40.61246131213068], [-73.76510948711574, 40.61245729185027], [-73.7651052819417, 40.61245023177142], [-73.76510029414598, 40.6124439266564], [-73.76509451138658, 40.61243838569407], [-73.76504425370344, 40.6124018566937], [-73.76499541692499, 40.6123608104616], [-73.76499016553302, 40.61235551433581], [-73.76498561413864, 40.61234964166405], [-73.764981866706, 40.61234332199183], [-73.76497900646008, 40.61233670609842], [-73.76497772419994, 40.61233164257704], [-73.76497737641814, 40.61232627472219], [-73.76497795224616, 40.61232069247531], [-73.76497946113022, 40.6123150225078], [-73.76498505323232, 40.61230396576788], [-73.764988994485, 40.612298940401224], [-73.76499352404916, 40.61229446523158], [-73.76500647572327, 40.61228427056625], [-73.76503932155175, 40.612261887413986], [-73.7650533511491, 40.61225116421339], [-73.76508677691687, 40.612221831265], [-73.76509205464984, 40.61221639248281], [-73.76509631619642, 40.612210863838186], [-73.7650995138344, 40.612205313250726], [-73.76510161275102, 40.612199825754374], [-73.76510385766348, 40.612186645935196], [-73.76510355561396, 40.61217177723917], [-73.765093651216, 40.61211864393701], [-73.76509191260185, 40.612082088957], [-73.76508948257325, 40.612067287279444], [-73.76508687389499, 40.61205996157751], [-73.76508330393696, 40.612053623675365], [-73.76507871435258, 40.61204817461116], [-73.76507602880078, 40.61204576524109], [-73.76507307015397, 40.612043561709164], [-73.76505961679173, 40.612035230610466], [-73.76504367100014, 40.61202689609064], [-73.76498676806158, 40.6120016362585], [-73.7648851174193, 40.61195567304717], [-73.7648108217638, 40.61192368792771], [-73.764737825289, 40.61189484710063], [-73.76466588211673, 40.61186905289796], [-73.76459483901746, 40.611846242016604], [-73.76455265822071, 40.611834944713], [-73.76450473059718, 40.61182484116257], [-73.76445501191374, 40.611816328133095], [-73.76435372713276, 40.61180098289912], [-73.7641892443344, 40.611773359535405], [-73.76413999058126, 40.611767360433994], [-73.76409546481764, 40.611764134414926], [-73.76399822638167, 40.61176085527847], [-73.76384171496271, 40.61175768850187], [-73.76372229932802, 40.611756857410356], [-73.7636707130079, 40.61175539202343], [-73.76366337770511, 40.61175477366764], [-73.76365590117706, 40.61175362311462], [-73.76364027088391, 40.61174962906869], [-73.76362356072626, 40.61174347920745], [-73.76357485503232, 40.611722721134804], [-73.76355883664321, 40.61171788863157], [-73.76354389811969, 40.611715273926], [-73.7635366099537, 40.61171481699], [-73.76352858551707, 40.61171493551054], [-73.7634931119177, 40.6117183660714], [-73.76348472026018, 40.611718700113975], [-73.76347721809432, 40.6117184590145], [-73.76342545508612, 40.61171382416491], [-73.7633741106811, 40.61170750363372], [-73.76332337442068, 40.611699546897825], [-73.7632734247846, 40.611689961362096], [-73.76323678796824, 40.61168139449969], [-73.76322255859913, 40.61167742690416], [-73.76319855273074, 40.611670733898855], [-73.7631892622385, 40.61166777076174], [-73.7631581383879, 40.61165782691472], [-73.76311389262129, 40.6116421392386], [-73.763026155521, 40.61160862050955], [-73.76294809170496, 40.61157639203976], [-73.76289698451858, 40.6115534580457], [-73.76275848988121, 40.611488523961604], [-73.76271397442427, 40.6114692958422], [-73.76266773125485, 40.611450776138646], [-73.76261974879229, 40.6114329450566], [-73.76256997841372, 40.61141581405303], [-73.76251873787662, 40.61140065131079], [-73.76246108507048, 40.61138712328248], [-73.76239615019814, 40.611374840497376], [-73.7623044835184, 40.611360143582466], [-73.76229944090419, 40.61135920604586], [-73.76172413788876, 40.61184724275769], [-73.76101512745583, 40.611359712588005], [-73.76023635756965, 40.610820864950284], [-73.76076730398712, 40.610184642971014], [-73.76087049521189, 40.61019812235988], [-73.76103107637614, 40.61001265160484], [-73.76106712691544, 40.6099710131896], [-73.76117344842932, 40.609839344137896], [-73.76178242960594, 40.60915167005445], [-73.76155051905889, 40.60903136580077], [-73.76154422819783, 40.60903918643414], [-73.761468385922, 40.60913355798008], [-73.76142052284622, 40.609215307072176], [-73.76140457135281, 40.60924480256101], [-73.7612613099961, 40.609380924608374], [-73.76110660758368, 40.60955759317357], [-73.76084686113951, 40.60985879158432], [-73.760668143153, 40.61006739366464], [-73.76056913844502, 40.61018295125937], [-73.76032133549326, 40.61047700509601], [-73.75998446974822, 40.61087937784668], [-73.7594911983337, 40.61081579591091], [-73.75973571099013, 40.61015545765216], [-73.7597209903858, 40.61015159808286], [-73.75970578731508, 40.610149046396195], [-73.75968243621176, 40.61014773915803], [-73.75965913710012, 40.61014950482124], [-73.7596365810579, 40.610154291014275], [-73.75961543712306, 40.61016195577321], [-73.75959633244965, 40.61017227175256], [-73.75957983370535, 40.610184932968565], [-73.75956643026325, 40.61019956387472], [-73.75955651968803, 40.61021573050111], [-73.75950902919092, 40.61029655950143], [-73.7595011327095, 40.61030982921634], [-73.759490367694, 40.61032187057974], [-73.75947705298303, 40.61033232694855], [-73.75946158293257, 40.61034088862442], [-73.75944441573677, 40.610347302026], [-73.75936292845843, 40.6103609209809], [-73.75928012157641, 40.61036863836023], [-73.75919672932062, 40.610370385735784], [-73.75911349111223, 40.61036614761372], [-73.7591044220478, 40.61036436368268], [-73.75907022522372, 40.61035787076551], [-73.75904511473067, 40.610351667614275], [-73.75900812829903, 40.61034226101927], [-73.75898721665443, 40.6103380203889], [-73.75895697014224, 40.610333895710504], [-73.75892030237395, 40.61033419709067], [-73.75888924674067, 40.61033827495676], [-73.75885114200875, 40.610337348491576], [-73.75883136120025, 40.6103309664966], [-73.75880039730679, 40.61031920427265], [-73.75878071065507, 40.61031304863154], [-73.75874562442633, 40.61030748152985], [-73.75871310860462, 40.61030337899627], [-73.75868333119982, 40.610299993030964], [-73.75864883787516, 40.61029739978981], [-73.75861193268273, 40.61029479158901], [-73.75858266022281, 40.610292622085076], [-73.75854665481931, 40.61028963864403], [-73.75850903056701, 40.610286731869614], [-73.75848022846047, 40.61028488833536], [-73.75844677519656, 40.610282098528906], [-73.75841702630294, 40.61028080248619], [-73.75838338936416, 40.610282784858406], [-73.75834929653381, 40.61028694565507], [-73.75832147772391, 40.61029110974947], [-73.75829926069888, 40.61029509798366], [-73.75829550033363, 40.61029577460223], [-73.75829125477554, 40.6102967653235], [-73.7582577507584, 40.610304592208756], [-73.7582269006985, 40.61031053409526], [-73.75819491433819, 40.610313628260144], [-73.75815889909484, 40.61031336250973], [-73.75812105252575, 40.61030962737675], [-73.75808825488171, 40.61030541534655], [-73.75805437268517, 40.61030330904905], [-73.75802880460462, 40.61030257120322], [-73.75800190081794, 40.61030227080735], [-73.75797402450189, 40.61030240091894], [-73.75794434716126, 40.61030459086045], [-73.75791022597653, 40.61030643654317], [-73.75788559499932, 40.61030485360956], [-73.75784908810411, 40.61030013848025], [-73.75781820510514, 40.61029551761364], [-73.7577894051914, 40.6102929525283], [-73.75775408741215, 40.610289672755385], [-73.75772433905365, 40.61028798954109], [-73.75768786959213, 40.61028604839457], [-73.75765998908317, 40.610284367928585], [-73.75762074385337, 40.61028195266314], [-73.7575877699896, 40.61027748865654], [-73.75755753440913, 40.61027347185674], [-73.75752779240311, 40.61027028502671], [-73.75749483565947, 40.610270430606604], [-73.75747534560298, 40.610272037473244], [-73.75743848238733, 40.6102776156533], [-73.75740623638153, 40.61028059161272], [-73.75738300800195, 40.61027983058391], [-73.75737380421911, 40.610276298898036], [-73.75736975048056, 40.61027011285435], [-73.75735905593557, 40.61027006397637], [-73.7573383590768, 40.61028804814461], [-73.75731666566746, 40.61029105506836], [-73.75729976071221, 40.610283238964485], [-73.75729703527199, 40.61028198159578], [-73.7572956814805, 40.61028086215243], [-73.7572741228047, 40.610263076415215], [-73.75724254710155, 40.610260596467185], [-73.75721192022327, 40.61026335951094], [-73.75718049765929, 40.61026748941501], [-73.75714700339951, 40.610259791363504], [-73.75711602497977, 40.61024890969251], [-73.75708351664257, 40.61024598695955], [-73.75704800614409, 40.61024681221035], [-73.75701247800151, 40.61024904214853], [-73.756978170497, 40.610253580366674], [-73.75695101212871, 40.61025462143374], [-73.7569314958743, 40.61025376111171], [-73.75689086851851, 40.61025427846316], [-73.75686929194647, 40.61025447607804], [-73.75683630817768, 40.610252326256266], [-73.75681050712097, 40.61024754348165], [-73.75677843192791, 40.61024272983445], [-73.75674660315728, 40.61024153768482], [-73.75672036013536, 40.61023799618673], [-73.75668271214545, 40.61023174018212], [-73.75665225558224, 40.6102267948972], [-73.75662364865083, 40.61022027689952], [-73.75659942149335, 40.610205024296256], [-73.75658046085303, 40.61018140904115], [-73.75656956469595, 40.610165518230076], [-73.75655062478583, 40.6101552125715], [-73.75652563163669, 40.61017538498005], [-73.75649716738332, 40.610187977691176], [-73.7564735551872, 40.61019512289571], [-73.75643362895231, 40.61019833310154], [-73.75640467161288, 40.61019050070696], [-73.75637780586109, 40.610199007545866], [-73.7563521235751, 40.61021018261843], [-73.75632916761667, 40.61021254721898], [-73.75629508193786, 40.61021791464015], [-73.75626657727715, 40.61022204997856], [-73.75624220788647, 40.61022644755194], [-73.75621022387813, 40.610229063542874], [-73.7561919008085, 40.61023128482026], [-73.7561835188916, 40.61023230287607], [-73.75614528566263, 40.610237633477404], [-73.75611605822041, 40.61024242479879], [-73.756082024449, 40.610252889355536], [-73.75606303287732, 40.61025685585744], [-73.756039521865, 40.61025247464753], [-73.7560249513726, 40.610239791884354], [-73.75600650505237, 40.61022075972843], [-73.75598972222198, 40.61021229535099], [-73.75598031214558, 40.61021320315286], [-73.75597348586042, 40.61021887093002], [-73.75597162609579, 40.61023638270558], [-73.7559674738124, 40.61025676995049], [-73.75595506330369, 40.61028372340788], [-73.75594948952418, 40.61029241155394], [-73.75591508294892, 40.61030472027324], [-73.75588061031844, 40.61031573233037], [-73.7558553156395, 40.61031766081564], [-73.75582482776083, 40.61031160686896], [-73.75580362404574, 40.610300251060885], [-73.75577995993108, 40.61028309952759], [-73.75576265399067, 40.61026566564261], [-73.75574522332289, 40.61024663560284], [-73.755727601822, 40.61022815582459], [-73.75571074035422, 40.61020220299001], [-73.75569710325406, 40.61017980494853], [-73.75567967220253, 40.61015794783541], [-73.75566930431309, 40.61013676328769], [-73.75566121883598, 40.6101116845066], [-73.75564946697222, 40.61008766928267], [-73.75563040563364, 40.61007827222955], [-73.7556083701559, 40.61007791963748], [-73.75559880868329, 40.61008163605521], [-73.75559496353753, 40.610099161194164], [-73.75559789661662, 40.610104877175395], [-73.75560182951712, 40.61010859598401], [-73.75561374660208, 40.61011984094885], [-73.75562259040024, 40.61012818145094], [-73.75580892838548, 40.61036528895029], [-73.75587251467283, 40.61037425785732], [-73.75587637625526, 40.610374824937836], [-73.7558756121628, 40.61038028930362], [-73.75587412912144, 40.61038518505482], [-73.75569925964342, 40.610365686657296], [-73.75569746400105, 40.61035580428942], [-73.75569974469146, 40.61035262071335], [-73.75570173151115, 40.610352130388634], [-73.75571634332071, 40.610353511900286], [-73.75575727259817, 40.6103606771733], [-73.75571167394627, 40.61030222568417], [-73.75556369084146, 40.61011842794625], [-73.75550218966305, 40.610120801540084], [-73.75545094849186, 40.61011499913182], [-73.75539915499955, 40.610115783533786], [-73.75534825676537, 40.610123132822736], [-73.75529967634799, 40.61013684159296], [-73.75527347337746, 40.6101494480493], [-73.755244821347, 40.61015844226332], [-73.75521456958829, 40.6101635576193], [-73.75518361485531, 40.61016464248254], [-73.75515287474015, 40.61016166469454], [-73.75512326047287, 40.610154712525734], [-73.75509564991016, 40.610143992059655], [-73.75507086151059, 40.61012982108318], [-73.75505193518073, 40.61012716504604], [-73.75503270124402, 40.610126678385924], [-73.75501358543141, 40.610128371874794], [-73.75499501085996, 40.61013220802829], [-73.75497738866586, 40.61013810193554], [-73.75496110890589, 40.61014592313847], [-73.75494580155386, 40.610156060557486], [-73.7548940656952, 40.610192240178726], [-73.754847762547, 40.61023246843121], [-73.75480743435165, 40.61027627422274], [-73.75477355338558, 40.61032314456516], [-73.75474651642602, 40.610372530581905], [-73.75473447491667, 40.61040110303178], [-73.75473218890687, 40.61040730222276], [-73.7534583619849, 40.61051986270308], [-73.75008771088383, 40.61164055391355], [-73.74912000690117, 40.6117943267718], [-73.74821585891324, 40.61201107779471], [-73.74799307308612, 40.61231015798392], [-73.74761418914572, 40.61199801094233], [-73.74721567892574, 40.612178838825315], [-73.74694207111293, 40.611774999155706], [-73.74685539012165, 40.611574100529694], [-73.7457615018933, 40.611991654374336], [-73.7456211250613, 40.61174249247841], [-73.74536751706268, 40.611203552050156], [-73.74456658083739, 40.61011745296871], [-73.74386981824344, 40.60889248424052], [-73.74338029298465, 40.60817868098387], [-73.74307247919678, 40.60788609719336], [-73.74332648806525, 40.607538062927716], [-73.74314655207264, 40.60725986604785], [-73.74121366909178, 40.60551892016551], [-73.74029138988146, 40.60459823188922], [-73.73922512581528, 40.60414236691341], [-73.73815143112287, 40.60271044008338], [-73.73799702694568, 40.601601233828404], [-73.73830554702238, 40.59824293796482], [-73.73830871892699, 40.59820837383979], [-73.73828313185064, 40.59816556815509], [-73.73807514912781, 40.59781762609574], [-73.73808925304178, 40.59768829704553], [-73.73825953089265, 40.59762564118727], [-73.73827146702239, 40.59763212570347], [-73.73828458902938, 40.597637103992284], [-73.73829857253124, 40.59764045298744], [-73.73831307184743, 40.59764208989999], [-73.73832772854637, 40.59764197426433], [-73.73834218030606, 40.59764010893923], [-73.73835606987075, 40.59763654003658], [-73.73836905388242, 40.59763135578154], [-73.73838081136934, 40.59762468433206], [-73.73839691743511, 40.59761090433321], [-73.73840997547153, 40.59759536286517], [-73.73841965958415, 40.597578447804516], [-73.73842572808496, 40.59756058130902], [-73.73842802952184, 40.59754220928152], [-73.73843000496976, 40.59752114632383], [-73.7384365132318, 40.59750062061242], [-73.7384473758534, 40.59748119494373], [-73.73846229498801, 40.597463401951494], [-73.73852716400212, 40.597415479198546], [-73.73852725571271, 40.59739205956135], [-73.73864871908607, 40.597344265996654], [-73.73863472627416, 40.5973131923762], [-73.73862744479486, 40.5972808113883], [-73.73857716826399, 40.59726344429299], [-73.73857382102462, 40.59726837127244], [-73.73855884632226, 40.597263214810766], [-73.73850355568756, 40.59736319785864], [-73.73848900400793, 40.597358894240195], [-73.73850854977917, 40.59732414727481], [-73.73852423807804, 40.5972962589159], [-73.73854295052871, 40.5972629936935], [-73.73856037839491, 40.597232013669405], [-73.73856611221647, 40.59722181928265], [-73.73858809179461, 40.5971827453487], [-73.73861389690552, 40.5971368706904], [-73.73862486165704, 40.59711737906302], [-73.73864407374766, 40.59708322671648], [-73.73865537843393, 40.59706313023849], [-73.73865954991007, 40.59705571446355], [-73.73868423105195, 40.59701183770795], [-73.73869078548499, 40.59700018716796], [-73.7387154873913, 40.59695627309346], [-73.73871968076894, 40.596948817326414], [-73.73873823189528, 40.596955440413964], [-73.73857461617719, 40.5972446315777], [-73.73858805052276, 40.59724807773863], [-73.73858466737627, 40.59725319461563], [-73.73862641254613, 40.59726591462481], [-73.73862781419399, 40.59724144290196], [-73.73863304587005, 40.59721727466488], [-73.738760045504, 40.596994466345315], [-73.73876263070068, 40.59698988947153], [-73.73876621138677, 40.59698572037735], [-73.73877068055083, 40.59698208365872], [-73.73877590462943, 40.59697908800116], [-73.7387817274975, 40.596976822931595], [-73.73878797513527, 40.59697535614313], [-73.73879446082829, 40.59697473147134], [-73.73880099074813, 40.59697496758506], [-73.7388073697443, 40.59697605742797], [-73.7388134071765, 40.59697796842924], [-73.73881892261252, 40.596980643477615], [-73.73882375122074, 40.596984002627835], [-73.73882774869459, 40.59698794548945], [-73.73883079556741, 40.59699235422792], [-73.73883280078138, 40.596997097085264], [-73.738882685792, 40.597046439024254], [-73.73893943794074, 40.59709126249315], [-73.73900235793, 40.59713101516901], [-73.73907067045685, 40.59716520721108], [-73.7391435337668, 40.597193417297234], [-73.73922005002483, 40.597215297816014], [-73.73929927638089, 40.59723057914991], [-73.7393802365885, 40.5972390729983], [-73.73946193303371, 40.59724067469769], [-73.73954335902998, 40.59723536451154], [-73.73962897165448, 40.59721149558431], [-73.73971099873867, 40.597181181110095], [-73.73978860759591, 40.59714472882414], [-73.7398610103941, 40.597102508768735], [-73.73992747215387, 40.597054949536634], [-73.73998731821021, 40.59700253391936], [-73.74003994106113, 40.59694579400635], [-73.74008480653316, 40.59688530578333], [-73.74010543603896, 40.59684794614482], [-73.74011797914022, 40.596808555742726], [-73.74012211431592, 40.596768144328436], [-73.74011773557845, 40.59672774782453], [-73.74010495518917, 40.596688401769434], [-73.74008410077897, 40.59665111477248], [-73.7400627513751, 40.596612011334976], [-73.74003427764893, 40.59657564054258], [-73.73999926623331, 40.59654275171455], [-73.73995843844962, 40.59651402243284], [-73.73991263544504, 40.596490044582794], [-73.7398628008646, 40.59647131215955], [-73.73980996140897, 40.59645821109084], [-73.73980543695005, 40.59645411751473], [-73.73980187243376, 40.596449505667046], [-73.73979936788847, 40.59644450496627], [-73.73979799359657, 40.59643925574317], [-73.73979778812388, 40.59643390530248], [-73.739798757236, 40.59642860378924], [-73.73990407814098, 40.596373376738526], [-73.74003703042294, 40.596281233554514], [-73.74006954777121, 40.59623570144109], [-73.74009714262667, 40.59620864729175], [-73.74011818218939, 40.59620376463578], [-73.7401537610051, 40.59620261274076], [-73.74025075069052, 40.596210227619046], [-73.74030905351357, 40.59618693954036], [-73.74039153834977, 40.59618219567315], [-73.74039294483362, 40.59623643247847], [-73.7403475207975, 40.59627454091755], [-73.74034900829132, 40.596307823429576], [-73.74033279524065, 40.596318879119906], [-73.74028393005828, 40.596409979862386], [-73.74028204054135, 40.596480232284165], [-73.7403601274953, 40.59677499554009], [-73.74038232489124, 40.59679939856366], [-73.74040960334438, 40.59682059858816], [-73.74044118144295, 40.59683798831977], [-73.74047615460643, 40.59685106961344], [-73.74051352099603, 40.59685946774307], [-73.7405522102172, 40.596862942136205], [-73.74055955620689, 40.596861262801525], [-73.74056715739592, 40.59686048575863], [-73.74057482428888, 40.59686063037905], [-73.74058236575246, 40.59686169305737], [-73.74058959378046, 40.5968636473014], [-73.74059632818015, 40.59686644439246], [-73.74060240106479, 40.59687001459978], [-73.74060766103945, 40.59687426891931], [-73.74061197697394, 40.59687910129197], [-73.74061834076392, 40.59689615557484], [-73.74062102518255, 40.596913767850374], [-73.74061996040308, 40.59693148000702], [-73.74061517411833, 40.596948831335396], [-73.74060679082126, 40.59696537051088], [-73.74060921413064, 40.5969730997288], [-73.74061334857008, 40.59698039499648], [-73.74061907043597, 40.596987038036445], [-73.74062620852796, 40.596992830086], [-73.7406345492715, 40.59699759784441], [-73.7406438431081, 40.597001198658276], [-73.74068530132834, 40.59699944738718], [-73.74072577925703, 40.59699239499603], [-73.74076411879872, 40.59698024325797], [-73.74079922303956, 40.59696333984049], [-73.74080307578484, 40.596959059846064], [-73.74080769951492, 40.59695524918273], [-73.74081299715759, 40.5969519878522], [-73.74081885749266, 40.59694934432396], [-73.74082515748691, 40.596947374096615], [-73.7408579579203, 40.59695348046154], [-73.7408916351951, 40.596955329519346], [-73.74092524512245, 40.596952869428755], [-73.74095784540083, 40.59694616916218], [-73.74098852203713, 40.59693541657071], [-73.74099306854177, 40.59693438287048], [-73.74099733176617, 40.59693279586703], [-73.74100119672039, 40.596930698365874], [-73.74100455915755, 40.59692814694157], [-73.74100732838409, 40.59692521041234], [-73.74100942970746, 40.596921967983796], [-73.74101080644976, 40.59691850711215], [-73.74101142147724, 40.59691492114546], [-73.74101125820096, 40.5969113068061], [-73.7410103210251, 40.59690776158178], [-73.74100863522756, 40.59690438109566], [-73.74100624627896, 40.59690125652778], [-73.74100321861472, 40.596898472155324], [-73.74100179815535, 40.59690098534124], [-73.74099985226756, 40.596903284989516], [-73.74099743448241, 40.59690530783691], [-73.74099461131338, 40.5969069982345], [-73.740991460426, 40.5969083096797], [-73.74098806850112, 40.5969092060944], [-73.74098452885066, 40.596909662818334], [-73.74098093885092, 40.59690966728692], [-73.74097739726272, 40.59690921937726], [-73.74097400151531, 40.59690833141136], [-73.74097084502581, 40.59690702781715], [-73.74096836219971, 40.59690780998126], [-73.74096574706559, 40.596908278886644], [-73.74096306716235, 40.596908422423475], [-73.74096039170094, 40.5969082368846], [-73.74095778977808, 40.596907727061904], [-73.74095532859113, 40.59690690612187], [-73.74095307170278, 40.59690579526628], [-73.74095107739952, 40.59690442318411], [-73.74094939718606, 40.596902825310785], [-73.74094807445584, 40.59690104291308], [-73.7409471433695, 40.59689912202301], [-73.74094662797329, 40.59689711224983], [-73.74095619595559, 40.596879774582014], [-73.74096938067086, 40.59686387549446], [-73.74098582174452, 40.59684984955056], [-73.74100506979715, 40.59683808011593], [-73.74104337975957, 40.596828361748585], [-73.74108315342914, 40.5968230665761], [-73.74112351700808, 40.59682231092948], [-73.7411635837387, 40.5968261114096], [-73.74118553063452, 40.59682842558906], [-73.74120768649566, 40.59682840021409], [-73.7412296241696, 40.596826035773596], [-73.74125092071091, 40.59682137785288], [-73.7412518896136, 40.5968182426191], [-73.74125351590618, 40.596815269672824], [-73.7412557561001, 40.5968125385127], [-73.74125855029123, 40.59681012217198], [-73.74126182376057, 40.596808085265245], [-73.74126548897357, 40.59680648226088], [-73.74126944791945, 40.59680535602441], [-73.74127359473346, 40.59680473667221], [-73.74127781852663, 40.59680464076604], [-73.74128200635205, 40.59680507087047], [-73.74128604622405, 40.596806015484376], [-73.7412898301138, 40.59680744934806], [-73.74129325683718, 40.5968093341191], [-73.74129623476148, 40.59681161939718], [-73.7413031074393, 40.59681721848292], [-73.74131109804405, 40.59682187629571], [-73.7413199874715, 40.5968254651174], [-73.7413295319721, 40.59682788654153], [-73.74133946983298, 40.59682907417223], [-73.74134952855663, 40.59682899544437], [-73.741357765473, 40.59683365187186], [-73.74136495054117, 40.59683922929574], [-73.74137090842818, 40.59684559161302], [-73.74137549374574, 40.59685258356743], [-73.74137859460036, 40.59686003453777], [-73.74138013532291, 40.59686776270181], [-73.74138007831561, 40.59687557947273], [-73.74137842496869, 40.59688329410187], [-73.74139847117411, 40.59690473687481], [-73.74142192576541, 40.596924066747974], [-73.74142175588474, 40.596926193469066], [-73.74142207004247, 40.596928310628876], [-73.74142285881184, 40.59693035469747], [-73.74142409852432, 40.596932264337724], [-73.74142575197932, 40.59693398224683], [-73.74142776956123, 40.59693545687521], [-73.74143009072836, 40.59693664397304], [-73.74143264582878, 40.59693750791897], [-73.74143535819077, 40.59693802278842], [-73.74143814642423, 40.59693817313153], [-73.74144092686164, 40.596937954437045], [-73.74144361607023, 40.59693737326711], [-73.74144613335383, 40.596936447061154], [-73.74146607403786, 40.5969243241283], [-73.74148827777292, 40.59691475599329], [-73.74151218637357, 40.59690798319158], [-73.74153719879638, 40.59690417598596], [-73.74156268624876, 40.59690343008671], [-73.7415653811586, 40.596904102346244], [-73.74156818656417, 40.59690441667493], [-73.74157102127162, 40.59690436397548], [-73.74157380323898, 40.596903945773114], [-73.74157645195119, 40.59690317417132], [-73.74157889074901, 40.59690207150173], [-73.74158104904993, 40.596900669677765], [-73.74158286438781, 40.59689900927075], [-73.74158428422388, 40.59689713833607], [-73.7415852674651, 40.59689511102202], [-73.74158578565496, 40.59689298600293], [-73.74158582379607, 40.59689082478082], [-73.74158538078444, 40.59688868990544], [-73.7415973671053, 40.596866178110496], [-73.74161388139287, 40.59684539835504], [-73.7416345064164, 40.59682687563051], [-73.7416587210913, 40.596811077904945], [-73.74168591364314, 40.596798404300436], [-73.74171539706398, 40.5967891750093], [-73.74174642646953, 40.59678362320475], [-73.74177821791754, 40.59678188915041], [-73.74179629237203, 40.596767927614245], [-73.74181096661388, 40.59675181474797], [-73.74182180790893, 40.596734025712834], [-73.74182849655536, 40.596715085099284], [-73.74183083531219, 40.59669555145679], [-73.74182875521423, 40.59667600082241], [-73.74182451436532, 40.59665904436297], [-73.74181686745597, 40.596642796441266], [-73.74180600036239, 40.596627651996705], [-73.74179187067021, 40.596613720892826], [-73.74177502220874, 40.596601673530444], [-73.74175588115652, 40.59659181464368], [-73.74173493168095, 40.59658439361002], [-73.7417127036924, 40.59657959814215], [-73.7417037086474, 40.59658266372059], [-73.74169418510041, 40.59658459013184], [-73.74168437953736, 40.596585327517175], [-73.74167454574275, 40.596584856791765], [-73.74166493823267, 40.596583190138816], [-73.74165580566556, 40.59658037069413], [-73.7416473844081, 40.59657647142976], [-73.74163989241653, 40.59657159326545], [-73.74163352359669, 40.59656586245637], [-73.74159574046283, 40.59654887614067], [-73.74155506097243, 40.59653636656588], [-73.74151239169112, 40.59652861251448], [-73.74115347299904, 40.596540239701305], [-73.74094313576546, 40.59654705211172], [-73.7409414631812, 40.59655175339061], [-73.7409387973399, 40.59655618037783], [-73.7409352110988, 40.59656021208484], [-73.74093080246882, 40.59656373832592], [-73.74092569193654, 40.59656666272959], [-73.74092001917201, 40.59656890537255], [-73.74091393921043, 40.59657040496381], [-73.74087487147645, 40.59655573103651], [-73.7408331572998, 40.59654618609025], [-73.74078994633359, 40.596542033185855], [-73.74074642948112, 40.5965433867782], [-73.7407038060766, 40.596550209562075], [-73.74066111186984, 40.596562375683], [-73.74061674572245, 40.59657034051727], [-73.74057141605768, 40.59657397688532], [-73.7405258466843, 40.5965732267228], [-73.74052059824265, 40.59657185597492], [-73.74051574138403, 40.59656981191844], [-73.74051142341858, 40.59656715655039], [-73.74050777531151, 40.596563970409], [-73.740504907711, 40.596560350130865], [-73.74050290759243, 40.59655640552021], [-73.74050183561943, 40.59655225621845], [-73.74050172430566, 40.59654802807515], [-73.74050257702658, 40.59654384933126], [-73.74050436791904, 40.596539846729335], [-73.74050704266466, 40.59653614166962], [-73.74051052013718, 40.596532846527516], [-73.74051469486409, 40.59653006124564], [-73.74054563704607, 40.59649026444071], [-73.74057097581999, 40.596448228434745], [-73.74059043582385, 40.59640441005918], [-73.74060380558622, 40.59635928551451], [-73.74061093982257, 40.5963133451953], [-73.74061525971155, 40.59627245088244], [-73.74061079345005, 40.596231565707086], [-73.74059766007412, 40.59619177920585], [-73.74057620958588, 40.59615415163541], [-73.74054701362383, 40.596119685717944], [-73.74053082459658, 40.59612457940447], [-73.74051142026894, 40.596124535589944], [-73.74050176132286, 40.59611342145448], [-73.74049382946058, 40.59607396063], [-73.74047930089631, 40.596067764059136], [-73.74045917623138, 40.59606911296055], [-73.74044371824415, 40.59607014906631], [-73.74041452651389, 40.59609226945322], [-73.74030143748699, 40.59606613039738], [-73.74029022946958, 40.59603775498193], [-73.7402223263083, 40.59603513663349], [-73.74018670507758, 40.59604738195101], [-73.73992792985005, 40.596061587313315], [-73.73991492767561, 40.596078813973264], [-73.73981229328994, 40.59627456218199], [-73.73957904997508, 40.596376337848696], [-73.73953153596668, 40.596535232658084], [-73.73960988055163, 40.596763437540325], [-73.73946370303052, 40.596878420092], [-73.73948220989597, 40.59697762988142], [-73.73948781808575, 40.59697593479486], [-73.7394966087387, 40.5970203707591], [-73.73957870164308, 40.596980082262924], [-73.73958631560401, 40.596976346846645], [-73.73969757669637, 40.596921742250395], [-73.739705436777, 40.59691637748516], [-73.73975111242625, 40.596885195054845], [-73.73976010849762, 40.59687905384428], [-73.73981105771489, 40.596844270047], [-73.73983374627073, 40.59682117497731], [-73.73984650947756, 40.59680818193786], [-73.7398504168596, 40.59680420424124], [-73.73988306007304, 40.59677097418953], [-73.73979486697488, 40.59665124795648], [-73.7397915823696, 40.59664678841122], [-73.73973369321162, 40.59656819869528], [-73.73970950101993, 40.5965734083625], [-73.73965518724205, 40.59658510556953], [-73.73964853372279, 40.59656715206082], [-73.73969621382008, 40.5965552954201], [-73.73974724022848, 40.59654260483508], [-73.73975390043238, 40.59655202638888], [-73.73977602333036, 40.59658331679867], [-73.73977984293752, 40.59658871957666], [-73.73980566371779, 40.59662524227227], [-73.73981102756485, 40.59663283012041], [-73.73983351575824, 40.59666463683548], [-73.73983784246828, 40.596670757620046], [-73.73986424975341, 40.59670810956276], [-73.73986828861138, 40.596713821256586], [-73.73988420556657, 40.59673633511134], [-73.7398966257443, 40.596753902602096], [-73.73990994690175, 40.596772744520315], [-73.73979634028939, 40.596882675627874], [-73.73969866310973, 40.59694676530583], [-73.73969333890277, 40.59695025998227], [-73.73968736688427, 40.5969541787169], [-73.73963571050848, 40.59698224810066], [-73.73958504738884, 40.597005331340895], [-73.73957401693768, 40.59701035737161], [-73.73954671965845, 40.597022793742596], [-73.73951745995797, 40.597036125964884], [-73.73950885867268, 40.59704004409154], [-73.73949652209808, 40.597042578162096], [-73.73945528659308, 40.59704556873792], [-73.73944033499654, 40.59704665209554], [-73.73943095850046, 40.59704658024129], [-73.7394064975102, 40.59704639091866], [-73.73939810824884, 40.59704632565623], [-73.73936639462225, 40.59704113854307], [-73.73924172058136, 40.59700250359185], [-73.7392166950812, 40.59699474980928], [-73.73920841735517, 40.596992183229275], [-73.73914434758687, 40.59696717151354], [-73.73913856349672, 40.59696488952607], [-73.73908184815292, 40.59694251078459], [-73.73907319397102, 40.59693909580605], [-73.73901571928859, 40.59691402762481], [-73.73892850726293, 40.59687548946645], [-73.73894704511588, 40.59686319031951], [-73.73897266456387, 40.59687376554155], [-73.73904606722695, 40.59690608057036], [-73.7390548498901, 40.59690974227924], [-73.73910946934664, 40.596932518180445], [-73.73911830379909, 40.59693620194847], [-73.73916751427618, 40.59695672131168], [-73.73917528803719, 40.596959963069125], [-73.73922317497744, 40.59697993158166], [-73.7392355386211, 40.59698408506149], [-73.73928808236302, 40.597001738128064], [-73.73929610421013, 40.59700443194724], [-73.73934283227533, 40.597020130123084], [-73.7393979636384, 40.597029543017925], [-73.73944749042397, 40.59702863204729], [-73.73944497449672, 40.59697756908488], [-73.73945583288493, 40.596978282586775], [-73.73943957124749, 40.596897402307455], [-73.73933278706397, 40.59691318437725], [-73.73910683002381, 40.596804204365206], [-73.73912307710913, 40.59678451979306], [-73.73894690694097, 40.59676316642691], [-73.73889308019422, 40.59679519246272], [-73.73884642112992, 40.59682210248287], [-73.73883031859042, 40.59680480879736], [-73.73862636110987, 40.5968598115698], [-73.73858431403262, 40.59686094777612], [-73.73860200843423, 40.59688440787326], [-73.73852427660151, 40.59691381296728], [-73.73848679652468, 40.59698768159276], [-73.73844962513597, 40.59698266793203], [-73.73826706178608, 40.59735449138662], [-73.73822344542049, 40.59735112738425], [-73.73812220887619, 40.597386102069386], [-73.73813316409253, 40.59728564583394], [-73.73803749195454, 40.5959131511688], [-73.73773567396867, 40.59478499423731], [-73.73770476300949, 40.59466942244817], [-73.73763679389783, 40.594415399837956], [-73.7376468285491, 40.59441408134138], [-73.7376479822356, 40.59441483987047], [-73.7376539685101, 40.594418788648994], [-73.73766540626953, 40.594433196371575], [-73.73767387555077, 40.59445767474746], [-73.73768487329951, 40.59447281876304], [-73.73770589774527, 40.594491832320074], [-73.73771202922693, 40.59449473670989], [-73.7377385952103, 40.59450733277356], [-73.73776863283119, 40.59451761311257], [-73.737804303554, 40.594528042498645], [-73.73784632428159, 40.594537277966346], [-73.73786658749289, 40.59453944160252], [-73.7380612247574, 40.594546116708024], [-73.7381686471151, 40.59454268675029], [-73.73825859616619, 40.594533417676665], [-73.73829117183324, 40.594525640075524], [-73.73836396686413, 40.5945045895076], [-73.73839885733365, 40.594494187093225], [-73.73842547703404, 40.594481153379654], [-73.73844602742476, 40.594464593914914], [-73.73845944615474, 40.59444079733848], [-73.73846105156619, 40.594389857304975], [-73.73847314107705, 40.594373414277754], [-73.73848811323195, 40.59436637171761], [-73.7385086654192, 40.59439126284707], [-73.73852246399807, 40.59441244741771], [-73.73852462983481, 40.594435578016046], [-73.73853764634308, 40.5944481524451], [-73.73856957886235, 40.59445413186819], [-73.73865814740493, 40.594480503520934], [-73.73867835139818, 40.59448881573182], [-73.73874923699084, 40.59451800985268], [-73.73877153079722, 40.59452520803298], [-73.73877815501379, 40.59444891235183], [-73.73878418806137, 40.59441333800904], [-73.73877005026992, 40.59441215745293], [-73.73877753514914, 40.59435500821518], [-73.73881662516276, 40.59435976495602], [-73.73881112762062, 40.594416435556795], [-73.73879338493006, 40.59441445740864], [-73.7387913253894, 40.59444907681658], [-73.73878644978721, 40.594529565560244], [-73.73883432550788, 40.59454304930055], [-73.73889349191411, 40.59455720483895], [-73.73900196778274, 40.5945862322444], [-73.73907212417535, 40.594596476534605], [-73.73910729317714, 40.59459655638528], [-73.73916764971513, 40.59459300170851], [-73.7391954400391, 40.59458593253053], [-73.73925589391581, 40.59455778446143], [-73.73927692202334, 40.59454553044771], [-73.73928729631761, 40.594533009676084], [-73.73928512794382, 40.59450742092013], [-73.73927780495323, 40.59448280202721], [-73.73928109875231, 40.59446533930758], [-73.73929430628124, 40.594453005560645], [-73.73930406142904, 40.5944539455886], [-73.73931941105549, 40.59446194143417], [-73.73932662910883, 40.59447398009358], [-73.73933385925235, 40.59448601878002], [-73.73934856041359, 40.59451206802289], [-73.73936472405694, 40.594523171714336], [-73.73939659170452, 40.59453070929159], [-73.73942749489994, 40.594537200626306], [-73.73945887454893, 40.59454245833712], [-73.73949465784864, 40.594545339725514], [-73.7395276572985, 40.594550187793], [-73.7395646533139, 40.59456157857743], [-73.73956502573112, 40.59451945535183], [-73.7395522688195, 40.59451943799904], [-73.73955249471965, 40.594426474456995], [-73.73958763297539, 40.5944265256111], [-73.73958740902066, 40.594518270376916], [-73.73957519758912, 40.59451825258765], [-73.73957493121861, 40.59456474305713], [-73.73958864884943, 40.594568966581505], [-73.73961965222445, 40.59458004187891], [-73.73964106023107, 40.59458245037932], [-73.73967453660494, 40.59458634372677], [-73.7396953946558, 40.59458939428621], [-73.73968123124612, 40.59450355888254], [-73.73995801592365, 40.59450076878709], [-73.73995275016908, 40.594636709613916], [-73.73998678749325, 40.59463285989959], [-73.74001746414017, 40.59462486123154], [-73.74004742639178, 40.59461463597358], [-73.74006098155643, 40.59460121264042], [-73.740072950435, 40.59457959082199], [-73.74008908945062, 40.594530657145285], [-73.74010803139174, 40.59451126617564], [-73.74013580364566, 40.59451157273891], [-73.74014139121738, 40.59451754659306], [-73.74013382011009, 40.594566031909075], [-73.74014286699474, 40.59458153196842], [-73.74018445146005, 40.59461192931863], [-73.74021122864659, 40.5946251649163], [-73.74027412098314, 40.594647071736055], [-73.74032962528412, 40.59466121825713], [-73.74036408761145, 40.59466969808866], [-73.7403919369275, 40.59467450780171], [-73.740419032672, 40.59467880098619], [-73.7405061595838, 40.594699475119604], [-73.74062832064128, 40.59472165227207], [-73.740651588069, 40.59472003049213], [-73.7406816322751, 40.59470719329445], [-73.74069758978415, 40.59469583776901], [-73.74070773498553, 40.594672014279574], [-73.74073935409572, 40.594670545350084], [-73.74076790338214, 40.59466251420721], [-73.74079275823493, 40.59465024325229], [-73.74083151673132, 40.59461116602116], [-73.74083862240138, 40.59459764797131], [-73.74083755553995, 40.59457433879751], [-73.74084054278646, 40.59454742062226], [-73.74085331865267, 40.59450336828137], [-73.74086643220772, 40.59449105173324], [-73.7409017059638, 40.5944882487506], [-73.74091740970762, 40.594505682310846], [-73.7409156959837, 40.59451789891997], [-73.7409053346814, 40.594554734495134], [-73.74088874912545, 40.59459718403629], [-73.74089520274937, 40.59461157919189], [-73.7409401621547, 40.594636607841124], [-73.74096717985206, 40.59465173387741], [-73.74102990553376, 40.59467729775638], [-73.74108997034794, 40.5946973351586], [-73.74112633401295, 40.59470622295994], [-73.74115233831625, 40.594714737780315], [-73.74118104185605, 40.594730633342806], [-73.7412611561602, 40.59476875309056], [-73.74129498270835, 40.59478271580657], [-73.7413182062532, 40.59479259288114], [-73.74137287779189, 40.59480814176124], [-73.74146578067547, 40.59480802697643], [-73.74150058113821, 40.59480557506032], [-73.74156118998467, 40.59480118137393], [-73.7415916774986, 40.594795685093054], [-73.74162091692338, 40.59478649313513], [-73.7416408809993, 40.59477144541601], [-73.74165498494048, 40.59474732500191], [-73.74165774557974, 40.594675225647016], [-73.74166197274509, 40.594654370132346], [-73.74167638401231, 40.59464243402208], [-73.74169766300271, 40.59464174405287], [-73.74171730107162, 40.5946498837432], [-73.74173218981556, 40.59466416328336], [-73.74174014598705, 40.59468094923735], [-73.7417376616572, 40.594727771793366], [-73.74174977224077, 40.59474256660813], [-73.74177588782638, 40.59475904205792], [-73.74180151991479, 40.59477199527085], [-73.7418488883716, 40.59479449776606], [-73.74191490578423, 40.5948181498347], [-73.74196836844072, 40.59483786620653], [-73.74202229932045, 40.59485881545237], [-73.7420847378624, 40.594879271628734], [-73.74214660413497, 40.59489483679036], [-73.74222603097068, 40.59490588332426], [-73.74225988208669, 40.59490455501748], [-73.74232229363123, 40.59489547413439], [-73.74242926334367, 40.59486851780651], [-73.7423874258487, 40.59472925606014], [-73.74261837534051, 40.5947368068089], [-73.7427604397498, 40.59474431144505], [-73.74277459101098, 40.594790810567766], [-73.74279563385534, 40.59480522068059], [-73.74280966836622, 40.594808377363236], [-73.74283993997102, 40.5948193134478], [-73.74287488141269, 40.59482601158156], [-73.74291424530844, 40.59483215904353], [-73.74297445988655, 40.594831969904114], [-73.74300969211616, 40.59482781672778], [-73.74304104329526, 40.59481910675949], [-73.74306717345472, 40.594804126358085], [-73.7430994470635, 40.59479203181844], [-73.74313202933371, 40.594785793298854], [-73.74316618149479, 40.59478908436607], [-73.74319069326747, 40.59479254268566], [-73.74322060926447, 40.594794580862654], [-73.74328649698155, 40.59479687178487], [-73.74331966019673, 40.594795982400626], [-73.74334951603205, 40.594795148276276], [-73.7433744085999, 40.59479188108], [-73.74340201112331, 40.59478428844293], [-73.74342845401051, 40.59477374753133], [-73.74345292601652, 40.59475678330378], [-73.74346867615824, 40.594734584393905], [-73.74347021013651, 40.59471074143902], [-73.74345368532323, 40.59463620381585], [-73.74345571627173, 40.59461200245114], [-73.74346642638272, 40.59459756333345], [-73.7434844228204, 40.594590516990586], [-73.74350188978259, 40.59460093027141], [-73.74351563536557, 40.59462094375595], [-73.74353464516977, 40.59464182422796], [-73.74358643822328, 40.59465040476105], [-73.74398868331865, 40.594664628781324], [-73.74414761988396, 40.59466528029151], [-73.74451941399786, 40.594653238665394], [-73.74461964043492, 40.59464731092186], [-73.74472011812641, 40.594633891375196], [-73.74506981727697, 40.59457025243548], [-73.74514756425398, 40.59454824576606], [-73.74524623370854, 40.594519592943556], [-73.7453919908722, 40.59448200412548], [-73.74576700628545, 40.594399419083096], [-73.74588152239984, 40.594375682763335], [-73.74597487469723, 40.594359696390406], [-73.74606380498341, 40.5943537429195], [-73.74633803574713, 40.59432789142369], [-73.74642483662306, 40.594310351461175], [-73.74657572303371, 40.59429685271138], [-73.74662298522834, 40.59428658867152], [-73.74664920374843, 40.594280895008744], [-73.74675560739023, 40.594243694148254], [-73.74683948398362, 40.59420899253098], [-73.74687686045283, 40.59416285051162], [-73.74692257807818, 40.59410975743752], [-73.7469877100281, 40.594090143051616], [-73.74705451165403, 40.59408158190828], [-73.74712039646221, 40.59408129602737], [-73.74712916578538, 40.59408031567385], [-73.74715073481806, 40.594077894414376], [-73.74718527378876, 40.594075449584224], [-73.74721470033396, 40.59407279480699], [-73.74727767310603, 40.59406444960137], [-73.7473428051014, 40.59405715550065], [-73.74741792429445, 40.59403588753023], [-73.74754294839317, 40.593938229632755], [-73.74760282517072, 40.59391154380944], [-73.74765788192904, 40.593881993330605], [-73.74775402443677, 40.59384325554806], [-73.74786754387374, 40.59379812802524], [-73.74793001816698, 40.593768510763624], [-73.7480968163944, 40.59362318914886], [-73.74814789619523, 40.59360773255468], [-73.74830456677556, 40.59358869579131], [-73.7485123044914, 40.59354496191742], [-73.74895028827507, 40.59340538311677], [-73.74918491752474, 40.59327851543212], [-73.74923723499444, 40.59322905616515], [-73.7493317329166, 40.59316846807457], [-73.74938620259219, 40.593140561804695], [-73.74950615034876, 40.59307241936008], [-73.74963881698261, 40.59301136471626], [-73.74971695134795, 40.59297329973449], [-73.74980984069296, 40.59291610140136], [-73.74999063621222, 40.592810305377085], [-73.75022538179961, 40.59270823678695], [-73.75037372333664, 40.59261259832038], [-73.75064164562772, 40.59241330843881], [-73.75074472812084, 40.59233727580714], [-73.75083983764087, 40.59224470968875], [-73.75092985909811, 40.59211661561272], [-73.75103368083255, 40.5919686499819], [-73.7511534608213, 40.591808282565914], [-73.75131940156294, 40.59161696509918], [-73.7513584707491, 40.59157819191071], [-73.75147213875667, 40.591472616011835], [-73.75154772812445, 40.5914220086854], [-73.75163854884676, 40.59138023139625], [-73.75175127901, 40.59134251756634], [-73.7518092615259, 40.591318923257134], [-73.75187164604968, 40.59130308266377], [-73.75208039307323, 40.59127735483698], [-73.75223475926043, 40.59125640779624], [-73.75235329623182, 40.59123572593447], [-73.75257921234481, 40.591183460100424], [-73.75263919939694, 40.59117343123855], [-73.7526758422693, 40.59116746764309], [-73.75271719872373, 40.59116105531522], [-73.7527511645542, 40.59115695908706], [-73.75278463818802, 40.5911519521024], [-73.75284115059198, 40.59114564370349], [-73.75286727165079, 40.59114163862416], [-73.75293148208962, 40.59113669769829], [-73.75299279987641, 40.59112775191378], [-73.75305170830612, 40.59111573930899], [-73.75311212524532, 40.59109805751856], [-73.75313483211737, 40.591091000521054], [-73.75317176753282, 40.59108262362464], [-73.7532677610427, 40.59103419232235], [-73.7532993144495, 40.59102111299008], [-73.75338089874752, 40.59099413754488], [-73.75343228226795, 40.59097256273838], [-73.75345522617467, 40.59096509221967], [-73.75349168911401, 40.59095322941643], [-73.75352988157324, 40.59094648500059], [-73.75355794778345, 40.59094362749599], [-73.75359114847551, 40.590942088063755], [-73.75365382039506, 40.59093445022406], [-73.75372199695761, 40.59093001207567], [-73.75374861134372, 40.59093015137155], [-73.75378173188352, 40.590930797643225], [-73.75381427976859, 40.59092982266386], [-73.75385060867407, 40.590922462664544], [-73.75390232431987, 40.59090975840006], [-73.75394637764788, 40.59090268497197], [-73.75398349406454, 40.59090256575355], [-73.75406364294247, 40.59090606949451], [-73.75426138209615, 40.590924504024116], [-73.75433376822224, 40.590931871638745], [-73.75438328246368, 40.590936246149674], [-73.75440305171236, 40.590937999555855], [-73.75447308112479, 40.59094029175186], [-73.75453500647647, 40.59093992950473], [-73.75457182984402, 40.590932848601135], [-73.75458831317428, 40.59092815611133], [-73.754611810522, 40.590921461202385], [-73.75464087599367, 40.59091655299063], [-73.754676719751, 40.59091243272865], [-73.75473532047647, 40.59090675743492], [-73.75477000716293, 40.590902581399654], [-73.75480303925438, 40.59089834704099], [-73.75483172480736, 40.5908942568565], [-73.75486536388713, 40.59088870129376], [-73.75495964643265, 40.59087474627759], [-73.75498975915815, 40.59087129822395], [-73.75501860929782, 40.59087026073922], [-73.75505442583602, 40.59087019268324], [-73.75508259331284, 40.590868937948734], [-73.75521122361054, 40.59086253882331], [-73.75528072638602, 40.59086046307248], [-73.75533106525006, 40.590859120464394], [-73.75537689752801, 40.59085974006773], [-73.75541498594802, 40.59086207358678], [-73.7555038248681, 40.59086978122559], [-73.75553036586294, 40.59086745089645], [-73.75556957875274, 40.59086577810023], [-73.75571850574238, 40.59086532131434], [-73.75577045063802, 40.59086727791204], [-73.75580079510667, 40.59087153819075], [-73.75583628405437, 40.59087361341051], [-73.75586357310075, 40.590873508888954], [-73.75592115806575, 40.590871109225816], [-73.7559546362305, 40.590874198049825], [-73.75602036595409, 40.59088630496383], [-73.75605024598252, 40.59089781371333], [-73.75607723387715, 40.59091181133894], [-73.75610479724386, 40.59092691805504], [-73.7561318000948, 40.59093718765598], [-73.75616278331961, 40.59094419618295], [-73.75619478638879, 40.59095315172029], [-73.75623154401504, 40.590964153494056], [-73.75625999802007, 40.590971464046554], [-73.75629179897616, 40.59098095003038], [-73.75632502885645, 40.59099079101608], [-73.75635783011097, 40.59100126166589], [-73.75638878883085, 40.59101180914955], [-73.75642199513254, 40.59102143478119], [-73.75645231340867, 40.591029352010295], [-73.75648210550776, 40.59103602403912], [-73.75651228449009, 40.59104303126979], [-73.75654462693286, 40.59105018748697], [-73.75657827475327, 40.59105541902863], [-73.75661224706137, 40.591058750117995], [-73.75664012057273, 40.59106032211658], [-73.75667054293734, 40.59105904546611], [-73.75669866323629, 40.591060860556546], [-73.75675525782049, 40.59107102144743], [-73.7568249426794, 40.59108706250415], [-73.75691806701087, 40.591111907381965], [-73.75694734161965, 40.59112124510418], [-73.75700245101464, 40.59113681495333], [-73.75703419417235, 40.59114937160005], [-73.75706297359264, 40.591158131775074], [-73.75711328332214, 40.5911748798808], [-73.75714554578028, 40.59118430461713], [-73.75717655040047, 40.59119183544117], [-73.75720108105554, 40.59119960454252], [-73.7572211627071, 40.591203239987166], [-73.7572407067868, 40.59120501102491], [-73.75730257288988, 40.59121411946992], [-73.75733639369399, 40.59122053039772], [-73.75739663764747, 40.59123729047855], [-73.75745186056763, 40.591251186057356], [-73.75748350555223, 40.59126146463545], [-73.75751466374516, 40.59126914831187], [-73.7575576526475, 40.5912784687471], [-73.75762331559734, 40.591292692514706], [-73.75767587486234, 40.591305013951896], [-73.75773334295016, 40.591321768671946], [-73.7577894312456, 40.59133456574357], [-73.75785285721308, 40.59135045845573], [-73.75789189861872, 40.59135698827763], [-73.75795637515171, 40.59135995234955], [-73.75798801159918, 40.59136273799488], [-73.7580128027422, 40.591367013969894], [-73.75804097641971, 40.591373854573675], [-73.75811181401144, 40.59137839818963], [-73.75813777460367, 40.591379498054145], [-73.75816643522687, 40.591378953928334], [-73.75829781442644, 40.591389955448335], [-73.75856200201768, 40.59142856761067], [-73.75871537076522, 40.59145097941272], [-73.75878945264255, 40.59146647080124], [-73.75885981629324, 40.59147504724865], [-73.75892785511789, 40.59147917052124], [-73.75904521901246, 40.59149193468693], [-73.75906249579332, 40.591493620078055], [-73.75914462613561, 40.59150163466767], [-73.7592283619613, 40.59150980685747], [-73.75928714482238, 40.5914932070092], [-73.75938010754977, 40.591504577464306], [-73.75947486517903, 40.59152563198178], [-73.75960054182336, 40.591552496045956], [-73.7596717559462, 40.591567602582735], [-73.75969570621598, 40.59157267789848], [-73.75976734920664, 40.591586605527176], [-73.75979294806714, 40.59159164523478], [-73.75989658654491, 40.591612045000495], [-73.75994809255582, 40.59161805117161], [-73.76007031564953, 40.591623169161466], [-73.76016560077571, 40.591619225565495], [-73.76023595718786, 40.591620057084604], [-73.76030924413571, 40.59162328130438], [-73.76038205920773, 40.59163013268843], [-73.76041824810322, 40.59163199116775], [-73.76054954013327, 40.59163800155924], [-73.76065279969747, 40.59164727620251], [-73.76078573008135, 40.59166078133847], [-73.76084382257397, 40.59166493654668], [-73.76095548743734, 40.59168271095222], [-73.76108305835777, 40.59169203527808], [-73.7612089514512, 40.591708047789616], [-73.7612787298498, 40.59171832330683], [-73.76134311678325, 40.59172663385036], [-73.76141229292924, 40.59172329321627], [-73.76157030497218, 40.59169996479758], [-73.76172098788959, 40.591681222340064], [-73.76229978054906, 40.59164087015591], [-73.76236416229632, 40.591637683712996], [-73.76243967044952, 40.59163394611224], [-73.76276146869947, 40.59162420917798], [-73.76295275572143, 40.591611177431], [-73.76303822435928, 40.59159747704603], [-73.76308430596761, 40.59158769299509], [-73.76347652151587, 40.59148431951076], [-73.76354057868238, 40.59147225835319], [-73.76376223423699, 40.591439709016804], [-73.76385620499985, 40.59175407799996], [-73.763912182, 40.592039344999925], [-73.7640494709999, 40.59273904699989], [-73.76414300399986, 40.593205171999905], [-73.76429039299992, 40.5939396449999], [-73.76440183099987, 40.59457168299989], [-73.76362979899989, 40.59466694999985], [-73.76277845599988, 40.59475654399989], [-73.762981598, 40.59589946999991], [-73.76299716399983, 40.595987023999875], [-73.76301370999988, 40.596081549999916], [-73.76319972899987, 40.59714459099989], [-73.7633344969999, 40.59782287399992], [-73.76255289900001, 40.59831811799991], [-73.76181486399989, 40.59895137099994], [-73.76128753399995, 40.59954927099993], [-73.76154124799989, 40.599846560999886], [-73.76158779199984, 40.59990109899993], [-73.76165797299986, 40.599973069999876], [-73.76336096499998, 40.599105042999945], [-73.76386778499985, 40.59972731299991], [-73.76444582499997, 40.6004352409999], [-73.76654204999997, 40.599434441999904], [-73.76599689399987, 40.59872106199988], [-73.76618488799977, 40.59865569699989], [-73.76636755699995, 40.59857174399987], [-73.76654034499992, 40.598469425999895], [-73.76669864799996, 40.598349814999935], [-73.76683818399978, 40.598214847999884], [-73.76695532899997, 40.598067273999895], [-73.76704742299998, 40.59791042699988], [-73.76711296699986, 40.5977480419999], [-73.76715168699985, 40.59758390099991], [-73.76716449199986, 40.597421597999904], [-73.76715321499994, 40.59726433399986], [-73.76712577699985, 40.59712732699992], [-73.76711717700006, 40.5969881329999], [-73.767127717, 40.59684901399989], [-73.76715706599988, 40.59671223899986], [-73.76717720099997, 40.5966231809999], [-73.76721443599983, 40.596535878999845], [-73.76726813399988, 40.59645316599989], [-73.76733661099998, 40.59637765199988], [-73.76742090699994, 40.59630520399985], [-73.76771811799993, 40.596261309999925], [-73.76776273800007, 40.59654905699993], [-73.76795322999999, 40.5974260569999], [-73.76809925802587, 40.59784597005487], [-73.76808948115853, 40.59786141186906], [-73.76806678092747, 40.59791051677544], [-73.76803688591761, 40.597983245938565], [-73.76802839158346, 40.59800389559855], [-73.7680081292507, 40.59804505428412], [-73.76800671591312, 40.59804723953703], [-73.76800499434478, 40.59804938798138], [-73.76800049037357, 40.59805366530517], [-73.76797967142298, 40.59806790552089], [-73.76797547820793, 40.5980713022647], [-73.76797219339382, 40.598074627296874], [-73.76795030970385, 40.598099643719394], [-73.76793987039952, 40.59811524830741], [-73.76793669062471, 40.59812275107256], [-73.76793516521563, 40.59812979898456], [-73.76793501234442, 40.59813318512241], [-73.76793526089354, 40.59813648160468], [-73.76793592288796, 40.59813970638148], [-73.76793698739951, 40.59814284200773], [-73.7679383588175, 40.598145699986695], [-73.76794006182584, 40.59814845041087], [-73.76794453414253, 40.59815366358615], [-73.76795043655434, 40.5981585114185], [-73.76795788671298, 40.598163101364314], [-73.7679735866159, 40.59817025642905], [-73.76801725401066, 40.59818645491745], [-73.76804356253508, 40.59819727867728], [-73.7681062491692, 40.59822061090645], [-73.76813551682744, 40.598233358664366], [-73.76814108283848, 40.59823635075125], [-73.76814644628057, 40.598239828265406], [-73.76815671757392, 40.59824835899617], [-73.76816672252433, 40.59825915854686], [-73.76819439128423, 40.598293236805254], [-73.76820476656758, 40.59830277560051], [-73.76821010481004, 40.598306676911044], [-73.76821563403921, 40.59831007400694], [-73.76823995164165, 40.59832227984716], [-73.76826549483133, 40.59833244429065], [-73.76829168302484, 40.598340350555176], [-73.76831805693715, 40.598345851628416], [-73.76839693702648, 40.598356879620326], [-73.76843153022395, 40.598360182655995], [-73.76846416625214, 40.59836214836891], [-73.76848525498117, 40.59836194760225], [-73.76853280608204, 40.59835872931362], [-73.76855413008346, 40.59835900613289], [-73.76856615207234, 40.59836036321092], [-73.76857915096119, 40.59836281253829], [-73.76863287860479, 40.59837725624019], [-73.76864559342032, 40.59837999263673], [-73.76865718894716, 40.598381754101055], [-73.76870119627677, 40.598385372366586], [-73.7687851600615, 40.59838854966014], [-73.7687971144413, 40.59838899638874], [-73.76883903289473, 40.59839186455648], [-73.76888469830521, 40.59839453099987], [-73.76890472549103, 40.59839705641079], [-73.76889176380759, 40.59844309700456], [-73.76893146064765, 40.59846776382943], [-73.7689799009702, 40.598453009537664], [-73.76900728611821, 40.598485604222866], [-73.76901642123492, 40.59849509605623], [-73.76902297252785, 40.59850001801285], [-73.7690308024206, 40.59850421194153], [-73.76903989945066, 40.5985076875357], [-73.76905027444222, 40.598510429069286], [-73.76907474247716, 40.598513711060114], [-73.76910411185766, 40.59851403962738], [-73.76918917846884, 40.59850927592098], [-73.76922076994283, 40.598505710169235], [-73.7692324467041, 40.59850439136927], [-73.76925114099092, 40.59850352860201], [-73.7692560185019, 40.59850384413721], [-73.76926090713974, 40.59850469227168], [-73.76926586446571, 40.59850607965441], [-73.7692709962844, 40.59850804469486], [-73.76930264378103, 40.59852579417719], [-73.76930921498162, 40.598528905160784], [-73.7693153972632, 40.59853115945395], [-73.76941553633635, 40.59856154676938], [-73.76958690877635, 40.598614372870564], [-73.76961311333139, 40.59862099039476], [-73.76967401111023, 40.59863268407702], [-73.76970071987024, 40.59864013052369], [-73.7697177925864, 40.59864685537092], [-73.76973574379937, 40.598655554470966], [-73.76980002538818, 40.598693091134756], [-73.76982828415264, 40.59870722425542], [-73.76993238412204, 40.59874703640101], [-73.76997616916023, 40.59876378404294], [-73.77001557170144, 40.59878377364488], [-73.77005175466385, 40.598805467437195], [-73.77007316919209, 40.59882066602884], [-73.77011765933023, 40.59885549768401], [-73.77017682425011, 40.598896986313505], [-73.77027408869283, 40.59895978547217], [-73.77028053833372, 40.598963310122805], [-73.7702923541327, 40.59896976420225], [-73.77031339088283, 40.59897801872385], [-73.77033634816473, 40.598984223332735], [-73.77036024774937, 40.5989881078579], [-73.77038390667956, 40.598989496709535], [-73.77040662910537, 40.598988389335254], [-73.77042764768771, 40.59898480212829], [-73.77043736736675, 40.59898212012738], [-73.7704464760719, 40.59897885172398], [-73.7704683455717, 40.598968259780925], [-73.77049004964127, 40.59895419260177], [-73.77051209601598, 40.59893651399356], [-73.77055524769148, 40.59889729232011], [-73.77057450087534, 40.59888111170244], [-73.77059633936453, 40.59886564874629], [-73.77061795344157, 40.59885383212521], [-73.77064482646136, 40.5988414128369], [-73.77066458708046, 40.59883227583165], [-73.7706956755438, 40.59882045129777], [-73.77072503590009, 40.59881308010125], [-73.77073894803341, 40.59881121728292], [-73.77075252398886, 40.598810542248174], [-73.77078720521955, 40.59881244001284], [-73.7708235249685, 40.598818499960174], [-73.77086131911726, 40.59882870584586], [-73.77090026970177, 40.59884297578282], [-73.77093086474137, 40.59885759912505], [-73.77095834516885, 40.59887485333718], [-73.77098168371995, 40.59889409104954], [-73.7710002428972, 40.59891476786325], [-73.77101070690539, 40.59892979083342], [-73.7710202328009, 40.59894585748658], [-73.77103672115965, 40.598981532790056], [-73.77105014425419, 40.59902249776632], [-73.77106245222207, 40.59907466205954], [-73.77106471104767, 40.59909118264467], [-73.77106497919539, 40.599109770434104], [-73.77105912334285, 40.59917048108935], [-73.77105828189103, 40.59919198386322], [-73.77105997150446, 40.59921233940989], [-73.77106455326746, 40.59923046704428], [-73.77107042772171, 40.59924356265017], [-73.77107908782979, 40.5992572858428], [-73.77109056842833, 40.599271861682865], [-73.77112720101427, 40.59931407892318], [-73.77113690037558, 40.59932803855502], [-73.7711437557007, 40.59934115487388], [-73.77115249540978, 40.59936616151324], [-73.77115807337863, 40.59939324240757], [-73.7711605243632, 40.599422452073746], [-73.77115986181745, 40.599453968287726], [-73.7711557024036, 40.59948176876551], [-73.7711467245462, 40.59951232305057], [-73.77113534274287, 40.599541045635874], [-73.77110581598824, 40.59960751824148], [-73.7710939261192, 40.59963985912255], [-73.77108674822033, 40.59966934546764], [-73.77108437410661, 40.599696573028744], [-73.7710871200451, 40.59979180908827], [-73.77109550611405, 40.599895809473374], [-73.77110624222676, 40.60000025648079], [-73.77111419742232, 40.60006435310831], [-73.77111914447596, 40.60008602941045], [-73.77113493157037, 40.60013350978704], [-73.77113905426296, 40.600154076572565], [-73.7711393089279, 40.60015922794802], [-73.77113891311716, 40.60016454002929], [-73.77113621586231, 40.600175944268535], [-73.77111624260783, 40.600219057797865], [-73.77111292431104, 40.60022883090313], [-73.77111126381162, 40.600237733977686], [-73.77111054072343, 40.600255805977156], [-73.77111187877777, 40.60027562941646], [-73.77112817318242, 40.60036459802649], [-73.77113028499313, 40.6003858091471], [-73.7711302089682, 40.60040484589827], [-73.77112590127966, 40.60045504211783], [-73.77111953843155, 40.60050186572423], [-73.7711110344584, 40.600546028547384], [-73.77110030469566, 40.600587863299594], [-73.77105282368589, 40.60074930554536], [-73.77093512613936, 40.601152308285606], [-73.77091239686352, 40.6012104189018], [-73.7708673092232, 40.60129269130317], [-73.77073733384829, 40.60153550225482], [-73.77074244654368, 40.6015817158007], [-73.77075283595511, 40.60162593119095], [-73.77077908903945, 40.601656115444754], [-73.77082643937688, 40.60168232507241], [-73.77084737913124, 40.60172455230795], [-73.77084198990094, 40.601758692487834], [-73.77083917528664, 40.60181091609616], [-73.77080471264298, 40.60186910380908], [-73.77078080648036, 40.60192329503747], [-73.77074383318684, 40.60194531862101], [-73.77071223428813, 40.60193721985457], [-73.77068865897292, 40.6018949885093], [-73.77065189914457, 40.601854737184134], [-73.7706175881491, 40.601868730478955], [-73.77060167895182, 40.601896822434604], [-73.77058311147579, 40.60193093629699], [-73.77054596552425, 40.60200317950911], [-73.77050356147332, 40.60207139551764], [-73.77044533066837, 40.60214359776326], [-73.77039741390276, 40.60228211207762], [-73.77034443950986, 40.60235834196636], [-73.77027546523118, 40.60248877892168], [-73.77026469363348, 40.60255504888111], [-73.7702328460693, 40.60261926802468], [-73.77022211742572, 40.60267348564586], [-73.77018588274089, 40.602748823325115], [-73.77023981935425, 40.602779886972485], [-73.770288763362, 40.602815430513466], [-73.77033207783559, 40.60285499141114], [-73.77036919910219, 40.60289805484946], [-73.77039964407953, 40.60294406043146], [-73.77039682621424, 40.602967653542606], [-73.77038875846117, 40.60299053226202], [-73.7703756783063, 40.603012023104945], [-73.7703579707886, 40.60303149344139], [-73.7703361571643, 40.60304837011899], [-73.77031087956307, 40.60306215633484], [-73.77028288208662, 40.60307244626041], [-73.77025298890355, 40.6030789369883], [-73.77025152605673, 40.60307914816689], [-73.7702408196256, 40.603079820671525], [-73.77019158542402, 40.60307987485414], [-73.77017898589695, 40.6030814251088], [-73.77017334566268, 40.603082674825664], [-73.77016805885243, 40.60308424858135], [-73.77016577454368, 40.60308518050411], [-73.77016484016474, 40.60308569194633], [-73.77016404658139, 40.60308623164771], [-73.77016339512008, 40.60308679743287], [-73.77016288511355, 40.603087391478326], [-73.77016251656966, 40.60308801160619], [-73.77016227802436, 40.6030886676779], [-73.77016219214377, 40.60308935203229], [-73.77016223713731, 40.60309007350495], [-73.77016242403113, 40.60309082156364], [-73.77016274026438, 40.60309160556471], [-73.77016379701433, 40.60309328196956], [-73.77016543416676, 40.603095113327136], [-73.77016908515742, 40.6030982810755], [-73.77017429474198, 40.60310202924763], [-73.77020712683694, 40.60312326562137], [-73.77021459537006, 40.603129620043916], [-73.77021718414757, 40.6031325077352], [-73.77021903074902, 40.603135240317805], [-73.77024599008497, 40.60318723578144], [-73.7702718933096, 40.60324447062789], [-73.77029746555604, 40.60330841369927], [-73.7703263389662, 40.60338816794179], [-73.77033952766757, 40.60342845736956], [-73.77035205529455, 40.60347217530938], [-73.77037750437735, 40.60357862357121], [-73.77040862113736, 40.603719023818094], [-73.77041588743104, 40.60375180813819], [-73.77043556340162, 40.60384686154675], [-73.77045185260809, 40.60393739745087], [-73.77046357455843, 40.60401935994732], [-73.77047139482067, 40.604099008939336], [-73.7704744997586, 40.60417177661492], [-73.77047540249363, 40.60431832145352], [-73.77047105451693, 40.60441441715326], [-73.77047158381761, 40.604453032116616], [-73.7704746817832, 40.604476235797506], [-73.77047953587791, 40.60459547958392], [-73.77048062362502, 40.60471477786669], [-73.77048893638917, 40.604756983507215], [-73.7704891070582, 40.60479966184392], [-73.77048113198757, 40.60484190531993], [-73.77046518075221, 40.60488281562423], [-73.7704415925433, 40.60492152279403], [-73.77041086895497, 40.60495720371523], [-73.77037366331972, 40.604989099625804], [-73.77033076681522, 40.60501653225206], [-73.7702682118161, 40.605049823403746], [-73.77021002042594, 40.60508743150987], [-73.77015670371276, 40.60512902628116], [-73.77010872993579, 40.60517424241473], [-73.77006652043227, 40.605222682803124], [-73.77003044591854, 40.605273922020714], [-73.77000082323222, 40.605327510060555], [-73.76999690762636, 40.60535766861426], [-73.76999420287767, 40.60538803833292], [-73.76999408117247, 40.6054166658867], [-73.76999691052053, 40.60544251718156], [-73.77000011297223, 40.605456093322296], [-73.77000524884225, 40.60547101725451], [-73.77003373282179, 40.60553707304843], [-73.77003849335085, 40.60555459008922], [-73.77004034008233, 40.60557070360015], [-73.77004040748075, 40.605609272572], [-73.77004041197118, 40.605611542607335], [-73.77003782936973, 40.6056444871658], [-73.77003224550127, 40.60567385147461], [-73.77002346997727, 40.60570028231908], [-73.7700171718621, 40.605714353261696], [-73.77001053824887, 40.60572914491508], [-73.76999819959372, 40.60575078716462], [-73.76998434837355, 40.6057688685591], [-73.76996865368979, 40.60578366904808], [-73.76996089553964, 40.60578913760247], [-73.76995154367613, 40.60579450362133], [-73.76995140139938, 40.60579458458842], [-73.76990388214686, 40.60581772314249], [-73.76989263053164, 40.605825553064655], [-73.76988816153622, 40.605829642059106], [-73.76988439044467, 40.605833920754215], [-73.76988039970524, 40.60583993763161], [-73.76987704256089, 40.60584700920382], [-73.7698696351318, 40.60587086700765], [-73.76986025918269, 40.60590349221747], [-73.76985219369494, 40.60594001930243], [-73.76983506067754, 40.6060496411682], [-73.76982781931672, 40.60610762977467], [-73.7698237091003, 40.60616538106313], [-73.76982277888568, 40.606222689070265], [-73.7698249935709, 40.6062793451514], [-73.76983037846216, 40.60633515234232], [-73.76983889691651, 40.606389966158574], [-73.76985052748687, 40.60644363393708], [-73.76986524728281, 40.6064960386955], [-73.76991551796586, 40.60664479740757], [-73.76995807121004, 40.60678398710599], [-73.76997746213935, 40.60684515327842], [-73.77000440332164, 40.60692723621898], [-73.77001192144186, 40.60695013362844], [-73.77001531520949, 40.60696313485274], [-73.77001735907474, 40.60697684487389], [-73.77001742079666, 40.607006976368396], [-73.77001285560377, 40.60703922421976], [-73.7700102145254, 40.60705128558923], [-73.77000025048235, 40.60709673335051], [-73.77000884974875, 40.607121713473376], [-73.77001450545818, 40.607143445505145], [-73.77001452895217, 40.607143516919805], [-73.77001456325434, 40.60714376895291], [-73.77001760359343, 40.60716327196342], [-73.7700180619659, 40.60718112181868], [-73.77001587858616, 40.60719731672403], [-73.77001104335504, 40.6072117249807], [-73.77000354472193, 40.60722438107633], [-73.76999878643595, 40.60723004393906], [-73.769993369065, 40.607235282805426], [-73.76996315560298, 40.607259546295985], [-73.76988784485623, 40.607313508270565], [-73.76985535749172, 40.60733899952773], [-73.76976899709642, 40.60741074266819], [-73.76974492340653, 40.60743289278435], [-73.76972484641294, 40.60745363561274], [-73.7697110570677, 40.607470988677434], [-73.76970030178391, 40.60748912180668], [-73.76969288938248, 40.607507468029716], [-73.76968900989174, 40.60752561493523], [-73.76968855777348, 40.607536816733024], [-73.76968970945995, 40.6075489669464], [-73.76969718031438, 40.60757860935677], [-73.76970211276303, 40.60759097471478], [-73.76970929856138, 40.607604038495026], [-73.7697521300489, 40.60766260402667], [-73.7697607528582, 40.60767718246931], [-73.76976659007242, 40.60769076456131], [-73.7697701968254, 40.6077036404038], [-73.76977235935462, 40.60771716186028], [-73.76977241260262, 40.607746212937585], [-73.76976677311299, 40.607778044157016], [-73.76975533215408, 40.60781326778836], [-73.76970592739838, 40.60792694099788], [-73.76970090617823, 40.60794387018821], [-73.76969151386992, 40.60798820197771], [-73.76968721740275, 40.608000728266965], [-73.76968186242391, 40.608011623209634], [-73.76967128998126, 40.60802770362672], [-73.76965766717895, 40.60804472262901], [-73.76965760798781, 40.608044803762226], [-73.76965731146738, 40.60804511829115], [-73.76960694482013, 40.60809849110895], [-73.76958957749022, 40.60811860038658], [-73.76950882786181, 40.60822781620049], [-73.76947829292341, 40.60826637874011], [-73.76946420453096, 40.60828114552284], [-73.76944853940672, 40.60829392910921], [-73.76943176069963, 40.60830435047032], [-73.76941417394845, 40.60831222174791], [-73.76940167237193, 40.608315988363856], [-73.76938718798358, 40.608318813336915], [-73.76933748366145, 40.60832402646428], [-73.76931943037118, 40.60832682784867], [-73.769299481436, 40.60833145233542], [-73.76928031925691, 40.60833708675856], [-73.76926228969427, 40.60834364251946], [-73.76924561573853, 40.60835102876289], [-73.76923055805143, 40.608359130919716], [-73.76921716314595, 40.60836791105494], [-73.76920552729723, 40.608377307208926], [-73.76919569817498, 40.60838726486325], [-73.7691790720129, 40.60840809745203], [-73.76916395114543, 40.60843071507354], [-73.76915015931363, 40.60845536146521], [-73.76913743491646, 40.60848248642137], [-73.769117804581, 40.608534741012946], [-73.76907784265943, 40.60866134609502], [-73.7690280579699, 40.60880254804337], [-73.76901324996051, 40.60885084001258], [-73.76900175529865, 40.60889474534333], [-73.76898858447721, 40.60895510674909], [-73.76897685578123, 40.60901835473809], [-73.76896647581314, 40.60908505403272], [-73.7689572722044, 40.60915629339664], [-73.76895214115504, 40.60920131821295], [-73.76895070639367, 40.60923021317384], [-73.76895228222197, 40.6092564665492], [-73.76895714023856, 40.60927991805636], [-73.76896899693551, 40.60931607885401], [-73.76898446840087, 40.60935567038207], [-73.76900078034787, 40.609391553487406], [-73.76903693493541, 40.60947111456458], [-73.76906734519808, 40.60953713992963], [-73.76908776343276, 40.609578325038484], [-73.7691085136945, 40.60961554957389], [-73.76912963328095, 40.60964845224971], [-73.76913964340078, 40.60966166547592], [-73.76915128441289, 40.609674745942776], [-73.76917976560823, 40.60970087375623], [-73.76921556009525, 40.60972739402596], [-73.76926359313588, 40.60975796492355], [-73.76930923097416, 40.60978428747008], [-73.76941724136196, 40.6098430207667], [-73.76946722660614, 40.60987283021983], [-73.76950134222372, 40.609896248070775], [-73.76957257646704, 40.60994953965], [-73.7696019995385, 40.60996952761247], [-73.76966090947737, 40.61000587377956], [-73.76971531021152, 40.61003699685999], [-73.76976745552511, 40.610064188671736], [-73.76981818454574, 40.610087875418095], [-73.76982838609898, 40.61009216718075], [-73.76985762894924, 40.61010446980088], [-73.76990145566118, 40.610121550569865], [-73.77001763216501, 40.610162964525784], [-73.77014896854337, 40.610208099859044], [-73.77019373076314, 40.61022134629524], [-73.77023486320056, 40.610231604605076], [-73.77024458117825, 40.61023328022782], [-73.77025485957076, 40.61023410223413], [-73.77026559974071, 40.61023405183104], [-73.77027667569713, 40.61023313798061], [-73.77028797772111, 40.61023136816997], [-73.77029925976277, 40.610228761340544], [-73.77031028441786, 40.61022537917164], [-73.77032080237387, 40.61022129454294], [-73.77032436638449, 40.6102194456048], [-73.77032759987875, 40.61021712005475], [-73.77033053985504, 40.6102142981982], [-73.77033324566118, 40.6102109176651], [-73.77033731097586, 40.61020405439635], [-73.77034527335127, 40.61018728535329], [-73.77034989420949, 40.61018021629505], [-73.7703523373575, 40.61017752847406], [-73.77035498094149, 40.610175236925734], [-73.77035785802019, 40.61017331692192], [-73.77036098383705, 40.61017174654638], [-73.77038237359653, 40.61016384636806], [-73.77040557876032, 40.61015758020957], [-73.77043045720318, 40.610152982966994], [-73.77045679265326, 40.610150108487176], [-73.77048443534659, 40.61014897573584], [-73.7705129103782, 40.61014960956341], [-73.77054178049336, 40.610152017305175], [-73.77057050360519, 40.610156135390476], [-73.77058023384357, 40.6101582083894], [-73.77058919293884, 40.610160792656266], [-73.77058996001047, 40.61016101867684], [-73.77060921844134, 40.61016882009694], [-73.77062780467537, 40.61017936581403], [-73.77064497700059, 40.61019221508597], [-73.77066798471779, 40.610215520962484], [-73.77071371980345, 40.61027240823682], [-73.77072621280226, 40.610285113635726], [-73.77073890354532, 40.610295800864215], [-73.77076212825757, 40.61031130865436], [-73.77078978749671, 40.61032592665456], [-73.77082244546227, 40.610340002437646], [-73.77086443569365, 40.61035542348201], [-73.77091857291383, 40.61037396382913], [-73.77095493259962, 40.61038405974893], [-73.77098868403316, 40.610389935584955], [-73.77100447736629, 40.61039116488701], [-73.77101975640439, 40.610391276577886], [-73.77102585543037, 40.61039083857602], [-73.77103212068106, 40.61038987720768], [-73.77104543864475, 40.61038633820573], [-73.7710862987221, 40.61036941718363], [-73.77110034057867, 40.61036459181746], [-73.77111357298146, 40.61036189799134], [-73.77111984842713, 40.610361424149346], [-73.77112600479279, 40.61036150827835], [-73.7711689452305, 40.610364990268245], [-73.77119969623142, 40.61037030119035], [-73.7712289122568, 40.61037905814711], [-73.77124075766046, 40.610384160715654], [-73.77125101879477, 40.61038983710174], [-73.7712912175345, 40.61041752596299], [-73.77136135007882, 40.61047169694773], [-73.7713897690596, 40.610492528309244], [-73.7714934010843, 40.61056422777396], [-73.77149370721929, 40.61056444466316], [-73.77156633578339, 40.61061469325633], [-73.7716783827704, 40.6106928023703], [-73.77175068390629, 40.610745735191124], [-73.77177270190808, 40.61076523897404], [-73.77177276151261, 40.610765294042025], [-73.77179266153695, 40.61078883677322], [-73.77181070491237, 40.610816689532804], [-73.77182798309488, 40.6108505205691], [-73.77183567042678, 40.610869501343835], [-73.77185132437893, 40.610915089557274], [-73.77186131675909, 40.61093724456386], [-73.77187277684048, 40.610955052578454], [-73.77190585729397, 40.61099785702015], [-73.77191334262928, 40.61101040794046], [-73.77191837105117, 40.61102222221602], [-73.77194457459913, 40.6111103555345], [-73.77197043212044, 40.6112059792419], [-73.7719796331746, 40.61123161744909], [-73.77198997844316, 40.61125384451127], [-73.77199000196882, 40.611253907716595], [-73.77199003780456, 40.61125397094622], [-73.77199007357872, 40.611254052269096], [-73.77200909650384, 40.61128557231708], [-73.7720532149166, 40.61134901093654], [-73.77211190787796, 40.61143890990163], [-73.77211789530061, 40.61144672027096], [-73.7721252314717, 40.61145424516313], [-73.7721438888505, 40.611468402542336], [-73.77216765565773, 40.61148122779965], [-73.77219408886714, 40.611491645073976], [-73.772195965905, 40.61149237838507], [-73.77221438055132, 40.611497664583986], [-73.77223450234523, 40.61150167675028], [-73.77225630686416, 40.61150442086574], [-73.77227984029001, 40.61150589149225], [-73.77235885560256, 40.611506317561414], [-73.77235963567736, 40.611506282249785], [-73.77245013985568, 40.61150240908259], [-73.7725317703667, 40.61149506030717], [-73.77263219715753, 40.61148144552586], [-73.77268245759265, 40.611473053031155], [-73.77272806684849, 40.61146363357612], [-73.77277007489664, 40.611452973464196], [-73.77280893259515, 40.61144094677056], [-73.77291336710068, 40.611402989418536], [-73.77304604538075, 40.6113500764281], [-73.77317979315391, 40.61129528186083], [-73.77326368427093, 40.61125874165951], [-73.77330125585432, 40.61123931064123], [-73.77331668625673, 40.61122934508988], [-73.773330120849, 40.611219051096064], [-73.77334370614358, 40.61120638970322], [-73.77335673062024, 40.611191665417344], [-73.77336952707269, 40.611174561928834], [-73.773383685269, 40.611152950391634], [-73.77339309446671, 40.61113458978192], [-73.77339979420599, 40.61111385446016], [-73.77340383065187, 40.61109080935154], [-73.77340389078967, 40.611090448779], [-73.77340563790028, 40.61106301306704], [-73.77340520518642, 40.611051512485744], [-73.77340338988809, 40.611039649157746], [-73.77339559311453, 40.61101456383304], [-73.77338222622511, 40.610987315106684], [-73.77336131902807, 40.61095385476212], [-73.7733244987483, 40.61090188452717], [-73.77323628183373, 40.61078552461406], [-73.77319873884215, 40.61073086959151], [-73.77316210701994, 40.61067196629814], [-73.7731307266893, 40.610618368630945], [-73.77304946279641, 40.610463119592424], [-73.773019690345, 40.610416279523584], [-73.77295676540665, 40.61033058652429], [-73.77293272002777, 40.61029549117771], [-73.77291139315624, 40.61026016565034], [-73.77289590086995, 40.61022937360428], [-73.772884997904, 40.610200643531165], [-73.77287854345656, 40.61017342532192], [-73.77287840879376, 40.6101715251035], [-73.77287865131598, 40.6101696350119], [-73.77287923617226, 40.61016778999218], [-73.77288018741419, 40.61016596328696], [-73.77288150513638, 40.61016419175308], [-73.77288316631983, 40.61016245691658], [-73.77288751313765, 40.610159161320624], [-73.77289320683816, 40.610156119344964], [-73.772900070499, 40.61015344958591], [-73.77290788845032, 40.61015122298471], [-73.77291637813553, 40.610149528108415], [-73.7729341192848, 40.610147987039554], [-73.77294255492997, 40.61014821998454], [-73.77295028088521, 40.6101491273411], [-73.77295702145892, 40.610150653615136], [-73.77296268686534, 40.610152762945724], [-73.77296508104024, 40.61015403738109], [-73.77296717978257, 40.61015543755019], [-73.77296895866235, 40.61015697211622], [-73.77297042983133, 40.61015862300996], [-73.77299992670021, 40.610199383907386], [-73.77306601457155, 40.61029673598037], [-73.77339019496708, 40.61075314800841], [-73.77340971200925, 40.61078269915479], [-73.77342761373566, 40.61080980202038], [-73.77349222861594, 40.6109170832587], [-73.77351890708198, 40.61095946065268], [-73.77354839797641, 40.61100218567637], [-73.77357751691721, 40.6110394789879], [-73.77360460444339, 40.61106958244737], [-73.77367760714131, 40.611142705083424], [-73.77370915833684, 40.611177356686206], [-73.7737449292238, 40.611221786404634], [-73.77376767753121, 40.61125622247544], [-73.77378104179124, 40.61128430946837], [-73.77379138951497, 40.61131637102398], [-73.77379900757467, 40.611352342874014], [-73.77380720929449, 40.61140784143564], [-73.77380978495356, 40.611432583240706], [-73.77381081335125, 40.611460275881086], [-73.77380863014407, 40.61154996353017], [-73.77380884252659, 40.61158168130349], [-73.77381102073075, 40.61161168141205], [-73.77381536140189, 40.61163869704945], [-73.77382415413824, 40.61167302439311], [-73.77384727397728, 40.61174793881712], [-73.77385497769873, 40.611779283162846], [-73.77388591019262, 40.611937197679545], [-73.77391081130263, 40.612041049611385], [-73.77391703488719, 40.61208027079028], [-73.77392076775064, 40.612094902451915], [-73.7739239440798, 40.612102500800724], [-73.7739278681188, 40.61210917183659], [-73.7739326342091, 40.612115035361], [-73.77393827753895, 40.61212015175367], [-73.77394229745471, 40.612122789705175], [-73.77394430327868, 40.6121237390188], [-73.77394630995796, 40.612124435699705], [-73.77394831660732, 40.61212488158879], [-73.7739503249933, 40.61212507434425], [-73.77395234585147, 40.612125024541434], [-73.77395435588404, 40.61212473096199], [-73.7739563775416, 40.61212418544094], [-73.7739584015895, 40.61212338846286], [-73.7739624744208, 40.612121046368415], [-73.77396659812177, 40.61211770422277], [-73.77397078374364, 40.612113344289575], [-73.77398539252552, 40.61209136466629], [-73.77400930728074, 40.6120431888255], [-73.77401764033394, 40.612028472162585], [-73.77405276935937, 40.61197332182517], [-73.7740809475436, 40.6119256939622], [-73.77409284355927, 40.61190221412751], [-73.77411806047178, 40.61184285571593], [-73.7741312788372, 40.61181616324954], [-73.77413182408637, 40.611815362859815], [-73.77413250067724, 40.61181466207327], [-73.77413332974614, 40.611814051047034], [-73.77413430026861, 40.61181353964375], [-73.77413541249396, 40.61181311915227], [-73.77413664243689, 40.611812796562006], [-73.77413801402106, 40.6118125829765], [-73.77413949166653, 40.61181246894438], [-73.77414282314955, 40.61181254736078], [-73.77414652034192, 40.61181304163379], [-73.77415045182457, 40.61181394028045], [-73.77415446562193, 40.61181519977985], [-73.77416205233125, 40.611818600792404], [-73.77416525980846, 40.61182057003741], [-73.77416789935114, 40.61182263767906], [-73.77416988911098, 40.6118246948298], [-73.77417063107842, 40.61182572274073], [-73.77417119391467, 40.6118267415883], [-73.77417159328917, 40.61182773330986], [-73.77417180232338, 40.61182871527586], [-73.77417184679693, 40.6118296699464], [-73.77417171308002, 40.611830597461996], [-73.774161726854, 40.611865619651276], [-73.77414786993229, 40.611903603526855], [-73.7741301655297, 40.61194489725949], [-73.77410192392666, 40.61200510278893], [-73.77408950538367, 40.61202740570797], [-73.77405335830696, 40.61208262542263], [-73.77404497735554, 40.61209895865359], [-73.77403932881545, 40.61211414884065], [-73.77403522797262, 40.61213158629511], [-73.77403265331083, 40.61215119826687], [-73.77403016171108, 40.612218812626736], [-73.7740276516538, 40.61224473020021], [-73.77400830908317, 40.61235765948454], [-73.77398615369667, 40.61245779570801], [-73.77397767570855, 40.61248913519638], [-73.7739697707448, 40.61251841821367], [-73.77394965118486, 40.612584257775346], [-73.77392543466452, 40.612656751420296], [-73.77389100971483, 40.61275422573868], [-73.77384604655711, 40.61287380911937], [-73.77382494541348, 40.61292460023349], [-73.77380379155755, 40.612971799407745], [-73.7737736886651, 40.61303047834605], [-73.77371081639753, 40.61313984431636], [-73.77368650717207, 40.61318455916043], [-73.77364588445435, 40.613268611302004], [-73.7735758160344, 40.6134272535469], [-73.7735466529197, 40.613490419876555], [-73.7735141321426, 40.61355476753436], [-73.773481748665, 40.61361161063556], [-73.77346587818103, 40.613635469564464], [-73.77345045782228, 40.613653395149335], [-73.77345031409877, 40.61365351012634], [-73.77343386792192, 40.6136667877601], [-73.77342509849096, 40.6136720153198], [-73.77341596206944, 40.61367629561389], [-73.77339803484179, 40.61368296748421], [-73.77338277882686, 40.6136870541096], [-73.77337035980797, 40.61368919578096], [-73.77336232912107, 40.61368901911916], [-73.77335608646888, 40.61368855280565], [-73.77332964754102, 40.613684430711444], [-73.77329896191252, 40.61367497196102], [-73.77326127524387, 40.61365136745161], [-73.77321979995166, 40.613631797089205], [-73.77317696868322, 40.61361842478523], [-73.7731332193545, 40.61360690709986], [-73.77304977884343, 40.61359451210485], [-73.77296528060725, 40.61358740749043], [-73.7728803012121, 40.61358564173448], [-73.77279542050722, 40.61358922688536], [-73.77271121766803, 40.61359813848014], [-73.77262826724544, 40.61361231571145], [-73.77252205525943, 40.61364825941569], [-73.77241812289253, 40.613687884674846], [-73.77231668973992, 40.613731107767585], [-73.77221797011781, 40.6137778373709], [-73.77215915970974, 40.61381372551739], [-73.77209575329097, 40.61384473767983], [-73.77202844410407, 40.61387053478952], [-73.771957968065, 40.613890834795654], [-73.77187500460022, 40.6139086087822], [-73.77179004380021, 40.61391961497738], [-73.77170403355123, 40.61392373058756], [-73.771654144762, 40.613924710645136], [-73.77160464827602, 40.61391985236653], [-73.77157232045141, 40.613913391685294], [-73.77154415104286, 40.61390284819679], [-73.77151470107658, 40.613894582888705], [-73.77149192825127, 40.61388992737643], [-73.77141220326567, 40.613890766362616], [-73.7713326800468, 40.61389516680603], [-73.77127396321097, 40.61390072517808], [-73.771215619475, 40.613908225831494], [-73.77112286825097, 40.61392432219383], [-73.771031821726, 40.61394532343614], [-73.7709429259498, 40.6139711266706], [-73.77091678395117, 40.61397276719949], [-73.77089122605439, 40.61397726480554], [-73.77086678706019, 40.613984525376075], [-73.77084397835702, 40.61399439698316], [-73.77080663362396, 40.61400979861785], [-73.77076680152705, 40.61402101149677], [-73.77072528617398, 40.614027809260314], [-73.77068292565424, 40.61403005467906], [-73.77064057511912, 40.61402770242352], [-73.77038968748185, 40.614044682072375], [-73.77019814694472, 40.61406330653122], [-73.77010609275644, 40.61406317623786], [-73.77006907432802, 40.61406201308636], [-73.77002788909296, 40.6140592555852], [-73.76985886892095, 40.614043033167775], [-73.76965068224214, 40.61403117088185], [-73.76963849147283, 40.61402992934977], [-73.76962599613113, 40.61402782442917], [-73.76959938519454, 40.61402089082527], [-73.76957481264448, 40.61401222402047], [-73.76951917147099, 40.61399014801285], [-73.76949266011108, 40.61398137846054], [-73.76946947006188, 40.61397598290578], [-73.76944759498394, 40.61397341704376], [-73.76940997385499, 40.61397218060091], [-73.76937109456867, 40.61397254337085], [-73.76933085131219, 40.61397452507587], [-73.76928901812339, 40.61397811587945], [-73.76924563215687, 40.61398331417909], [-73.76920000707541, 40.61399018393143], [-73.76915158682361, 40.61399881247275], [-73.76909892905647, 40.61400944987172], [-73.76902068965084, 40.61402631317462], [-73.76896496255547, 40.61403933884359], [-73.76891513219671, 40.61405241319663], [-73.76887037118318, 40.614065832110235], [-73.76858268472621, 40.61416176216617], [-73.76846044441163, 40.61420061738405], [-73.76841416033392, 40.61421360802995], [-73.76837140539077, 40.614224165701835], [-73.76835465017773, 40.61422714847352], [-73.76833706074915, 40.61422865345921], [-73.76831847311404, 40.61422867848521], [-73.76829854249856, 40.614227243462224], [-73.76825921241463, 40.6142211393147], [-73.7681828504966, 40.61420436172427], [-73.76815131971195, 40.614198489046444], [-73.76811864665672, 40.61419476714477], [-73.76808937643767, 40.61419447472638], [-73.76803145576746, 40.61419976068117], [-73.76789952592243, 40.614218188647286], [-73.76786697167925, 40.6142211406185], [-73.76783729461303, 40.61422240416597], [-73.76778765465387, 40.614221403090404], [-73.76766549195821, 40.614214042266255], [-73.76763170757866, 40.61421381147214], [-73.76760132093918, 40.6142153085681], [-73.7675607760041, 40.61421920708899], [-73.76754530082158, 40.614221131702], [-73.7675186061643, 40.61422445276445], [-73.7674275279154, 40.61423919037484], [-73.76737638307308, 40.614249820664746], [-73.76726342522326, 40.61427650864115], [-73.76721373576882, 40.61428641338739], [-73.76715921956162, 40.61429454290923], [-73.76710253635196, 40.61430041558083], [-73.76704379234957, 40.61430402508025], [-73.76698308457676, 40.61430535183291], [-73.76694157398309, 40.61430453808634], [-73.76690019985251, 40.61430185212442], [-73.76685925777915, 40.61429732034635], [-73.76681903151112, 40.614290961923935], [-73.7667529623521, 40.614261005953615], [-73.76669071688973, 40.61422662157113]]]}}, {\"id\": \"86\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 87, \"Shape_Leng\": 0.0369015496117, \"Shape_Area\": 6.72050210692e-05, \"zone\": \"Financial District North\", \"LocationID\": 87, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00372234899996, 40.70693234399989], [-74.00360537999994, 40.706780592999905], [-74.00316110499998, 40.70623201399992], [-74.00306957, 40.706142802999885], [-74.0030261529999, 40.70610867499988], [-74.00299474599991, 40.70608024599991], [-74.00295417200002, 40.706027114999934], [-74.00256965079724, 40.705523579761724], [-74.00268512768783, 40.70544924811862], [-74.00236285311736, 40.7051589666926], [-74.00241573476919, 40.705129311282505], [-74.00238385458334, 40.705101270250836], [-74.00197070397937, 40.7047378684872], [-74.00197014837809, 40.704731159202304], [-74.00217184941894, 40.70460659646366], [-74.00218562331561, 40.70460701887392], [-74.00233344621154, 40.70474813530697], [-74.00233393075767, 40.70474260740423], [-74.00242059318663, 40.70468033460375], [-74.0024307107586, 40.70467785800438], [-74.00248839829254, 40.70472677235117], [-74.00248790148628, 40.704733761091624], [-74.00239832748949, 40.70479492514084], [-74.00238670740669, 40.70479566367449], [-74.00332307270776, 40.70562859522001], [-74.00358262337747, 40.70546112883118], [-74.00363187169316, 40.70542980675177], [-74.00360802644228, 40.70540778137035], [-74.00355934714837, 40.705438338158096], [-74.00287843069457, 40.70483135798975], [-74.00297246475286, 40.704766492442566], [-74.00358383172679, 40.70530161130755], [-74.00364968772803, 40.705377950512094], [-74.00362197523796, 40.705398899984345], [-74.00364446183858, 40.70542168463882], [-74.00383327532252, 40.70530324302638], [-74.00394758957431, 40.705402871900304], [-74.0043834441722, 40.7051132598548], [-74.00426687037424, 40.70501166020209], [-74.00458622952459, 40.70479945067928], [-74.00459003883662, 40.70478775294495], [-74.00349632042315, 40.70379676926777], [-74.00351016434425, 40.70378808825903], [-74.00371987043046, 40.70365448194469], [-74.00396975693849, 40.70388164511804], [-74.00480628900739, 40.70464206905123], [-74.00484062824778, 40.704639779526524], [-74.00492037826484, 40.70459100497326], [-74.00518805067318, 40.704420535690545], [-74.00511107204132, 40.704350605261965], [-74.00508460380269, 40.704366355828846], [-74.00492580498366, 40.70421180838465], [-74.00500393507072, 40.70416531254446], [-74.00516296842157, 40.70432009323669], [-74.00513867768034, 40.70433455643976], [-74.00521160900396, 40.70440553223528], [-74.00526035829189, 40.704374488410075], [-74.0053502722475, 40.70431007996148], [-74.00418492090171, 40.703248066176315], [-74.00418130143021, 40.703250361430186], [-74.0041574215416, 40.703231632087295], [-74.00416265161378, 40.703227768889136], [-74.00420723493835, 40.70319937375783], [-74.00411763228837, 40.70311193702741], [-74.004272587521, 40.70301566641148], [-74.00429788566103, 40.70301726836962], [-74.00432634521925, 40.70304454424587], [-74.00434743166828, 40.703046947298816], [-74.00436534740585, 40.70307021681555], [-74.00438643514065, 40.70308866795962], [-74.00441805325255, 40.70311673525835], [-74.00444124738806, 40.703137598367604], [-74.00448447402037, 40.70317690392736], [-74.00462520935534, 40.70330003518526], [-74.00462626412609, 40.703314479168114], [-74.00465050979112, 40.70332972316352], [-74.00466632025643, 40.70334817446862], [-74.00467686257008, 40.70334817404074], [-74.00474855169037, 40.703419564148824], [-74.0048022288324, 40.70348400677523], [-74.00483652285267, 40.703512926391674], [-74.00498836312711, 40.703637436151816], [-74.0050337744232, 40.7036749205554], [-74.00510304451781, 40.70373114347389], [-74.00520771937107, 40.70382133725155], [-74.00527929789541, 40.703875218096265], [-74.00534891537728, 40.70392373409762], [-74.00535945878065, 40.703919725687555], [-74.0053963553808, 40.703942183324386], [-74.00538791847218, 40.703949405145195], [-74.00559242542963, 40.70408256373169], [-74.00560507496257, 40.70408496696345], [-74.00563775955965, 40.70410583053918], [-74.00566330640648, 40.7041042444782], [-74.00589012878147, 40.70395873810608], [-74.00601862724882, 40.70387315289086], [-74.00614010577931, 40.70397866027171], [-74.00634162385869, 40.703848533682745], [-74.00643225275432, 40.70379036104623], [-74.00659591913875, 40.70368590245183], [-74.00634518899977, 40.70348864644174], [-74.00632434161025, 40.70346987449462], [-74.00591217944537, 40.70309876190139], [-74.00588747266426, 40.70307651522387], [-74.0054838408244, 40.70271432213403], [-74.00552682615245, 40.70268528761711], [-74.00550531272066, 40.70266684658552], [-74.00547130861277, 40.70268982908357], [-74.00543599763346, 40.70265955541132], [-74.00542728826254, 40.70266543619403], [-74.00530412596079, 40.70255986563306], [-74.00549427805105, 40.70243138195557], [-74.00550166067457, 40.70243086779703], [-74.00561361797617, 40.702530242032324], [-74.00560163196401, 40.70253805976984], [-74.00560905264518, 40.70255281855374], [-74.00559479485831, 40.70256802032948], [-74.00559935100472, 40.702580167839244], [-74.00559022977765, 40.70258581428382], [-74.00559194611432, 40.702598835921144], [-74.00558509547798, 40.70260405020236], [-74.00558909594902, 40.70261273059688], [-74.00553546138171, 40.70264552110693], [-74.0055577199881, 40.702664394651975], [-74.00567522249852, 40.70258407270014], [-74.005984303395, 40.702860018041086], [-74.00623870508527, 40.70307978930156], [-74.0063637273041, 40.70318779225255], [-74.00649425191104, 40.703300548171775], [-74.00654880812648, 40.70336761508583], [-74.00655720878508, 40.70336314833331], [-74.00675541584022, 40.70354148947679], [-74.00678502191047, 40.70356812790525], [-74.00699466002446, 40.70344118856669], [-74.00722046663674, 40.70330197730327], [-74.007519052838, 40.70312182023711], [-74.00752714979265, 40.70311498141609], [-74.00760498189766, 40.70299314132918], [-74.00763657899991, 40.70300436799988], [-74.0077967509999, 40.703063514999876], [-74.00782878499996, 40.70308642199992], [-74.00785483099992, 40.70310778999987], [-74.00797948599994, 40.703188755999875], [-74.00854172499996, 40.70369761199989], [-74.00898541299989, 40.70407915999986], [-74.00907940599997, 40.70416904299986], [-74.00945463899993, 40.7045278569999], [-74.00974639299997, 40.704775247999905], [-74.00998418599995, 40.70514773299991], [-74.00999520599994, 40.705331238999904], [-74.0098658089999, 40.705796426999925], [-74.01118994199994, 40.706278727999916], [-74.01187330299993, 40.706551406999864], [-74.01240001199992, 40.70678480799989], [-74.01250773999998, 40.706767065999934], [-74.01194691399999, 40.7074515319999], [-74.01170249899995, 40.707755712999834], [-74.01124834999997, 40.708304700999854], [-74.01118228399991, 40.70838557399989], [-74.01101198499991, 40.70859402199989], [-74.01084638199993, 40.70879754499991], [-74.01054246499999, 40.70915414799992], [-74.00998739199997, 40.70980567299989], [-74.00954277299995, 40.71032749299992], [-74.00947357899996, 40.710412542999954], [-74.00906264799988, 40.71088670399992], [-74.00774466599994, 40.71022077299992], [-74.00705335899995, 40.70987195099989], [-74.00643549199992, 40.70955732099991], [-74.00551253, 40.709093407999866], [-74.0052621729999, 40.70892071099989], [-74.00505672899992, 40.70878876899998], [-74.00478782699994, 40.70840189299991], [-74.00432857299997, 40.70782349599986], [-74.00405660199985, 40.70743678099992], [-74.00392127499991, 40.70723974499993], [-74.00372234899996, 40.70693234399989]]]}}, {\"id\": \"87\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 88, \"Shape_Leng\": 0.0352046035327, \"Shape_Area\": 5.72999455414e-05, \"zone\": \"Financial District South\", \"LocationID\": 88, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00974639299997, 40.704775247999905], [-74.00945463899993, 40.7045278569999], [-74.00898541299989, 40.70407915999986], [-74.00854172499996, 40.70369761199989], [-74.00797948599993, 40.70318875499991], [-74.0078548309999, 40.70310778899988], [-74.00782878499996, 40.70308642199992], [-74.0077967509999, 40.703063514999876], [-74.00763657899991, 40.70300436799988], [-74.00760498189766, 40.70299314132918], [-74.00764555180645, 40.702928174389456], [-74.00765095550902, 40.702919839105114], [-74.00765840854532, 40.70290916746315], [-74.00770527479769, 40.702881825407246], [-74.00770614970409, 40.702875161620035], [-74.00796578643664, 40.70271815707015], [-74.0083739580163, 40.70246975865777], [-74.009127727589, 40.70200832442169], [-74.00915955533813, 40.70199730764995], [-74.00930907655483, 40.70195460302249], [-74.00869964313567, 40.701139867542174], [-74.00775549150644, 40.70154800167175], [-74.00759136204225, 40.70132808104815], [-74.00853611510765, 40.700919695745085], [-74.0083492755909, 40.7006693483882], [-74.00862036163882, 40.70055216421584], [-74.00960052263278, 40.70186548191784], [-74.01026710345066, 40.70169866881413], [-74.01083741653544, 40.70155334480754], [-74.01116664151876, 40.70147720220832], [-74.01109588602684, 40.70127499934889], [-74.01111949092332, 40.701268748305765], [-74.01112667396998, 40.701274996376725], [-74.01128983905525, 40.70124998224428], [-74.0112097577877, 40.701044583273756], [-74.01112868539599, 40.701064897450436], [-74.0110783797454, 40.70095712031517], [-74.0110670856897, 40.70091494089483], [-74.01110886351543, 40.7009014206063], [-74.0110951435839, 40.7008665271816], [-74.01109514108846, 40.700851532992615], [-74.01108063438, 40.70085194885256], [-74.01105929378232, 40.700811562771484], [-74.01106312665488, 40.70080281982734], [-74.01104287442345, 40.70075743496105], [-74.011021525765, 40.70074120110607], [-74.01103575886044, 40.700731617687474], [-74.01102699993733, 40.700712878931306], [-74.01101386500967, 40.70071163747197], [-74.01100072817277, 40.700690819617186], [-74.01101659417203, 40.70068415386645], [-74.01101221338783, 40.700668746136365], [-74.01099990958122, 40.70067374615791], [-74.01097965965197, 40.700645849111844], [-74.01099825777675, 40.700635024239574], [-74.01097034745736, 40.70057548420756], [-74.01094845841567, 40.70057631454651], [-74.01093752366046, 40.7005671573749], [-74.0109386101689, 40.7005563341749], [-74.01096049734583, 40.70053884409263], [-74.0109344927399, 40.700487624779925], [-74.01092027169815, 40.70049262380237], [-74.01090112139877, 40.70046223399988], [-74.01092026515036, 40.70045265012994], [-74.01081026034738, 40.70023406243364], [-74.01082511780585, 40.70020930486715], [-74.01083751798708, 40.70020722317945], [-74.01093093205073, 40.700415152334415], [-74.01100364429968, 40.70058110994409], [-74.01104377249544, 40.70068284521962], [-74.01107427475966, 40.70074833789612], [-74.01110436404349, 40.700810649429386], [-74.01113362222917, 40.70086787440548], [-74.01115033556829, 40.70089840010092], [-74.01135836291361, 40.70085195863628], [-74.0113031892115, 40.700647844036354], [-74.01129816432261, 40.70059888503188], [-74.01130986410945, 40.70057980119103], [-74.01133409410409, 40.70056835368226], [-74.01136082267426, 40.700565810999194], [-74.0113817187564, 40.7005702571179], [-74.01139508123386, 40.700590607673846], [-74.01143269912247, 40.70067454111621], [-74.011469474374, 40.70074892825803], [-74.011502911437, 40.70081505897416], [-74.0117292681748, 40.700770118059616], [-74.01170009248334, 40.700721538635975], [-74.011682753977, 40.70069334506204], [-74.01166539833068, 40.700637560590245], [-74.01163149614034, 40.70055539174732], [-74.01161019835872, 40.70049300642145], [-74.01161413587107, 40.70047261846256], [-74.01163147918638, 40.7004582182481], [-74.01167798034295, 40.70046181520738], [-74.01170321011998, 40.70048100238685], [-74.01172605835686, 40.70047860442191], [-74.01175129264924, 40.70053078758623], [-74.0117773082774, 40.70057097487754], [-74.01182619233424, 40.700666342782014], [-74.01183407601519, 40.70068613447612], [-74.01184827026111, 40.700729925805916], [-74.01185378549594, 40.700738921450146], [-74.01199159781133, 40.70070483146649], [-74.01208668214687, 40.70068130873922], [-74.01208509377949, 40.70066451480386], [-74.0120693217167, 40.7005997333441], [-74.01203856797002, 40.700528956309654], [-74.0119767064249, 40.7003784242992], [-74.01190488098494, 40.700226930002], [-74.01185672556994, 40.70010710340836], [-74.01193096316563, 40.70005120965432], [-74.01196196116912, 40.70003754559547], [-74.01204027930196, 40.70004809164558], [-74.01208842091677, 40.7000865836894], [-74.01212759506136, 40.7001430865068], [-74.01215779158525, 40.700211378648156], [-74.01217247961002, 40.700233115597726], [-74.01223694549928, 40.70032127784431], [-74.01225490284888, 40.70036473434146], [-74.01231855974255, 40.70048828759264], [-74.01236222440257, 40.70057582996201], [-74.01239160032279, 40.70062052809406], [-74.01259392327509, 40.700570833472426], [-74.01259064684221, 40.70051185050813], [-74.01255471870927, 40.700359730657794], [-74.01252899955234, 40.70025232035699], [-74.01251593564186, 40.70018464848017], [-74.01251511280114, 40.700150492110794], [-74.01254202780153, 40.7001349741483], [-74.01261382087365, 40.7001082658111], [-74.0126480885809, 40.70011508828302], [-74.01268072728439, 40.70014675538139], [-74.0127349915856, 40.70021877320353], [-74.01279619139247, 40.70030009988461], [-74.01283454933404, 40.70035287473442], [-74.01286637756563, 40.700407513371665], [-74.01288188403379, 40.700441659886145], [-74.01290474249377, 40.70049691216282], [-74.01290881793962, 40.70052361169349], [-74.01300205335563, 40.700516028968046], [-74.01310747847366, 40.70050745218069], [-74.01311155389506, 40.7004720530446], [-74.01311153191459, 40.70036029060313], [-74.01309395831619, 40.70022369323981], [-74.01307843485947, 40.70010324258431], [-74.01302495705559, 40.699882830206], [-74.0130118926586, 40.69981825663122], [-74.01302085702764, 40.699793419367936], [-74.01304370092431, 40.699774784735816], [-74.01311303826309, 40.69977416588525], [-74.01314078965082, 40.69980333792552], [-74.01315384897737, 40.69984494048546], [-74.01316691626799, 40.699931244337535], [-74.01317753563323, 40.7000175474625], [-74.01318571684631, 40.70010323034394], [-74.01320084197312, 40.70023982797609], [-74.01320736787193, 40.700275216518826], [-74.01322288016905, 40.700333586085485], [-74.0132734948308, 40.7004968702278], [-74.01348560832729, 40.700478222908394], [-74.01348068493708, 40.700357771068354], [-74.01348311536108, 40.70026338811632], [-74.01350349660855, 40.700176459528265], [-74.01353285018102, 40.70010816142327], [-74.01358669315087, 40.70007959034501], [-74.01361029865362, 40.7000842982948], [-74.01367479307396, 40.70012582122537], [-74.01366274984949, 40.700136656517614], [-74.01364265243346, 40.700161782904686], [-74.01361182284575, 40.700245174325296], [-74.01357049004972, 40.700232202686955], [-74.01356130915947, 40.70029512207909], [-74.01353310202803, 40.70029712412281], [-74.01353047798223, 40.70031110017328], [-74.01356131218263, 40.70031009779257], [-74.01356034122136, 40.70036653321862], [-74.01352950894581, 40.70037352406205], [-74.01353147488012, 40.70038201609177], [-74.01356297105794, 40.700380013103185], [-74.01355970574456, 40.70043893421369], [-74.0135236177836, 40.700441937789115], [-74.01352295909443, 40.70046041520351], [-74.01358070009414, 40.70045691424639], [-74.0135854179925, 40.70050191287807], [-74.01355283214755, 40.70050335758483], [-74.01357712861271, 40.70064368123795], [-74.01351039646312, 40.70065037107601], [-74.0135582697974, 40.70093987091524], [-74.01395152778245, 40.70099098222993], [-74.01415893499998, 40.70109628999988], [-74.01376867899998, 40.7011432629999], [-74.01377528899994, 40.70118271299988], [-74.01378191199989, 40.70122215499992], [-74.01379120799997, 40.70130547599994], [-74.01389890399994, 40.702270669999976], [-74.014192151, 40.70275671099995], [-74.01442983199998, 40.70315064799991], [-74.014391769, 40.70359878299993], [-74.01425523899994, 40.704425471999905], [-74.01425342599994, 40.70445756399993], [-74.01425861399996, 40.70448944299991], [-74.01427065899998, 40.70452022899989], [-74.01428922899998, 40.70454907499989], [-74.01419226999991, 40.70457040399989], [-74.01326979499997, 40.704553495999974], [-74.01332610899989, 40.7050307879999], [-74.01335309799998, 40.70519857699984], [-74.01335946199991, 40.705368655999926], [-74.01334484799996, 40.7055388269999], [-74.01330942899993, 40.70570683099988], [-74.01325396399993, 40.705870513999905], [-74.01250773999998, 40.706767065999934], [-74.01240001199992, 40.70678480799989], [-74.01187330299997, 40.70655140599994], [-74.01118994199994, 40.706278727999916], [-74.0098658089999, 40.705796426999925], [-74.00999520599994, 40.705331238999904], [-74.00998418599995, 40.70514773299991], [-74.00974639299997, 40.704775247999905]]]}}, {\"id\": \"88\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 89, \"Shape_Leng\": 0.122794569553, \"Shape_Area\": 0.000447548142373, \"zone\": \"Flatbush/Ditmas Park\", \"LocationID\": 89, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95374853799996, 40.63858633799993], [-73.9536000469999, 40.63842234699991], [-73.95307699899985, 40.638446260999864], [-73.95213852299987, 40.63844349599993], [-73.95116311199983, 40.63844818299984], [-73.95017361500003, 40.63851059699987], [-73.94923295699995, 40.63854767299993], [-73.94826499699982, 40.63860718999994], [-73.94715150299994, 40.63867640599994], [-73.94638150299986, 40.638726249999905], [-73.94537272499998, 40.638786954999865], [-73.9451233509999, 40.636448278999865], [-73.94488042799988, 40.634176953999926], [-73.94462885499995, 40.63183773499993], [-73.94561754699987, 40.63177742999988], [-73.94647961799998, 40.631708161999896], [-73.9455109549998, 40.63079148299986], [-73.94480816099995, 40.630127166999884], [-73.94453995099988, 40.62986656299989], [-73.94415535299993, 40.629508284999886], [-73.94446784099986, 40.629480051999884], [-73.94540053199998, 40.62937781199983], [-73.9463262219999, 40.629276641999915], [-73.94732672199994, 40.629166566999835], [-73.94832386299989, 40.62905666099993], [-73.94925229799985, 40.628954764999925], [-73.95017656999988, 40.6288527719999], [-73.95110641899993, 40.62875046699989], [-73.9520690179999, 40.628645780999875], [-73.95303133199995, 40.62854011499994], [-73.95395983399986, 40.62843938999987], [-73.95488499199999, 40.62833554799986], [-73.95581207199996, 40.62823349199987], [-73.95684697299988, 40.62811987899991], [-73.95805293100001, 40.627971118999895], [-73.95899416299996, 40.62788559399996], [-73.95995565099992, 40.627778671999884], [-73.96014773499996, 40.628915184999876], [-73.96022560099996, 40.62925450099989], [-73.96024902900002, 40.62937062199989], [-73.96025855099994, 40.62941826299991], [-73.96040734299989, 40.6301625309999], [-73.96046280599994, 40.630155660999904], [-73.96136921599995, 40.63005838799989], [-73.96163639899993, 40.63002222299986], [-73.96182099799995, 40.630004876999855], [-73.96197454299985, 40.629995040999866], [-73.96232599399991, 40.6299687169999], [-73.96322149299982, 40.629841060999865], [-73.96414862899985, 40.62973896499996], [-73.96508072799992, 40.629654674999856], [-73.96608191399986, 40.629542706999906], [-73.96707656900001, 40.629434681999946], [-73.96800084399993, 40.62933305799992], [-73.96892915899984, 40.62923162599992], [-73.97000109799986, 40.62911347399987], [-73.97109232299984, 40.628966140999886], [-73.97136622199992, 40.62892916199996], [-73.97120750499982, 40.62808971199992], [-73.97111421999993, 40.62758588399991], [-73.97092126300004, 40.62656998499993], [-73.97108160299985, 40.62655370899987], [-73.97119209799986, 40.6265426959999], [-73.97229894799983, 40.626431167999876], [-73.97322001099991, 40.62631854299984], [-73.97414911500006, 40.626214897999894], [-73.97507611299997, 40.62611376699993], [-73.975635809, 40.62603520899995], [-73.97623453700001, 40.625976349999895], [-73.97644970399986, 40.62700825499989], [-73.9765146209999, 40.62729448999989], [-73.97657775699989, 40.627573185999886], [-73.97669604399998, 40.62835456399994], [-73.97684964299987, 40.62908858099994], [-73.97699848899988, 40.62987149599992], [-73.97716511999984, 40.63074665399988], [-73.97604935699995, 40.63127590599986], [-73.975171463, 40.631688580999906], [-73.97431883499996, 40.632090573999946], [-73.97320194999996, 40.632618080999855], [-73.97281084499981, 40.63280454799989], [-73.97241686999978, 40.632992044999874], [-73.97231136199997, 40.63304155499987], [-73.97216145999994, 40.63311162099996], [-73.97190427099986, 40.63323009399994], [-73.97087567699992, 40.63371951299991], [-73.96986869299992, 40.63419467499989], [-73.97029036199991, 40.63640634099988], [-73.96935965799996, 40.63650554999993], [-73.96803868499985, 40.636650984999925], [-73.96822343400004, 40.63729690599994], [-73.96872218399989, 40.63901853799988], [-73.968790992, 40.639255932999895], [-73.96891461099989, 40.63975715999986], [-73.9693990309999, 40.64138244599991], [-73.96948468100001, 40.641689896999935], [-73.9696861179999, 40.64235600999992], [-73.96988464699999, 40.643030296999896], [-73.9700889779999, 40.643745839999944], [-73.97031677099991, 40.64453794099992], [-73.97054554399976, 40.64531666799991], [-73.97076976099986, 40.646110906999866], [-73.97084113799984, 40.64637857099989], [-73.97121039100001, 40.647620153999924], [-73.97139631700004, 40.64825778599986], [-73.97069344099991, 40.648496775999945], [-73.96967173699989, 40.64889793599989], [-73.96879367199982, 40.649248262999926], [-73.96790948399995, 40.649599373999884], [-73.9670280919999, 40.649947273999885], [-73.96614503299999, 40.650297679999866], [-73.96526281999996, 40.65064244999991], [-73.96465438699984, 40.650887727999894], [-73.96524103499988, 40.651742046999864], [-73.96575121499994, 40.65249922899995], [-73.96621321999996, 40.653189404999885], [-73.964144226, 40.65399426399989], [-73.96190028499997, 40.65487096199992], [-73.96156461500001, 40.655004116999905], [-73.96069809399995, 40.6553409239999], [-73.95986103199981, 40.65563478899989], [-73.95969665899997, 40.65483921499991], [-73.959451838, 40.65357538799987], [-73.95932926499988, 40.652928178999915], [-73.95915000999992, 40.65214455699985], [-73.95892527299989, 40.65121154999988], [-73.9587078989999, 40.65038727299989], [-73.95844335399988, 40.64930344299989], [-73.95825517200004, 40.64853244899996], [-73.95815003599986, 40.64776767399996], [-73.95814068499989, 40.647503403999934], [-73.95810672200003, 40.64651800899988], [-73.95803664499991, 40.64530249999989], [-73.95802176499988, 40.6450699619999], [-73.95797764699991, 40.64457625599989], [-73.95791873699986, 40.64414326499991], [-73.95789394399986, 40.64401556199989], [-73.95763644599988, 40.64268949899989], [-73.95746359899987, 40.64237166999987], [-73.95707031, 40.64196067099995], [-73.95571599999984, 40.64058252599991], [-73.95563820699984, 40.64050392599988], [-73.95476763299989, 40.63962444699996], [-73.95427358399985, 40.63911553899988], [-73.95404358599993, 40.6388775489999], [-73.95374853799996, 40.63858633799993]]]}}, {\"id\": \"89\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 90, \"Shape_Leng\": 0.0307591620819, \"Shape_Area\": 5.53132476305e-05, \"zone\": \"Flatiron\", \"LocationID\": 90, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9963368949999, 40.7380437469999], [-73.99683993899998, 40.73736088899995], [-73.99968392199986, 40.73855224899986], [-74.00252381499989, 40.73975264399995], [-74.00201937799989, 40.74044408899989], [-74.00155531499993, 40.741080025999935], [-74.00112917499989, 40.741668097999884], [-74.00070275499995, 40.74225361099988], [-74.0002751799999, 40.74283879799987], [-73.9998482359998, 40.74342501899993], [-73.99942208999991, 40.74400957299992], [-73.99897237399989, 40.74463005499994], [-73.99848652499993, 40.745294163999894], [-73.99800822000003, 40.74595130199988], [-73.99755164599995, 40.746580899999834], [-73.99709902899994, 40.74720510199991], [-73.99426258599996, 40.74600398099986], [-73.99141858599994, 40.74480268199988], [-73.9918710239999, 40.74418556999991], [-73.99206647199993, 40.74391620299988], [-73.99231888499988, 40.74356832299989], [-73.99280340900002, 40.74291001999997], [-73.99329166800001, 40.74223330099994], [-73.99373928799994, 40.74161720799987], [-73.99416492399986, 40.7410309369999], [-73.99459264899988, 40.740441980999925], [-73.99501798599992, 40.73986001099995], [-73.99544664199995, 40.7392721329999], [-73.99587409499989, 40.73868594599991], [-73.9963368949999, 40.7380437469999]]]}}, {\"id\": \"90\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 91, \"Shape_Leng\": 0.124996624585, \"Shape_Area\": 0.000537330013243, \"zone\": \"Flatlands\", \"LocationID\": 91, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91945472999996, 40.6401731369999], [-73.91941447399985, 40.63977466499991], [-73.91928984599986, 40.638596032999914], [-73.91917180399989, 40.63746984099993], [-73.91916482699992, 40.63740334199989], [-73.91905790399989, 40.63638510999988], [-73.9190277079998, 40.63609754399988], [-73.919006295, 40.63589361799995], [-73.91899154599992, 40.63575906299996], [-73.91895156599989, 40.63539431799991], [-73.91893983700002, 40.63528732099991], [-73.91892161299991, 40.635121060999914], [-73.91891212099982, 40.63503445999987], [-73.91889071099986, 40.6348391329999], [-73.91888284299993, 40.634767351999905], [-73.91886934599994, 40.63464421599985], [-73.91886199299982, 40.634577127999876], [-73.91885164499996, 40.634482718999955], [-73.91876282599999, 40.6336375289999], [-73.9186298239999, 40.63234687299989], [-73.91860649599992, 40.632090414999894], [-73.91824254499994, 40.62877007499986], [-73.91816260199982, 40.62800449499984], [-73.91799464599991, 40.62645772299993], [-73.91775240199999, 40.624164716999864], [-73.91770927100005, 40.6237176859999], [-73.91751556199989, 40.62194031599993], [-73.91746991599989, 40.62151242899988], [-73.91741279799999, 40.62095853099987], [-73.91726440900005, 40.61954803699993], [-73.91717137699983, 40.61871125199989], [-73.91716779499997, 40.618387961999886], [-73.91711554399998, 40.617925190999884], [-73.9170081, 40.6172019239999], [-73.91697651699978, 40.61693221199992], [-73.91690302699992, 40.61627422899993], [-73.91719587399997, 40.616091715999886], [-73.9173907319999, 40.61596630599988], [-73.91788753399999, 40.61564657999988], [-73.91862838199995, 40.615168606999916], [-73.91893348499988, 40.61496777999992], [-73.9193708869999, 40.61469022299991], [-73.9196079709999, 40.61453446399989], [-73.92010700000002, 40.614212128999874], [-73.9205269409999, 40.61394269499987], [-73.92062400299997, 40.6148386819999], [-73.92159370099995, 40.614778183999874], [-73.92252930300005, 40.61471903299993], [-73.92349786399998, 40.61465815799991], [-73.92446651399993, 40.61459941799995], [-73.9253954579999, 40.614540562999885], [-73.92640044599996, 40.61447853499988], [-73.9263159899999, 40.61343967599986], [-73.92629769899993, 40.61319215499985], [-73.92629604599998, 40.61311679899991], [-73.92739580799999, 40.614052690999884], [-73.92769203599991, 40.614312640999835], [-73.92776130099995, 40.61437342499987], [-73.92808935399992, 40.614667884999946], [-73.92842985399992, 40.61497277999991], [-73.92946094800007, 40.61589681999992], [-73.92989884499987, 40.616294271999934], [-73.93060244599985, 40.61692226299993], [-73.93173179199982, 40.61792692099988], [-73.93248727399993, 40.61743170199987], [-73.93320457599995, 40.6169704419999], [-73.93501858699992, 40.61860011099992], [-73.93686699599992, 40.620253280999904], [-73.93696272700001, 40.62019914099994], [-73.93757616399986, 40.619795528999866], [-73.93781351899986, 40.6196520959999], [-73.93828802399999, 40.619329018999906], [-73.93864732399993, 40.61909724599994], [-73.93900236399998, 40.61886727099992], [-73.93949341999998, 40.618550790999905], [-73.9397167159999, 40.618406095999866], [-73.94037010899984, 40.617992385999884], [-73.94045274699987, 40.617929049999894], [-73.94078914099987, 40.61769678999986], [-73.941282159, 40.617629269999924], [-73.94138899899984, 40.617617418999885], [-73.94221251499997, 40.617526054999864], [-73.94314510099991, 40.617425616999896], [-73.94325340699987, 40.617412063999936], [-73.94346961999989, 40.61738500099992], [-73.94395779499983, 40.616816308999894], [-73.94467599499988, 40.61616511299997], [-73.9448277099998, 40.61604620899986], [-73.94573691999982, 40.61536340699989], [-73.94576359499983, 40.61550406399987], [-73.94606699600003, 40.617103738999894], [-73.94651373199997, 40.61949401899995], [-73.94551663599992, 40.61960561199991], [-73.94451833400001, 40.61971364699989], [-73.9449703129999, 40.62210451199989], [-73.94542442899984, 40.624495609999926], [-73.94642266100001, 40.6243855669999], [-73.94687439899997, 40.62677366799994], [-73.94732672199994, 40.629166566999835], [-73.9463262219999, 40.629276641999915], [-73.94540053199998, 40.62937781199983], [-73.94446784099986, 40.629480051999884], [-73.94415535299993, 40.629508284999886], [-73.94453995099988, 40.62986656299989], [-73.94480816099995, 40.630127166999884], [-73.9455109549998, 40.63079148299986], [-73.94647961799998, 40.631708161999896], [-73.94561754699987, 40.63177742999988], [-73.94462885399984, 40.631837734999884], [-73.94365994499996, 40.63189942699991], [-73.94272877999995, 40.631957293999896], [-73.94175680799997, 40.63201662699987], [-73.94078821499988, 40.632075845999886], [-73.94032359799989, 40.63210526799983], [-73.93985458399987, 40.63213497499991], [-73.93892456999993, 40.63219308999986], [-73.93799799799994, 40.63224680599991], [-73.93759273599986, 40.63226785699986], [-73.9374083309999, 40.63228068499994], [-73.93701746200003, 40.63231097899989], [-73.93703669099992, 40.63250924999988], [-73.93706492799993, 40.63277116699994], [-73.9372670019998, 40.634650624999885], [-73.93629596699988, 40.634709525999874], [-73.93535985000001, 40.6347673119999], [-73.93439440199988, 40.634826802999925], [-73.93342687599988, 40.634884050999915], [-73.93249995699995, 40.63494706899993], [-73.93223554799995, 40.634963338999924], [-73.9315273509998, 40.63500796799989], [-73.93055333099983, 40.63506291499989], [-73.92962453299997, 40.63512131599987], [-73.92861283799982, 40.63518764599997], [-73.92766858599995, 40.63524502299991], [-73.92744822299987, 40.635258409999864], [-73.92712769499995, 40.63566745599995], [-73.9269145769998, 40.6359394399999], [-73.92682085699984, 40.63606066199992], [-73.92674442999993, 40.636173105999895], [-73.92668615699995, 40.636257087999915], [-73.92652490899994, 40.63649272299989], [-73.9262739919998, 40.63721532799985], [-73.92616377799996, 40.63757296099988], [-73.9261268489999, 40.63774352299986], [-73.9260813459999, 40.63816294599993], [-73.92606338099995, 40.63852406099986], [-73.92604638799999, 40.638603340999964], [-73.92596691799999, 40.63880208899997], [-73.92590483099994, 40.63900510599991], [-73.92586068599985, 40.63921107099989], [-73.925834756, 40.63941858799992], [-73.92582711299985, 40.63962626999988], [-73.92574389399985, 40.64001446599985], [-73.92570388299987, 40.6403609499999], [-73.92562125699988, 40.64081121999991], [-73.9255828079999, 40.64107003099984], [-73.92547887299986, 40.64158704199992], [-73.92538233099987, 40.64238453199987], [-73.92518618999996, 40.642397290999945], [-73.92454228399995, 40.64243917699988], [-73.92356525899983, 40.642499297999905], [-73.92341500199993, 40.640923605999916], [-73.92337418899983, 40.640495543999904], [-73.92335409199994, 40.64017632399992], [-73.92332654599986, 40.63990021599991], [-73.92154948099989, 40.64134469199992], [-73.92025802399995, 40.64236629899986], [-73.91973736999988, 40.64277814799987], [-73.91965951399997, 40.642078236999915], [-73.91958298599987, 40.64139576199993], [-73.919555766, 40.641184957999876], [-73.91953493999996, 40.640857740999884], [-73.91946588499995, 40.64028418699997], [-73.91946141199999, 40.64023929299983], [-73.91945472999996, 40.6401731369999]]]}}, {\"id\": \"91\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 92, \"Shape_Leng\": 0.117830066799, \"Shape_Area\": 0.000374946617289, \"zone\": \"Flushing\", \"LocationID\": 92, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82656578899986, 40.77064147599989], [-73.82655841999986, 40.770137083999906], [-73.82604927499997, 40.77021081399986], [-73.8255648519999, 40.77028295999992], [-73.82466908999987, 40.77040760799985], [-73.82444330299992, 40.7692855139999], [-73.82421202499988, 40.76816922799988], [-73.82395032599992, 40.76688720199994], [-73.82372658199988, 40.7658255159999], [-73.82348245199985, 40.76465179199993], [-73.82302513300002, 40.7636663409999], [-73.82260068399988, 40.76275987799985], [-73.82207166499991, 40.761625090999885], [-73.8218107709998, 40.76107223699987], [-73.82176735399993, 40.760979813999924], [-73.82155809699998, 40.76053432399992], [-73.82116404999985, 40.75968607499993], [-73.82080684899987, 40.7589417829999], [-73.82063593299985, 40.75887226799989], [-73.8200450059999, 40.75848322799985], [-73.819232864, 40.75794681199992], [-73.81856914499988, 40.7575188489999], [-73.81789245699991, 40.75708633799991], [-73.818936954, 40.75640957799994], [-73.81926771999996, 40.75619984799989], [-73.81961754399985, 40.75596557899992], [-73.82030067699999, 40.75551497999992], [-73.8184078289999, 40.75383042199998], [-73.819092321, 40.7533848199999], [-73.81977819399994, 40.752936894999884], [-73.820849288, 40.75224154499988], [-73.81998860999985, 40.751428620999896], [-73.81941130799996, 40.75088775499989], [-73.81911566899988, 40.750621072999856], [-73.81883698299988, 40.75035215799989], [-73.81826477299987, 40.749826989999875], [-73.81779541099995, 40.749392014999906], [-73.8176865099999, 40.749288211999925], [-73.81710925599998, 40.74876054399992], [-73.81654271199997, 40.748260041999856], [-73.81598751399984, 40.74776993599989], [-73.8153731739999, 40.747218337999946], [-73.81575069499985, 40.7469794799999], [-73.81657996199982, 40.74713778799989], [-73.81691898700001, 40.747204667999895], [-73.81782025899999, 40.74738541599988], [-73.8187314289999, 40.747711900999896], [-73.81961962699988, 40.748042042999934], [-73.82052281799989, 40.74835684999989], [-73.82108427599998, 40.748558445999954], [-73.82131898199997, 40.7487581829999], [-73.82239598999983, 40.74986422299986], [-73.82321319899984, 40.75070793199988], [-73.82426493900002, 40.75183288899994], [-73.8246103029999, 40.751626747999865], [-73.82492269399982, 40.75144029499993], [-73.82518837299989, 40.75128858499995], [-73.82576557299983, 40.750969023999914], [-73.82566698599992, 40.7509087479999], [-73.82565209100002, 40.75078150099995], [-73.82565201299991, 40.75065299899992], [-73.8256669259999, 40.7505247439999], [-73.82569664700004, 40.75039824999985], [-73.82574065099986, 40.750274963999864], [-73.82584440999999, 40.74934839099993], [-73.82673694299987, 40.74967723499986], [-73.82723921499992, 40.749859602999905], [-73.82737756299996, 40.74987437099992], [-73.82751792599984, 40.749874789999886], [-73.82765641899984, 40.749860830999886], [-73.82786693899982, 40.74981128499992], [-73.82807314199995, 40.74975046599994], [-73.82827349999988, 40.749678714999966], [-73.82846651899989, 40.74959657899997], [-73.82865083499996, 40.749504731999856], [-73.82882914999982, 40.74940718399992], [-73.82902159099983, 40.74932374899987], [-73.82922511999989, 40.74925608099989], [-73.82943632699994, 40.74920530899993], [-73.82965157399984, 40.749171992999905], [-73.83060941400002, 40.74902588399991], [-73.83071466599985, 40.74901721399987], [-73.83081942999986, 40.748995791999924], [-73.83091972599982, 40.748961710999886], [-73.8310116789999, 40.748916018999886], [-73.83109206499991, 40.74886061199991], [-73.83115861599995, 40.7487979979999], [-73.83121778600005, 40.74869012299991], [-73.83126084299984, 40.748576271999895], [-73.83128640999995, 40.74845858499988], [-73.83129377999995, 40.74833944999994], [-73.83128299699986, 40.7482212609999], [-73.8312547989999, 40.74810635999991], [-73.83300302099987, 40.74783363099988], [-73.83308582399988, 40.74798912499988], [-73.833116784, 40.74804727099989], [-73.83277623900001, 40.749313493999914], [-73.83228373899995, 40.751425268999895], [-73.83227581899978, 40.75145759599989], [-73.83225199799978, 40.75157800299994], [-73.832247237, 40.751700590999896], [-73.83226187399998, 40.75182272099988], [-73.8322953569999, 40.751941790999844], [-73.83560230899995, 40.75139485299991], [-73.83599840100001, 40.75200573699994], [-73.83701406699998, 40.75402742099987], [-73.83743572399989, 40.75408295599989], [-73.83766909799985, 40.75411369599991], [-73.83778038200002, 40.75413068299993], [-73.83798921299994, 40.75451504299986], [-73.83805974199996, 40.75466660699988], [-73.8381423329998, 40.75485403699987], [-73.83827816299996, 40.75523337799995], [-73.83834041199987, 40.75542546299986], [-73.83838479599993, 40.755640366999884], [-73.8384675259999, 40.7560401559999], [-73.83867321899987, 40.75701997899995], [-73.83861811099983, 40.75709081599992], [-73.83679107299986, 40.76116667099992], [-73.83689455800001, 40.76260268399992], [-73.83689970199995, 40.7626981179999], [-73.83691694399995, 40.762766679999885], [-73.8369514729998, 40.76283036999996], [-73.83790429599993, 40.764587796999876], [-73.83908354399988, 40.76525691299991], [-73.83922344399986, 40.765336291999844], [-73.83935912299995, 40.765409871999886], [-73.8394581342306, 40.76544576822929], [-73.83913189281712, 40.765991105784146], [-73.83871102200001, 40.76666838599987], [-73.83866554599997, 40.76674036599996], [-73.8386015299999, 40.76683050999985], [-73.83805766700006, 40.76759634399986], [-73.83746794599983, 40.76830671699987], [-73.83711212099999, 40.76864287199989], [-73.83678033699985, 40.76896614799991], [-73.83614052499993, 40.76952737399995], [-73.83536623599981, 40.77012144099994], [-73.83511636199991, 40.77028363399989], [-73.83433960199982, 40.77078781999989], [-73.83338983199998, 40.77134538399988], [-73.83209910199999, 40.77210429399989], [-73.83161455499989, 40.77241020899988], [-73.83150226899997, 40.77248795599991], [-73.83090463599999, 40.7729830689999], [-73.83000011099983, 40.77373240699996], [-73.82908256199991, 40.7747943209999], [-73.828793895, 40.77524292899992], [-73.82830621299996, 40.776000787999905], [-73.82789985099988, 40.7766822329999], [-73.82665895999996, 40.778818601999895], [-73.82642473899999, 40.779228734999954], [-73.82616437200001, 40.77968464499987], [-73.82607916299985, 40.77985255099991], [-73.82576928399986, 40.78046316399985], [-73.82557907999986, 40.78083309199997], [-73.82513740299993, 40.78169970599994], [-73.82498993100002, 40.7817014639999], [-73.82462418799997, 40.78170582499991], [-73.82384132799996, 40.781716885999835], [-73.82224127699999, 40.781693878999924], [-73.82243697299995, 40.781413836999896], [-73.8226154139999, 40.781126823999934], [-73.82277596999982, 40.780833776999906], [-73.82291816299993, 40.78053566699987], [-73.82331286899986, 40.77957865999986], [-73.82354681599996, 40.77903553799989], [-73.82376244899997, 40.77850455199988], [-73.82382108999985, 40.7783601379999], [-73.82405384599981, 40.7777178299999], [-73.82414480499983, 40.77747809299995], [-73.8242214729999, 40.777235226999935], [-73.8242835999999, 40.77698996099983], [-73.82433101899989, 40.776743069999924], [-73.82486800999989, 40.77654235499993], [-73.82583089999997, 40.77591527099986], [-73.82625348499994, 40.77452065899989], [-73.82647720799984, 40.77311644299986], [-73.82657481899986, 40.772345397999885], [-73.82657511799988, 40.771630352999935], [-73.82656578899986, 40.77064147599989]]]}}, {\"id\": \"92\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 93, \"Shape_Leng\": 0.170998027597, \"Shape_Area\": 0.000594278183978, \"zone\": \"Flushing Meadows-Corona Park\", \"LocationID\": 93, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8569635593979, 40.76407442786308], [-73.8566748236235, 40.763930188326135], [-73.85668299802357, 40.76392072858619], [-73.8568029095629, 40.763980845460814], [-73.8568542717766, 40.763922543905565], [-73.85686149910755, 40.76391434059379], [-73.85691477947755, 40.76385386154618], [-73.8569207286196, 40.76384710808833], [-73.85697496767924, 40.76378553994385], [-73.85698123375121, 40.76377842954332], [-73.85702898355706, 40.76372422680603], [-73.85703613508508, 40.76371611016756], [-73.8570833361649, 40.76366252925309], [-73.85708959006736, 40.7636554318984], [-73.85713828854304, 40.76360015443993], [-73.8571452328759, 40.76359227023111], [-73.8571946426761, 40.76353618498254], [-73.85720087844675, 40.76352910686474], [-73.85724903760318, 40.76347443965748], [-73.85725666284321, 40.76346578332061], [-73.85730643291356, 40.763409287863276], [-73.85731353865413, 40.76340122325119], [-73.85735990197853, 40.763348593736616], [-73.85736795787706, 40.76333944992225], [-73.85741662254271, 40.76328420899041], [-73.85742519945238, 40.76327447243974], [-73.85747258321598, 40.763220687374904], [-73.85747974744922, 40.76321255480913], [-73.85752810581124, 40.763157660396914], [-73.85753555737742, 40.763149201507154], [-73.85755915818416, 40.763122411786256], [-73.85754793889828, 40.763115565701206], [-73.85758285470693, 40.76307290062661], [-73.85746422874625, 40.76301478271995], [-73.85737359701484, 40.762970378519576], [-73.85718496087118, 40.762877960462795], [-73.85717606828264, 40.762873603814086], [-73.85709111433461, 40.76283198149714], [-73.85708275835432, 40.762827888020134], [-73.85698207201663, 40.76277855800093], [-73.85697105684375, 40.7627731613803], [-73.85688415442013, 40.7627305839378], [-73.85682507576576, 40.76270163909816], [-73.85682436749566, 40.762701292444234], [-73.85675571120355, 40.76266765415096], [-73.85674350380765, 40.76266167353797], [-73.85669035223006, 40.762635632858746], [-73.85660302580746, 40.76273141793303], [-73.8565867538878, 40.762722840114236], [-73.8567823696347, 40.76250827740658], [-73.85679858100602, 40.76251682329276], [-73.85671169399488, 40.76261568695248], [-73.85676092804188, 40.762639826189165], [-73.85677318714824, 40.76264583651674], [-73.85682558732012, 40.7626715270996], [-73.8568366081776, 40.76267693094509], [-73.85688788040733, 40.76270206793208], [-73.85690044326856, 40.762708227386334], [-73.85698999094755, 40.76275213155589], [-73.85709311210883, 40.76280268870123], [-73.85710529468155, 40.76280866153997], [-73.85718716762985, 40.76284880177785], [-73.85719629175131, 40.76285327497573], [-73.85728240762211, 40.76289549655458], [-73.85729174591326, 40.76290007370879], [-73.8573806782753, 40.76294367543134], [-73.85738951387346, 40.76294800770227], [-73.85747499963058, 40.7629899192708], [-73.85748453757667, 40.76299459449141], [-73.85760424037194, 40.76305328178898], [-73.85763578363799, 40.763012321964815], [-73.85765037151214, 40.763019173878945], [-73.8577084468895, 40.762950395280726], [-73.85771567256866, 40.762941835081996], [-73.85776740573347, 40.762880552278006], [-73.85777623039976, 40.7628700979901], [-73.85783497417715, 40.76280051049886], [-73.85784136663887, 40.76279293749988], [-73.85789701257401, 40.76272701849153], [-73.85794433387208, 40.762670962169565], [-73.8579670234807, 40.76264408295382], [-73.85797500120835, 40.76263463209225], [-73.8580451152925, 40.76255157517708], [-73.85807199853592, 40.762519729773025], [-73.85813288714913, 40.76244759870893], [-73.85814119991119, 40.762437750876416], [-73.85817631070509, 40.7623962942238], [-73.85781983894705, 40.76222613388971], [-73.85780788168364, 40.76222042710023], [-73.85768834724743, 40.76216337375971], [-73.85767257542702, 40.7621558457594], [-73.85752950976803, 40.76208756021165], [-73.8574600419264, 40.76205440229832], [-73.85737149406725, 40.762012137796255], [-73.85735065780416, 40.762002192446865], [-73.85718801666012, 40.76192456443414], [-73.85716303200887, 40.76191263827355], [-73.85700279501378, 40.76183615483602], [-73.85698328892109, 40.76182684448665], [-73.85683800035645, 40.761757495880914], [-73.85677632095462, 40.76172805592267], [-73.85667457135632, 40.76167948956146], [-73.85665617597306, 40.76167070874793], [-73.85661195813331, 40.761649542759294], [-73.85662772941468, 40.761629062393226], [-73.85677893473327, 40.76170097456675], [-73.85679154181352, 40.76170696991361], [-73.85688557543547, 40.76175169107733], [-73.85690054310561, 40.76175880925622], [-73.85699014273463, 40.76180142237466], [-73.85700272542273, 40.761807405606], [-73.85709011185035, 40.76184896685109], [-73.85710534324438, 40.761856210472295], [-73.85719139863377, 40.761897137075714], [-73.85720516175633, 40.761903682792855], [-73.8572941135809, 40.761945985785], [-73.85730868889425, 40.761952918978366], [-73.85739929667307, 40.761996009469485], [-73.85741179997495, 40.762001956372046], [-73.85750467691464, 40.762046126613114], [-73.85751841919019, 40.76205266188022], [-73.85761007602207, 40.762096253064605], [-73.85762293989328, 40.762102369423516], [-73.85771566567898, 40.76214646911336], [-73.85773024174472, 40.762153400578725], [-73.85781948058784, 40.76219584021852], [-73.8578332359116, 40.762202381494895], [-73.85819433132511, 40.7623741079845], [-73.85827164316137, 40.762287628352816], [-73.85828091939457, 40.76227725147556], [-73.85836158883386, 40.76218701512108], [-73.85844323288775, 40.76209568947986], [-73.85845113610489, 40.762099465008106], [-73.85851749742233, 40.7620149112025], [-73.85850740232593, 40.76200806491507], [-73.8585852628444, 40.76191700859695], [-73.85747662435531, 40.76130891668733], [-73.8562732064007, 40.76090240843036], [-73.85537633453939, 40.760669385032244], [-73.85530351821596, 40.76074715385173], [-73.85529349219507, 40.760742320360684], [-73.85522306397422, 40.76081695051194], [-73.8552098725408, 40.76080930077564], [-73.85527660311541, 40.76073426476228], [-73.85526921633013, 40.76072983580968], [-73.85533454594008, 40.760658154889406], [-73.85530266029582, 40.76064958525487], [-73.85499808982104, 40.760391170767384], [-73.85466366458877, 40.7601711695621], [-73.8545203082371, 40.76015934488466], [-73.85425219901792, 40.76025067778132], [-73.85396489201136, 40.76027142272702], [-73.85372403934942, 40.76024638863483], [-73.8532844944673, 40.76012651275205], [-73.85306618709187, 40.76003870448119], [-73.85294927727757, 40.76002140181005], [-73.85277928565588, 40.75997298911409], [-73.85210362063094, 40.759682826699695], [-73.85200759608603, 40.75957250068762], [-73.85188711257045, 40.75952236727886], [-73.85171252421836, 40.759401258516625], [-73.85158552901191, 40.75935542065696], [-73.85144347566316, 40.75941908427621], [-73.85133438850518, 40.75942068096519], [-73.85121495373862, 40.75941585413536], [-73.8509512195169, 40.759419583092814], [-73.8506905306545, 40.75944879109037], [-73.85042437822479, 40.75950641103758], [-73.85029807674991, 40.7595462437419], [-73.8501750399337, 40.7595916012256], [-73.85005568532608, 40.75964232956161], [-73.84994041798382, 40.759698256595414], [-73.84970908350634, 40.759838837957474], [-73.84950202299183, 40.76000672901061], [-73.84933909178683, 40.76018858944728], [-73.84934906664418, 40.7601982826528], [-73.84930110814332, 40.76024287534997], [-73.84987484496149, 40.76110381294847], [-73.8518167929994, 40.76035373284717], [-73.85184046510508, 40.760305978821485], [-73.85197412962866, 40.76024940738178], [-73.85207200177854, 40.7603928846721], [-73.84981162475142, 40.76127097400822], [-73.84916662406035, 40.760296584790396], [-73.84914140272522, 40.760306890639015], [-73.84891393714967, 40.76027578667359], [-73.8486865565891, 40.76013849511064], [-73.84858834617657, 40.76017956308828], [-73.84821242063795, 40.760437594507884], [-73.84796233649804, 40.76055108883418], [-73.84670933714987, 40.76134526931977], [-73.84663046699997, 40.76129165899989], [-73.84606731299985, 40.76090886399986], [-73.84601517799996, 40.76080248899988], [-73.84597463499995, 40.76071730199993], [-73.8459114399999, 40.76058469999987], [-73.84585235799983, 40.76045539799988], [-73.8454898629999, 40.75972867199994], [-73.84513404100002, 40.75906847499993], [-73.84479027099985, 40.75840315899994], [-73.84444970399977, 40.75774348399989], [-73.84404431200002, 40.756944772999915], [-73.84363442799987, 40.75615699399987], [-73.84324381399976, 40.75533680399995], [-73.84253360699991, 40.75554469899986], [-73.84183146799998, 40.75576926799991], [-73.84113839099994, 40.756010244999864], [-73.84045537199995, 40.756267296999965], [-73.83978335899995, 40.75654003599987], [-73.8391232999999, 40.75682806199994], [-73.83875129700003, 40.756994991999925], [-73.83867321899987, 40.75701997899995], [-73.8384675259999, 40.7560401559999], [-73.83838479599993, 40.755640366999884], [-73.83834041199997, 40.755425461999906], [-73.83827816299996, 40.75523337799995], [-73.83814233299988, 40.75485403799992], [-73.83805974199996, 40.75466660699988], [-73.83798921299986, 40.7545150419999], [-73.83778038200002, 40.75413068299993], [-73.83766909799985, 40.75411369599991], [-73.83743572399989, 40.75408295599989], [-73.83701406699998, 40.75402742099987], [-73.83599840099994, 40.752005735999916], [-73.83560230899995, 40.75139485299991], [-73.8322953569999, 40.751941790999844], [-73.83226187399998, 40.75182272199993], [-73.832247237, 40.751700590999896], [-73.83225199799978, 40.75157800399993], [-73.83227581899978, 40.7514575949999], [-73.83228373899995, 40.751425268999895], [-73.83277623900001, 40.74931349499994], [-73.833116784, 40.74804727099989], [-73.83308582399988, 40.74798912499988], [-73.83300302099987, 40.74783363099988], [-73.83364897199982, 40.74764609899995], [-73.83370364199992, 40.74762634799989], [-73.83392204099982, 40.74753213699994], [-73.83392218299996, 40.74753207499989], [-73.83392232599984, 40.747532001999936], [-73.83413036799998, 40.7474264389999], [-73.83426490799998, 40.747337797999926], [-73.83426506200006, 40.74733769899993], [-73.834265193, 40.74733759999988], [-73.83438980999983, 40.74723985799992], [-73.83450304299996, 40.747133967999886], [-73.83450311499995, 40.74713390399994], [-73.83450317399993, 40.74713383299994], [-73.83460358999982, 40.74702113399993], [-73.834603649, 40.747021071999875], [-73.83460369599997, 40.747020998999886], [-73.83469042599992, 40.746902697999936], [-73.83506574799989, 40.74620410899988], [-73.83543777199996, 40.74552397399991], [-73.83556958700001, 40.745150630999916], [-73.83567553899992, 40.74485051799991], [-73.83584967799999, 40.7441757629999], [-73.83595071499995, 40.74373056899992], [-73.835925305, 40.743500569999846], [-73.83591564899993, 40.74343089299987], [-73.83590196999995, 40.74337980599992], [-73.83586131899989, 40.743227984999926], [-73.83609989199991, 40.7424290689999], [-73.83687304099996, 40.74213754699986], [-73.83690407799999, 40.74212583899989], [-73.83699037399995, 40.74209330099992], [-73.8371072069999, 40.7413962279999], [-73.8369627279999, 40.74065666099988], [-73.83679277399987, 40.73990191199988], [-73.83675088199985, 40.73972053399992], [-73.83669458799986, 40.73954125399997], [-73.83662419799995, 40.739365059999905], [-73.83627527100006, 40.73956837599992], [-73.83564585599999, 40.738491010999894], [-73.83563372699986, 40.737111629999916], [-73.83574733199998, 40.73699114199996], [-73.8361887679999, 40.73562665299992], [-73.83628669399984, 40.73513136699995], [-73.83635625099998, 40.73500859199989], [-73.8361277859999, 40.734241311999924], [-73.83606223299994, 40.734098694999865], [-73.8360070899999, 40.733978740999916], [-73.83584824699989, 40.73383374899988], [-73.83489939499987, 40.732967641999906], [-73.83448336599982, 40.73216391599991], [-73.83400368799985, 40.7315677099999], [-73.83386267199987, 40.730553188999885], [-73.83307087999992, 40.729919030999966], [-73.832895707, 40.7295863309999], [-73.83269855399999, 40.72925986199992], [-73.83248002199991, 40.7289408539999], [-73.83224087999997, 40.7286305209999], [-73.83198207999989, 40.728329997999865], [-73.83170465899991, 40.728040331999836], [-73.83140452799991, 40.72759762199997], [-73.83086471499999, 40.72670963399988], [-73.83073214599987, 40.72654932799991], [-73.83057976800002, 40.72639650399984], [-73.83057961499996, 40.72639635099986], [-73.83057943699991, 40.72639619699987], [-73.83040824999996, 40.7262528629999], [-73.83040820199987, 40.726252827999915], [-73.83040815599992, 40.72625279999995], [-73.83021964699995, 40.72612077799993], [-73.83021953999997, 40.72612069599989], [-73.83021940999994, 40.7261206249999], [-73.8300158949998, 40.72600180799986], [-73.83001569399997, 40.72600168999989], [-73.83001549299993, 40.72600159099995], [-73.82979916899991, 40.72589728099996], [-73.82979910999985, 40.72589725399995], [-73.82979903999995, 40.72589722599985], [-73.82957260399982, 40.72580846999996], [-73.82957252099986, 40.725808433999866], [-73.82957243899995, 40.725808406999896], [-73.82933887099998, 40.72573593899991], [-73.82933877699988, 40.72573592099995], [-73.82910097699988, 40.72567999799992], [-73.82887984399996, 40.725638253999925], [-73.828879714, 40.72563822699989], [-73.82887959600001, 40.725638191999906], [-73.82866154299994, 40.725581422999866], [-73.82844909199999, 40.72550982199991], [-73.82824491699995, 40.72542399599991], [-73.82805160699986, 40.72532493099987], [-73.82805145299996, 40.725324849999915], [-73.8280513119999, 40.725324759999864], [-73.82787086399985, 40.72521360999989], [-73.82770512299993, 40.72509185599994], [-73.8277049809999, 40.725091747999905], [-73.827704851, 40.72509162999989], [-73.8275552779999, 40.72496094999987], [-73.82742447699991, 40.72480770299991], [-73.82742443099994, 40.724807657999875], [-73.82742439499987, 40.72480760399992], [-73.8273090039999, 40.72464607699988], [-73.82721028599994, 40.7244775779999], [-73.8271293019999, 40.72430378399993], [-73.82712926599994, 40.72430371199989], [-73.82712924299986, 40.7243036399999], [-73.82706662699995, 40.7241260829999], [-73.82702267100002, 40.72394639099995], [-73.82672635699993, 40.72308159999988], [-73.82585945699992, 40.72046263199995], [-73.8257413029999, 40.72022165599988], [-73.82564380900001, 40.719974137999934], [-73.825643726, 40.71997393899991], [-73.82564366799977, 40.719973732999904], [-73.825567704, 40.719721168999854], [-73.82556765699992, 40.71972101499993], [-73.82556762199991, 40.719720870999936], [-73.82551375400004, 40.71946457499994], [-73.82551374199984, 40.719464530999865], [-73.82551374199986, 40.71946448499992], [-73.82548231999989, 40.71920620699995], [-73.82548231999995, 40.71920613499989], [-73.8254734689999, 40.71894785499994], [-73.82547345799988, 40.7189476569999], [-73.82547346999992, 40.718947467999875], [-73.82548692499986, 40.71869066199987], [-73.82545983199994, 40.71799986599993], [-73.8254192649999, 40.71739254399993], [-73.82531644999993, 40.71727569999994], [-73.82518614699985, 40.717288189999884], [-73.82487503799997, 40.71732243199995], [-73.8237298389999, 40.717437003999855], [-73.82332432099994, 40.71745250999989], [-73.82291752899988, 40.71745256099993], [-73.82251099399988, 40.717437075999904], [-73.82210623799986, 40.71740613099991], [-73.821704791, 40.717359889999905], [-73.82085213599989, 40.717348876999935], [-73.81962622499992, 40.7173404429999], [-73.81952343899992, 40.71733973299991], [-73.8191283259999, 40.71734991899995], [-73.81732944799997, 40.717660456999916], [-73.81725907799989, 40.71763787699993], [-73.81729568999994, 40.71757964499995], [-73.81887154599995, 40.71725932899994], [-73.81907861099985, 40.71720434499988], [-73.81930469499987, 40.71713608099988], [-73.82005442499994, 40.716810333999916], [-73.82018883899984, 40.71673522399984], [-73.82051599199986, 40.71681199099988], [-73.82100705300005, 40.71687995099992], [-73.82187417699991, 40.716917117999856], [-73.82346833799978, 40.71673105099993], [-73.825090876, 40.71645279299986], [-73.82524731699984, 40.71645544499987], [-73.82561290699984, 40.7162098979999], [-73.82566233799984, 40.716152637999926], [-73.82591945199992, 40.71598987599992], [-73.82627826, 40.71580688099987], [-73.82630699099998, 40.71579072399993], [-73.82646966099986, 40.715695567999916], [-73.82681044699991, 40.715550945999965], [-73.82700619999987, 40.715440297999855], [-73.82715524299999, 40.71538800399986], [-73.82850807699991, 40.71504432899996], [-73.82926493499993, 40.714792229999894], [-73.82995752699982, 40.714580886999876], [-73.83031312099993, 40.71449974499997], [-73.83082284299995, 40.71438342799993], [-73.8309675249999, 40.71435449099987], [-73.83103141800002, 40.71433672599993], [-73.83106922200002, 40.71435395599992], [-73.83115849800005, 40.71439336699996], [-73.83125888999979, 40.71444125999994], [-73.83145866999996, 40.714554448999934], [-73.83138630199987, 40.71457739699988], [-73.83122975599984, 40.71462702899991], [-73.83050460999999, 40.7148569439999], [-73.83035327299996, 40.7149518069999], [-73.83021283199997, 40.71505777899988], [-73.83008542999981, 40.71517363999986], [-73.8300853579999, 40.71517370299991], [-73.83008529899986, 40.71517376599995], [-73.82997259699991, 40.71529812199986], [-73.82987589099987, 40.71542940099992], [-73.8297961659998, 40.71556571899986], [-73.83072818099994, 40.71625964099996], [-73.8320714099999, 40.717290595999884], [-73.83348148399978, 40.71833345599985], [-73.83456936899987, 40.71902728199992], [-73.83457034900002, 40.71902795899989], [-73.83468657499992, 40.71910892099989], [-73.83479214399989, 40.71920142799992], [-73.83488359399998, 40.719303271999934], [-73.8349589409999, 40.71941247699992], [-73.83501684199989, 40.71952679899991], [-73.83505666300006, 40.719643840999886], [-73.83511286899991, 40.71991727199992], [-73.83518572100002, 40.72018871899989], [-73.83527502199999, 40.72045730599992], [-73.83538045099985, 40.720722179999925], [-73.83556698599992, 40.72104319099987], [-73.83576124499994, 40.72136157399993], [-73.83596312399996, 40.72167716799987], [-73.83603385999993, 40.72176720199997], [-73.83652367699997, 40.722370553999944], [-73.83836161400004, 40.725561841999856], [-73.83871473099992, 40.72622079099988], [-73.83906354600003, 40.7268795339999], [-73.8394453729999, 40.7275367999999], [-73.83985434299984, 40.72817756899988], [-73.84032703199988, 40.728807114999896], [-73.84146810899993, 40.72999809299991], [-73.84162181800001, 40.73011794699988], [-73.8428141259999, 40.73115341199989], [-73.84350489099987, 40.73181434399993], [-73.84370558699993, 40.73203240499986], [-73.84389098199985, 40.73225885599994], [-73.844060203, 40.7324927399999], [-73.84421250399983, 40.73273304099995], [-73.84434729799992, 40.73297869299997], [-73.84475302099987, 40.73361999799993], [-73.8449680009999, 40.734007128999906], [-73.84511542099983, 40.734277407999855], [-73.84542613099978, 40.734949802999935], [-73.84569341599997, 40.7356664799999], [-73.84590881499994, 40.736399800999884], [-73.84605541899985, 40.73709918799988], [-73.84689366199993, 40.73867121699989], [-73.84721361599989, 40.73864398699985], [-73.84732494199989, 40.738771457999896], [-73.84746313999992, 40.73895203099994], [-73.84750820299992, 40.73900780699995], [-73.8475557979999, 40.7390639149999], [-73.84767033700003, 40.7391952349999], [-73.84775740199994, 40.739299088999864], [-73.84777844199982, 40.7393448349999], [-73.84823627999982, 40.739290400999955], [-73.84794174099986, 40.73981640499991], [-73.84780164699998, 40.740049676999874], [-73.84795885999989, 40.74009701599994], [-73.84892344899993, 40.740391914999854], [-73.8493530509999, 40.74052324999996], [-73.84965207399982, 40.740614663999935], [-73.84984431299986, 40.740665549999896], [-73.84996425999991, 40.74070177199997], [-73.85011556599989, 40.74072356299995], [-73.85020444599994, 40.74090203399989], [-73.85051160899988, 40.741518856999946], [-73.85086917499987, 40.74218041099995], [-73.85099327299999, 40.74242254299989], [-73.85120929199985, 40.74284402099991], [-73.85135102799995, 40.743119057999884], [-73.85155107299987, 40.74350722699996], [-73.85189537499998, 40.74417973799992], [-73.85224097499999, 40.74484096599985], [-73.85257733899984, 40.7455056469999], [-73.8529235349999, 40.74618194799996], [-73.85325493199986, 40.74682354899991], [-73.85360568399982, 40.74749682799991], [-73.8537828699998, 40.74780299099994], [-73.85382450799997, 40.74785403999988], [-73.85405453299994, 40.748158316999906], [-73.85436928099995, 40.748738929999874], [-73.85442575299999, 40.748851725999884], [-73.8513110499998, 40.74984337599988], [-73.85086439699992, 40.74999564199986], [-73.85098691399993, 40.75023929799988], [-73.85118150899994, 40.75062629599989], [-73.85151883499987, 40.751297136999916], [-73.85171508699987, 40.751687413999875], [-73.85185614699996, 40.75196793299992], [-73.85220450599998, 40.752660681999934], [-73.85246329699996, 40.75312740899989], [-73.85261725399984, 40.75340506599992], [-73.85300762299985, 40.75455456599985], [-73.85302119999993, 40.754618001999866], [-73.85312635499996, 40.75510934299989], [-73.85318740799981, 40.755400028999944], [-73.85335273999982, 40.756017825999876], [-73.85339582599983, 40.75614235899991], [-73.8535413749999, 40.75656303299997], [-73.85377320199979, 40.75711640099996], [-73.85379813799992, 40.757177611999914], [-73.85405570299993, 40.75760316299988], [-73.85436796999998, 40.75811908299986], [-73.85449593299987, 40.75827863899985], [-73.8545797199998, 40.7583876199999], [-73.85463995799994, 40.75846615799992], [-73.85500920899997, 40.758879225999884], [-73.855067026, 40.758943277999926], [-73.85525140099983, 40.7591475299999], [-73.85553311699984, 40.75934259099992], [-73.85576997499993, 40.75953834299993], [-73.85603980899994, 40.75972578699984], [-73.85638216199997, 40.7599066399999], [-73.85655895700003, 40.75998156099994], [-73.856705981, 40.76004386599993], [-73.85734910199996, 40.76030063399988], [-73.85754427999996, 40.76037765399996], [-73.85773604799981, 40.760459477999845], [-73.857924202, 40.760546017999935], [-73.85810853999996, 40.7606371819999], [-73.85828886699981, 40.760732872999895], [-73.8584649899998, 40.760832988999894], [-73.858636721, 40.76093742299987], [-73.85880387699986, 40.76104606399992], [-73.85896628099991, 40.761158795999904], [-73.85912375899987, 40.76127549899994], [-73.85913545299991, 40.76141844099987], [-73.859666663, 40.761901912999896], [-73.85992937999985, 40.76220030099993], [-73.86109724399991, 40.76366447799989], [-73.86093691499987, 40.76372861499989], [-73.86057039699999, 40.76383121299987], [-73.86043337809801, 40.76386888243776], [-73.86001698390152, 40.76326428551858], [-73.85975083822206, 40.7628778391954], [-73.85949072467386, 40.76256261539987], [-73.8594133939809, 40.7624832653393], [-73.85922062887968, 40.762482744481545], [-73.85915607632676, 40.76251963069956], [-73.85907893363806, 40.76252283227078], [-73.85892056312618, 40.762406401185224], [-73.85899197332793, 40.762246523709955], [-73.85898423264058, 40.7621714744713], [-73.85892298182517, 40.762122346521515], [-73.85879204218736, 40.76227933874763], [-73.85867190067573, 40.76252227000638], [-73.85864510487932, 40.76251439590902], [-73.85876525911517, 40.76226519184914], [-73.85888722726258, 40.762133609398525], [-73.85883984926524, 40.76210532221611], [-73.85871170031803, 40.762233761955024], [-73.85869110268315, 40.762221190500945], [-73.85884371506147, 40.76205876728172], [-73.85861915281858, 40.7619355969989], [-73.85854218676641, 40.762026898691005], [-73.85853208671435, 40.76202176133262], [-73.8584808798073, 40.762082699861345], [-73.85847289638568, 40.762092199501595], [-73.85846125776057, 40.76210605116506], [-73.85846562340063, 40.76210880228016], [-73.85840968995647, 40.76217335587738], [-73.85840137452446, 40.762183042901206], [-73.85833010330997, 40.762266065611094], [-73.85832178366148, 40.76227575597448], [-73.85824801084416, 40.762361693691965], [-73.85823701549468, 40.76237450310679], [-73.85816550201432, 40.76245780652361], [-73.85815675514272, 40.76246799574389], [-73.85808172268878, 40.76255539983264], [-73.85807451463484, 40.76256379539836], [-73.85800186588557, 40.76264842076873], [-73.85799432914435, 40.762657200731425], [-73.85793183124306, 40.762730003860696], [-73.85792332108373, 40.76273991559671], [-73.85786819502788, 40.76280413118438], [-73.857859431059, 40.76281433962614], [-73.85780092280253, 40.762882493565314], [-73.85779250823931, 40.76289229618249], [-73.85773856319483, 40.76295513258302], [-73.85773293300596, 40.76296169164485], [-73.85767843741588, 40.76302774852689], [-73.85768965370534, 40.763036305706244], [-73.85765473616219, 40.76307982335119], [-73.85777497639125, 40.76313883512948], [-73.85786962064152, 40.763185283759206], [-73.85788103167226, 40.763190884493554], [-73.85796457800302, 40.76323188698831], [-73.85805470915611, 40.76327612040054], [-73.85815033962187, 40.763323053661004], [-73.85815280980245, 40.7633242660726], [-73.85816421582143, 40.76332986291929], [-73.85825464723334, 40.763374244804325], [-73.85826783696172, 40.763380717148394], [-73.85835219792034, 40.76342211890684], [-73.8584153868299, 40.76345313042265], [-73.85848633892353, 40.763369346378035], [-73.85850081255238, 40.763376433585044], [-73.85834319308863, 40.76356256146058], [-73.85832948824915, 40.763555851275136], [-73.85839531409529, 40.76347705912173], [-73.8583507077488, 40.76345478950192], [-73.8583381131546, 40.763448503350624], [-73.8582525680361, 40.763405793589826], [-73.8582413092282, 40.763400172809334], [-73.85813903143266, 40.763349110468035], [-73.85812806185179, 40.76334363326728], [-73.85805097021218, 40.76330514587175], [-73.85803917484533, 40.76329925703871], [-73.85795122974932, 40.763255349132415], [-73.85794243473829, 40.763250958823285], [-73.85786245501016, 40.76321102718345], [-73.85784979492291, 40.76320470688977], [-73.85777061776874, 40.76316517706584], [-73.85775931084058, 40.76315953255724], [-73.85763896602008, 40.763099448634655], [-73.85760068527979, 40.76314040113684], [-73.85758834275414, 40.763133551992205], [-73.85683888701782, 40.76400017911488], [-73.85697078354372, 40.76406606756879], [-73.8569635593979, 40.76407442786308]]]}}, {\"id\": \"93\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 94, \"Shape_Leng\": 0.0498326083147, \"Shape_Area\": 6.25819407393e-05, \"zone\": \"Fordham South\", \"LocationID\": 94, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89964261700001, 40.862210436999874], [-73.89943670799998, 40.862116911999856], [-73.89841038399996, 40.862296966999935], [-73.89798099399988, 40.86240253999991], [-73.89737274799985, 40.86243236399985], [-73.897143798, 40.86244506499991], [-73.89773815399992, 40.86135113699993], [-73.89802453099999, 40.86082046099989], [-73.89822728999994, 40.860448863999885], [-73.89828973399992, 40.86033442099995], [-73.898590972, 40.859766782999934], [-73.89880457699982, 40.8593642709999], [-73.89883814599983, 40.859307307999906], [-73.89866157499996, 40.859246901999896], [-73.89818244699991, 40.85908368099995], [-73.89735765499988, 40.85876273499986], [-73.89636314599984, 40.85837531899991], [-73.89573924699978, 40.85813820699986], [-73.8946932639999, 40.85942628199991], [-73.89381026099987, 40.858986564999924], [-73.89321520799987, 40.85987420599986], [-73.89256392599998, 40.860522728999925], [-73.89141802399983, 40.86166298399996], [-73.89138023399983, 40.86170058799992], [-73.89106280699994, 40.8615294119999], [-73.89106059999996, 40.861471814999945], [-73.89115375599981, 40.86091361599996], [-73.89174394999986, 40.860177146999945], [-73.89225707599982, 40.85940997599994], [-73.8929935429999, 40.85838611999985], [-73.89441355099994, 40.85643377599997], [-73.89477596099992, 40.85592952599996], [-73.89549889799987, 40.85474976599994], [-73.89560883299981, 40.85457036999986], [-73.89644052599982, 40.854907758999914], [-73.89709926799998, 40.855172290999874], [-73.89831118599984, 40.85406115099989], [-73.89863543699985, 40.85390697499993], [-73.89816577499982, 40.85514637599989], [-73.89877598599995, 40.85536324699995], [-73.89967155800004, 40.85566890799991], [-73.90060385999978, 40.85598723799995], [-73.90077922699986, 40.85604710999993], [-73.90085477199993, 40.8559374399999], [-73.90116590999993, 40.85549567499992], [-73.90130126599995, 40.85530348999985], [-73.90153703400001, 40.85499464299988], [-73.90185996099993, 40.85458614899994], [-73.90208001099982, 40.85462259899992], [-73.90298864599995, 40.854773108999936], [-73.90395154299999, 40.854973922999896], [-73.90472622199998, 40.85537039499996], [-73.90557007299988, 40.855787321999884], [-73.90522169999987, 40.856203594999926], [-73.90508614099994, 40.856410830999955], [-73.90468371399987, 40.85706030199991], [-73.90565953499988, 40.85739873399997], [-73.90526389299997, 40.85804717799987], [-73.90486847199993, 40.8586932249999], [-73.90447464099996, 40.85934159499988], [-73.90408132700006, 40.859987847999875], [-73.90351390599986, 40.86091750999989], [-73.90254479399998, 40.86057825499988], [-73.90233887199986, 40.86095497099995], [-73.90211720400006, 40.8613269479999], [-73.90188007299994, 40.86169362499991], [-73.90162785099997, 40.86205443399997], [-73.90136090399993, 40.8624088639999], [-73.90107966999994, 40.86275638299992], [-73.90028368099995, 40.86247183699992], [-73.89996177599984, 40.86235027999989], [-73.89964261700001, 40.862210436999874]]]}}, {\"id\": \"94\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 95, \"Shape_Leng\": 0.113629605476, \"Shape_Area\": 0.00057277033803, \"zone\": \"Forest Hills\", \"LocationID\": 95, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84732494199989, 40.738771457999896], [-73.84721361599989, 40.73864398699985], [-73.84689366199993, 40.73867121699989], [-73.84605541799989, 40.737099187999874], [-73.84590881499994, 40.736399800999884], [-73.84569341599997, 40.7356664799999], [-73.84542613099978, 40.734949802999935], [-73.84511541999991, 40.734277407999855], [-73.8449680009999, 40.734007128999906], [-73.84475302099987, 40.73361999799993], [-73.84434729799992, 40.73297869299997], [-73.84421250399983, 40.73273304099995], [-73.844060203, 40.7324927399999], [-73.84389098199985, 40.73225885599994], [-73.84370558699993, 40.73203240499986], [-73.84350489099987, 40.73181434399993], [-73.8428141259999, 40.73115341199989], [-73.84162181699999, 40.73011794699993], [-73.84146810899993, 40.72999809299991], [-73.84032703199988, 40.728807114999896], [-73.83985434299984, 40.72817756899988], [-73.8394453729999, 40.7275367999999], [-73.83906354600003, 40.7268795339999], [-73.83871473099992, 40.72622079099988], [-73.83836161400004, 40.725561841999856], [-73.8365236769999, 40.722370552999884], [-73.83603385999993, 40.72176720199997], [-73.83596312399996, 40.72167716799987], [-73.83576124499994, 40.72136157399993], [-73.83556698599992, 40.72104319099987], [-73.83538045099985, 40.720722179999925], [-73.83527502199999, 40.72045730599992], [-73.83518572100002, 40.72018871899989], [-73.83511286899991, 40.71991727199992], [-73.8350566629999, 40.71964384199989], [-73.83501684199989, 40.71952679899991], [-73.8349589409999, 40.71941247699992], [-73.83488359399998, 40.719303271999934], [-73.83479214399989, 40.71920142799992], [-73.83468657499992, 40.71910892099989], [-73.834570349, 40.719027959999934], [-73.83456936899987, 40.71902728199992], [-73.83348148399978, 40.71833345599985], [-73.8320714099999, 40.717290595999884], [-73.83072818099994, 40.71625964099996], [-73.8297961659998, 40.71556571899986], [-73.82987589099987, 40.71542940099992], [-73.82997259699991, 40.71529812199986], [-73.83008529899986, 40.71517376599995], [-73.8300853579999, 40.71517370299991], [-73.83008542999981, 40.71517363999986], [-73.83021283199997, 40.71505777899988], [-73.83035327299996, 40.7149518069999], [-73.83050460999999, 40.7148569439999], [-73.83122975599984, 40.71462702899991], [-73.83138630199986, 40.71457739799996], [-73.83145866999996, 40.714554448999934], [-73.83125888999979, 40.71444125999994], [-73.83115849800005, 40.71439336699996], [-73.83106922200002, 40.71435395599992], [-73.83103141799982, 40.71433672499987], [-73.83115264699987, 40.7143068559999], [-73.83129817899982, 40.714264132999844], [-73.83135959999987, 40.71423170499988], [-73.83243372799994, 40.71365996799988], [-73.83317906799991, 40.71326322499991], [-73.83330196199995, 40.71320007099996], [-73.83355895899987, 40.713068003999865], [-73.833657742, 40.71301572899994], [-73.83374150300001, 40.712969680999905], [-73.83426226999985, 40.71268337399986], [-73.83505175599991, 40.712253576999885], [-73.83600510799992, 40.71172967199989], [-73.83602786500002, 40.711749063999896], [-73.83611476599982, 40.71182514699994], [-73.83640857599988, 40.71164176899991], [-73.83640880199984, 40.71164162599989], [-73.83640902599988, 40.71164149999986], [-73.83671639099991, 40.711470315999854], [-73.83671661599983, 40.711470191999894], [-73.83671685300001, 40.71147007499992], [-73.83703669299996, 40.711311803999926], [-73.83703688199994, 40.711311704999915], [-73.83703708399995, 40.7113116139999], [-73.837368179, 40.711166928999944], [-73.8377089419999, 40.71103648399989], [-73.83805756099987, 40.71092087199988], [-73.83952312699982, 40.7104964509999], [-73.83972439899992, 40.71039876599992], [-73.83978025199983, 40.710385523999925], [-73.840191993, 40.710276828999916], [-73.84024206299989, 40.71026193199985], [-73.84069154199982, 40.710153213999924], [-73.84133936599986, 40.709946233999894], [-73.84263067999993, 40.709432976999885], [-73.84267962199995, 40.709414123999906], [-73.8430466589999, 40.7092641499999], [-73.84314625299982, 40.70922345599992], [-73.84321861899991, 40.70918746199985], [-73.84347552199995, 40.70906047099992], [-73.84381825699988, 40.70891181999989], [-73.84417281299982, 40.70877842099986], [-73.84453738799996, 40.708661054999894], [-73.84491011200004, 40.708560330999944], [-73.84528898599984, 40.70847667899995], [-73.84567198199987, 40.70841036599989], [-73.84615958399999, 40.70835081999995], [-73.84665724099989, 40.70831864199995], [-73.8476241019999, 40.708332622999855], [-73.84857838799986, 40.70843871999993], [-73.84876725199985, 40.70844457899986], [-73.84895712599986, 40.70843813799993], [-73.84914591599988, 40.7084193199999], [-73.84933156199999, 40.70838833399994], [-73.84951207399992, 40.708345660999925], [-73.8504290539999, 40.7080870669999], [-73.85059051099984, 40.70803988299985], [-73.85110121699995, 40.707900982999895], [-73.85268225699998, 40.70741814499989], [-73.85275370599992, 40.7073971469999], [-73.85473240499985, 40.70676414299989], [-73.85593991699984, 40.7063980339999], [-73.85595536199988, 40.70642977899992], [-73.85667714399985, 40.70769187399986], [-73.85729287299985, 40.711467676999874], [-73.85744857899988, 40.7136927999999], [-73.85777168499992, 40.71645629299992], [-73.85823757099998, 40.719612410999936], [-73.85852214099987, 40.72153379399993], [-73.8586809759999, 40.72261243999988], [-73.85990796499992, 40.72434623399985], [-73.86203833799982, 40.72595678499994], [-73.86015205599992, 40.725227040999876], [-73.85960517199996, 40.7250223999999], [-73.85805416899993, 40.72444540599986], [-73.85776316799985, 40.72490571199994], [-73.85727108100001, 40.72568129799988], [-73.8594198799999, 40.726464837999885], [-73.85901374300002, 40.7271089069999], [-73.85860783699987, 40.72775228399992], [-73.85820066499991, 40.72839135199992], [-73.85812618599992, 40.728508245999905], [-73.8580876569999, 40.728568721999864], [-73.85775002599989, 40.72844560099985], [-73.85741240699991, 40.72832247999986], [-73.85771592500002, 40.728505460999926], [-73.85762923599995, 40.728606921999884], [-73.857827674, 40.72897471999988], [-73.85816500199988, 40.72962493499991], [-73.85837971299996, 40.73003567799985], [-73.85851097899999, 40.73028680199993], [-73.85625665099991, 40.73095912299993], [-73.85659673999993, 40.731623059999926], [-73.85704937999998, 40.732512433999894], [-73.85730927099998, 40.73300411099996], [-73.85765183100001, 40.73366788599986], [-73.85799293699985, 40.73433365699993], [-73.85834345899994, 40.73500123999988], [-73.8585833349998, 40.73547194199987], [-73.85867078199992, 40.73561895999997], [-73.85872465799989, 40.73570864399988], [-73.85862016399997, 40.73573990699991], [-73.85801767699988, 40.73590708399994], [-73.8566154109999, 40.73633083699988], [-73.85628567699989, 40.73642604299987], [-73.85567427699986, 40.73658665799995], [-73.85505083399991, 40.736720220999885], [-73.85415418099988, 40.73687781299987], [-73.85340136499995, 40.73700275799991], [-73.85244443499992, 40.737202311999916], [-73.85187067499986, 40.73734292899995], [-73.85130352699998, 40.73751172799992], [-73.8508760629999, 40.73765385999991], [-73.85079216499992, 40.73768175599991], [-73.8499544029998, 40.737960303999884], [-73.84956453799992, 40.738106091999875], [-73.84859447999989, 40.73850117099989], [-73.84801295299997, 40.73876334399993], [-73.84750820299992, 40.73900780699995], [-73.84746313999992, 40.73895203099994], [-73.84732494199989, 40.738771457999896]]]}}, {\"id\": \"95\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 96, \"Shape_Leng\": 0.185180248095, \"Shape_Area\": 0.000547566253402, \"zone\": \"Forest Park/Highland Park\", \"LocationID\": 96, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83804164799996, 40.70832329499994], [-73.83773266099998, 40.70821535699991], [-73.83743395899987, 40.70809301499986], [-73.83743389999984, 40.70809298699988], [-73.83743384099994, 40.70809296099992], [-73.83714702999997, 40.70795704599989], [-73.83781050699982, 40.70698817599987], [-73.83790705599984, 40.706871460999864], [-73.83790716199995, 40.7068713259999], [-73.83790724599992, 40.70687119099992], [-73.83798758599988, 40.706743142999834], [-73.83798760999981, 40.70674309799987], [-73.83798763399989, 40.70674305399993], [-73.83804872099981, 40.70660560399997], [-73.83804876799985, 40.706605495999945], [-73.83804879199987, 40.70660539699989], [-73.83808807400005, 40.706461289999936], [-73.83808812099979, 40.70646110899985], [-73.83808814499993, 40.70646092899994], [-73.83810401799988, 40.7063130239999], [-73.83809585899986, 40.70616468899991], [-73.83809584699989, 40.70616458099993], [-73.83809582399991, 40.7061644729999], [-73.83806408500001, 40.706019256999866], [-73.83806406099988, 40.70601912199987], [-73.83806401399981, 40.70601898699988], [-73.83801015599994, 40.70587982699987], [-73.83801013199992, 40.7058797729999], [-73.83801010900005, 40.70587971799994], [-73.83793647999991, 40.705749291999865], [-73.83793643299985, 40.705749219999866], [-73.83793637399988, 40.70574914799994], [-73.83784595199988, 40.70562953999993], [-73.83733095699989, 40.704854975999886], [-73.83696237799982, 40.70421245199992], [-73.83856418299983, 40.70329696899991], [-73.83947815200001, 40.70269019399994], [-73.84156378099983, 40.70110059799985], [-73.84215844799995, 40.700902563999875], [-73.842365123, 40.700832543999915], [-73.84236528999996, 40.700832489999904], [-73.84236544299989, 40.7008324279999], [-73.84256506999995, 40.70075073599988], [-73.84256525999987, 40.70075065499986], [-73.8425654379999, 40.700750565999925], [-73.8427564879998, 40.70065776799991], [-73.84275664199988, 40.70065769499992], [-73.84275678399989, 40.70065761499992], [-73.84293789599988, 40.700554428999894], [-73.84359398099988, 40.70017235199988], [-73.84427352200001, 40.69978050599992], [-73.84502472899999, 40.6993593769999], [-73.84700405300002, 40.698825210999914], [-73.84789669799994, 40.69860032899992], [-73.84820912899981, 40.69851840899985], [-73.84913837299978, 40.69828274099995], [-73.85070842300001, 40.70020312499988], [-73.85410116499996, 40.70051374799992], [-73.85443455699989, 40.70060295199994], [-73.85429389599985, 40.700445760999926], [-73.85377512599996, 40.6999095679999], [-73.85328620599985, 40.69928735699989], [-73.8527113089999, 40.69841496199995], [-73.85265915099988, 40.698266471999915], [-73.85262446500002, 40.698114106999896], [-73.85260789899989, 40.697959664999885], [-73.8526096609999, 40.69780503999987], [-73.85262948599983, 40.697652077999926], [-73.85277238699999, 40.69755711199987], [-73.85283544399991, 40.69754896199987], [-73.85478796099983, 40.69729662299988], [-73.85567421900002, 40.69717935799988], [-73.85657185199995, 40.69706469199983], [-73.85744447700002, 40.6969129849999], [-73.8583307899999, 40.69674371099995], [-73.85928498399986, 40.69655371099988], [-73.86016364299988, 40.6963846579999], [-73.86236834199985, 40.69595014299991], [-73.86368656399996, 40.695691417999925], [-73.8645570889999, 40.695520456999915], [-73.86681433899996, 40.69506535099989], [-73.86842489800001, 40.69471811999993], [-73.86891704399994, 40.69515042299987], [-73.870042929, 40.69494148499996], [-73.874020533, 40.69419129499994], [-73.87939315099985, 40.691209631999875], [-73.87950641199996, 40.69114676899987], [-73.87956139399984, 40.691104780999915], [-73.88118296199987, 40.689866269999904], [-73.88226665499997, 40.689038550999896], [-73.88377698600002, 40.68786341099992], [-73.88452250999984, 40.68668474899988], [-73.88728392899995, 40.68562247499987], [-73.88808341799984, 40.68529363199983], [-73.8890889529999, 40.68464055299995], [-73.88962787599998, 40.684236453999894], [-73.89015418699982, 40.68500398999987], [-73.8916601069998, 40.6840441779999], [-73.89252316899987, 40.683424532999865], [-73.89274142399994, 40.683670185999866], [-73.89404051299984, 40.68513229799984], [-73.89414204200001, 40.68524656699989], [-73.89417463299992, 40.6852832479999], [-73.8950492689999, 40.68436259899988], [-73.89530504899982, 40.684080519999966], [-73.89554859499991, 40.68379189199988], [-73.89561476599985, 40.68370742199992], [-73.89577945499994, 40.6834971829999], [-73.8959972739999, 40.683196878999894], [-73.89620169599988, 40.68289148199994], [-73.89628634799988, 40.682758382999914], [-73.896358989, 40.68262086699989], [-73.896367991, 40.68259975699988], [-73.89641909099993, 40.68247987599991], [-73.89643010499985, 40.682446358999925], [-73.89646625099995, 40.68233642199988], [-73.8965264419999, 40.682403298999894], [-73.90116155099986, 40.687877934999946], [-73.90042465099988, 40.688183897999856], [-73.90180467199995, 40.6907662979999], [-73.90123290699994, 40.69144227899997], [-73.90103661799981, 40.691845252999904], [-73.9003866789998, 40.69313995999992], [-73.900030392, 40.693437450999895], [-73.89973223499982, 40.69351918299989], [-73.89935202799984, 40.69364779599994], [-73.89896466800003, 40.69376479299993], [-73.89857107699993, 40.69386983399992], [-73.89817221699985, 40.693962657999926], [-73.89776907799997, 40.69404304999986], [-73.89736265799978, 40.6941108469999], [-73.89691021299986, 40.69384226999992], [-73.89742730199983, 40.69328897999988], [-73.89764233499987, 40.69340865099986], [-73.89768631800001, 40.69336525799989], [-73.89695653499987, 40.6919492069999], [-73.89576145299988, 40.69233462199984], [-73.89474270999995, 40.6926369519999], [-73.89326209899984, 40.69220094699994], [-73.89357935000005, 40.69308532899988], [-73.89398113199998, 40.69307596499988], [-73.89558625499984, 40.693979161999884], [-73.8950769439999, 40.694591389999886], [-73.89245207599993, 40.695225727999855], [-73.89195658699997, 40.694410569999896], [-73.88919365299998, 40.694935769999894], [-73.88828642399997, 40.69523691699991], [-73.88865416899996, 40.69582954899994], [-73.88648530299996, 40.6964399149999], [-73.88642048199995, 40.696458148999945], [-73.88681137199994, 40.69666794799984], [-73.88699989899986, 40.6967711439999], [-73.88717721900005, 40.69689008599989], [-73.88734003299987, 40.6970236659999], [-73.88748536199982, 40.69717024899989], [-73.88761064199986, 40.69732774499996], [-73.88771392599995, 40.697493692999906], [-73.88779396299992, 40.697665426999905], [-73.88785023699991, 40.69784014499988], [-73.88788297599991, 40.69801511999985], [-73.88809909499986, 40.69876329599988], [-73.88818025800003, 40.69885791099992], [-73.88771297999986, 40.699678075999906], [-73.8867672829999, 40.69982608499987], [-73.88584205499986, 40.69996547099991], [-73.88492223099993, 40.70010402699988], [-73.88431619499985, 40.70020119599994], [-73.88384293499996, 40.70034725999994], [-73.88343086899987, 40.70047443799989], [-73.88280948499987, 40.7006834549999], [-73.88273014799978, 40.700549908999925], [-73.88182690599986, 40.69952929899987], [-73.88106992699986, 40.69979988799984], [-73.880246012, 40.70009334899991], [-73.87942953599995, 40.70040484899988], [-73.87860797499998, 40.700704187999875], [-73.87778919199995, 40.701013620999916], [-73.87621714199997, 40.70159500499993], [-73.87625758399986, 40.70174876599991], [-73.87639785199997, 40.70229404199989], [-73.87543139399997, 40.702374352999875], [-73.8744627409999, 40.70245683199996], [-73.87351208999992, 40.70253545899994], [-73.87256153500006, 40.7026120979999], [-73.87210385799979, 40.70264976299988], [-73.87164751899996, 40.702687309999895], [-73.87073324199999, 40.70276203899995], [-73.86981898499995, 40.702837929999845], [-73.86915596200001, 40.702893432999865], [-73.86903271499997, 40.70290390099996], [-73.86873243499991, 40.702929097999856], [-73.86824273999996, 40.70297019599989], [-73.86724652399994, 40.703051654999875], [-73.86653775700002, 40.703109737999895], [-73.86564021600002, 40.70318258999988], [-73.8647283009999, 40.70325882199997], [-73.86379389300001, 40.70333379599989], [-73.86341506899986, 40.7033788049999], [-73.86285374199994, 40.7034144919999], [-73.86241891699996, 40.703377473999865], [-73.86142263099983, 40.70326975799987], [-73.86068159399994, 40.70318418699986], [-73.85997492299997, 40.70310890199989], [-73.85874741999989, 40.702974297999894], [-73.858530295, 40.7029497089999], [-73.85824915999993, 40.702916934999905], [-73.85819890699996, 40.70291107399986], [-73.85823494999983, 40.7028328909999], [-73.85893224399979, 40.70132034499989], [-73.85827405799986, 40.70122365899992], [-73.85739221099976, 40.701213978999895], [-73.85725351500002, 40.70121217699989], [-73.85716688299983, 40.70167410499991], [-73.85630830699988, 40.70183818199996], [-73.85538910499996, 40.701976915999836], [-73.85529571500001, 40.70199100999991], [-73.85529275899978, 40.702595501999944], [-73.85518109699989, 40.702583266999895], [-73.85252019099998, 40.70229172399997], [-73.85305663899999, 40.703137045999945], [-73.85371219899997, 40.70391788699991], [-73.85389480199986, 40.704183418999904], [-73.85392092799985, 40.70422140899995], [-73.85396105499991, 40.7042642169999], [-73.85411264799981, 40.70442593399984], [-73.854739573, 40.70518484999995], [-73.85473254599995, 40.706731913999924], [-73.85473240499985, 40.70676414299989], [-73.85275370600007, 40.70739714799987], [-73.85268225699998, 40.70741814499989], [-73.85110121699995, 40.707900982999895], [-73.85059051099984, 40.70803988299985], [-73.8504290539999, 40.7080870669999], [-73.84951207399992, 40.708345660999925], [-73.84933156199999, 40.70838833399994], [-73.84914591599988, 40.7084193199999], [-73.84895712599986, 40.70843813799993], [-73.84876725200003, 40.70844457799991], [-73.8485783879999, 40.708438720999894], [-73.8476241019999, 40.708332622999855], [-73.84665724099989, 40.70831864199995], [-73.84615958399984, 40.70835082099995], [-73.84567198199987, 40.70841036599989], [-73.84528898599984, 40.70847667899995], [-73.84491011200004, 40.708560330999944], [-73.84453738799996, 40.708661054999894], [-73.84417281299982, 40.70877842099986], [-73.84381825699984, 40.70891182099991], [-73.84347552199995, 40.70906047099992], [-73.84321861899991, 40.70918746199985], [-73.84314625299982, 40.70922345599992], [-73.84296437499994, 40.7091949589999], [-73.84290926199998, 40.70918631999993], [-73.84280177899991, 40.70916947799993], [-73.84220736299994, 40.709076316999884], [-73.84159160299998, 40.70898160199991], [-73.84151213299992, 40.7089693809999], [-73.84144586899984, 40.70895751099992], [-73.8409080659998, 40.70887501099991], [-73.84069937599986, 40.708841168999896], [-73.83975789399979, 40.70868851299985], [-73.83938086799994, 40.70862603399992], [-73.83911666099988, 40.70857492899989], [-73.83901101199996, 40.70855449199996], [-73.83868296400001, 40.70849332099987], [-73.8386827869999, 40.70849328499984], [-73.83868260999992, 40.70849323899989], [-73.83835899099991, 40.70841610699994], [-73.83829235499985, 40.70839661699987], [-73.83804164799996, 40.70832329499994]]]}}, {\"id\": \"96\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 97, \"Shape_Leng\": 0.0624760147423, \"Shape_Area\": 0.000163303970435, \"zone\": \"Fort Greene\", \"LocationID\": 97, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9693409769999, 40.69576842999993], [-73.96880989600005, 40.69312038199996], [-73.96983118499983, 40.69313424399997], [-73.96952115199994, 40.69156355899989], [-73.96910362499993, 40.68946752199986], [-73.97013154199992, 40.68950630799986], [-73.9697831419999, 40.68775161199994], [-73.96948939899981, 40.68628830699986], [-73.96906202000002, 40.68415736099991], [-73.96867398499988, 40.68208599299987], [-73.96868961400001, 40.68200525799988], [-73.96969385699984, 40.68221680499988], [-73.97067424999982, 40.68242096299987], [-73.97186424199982, 40.68267028699993], [-73.97264904900005, 40.68282745499995], [-73.97278445899987, 40.68285242199992], [-73.97375598499991, 40.68303401999988], [-73.97483191299989, 40.68323523699998], [-73.97595299799993, 40.68347469799988], [-73.97624898299998, 40.68355587799986], [-73.97653928699988, 40.683649050999854], [-73.97676303199997, 40.68373178199996], [-73.97682267999997, 40.68375383799986], [-73.97709796699996, 40.6838697799999], [-73.97745722099988, 40.68405228699988], [-73.97804289599988, 40.68485661499987], [-73.97827416099987, 40.685195951999944], [-73.97899402899994, 40.686156131999894], [-73.97909065799995, 40.68629386599988], [-73.97936297699988, 40.686681999999934], [-73.97959037699985, 40.68700390199995], [-73.97984055899983, 40.68734433199992], [-73.98012756199996, 40.68775000899993], [-73.98064972099985, 40.688459476999874], [-73.980691182, 40.68851581099992], [-73.9808430809999, 40.688761361999916], [-73.98092822899991, 40.68890247699988], [-73.98101786899988, 40.68893792699991], [-73.98155298299992, 40.689146163999936], [-73.98209737999989, 40.68935800999991], [-73.98295657299988, 40.689699730999905], [-73.98372011099993, 40.69000671899992], [-73.9835048569999, 40.69000570599997], [-73.98313467299995, 40.690002276999884], [-73.98276720199986, 40.68998648099992], [-73.98157420399987, 40.68993518299992], [-73.98147568199991, 40.689930943999876], [-73.98150118599992, 40.68998815499986], [-73.98213180199994, 40.69122723699989], [-73.98206215699987, 40.69131209099991], [-73.98178907299996, 40.691644867999905], [-73.98179025899985, 40.69202541599988], [-73.98175888999991, 40.692955172999895], [-73.98253983699992, 40.693002615999944], [-73.98251122999989, 40.69361433399992], [-73.98242892099991, 40.693610852999925], [-73.98234987399988, 40.69360788799985], [-73.9822668969999, 40.693604775999866], [-73.9817531689998, 40.69358338499997], [-73.98154756999999, 40.69357778999993], [-73.98147150299985, 40.693574934999965], [-73.98098333199992, 40.69355661699993], [-73.98011299200002, 40.6935238809999], [-73.97916718399999, 40.69348832299992], [-73.97921799199992, 40.69362565199984], [-73.97921806199986, 40.69362583099995], [-73.97921815699976, 40.69362600299992], [-73.97928950399996, 40.69375974199993], [-73.97928957499984, 40.693759866999926], [-73.9792896699999, 40.69375999399987], [-73.97938094200002, 40.69388775699988], [-73.97949049699993, 40.69400690499993], [-73.979490615, 40.694007030999906], [-73.97949074499996, 40.69400713999986], [-73.97961626399982, 40.69411542199983], [-73.97961635799989, 40.6941155019999], [-73.97961647699987, 40.694115582999935], [-73.97975539199989, 40.69421147699993], [-73.97986236999992, 40.694290091999946], [-73.97995651799985, 40.69438029499986], [-73.9800347509999, 40.69448012199993], [-73.98009475099997, 40.694587003999835], [-73.98013518099992, 40.694698016999936], [-73.9801557449999, 40.694810117999964], [-73.98021990699989, 40.6952871029999], [-73.98024270299999, 40.69545652899992], [-73.9802635679999, 40.69563447299992], [-73.98031019799991, 40.695905994999876], [-73.9803568289999, 40.69617751699992], [-73.98037063399991, 40.69626304999989], [-73.98038723500002, 40.696382911999926], [-73.98039561699981, 40.69644348999992], [-73.9804026989999, 40.696502104999915], [-73.9804096149999, 40.696559343999915], [-73.98050926199988, 40.69744627499991], [-73.98054419199988, 40.69824651199988], [-73.97808892799996, 40.69813832499989], [-73.97753101699978, 40.698115812999916], [-73.97678642599985, 40.69808694699989], [-73.97578275800007, 40.698047729999864], [-73.97478615599996, 40.69800455299992], [-73.97377513399991, 40.697959390999884], [-73.97332378599988, 40.697939650999906], [-73.97279477799988, 40.69792083799993], [-73.97178445299986, 40.697882663999955], [-73.97076190899998, 40.69783928399989], [-73.9697278989999, 40.69780401399986], [-73.9693945009999, 40.696080388999846], [-73.96937861700003, 40.695991333999906], [-73.9693719569999, 40.695937759999936], [-73.96935567900005, 40.69585472899994], [-73.9693409769999, 40.69576842999993]]]}}, {\"id\": \"97\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 98, \"Shape_Leng\": 0.121661018278, \"Shape_Area\": 0.000485773020954, \"zone\": \"Fresh Meadows\", \"LocationID\": 98, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76360860199983, 40.73917476499987], [-73.76268297099985, 40.73725583599988], [-73.76259694699998, 40.73708732599994], [-73.76253086199996, 40.73691167999987], [-73.76248609299991, 40.73673098099987], [-73.76246348299985, 40.73654750099995], [-73.76246334599998, 40.73636362499993], [-73.76248538699991, 40.73618169499987], [-73.76252875800003, 40.73600396099994], [-73.76273303499985, 40.73526216299995], [-73.76311240499989, 40.73516391399993], [-73.76318167799995, 40.735144947999906], [-73.76349692299982, 40.73504763299994], [-73.76388970599984, 40.73492638099988], [-73.7642906499999, 40.73480260699987], [-73.76503749400005, 40.73457204399988], [-73.76510920699994, 40.73455658499992], [-73.76470929199999, 40.73408485499987], [-73.76374010499987, 40.73291287299987], [-73.7636997919999, 40.73286411699993], [-73.76434014999984, 40.73275295499994], [-73.76491667699993, 40.73265467299992], [-73.76555086000002, 40.732544282999925], [-73.76564374500005, 40.732533126999954], [-73.76558856199988, 40.732467520999876], [-73.765288304, 40.732114664999905], [-73.7648685299999, 40.73166863999996], [-73.76435071499999, 40.73115119199987], [-73.76381866799998, 40.73068357399991], [-73.76327537799996, 40.730240540999915], [-73.76234209799992, 40.72953829099986], [-73.76160814399992, 40.72904753399989], [-73.76114083699993, 40.72879441399988], [-73.76024211099985, 40.728307611999874], [-73.75993818299987, 40.72815841199989], [-73.75831675599993, 40.727362406999916], [-73.75787141499994, 40.72710685099989], [-73.75684854799977, 40.72635581199993], [-73.75678272799996, 40.72630578399985], [-73.7567473959999, 40.72627289499993], [-73.75669771299987, 40.7262287109999], [-73.75677678599992, 40.72620833599996], [-73.75683174999999, 40.726194017999916], [-73.75698318299987, 40.72616032499988], [-73.75721902199992, 40.72610785499997], [-73.75755793699986, 40.7260324399999], [-73.75796018499997, 40.72595204999991], [-73.75835433199994, 40.72588850899994], [-73.75879801899991, 40.72584059599993], [-73.75922708600002, 40.72580597799991], [-73.75955973699996, 40.7258022329999], [-73.75994854100003, 40.72580527799991], [-73.76017281699988, 40.7258140209999], [-73.76083705999996, 40.725877042999855], [-73.76102756099999, 40.72589417899992], [-73.76181677799993, 40.72600393899992], [-73.7620440879999, 40.72603512499993], [-73.76241922899995, 40.726086592999934], [-73.76368265699995, 40.72625991699991], [-73.76376765199994, 40.72627157299995], [-73.76386689499986, 40.7262841589999], [-73.76475781199981, 40.72639712399995], [-73.76545422799997, 40.72638236199985], [-73.7654693229998, 40.72638204199991], [-73.76568253199987, 40.726377523999886], [-73.76610679099996, 40.72633693499997], [-73.76618933199987, 40.72632947499991], [-73.76629387899985, 40.726318367999944], [-73.76660888099983, 40.726267875999966], [-73.76717559599999, 40.72618267299988], [-73.76745788000002, 40.72613928099987], [-73.76856605399993, 40.72585397099989], [-73.76856674299998, 40.72586018599985], [-73.76856792799998, 40.72586650099986], [-73.7685696449999, 40.72587288899995], [-73.768571885, 40.72587934999989], [-73.76857466699992, 40.72588583999987], [-73.76857799399986, 40.72589235699989], [-73.76858186599988, 40.72589887499988], [-73.76858628199994, 40.72590537699995], [-73.76859122999998, 40.7259118259999], [-73.76859671300002, 40.72591821299991], [-73.76860271599992, 40.725924518999896], [-73.76860920399997, 40.725930700999896], [-73.7686161789998, 40.72593676599994], [-73.76862361599991, 40.725942669999874], [-73.76863146699995, 40.72594840399995], [-73.76863972199986, 40.72595394099988], [-73.76864834499995, 40.72595927999993], [-73.7686572999999, 40.72596439499989], [-73.76866656199985, 40.7259692859999], [-73.76867607499993, 40.725973915999894], [-73.76868582499995, 40.72597829299986], [-73.76869575399984, 40.72598241999985], [-73.7687058369998, 40.725986266999875], [-73.76871602799994, 40.72598984399985], [-73.76872630299988, 40.72599314299992], [-73.76924074899993, 40.72590270099988], [-73.7697527519999, 40.72578523199992], [-73.76991329299986, 40.72574839799991], [-73.77139578199996, 40.725350500999866], [-73.77163268999986, 40.72527440499992], [-73.77165390499994, 40.72526758999993], [-73.77224702999986, 40.7250770729999], [-73.77286784099987, 40.72486798499995], [-73.77348555899981, 40.7246249519999], [-73.77372529499999, 40.724530629999926], [-73.77423835499987, 40.72431652299992], [-73.774371087, 40.72426113199986], [-73.77496975299988, 40.72398806099991], [-73.77524876600002, 40.723843161999895], [-73.77548473600002, 40.723720613999866], [-73.77587312899986, 40.7234728209999], [-73.7760563109999, 40.72335593999985], [-73.776032805, 40.72336247699992], [-73.77583631399985, 40.72343173999993], [-73.77600685799993, 40.723303568999924], [-73.77727808000003, 40.72234633599988], [-73.77732562899998, 40.722392700999926], [-73.77738048899985, 40.72244906299987], [-73.77757868099984, 40.72264221799984], [-73.77774895399996, 40.72284210999989], [-73.7778760189999, 40.72303670299993], [-73.77803765999981, 40.723284268999855], [-73.77781693000003, 40.723339142999876], [-73.77759997899993, 40.72341057599986], [-73.77738999299997, 40.72349836699989], [-73.77719009799996, 40.723601881999905], [-73.7770032789999, 40.723720008999905], [-73.77683220599998, 40.7238512149999], [-73.77667910999996, 40.72399361299994], [-73.77654571399995, 40.72414504599983], [-73.7764331279998, 40.724303181999886], [-73.77634188999991, 40.72446564699988], [-73.7752338719998, 40.725237532999884], [-73.77556110500004, 40.72596986399986], [-73.77596177499996, 40.72681284899995], [-73.77672825799984, 40.72843008099995], [-73.7775479359998, 40.730136776999885], [-73.77757506899988, 40.73019327299987], [-73.77848636199998, 40.72994389499989], [-73.77936529399982, 40.72970186799997], [-73.78024774699996, 40.72945982399991], [-73.78116054999998, 40.72921275899992], [-73.78205851399986, 40.728918895999875], [-73.78292253099987, 40.72863594899996], [-73.78360140199989, 40.72841281999988], [-73.78397695399983, 40.728291396999964], [-73.78482734399982, 40.72801171499994], [-73.78568492899991, 40.727730192999914], [-73.78619017499982, 40.727563574999905], [-73.78655584699987, 40.727441503999906], [-73.78738339699986, 40.72717198199993], [-73.78825003899982, 40.726886972999935], [-73.78911280699997, 40.7266001679999], [-73.78964444799998, 40.72642499899994], [-73.78998042299995, 40.726317244999954], [-73.79085659799983, 40.72602748899991], [-73.7916922309998, 40.7257923849999], [-73.79174472699992, 40.725788849999944], [-73.79176169599994, 40.7258396519999], [-73.79191210299997, 40.72620816199992], [-73.79204513299999, 40.72655107399986], [-73.79229974199997, 40.72714071899992], [-73.79283910399985, 40.72842900799989], [-73.79322750899982, 40.72934789399992], [-73.79352516299998, 40.73005381599991], [-73.79378938699999, 40.730701180999866], [-73.79384983499995, 40.73124678299997], [-73.79381937199997, 40.731657387999874], [-73.793909425, 40.73171777599989], [-73.79392707899994, 40.73199969399992], [-73.79397708699996, 40.73229685299988], [-73.7942776749999, 40.73269614499986], [-73.79438449799987, 40.73286592899996], [-73.79447360099982, 40.73304250799991], [-73.79454395199986, 40.73322422999991], [-73.79459486499994, 40.73340935799989], [-73.794626052, 40.73359608899993], [-73.79459579399989, 40.7345442459999], [-73.79458633199991, 40.73505225199988], [-73.79516450899986, 40.73641140399995], [-73.79594852799994, 40.73781371899989], [-73.79584803299987, 40.7380746799999], [-73.79651029099988, 40.7379086369999], [-73.7965035009998, 40.73800383599993], [-73.79607605199982, 40.738144297999945], [-73.79636108899999, 40.73812415299987], [-73.79639000899994, 40.73823357999985], [-73.79640218700004, 40.73828235799987], [-73.79611584799986, 40.73833613999987], [-73.79588681699978, 40.73837637499989], [-73.79533772999987, 40.738493672999915], [-73.79440801799996, 40.73874822999989], [-73.79369341399999, 40.73895006199987], [-73.79297881699995, 40.73915188099988], [-73.79291603199995, 40.73916776099987], [-73.79285797099998, 40.73917973299994], [-73.79202817599997, 40.73940358099989], [-73.7911152649998, 40.73964983899984], [-73.79020750099997, 40.739894702999855], [-73.78934727299989, 40.74012633799987], [-73.7886511029998, 40.7403257139999], [-73.78848243499986, 40.7403682779999], [-73.78759444399994, 40.74061795799991], [-73.78673516799984, 40.74084177599985], [-73.78669085300001, 40.74085298599988], [-73.78664844699985, 40.74086674899993], [-73.78577051299996, 40.7411106799999], [-73.78484244599998, 40.74136852999993], [-73.78400944099984, 40.74159935699984], [-73.78349697199984, 40.74174135999992], [-73.7831101189999, 40.741848304999905], [-73.7821600969999, 40.742115548999905], [-73.78176526799983, 40.74221640599993], [-73.78137698899997, 40.74231999399989], [-73.78121857699983, 40.742362250999925], [-73.78100566399996, 40.74241954099993], [-73.78068584999994, 40.74250560699994], [-73.78036603499997, 40.74259166299987], [-73.77965641299991, 40.742729309999895], [-73.778599248, 40.742908122999886], [-73.77805835699986, 40.74298648899991], [-73.77579713399984, 40.743317665999925], [-73.77573836999991, 40.74332564699993], [-73.77569182399996, 40.74333358899985], [-73.77551238399991, 40.74335684299992], [-73.77486382100001, 40.74345333599993], [-73.77444330499989, 40.74351589999993], [-73.77403143100003, 40.74356760099993], [-73.77362185199992, 40.74363046299994], [-73.77321534699996, 40.74370439599986], [-73.77281267599984, 40.7437892489999], [-73.7724146099999, 40.743884844999926], [-73.77162097999985, 40.74415973099986], [-73.77156671899986, 40.74417951599989], [-73.77153402299984, 40.74419162499989], [-73.77145218799996, 40.7442215229999], [-73.77137678500003, 40.74425103599991], [-73.770389824, 40.74473791399991], [-73.76937164099995, 40.745220308999876], [-73.76857701200005, 40.74552220499988], [-73.76772991399994, 40.74577622599993], [-73.76770614499998, 40.74572274199992], [-73.76761507999991, 40.745526354999924], [-73.76697863000001, 40.74422296799987], [-73.76687853500006, 40.74409357899987], [-73.7667631489999, 40.74397036299992], [-73.76663352600005, 40.74385485299991], [-73.76649109699986, 40.74374842999987], [-73.76633756199995, 40.74365225899987], [-73.76617484699997, 40.743567271999936], [-73.76556849699985, 40.743258154999886], [-73.76515128799994, 40.74243734599991], [-73.76431495499992, 40.740672626999924], [-73.76360860199983, 40.73917476499987]]]}}, {\"id\": \"98\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 99, \"Shape_Leng\": 0.1833714893, \"Shape_Area\": 0.00121016463877, \"zone\": \"Freshkills Park\", \"LocationID\": 99, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.16842916199994, 40.58649792299988], [-74.16895369199989, 40.5830983639999], [-74.16901204299992, 40.58268826899994], [-74.16903999799996, 40.582491778999945], [-74.1690483779999, 40.582432934999915], [-74.16905498199999, 40.58238696299988], [-74.16907820699998, 40.58222502299985], [-74.16908500799998, 40.58217764599992], [-74.16909417599999, 40.58211339699994], [-74.16913807999991, 40.58180581199991], [-74.16948145799996, 40.57940007499996], [-74.16955946599997, 40.57891524199987], [-74.16962344999997, 40.57842904399988], [-74.16967337599986, 40.577941821999964], [-74.16970918699998, 40.5774539449999], [-74.16973086899992, 40.57696575699988], [-74.16975526299997, 40.57668893399995], [-74.16980912399991, 40.57579489399991], [-74.16991886099999, 40.57397335499995], [-74.16995465599994, 40.57345803799992], [-74.16997053699991, 40.57294169899994], [-74.16996647099997, 40.5724249999999], [-74.16994243599997, 40.5719086199999], [-74.16989846899996, 40.57139322399989], [-74.16983467899996, 40.57087948699992], [-74.16974893399991, 40.570236483999906], [-74.16968558799996, 40.56959152099988], [-74.169644733, 40.568945264999904], [-74.16962643399991, 40.568298409999926], [-74.16963069799989, 40.567651638999905], [-74.16966171099988, 40.566903382999904], [-74.16967801699991, 40.56647024499989], [-74.16969147299992, 40.56607039999989], [-74.16972490199993, 40.56447115899986], [-74.16972693399991, 40.56437399799987], [-74.16973775199993, 40.56383841899987], [-74.16974554099995, 40.56345282199995], [-74.16974968899994, 40.56333125299989], [-74.16974974799987, 40.56332951299986], [-74.16976493899996, 40.56288426399993], [-74.16982582400001, 40.56109042099991], [-74.17003462899996, 40.561109990999896], [-74.17024259599992, 40.561134199999856], [-74.1704495479999, 40.5611630259999], [-74.17065530999997, 40.561196445999926], [-74.17085970499996, 40.56123442999989], [-74.17106256099997, 40.561276946999875], [-74.17126370299995, 40.561323959999875], [-74.17146296099997, 40.56137542899988], [-74.17177723899994, 40.56149241599994], [-74.17185166499996, 40.56152092299991], [-74.17191921399996, 40.56155171899995], [-74.17225507699992, 40.56170483699991], [-74.17348508099991, 40.56226748999991], [-74.17421738099996, 40.56256808499993], [-74.17488750399997, 40.56282641899986], [-74.17539949099991, 40.56296466199991], [-74.17659365999994, 40.56324132999986], [-74.17758036699996, 40.56375543999989], [-74.1779454409999, 40.56398336699991], [-74.18139492399996, 40.56523797899997], [-74.1815955649999, 40.56531796099987], [-74.18180816599994, 40.56538457899992], [-74.18203038699987, 40.56543640499994], [-74.18225953199997, 40.565472349999936], [-74.18241339499998, 40.56548514599992], [-74.1824925989999, 40.56549172899984], [-74.18272650699997, 40.56549428299992], [-74.18295815999996, 40.56548019999992], [-74.18318459699992, 40.56545009399985], [-74.18340311699991, 40.56540496199994], [-74.18369189099995, 40.5653190679999], [-74.18397454599993, 40.56522036799989], [-74.18397488799997, 40.56522024099992], [-74.18397521799992, 40.56522011499991], [-74.18425030499996, 40.565108989999906], [-74.18425062399993, 40.56510885399991], [-74.18425093099988, 40.56510871899992], [-74.18451749399999, 40.56498549599994], [-74.18451768299992, 40.564985404999916], [-74.18451788299991, 40.56498530499989], [-74.18477472999997, 40.56485049699993], [-74.18502026899988, 40.56470502799989], [-74.18502052799994, 40.56470486499994], [-74.18502077599993, 40.564704711999916], [-74.18525395899995, 40.56454935499987], [-74.18576667699995, 40.56444512799989], [-74.18576687799995, 40.56444508299994], [-74.185767078, 40.56444503799991], [-74.18627446199993, 40.564325526999866], [-74.18677554199998, 40.56419095799989], [-74.18726897600001, 40.56404169299989], [-74.18909923999989, 40.56368927199991], [-74.18929075199998, 40.56364274099996], [-74.189477545, 40.56358098999988], [-74.18965654999995, 40.56350449199993], [-74.18982471199989, 40.56341428799993], [-74.18982482999994, 40.563414223999885], [-74.18982494799998, 40.56341415199991], [-74.18997973099991, 40.56331160499996], [-74.19011908499989, 40.56319841199991], [-74.19024124699997, 40.56307668499993], [-74.19032832899994, 40.56296424499989], [-74.19043537599994, 40.56285950699993], [-74.19056041299997, 40.56276545599995], [-74.1907004609999, 40.56268463499989], [-74.19085177899994, 40.562618932999946], [-74.19101021699997, 40.562569401999916], [-74.19123302399998, 40.56253757599988], [-74.19146034299997, 40.5625194289999], [-74.19168998299995, 40.5625154089999], [-74.19191967499992, 40.56252570499986], [-74.19214711699996, 40.562550214999916], [-74.19237007799995, 40.56258854699988], [-74.1925864619999, 40.562640053999885], [-74.192684759, 40.562651716999945], [-74.19278405399993, 40.5626563249999], [-74.19288347999992, 40.56265383899991], [-74.19298216699994, 40.562644280999905], [-74.19307925099996, 40.5626277349999], [-74.19317388399995, 40.56260434399992], [-74.193410946, 40.56248503899994], [-74.19364051199993, 40.56235749799986], [-74.19364075799996, 40.562357353999914], [-74.19364100699988, 40.56235719999993], [-74.19386239999993, 40.56222177399991], [-74.19386255299989, 40.56222168499994], [-74.1938627059999, 40.56222158499984], [-74.19407564499997, 40.56207847399989], [-74.19769481199992, 40.55888705399984], [-74.19946683299995, 40.557181180999905], [-74.20005953999996, 40.556577939999876], [-74.20018273999989, 40.5564544339999], [-74.20031905499991, 40.5563382259999], [-74.20046730799993, 40.556230489999884], [-74.20062607299995, 40.55613226099993], [-74.20079377099992, 40.556044364999885], [-74.20091501299991, 40.55600525999987], [-74.20103876099995, 40.55597102599994], [-74.20116467899999, 40.555941755999896], [-74.20129242599991, 40.55591752799988], [-74.20142165299998, 40.55589840999992], [-74.20155201099993, 40.55588445199986], [-74.20168314599995, 40.555875692999926], [-74.20181469999991, 40.5558721559999], [-74.2019463179999, 40.55587385199995], [-74.20207764, 40.55588077499988], [-74.2022083119999, 40.55589290699985], [-74.20233797799996, 40.55591021399987], [-74.20247541199996, 40.55592731199989], [-74.20261207099998, 40.5559477059999], [-74.20274781999996, 40.55597137599992], [-74.20288252399993, 40.555998296999846], [-74.20301604799995, 40.55602844299987], [-74.20314825999996, 40.55606178499986], [-74.2032263689999, 40.55608719199989], [-74.20330246299997, 40.55611593799991], [-74.20337629799997, 40.55614792799985], [-74.20344936699998, 40.556183969999886], [-74.20351956699993, 40.55622318999985], [-74.20358106099994, 40.55626173299989], [-74.20363978099992, 40.55630271299988], [-74.20369556299995, 40.55634601699991], [-74.2038442639999, 40.55648572799987], [-74.20398450099998, 40.55661748499993], [-74.20492823999997, 40.557205489999916], [-74.20506288299997, 40.5572778759999], [-74.20520041199993, 40.557332220999875], [-74.20530543999998, 40.55736862199989], [-74.20557588399996, 40.5574435929999], [-74.20585448599998, 40.55750245699987], [-74.20613872599988, 40.55754450599986], [-74.20642597199998, 40.55756934899993], [-74.20671355399998, 40.55757693599984], [-74.20699885299997, 40.55756748899989], [-74.2056960519999, 40.55851600599987], [-74.21324031454323, 40.55667262386906], [-74.21324697320209, 40.55670469421521], [-74.21324397315513, 40.55676445719357], [-74.2132387152298, 40.556779081827386], [-74.21322929596316, 40.556803138867814], [-74.21321916900892, 40.556827027794135], [-74.21320833788467, 40.5568507359886], [-74.21319680781782, 40.55687425274997], [-74.21318912697654, 40.55688747517424], [-74.2131822271303, 40.55690094810482], [-74.21317612319916, 40.55691464095038], [-74.2131708250863, 40.5569285289722], [-74.21316634793226, 40.55694258224588], [-74.21316269589218, 40.556956775783924], [-74.21315987847005, 40.556971077999314], [-74.21315790098974, 40.55698546215116], [-74.21315792607356, 40.55699403865189], [-74.2131571405387, 40.557002594094676], [-74.21315554408262, 40.55701108219649], [-74.21315314775063, 40.557019461924], [-74.213149962119, 40.55702768614652], [-74.2131460060169, 40.557035713990224], [-74.21314129760212, 40.55704350232929], [-74.2131358631446, 40.55705100994545], [-74.21312973098384, 40.55705819821812], [-74.21312293480051, 40.55706502984931], [-74.21311550585351, 40.5570714683029], [-74.2131074872238, 40.557077480950554], [-74.21309756484123, 40.55708957912589], [-74.2130869251905, 40.557101322797166], [-74.21307559717893, 40.55711268587346], [-74.21306360191923, 40.55712364318667], [-74.21305096411395, 40.55713417215336], [-74.2130377119574, 40.55714425111], [-74.2130238729699, 40.557153856886956], [-74.21300947810742, 40.55716297024143], [-74.21299661597317, 40.55717262231916], [-74.21298321617033, 40.557181839437604], [-74.21296930618101, 40.55719060536247], [-74.21295491392856, 40.55719890486157], [-74.21294006307731, 40.557206720203176], [-74.21292478382288, 40.5572140392427], [-74.21290910193055, 40.55722084708401], [-74.2128930487575, 40.55722713191879], [-74.21287665675652, 40.557232880099], [-74.21285152687292, 40.55723026778288], [-74.21282631094371, 40.55722820980644], [-74.21280102614703, 40.55722670790734], [-74.21279345113571, 40.557227556910505], [-74.2127860267915, 40.55722898693911], [-74.21277883112765, 40.55723098304876], [-74.21277193658419, 40.55723352470325], [-74.21276541375505, 40.55723658677328], [-74.21275932997062, 40.557240134106685], [-74.21275374880909, 40.55724413497832], [-74.21274872592612, 40.55724854514076], [-74.2127443139994, 40.557253320848], [-74.21274055956393, 40.55725841284807], [-74.21272721547317, 40.55726180513732], [-74.21271358840103, 40.55726449195031], [-74.21269974875094, 40.55726645910471], [-74.21268576576317, 40.55726769817959], [-74.21267170985122, 40.5572682022597], [-74.212665089788, 40.55727169903635], [-74.21265890471648, 40.55727563320729], [-74.21265320401349, 40.557279971973486], [-74.21264803491319, 40.557284683962806], [-74.21264343799604, 40.55728972727975], [-74.21263944838286, 40.55729506412569], [-74.21263610135786, 40.5573006511085], [-74.21263342054272, 40.55730644501077], [-74.21263142895256, 40.55731239710378], [-74.21263014114162, 40.557318461921135], [-74.21262956902495, 40.55732459099601], [-74.21262971693675, 40.55733073302433], [-74.212630581985, 40.557336841221975], [-74.2126321588445, 40.55734286604855], [-74.21263443479033, 40.55734875997723], [-74.21263739090948, 40.55735447406641], [-74.2126410044309, 40.55735996514136], [-74.21264524623795, 40.55736518903256], [-74.21265008206916, 40.55737010282704], [-74.2126723150813, 40.557412797144444], [-74.2126761341738, 40.55743291113687], [-74.21267905301579, 40.55745311519104], [-74.21268106803171, 40.55747338599887], [-74.21268217382365, 40.55749369683106], [-74.21268237383482, 40.55751402679703], [-74.21268166310496, 40.55753434816385], [-74.21268368228385, 40.55754852492697], [-74.2126848169559, 40.55756275823561], [-74.21268506353324, 40.557577017928914], [-74.2126844214028, 40.55759127008814], [-74.21268289174363, 40.55760548296424], [-74.21268047879693, 40.55761962279826], [-74.21267718710236, 40.55763365984381], [-74.21267302682733, 40.55764756000401], [-74.21266221111956, 40.55766350990953], [-74.21265063396818, 40.55767914311732], [-74.21263830793275, 40.5576944412336], [-74.21262525131444, 40.55770938310279], [-74.21261148120676, 40.557723947238955], [-74.21259701696337, 40.557738114990975], [-74.21258187815579, 40.557751865204494], [-74.21256608514139, 40.5577651814006], [-74.21255225997128, 40.557778444229236], [-74.21253785107135, 40.55779134390751], [-74.21252287597689, 40.55780386020303], [-74.2125073528959, 40.55781598315396], [-74.21249129782066, 40.557827696871264], [-74.21247473304155, 40.55783898704916], [-74.2124576733534, 40.55784984144864], [-74.21245156909004, 40.557853466033635], [-74.21244014147611, 40.557860247186035], [-74.21240973253926, 40.55787731294612], [-74.21237984165433, 40.557894902109915], [-74.21235120957739, 40.55791255812843], [-74.21235048482772, 40.55791300576909], [-74.21232168091373, 40.5579316128322], [-74.21231012772952, 40.557938977711714], [-74.21229910426533, 40.557946801146514], [-74.21228863856409, 40.55795506030065], [-74.2122787628412, 40.55796373199563], [-74.2122695040372, 40.557972792057406], [-74.21226088820295, 40.55798221279557], [-74.2122529392056, 40.55799197054422], [-74.21224568001387, 40.55800203544087], [-74.21223913140844, 40.558012380139694], [-74.21223369481807, 40.55802228251367], [-74.21223331416364, 40.55802297545222], [-74.21222824092033, 40.55803379002687], [-74.21222392651748, 40.5580447931786], [-74.21222038623903, 40.55805595589611], [-74.21221762722817, 40.55806724482816], [-74.21221565883023, 40.558078628461566], [-74.21219844901638, 40.558094821234896], [-74.21218188598277, 40.55811140030813], [-74.2121659841867, 40.558128353090176], [-74.21215075917685, 40.55814566481006], [-74.21213622297513, 40.55816331668222], [-74.21212239135275, 40.55818129494037], [-74.21210927501949, 40.55819958264444], [-74.21209688644561, 40.558218163856374], [-74.21208523721299, 40.55823701962404], [-74.21207433957846, 40.558256136019835], [-74.21206419984549, 40.558275491923744], [-74.21204963603087, 40.55829546379474], [-74.21203576875757, 40.558315720854715], [-74.21202260611177, 40.558336250021625], [-74.21201015815413, 40.558357037539544], [-74.21199843164408, 40.5583780674809], [-74.21198743707826, 40.55839932491638], [-74.21197718232703, 40.55842079793707], [-74.21196767172765, 40.558442468776825], [-74.21195891467916, 40.55846432267588], [-74.21195091597077, 40.558486345888156], [-74.21194367928562, 40.558508521988905], [-74.21193721380439, 40.55853083672145], [-74.21193152100928, 40.55855327232471], [-74.21192660612942, 40.55857581521946], [-74.21192247087268, 40.558598449487114], [-74.21191912111586, 40.558621158028856], [-74.21191655681484, 40.55864392660494], [-74.21189867612937, 40.55870442847091], [-74.2118801739002, 40.558764827779044], [-74.21186105230399, 40.558825118829205], [-74.21184131197964, 40.55888529575616], [-74.21182095510497, 40.558945353194765], [-74.21179998319977, 40.5590052859482], [-74.21177840042363, 40.55906509015543], [-74.21175620455124, 40.55912475727625], [-74.21175573559012, 40.559131620180054], [-74.21175602577777, 40.55913848771529], [-74.21175707738473, 40.55914531380703], [-74.2117588808004, 40.5591520462034], [-74.2117614202853, 40.5591586395327], [-74.21176468316598, 40.55916504540211], [-74.2117686427182, 40.559171217789896], [-74.21177327156883, 40.55917711385877], [-74.21177853641149, 40.55918268994474], [-74.21178440022126, 40.559187907416344], [-74.21179081872265, 40.559192726985756], [-74.21179774898178, 40.55919711690098], [-74.2118131007146, 40.55919930199983], [-74.21182827737275, 40.55920210778403], [-74.21184324003018, 40.55920551941539], [-74.21185794473554, 40.55920953044113], [-74.21187234972683, 40.559214132227126], [-74.21188641321727, 40.55921930826475], [-74.21190010067754, 40.5592250448807], [-74.21191336724817, 40.559231326409744], [-74.21192617926559, 40.559238135156086], [-74.211933493293, 40.559243498711034], [-74.21194027809761, 40.55924925576557], [-74.21194649535678, 40.55925537405696], [-74.21195211289572, 40.55926182064137], [-74.21195710358683, 40.55926856089026], [-74.21196143657221, 40.55927556152188], [-74.2119650913001, 40.559282783539885], [-74.21196804875582, 40.559290187944946], [-74.21197029168968, 40.55929773807948], [-74.21197181096326, 40.55930539224575], [-74.21197259700165, 40.55931310975155], [-74.21197264462789, 40.559320851236585], [-74.21197195436712, 40.559328574650024], [-74.21197053048114, 40.559336238603834], [-74.21196838009425, 40.559343804050414], [-74.21196551296323, 40.55935123076457], [-74.21195716802121, 40.55937783958228], [-74.21194799863947, 40.559404291772175], [-74.21193801509246, 40.55943057206978], [-74.21192721864446, 40.55945666388711], [-74.2119156180345, 40.55948255246527], [-74.21190768055533, 40.559501207698666], [-74.21190056125081, 40.55952005575716], [-74.2118942710297, 40.55953907299892], [-74.21188881181226, 40.5595582413272], [-74.21188419516436, 40.55957753659651], [-74.21188042321681, 40.55959693786174], [-74.21187398427489, 40.559611522526275], [-74.21186683213901, 40.559625911992946], [-74.21185898256296, 40.559640086966695], [-74.21185044273767, 40.55965402934088], [-74.21184122621729, 40.559667718819156], [-74.21183134501631, 40.55968113443794], [-74.21182081248342, 40.55969426042454], [-74.21180964546389, 40.55970707513636], [-74.211797858413, 40.559719565479206], [-74.21178650572082, 40.559735343303515], [-74.21177445749836, 40.559750817169075], [-74.21176172643324, 40.55976597063408], [-74.21174832586338, 40.55978078440814], [-74.21173427396971, 40.55979524254179], [-74.21171958563137, 40.55980932691425], [-74.21170427684127, 40.55982302426096], [-74.21168836796686, 40.55983631544542], [-74.21167187610496, 40.55984918887652], [-74.21165706886426, 40.559860348046946], [-74.21164290739024, 40.559871984156736], [-74.21162941665453, 40.55988407521373], [-74.2116166233805, 40.559896597547144], [-74.21160454903074, 40.55990953084658], [-74.21159321724679, 40.559922849268865], [-74.21158264815574, 40.55993652697761], [-74.21156059716209, 40.55996613584748], [-74.21153925748023, 40.559996045292465], [-74.21151863853387, 40.56002624825926], [-74.21149874511931, 40.56005673317995], [-74.2114795848924, 40.560087489486214], [-74.2114706758224, 40.560103766207384], [-74.21146107650561, 40.5601198125045], [-74.21145079479376, 40.56013561077206], [-74.21143984403734, 40.5601511457401], [-74.21142823472547, 40.560166400301036], [-74.21141597822462, 40.560181356842996], [-74.21140309140515, 40.560196001764886], [-74.2113895825626, 40.5602103188008], [-74.21137547009187, 40.56022429015838], [-74.21133713481073, 40.56025785569348], [-74.21129944575641, 40.5602918453861], [-74.21126241146871, 40.560326250677036], [-74.21122604291112, 40.56036106518069], [-74.2112170869799, 40.560372555629094], [-74.21120881578622, 40.56038434218381], [-74.21120124704034, 40.56039639867766], [-74.21119439340815, 40.56040870146574], [-74.21118827150941, 40.5604212268958], [-74.21118289114916, 40.56043394965295], [-74.21117826608337, 40.560446843409856], [-74.21117440194196, 40.5604598823563], [-74.21117131028556, 40.56047304067166], [-74.21116899542127, 40.560486290872966], [-74.2111674627594, 40.560499607150945], [-74.2111667139708, 40.560512962027325], [-74.21116675116954, 40.56052632919614], [-74.21116521298025, 40.56055798150036], [-74.21116463850305, 40.56058965213629], [-74.21116503010131, 40.56062132384411], [-74.21116638706465, 40.56065298003938], [-74.21116870824625, 40.560684604809055], [-74.21117199314888, 40.56071617955784], [-74.21117209081683, 40.56072317259265], [-74.21117303560405, 40.560730128385956], [-74.21117482073525, 40.56073698831464], [-74.2111774339364, 40.56074369125278], [-74.21118084823422, 40.56075018196485], [-74.21118503686579, 40.560756402199516], [-74.21118996452417, 40.5607623010924], [-74.21119558623057, 40.56076782545153], [-74.21120185505099, 40.560772929124745], [-74.2112087172516, 40.560777568987966], [-74.21121857348905, 40.5607810400149], [-74.21122807010568, 40.56078505584053], [-74.21123715233051, 40.5607895927764], [-74.21124577263936, 40.56079462628309], [-74.21125388482164, 40.56080013064596], [-74.2112614448448, 40.56080607411551], [-74.21126841087217, 40.5608124249379], [-74.21127474743373, 40.560819150007596], [-74.21128041882379, 40.560826211360954], [-74.21128669305928, 40.560837538097424], [-74.21129215619294, 40.56084910790434], [-74.21129678768462, 40.56086088563805], [-74.21130057358947, 40.560872837818394], [-74.21130350610588, 40.56088492877594], [-74.21130557523796, 40.56089712351527], [-74.2113067742804, 40.56090938569459], [-74.21131155011123, 40.56092234843845], [-74.2113154561791, 40.560935481652926], [-74.21131848314432, 40.56094874950353], [-74.21132062144908, 40.56096211665934], [-74.21132186593334, 40.56097554912106], [-74.2113222123134, 40.56098901238534], [-74.21132425778843, 40.561017406502934], [-74.21132550948525, 40.56104582670767], [-74.21132596868435, 40.5610742616054], [-74.21132563160215, 40.56110269579036], [-74.21132637809602, 40.56112202556187], [-74.21132815797952, 40.56114131506759], [-74.21133096698274, 40.561160533489726], [-74.21133479841416, 40.56117964850716], [-74.21133964799989, 40.56119862812938], [-74.21134026248816, 40.561214488257534], [-74.21134005281321, 40.561230353085314], [-74.21133901823953, 40.56124619865709], [-74.21133716220588, 40.561262001177326], [-74.21133448308989, 40.561277734011576], [-74.21133098696738, 40.561293373694554], [-74.21132668188982, 40.561308896421856], [-74.21132157194675, 40.561324275716174], [-74.21131566584408, 40.561339486264416], [-74.21130897252331, 40.56135450777871], [-74.21127926148336, 40.56140113918789], [-74.21125007208866, 40.56144797384898], [-74.21122141090954, 40.56149500521674], [-74.21119328295505, 40.561542219377195], [-74.21119323731205, 40.56155738245365], [-74.21119242214162, 40.56157253253728], [-74.21119084176642, 40.5615876473388], [-74.2111884967697, 40.56160270306774], [-74.21118539060336, 40.56161768011681], [-74.21118153043909, 40.56163255418076], [-74.21117691796499, 40.56164730347756], [-74.21117156563142, 40.5616619058701], [-74.21116105263172, 40.56169434748836], [-74.2111513036203, 40.561726928761544], [-74.21114232317109, 40.561759637116666], [-74.21113411608856, 40.561792463163314], [-74.21112626601061, 40.56181481104591], [-74.21111924501375, 40.561837319405896], [-74.21111305501641, 40.561859969476096], [-74.21110770474597, 40.56188274214254], [-74.21110319547003, 40.561905621319816], [-74.21109953372239, 40.56192858890303], [-74.2110967214154, 40.56195162411486], [-74.21109475872257, 40.56197471221232], [-74.21109364843628, 40.56199783275171], [-74.21109339203943, 40.56202096813977], [-74.21109399057063, 40.56204409927601], [-74.2110939191909, 40.56207323173792], [-74.21109459070571, 40.56210235847845], [-74.21109600355396, 40.56213147179443], [-74.21109815988439, 40.56216055626874], [-74.21110105594312, 40.56218960537483], [-74.21110469124471, 40.56221860420325], [-74.21110906466804, 40.56224754521719], [-74.21115994766821, 40.562393063576614], [-74.21116656016068, 40.562415668164995], [-74.21117394588556, 40.56243813295121], [-74.2111820940348, 40.5624604437151], [-74.21119100455726, 40.56248258403874], [-74.21120066840547, 40.56250454070351], [-74.21121107937378, 40.562526296129974], [-74.21122223259422, 40.56254783893502], [-74.211273723836, 40.562715288841474], [-74.21128348460323, 40.562735365129285], [-74.21129255643, 40.56275562931218], [-74.21130093487787, 40.56277606715855], [-74.21130861550208, 40.56279666276122], [-74.21131559145635, 40.5628174045735], [-74.21132185719213, 40.562838275014876], [-74.21132740982452, 40.562859264876735], [-74.21133370130121, 40.56288049261196], [-74.21133920984406, 40.562901847602625], [-74.21134392485533, 40.56292331328281], [-74.2113478478115, 40.562944870383944], [-74.21135097251253, 40.56296650367217], [-74.21135329846459, 40.56298819606043], [-74.21135529340576, 40.56311821424979], [-74.21135292461402, 40.563134433353675], [-74.21135129758787, 40.563150703194374], [-74.21135041689111, 40.56316700801578], [-74.21135028004271, 40.56318332721618], [-74.21135088807269, 40.56319963901492], [-74.21135224180138, 40.56321592481458], [-74.21135433809455, 40.56323216585749], [-74.21135717556896, 40.563248341372415], [-74.21136074998377, 40.56326442992312], [-74.2113650562325, 40.56328041426349], [-74.21136685536173, 40.56329507269257], [-74.2113693778171, 40.56331561680146], [-74.21137296608772, 40.563350870612695], [-74.21137582167343, 40.56338616648183], [-74.2113779423426, 40.56342149385878], [-74.21137932828424, 40.56345684336147], [-74.21137997858774, 40.56349220527486], [-74.21137008619401, 40.56351582999108], [-74.21135936781704, 40.56353924262531], [-74.2113478306638, 40.563562430096766], [-74.2113354847796, 40.563585373790794], [-74.21132233714144, 40.563608057443965], [-74.21130839824029, 40.56363046428353], [-74.21129367571348, 40.56365257837988], [-74.21127639770533, 40.56367843393571], [-74.2112598193029, 40.563704556082854], [-74.2112439507954, 40.563730933243185], [-74.211228798512, 40.56375755233794], [-74.21121436747669, 40.56378440397643], [-74.21120066709557, 40.56381147540934], [-74.21118769931408, 40.56383875641406], [-74.21117547287594, 40.5638662330698], [-74.21114908650823, 40.563912529458506], [-74.21112338114351, 40.56395905142522], [-74.21109836004746, 40.56400578941492], [-74.21107402518696, 40.56405273973875], [-74.21105038247171, 40.56409989518279], [-74.2110274327638, 40.564147250217], [-74.21100517978078, 40.56419479897163], [-74.21098362503817, 40.564242534072925], [-74.21096277401229, 40.56429045015019], [-74.21093475874353, 40.56441163027387], [-74.21092165530236, 40.56446260026247], [-74.21090755600585, 40.56451341510195], [-74.21089245905263, 40.56456406156065], [-74.21087637317956, 40.56461452437678], [-74.21086774089825, 40.56463778888891], [-74.21085993776386, 40.56466122409922], [-74.21085297008612, 40.56468481039525], [-74.21084684089459, 40.56470853302902], [-74.21084155693562, 40.56473237154946], [-74.21083712189804, 40.564756311207475], [-74.21081321472644, 40.564856374954054], [-74.21078819559088, 40.564968656364776], [-74.2107797307137, 40.56506916067951], [-74.21075976100575, 40.565139409047134], [-74.21075316824248, 40.56518284412434], [-74.21074330542251, 40.56522817912912], [-74.21072667691, 40.565273911362134], [-74.2107179266907, 40.56529701960465], [-74.2107085089515, 40.5653218960115], [-74.21069743518801, 40.565348624485516], [-74.21067733198504, 40.56538802089522], [-74.21063993391671, 40.5654783944366], [-74.2106234374394, 40.56552020017456], [-74.21058462277247, 40.56557239533419], [-74.21056232688464, 40.56561278551075], [-74.21047559606595, 40.56576467669574], [-74.21045176120732, 40.5658420973515], [-74.21039639550105, 40.56595306168716], [-74.21037558522801, 40.56599685214434], [-74.2103607965635, 40.56603248005558], [-74.21033396826797, 40.566096621696246], [-74.21030851498715, 40.566139788216304], [-74.21028473121008, 40.5661768033178], [-74.21025750938672, 40.56620745889705], [-74.21023567751136, 40.566235220405765], [-74.21020935035405, 40.566275425732], [-74.21011458775692, 40.566414706259245], [-74.2100369420801, 40.566560405318604], [-74.2099764118816, 40.566700364112386], [-74.20992198936453, 40.56679170233913], [-74.20988348982404, 40.56685300947939], [-74.20985586162145, 40.566915010423706], [-74.20979737148639, 40.56702877942583], [-74.20977691342875, 40.56707037330834], [-74.2097613438044, 40.567098207219914], [-74.20973856379582, 40.56713340987282], [-74.20971519072341, 40.56717473398203], [-74.20969971286483, 40.56720907240954], [-74.20967795471621, 40.56724902232857], [-74.20964655233293, 40.56728992198323], [-74.20962824988973, 40.56731836464221], [-74.2096095952876, 40.567361162927064], [-74.20959638520986, 40.5673953867992], [-74.20957929223879, 40.56744243665677], [-74.20955643852933, 40.567488948024966], [-74.20954125857394, 40.56751458487445], [-74.20952892037968, 40.56755182704577], [-74.20952461322831, 40.567578157067466], [-74.20952390733657, 40.567606018781696], [-74.20952991213728, 40.56763828686198], [-74.20954069373899, 40.56766725241841], [-74.2095508790996, 40.56770146225176], [-74.20957207113801, 40.56779071178188], [-74.20956174056079, 40.5678252054795], [-74.20955122470804, 40.56784581076466], [-74.20953849782619, 40.567885221484595], [-74.20953467232897, 40.56792804865804], [-74.20952755116922, 40.56793531838229], [-74.20951393515091, 40.56794922158046], [-74.20948488887286, 40.56794954846761], [-74.2094476662524, 40.56794774989156], [-74.2094237327763, 40.56794823379797], [-74.20938515363255, 40.56796172513281], [-74.2093478026972, 40.56798817200064], [-74.20932175268325, 40.5680253007569], [-74.20931088354243, 40.56804810264814], [-74.20930302509844, 40.56809071635698], [-74.20930222761174, 40.56812338165856], [-74.20930338344986, 40.56815895122125], [-74.20926886123786, 40.56824962093698], [-74.20926525883027, 40.56828286664633], [-74.20927042628897, 40.568324305018955], [-74.20925794317955, 40.568371977176355], [-74.20924168547243, 40.56840985887977], [-74.20921948028858, 40.56844547099358], [-74.20918588410103, 40.56847520360477], [-74.20916235741977, 40.568491303034364], [-74.20912500684162, 40.568517776129305], [-74.20909031328969, 40.56854191135124], [-74.20905816585409, 40.56857501852686], [-74.20904489974667, 40.56860273723909], [-74.2090342903802, 40.56864030506821], [-74.20901810631887, 40.568679036484866], [-74.2089929848352, 40.568725689570655], [-74.20896988848015, 40.56876393934233], [-74.20894236847846, 40.56880332262291], [-74.20891142961004, 40.56884290477425], [-74.20888426318028, 40.56888009091043], [-74.20886661112198, 40.56890979392917], [-74.20885566763602, 40.568943905029386], [-74.20884572715461, 40.56898838843852], [-74.20884299766729, 40.56902462547019], [-74.20884318638387, 40.56906200946808], [-74.20884309550478, 40.56909028067201], [-74.20884538245129, 40.56910798109466], [-74.20885305891247, 40.569134070337], [-74.20887190552189, 40.56917562055676], [-74.20888640180824, 40.569195658773594], [-74.20889248059125, 40.569228805610706], [-74.20889153469763, 40.56927189975651], [-74.20889159336394, 40.569301900436244], [-74.2088908511528, 40.569341070321585], [-74.20889579663681, 40.56936861919062], [-74.20890031354503, 40.56939751444051], [-74.20887997055323, 40.569475613922556], [-74.20889013891401, 40.56953897206492], [-74.20891141734027, 40.56956426911142], [-74.20892489464279, 40.569591774484344], [-74.20892636328658, 40.569624298450016], [-74.20888302882722, 40.5696711208332], [-74.20885307543493, 40.56970323469199], [-74.20884705448222, 40.569723585299165], [-74.20882741271222, 40.56975035495678], [-74.20880496367919, 40.569765520213934], [-74.2087781874855, 40.56978922915667], [-74.20875830301402, 40.569807737329675], [-74.20872712118614, 40.56981558983796], [-74.20870130896726, 40.56981398924639], [-74.20868227933643, 40.56981767446649], [-74.20865187713905, 40.56983332021022], [-74.20861881832602, 40.56986258560527], [-74.20858691045257, 40.56990395244668], [-74.20855662348757, 40.56995610343071], [-74.20850409350268, 40.57007864602345], [-74.20850161235694, 40.57010540150904], [-74.2084931272451, 40.57013143166719], [-74.20847889159194, 40.57015595903444], [-74.20845933057952, 40.570178251031294], [-74.20843502844855, 40.57019764184334], [-74.20840671104904, 40.57021355230792], [-74.20837522415977, 40.570225507212506], [-74.20833339151348, 40.570246581978424], [-74.20829081802128, 40.570272020722946], [-74.20825567636716, 40.57029184662071], [-74.20821981084153, 40.57032169329083], [-74.20817926338377, 40.57036225276879], [-74.20811163891236, 40.570411726479215], [-74.20807647994936, 40.57043717968815], [-74.20805897478249, 40.570456452558986], [-74.20803424151269, 40.57048957231308], [-74.20801705381407, 40.570517957400746], [-74.20800811544828, 40.57054931930024], [-74.20800502986653, 40.570564284652754], [-74.2080054602375, 40.5705864345317], [-74.20800746847948, 40.57060718116962], [-74.20801360133788, 40.57062336385578], [-74.20802776229282, 40.57063997294266], [-74.20804694132815, 40.570649324918755], [-74.20807383412324, 40.57066214938773], [-74.20809988826458, 40.57067198345026], [-74.20812547988973, 40.570683135968906], [-74.20814699968129, 40.570693225342815], [-74.20816851102184, 40.57071942341425], [-74.20783559015278, 40.572610001451686], [-74.20727340079816, 40.57407098541118], [-74.20726298207298, 40.574113027772725], [-74.2072315961311, 40.57414395967348], [-74.20719165903249, 40.574174557353345], [-74.20712711526238, 40.574214592628], [-74.20707800410406, 40.574266340216575], [-74.20708127254849, 40.57432738494099], [-74.20708141086799, 40.57437199994334], [-74.20711849533382, 40.57441419889951], [-74.20715858686431, 40.57443291258793], [-74.2071814237486, 40.574442147205986], [-74.20720483655512, 40.57445161465258], [-74.20722650206048, 40.57448679641168], [-74.20719267147533, 40.574500946538635], [-74.20715669106241, 40.574541953079205], [-74.20711896133153, 40.57456447917679], [-74.2070913697977, 40.574604447801164], [-74.20707313358393, 40.574681968326765], [-74.20707024366172, 40.57474302532235], [-74.20709287504637, 40.57479947162327], [-74.20712612520137, 40.57488850882439], [-74.20715404454762, 40.574954207139406], [-74.20720049854935, 40.575038655315375], [-74.20722841802986, 40.57510435310896], [-74.20727167599361, 40.5751512377485], [-74.20728723368893, 40.57520286855747], [-74.20728431413171, 40.575254532872584], [-74.207232050259, 40.57528280487044], [-74.2071736424719, 40.57531578479528], [-74.20710601770283, 40.57535582393274], [-74.20697412386814, 40.57552512720597], [-74.20692512074686, 40.57561209703668], [-74.20687395306429, 40.575682874614486], [-74.20685311460248, 40.575766961597125], [-74.20683148750715, 40.575854230049295], [-74.20681205626771, 40.575922020188194], [-74.20681884607389, 40.5759610605443], [-74.20682054032525, 40.575993858524676], [-74.20683824659714, 40.57605213964548], [-74.20684637155793, 40.576103148282904], [-74.20684971914962, 40.576155380449066], [-74.2068737187741, 40.576188138264094], [-74.20688655587759, 40.57621727105957], [-74.20692807000994, 40.57624635220359], [-74.206971117961, 40.576255994075986], [-74.20702213223713, 40.57626562040431], [-74.20705559440268, 40.57626677552938], [-74.20709545812728, 40.576277636848985], [-74.20708917179493, 40.576305590338734], [-74.20707333773304, 40.576337203977516], [-74.20705113024823, 40.57636883127879], [-74.20704010235717, 40.57640894023964], [-74.20701468863136, 40.57643449733458], [-74.20698130610009, 40.57645885402143], [-74.20693992288872, 40.5764722919367], [-74.20693678598774, 40.57648809130825], [-74.20693688761224, 40.576520892154086], [-74.20692739963445, 40.57654399035161], [-74.20690513533927, 40.57655739381555], [-74.20688441149332, 40.576553786690425], [-74.20685407570674, 40.57653318872325], [-74.20682535188581, 40.576518662147976], [-74.20679189402031, 40.576518722377855], [-74.20673936541259, 40.5765346106395], [-74.20670274735936, 40.5765431800168], [-74.2066470961085, 40.57657972560937], [-74.20661471962363, 40.57661046209855], [-74.206580405322, 40.57665273733926], [-74.2065550934208, 40.57671109417131], [-74.20653617393428, 40.576775514812084], [-74.2065268053306, 40.5768374887604], [-74.20652532907332, 40.576875151816175], [-74.20653125429679, 40.57690163736891], [-74.20654779575953, 40.576927350117614], [-74.20659246376154, 40.5769454920353], [-74.20663554593676, 40.57696606647526], [-74.20666586305758, 40.57698059023145], [-74.20669297509777, 40.57698904594725], [-74.20669306156194, 40.577016986272014], [-74.20670587775237, 40.57704004549179], [-74.20672986390356, 40.5770679444875], [-74.20674745004618, 40.57708735050466], [-74.20677775928132, 40.57709944338576], [-74.20679849745787, 40.577107910046166], [-74.20681606482219, 40.57712124178519], [-74.20681616638838, 40.57715404262922], [-74.20680827961804, 40.577179568792545], [-74.20677966547655, 40.57720027222004], [-74.20675099027359, 40.57720153925896], [-74.20671602826916, 40.577230758082706], [-74.2066890479301, 40.5772648230829], [-74.20667480992525, 40.5772976480489], [-74.20667013969313, 40.577332888336535], [-74.20666067340031, 40.57736327502945], [-74.2066352823778, 40.57739612172753], [-74.20659709323164, 40.577411984631745], [-74.20654610443377, 40.577410860381455], [-74.20650468364701, 40.57741215077526], [-74.20647287503958, 40.57743042917538], [-74.20644429360412, 40.57746206663754], [-74.20641889868126, 40.57749369786452], [-74.20639507149829, 40.57751682295124], [-74.20635176963043, 40.57752778113748], [-74.20632344195356, 40.57753881896482], [-74.20629493544428, 40.577594752789274], [-74.20626642888944, 40.5776506872764], [-74.20623005707404, 40.57773943572741], [-74.20620636100908, 40.57780507980824], [-74.20619853327649, 40.5778500429818], [-74.20618117210455, 40.5779035275872], [-74.20616863938997, 40.577972796740724], [-74.20615140873657, 40.57806879936998], [-74.20615951026187, 40.57811251953037], [-74.20617482449852, 40.57814505551433], [-74.20617390206725, 40.57816347474193], [-74.20616518055131, 40.578173804877004], [-74.20615112441064, 40.57818846049042], [-74.20613343399746, 40.57819994960594], [-74.20610886698275, 40.57823214804006], [-74.2060954249185, 40.57825947316978], [-74.20607561715705, 40.57828681021901], [-74.2060629581204, 40.57831049368194], [-74.20604632007586, 40.57833357832398], [-74.2060392303421, 40.57835664583984], [-74.20603132914712, 40.578374860295206], [-74.2060194179058, 40.57838276913292], [-74.20600673956517, 40.57839977890129], [-74.20598931391189, 40.57842589790918], [-74.20597744326368, 40.578446546484145], [-74.20596399604793, 40.57847266003047], [-74.20595372759396, 40.57849694531836], [-74.20594901152563, 40.57851576118643], [-74.20593715361618, 40.578541263076445], [-74.20593482006235, 40.578558255465424], [-74.20591732509392, 40.57861664755361], [-74.20595771087191, 40.579310888030335], [-74.20596154297344, 40.580046993882775], [-74.205950418, 40.57995433999984], [-74.2059328709999, 40.5798924379999], [-74.20583829199992, 40.57984126099989], [-74.20575100199994, 40.57983323599993], [-74.20568486299997, 40.57983453099987], [-74.20561948299995, 40.57984225399984], [-74.20555593099994, 40.57985627799989], [-74.20549524799993, 40.5798763739999], [-74.20543842599989, 40.57990221299989], [-74.2053863959999, 40.57993337199984], [-74.2053762709999, 40.5799506069999], [-74.20536261699996, 40.579966361999915], [-74.20534579299989, 40.57998022199993], [-74.20532624599998, 40.579991818999915], [-74.20530449199997, 40.58000084699988], [-74.20528110599997, 40.580007067999915], [-74.20525670699998, 40.58001031599987], [-74.20523194099997, 40.580010506999926], [-74.20520746099992, 40.5800076339999], [-74.20512751299994, 40.5800157669999], [-74.20504901799995, 40.580029906999854], [-74.20497275099994, 40.5800499149999], [-74.20489946199996, 40.580075591999915], [-74.20482987599989, 40.580106686999926], [-74.20476467899992, 40.5801428909999], [-74.20470451399991, 40.580183847999876], [-74.20469757299996, 40.580202453999895], [-74.20468682999994, 40.58021997899994], [-74.20467256199997, 40.58023597499993], [-74.20465513199991, 40.58025003199989], [-74.20463498699989, 40.580261791999874], [-74.20461264099993, 40.58027095299988], [-74.2045886649999, 40.5802772819999], [-74.20456367299995, 40.58028061599988], [-74.20453830399995, 40.58028087099992], [-74.20437042899988, 40.5803047879999], [-74.20420118899996, 40.580322231999894], [-74.20403100999988, 40.580333158999906], [-74.20394595099991, 40.580344580999906], [-74.20385400899988, 40.58037563699988], [-74.2037591199999, 40.5804010189999], [-74.20366188299988, 40.580420565999894], [-74.20356291499995, 40.58043415399991], [-74.20354150599995, 40.58043609799987], [-74.20350663899988, 40.580436159999884], [-74.20346962599996, 40.58043264699992], [-74.20343081399989, 40.58041929399991], [-74.20337531999998, 40.58040201699984], [-74.20332116299994, 40.580385643999925], [-74.20327432799998, 40.580374143999904], [-74.20323381699988, 40.58037045599987], [-74.20319223099993, 40.58036874499986], [-74.20314675199991, 40.580365310999866], [-74.20309425099991, 40.58036082099994], [-74.20300418199997, 40.58035008299991], [-74.20285677399994, 40.58032983999995], [-74.20281236899989, 40.580324427999884], [-74.20276536999991, 40.58031819799994], [-74.20269703699992, 40.580304951999935], [-74.20264545999996, 40.58029656099989], [-74.20260239099991, 40.58029205299991], [-74.20254855399999, 40.58028665799987], [-74.20251156099994, 40.580279916999906], [-74.20240475299997, 40.580257130999875], [-74.20230969599996, 40.580239456999934], [-74.2021713749999, 40.58020105299987], [-74.20214420399996, 40.58019190599988], [-74.20210029899994, 40.58018503899992], [-74.20204828299998, 40.58018625699991], [-74.20200397499998, 40.58018899599993], [-74.20195345799995, 40.58018581899987], [-74.20191105199993, 40.58017458399995], [-74.20187663199988, 40.580161496999885], [-74.20184248899996, 40.58014407399991], [-74.20179998599995, 40.58012471499987], [-74.20176039899995, 40.58011715299987], [-74.20173840099989, 40.58011683599991], [-74.2016748199999, 40.58011480599988], [-74.201626822, 40.58011151499991], [-74.20158593699988, 40.580103048999874], [-74.20154141299999, 40.58008234999988], [-74.20148133599993, 40.58004855599993], [-74.2014525309999, 40.5800347179999], [-74.20143434499994, 40.58002080599988], [-74.20140358699999, 40.57999220499992], [-74.2013805979999, 40.57997327799995], [-74.20135420199988, 40.5799583649999], [-74.2013187079999, 40.57994725599987], [-74.20128744299997, 40.579941655999875], [-74.20125254599999, 40.57993721699993], [-74.20121027499995, 40.57993503899993], [-74.20115662599996, 40.57993156499993], [-74.20109537699989, 40.579927500999915], [-74.20103374199996, 40.57992585099987], [-74.20094091399994, 40.579919562999876], [-74.200851071, 40.57991170499991], [-74.20079777099993, 40.5799048269999], [-74.20073234099989, 40.5799105679999], [-74.20069287199986, 40.57991832299988], [-74.20065720799997, 40.57993400299995], [-74.20062395199992, 40.57994860899994], [-74.20058554299995, 40.579961575999945], [-74.20055647099991, 40.57996640299992], [-74.20053829099999, 40.57996684599991], [-74.20051512399985, 40.579960381999875], [-74.20048848799996, 40.57994977799989], [-74.19964981299998, 40.58226262599991], [-74.19423261299991, 40.58514680599988], [-74.19332419799998, 40.585038280999875], [-74.19251747899993, 40.585158981999875], [-74.1923211319999, 40.5849132419999], [-74.19226048399997, 40.585206968999955], [-74.19131783499998, 40.5857044889999], [-74.19078902199995, 40.58518836699991], [-74.18971488199998, 40.58558930699995], [-74.18951740799994, 40.58566073099988], [-74.18932391199996, 40.58573822599987], [-74.18913471499995, 40.58582166299985], [-74.18895013099997, 40.58591090399989], [-74.18877046599995, 40.586005800999864], [-74.18859601799987, 40.586106195999946], [-74.1883043219999, 40.586480084999884], [-74.18819964399991, 40.586561903999936], [-74.18808950399989, 40.586639443999914], [-74.18797420199998, 40.58671249399986], [-74.18785405399997, 40.58678085299987], [-74.18772938699996, 40.58684433499992], [-74.18760054199991, 40.58690276699988], [-74.18746786899996, 40.586955989999886], [-74.18733173099997, 40.587003856999935], [-74.18719250099991, 40.58704623999991], [-74.18705055599993, 40.587083020999955], [-74.18681414999996, 40.58710057399991], [-74.18650033699991, 40.58719476299993], [-74.18636768699997, 40.587245200999895], [-74.18623192299994, 40.5872905709999], [-74.18609337999993, 40.58733075999993], [-74.18595239599988, 40.587365671999876], [-74.18580931799991, 40.58739521799987], [-74.18566449699989, 40.58741932799991], [-74.185518287, 40.58743794299991], [-74.185371048, 40.58745101499987], [-74.18730248699987, 40.588488221999896], [-74.18683246699995, 40.5890713609999], [-74.18468224999998, 40.59147895099995], [-74.18183084799989, 40.59409424599988], [-74.18095672799987, 40.59502355399995], [-74.18075192499991, 40.595241280999886], [-74.1803101799999, 40.595710899999894], [-74.17932277499995, 40.59676057899984], [-74.17822457199993, 40.59982806099992], [-74.17582749699989, 40.60152685299991], [-74.17488959399996, 40.602216572999914], [-74.17260229799992, 40.602971490999934], [-74.172351093, 40.60284533499991], [-74.1720855709999, 40.602733770999926], [-74.17180796899991, 40.60263814499992], [-74.171520817, 40.60255950899988], [-74.1712268479999, 40.602498611999835], [-74.1709288839999, 40.60245585699987], [-74.17062978699994, 40.60243131199995], [-74.16900518100002, 40.6022733739999], [-74.16719634899995, 40.602075311999926], [-74.16762086799996, 40.599849696999925], [-74.1679964909999, 40.597922359999885], [-74.16828458099997, 40.59643316499995], [-74.16886362299992, 40.59472808699995], [-74.16778450099996, 40.59418566499994], [-74.166390049, 40.59373792599991], [-74.16439050499997, 40.59348246099993], [-74.16360268199993, 40.593374198999854], [-74.16350785499993, 40.59327217999993], [-74.16349571599993, 40.593101060999864], [-74.16350123599995, 40.59292868299988], [-74.16352460699994, 40.59275663199986], [-74.16356570199993, 40.592586526999874], [-74.16362405199996, 40.592419936999896], [-74.16363304199993, 40.59240052699994], [-74.16369888299994, 40.592258338999905], [-74.16578641299996, 40.59070780399986], [-74.16617917099994, 40.59036865799992], [-74.1665588039999, 40.590020541999955], [-74.16692477899988, 40.589663906999924], [-74.16727661599998, 40.58929921399985], [-74.16761386399995, 40.58892696699992], [-74.1676915249999, 40.58883149299986], [-74.16776136599995, 40.58873256899993], [-74.16782312399997, 40.588630568999854], [-74.1678765679999, 40.588525874999895], [-74.16792283599987, 40.58841412299991], [-74.16796126199993, 40.588300656999905], [-74.1679917399999, 40.58818579699995], [-74.16801418199992, 40.5880698669999], [-74.16842916199994, 40.58649792299988]]]}}, {\"id\": \"99\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 100, \"Shape_Leng\": 0.0248131090342, \"Shape_Area\": 3.74700210291e-05, \"zone\": \"Garment District\", \"LocationID\": 100, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98729377099981, 40.75045160899988], [-73.98776905899994, 40.749787028999926], [-73.98793687499989, 40.74985496699991], [-73.99062219999986, 40.75099175399992], [-73.99211708899992, 40.751622077999876], [-73.99346417699986, 40.75219005499987], [-73.99297937799993, 40.75285340299989], [-73.99252855999994, 40.75347391299987], [-73.99207900099996, 40.75409111599986], [-73.99163271299996, 40.75470680499996], [-73.99117738199989, 40.75532984899992], [-73.99072636299992, 40.75595040599995], [-73.99027621399986, 40.756567564999926], [-73.98979085399996, 40.7572331319999], [-73.98841835499994, 40.7566549859999], [-73.98694873899993, 40.7560359019999], [-73.98640436699999, 40.755808028999866], [-73.98411754799999, 40.75484205299995], [-73.98459584300001, 40.754176941999916], [-73.98504363799995, 40.753554148999854], [-73.98548819099985, 40.752935179999945], [-73.98594653099995, 40.752316804999886], [-73.98639930899988, 40.751699985999934], [-73.986849911, 40.751081678999874], [-73.98729377099981, 40.75045160899988]]]}}, {\"id\": \"100\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 101, \"Shape_Leng\": 0.101709836277, \"Shape_Area\": 0.000452342528877, \"zone\": \"Glen Oaks\", \"LocationID\": 101, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.70134715908382, 40.750780580935846], [-73.70117316524451, 40.74892430181051], [-73.70090127406785, 40.74712854378895], [-73.700893400139, 40.747057265812565], [-73.70077960803322, 40.745041453900036], [-73.70058164018724, 40.74318384179423], [-73.70029222220697, 40.74101060200636], [-73.70002140398307, 40.73940416608862], [-73.7000090639354, 40.739323413081635], [-73.70002020501624, 40.73923654186594], [-73.70031835188448, 40.73896808279069], [-73.70081147181156, 40.73843915219474], [-73.7014707301906, 40.73751156098204], [-73.70206036378326, 40.736442756832986], [-73.7022808428287, 40.73612579680926], [-73.70268794512972, 40.735490222643136], [-73.70331490976335, 40.7345632569417], [-73.703522490113, 40.734207944901655], [-73.70394463088694, 40.73352626210759], [-73.70460114375578, 40.73256974721385], [-73.70499536283172, 40.731970558012414], [-73.7052216578116, 40.73161963010228], [-73.70584734777326, 40.7306728541998], [-73.70648028283007, 40.72971129886389], [-73.70709970121251, 40.72874771477672], [-73.70766217302989, 40.727830934754685], [-73.70856949090842, 40.72763641821302], [-73.71043109587094, 40.72724106148464], [-73.71049995381028, 40.72722708720663], [-73.71050351799981, 40.72727838999988], [-73.7105550639999, 40.728020849999915], [-73.71061871899991, 40.72883987399995], [-73.710655187, 40.729382965999925], [-73.71066275899989, 40.72949571099991], [-73.71075217299996, 40.73052537499992], [-73.710893249, 40.730971041999936], [-73.71147804899985, 40.73250140099988], [-73.71186191499989, 40.73315683699991], [-73.71208085999997, 40.73351852599989], [-73.7122862199999, 40.73385241799987], [-73.71250100999993, 40.73420163299987], [-73.71278769000006, 40.7346703809999], [-73.71283004500002, 40.73473962799994], [-73.71289825099977, 40.73485113699988], [-73.71328662200003, 40.73547312499993], [-73.71370155699985, 40.73616012199993], [-73.71436629299991, 40.737243891999896], [-73.71476982799997, 40.7379003479999], [-73.71494063099992, 40.73823459999989], [-73.71496199799985, 40.738276409999855], [-73.71511121299982, 40.73856840999988], [-73.71544946799989, 40.73921680399994], [-73.71578267699999, 40.73989578599985], [-73.71656758399989, 40.74164908599988], [-73.71740598499986, 40.74351092999989], [-73.71760306099992, 40.743974627999876], [-73.71762516399986, 40.74402847899988], [-73.71764302999979, 40.744070413999964], [-73.71823250200002, 40.74539666299995], [-73.71853063699987, 40.74603933099996], [-73.71869037499981, 40.746392765999936], [-73.71892134899986, 40.74689486899993], [-73.71894406299995, 40.74694425599994], [-73.71896368299996, 40.74699474099987], [-73.71988664100002, 40.74936951099994], [-73.72010148299994, 40.74977568599991], [-73.720144294, 40.74976283099995], [-73.72258571899978, 40.74885045099991], [-73.72339596299987, 40.750389075999855], [-73.72627213299992, 40.74930649099986], [-73.727664654, 40.749654641999946], [-73.72781151199987, 40.74929691999998], [-73.7279417009999, 40.74893498799988], [-73.72805493499999, 40.74856952199987], [-73.72815098799995, 40.74820123199985], [-73.72822968799987, 40.74783080399989], [-73.72829094199989, 40.74745895699992], [-73.72888621599986, 40.74756663599992], [-73.728962516, 40.74763288799986], [-73.72871952799996, 40.74819708099994], [-73.72862207399999, 40.74851209199989], [-73.728491785, 40.74904470699989], [-73.72837798399998, 40.74938116699989], [-73.7282538099999, 40.74969372899995], [-73.72805355599995, 40.750082150999894], [-73.72788401199996, 40.750387482999884], [-73.72771356199995, 40.750626894999904], [-73.72737889099987, 40.751029566999854], [-73.72709389599997, 40.75134160399987], [-73.726777395, 40.75162309199987], [-73.72666529899992, 40.75171358799987], [-73.7264147219999, 40.75191587799986], [-73.72602966399984, 40.752174354999916], [-73.72582988899985, 40.752291720999935], [-73.72551247899986, 40.75247819799989], [-73.72509445299985, 40.75270759199992], [-73.72506847999986, 40.752721843999915], [-73.72475823099984, 40.7529081479999], [-73.72384451899983, 40.75345681199988], [-73.72382698899992, 40.75346908899995], [-73.72351867199991, 40.753721437999886], [-73.72344135499989, 40.75379084499986], [-73.72326673399982, 40.75394759899992], [-73.72299301299984, 40.754217670999914], [-73.72279703199989, 40.75443353799988], [-73.72264543599995, 40.75460052399995], [-73.72241094799993, 40.754920188999925], [-73.72207071399988, 40.75549670299993], [-73.72180289199997, 40.75606989699991], [-73.72176339599984, 40.75613846699984], [-73.72161357099998, 40.756398594999894], [-73.72159631499989, 40.7564275039999], [-73.72145870299994, 40.75665808099994], [-73.72121176099988, 40.75697920799989], [-73.72111001499998, 40.75711152599996], [-73.72096511699988, 40.757293788999945], [-73.7206331939999, 40.75761683499996], [-73.72031595299998, 40.75789725799992], [-73.71998297099998, 40.75816082099986], [-73.7196773229999, 40.75837249899988], [-73.71956460799996, 40.7584505599999], [-73.71917909599986, 40.758666436999924], [-73.7189705039999, 40.75878323799989], [-73.71877787700004, 40.758871691999886], [-73.71831971899985, 40.7590820759999], [-73.71767237099996, 40.75931121199987], [-73.71685440399995, 40.75952551499989], [-73.71613842699988, 40.759658278999964], [-73.71612528199985, 40.75966071399991], [-73.71498213999979, 40.759782864999885], [-73.71455685899994, 40.759808807999896], [-73.71426287799991, 40.7598267469999], [-73.71351828186042, 40.75983773091619], [-73.71332542122421, 40.75973018978564], [-73.71244940715339, 40.75924170410932], [-73.71085768999157, 40.75819506482038], [-73.71054499713267, 40.757991034162465], [-73.70782538615794, 40.75621640416214], [-73.70527128308203, 40.75477044122686], [-73.70346781908046, 40.753683363268976], [-73.7027349570185, 40.75325051690488], [-73.70217841689814, 40.75288593949997], [-73.70169386980766, 40.75253685502325], [-73.70163345846068, 40.75249332984642], [-73.70162212285493, 40.75242551794639], [-73.70134715908382, 40.750780580935846]]]}}, {\"id\": \"101\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 102, \"Shape_Leng\": 0.136900484646, \"Shape_Area\": 0.000296595466345, \"zone\": \"Glendale\", \"LocationID\": 102, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85595536199988, 40.70642977899992], [-73.85593991699984, 40.7063980339999], [-73.85473240499985, 40.70676414299989], [-73.85473254599995, 40.706731913999924], [-73.854739573, 40.70518484999995], [-73.85411264799981, 40.70442593399984], [-73.85396105499991, 40.7042642169999], [-73.85392092799985, 40.70422140899995], [-73.85389480199986, 40.704183418999904], [-73.85371219899997, 40.70391788699991], [-73.85305663899999, 40.703137045999945], [-73.85252019099998, 40.70229172399997], [-73.85518109699989, 40.702583266999895], [-73.85529275899978, 40.702595501999944], [-73.85529571500001, 40.70199100999991], [-73.85538910499996, 40.701976915999836], [-73.85630830699988, 40.70183818199996], [-73.85716688299983, 40.70167410499991], [-73.85725351500002, 40.70121217699989], [-73.85739221099976, 40.701213978999895], [-73.85827405799986, 40.70122365899992], [-73.85893224399979, 40.70132034499989], [-73.85823494999983, 40.7028328909999], [-73.85819890699996, 40.70291107399986], [-73.85824915999993, 40.702916934999905], [-73.858530295, 40.7029497089999], [-73.85874741999989, 40.702974297999894], [-73.85997492299997, 40.70310890199989], [-73.86068159399994, 40.70318418699986], [-73.86142263099983, 40.70326975799987], [-73.86241891699996, 40.703377473999865], [-73.86285374199994, 40.7034144919999], [-73.86341506899986, 40.7033788049999], [-73.86379389300001, 40.70333379599989], [-73.8647283009999, 40.70325882199997], [-73.86564021600002, 40.70318258999988], [-73.86653775700002, 40.703109737999895], [-73.86724652399994, 40.703051654999875], [-73.86824273999996, 40.70297019599989], [-73.86873243499991, 40.702929097999856], [-73.86903271499997, 40.70290390099996], [-73.86915596200001, 40.702893432999865], [-73.86981898499995, 40.702837929999845], [-73.87073324199999, 40.70276203899995], [-73.87164751899996, 40.702687309999895], [-73.87210385799979, 40.70264976299988], [-73.87256153500006, 40.7026120979999], [-73.87351208999992, 40.70253545899994], [-73.8744627409999, 40.70245683199996], [-73.87543139400003, 40.70237435399989], [-73.87639785199997, 40.70229404199989], [-73.87625758399986, 40.70174876599991], [-73.87621714199997, 40.70159500499993], [-73.8777891919999, 40.70101362199989], [-73.87860797499998, 40.700704187999875], [-73.87942953599995, 40.70040484899988], [-73.880246012, 40.70009334899991], [-73.88106992699986, 40.69979988899989], [-73.88182690599986, 40.69952929899987], [-73.88273014799978, 40.700549908999925], [-73.88280948499988, 40.70068345599986], [-73.88343086899987, 40.70047443799989], [-73.88384293499996, 40.70034725999994], [-73.88431619499985, 40.70020119599994], [-73.88492223099999, 40.70010402799989], [-73.88584205499986, 40.699965471999896], [-73.8867672829999, 40.69982608499987], [-73.88771297999986, 40.699678075999906], [-73.88818025800003, 40.69885791099992], [-73.88809909499986, 40.69876329599988], [-73.88788297599991, 40.69801511999985], [-73.88785023699991, 40.69784014499988], [-73.88779396299992, 40.697665426999905], [-73.88771392599995, 40.697493692999906], [-73.88761064199986, 40.69732774499996], [-73.88748536199982, 40.69717024899989], [-73.88734003299987, 40.6970236659999], [-73.88717721900005, 40.69689008599989], [-73.88699989899986, 40.6967711439999], [-73.88681137100004, 40.696667947999885], [-73.88642048199995, 40.696458148999945], [-73.88648530299996, 40.6964399149999], [-73.88865416899996, 40.69582954899994], [-73.88828642399997, 40.69523691699991], [-73.88919365299998, 40.694935769999894], [-73.89195658699997, 40.694410569999896], [-73.89245207599993, 40.695225727999855], [-73.8950769439999, 40.694591389999886], [-73.89558625499984, 40.693979161999884], [-73.89398113199998, 40.69307596499988], [-73.89357935000005, 40.69308532899988], [-73.89326209899984, 40.69220094699994], [-73.89474270999993, 40.69263695299984], [-73.89576145299988, 40.69233462199984], [-73.89695653499987, 40.6919492069999], [-73.89768631800001, 40.69336525799989], [-73.89764233499987, 40.69340865099986], [-73.89742730199983, 40.69328897999988], [-73.89691021299981, 40.69384227099989], [-73.89736265799978, 40.6941108469999], [-73.89776907799997, 40.69404304999986], [-73.89817221699988, 40.69396265899992], [-73.89857107699993, 40.69386983399992], [-73.89896466800003, 40.69376479299993], [-73.89935202799984, 40.69364779599994], [-73.89973223499982, 40.69351918299989], [-73.900030392, 40.693437450999895], [-73.89954093099989, 40.69428573099985], [-73.89888585799994, 40.695389474999935], [-73.89838185199996, 40.69613599799993], [-73.89756262999987, 40.696480707999896], [-73.89677249999983, 40.696799116999884], [-73.89558457599986, 40.697127641999884], [-73.89578005099996, 40.69851096199989], [-73.89484994999988, 40.698589622999876], [-73.89500322699999, 40.69971237699985], [-73.8942016529999, 40.70050730899993], [-73.89384018399997, 40.70085972599988], [-73.89283458099982, 40.70191027699992], [-73.89194157299991, 40.70288622799989], [-73.8912683769999, 40.70359657599989], [-73.89105316999985, 40.70381641299988], [-73.88984330199986, 40.704972401999896], [-73.88911917799989, 40.705598876999915], [-73.88826271699995, 40.70580991299987], [-73.8873957249999, 40.70601812999995], [-73.88652826799989, 40.706225205999864], [-73.88554946899997, 40.70615461899994], [-73.88553214199999, 40.70644568299984], [-73.88456244600002, 40.70636218699992], [-73.88377296899992, 40.70629420199989], [-73.88266573799994, 40.70635551699991], [-73.8817801769999, 40.706404547999966], [-73.88088332699985, 40.7064539769999], [-73.88000104599993, 40.706492933999876], [-73.87900684300003, 40.706536838999924], [-73.87814452399996, 40.7066096259999], [-73.87720614800003, 40.70668883999991], [-73.87625034900005, 40.706744642999894], [-73.87532966799982, 40.70679838799992], [-73.87482359699989, 40.70681947799993], [-73.87425518099995, 40.706857030999906], [-73.87326053099987, 40.70692273699991], [-73.87153400399993, 40.70703676799997], [-73.86911007199988, 40.707089336999914], [-73.86949738599989, 40.70787206999989], [-73.86989582499986, 40.7085070939999], [-73.87016462499984, 40.70920774299987], [-73.86661823199995, 40.71040164899995], [-73.86557714699987, 40.71065921899992], [-73.86500981999998, 40.710776908999875], [-73.86207872699998, 40.71220105999993], [-73.85978566499996, 40.71331129599985], [-73.85965416799995, 40.713379960999866], [-73.85966066799995, 40.71329464599993], [-73.8597131359999, 40.71260553899993], [-73.85975932900001, 40.711845058999856], [-73.85964591099994, 40.711827343999886], [-73.8581874529999, 40.7115995459999], [-73.85729287299985, 40.711467676999874], [-73.85667714399985, 40.70769187399986], [-73.85595536199988, 40.70642977899992]]]}}, {\"id\": \"102\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 103, \"Shape_Leng\": 0.0143055167343, \"Shape_Area\": 6.33056361314e-06, \"zone\": \"Governor's Island/Ellis Island/Liberty Island\", \"LocationID\": 103, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.04388559600675, 40.690184824096946], [-74.04350596056258, 40.689687359818144], [-74.04273533896499, 40.69005019121004], [-74.04278433306983, 40.690120976845634], [-74.04270428413375, 40.69015520482211], [-74.04255372018648, 40.68996275914475], [-74.04263929302562, 40.68992817595552], [-74.04269380809586, 40.68999725905358], [-74.04346752305135, 40.68963699005171], [-74.04351637222922, 40.68919103337114], [-74.04364078613705, 40.688766559785], [-74.04397458578079, 40.68858240702793], [-74.04438521705613, 40.68851617820147], [-74.04478399103188, 40.688595660057736], [-74.04627538918811, 40.68932742544849], [-74.04680284899291, 40.689953256132966], [-74.04747651481168, 40.689611369999135], [-74.04772962698515, 40.689915318233105], [-74.04758571912386, 40.68998250691306], [-74.04743126078736, 40.689803890235574], [-74.04689205500284, 40.69005909816131], [-74.0472002938312, 40.69042481531183], [-74.04711050616285, 40.69047041292517], [-74.04711581988163, 40.69065580605927], [-74.04718321406033, 40.69074735502444], [-74.04719492423845, 40.69076326392415], [-74.04721324509907, 40.69079048716872], [-74.04722568396599, 40.69081957224158], [-74.04723192804273, 40.69084978818733], [-74.04723182036673, 40.69088037534203], [-74.04722536428265, 40.69091056537457], [-74.04721272190851, 40.690939599444675], [-74.04719421009764, 40.690966748235454], [-74.04717029494704, 40.690991329122504], [-74.04714091071601, 40.69101051633722], [-74.04710814069993, 40.69102620832851], [-74.0470726967221, 40.69103806420939], [-74.04703534814037, 40.69104582624724], [-74.0469969069895, 40.69104932615995], [-74.04617166999705, 40.69109798124324], [-74.04614707204328, 40.69112264601653], [-74.04609551081596, 40.691120063192244], [-74.04604442476779, 40.691114152164126], [-74.04599418397756, 40.69110495584743], [-74.04592592783412, 40.6910826032921], [-74.04586027413703, 40.69105611629163], [-74.04460616491457, 40.69057348447534], [-74.04388559600675, 40.690184824096946]]]}}, {\"id\": \"103\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 104, \"Shape_Leng\": 0.0212208330928, \"Shape_Area\": 1.19205339715e-05, \"zone\": \"Governor's Island/Ellis Island/Liberty Island\", \"LocationID\": 103, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.03995040794246, 40.70089063032128], [-74.03945262906649, 40.70053315991193], [-74.0393827810941, 40.700577694280575], [-74.03771124798313, 40.69934404017382], [-74.03809786207383, 40.6990395051221], [-74.03822954422694, 40.69836859134884], [-74.03900043889001, 40.698369890195494], [-74.03934037685066, 40.698115514418], [-74.0399327211647, 40.698544422145524], [-74.04025555058615, 40.69880482315377], [-74.04124261816106, 40.69953674121738], [-74.04174768884, 40.69914786378335], [-74.0399124888631, 40.69770204019913], [-74.0416605190741, 40.696452971319864], [-74.04367371215382, 40.69802040416498], [-74.04363686205573, 40.69804823209787], [-74.04365651081106, 40.698064091043896], [-74.04270395805347, 40.69880060029328], [-74.04296505213769, 40.698999362162375], [-74.0410986121853, 40.700492943194774], [-74.04080080623118, 40.700268549963226], [-74.04031250903736, 40.70062574909617], [-74.04039104391973, 40.700687376104355], [-74.04041179180665, 40.700678484059274], [-74.04053637716099, 40.70078209438744], [-74.04048062492674, 40.700820614335036], [-74.04034955315318, 40.70071601882726], [-74.04036511100502, 40.70070416692277], [-74.04028808718954, 40.70064361323363], [-74.03995040794246, 40.70089063032128]]]}}, {\"id\": \"104\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 105, \"Shape_Leng\": 0.0774253398314, \"Shape_Area\": 0.000368636392805, \"zone\": \"Governor's Island/Ellis Island/Liberty Island\", \"LocationID\": 103, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01674756096064, 40.693343368217555], [-74.01540359141636, 40.693066497309154], [-74.01535466389976, 40.69322485341718], [-74.01540916665937, 40.69323867317311], [-74.01538919768244, 40.693298134665504], [-74.01517482202624, 40.69325668033539], [-74.01518934209336, 40.69319860354291], [-74.01525474322258, 40.693206891183976], [-74.01530915985025, 40.69304704289151], [-74.01525951035647, 40.69303681423183], [-74.01512030609706, 40.6931875506329], [-74.01516209441294, 40.69320828569578], [-74.01512395634913, 40.693258069299695], [-74.0149313669649, 40.69315577223661], [-74.0149785911563, 40.69311566557114], [-74.01502946389674, 40.693146079734866], [-74.01515612159491, 40.69301551489163], [-74.01422123084896, 40.69282290634399], [-74.01418643053366, 40.69280348533486], [-74.01415791009549, 40.692782768813885], [-74.01413296148374, 40.69275953477481], [-74.01411196509743, 40.69273413747719], [-74.0134656664421, 40.692134996190894], [-74.01346559490221, 40.69213495063172], [-74.01345880318475, 40.69213036821136], [-74.01345142020139, 40.69212641492502], [-74.01344381222603, 40.692123273167304], [-74.01343629886243, 40.692121076587725], [-74.0134289639662, 40.692119825679], [-74.01342192440899, 40.69211953014426], [-74.01341855282882, 40.69211975502416], [-74.01341527571654, 40.69212021493394], [-74.01337499385887, 40.69212914447782], [-74.01329253788359, 40.692152143650446], [-74.01325647934522, 40.69216088218619], [-74.0131275881807, 40.6921850043228], [-74.01308076367873, 40.692194878694266], [-74.01305857042327, 40.69220153624106], [-74.0128533381957, 40.69224524345787], [-74.01272612664187, 40.692271335581886], [-74.01260406086602, 40.692294240965545], [-74.01258803080069, 40.69229831212829], [-74.01257261595585, 40.692303852771985], [-74.0125582903823, 40.692310688784325], [-74.01254543217101, 40.69231865108907], [-74.01253368614769, 40.69232805434204], [-74.01252307563139, 40.69233884241937], [-74.01252289827039, 40.69233903241466], [-74.0124088889432, 40.69249352748011], [-74.01240885439651, 40.69249357187864], [-74.01237989918387, 40.69253216208628], [-74.01233661589079, 40.692559893859546], [-74.0122978356522, 40.69256814655674], [-74.0122973149255, 40.69256825483513], [-74.01211815614386, 40.69260636571759], [-74.01211726765081, 40.692606554614414], [-74.01210359185654, 40.69260946499937], [-74.01204369179824, 40.69260181662152], [-74.01184920038554, 40.69251819825302], [-74.01184889307419, 40.6925178732815], [-74.0118118564723, 40.69247933506435], [-74.01179881230529, 40.692444594736955], [-74.0117837450172, 40.69240476697541], [-74.01175097032225, 40.69231811359295], [-74.01175033086639, 40.69231641141226], [-74.01175034252032, 40.69231634875577], [-74.01175749051248, 40.69226373956153], [-74.01183446726385, 40.6921590560231], [-74.01183833471082, 40.69215379727498], [-74.01183900825662, 40.69215287764775], [-74.01186468706686, 40.692117962626], [-74.01190309722556, 40.69209058350785], [-74.01191118868, 40.6920888440712], [-74.011968932587, 40.69207647455717], [-74.01213630770874, 40.69204064385554], [-74.01213732516256, 40.69204041808883], [-74.01214293316167, 40.692039219676275], [-74.01214885984382, 40.69203794936049], [-74.0122011411636, 40.69204490509602], [-74.01232430689473, 40.69209118800495], [-74.01235951844633, 40.692107770280096], [-74.01239001029668, 40.69212308503785], [-74.01240858613572, 40.692134240529455], [-74.0124292705935, 40.692154238892336], [-74.01246140165121, 40.69221262441595], [-74.01253982481636, 40.69219697386313], [-74.01255899015489, 40.69219315382361], [-74.01282826026977, 40.69213943594412], [-74.01322338051708, 40.69205919065682], [-74.01324195439297, 40.69205541613878], [-74.01324784563468, 40.692054217639885], [-74.0132851579289, 40.69204736900613], [-74.01330314021662, 40.69204264868684], [-74.01331123156653, 40.69203949572284], [-74.01331804474162, 40.69203590247678], [-74.01332367657382, 40.692031822532485], [-74.01332606546653, 40.692029579231274], [-74.01332818317637, 40.69202720143703], [-74.01333064247271, 40.69202372579595], [-74.01333264149613, 40.69201997010238], [-74.01333505303288, 40.692012099552606], [-74.013335394941, 40.69200820902383], [-74.01333515825114, 40.69200446297076], [-74.01333432976998, 40.692000924552666], [-74.01333293283652, 40.691997636989036], [-74.01332546520501, 40.69198520246153], [-74.01331621035997, 40.691972497416074], [-74.01327481959478, 40.69193534679518], [-74.01326620460138, 40.69192718049207], [-74.0132647611728, 40.69192580274551], [-74.01325528699353, 40.691916817498964], [-74.01318656537055, 40.691957929417796], [-74.01291228751674, 40.691690730833784], [-74.01292800393819, 40.69167610442813], [-74.0128598710889, 40.69160299086556], [-74.01289688430354, 40.691576895500305], [-74.01289518355655, 40.69157528340696], [-74.0128751983542, 40.69155633025172], [-74.01286547266726, 40.69154710124848], [-74.01281595185874, 40.69150012672344], [-74.0127543683677, 40.6914764676359], [-74.01265071760986, 40.69139945918043], [-74.01262511694793, 40.69140873714776], [-74.01260856859405, 40.69136410462975], [-74.01258434695997, 40.69137064972958], [-74.01258434935933, 40.69138336238062], [-74.0125450515151, 40.6913915932729], [-74.01255488211348, 40.69142300124786], [-74.01253228594773, 40.691428986617765], [-74.01251950752707, 40.691396831454185], [-74.0120341761657, 40.69150307668566], [-74.01201746659368, 40.69145820805393], [-74.01259753152567, 40.69133433554453], [-74.01252783209965, 40.69114633733795], [-74.01218355071047, 40.69113621594061], [-74.01218682075256, 40.69107802427063], [-74.01182575354963, 40.69106481621614], [-74.01182847957557, 40.69101135776438], [-74.01219500090075, 40.69102884756879], [-74.01219552958608, 40.691020573667956], [-74.01221017410666, 40.690791696801234], [-74.01208414396382, 40.69078062230048], [-74.01208232424358, 40.690759881924535], [-74.01205325498773, 40.690739143903045], [-74.01199875966572, 40.69075021210482], [-74.0119878628144, 40.69077233607646], [-74.01197151318819, 40.690770955509436], [-74.01198603276991, 40.69069766811433], [-74.01199329167372, 40.69065065379713], [-74.01198056761243, 40.690606407271744], [-74.01204959850608, 40.69061054854669], [-74.01205685428478, 40.69055385448957], [-74.0120986343193, 40.69054693658221], [-74.01210771369418, 40.69052895959737], [-74.01212769712372, 40.690528957493186], [-74.01214221600921, 40.690454287807185], [-74.012231311469, 40.69046133077506], [-74.01230688304486, 40.68928018293753], [-74.01270320924039, 40.68879053981696], [-74.01275451203071, 40.68872055623788], [-74.01316926402752, 40.6881547712032], [-74.01326486705615, 40.68812243009664], [-74.01333019977945, 40.68809316604797], [-74.01341858951315, 40.688052198317855], [-74.01347243916622, 40.68801205796619], [-74.01488097975005, 40.68725457048855], [-74.01604730539512, 40.68733800629771], [-74.01609789228392, 40.68731459228818], [-74.01651927578641, 40.68711955114744], [-74.01698146864109, 40.68690738448149], [-74.0170498660923, 40.68687598791896], [-74.01671582671982, 40.6864455937096], [-74.01664664084757, 40.68641634723339], [-74.0151516167432, 40.68652480204175], [-74.0151439102199, 40.686437034043024], [-74.01661971233223, 40.68631980484093], [-74.01667351529983, 40.68630516855345], [-74.01727291238646, 40.685690697681224], [-74.01753032434085, 40.68536298793239], [-74.0176225750659, 40.685409783720694], [-74.01736132076928, 40.68574627202702], [-74.01678113957136, 40.686349037919165], [-74.01677346463228, 40.68639877544977], [-74.01712099101067, 40.68684333763659], [-74.02015656652381, 40.68544978729508], [-74.02054367293081, 40.68527206792595], [-74.02020686710522, 40.68483383307706], [-74.02016278941748, 40.68483301741088], [-74.01945957389425, 40.685154957922364], [-74.01939842391928, 40.685087404047145], [-74.0209525501029, 40.684384431647196], [-74.0210087789929, 40.6844535390843], [-74.02028899992284, 40.68478444841858], [-74.02027819846491, 40.68480913517448], [-74.02061267235811, 40.68524039019806], [-74.02106226329632, 40.68504102470039], [-74.02183909525692, 40.68469654054689], [-74.0222407889853, 40.68451840810193], [-74.02233552903553, 40.68447639360419], [-74.02239732580504, 40.6844489893264], [-74.02259359581036, 40.68435969594992], [-74.02213250186877, 40.683767672385194], [-74.02305574749599, 40.6829169454452], [-74.02319462834667, 40.68300049806215], [-74.02236823557664, 40.683779916793995], [-74.0227589902996, 40.68428444833225], [-74.0230170445636, 40.68416704426917], [-74.02313509710321, 40.684124156652665], [-74.0232567831563, 40.68408541432076], [-74.0233977685456, 40.684040969821744], [-74.0234147937385, 40.68403560313048], [-74.02354160370115, 40.684002983592535], [-74.0237826733674, 40.683968915869016], [-74.0239473535509, 40.68394921598666], [-74.02421537269672, 40.68394424346432], [-74.02443173023701, 40.68395403059442], [-74.02464809241224, 40.68397856661301], [-74.02484185637266, 40.684012941142406], [-74.02499306057516, 40.68405211253555], [-74.02521646935898, 40.68409398087132], [-74.02546836768583, 40.68415292257828], [-74.02567183758212, 40.68423645620504], [-74.02588501823519, 40.68437652684006], [-74.02604653184764, 40.68451415115492], [-74.02612578192962, 40.684593511439715], [-74.02623238290327, 40.68470026105373], [-74.026334042424, 40.6848016977242], [-74.02644066624605, 40.684951624519876], [-74.02651822912101, 40.685108933148456], [-74.02657640943082, 40.68524166303809], [-74.02660875559273, 40.68537685800988], [-74.0266281908386, 40.685526804749706], [-74.02662178340184, 40.6856546341114], [-74.02660890959791, 40.685762798069575], [-74.02657021545329, 40.685905384681284], [-74.02649600144045, 40.686055352516355], [-74.02642577243199, 40.68617533982955], [-74.02637621613817, 40.6862313188001], [-74.0263127037025, 40.68629511626799], [-74.02622025384302, 40.686384568093466], [-74.02617488097594, 40.6864290722291], [-74.02606584987554, 40.686530566227816], [-74.0257838705966, 40.68679470641727], [-74.02557971866312, 40.6869974566], [-74.02538699247003, 40.68716690428223], [-74.02518794246518, 40.68735622748171], [-74.02515426770418, 40.68738749447694], [-74.02498452676963, 40.68754509996105], [-74.02495996488949, 40.6875682036494], [-74.02474903514577, 40.6877870294122], [-74.0245639021157, 40.68796465845378], [-74.02445685221299, 40.68806055942187], [-74.024339228089, 40.68816581005809], [-74.02432308577353, 40.68818025702191], [-74.0241951143645, 40.68829476558986], [-74.02411997621049, 40.688368243510276], [-74.02397192277759, 40.68851302540427], [-74.02396924880294, 40.6885155500981], [-74.02381144169377, 40.6886645158027], [-74.02367436482533, 40.688792309760025], [-74.02353644452148, 40.68891577045838], [-74.02339034096943, 40.68904924761344], [-74.02313394750364, 40.68929545426466], [-74.0226846593962, 40.68971747085378], [-74.02256537904927, 40.68982575842605], [-74.02243973951947, 40.68993981943943], [-74.02225114057582, 40.69012221857815], [-74.02189942193753, 40.69044347236064], [-74.02149639768447, 40.69082705033472], [-74.02147636060457, 40.690846622621194], [-74.02121292061638, 40.69110396565088], [-74.02097996202453, 40.69132439086214], [-74.02071969701748, 40.69158098499546], [-74.02071619151157, 40.691584232140485], [-74.02046388462907, 40.691817991104024], [-74.02020527434266, 40.69203392385426], [-74.01985623976917, 40.69290623279892], [-74.01979706616879, 40.693041562744504], [-74.01979706638019, 40.69304227473585], [-74.01975820423938, 40.69309283565343], [-74.01975203020818, 40.693098528288935], [-74.01971454942077, 40.69313300682718], [-74.0196683350802, 40.69316438792429], [-74.01962614022352, 40.69318563860613], [-74.01960136832818, 40.69319203651079], [-74.01952625786083, 40.693206790851846], [-74.01945183356794, 40.69322738318996], [-74.01938401317452, 40.693234769858826], [-74.01934089597206, 40.69323477711305], [-74.01928712492476, 40.69323478613706], [-74.01917408781627, 40.693232346225926], [-74.01899645645297, 40.69322254283086], [-74.01886404103503, 40.69321764868789], [-74.01879710616217, 40.69321765963996], [-74.01875100400217, 40.69321766716067], [-74.0186541185004, 40.69322505712626], [-74.01852170583395, 40.69322999493901], [-74.01837960490766, 40.693239851076], [-74.01830295903022, 40.69324774729764], [-74.01801807424182, 40.6932657952483], [-74.017728751353, 40.69328178778384], [-74.01742068986829, 40.69329880082907], [-74.01723188418137, 40.69332043274831], [-74.01711872274268, 40.693326078054056], [-74.01700668693556, 40.693344068510434], [-74.01690615078013, 40.69335341225322], [-74.01682187641964, 40.69334558113752], [-74.01674756096064, 40.693343368217555]]]}}, {\"id\": \"105\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 106, \"Shape_Leng\": 0.0492110119798, \"Shape_Area\": 0.000113616826567, \"zone\": \"Gowanus\", \"LocationID\": 106, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98610388199985, 40.68100852199994], [-73.98413878099997, 40.68024537099991], [-73.98185715599982, 40.67935261499984], [-73.98175642299985, 40.67931462399987], [-73.98218939700001, 40.67867580399995], [-73.98261725399998, 40.678042429999905], [-73.98304811899982, 40.677406685999834], [-73.98347361499994, 40.67677916099998], [-73.98390048699991, 40.67614865299992], [-73.98420294599991, 40.67577119999985], [-73.98469872999982, 40.67517562999993], [-73.98520256000002, 40.67457380999991], [-73.98570017199982, 40.67397483299985], [-73.98670905399989, 40.67276673299988], [-73.98721100499988, 40.67216061599988], [-73.98744496399998, 40.67188008599991], [-73.98771575999984, 40.671555380999976], [-73.98821210799997, 40.67096081399996], [-73.988713226, 40.67035690699993], [-73.9890700769999, 40.66992948399993], [-73.98907079799996, 40.66992861999993], [-73.98921685399992, 40.669753474999915], [-73.98972203499986, 40.66914780699988], [-73.99022252199988, 40.668548097999896], [-73.99072558999995, 40.66794617299989], [-73.99123399599989, 40.6673356529999], [-73.99173365199998, 40.66673659499987], [-73.9922346229999, 40.66613762499993], [-73.99277080699981, 40.665507280999876], [-73.99284786000001, 40.66555439199987], [-73.99490350199993, 40.66678543499988], [-73.99507117699984, 40.66707573199994], [-73.99585129199981, 40.667972495999884], [-73.99616566099998, 40.668435497999965], [-73.99648142699999, 40.668925235999936], [-73.99659742999994, 40.66881192799984], [-73.99667191699996, 40.66873916799991], [-73.9969179149999, 40.6691534839999], [-73.99696465099986, 40.669234548999924], [-73.99757732099995, 40.67019948399994], [-73.99764875199986, 40.670311734999906], [-73.99771474299992, 40.67041544799982], [-73.99787130000004, 40.67066149999993], [-73.99798931599987, 40.670842081999915], [-73.99822510099992, 40.67120286799991], [-73.99845678099994, 40.671568042999894], [-73.9986188799999, 40.67183017199989], [-73.99855090599986, 40.67192594199986], [-73.99847134099994, 40.67205910099987], [-73.99812844699997, 40.672440749999964], [-73.99805741699998, 40.67246954499988], [-73.99745115699996, 40.67319456799988], [-73.99717696100001, 40.67352829399992], [-73.99696673299994, 40.673784159999876], [-73.99666043499988, 40.67414278099988], [-73.9959621319999, 40.67496822399994], [-73.99572187999998, 40.675222292999905], [-73.99540989400003, 40.67556309699996], [-73.99520266299999, 40.67572952499992], [-73.99506451099984, 40.67584047499984], [-73.99493096199983, 40.67590705299988], [-73.99479291199987, 40.67596807599986], [-73.99465075899997, 40.67602336999993], [-73.99450491199987, 40.67607277499994], [-73.99435579199994, 40.67611614799993], [-73.99420802699994, 40.67615242699986], [-73.9940579759999, 40.676182783999884], [-73.99396082100002, 40.67616263899984], [-73.99380485099994, 40.67617660299988], [-73.99364806799993, 40.67618349899993], [-73.993491024, 40.67618330199986], [-73.9933342719999, 40.676176012999846], [-73.99317836299984, 40.676161657999906], [-73.9931237449998, 40.67617871399997], [-73.99280910199987, 40.67614874999984], [-73.99279927099997, 40.67613190599994], [-73.99256575599999, 40.6761058199999], [-73.99069861899994, 40.67580741899994], [-73.99036651399997, 40.67621832099995], [-73.98986566199996, 40.676826601999934], [-73.98940896399986, 40.67745451699993], [-73.98931805499996, 40.67825393999993], [-73.98931813699981, 40.67834938899991], [-73.98897369299988, 40.67884987699992], [-73.98850509899982, 40.6795371569999], [-73.98808029299991, 40.68017216599984], [-73.9876680399999, 40.68081780999993], [-73.98729306099992, 40.6806696979999], [-73.98653302999993, 40.680372638999884], [-73.98610388199985, 40.68100852199994]]]}}, {\"id\": \"106\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 107, \"Shape_Leng\": 0.0380413645908, \"Shape_Area\": 7.46112192675e-05, \"zone\": \"Gramercy\", \"LocationID\": 107, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98238628799993, 40.73141124499993], [-73.98255629299986, 40.731350133999875], [-73.98494102799992, 40.73234760399984], [-73.98718049699994, 40.733292147999876], [-73.98877931699997, 40.733965399999924], [-73.98990295999991, 40.7344347899999], [-73.98996936399989, 40.73490456699994], [-73.98997766699992, 40.73497925499991], [-73.9899791279999, 40.73505420099983], [-73.98997373899982, 40.735129042999894], [-73.98996152699993, 40.73520341999988], [-73.98994254999981, 40.73527697099995], [-73.98991689999981, 40.73534934199992], [-73.989874214, 40.73540825799992], [-73.98945472099987, 40.7359872469999], [-73.98920953899987, 40.736307120999854], [-73.98901559399987, 40.73656015399992], [-73.98859298399994, 40.73714256099993], [-73.98817910299996, 40.73773074899986], [-73.98774074499983, 40.738312187999945], [-73.98730154899981, 40.73890302599993], [-73.98703592799987, 40.739266378999886], [-73.9868491859999, 40.73952182199985], [-73.98636141699991, 40.74018480099992], [-73.98587913199991, 40.74085299999997], [-73.98542767799994, 40.741470348999876], [-73.98497775499983, 40.74208895699995], [-73.98452636699996, 40.7427070419999], [-73.98407485299995, 40.74332471799989], [-73.98400852, 40.743299017999945], [-73.98245520799999, 40.74264695999995], [-73.98085965399994, 40.74196976799989], [-73.98131180499989, 40.741350923999896], [-73.98176150799992, 40.74073311299989], [-73.98221134499993, 40.7401145349999], [-73.98266497499989, 40.73949733299987], [-73.98041832999985, 40.73855053099987], [-73.97949062699986, 40.73815979199986], [-73.97891574499987, 40.737917658999876], [-73.97805226499996, 40.7375539539999], [-73.97849845599995, 40.73679095699989], [-73.97857449799984, 40.736677280999864], [-73.97890917399987, 40.736164125999906], [-73.9793481959999, 40.735553111999884], [-73.97944606000006, 40.73541451499991], [-73.97978904000004, 40.734961816999906], [-73.98009050499998, 40.73454437099989], [-73.98020496200002, 40.73438532199987], [-73.98041991299985, 40.73408982099988], [-73.980644989, 40.733779319999904], [-73.98105323699987, 40.73320848199997], [-73.98149910399985, 40.732618595999924], [-73.98180339299985, 40.7322047979999], [-73.98196782199994, 40.731982560999874], [-73.98232282599997, 40.731497889999915], [-73.98238628799993, 40.73141124499993]]]}}, {\"id\": \"107\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 108, \"Shape_Leng\": 0.11117141227, \"Shape_Area\": 0.000352866009465, \"zone\": \"Gravesend\", \"LocationID\": 108, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97148350899987, 40.59717158499996], [-73.97132951099988, 40.59635243099994], [-73.9715332449999, 40.59636870899992], [-73.97169554999981, 40.59548854199992], [-73.97176509099998, 40.59511141099989], [-73.97189684799982, 40.5943525109999], [-73.97178220299999, 40.593756083999914], [-73.97155530499995, 40.59256045799992], [-73.97141847599984, 40.59183850999992], [-73.97128748899985, 40.59115950199986], [-73.97115101799994, 40.59044244399989], [-73.97022337700001, 40.590540175999934], [-73.96929959299999, 40.590647588999964], [-73.96837099999982, 40.59074946599986], [-73.96787702199988, 40.590803756999925], [-73.96744527499996, 40.59085120999989], [-73.96651951199983, 40.590952872999885], [-73.96542523899983, 40.59107142799992], [-73.9652882789999, 40.59108660999993], [-73.96514385199995, 40.5911019159999], [-73.96530164399988, 40.59035822999992], [-73.96546500699992, 40.589643336999856], [-73.9656408479998, 40.58888742799993], [-73.96581402699984, 40.58813321799995], [-73.96597498399979, 40.58743213599996], [-73.96615022799989, 40.58666882399992], [-73.96633012800002, 40.58589875099992], [-73.96636486799986, 40.585751243999916], [-73.96643740799989, 40.58541469399988], [-73.96649796199982, 40.58514249299987], [-73.96653588899984, 40.584957806999974], [-73.96660079799997, 40.58464174199989], [-73.96661531499983, 40.58457103699992], [-73.96670868800001, 40.584569082999906], [-73.9668442979999, 40.584569328999876], [-73.96781584999998, 40.5845390509999], [-73.96899979299988, 40.58406350399991], [-73.96985445599992, 40.583966982999875], [-73.970055463, 40.58394772999985], [-73.97045373499985, 40.583892278999876], [-73.970846422, 40.58382021499993], [-73.97174499699994, 40.58366274899988], [-73.97308506499995, 40.5834298259999], [-73.97449712999993, 40.583153389999886], [-73.97461091899993, 40.5831298759999], [-73.97468194399984, 40.58312411799988], [-73.97742525299988, 40.58290659299994], [-73.98006567999988, 40.583317863999845], [-73.98044790899985, 40.58341064299993], [-73.98175452099993, 40.58358376299985], [-73.98281157699994, 40.58379238799989], [-73.983875933, 40.58397574499992], [-73.98482192899988, 40.584168522999924], [-73.98492731, 40.584595951999844], [-73.98548157999991, 40.58451967699994], [-73.9858605929999, 40.5845257129999], [-73.98627573699999, 40.584479980999895], [-73.98671582299981, 40.58456849999991], [-73.98701022999991, 40.58462626699987], [-73.98710120299997, 40.58464459199988], [-73.98687540199994, 40.58425158799985], [-73.98683590499999, 40.58418633099989], [-73.98679577199978, 40.58411362799992], [-73.98660338299986, 40.58376512099988], [-73.98654457099988, 40.583658590999946], [-73.98662350499995, 40.58367393699996], [-73.98730893199986, 40.58359445299993], [-73.98829948299982, 40.58346674699986], [-73.98852399299994, 40.58314353499986], [-73.98874850099989, 40.58282032299996], [-73.98888884999998, 40.58256733499994], [-73.98928724999996, 40.58195758299993], [-73.98971762499994, 40.5808382269999], [-73.98995504960881, 40.58066166976562], [-73.9911944079673, 40.581028307699384], [-73.99163007472373, 40.58107988438188], [-73.99174273416997, 40.58120016258747], [-73.99168261881185, 40.581383427077085], [-73.99173078128501, 40.581486505047906], [-73.99182532449969, 40.58150370686698], [-73.99185535656628, 40.5816239785461], [-73.99182529575866, 40.58173852030272], [-73.99169553014772, 40.58185850327888], [-73.99033762504436, 40.58421825488588], [-73.99035067235161, 40.584276528110806], [-73.99035573816279, 40.584335517948475], [-73.99035276803677, 40.58439459069805], [-73.99034179385924, 40.58445311176603], [-73.99032293350066, 40.584510452484004], [-73.99029638955085, 40.58456599686182], [-73.99026244714351, 40.5846191482053], [-73.99097612298843, 40.584350030809986], [-73.99267391275022, 40.583588443369415], [-73.99272664752623, 40.58356177434858], [-73.99277367838121, 40.583529494172915], [-73.99281398707836, 40.58349230172691], [-73.99284670092436, 40.583451002247294], [-73.99288601634854, 40.583424157067896], [-73.99292971084553, 40.583401587909094], [-73.9929769901788, 40.58338370500898], [-73.99302699495362, 40.583370833423814], [-73.99304892967515, 40.58336317452515], [-73.99306880707216, 40.58335275049628], [-73.99308603338487, 40.58333987271512], [-73.99310009404569, 40.58332492585561], [-73.99311056904914, 40.58330835639695], [-73.99311714549789, 40.58329065928717], [-73.99314662510577, 40.58327194844545], [-73.99317951603533, 40.58325690118909], [-73.99321504197076, 40.58324587267355], [-73.99325236440465, 40.583239123201075], [-73.99339745701728, 40.58314812948715], [-73.99354754864491, 40.58306197803178], [-73.9937023614738, 40.58298082828728], [-73.99386160895484, 40.582904830448165], [-73.99402499633493, 40.58283412517319], [-73.99419222120069, 40.58276884332482], [-73.99436297403963, 40.58270910572781], [-73.99453693881253, 40.58265502294484], [-73.99458961848661, 40.58264986405606], [-73.9946405256929, 40.58263831674982], [-73.99468833671246, 40.58262068128622], [-73.99473180833827, 40.58259741623282], [-73.99476981020157, 40.58256912654064], [-73.99479586376927, 40.5825394805631], [-73.99482767692096, 40.58251330524204], [-73.9948644670117, 40.58249124451884], [-73.99490532896193, 40.58247384111093], [-73.994949257522, 40.58246152316033], [-73.99499517200287, 40.58245459370131], [-73.99504973843936, 40.58245098811909], [-73.9951028057891, 40.582440654087556], [-73.99515295656298, 40.58242386764041], [-73.99519885117982, 40.58240107716288], [-73.99523926374887, 40.582372891413506], [-73.99527311481427, 40.582340063264155], [-73.99528742810297, 40.582311998131665], [-73.99530754680039, 40.58228608565947], [-73.99533293339671, 40.58226301814176], [-73.99536290964227, 40.58224341186448], [-73.9953966746708, 40.582227790640644], [-73.99543332639404, 40.5822165718157], [-73.99547188560318, 40.58221005511802], [-73.99551132212923, 40.582208414651085], [-73.99555058236467, 40.582211694242616], [-73.99558861741181, 40.58221980627343], [-73.99589671851147, 40.58215701878393], [-73.99605528820516, 40.58212890385413], [-73.99622429809615, 40.58220193058146], [-73.99672570402983, 40.582253490579355], [-73.99739612012652, 40.582429616999576], [-73.99781301710898, 40.58261003108302], [-74.00017922859408, 40.583073950747675], [-74.0003088081067, 40.58317703751469], [-74.00041739503702, 40.58354592150335], [-74.00043685358045, 40.58404896948651], [-74.00035951790473, 40.58418644588039], [-74.00024683907247, 40.58426376283366], [-74.00010035579987, 40.58437544199797], [-73.99998204289255, 40.58443128159514], [-73.99985809444573, 40.58454296095274], [-73.99973977980433, 40.58457732286521], [-73.99963273341196, 40.58463316229419], [-73.99953132177025, 40.58470188733625], [-73.99945807926105, 40.58475343161644], [-73.99933976309926, 40.584860814817176], [-73.99915383954428, 40.58494242587021], [-73.99898481729464, 40.585015445530836], [-73.99878762307108, 40.5851528939333], [-73.99860169877729, 40.58523020847001], [-73.99846084611694, 40.58528604638743], [-73.99826365291965, 40.58531181562509], [-73.99808523776004, 40.585364721581485], [-73.998088991884, 40.58547074076327], [-73.99828673553586, 40.58548306298668], [-73.9986711016601, 40.585507013215405], [-73.99928905009463, 40.58555236605288], [-73.99946933901515, 40.58568981900932], [-73.99970597099518, 40.585689819858544], [-73.9997003362606, 40.58575424962766], [-73.99952567990363, 40.58575854465988], [-73.99963423823505, 40.586069747433356], [-73.99956511570304, 40.5862439200957], [-73.99935665157753, 40.58638136941568], [-73.9989735281156, 40.58642861560582], [-73.99877633053215, 40.58648445338662], [-73.99863826384758, 40.58649162946082], [-73.99828052442538, 40.58651021958485], [-73.99821854784219, 40.58651021864066], [-73.99814530448992, 40.586531693907126], [-73.99623653794225, 40.587421255879605], [-73.99417305638372, 40.5883828578208], [-73.99423502922512, 40.58843870064835], [-73.99559857410351, 40.58784170755412], [-73.9956605451227, 40.587936207343844], [-73.9940790462178, 40.58869344199059], [-73.99421554608305, 40.588868530482756], [-73.99497484847483, 40.588505372015895], [-73.99533140797436, 40.588330066781644], [-73.99534566885502, 40.588345017174476], [-73.99548294316648, 40.58828522272529], [-73.99549007351672, 40.58829745499158], [-73.99555247167991, 40.588264839296585], [-73.99556316619993, 40.58827707219872], [-73.99513351269499, 40.58848091554354], [-73.99515490404112, 40.588514893739465], [-73.99505684930178, 40.58856109833817], [-73.99509949490815, 40.58859250504379], [-73.99552212251818, 40.58839062298806], [-73.9955277563281, 40.58840351006567], [-73.99777708135538, 40.58735323755155], [-73.99795639739294, 40.58726950636562], [-73.99796055106226, 40.58727742354104], [-73.99797893506641, 40.58731246396774], [-73.99555029453958, 40.58842069392083], [-73.99555029368085, 40.58843358128157], [-73.99515584195441, 40.58862687289014], [-73.99487677695376, 40.58877174667522], [-73.99490226050743, 40.588802987503094], [-73.99534737661207, 40.589331380439994], [-74.00011449538408, 40.58695164090261], [-74.00010322572263, 40.58659079894537], [-74.000108860195, 40.586560729096675], [-74.00016520656312, 40.586504884962544], [-74.00023845644195, 40.58647051937003], [-74.00031734303268, 40.586453335023286], [-74.0003736889094, 40.58647481463], [-74.0004327749314, 40.586544594921556], [-74.00042529438699, 40.5866924315609], [-74.0004694808748, 40.58694304876059], [-74.00048912050953, 40.58704285506199], [-74.00038207107852, 40.58713305717468], [-73.99788311275509, 40.58834343796131], [-73.9979338240619, 40.588399282951634], [-73.99801867061471, 40.588425400020526], [-73.99375761899992, 40.590331843999856], [-73.99330855, 40.58958389899994], [-73.99297835999997, 40.58973003299996], [-73.99291591599997, 40.58975737899986], [-73.99286067699995, 40.58978269799995], [-73.99255747799991, 40.5899216749999], [-73.99161412399994, 40.59035644499992], [-73.99153934399993, 40.59039090299989], [-73.99147342099985, 40.590449305999904], [-73.99066311999997, 40.5911671669999], [-73.99036425099996, 40.591431931999935], [-73.98962150099992, 40.59098199299996], [-73.98798856399989, 40.59255277799986], [-73.98601936799993, 40.594446048999906], [-73.98467794099993, 40.5936586499999], [-73.98463965399982, 40.59355217599988], [-73.98412888499986, 40.593610666999915], [-73.98407040399994, 40.593617363999876], [-73.98366766399984, 40.593663481999926], [-73.98354622399994, 40.59367650999994], [-73.9834412439999, 40.593687772999864], [-73.98303296899982, 40.593730292999915], [-73.98273346399986, 40.593763698999915], [-73.98179084800002, 40.593867455999906], [-73.98108086699995, 40.593952890999915], [-73.98071099999997, 40.59399739599998], [-73.97995443499985, 40.594073811999884], [-73.98014301299985, 40.595066175999904], [-73.98035785499995, 40.59619154099992], [-73.9794258679999, 40.596297522999876], [-73.97890571399992, 40.59635272999989], [-73.9785016919999, 40.596394852999914], [-73.97750674199997, 40.596506452999904], [-73.97650934500001, 40.59661610899983], [-73.97558819299998, 40.59671920599991], [-73.97503940399993, 40.59678002699995], [-73.974926956, 40.596793537999936], [-73.97408390599992, 40.59688294399995], [-73.97323983499987, 40.59697881799983], [-73.97148350899987, 40.59717158499996]]]}}, {\"id\": \"108\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 109, \"Shape_Leng\": 0.178267819599, \"Shape_Area\": 0.00116960076185, \"zone\": \"Great Kills\", \"LocationID\": 109, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.14706472699994, 40.5705086149999], [-74.14699254699993, 40.570504754999874], [-74.14692022299994, 40.57050644499987], [-74.14684848799997, 40.57051367099988], [-74.14677806799996, 40.5705263589999], [-74.14671789599998, 40.57054077199993], [-74.14665601599995, 40.570550074999886], [-74.146593173, 40.5705541569999], [-74.14653012099988, 40.5705529679999], [-74.1464676189999, 40.57054652199994], [-74.14640641799993, 40.5705348979999], [-74.146347254, 40.57051823399995], [-74.14629083699998, 40.570496730999885], [-74.14559994099997, 40.57021110699987], [-74.14489975699995, 40.569915169999945], [-74.14438744999991, 40.56970610099991], [-74.1428731119999, 40.56908188399993], [-74.14151969499997, 40.56852030899988], [-74.14131233599991, 40.5684333479999], [-74.14026824299997, 40.56795018899985], [-74.13976413499992, 40.56771224499988], [-74.13889993799994, 40.56729303599993], [-74.13768350999999, 40.56671786699995], [-74.13651149199997, 40.566166621999876], [-74.13622008499992, 40.56598659899989], [-74.1359159259999, 40.5658179609999], [-74.135600208, 40.5656614559999], [-74.13527417299991, 40.565517760999874], [-74.13493919999993, 40.56538743599986], [-74.13459671299997, 40.5652709339999], [-74.13329816299988, 40.56497397399988], [-74.13240280399998, 40.564760999999876], [-74.13170009, 40.56453574899991], [-74.13264383999997, 40.563503334999915], [-74.13310979199996, 40.56286769099991], [-74.13340740099987, 40.5621298059999], [-74.13367116399996, 40.5618409339999], [-74.13396833899998, 40.56158699299987], [-74.134809734, 40.56082861799992], [-74.13519379599992, 40.55974841799983], [-74.13523859399992, 40.55958551299993], [-74.13530363499989, 40.55942508899991], [-74.13538846199997, 40.55926934199991], [-74.13549208699995, 40.559120344999876], [-74.13561300299993, 40.55898002699994], [-74.13574932499998, 40.55885004499987], [-74.13660029399993, 40.558388938999926], [-74.13739437899987, 40.558020936999874], [-74.13818000999993, 40.55759386599989], [-74.13885092699991, 40.5569511789999], [-74.13769985100001, 40.55602734599992], [-74.13754790999995, 40.555906577999956], [-74.137431255, 40.55581758299994], [-74.13609776299991, 40.55472110499991], [-74.13498726199998, 40.553810883999844], [-74.13436896499996, 40.55330607999987], [-74.13376761399991, 40.55281510299993], [-74.13321002499998, 40.552383875999936], [-74.13316825999992, 40.552356178999936], [-74.13455868199995, 40.55125507799986], [-74.13528075999987, 40.550863238999895], [-74.13607343399998, 40.550433299999874], [-74.137389882, 40.549592369999885], [-74.13769731599993, 40.54938354599991], [-74.13861798499994, 40.548492384999875], [-74.13861839799995, 40.54845247299989], [-74.13820137199997, 40.54803495999985], [-74.13744805299993, 40.54730532899985], [-74.13700629799997, 40.54696777099985], [-74.13585712583111, 40.546451696357344], [-74.13641153119026, 40.54625235004187], [-74.13649063050026, 40.546073224452435], [-74.13645912181923, 40.545975007476706], [-74.13651613439367, 40.545914748137704], [-74.13658511342636, 40.54592318217117], [-74.13680622121552, 40.545970497524166], [-74.13697129151097, 40.54597498790271], [-74.13706446059668, 40.54595749648294], [-74.13700894772, 40.545894655995845], [-74.13702342911233, 40.54588496435272], [-74.13709627808949, 40.545952218659515], [-74.13713259013673, 40.54593097435956], [-74.13706316637999, 40.545864007992286], [-74.1371798712987, 40.5457928646606], [-74.13715619537318, 40.5457731072957], [-74.1371427044082, 40.545774097764316], [-74.13712777420874, 40.545760371755634], [-74.13707821344494, 40.54571480392378], [-74.13706259040474, 40.54570044027624], [-74.1370106897442, 40.54565272160779], [-74.13699494197701, 40.54563824283913], [-74.13694478446438, 40.545592127167666], [-74.13692680169478, 40.54557559244442], [-74.13687689400858, 40.545529707636824], [-74.13685874691487, 40.5455130226567], [-74.13681049369205, 40.54546865702209], [-74.13679269845123, 40.545452294945505], [-74.13674365036569, 40.54540719896034], [-74.13672532379807, 40.54539034898803], [-74.13667927562506, 40.54534801180821], [-74.13666058054599, 40.54533082184289], [-74.13661515336315, 40.54528905484095], [-74.13658821554525, 40.54526428806667], [-74.13652810778214, 40.545209021823126], [-74.13650122481619, 40.54518430605995], [-74.1364428921272, 40.54513067191817], [-74.13626085280933, 40.544963297435935], [-74.13613448258246, 40.54503946882654], [-74.13611540724054, 40.54501984567978], [-74.1362395396176, 40.544944531201615], [-74.13638379801961, 40.54485467150623], [-74.13640287317463, 40.54487429511146], [-74.13628433241061, 40.54494704058925], [-74.13636510853374, 40.54502381925845], [-74.13643522519202, 40.545088236925906], [-74.13643917174414, 40.54509186194526], [-74.13660795667722, 40.54524692772415], [-74.13661259610625, 40.54525119037059], [-74.13717031824073, 40.5457567488644], [-74.1371728196836, 40.54576586973614], [-74.13719287519706, 40.54578429470008], [-74.13750833508846, 40.545577557238616], [-74.13748941991938, 40.54555909578281], [-74.1374795860376, 40.545564917837936], [-74.13746656856912, 40.545550519294814], [-74.13745005439412, 40.54553225147102], [-74.13742976155727, 40.545509803933946], [-74.1374154359189, 40.54549531581418], [-74.13735634623878, 40.54543555177376], [-74.13734181068475, 40.54542085046247], [-74.13731811323326, 40.545396882453105], [-74.13730652265437, 40.54538515948499], [-74.13724536078695, 40.5453232997171], [-74.13724077148176, 40.54531865707779], [-74.13684436011359, 40.54491771650742], [-74.13684118434955, 40.54491450555556], [-74.13681588562174, 40.54488891782572], [-74.13680670009057, 40.54487962598706], [-74.13674461047367, 40.54481682620692], [-74.1367337016138, 40.544803742023845], [-74.13671451130512, 40.544803653968565], [-74.13661187110705, 40.544862075452095], [-74.13658907071063, 40.544845486095916], [-74.13663388681591, 40.54482118663931], [-74.13683300071517, 40.54470532373605], [-74.13687348630866, 40.54468259391001], [-74.13689095294764, 40.54469944270676], [-74.13674888757022, 40.54478258448059], [-74.13742434617905, 40.5454662480497], [-74.13743335927994, 40.5454612400054], [-74.13741558802171, 40.54544262354305], [-74.13747166103315, 40.54541147405328], [-74.13748786286484, 40.545428445564696], [-74.13750289960466, 40.54542009326926], [-74.1375299444234, 40.54544842357019], [-74.13744723484227, 40.54549436974272], [-74.13750293136816, 40.545552714965865], [-74.13750022510571, 40.545554216261195], [-74.13751799428782, 40.54557179804659], [-74.13752877848904, 40.54556547209198], [-74.13754932972272, 40.54558486589734], [-74.13788456932015, 40.54589401852868], [-74.13794966050746, 40.54600033758068], [-74.13806608226764, 40.545913341314716], [-74.1382249378197, 40.54579463735954], [-74.13829297544638, 40.54570585894163], [-74.13841060885537, 40.54560641421048], [-74.1384876831793, 40.54557219703495], [-74.1385597655297, 40.545532145668496], [-74.13862609986093, 40.54548668020075], [-74.13868599042074, 40.545436277507605], [-74.13873880904883, 40.54538146624923], [-74.13878400176417, 40.54532282132456], [-74.13882109457529, 40.545260957841336], [-74.13884969845212, 40.545196524663865], [-74.13886951340419, 40.545130197607286], [-74.13869488246426, 40.54499107030514], [-74.13874315087111, 40.5449454245212], [-74.13872158725741, 40.54492928692764], [-74.13870784432727, 40.54492845649667], [-74.13868781593632, 40.544914183495905], [-74.13865026247518, 40.544887419662814], [-74.13862752649028, 40.54487121645503], [-74.138585032908, 40.54484093272433], [-74.13856177219148, 40.54482435570691], [-74.13852070494373, 40.54479508898214], [-74.13849651239545, 40.54477784664052], [-74.13845820251454, 40.544750543539486], [-74.1384337707142, 40.54473313243514], [-74.13842126769558, 40.5447242225052], [-74.13836893908898, 40.544686929192665], [-74.13834296955775, 40.544668420924], [-74.1381781226809, 40.54455093817383], [-74.1381604783657, 40.54453836398358], [-74.13812198137776, 40.544510927641255], [-74.13810404192895, 40.54449899057944], [-74.13809284654717, 40.5444998577775], [-74.13804143918792, 40.54454775280511], [-74.13762430745145, 40.54425356641828], [-74.13775620755665, 40.54414492877819], [-74.13777975695223, 40.54416283848299], [-74.13767915494255, 40.54424495880721], [-74.1376791831006, 40.54425862571802], [-74.1377049727256, 40.5442765315792], [-74.13773987886329, 40.544301325392084], [-74.13781048758284, 40.54435147844955], [-74.1378440051603, 40.54437528630685], [-74.13786250173463, 40.544388423261616], [-74.1380279169008, 40.544505916168994], [-74.13804694766422, 40.54450418431619], [-74.13810064050065, 40.544458651895795], [-74.13811742545273, 40.54447334940141], [-74.13817085058372, 40.54451069478396], [-74.13819352235129, 40.54452654182296], [-74.13824880814671, 40.5445651876446], [-74.13827073483874, 40.54458051354138], [-74.13832768600864, 40.544620322156916], [-74.1383481216329, 40.544634606792116], [-74.13840669397884, 40.54467554809705], [-74.13842530203979, 40.544688555325976], [-74.13848461013943, 40.544730011194396], [-74.13850257186327, 40.54474256685877], [-74.13856268411446, 40.54478458394825], [-74.13858052996494, 40.54479705832047], [-74.13864197640305, 40.544840009246634], [-74.13865724010041, 40.54485067790131], [-74.13871819443702, 40.5448932851063], [-74.1387350968491, 40.544905099185094], [-74.13873484040181, 40.54491782006107], [-74.13875534703455, 40.544933958426874], [-74.13901965079306, 40.54471556102991], [-74.1389909520358, 40.544696019702606], [-74.13896956674542, 40.54468187906661], [-74.13891755659161, 40.54464345914525], [-74.13889454402937, 40.54462752102231], [-74.13884074587871, 40.544589193192586], [-74.13881594374128, 40.54457335053199], [-74.13876268535788, 40.544536022838415], [-74.13874002777955, 40.544519037862], [-74.1386884970685, 40.54448093455953], [-74.1385405135774, 40.544376719615116], [-74.13853352313339, 40.54437179780327], [-74.1384801032704, 40.544334177665526], [-74.13845692519634, 40.54431785466995], [-74.13843350741398, 40.544301364093435], [-74.13837770364078, 40.544262065281515], [-74.13835673677613, 40.5442472998096], [-74.13828211671824, 40.54419475020519], [-74.13825826576176, 40.544177953199345], [-74.13818669933056, 40.54412755345665], [-74.13816260914278, 40.54411058868597], [-74.138085611978, 40.544056364166686], [-74.13806166435305, 40.5440394992188], [-74.13804820788745, 40.54403011879695], [-74.138001949898, 40.54399832356188], [-74.13798990164418, 40.54399004378926], [-74.13797997082938, 40.54399834997484], [-74.13789377550259, 40.544070451121904], [-74.13786798524545, 40.54405254413092], [-74.13793218253984, 40.54399840738909], [-74.138043468979, 40.54390456296631], [-74.1380658961542, 40.54391991024747], [-74.1380100182887, 40.54397122691453], [-74.13804967625836, 40.54399826619306], [-74.1380750508894, 40.54401556534103], [-74.13809952422086, 40.544033160923064], [-74.1381081262647, 40.54403934450325], [-74.13903307178627, 40.54470491029999], [-74.13953937915736, 40.54429514087214], [-74.13943253002638, 40.544217217516156], [-74.13942231998007, 40.544225401198446], [-74.13941436189043, 40.544231634439676], [-74.13942091245107, 40.54423610657838], [-74.13937093607659, 40.54427398199373], [-74.13936463648669, 40.54427875687849], [-74.13931996946161, 40.54431260795095], [-74.13931281145453, 40.544318032718806], [-74.1392930274949, 40.544333026379334], [-74.13928593614011, 40.54433840114066], [-74.13924473709938, 40.54436962445413], [-74.13923613063527, 40.544376146118566], [-74.13919652383179, 40.544406164282044], [-74.13918807961167, 40.544412562778945], [-74.13915187461556, 40.544440001529445], [-74.13914525114242, 40.5444450210537], [-74.13910838810352, 40.544472957984866], [-74.1391510849761, 40.544505211598924], [-74.13914541346325, 40.54450958112509], [-74.1391027172494, 40.54447732750823], [-74.13910029753151, 40.54447919136462], [-74.13903265990314, 40.544428097333046], [-74.13904134663564, 40.54442140537625], [-74.13909334926191, 40.544460688869236], [-74.13911264176714, 40.54444566936904], [-74.13922660687233, 40.5443569516712], [-74.13926952681004, 40.544323539170406], [-74.13927673845099, 40.544317925533655], [-74.13932514949839, 40.54428023701224], [-74.13933178866621, 40.544275070367064], [-74.1393777453498, 40.54423929246762], [-74.13938357415316, 40.5442347553762], [-74.13940497441412, 40.544218096330454], [-74.13941007968153, 40.54421412117469], [-74.13942069337877, 40.54420594353718], [-74.13933895032284, 40.54415259794936], [-74.1392876729529, 40.544116589404645], [-74.13897454226765, 40.54437610181971], [-74.13890743048366, 40.54432897553262], [-74.13891817604893, 40.5443200698636], [-74.13895983828961, 40.544349323664896], [-74.13897193186773, 40.54434889623473], [-74.13901051803721, 40.544318789922876], [-74.13915675876315, 40.544198110506606], [-74.1391556516453, 40.54418584042981], [-74.13916396309388, 40.544178548825215], [-74.13920592451963, 40.54414173279324], [-74.13921398233327, 40.544134662799756], [-74.13922249824198, 40.54414055607114], [-74.13924591987264, 40.54412151869098], [-74.13919553562482, 40.54408546512921], [-74.13922032859912, 40.544066700673795], [-74.1392094720652, 40.5440605659809], [-74.13912214559907, 40.54399697181448], [-74.13908266855009, 40.54396822488394], [-74.13905919087385, 40.54398533704194], [-74.13906669135486, 40.54399115683618], [-74.13900724995295, 40.544038766841076], [-74.13896631072416, 40.54407155598066], [-74.13895536858136, 40.544080319659315], [-74.13891491404725, 40.544112721199085], [-74.13890291514562, 40.54412233217856], [-74.13889225032634, 40.54413087320302], [-74.13883900359002, 40.544173520084], [-74.1388390213371, 40.54418206192403], [-74.13889733667764, 40.544225553770296], [-74.13887570388356, 40.54424355793479], [-74.13886309560387, 40.544233859280354], [-74.13881357204203, 40.54419575963726], [-74.13875374875403, 40.54414973668162], [-74.13874816730304, 40.54414544225615], [-74.1387281244596, 40.544130022535], [-74.13874595001212, 40.54411554950892], [-74.13880747279141, 40.54416325744312], [-74.13882839412233, 40.54415924124666], [-74.1390416557895, 40.54397471223546], [-74.13905058643817, 40.54398004831454], [-74.13907750658639, 40.543960153165706], [-74.13895116445453, 40.543877450046175], [-74.13883460585724, 40.54379135939304], [-74.13880325724112, 40.543812683845154], [-74.13881019318308, 40.54382320841886], [-74.13878904068672, 40.54383789237636], [-74.13872880510179, 40.54388665379488], [-74.13872431510785, 40.54389028842588], [-74.13865437747432, 40.543946903446276], [-74.1386286421722, 40.54396773640719], [-74.13854381696055, 40.54403640369566], [-74.1385375337886, 40.544041488613864], [-74.13853867546942, 40.54405173690058], [-74.13859811050895, 40.54409608178442], [-74.1385858115911, 40.54410463849037], [-74.1384415152368, 40.5439977944383], [-74.13841535838714, 40.5439784273163], [-74.13842317799855, 40.54396987653108], [-74.13846102593133, 40.54399777091345], [-74.13849606963167, 40.544023601845794], [-74.13851174337242, 40.54402358294013], [-74.1385228670691, 40.54401455438708], [-74.1385856815562, 40.5439635765561], [-74.13860340384755, 40.54394919380776], [-74.13862979916559, 40.54392777269479], [-74.13868576145339, 40.54388235586905], [-74.13870808505072, 40.54386424000037], [-74.13872636239431, 40.54384940523603], [-74.13874890783579, 40.54383110849366], [-74.13876624485728, 40.54381703795931], [-74.13878541161912, 40.54380148189836], [-74.1387971330161, 40.54380271231233], [-74.13882325461655, 40.5437840548958], [-74.1385399626842, 40.543587928692716], [-74.13852077238342, 40.54360399252532], [-74.13852347275918, 40.54361453211015], [-74.13846115296431, 40.54366427629003], [-74.13844809117643, 40.54367865239242], [-74.13844876639854, 40.54368614654391], [-74.13847431012198, 40.54370514666928], [-74.1384618395175, 40.543715287902266], [-74.13837773022571, 40.54365047218311], [-74.13835645886374, 40.54365220679083], [-74.13829833483882, 40.543700109025494], [-74.1382659207829, 40.5437274814488], [-74.1382066726708, 40.54377367836005], [-74.13820558623709, 40.543789054415555], [-74.13829305440626, 40.54385386621483], [-74.13827964456058, 40.54386669457056], [-74.13815068621103, 40.54377203675709], [-74.13836531286525, 40.54360179969038], [-74.13841801795566, 40.54364102702416], [-74.1384392933049, 40.543641855612044], [-74.13845605241477, 40.54362389912065], [-74.13846595774761, 40.5436291045691], [-74.13852649744571, 40.54357854230606], [-74.13850743752054, 40.54356575359463], [-74.13850743574893, 40.543564898857724], [-74.13844819325458, 40.543613657257005], [-74.13844148726754, 40.54361964466229], [-74.13841121199921, 40.54359832759451], [-74.13848274781854, 40.54353930304897], [-74.1384467162898, 40.543515551682574], [-74.13837491744677, 40.54357651886772], [-74.13834151288823, 40.543551828814046], [-74.13852282835339, 40.54340155175448], [-74.1385559775718, 40.54342415648043], [-74.13845971967868, 40.5435062859916], [-74.13849280865536, 40.54353074955564], [-74.1385900530627, 40.54345034036669], [-74.13862033031373, 40.54347251212417], [-74.13853314542555, 40.54354436734081], [-74.13872959207069, 40.54368233939185], [-74.13881702344845, 40.54374374545133], [-74.1390334816121, 40.54389577085168], [-74.13908167507363, 40.5439296180854], [-74.13910166037837, 40.54394365502056], [-74.13915570620213, 40.543981612355275], [-74.13918578603182, 40.54400273730637], [-74.1392274114774, 40.54403197178986], [-74.13928076576917, 40.54406944406655], [-74.13930377144462, 40.54408560292582], [-74.13931136009417, 40.54409093123115], [-74.13931717941932, 40.5440950189436], [-74.13942349832641, 40.54416968827454], [-74.13945300079233, 40.54419040876872], [-74.13945826107069, 40.54419410245182], [-74.13947061254812, 40.544202777408465], [-74.13953345753629, 40.54424691512262], [-74.13956971698569, 40.544272274830455], [-74.13983697786095, 40.54405246234075], [-74.1389463277329, 40.54340739868262], [-74.13891041386135, 40.54343226458148], [-74.13870632397268, 40.54329141083437], [-74.13890636539442, 40.543131777602284], [-74.13893037818052, 40.54315004000064], [-74.13877140226839, 40.543291332202934], [-74.13886061078239, 40.54336569387329], [-74.1390367192296, 40.54322699394062], [-74.13906073296394, 40.54324525631054], [-74.13894447378158, 40.543339462908676], [-74.13899592102271, 40.54337336940083], [-74.13898355608593, 40.54338441137294], [-74.13902553215638, 40.54341393868391], [-74.1396764048069, 40.54388899585858], [-74.13986496495076, 40.54373682332141], [-74.13988897406256, 40.54375247188492], [-74.13970350694184, 40.543908777186296], [-74.1398652726434, 40.544026844009515], [-74.14004115970624, 40.54388399962383], [-74.13999383559147, 40.54384582593514], [-74.14001283835921, 40.543830426844764], [-74.14005310717077, 40.54385885083263], [-74.14024685174626, 40.54399560363005], [-74.14024719836524, 40.543995850816316], [-74.14031078013868, 40.54405040205576], [-74.14035904817948, 40.54401661756249], [-74.14038573073617, 40.54403318500786], [-74.14041765335297, 40.544002725172966], [-74.14039544536466, 40.543989016695456], [-74.14046054656262, 40.543932017356624], [-74.14030865805942, 40.54382531175345], [-74.14032945189241, 40.54380783807347], [-74.14032848694484, 40.54380208727105], [-74.1403143311892, 40.54379395573368], [-74.14029681353321, 40.54379316796755], [-74.14024817472753, 40.543834286522475], [-74.14023491179564, 40.54382610831666], [-74.14028491492418, 40.5437852598345], [-74.14028458104862, 40.543771625127526], [-74.1402435066554, 40.54374204004763], [-74.14021461687776, 40.54372123189195], [-74.14019945499791, 40.54371031168871], [-74.14014813068958, 40.54367334296595], [-74.14012372242281, 40.543655761980226], [-74.14008209744807, 40.54362578013272], [-74.14005970538267, 40.543609651003294], [-74.14001232514708, 40.54357552424944], [-74.14000281637705, 40.543568674309434], [-74.13994566703632, 40.54352751027062], [-74.13989283556387, 40.54348945633452], [-74.13986402298173, 40.54346870328158], [-74.13982300152821, 40.5434391570209], [-74.13978359729647, 40.54341077311183], [-74.13974757593654, 40.54338482682479], [-74.13970183036301, 40.54335187719821], [-74.1396943683801, 40.54334650202356], [-74.13943438036881, 40.54315923245199], [-74.13934994841898, 40.54309608509422], [-74.13924181634614, 40.54318196207973], [-74.13921993485874, 40.54316608513788], [-74.13923987000207, 40.543150663097634], [-74.13926620242248, 40.54313029048412], [-74.13929760323849, 40.54310599821049], [-74.13940810077588, 40.54301658508783], [-74.13943356778354, 40.54303606758095], [-74.1393844546519, 40.543076869046324], [-74.13938508033534, 40.54308878186187], [-74.13942485990556, 40.54311746030541], [-74.13945177529892, 40.54313686363685], [-74.13947791081843, 40.543155704672564], [-74.13948893219914, 40.543163650141516], [-74.13954690313828, 40.54320544198287], [-74.1395574303407, 40.5432130312066], [-74.13961721033611, 40.54325612738172], [-74.13962652307069, 40.54326284139024], [-74.13968226733104, 40.543303027915776], [-74.13969380758878, 40.54331134716389], [-74.13974850481078, 40.54335077735956], [-74.1397617742255, 40.543360343936634], [-74.1398181529452, 40.543400987483835], [-74.13985313210576, 40.543426203748815], [-74.13988439282308, 40.54344874069938], [-74.13991412345699, 40.54347017330391], [-74.1399522746247, 40.543497676481486], [-74.13996556411055, 40.54350725674794], [-74.14002080798198, 40.54354708252372], [-74.14003465964629, 40.543557067184615], [-74.14022641531263, 40.54369606282558], [-74.14029571947157, 40.543640457155504], [-74.140209375695, 40.54357906228394], [-74.1402138440382, 40.543574786483035], [-74.14012638219599, 40.543513393416866], [-74.14028734539116, 40.54338507111919], [-74.14020997730147, 40.54333220714311], [-74.14019685632545, 40.5433226618805], [-74.14012345022545, 40.54326925674442], [-74.14011092948785, 40.543260147155586], [-74.14003694970397, 40.543206324843815], [-74.14002606731881, 40.5431984072355], [-74.1399503890821, 40.54314334905833], [-74.139939529593, 40.543135449004495], [-74.13986504774401, 40.54308126047014], [-74.1398541018354, 40.5430732956792], [-74.13977902783273, 40.54301867692245], [-74.13976927269164, 40.54301157915037], [-74.13969250376593, 40.5429557266972], [-74.13968218054617, 40.542948216814516], [-74.13960709890746, 40.54289359242594], [-74.13959620562682, 40.54288566658089], [-74.13955102413229, 40.5428527941092], [-74.13951023746552, 40.54282312063042], [-74.13945753440686, 40.54278560209839], [-74.1393871175655, 40.542843770656766], [-74.13936132804027, 40.542825863998594], [-74.1395368154802, 40.54268215086976], [-74.1395637247472, 40.542700056127785], [-74.13947877205688, 40.54276849257767], [-74.13953259730957, 40.54280686447543], [-74.13956044589743, 40.542826992388434], [-74.13961652514804, 40.54286752281435], [-74.13962918175764, 40.54287667015384], [-74.13970048414524, 40.542928201325104], [-74.13987173067858, 40.54305196570492], [-74.13987698012025, 40.543055759732844], [-74.13989432782361, 40.54306829616035], [-74.13996485006797, 40.54311926488114], [-74.13998476182942, 40.543133654739265], [-74.14005110782219, 40.54318160424294], [-74.14014720421119, 40.54325105541862], [-74.14022188806457, 40.54330502941964], [-74.14023345086983, 40.543313386950416], [-74.14030634258127, 40.54336711011437], [-74.14062715566955, 40.543113029692485], [-74.14053648767492, 40.54304727787413], [-74.14052328363455, 40.54303770242722], [-74.14043317229084, 40.54297235493394], [-74.14042171459113, 40.54296404554315], [-74.14035319257147, 40.54291435274512], [-74.14031895744444, 40.54288952590016], [-74.14023059544262, 40.54282544582132], [-74.14022024715962, 40.54281794154593], [-74.14013887318971, 40.54275892852513], [-74.14011692170484, 40.5427430091187], [-74.14005800038075, 40.542700279381044], [-74.1400158651306, 40.542669723334214], [-74.13999673966619, 40.542655852368775], [-74.13975584890447, 40.542481155655224], [-74.13965860486505, 40.5425607111017], [-74.13963169074167, 40.54254109756618], [-74.13985859891669, 40.5423571751291], [-74.13988550839119, 40.54237593505093], [-74.13977484776684, 40.542464048803886], [-74.14064614869828, 40.543094214902105], [-74.14084520303832, 40.54293884497903], [-74.1408723827923, 40.54295994171921], [-74.14104442644127, 40.54308698562637], [-74.14109069897191, 40.54305205535582], [-74.14096661369906, 40.54296326479877], [-74.14098421950682, 40.54294899758735], [-74.14110542112017, 40.54304172058754], [-74.14125673409733, 40.543150146888685], [-74.14132426794235, 40.54309413383019], [-74.14128807915924, 40.543069596322695], [-74.14128331627666, 40.543073695783754], [-74.14038500590046, 40.542420771366494], [-74.14040508359442, 40.54240382006747], [-74.14040897507606, 40.54240650081819], [-74.14043778311083, 40.54238217504695], [-74.14040685334767, 40.542360855450504], [-74.14028673369575, 40.542455789021446], [-74.14016510037457, 40.542366218433486], [-74.14028096291975, 40.54227225901737], [-74.14032683633252, 40.542235057997615], [-74.14033528229751, 40.54222820925949], [-74.14035874187903, 40.542245484948694], [-74.14020723606573, 40.54236522429834], [-74.14024073040927, 40.5423893295242], [-74.14028630244322, 40.54242212653012], [-74.14040656300683, 40.54232708255206], [-74.14048119210243, 40.542382038013756], [-74.14043095386094, 40.5424217411358], [-74.14042719207666, 40.54242466294015], [-74.14120934288349, 40.54298645830763], [-74.14130461980226, 40.54305489177875], [-74.14129718766546, 40.54306128868369], [-74.14133387500644, 40.54308616532608], [-74.14141073560397, 40.543022415080344], [-74.14107892119061, 40.54279117185158], [-74.1411614790294, 40.54271867371482], [-74.14106663614172, 40.5426540467147], [-74.14108436804705, 40.54263891370385], [-74.14117879550508, 40.54270346648169], [-74.14118693183593, 40.54269632170108], [-74.1412951184684, 40.542768770753604], [-74.14132653105813, 40.54274704748731], [-74.14122820814431, 40.54267805353793], [-74.14112375946439, 40.54260461374511], [-74.14114311299068, 40.54258765885143], [-74.14124949366264, 40.54266034085642], [-74.14133547192966, 40.5425887924703], [-74.1413155442272, 40.54257450606381], [-74.14128619376388, 40.54259913025637], [-74.14121958022263, 40.54255555315844], [-74.14125011203603, 40.54253149847261], [-74.14119811780873, 40.542495249074705], [-74.14118653657921, 40.54248710023829], [-74.14112691855341, 40.54244515259515], [-74.14111435667954, 40.54243631322439], [-74.14104399349495, 40.542386804199445], [-74.1410303146288, 40.542377178967136], [-74.14095350746899, 40.542323136224454], [-74.14094279413196, 40.54231559728068], [-74.1408591544129, 40.54225674620028], [-74.14084801466012, 40.54224890805907], [-74.14075996557567, 40.54218695412683], [-74.14074792898349, 40.542178484146234], [-74.14065797552682, 40.54211519189315], [-74.14064549734753, 40.54210641066902], [-74.14056897693308, 40.54205256868892], [-74.1405408621776, 40.54205821787937], [-74.14046378256693, 40.542127442535936], [-74.14041261732501, 40.54209062802874], [-74.14040510349558, 40.54206053639265], [-74.14026290400692, 40.54196151152258], [-74.1402951165258, 40.5419342304149], [-74.14047503718434, 40.542059787772274], [-74.14049581011302, 40.542060855323896], [-74.14053729713807, 40.54202655584857], [-74.14057844981443, 40.541992534030044], [-74.14059741115025, 40.54197685762793], [-74.14062821467097, 40.54195139082894], [-74.1406586485676, 40.5419262303846], [-74.14067998093041, 40.54190859297103], [-74.14070353028875, 40.54192481773058], [-74.14059472343459, 40.54201650761541], [-74.14063417283317, 40.54204226716762], [-74.14061919911953, 40.54205602164392], [-74.1407910422767, 40.54217700008026], [-74.14082098556521, 40.542198080377645], [-74.14089718575627, 40.542251725399495], [-74.14099109268012, 40.54231783482467], [-74.14100375613094, 40.542326749807884], [-74.14101543299061, 40.54233497158853], [-74.14111038150887, 40.54240181358666], [-74.14121924204682, 40.54248021825289], [-74.14134320138834, 40.54256413942904], [-74.14133758497195, 40.54256887136462], [-74.14134901379589, 40.54257752313157], [-74.14146151585437, 40.542483902223864], [-74.14127657503222, 40.54235307924103], [-74.14129442123689, 40.5423358758857], [-74.14148097474445, 40.54246770907046], [-74.14153109439735, 40.5424260008006], [-74.14134854694547, 40.542290464870966], [-74.1413633083767, 40.542278451367494], [-74.14154957905978, 40.54241061750455], [-74.14164911303284, 40.54232152214273], [-74.14179719957052, 40.54220041538235], [-74.14177252532792, 40.542181628520986], [-74.14089451090716, 40.54157111119268], [-74.14088702790175, 40.54158404784442], [-74.14084633142167, 40.54155550672839], [-74.14066095220976, 40.541709346456194], [-74.14063984172124, 40.54169454203723], [-74.14095816832888, 40.54143207796496], [-74.14098048587034, 40.54144768331346], [-74.14089578688294, 40.541516583899515], [-74.140894775834, 40.54153709486054], [-74.14181267772757, 40.54218676935904], [-74.14185192891291, 40.54215177993475], [-74.14187307971923, 40.542166658129425], [-74.1416722624308, 40.542335464095274], [-74.14170523639434, 40.54235532331576], [-74.1417231520644, 40.542359394828416], [-74.14174169529748, 40.54236120068911], [-74.14176037717463, 40.54236069328358], [-74.14181145940867, 40.542362063385895], [-74.1418620809595, 40.542356664752866], [-74.1419107124004, 40.54234466049322], [-74.14195588443337, 40.54232641329101], [-74.14203941343264, 40.54228535739979], [-74.14207881552542, 40.54226177785972], [-74.14210723175339, 40.54224838534302], [-74.14212200021255, 40.54224242945831], [-74.14213710288624, 40.54223698261742], [-74.14215556001572, 40.542239612944506], [-74.14217172891641, 40.5422468940603], [-74.14218366887137, 40.54225795084549], [-74.14221216682478, 40.542286377597726], [-74.14223949692784, 40.54230333583296], [-74.14224801929021, 40.54229976190776], [-74.14225613979161, 40.54229567700988], [-74.14225730330142, 40.54229498433866], [-74.14226380161546, 40.54229110935514], [-74.14227096090445, 40.542286088483095], [-74.14227757028851, 40.5422806444406], [-74.14228358438888, 40.54227481414055], [-74.1422889674892, 40.54226863448397], [-74.1422929736675, 40.54226337258864], [-74.14229632246158, 40.54225784915546], [-74.14229898630232, 40.54225211146236], [-74.14230094334164, 40.542246211973264], [-74.1423021723808, 40.54224019879562], [-74.14230266760137, 40.54223412387123], [-74.1423024207737, 40.54222804115481], [-74.1423014350806, 40.54222200157167], [-74.14229972051788, 40.542216058383964], [-74.14229743316865, 40.54220905208933], [-74.14229423382946, 40.54220225767781], [-74.14229015339535, 40.54219574597618], [-74.14228523416807, 40.542189581766515], [-74.14227952634577, 40.54218382579962], [-74.14227308935794, 40.5421785423336], [-74.14226289372438, 40.54217121479697], [-74.1422532871971, 40.54216343470077], [-74.14224430585027, 40.542155230145184], [-74.142235984228, 40.5421466325834], [-74.14222835050174, 40.54213767146605], [-74.14218927208897, 40.542067782530616], [-74.14218623118083, 40.542059684702195], [-74.14218400576316, 40.542051433746416], [-74.14218260779379, 40.54204307655703], [-74.1421820474815, 40.54203466371547], [-74.14218232602431, 40.542026242296394], [-74.14218344286675, 40.54201786105168], [-74.14218539130584, 40.54200956924344], [-74.14218815870504, 40.542001414298504], [-74.1421894647173, 40.54199225212865], [-74.14219151688192, 40.541983170286805], [-74.14219430868631, 40.54197420446524], [-74.14219782944357, 40.541965389188036], [-74.14220206539267, 40.5419567593187], [-74.14220699969682, 40.541948349053996], [-74.14221261310068, 40.541940191086226], [-74.1422188826151, 40.541932317776904], [-74.14223740154239, 40.541892933611805], [-74.14224204736009, 40.541881258036206], [-74.14224742417294, 40.54186976483291], [-74.14225352084085, 40.54185848216069], [-74.14226032117027, 40.54184743684433], [-74.14226781138109, 40.54183665470042], [-74.14227597153985, 40.5418261593753], [-74.14231438531455, 40.541769485716486], [-74.14231762708064, 40.541755995850174], [-74.14232174995551, 40.54174264544971], [-74.14232674237348, 40.54172946904025], [-74.1423325923287, 40.54171650081275], [-74.14233928539392, 40.54170377194568], [-74.14234350622392, 40.54169857629174], [-74.14234702765678, 40.541693083971026], [-74.14234981467342, 40.541687349809344], [-74.14235183775372, 40.541681432814094], [-74.1423530783546, 40.54167539047113], [-74.14235352145613, 40.54166928495267], [-74.1423531667455, 40.54166317539748], [-74.1423520126031, 40.541657126139064], [-74.14235007650028, 40.54165119478617], [-74.14232447297829, 40.541590346542336], [-74.14232408777632, 40.541579095522096], [-74.14232447686835, 40.5415678390187], [-74.14232564230801, 40.54155661355124], [-74.14232757735208, 40.541545449618695], [-74.14233027811335, 40.541534378051225], [-74.14233373829786, 40.54152343403803], [-74.14233794435431, 40.54151264775116], [-74.14234288668823, 40.54150205103334], [-74.14235181338329, 40.541489273828134], [-74.14236150914962, 40.54147682503301], [-74.14237195252855, 40.541464733824846], [-74.14238312051093, 40.541453023183536], [-74.14239498834318, 40.54144172178718], [-74.14240753323442, 40.541430852112924], [-74.1424166286494, 40.54142636073868], [-74.14242524863116, 40.54142135345751], [-74.14243334426878, 40.54141585814063], [-74.14244086644031, 40.541409906847015], [-74.14244777218299, 40.54140353631952], [-74.14245402116491, 40.54139678128667], [-74.14245957614196, 40.54138968283999], [-74.14246440689692, 40.54138228189439], [-74.1424684827766, 40.54137462087316], [-74.14247178257858, 40.541366745873425], [-74.14247428246301, 40.54135870199038], [-74.14247597330859, 40.541350536814214], [-74.14247692280713, 40.54134495436633], [-74.14247713962867, 40.54133932608661], [-74.14247662235951, 40.541333709774804], [-74.14247537660658, 40.541328160206355], [-74.14247341412687, 40.541322732819], [-74.14247075501662, 40.54131748036001], [-74.14246742486102, 40.541312455401886], [-74.14246345780855, 40.54130770966916], [-74.14245889128624, 40.54130328801346], [-74.14245377216538, 40.54129923577708], [-74.14244654535769, 40.54128972952814], [-74.14244022799775, 40.541279853918994], [-74.14243485400583, 40.54126965698906], [-74.14243045094854, 40.541259193654156], [-74.14242704046049, 40.54124851783215], [-74.1424239201753, 40.541238307216595], [-74.14241999563218, 40.54122825943281], [-74.14241528096458, 40.5412184114872], [-74.14240979315714, 40.54120879870763], [-74.1424035531511, 40.54119945825972], [-74.14239658429314, 40.54119042261506], [-74.14238890927422, 40.54118172558669], [-74.14238055824559, 40.54117339829759], [-74.142371563779, 40.541165474213415], [-74.1423605126076, 40.54115571470418], [-74.14235009040394, 40.54114556038316], [-74.14234032115439, 40.54113503500984], [-74.14233122928577, 40.54112416267861], [-74.14232283461494, 40.54111296866211], [-74.1423151567419, 40.54110147907077], [-74.14230389486428, 40.54108536723826], [-74.14229185500412, 40.54106958573329], [-74.14227905630845, 40.54105415430023], [-74.14226551705349, 40.54103909603549], [-74.14225199022492, 40.541025849258375], [-74.14223905010718, 40.541012265854526], [-74.14222671276296, 40.54099836104978], [-74.14221499316109, 40.54098415191382], [-74.1422093306499, 40.540973323071384], [-74.14220447533503, 40.54096226739818], [-74.14220044485783, 40.54095101971875], [-74.14219724939792, 40.54093961687728], [-74.1421949013318, 40.54092809638529], [-74.1421934073191, 40.54091649207567], [-74.14219638266205, 40.54090494639516], [-74.1422002584716, 40.54089356109865], [-74.1422050205657, 40.54088237942624], [-74.14221065102096, 40.540871440937266], [-74.14221712687538, 40.5408607907256], [-74.14222442361935, 40.54085046886115], [-74.14224031268571, 40.540842086907006], [-74.14226054076296, 40.54082628234095], [-74.14227115484987, 40.540813959235514], [-74.14226793838222, 40.54080754480223], [-74.14226405963667, 40.540801347973726], [-74.14225954394676, 40.54079540607753], [-74.14225441994756, 40.54078976029063], [-74.1422487191152, 40.54078444525272], [-74.14224247995415, 40.54077949626482], [-74.14223574008284, 40.540774944943635], [-74.14222619465933, 40.54076606359246], [-74.14221610188633, 40.540757539255196], [-74.14220549101721, 40.54074939417716], [-74.14219438448305, 40.540741643407955], [-74.14218280538176, 40.54073430601732], [-74.14217078229831, 40.54072739956056], [-74.14215834381515, 40.540720940587555], [-74.14213673870715, 40.5407114723906], [-74.14211552890573, 40.54070150094198], [-74.14209473837126, 40.54069103760447], [-74.14207439083921, 40.540680091227905], [-74.14206665867121, 40.54067483347683], [-74.14205946666839, 40.54066914299637], [-74.14205285596474, 40.54066305223675], [-74.14204686264567, 40.54065659516185], [-74.14204152478366, 40.54064981042436], [-74.14203687057113, 40.54064273786163], [-74.14203292556499, 40.54063541681177], [-74.14202971532565, 40.54062788845556], [-74.14202725883361, 40.54062019783535], [-74.14202556716027, 40.54061238816031], [-74.14202465094452, 40.54060450565539], [-74.14202501079866, 40.54059806970366], [-74.14202611672216, 40.54059168225033], [-74.14202796289642, 40.54058539540478], [-74.1420305338337, 40.54057925810483], [-74.14203380943752, 40.540573320299174], [-74.14203776301365, 40.54056762758931], [-74.14204236545184, 40.54056222507666], [-74.14204757731929, 40.54055715720532], [-74.14205331996206, 40.54055364553563], [-74.14205850822262, 40.540549662953964], [-74.14206307962891, 40.54054526230973], [-74.14206697390732, 40.54054049795718], [-74.14207014221716, 40.54053543043541], [-74.14207254362798, 40.54053012278637], [-74.14207415018065, 40.54052464490504], [-74.14207493786425, 40.54051906483825], [-74.14207489936211, 40.54051345279029], [-74.14206740568822, 40.54049920557377], [-74.142059238979, 40.540485174800864], [-74.14205041288535, 40.54047137720771], [-74.1420409375547, 40.540457834392804], [-74.14203082663339, 40.540444560914196], [-74.14202009267665, 40.54043157468216], [-74.14200874977743, 40.54041889393968], [-74.14200146000344, 40.540410890318874], [-74.1419933905225, 40.54040333446493], [-74.14198458886533, 40.540396272557366], [-74.14197510562117, 40.54038974340041], [-74.14196499796977, 40.540383786963005], [-74.14195309963941, 40.54038088296165], [-74.14194144744997, 40.54037744976184], [-74.14193008007078, 40.540373497032526], [-74.14191903816062, 40.5403690404714], [-74.14190836281274, 40.5403640937654], [-74.14190517802098, 40.540356230481144], [-74.14190281657854, 40.54034820166461], [-74.14190129022518, 40.54034005521493], [-74.14190060873084, 40.54033184188166], [-74.14190077967079, 40.54032361325498], [-74.14190179788412, 40.54031542094036], [-74.14190366018238, 40.540307314865686], [-74.14190635459853, 40.540299347147865], [-74.14190870883773, 40.54028909289499], [-74.14191034043938, 40.54027875828301], [-74.1419112488051, 40.540268371457834], [-74.14191142741133, 40.54025796275039], [-74.1419108785163, 40.54024756114078], [-74.14190960174705, 40.54023719778986], [-74.14190760177618, 40.540226901172346], [-74.14190488240092, 40.54021670143855], [-74.14189926617172, 40.54020517308427], [-74.14189276029074, 40.54019391856842], [-74.14188538527415, 40.540182981925966], [-74.14187716996324, 40.540172398134835], [-74.14186814079488, 40.540162207369335], [-74.14185605092108, 40.54014911758806], [-74.14184322869355, 40.54013643580874], [-74.14182970578898, 40.54012418812668], [-74.14181550751331, 40.540112399137385], [-74.14180974566287, 40.54011160746946], [-74.1418041589966, 40.54011027379219], [-74.14179883385656, 40.540108417767385], [-74.14179385529062, 40.54010606994809], [-74.14178930023023, 40.540103264918216], [-74.14178523794598, 40.54010004916562], [-74.14178173397713, 40.54009646985285], [-74.14177884532401, 40.540092586215465], [-74.14177661383266, 40.54008845633471], [-74.1417750741191, 40.540084146174806], [-74.14177425311996, 40.540079724725175], [-74.14177416085599, 40.540075257477426], [-74.14177458284607, 40.540070670313284], [-74.14177573470064, 40.540066155793575], [-74.14177759987497, 40.54006178212378], [-74.14178014842196, 40.54005761400742], [-74.14178334489904, 40.54005371347423], [-74.14178713892304, 40.540050138887175], [-74.14179147614927, 40.54004694409036], [-74.14179629277618, 40.54004417640602], [-74.1418015148865, 40.540041876300116], [-74.14180706394352, 40.54004008039146], [-74.14181285874656, 40.540038811731684], [-74.14181881523446, 40.540038090695916], [-74.14182484120643, 40.540037930632785], [-74.14183084684538, 40.540038329977904], [-74.1418367449673, 40.54003928615861], [-74.14184244637886, 40.54004078102421], [-74.14184957452491, 40.540037118886026], [-74.14185622819136, 40.54003296931812], [-74.14186234838104, 40.54002836690455], [-74.14186788489322, 40.540023352417215], [-74.14187278885099, 40.540017969307], [-74.14187701708526, 40.54001226034762], [-74.14188053676908, 40.54000627835169], [-74.14188331243463, 40.54000007328703], [-74.14188532465153, 40.539993698452484], [-74.14188655355558, 40.53998720982752], [-74.14188699003844, 40.53998066120081], [-74.14188663005051, 40.539974110375205], [-74.14188667786583, 40.53996343076526], [-74.14188595801129, 40.539952764836286], [-74.14188446901966, 40.5399421452584], [-74.14188221952347, 40.53993160418676], [-74.1418792126678, 40.53992117478839], [-74.14187546059314, 40.53991088636612], [-74.14187097346594, 40.53990076956517], [-74.14186576585055, 40.53989085787367], [-74.14185898274587, 40.53987747154069], [-74.1418528874464, 40.539863895551036], [-74.14184748745379, 40.539850146815866], [-74.14184279226167, 40.53983624961569], [-74.14183880849806, 40.53982222304075], [-74.14183554323496, 40.53980808852637], [-74.1418329998135, 40.539793868517265], [-74.14183237875211, 40.539785773217496], [-74.14183251854898, 40.539777663240315], [-74.14183342170934, 40.53976958080038], [-74.14183508130014, 40.53976157046924], [-74.14183749015511, 40.53975367011747], [-74.14184063475152, 40.539745923152125], [-74.14184449781987, 40.53973836678615], [-74.14184906363768, 40.539731042418914], [-74.1418543050504, 40.539723985600325], [-74.14186019556792, 40.53971723439248], [-74.1418667047398, 40.53971082317631], [-74.14187380123579, 40.539704785664156], [-74.14188144756507, 40.53969914987982], [-74.14188967926111, 40.53969410358076], [-74.14195889835926, 40.53967817719808], [-74.14198428949713, 40.53966199086007], [-74.14200666356471, 40.53962065642814], [-74.14200459282473, 40.539567959164366], [-74.14198181354386, 40.53953597330813], [-74.14192932905125, 40.53949978364434], [-74.14187458672595, 40.53946264148896], [-74.14182589311272, 40.53942218646411], [-74.14176721342444, 40.53938732786741], [-74.1417415875668, 40.53936550367508], [-74.14172892651293, 40.539339484096985], [-74.14173698496194, 40.539309712734386], [-74.14176433199539, 40.539275656425225], [-74.1418878357748, 40.53920974740635], [-74.14203045817284, 40.539175233398495], [-74.14209894139935, 40.53915770591602], [-74.1421719510025, 40.539154003918846], [-74.14221934705722, 40.53913924946704], [-74.14227296511635, 40.53910289983899], [-74.1423035127584, 40.53906997528961], [-74.14223700269802, 40.539030654248776], [-74.14225638639385, 40.539012914076714], [-74.14233831577496, 40.538949190408296], [-74.1421518306091, 40.538816553186884], [-74.14176587793762, 40.53854204099971], [-74.14226513007988, 40.53814035366466], [-74.14224928389066, 40.53812968335866], [-74.14220824925785, 40.53816462035977], [-74.14221322317795, 40.53816752905834], [-74.14186762291457, 40.538444090719196], [-74.14183789388267, 40.53842162042672], [-74.14220052571474, 40.53812644173447], [-74.14220665211691, 40.53813197269759], [-74.14222992294864, 40.538114453879814], [-74.14219794712481, 40.53809105549753], [-74.14207893644108, 40.5380039642851], [-74.1421671801914, 40.53793276106813], [-74.14214752525183, 40.53791658053479], [-74.14213937737509, 40.53792148518653], [-74.14121865335552, 40.53725399843248], [-74.1411977050377, 40.5372540241963], [-74.14106736882466, 40.537365643645124], [-74.14104157916547, 40.53734688546733], [-74.1412032253677, 40.53721597312261], [-74.14129825471821, 40.53713817794829], [-74.14132073591662, 40.53715352068053], [-74.14123742490064, 40.5372156176087], [-74.14124113051585, 40.53723469173408], [-74.14217576939241, 40.53789459909736], [-74.14216240779166, 40.53790214964632], [-74.14218574841878, 40.53791777829155], [-74.14238963689803, 40.53775326100733], [-74.14158287690883, 40.53708757795238], [-74.1417341057238, 40.537039511377365], [-74.14305162613793, 40.5372398738564], [-74.14348622872835, 40.53724188202979], [-74.14354145739786, 40.53724231720504], [-74.14359630012335, 40.537237325528444], [-74.14364984461051, 40.537226990035535], [-74.14370120016159, 40.53721148265441], [-74.14374951249268, 40.53719106134627], [-74.14379397794472, 40.53716606581355], [-74.14383385685329, 40.53713691184967], [-74.14391512656537, 40.53707560486414], [-74.14399097095605, 40.53701037676944], [-74.14406106575686, 40.536941506458405], [-74.14412511128542, 40.53686928839643], [-74.14418283372648, 40.53679403136107], [-74.14423398630267, 40.536716057123], [-74.14427835032906, 40.536635699069066], [-74.14428108600727, 40.5365674030764], [-74.14429650123257, 40.536528092654315], [-74.14437399056834, 40.536394870634886], [-74.14438874490745, 40.53635837336202], [-74.14455507323936, 40.535946931272015], [-74.14474069813576, 40.53580104421121], [-74.14478850976617, 40.53579095170865], [-74.14488929110058, 40.535802758424886], [-74.14506861008147, 40.53583966609008], [-74.14536324652758, 40.53585635689648], [-74.14555172527938, 40.53576489177913], [-74.14573534090819, 40.535667839442446], [-74.14581865021563, 40.53562104045176], [-74.14585411335841, 40.535595211228696], [-74.14588438604362, 40.53556578737187], [-74.14590884942612, 40.53553337038119], [-74.14592700342014, 40.535498622943486], [-74.14593847692211, 40.535462255384594], [-74.14594303539546, 40.535425011148924], [-74.14598880766482, 40.535381588737415], [-74.14625512927707, 40.535538684733424], [-74.14669647685385, 40.53541430854866], [-74.14701525183798, 40.53532447244462], [-74.14794320046134, 40.534923016992956], [-74.14851051812346, 40.534610813656755], [-74.14870158909109, 40.5345110030592], [-74.14921733263965, 40.53424158776087], [-74.14977717489465, 40.533887869806705], [-74.14977824913616, 40.533826181885516], [-74.14971830290298, 40.533491910022384], [-74.14975846488892, 40.533424302751406], [-74.14980633822482, 40.533359684349335], [-74.14986154448287, 40.5332985655814], [-74.14992364727622, 40.53324142954886], [-74.1499921557097, 40.533188727871206], [-74.15001653727084, 40.53317059720704], [-74.15012879899228, 40.53308711784299], [-74.1503004530457, 40.532922721865944], [-74.1502625151295, 40.53290285183018], [-74.15024247803049, 40.53291819300964], [-74.15021506106473, 40.53289734521976], [-74.15026788636534, 40.53285690093636], [-74.15029347623548, 40.532876358928156], [-74.15027891302235, 40.53289169244555], [-74.150315444394, 40.53290835175499], [-74.15048977709482, 40.53274929017209], [-74.15069553618274, 40.53254583893711], [-74.1510243472269, 40.53270667479153], [-74.15110392281035, 40.53274222306205], [-74.1511202569052, 40.53274138250669], [-74.15125007703935, 40.53268303076502], [-74.15145035861134, 40.53259535089076], [-74.15175798831623, 40.53240179402802], [-74.15176169985676, 40.53239766893707], [-74.1517644226307, 40.53239311699991], [-74.15176607492447, 40.53238827482501], [-74.15176660715161, 40.53238328773113], [-74.15176600333938, 40.53237830538616], [-74.15176428160906, 40.53237347731539], [-74.15176149363195, 40.532368948414266], [-74.1517577230782, 40.53236485459965], [-74.1517336092612, 40.53234535494257], [-74.1517102169699, 40.53232644001321], [-74.1517023907152, 40.53232228670496], [-74.15169553530163, 40.53231723221056], [-74.15168982705744, 40.53231140653698], [-74.15168541280488, 40.53230495952704], [-74.15168240608293, 40.53229805700479], [-74.1516808842272, 40.53229087651052], [-74.15168088638092, 40.53228360273435], [-74.15168241248837, 40.532276422765534], [-74.15168542329587, 40.5322695212804], [-74.1516898413618, 40.5322630757924], [-74.15169555304871, 40.53225725208631], [-74.15177191995262, 40.532208128107825], [-74.15185337744485, 40.532164021023306], [-74.15188502884477, 40.532156061849584], [-74.15191797942113, 40.5321522556412], [-74.15195130015276, 40.532152709711916], [-74.15198405158232, 40.532157411259384], [-74.15201531030324, 40.53216622772631], [-74.15204419499408, 40.532178910538015], [-74.15207329797543, 40.532178770152726], [-74.15210198442624, 40.53217502407327], [-74.15212949213605, 40.53216777183452], [-74.15215509021469, 40.532157206131274], [-74.15217809851231, 40.532143607698174], [-74.15218061639548, 40.53214220447677], [-74.1522249360828, 40.53212160703056], [-74.15227294935504, 40.53210658334698], [-74.15232346155292, 40.53209750724252], [-74.1523519720649, 40.532096098677826], [-74.15238852526379, 40.53209063761969], [-74.15242365238952, 40.53208118599126], [-74.15245660936695, 40.53206794400092], [-74.15277081604118, 40.5318130597192], [-74.15355245128475, 40.53117898506993], [-74.15365382450162, 40.531110493514326], [-74.15373066693097, 40.5310539419431], [-74.15383974058712, 40.53095675485172], [-74.15399336904485, 40.53082838075021], [-74.15413110791108, 40.530722911256305], [-74.15413681963841, 40.5307053971625], [-74.15412319527393, 40.53069081517855], [-74.15405094663906, 40.530624288822736], [-74.1540409411867, 40.53060724815499], [-74.15404749949809, 40.53059456484163], [-74.15406027841539, 40.530584815684534], [-74.15408340297849, 40.53058624166225], [-74.15420772346785, 40.53065889635527], [-74.15421467768634, 40.53066536709029], [-74.15422643744479, 40.530658595126454], [-74.15424379688706, 40.53064276380237], [-74.15430582451033, 40.53059216483583], [-74.15436818968881, 40.530545447061215], [-74.15438294913153, 40.53052909886251], [-74.15436288012884, 40.53050899371879], [-74.15434789896189, 40.53049350557085], [-74.1543450727081, 40.530480649903524], [-74.15453545404685, 40.530304909365576], [-74.15457237074557, 40.53032285745904], [-74.15460770600045, 40.530337311310745], [-74.15463043409935, 40.5303390818195], [-74.15464111999897, 40.530334638424286], [-74.15465480504844, 40.530315743834805], [-74.15469794899194, 40.53024888010371], [-74.15449652366432, 40.530102166861184], [-74.15454357374453, 40.53006213630362], [-74.15492894849395, 40.530225483575634], [-74.15504649891339, 40.53015381451283], [-74.15515949728098, 40.530078002615234], [-74.15526769306129, 40.52999821598076], [-74.15537084637016, 40.52991463151942], [-74.1554687285071, 40.529827434561604], [-74.1555611224612, 40.529736818447056], [-74.15564782339318, 40.52964298409564], [-74.1554708552024, 40.529477505470844], [-74.1555035725126, 40.52943802633765], [-74.15573153435155, 40.529600905035004], [-74.15586370784149, 40.52951269380084], [-74.15599000320377, 40.52941960371781], [-74.15611011148543, 40.52932186252861], [-74.15622373887356, 40.52921970935301], [-74.1563306074147, 40.52911339410393], [-74.15643045569396, 40.5290031768752], [-74.1564581409916, 40.52897042748831], [-74.15608512824117, 40.528639386976074], [-74.15616811130214, 40.52859614860073], [-74.15666771048716, 40.528992124356016], [-74.15676917912974, 40.529023163252845], [-74.15701861946262, 40.528977655809626], [-74.15720948462648, 40.52884150512297], [-74.15760975710916, 40.52865910433], [-74.15789753570577, 40.528528814141225], [-74.15826103107307, 40.52830049916295], [-74.15794886622446, 40.528085103953735], [-74.15797239457906, 40.52806708584441], [-74.15832645126979, 40.52827842678315], [-74.15848687127477, 40.528197229412555], [-74.15864056291043, 40.528117072097295], [-74.15867696304029, 40.5280980880003], [-74.15890180648944, 40.5279359095688], [-74.15917092406275, 40.527669751506764], [-74.15928316159182, 40.527511724834895], [-74.15939460542255, 40.527427752144376], [-74.15970134520207, 40.52729599283152], [-74.15988277075512, 40.52736935827443], [-74.16005732699996, 40.527635041999886], [-74.16042481199997, 40.528118481999854], [-74.16085825899998, 40.528718545999894], [-74.16134961699994, 40.5293734239999], [-74.16122606799999, 40.5294244229999], [-74.16037397499991, 40.529783172999906], [-74.15851066999988, 40.53056568999989], [-74.15730358599997, 40.53107601399994], [-74.15716510199995, 40.53112831799989], [-74.15703269699993, 40.53119055399988], [-74.15690813799995, 40.531262027999915], [-74.15679305199991, 40.53134182899985], [-74.1566887699999, 40.53142883599987], [-74.15602014099989, 40.53211509399988], [-74.15553157299992, 40.5326308329999], [-74.15502905699995, 40.533160203999884], [-74.15447193999992, 40.53374918899989], [-74.15438619599998, 40.53383715099992], [-74.15415844299997, 40.53407479199993], [-74.15397419799994, 40.53426814799993], [-74.15404693799991, 40.534308456999916], [-74.15593326499993, 40.53535375699991], [-74.15704288199998, 40.535967071999906], [-74.15721860099994, 40.53607598699994], [-74.15738901899991, 40.53618969999996], [-74.15755391199993, 40.536308060999936], [-74.15771305999992, 40.53643091299995], [-74.15786625499994, 40.53655809499985], [-74.15801329499999, 40.53668943899986], [-74.15815398399998, 40.536824769999896], [-74.15828813799993, 40.53696391199989], [-74.1584155779999, 40.537106678999876], [-74.15853613800002, 40.537252883999905], [-74.15914106399993, 40.53789864499992], [-74.15955749199995, 40.53833428299992], [-74.16102399199994, 40.539897758999885], [-74.16252750299994, 40.5415027069999], [-74.16359221499997, 40.54264214599987], [-74.16423140999989, 40.54330115899992], [-74.16478748999995, 40.543802109999916], [-74.16526055199988, 40.54423960899985], [-74.16540080799992, 40.54443272299989], [-74.16548821099998, 40.544652541999874], [-74.16574238499992, 40.545397704999914], [-74.16619699799993, 40.54711041099993], [-74.16623327399996, 40.54734247899988], [-74.16683255499997, 40.54777899599992], [-74.1668879359999, 40.54783668799997], [-74.16693751599995, 40.54789737399989], [-74.16698101599997, 40.547960714999846], [-74.16701819499993, 40.54802635699987], [-74.16704884399992, 40.548093932999905], [-74.16707279199991, 40.54816306699997], [-74.16708990499995, 40.54823337199991], [-74.16710008899994, 40.548304454999865], [-74.167103286, 40.54837591999994], [-74.16709947799988, 40.54844736699987], [-74.1672586319999, 40.54916666599995], [-74.16741722299999, 40.54988134599994], [-74.1675722339999, 40.550574733999916], [-74.16773525199996, 40.551291459999916], [-74.16776724799992, 40.551438072999865], [-74.16786573399993, 40.55188942699987], [-74.16789463099991, 40.55200100599988], [-74.16805222599993, 40.55271771199992], [-74.16810265799997, 40.55294619199988], [-74.16820548500002, 40.55343190299991], [-74.16836706799991, 40.554131592999944], [-74.16852396199995, 40.55485193799988], [-74.16868441299991, 40.555567584999864], [-74.16884272899996, 40.55626910699991], [-74.16900167899992, 40.55697216799992], [-74.1691669539999, 40.55770084699984], [-74.16931680999987, 40.55840978199988], [-74.16947719799997, 40.559126671999856], [-74.16973650699998, 40.560250200999896], [-74.16977469899997, 40.56038818799986], [-74.16980364499997, 40.560527475999876], [-74.16982326899996, 40.56066770699989], [-74.16983351999993, 40.56080851799988], [-74.16983437299987, 40.560949543999875], [-74.16982582400001, 40.56109042099991], [-74.16688971999996, 40.560476604999955], [-74.16589404599996, 40.560283361999915], [-74.1649414939999, 40.56011179699991], [-74.16377941599991, 40.56031697999986], [-74.16272123599997, 40.56077419499996], [-74.16193293199993, 40.56111113299994], [-74.1615313009999, 40.56126935399991], [-74.16096801899997, 40.56146020199987], [-74.1600564059999, 40.56170549999991], [-74.15915791699992, 40.56193583399991], [-74.15825784199991, 40.56220197699982], [-74.15741136799991, 40.5624985659999], [-74.15725989499994, 40.56255326399991], [-74.15725974199995, 40.562553318999896], [-74.15725961299992, 40.562553382999916], [-74.15711551399995, 40.562620641999935], [-74.15711540799994, 40.562620686999885], [-74.15711531399992, 40.562620740999876], [-74.15698121999995, 40.56269959399989], [-74.15698100799993, 40.562699720999944], [-74.15698080799989, 40.56269986499994], [-74.15685893499987, 40.562789021999926], [-74.15685888899992, 40.56278905699991], [-74.1568588409999, 40.562789093999896], [-74.15675091099989, 40.56288697499989], [-74.15585090399995, 40.56377444399988], [-74.15511310499993, 40.56494054499992], [-74.15502927899996, 40.565093413999904], [-74.15492589699993, 40.56524125699989], [-74.15480382999996, 40.56538193999996], [-74.154664477, 40.5655134429999], [-74.15450973599995, 40.5656339799999], [-74.15434186999994, 40.565742096999884], [-74.15416347399992, 40.56583667599993], [-74.15357985599994, 40.56625318599991], [-74.15295627699996, 40.56677601699993], [-74.15228114599992, 40.567392946999846], [-74.15171497199997, 40.56788728399988], [-74.15147304199998, 40.56805205899985], [-74.15123561999997, 40.568220604999894], [-74.1510028049999, 40.56839284899993], [-74.1507746989999, 40.56856871799991], [-74.15055139699999, 40.56874813699989], [-74.15033299399995, 40.56893102999989], [-74.14983771799996, 40.56935826299985], [-74.14789716899996, 40.57088728599995], [-74.14720574299992, 40.57053278099995], [-74.14713603399994, 40.57051798899991], [-74.14706472699994, 40.5705086149999]]]}}, {\"id\": \"109\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 110, \"Shape_Leng\": 0.103946292913, \"Shape_Area\": 0.000525745098785, \"zone\": \"Great Kills Park\", \"LocationID\": 110, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.11813477599993, 40.550592692999906], [-74.11656290099995, 40.54912774199988], [-74.11495831999993, 40.54958143899991], [-74.11458035199992, 40.54927646599987], [-74.11444772299996, 40.549169449999944], [-74.11395149899992, 40.548725322999935], [-74.11338340608127, 40.54807337507532], [-74.11340791195607, 40.54806970918334], [-74.11343842392581, 40.54806514350483], [-74.11374841507876, 40.54801875805332], [-74.11387354532663, 40.54809147694596], [-74.11473781165274, 40.54799661067656], [-74.11497175661175, 40.54789475160998], [-74.11523058951686, 40.54759949785568], [-74.11526784211863, 40.5475710010925], [-74.11531090308861, 40.547547749985135], [-74.11535851886553, 40.54753042140077], [-74.11540930329542, 40.54751951979494], [-74.11546177798424, 40.547515362525886], [-74.11551441533344, 40.54751807061639], [-74.11556568301016, 40.547527565230894], [-74.11561408855269, 40.54754356997034], [-74.11565280430547, 40.54759750456617], [-74.11569815233663, 40.54764834267649], [-74.11574971068985, 40.547695611244805], [-74.11580699961861, 40.54773887042871], [-74.11586948604845, 40.5477777176931], [-74.11593658853862, 40.54781179155582], [-74.1160076826915, 40.547840774951965], [-74.11608004748837, 40.547864946839105], [-74.11615549887546, 40.54788280164156], [-74.11623309254398, 40.54789411589802], [-74.11631185737198, 40.5478987480049], [-74.11639080757848, 40.54789663998902], [-74.11646895506243, 40.54788781823345], [-74.11654532176979, 40.5478723931466], [-74.11676361901095, 40.54775203154288], [-74.11697706667302, 40.547626711233896], [-74.1170796492492, 40.54756682395667], [-74.11718742988651, 40.54751250840691], [-74.11729988946024, 40.547464026189054], [-74.11741648631386, 40.54742161081151], [-74.11753665886786, 40.54738546656196], [-74.11765982832337, 40.547355767523975], [-74.11778540145028, 40.54733265673844], [-74.11791277344408, 40.547316245514594], [-74.11804133083707, 40.54730661289452], [-74.11817087783834, 40.54721002129322], [-74.1182950048036, 40.54710937346596], [-74.11841349369186, 40.54700484623081], [-74.11852613636927, 40.54689662322071], [-74.11863273497548, 40.54678489455996], [-74.11873310227173, 40.54666985653102], [-74.11882706196816, 40.54655171122869], [-74.1189144490349, 40.54643066620581], [-74.11899287798937, 40.54630571439796], [-74.11907706963265, 40.54618296423323], [-74.11916691789598, 40.54606257031249], [-74.11926230958805, 40.545944684267816], [-74.11934442053138, 40.54585017518081], [-74.11943009461557, 40.54575752500887], [-74.11948690777857, 40.54569180329381], [-74.11955056043125, 40.54562983659857], [-74.11962062815836, 40.54557203807892], [-74.11969664377663, 40.545518793098964], [-74.11977810044974, 40.54547045666164], [-74.11986445506753, 40.545427351042214], [-74.11995513186709, 40.54538976363983], [-74.12004952627139, 40.54535794506144], [-74.12014700891991, 40.54533210745096], [-74.1202469298642, 40.54531242307533], [-74.12034862290126, 40.54529902317587], [-74.12040531184229, 40.54528346247968], [-74.12046386191147, 40.54527263089338], [-74.12052359731865, 40.54526665343616], [-74.12059038294942, 40.54526578281469], [-74.12065683051817, 40.54527097601801], [-74.12078965312625, 40.545283164296634], [-74.12092326347042, 40.545288304336445], [-74.12105701960938, 40.545286371441826], [-74.12119027890229, 40.545277374899406], [-74.12132240109477, 40.54526135793375], [-74.12145275139761, 40.5452383974994], [-74.12158070353595, 40.545208603911554], [-74.1217056427596, 40.54517212031503], [-74.12182696879638, 40.545129121997306], [-74.12194409873757, 40.545079815545584], [-74.12205646983776, 40.545024437854735], [-74.12216354221981, 40.54496325498782], [-74.12245306050386, 40.544770545629824], [-74.12424449712773, 40.54311038091073], [-74.12575094543642, 40.541305425450304], [-74.1263818513107, 40.540520755459255], [-74.12762434074298, 40.53871795356038], [-74.1307118427266, 40.53453507543623], [-74.1335644349803, 40.5315111748056], [-74.13527548623271, 40.53028572843679], [-74.13618848993339, 40.52962079668727], [-74.1369079907634, 40.529281918882695], [-74.13728504251857, 40.529740467984816], [-74.13743797279717, 40.52967586404655], [-74.13793376881654, 40.52966815993488], [-74.13804028983962, 40.529670019251085], [-74.13814635487623, 40.5296777584785], [-74.13825140719548, 40.52969133699387], [-74.13835489538205, 40.52971068352444], [-74.13845627622914, 40.529735696521115], [-74.13855501758961, 40.529766244691295], [-74.13865060116915, 40.52980216768873], [-74.13874252524631, 40.529843276954324], [-74.13883030715276, 40.529889356352996], [-74.13888337295603, 40.52996067648437], [-74.13892891917789, 40.53003496123281], [-74.13896666349659, 40.530111750283844], [-74.1389963718173, 40.53019056743806], [-74.1390178598777, 40.530270923917264], [-74.13903099439075, 40.53035232139547], [-74.1390356938731, 40.53043425508947], [-74.1390319291496, 40.53051621688854], [-74.13901972353628, 40.53059769850599], [-74.13899915269602, 40.53067819463105], [-74.13899666667372, 40.53080021004376], [-74.13900228727267, 40.53092216481683], [-74.13901600009501, 40.53104374558269], [-74.13903776994901, 40.53116463993302], [-74.13906754093875, 40.53128453722161], [-74.13910523660833, 40.53140312936247], [-74.13915076013691, 40.531520111622044], [-74.13920399458759, 40.53163518340139], [-74.14027015269588, 40.53330001719592], [-74.1406109270312, 40.53413060019013], [-74.1406238382893, 40.534244513705545], [-74.14062780663788, 40.534358812277375], [-74.14062281785364, 40.53447308757021], [-74.14060888971382, 40.534586931329706], [-74.1405860719319, 40.534699936840994], [-74.14055444598172, 40.534811700381994], [-74.14051412480651, 40.534921822665545], [-74.14049228277464, 40.53499866806264], [-74.14046219147538, 40.535073873307795], [-74.14042405676581, 40.535146923847456], [-74.14037813953776, 40.535217319869474], [-74.14032475393472, 40.5352845797219], [-74.14026426520137, 40.535348243209405], [-74.14019708718655, 40.53540787474189], [-74.14015206197386, 40.53542706714786], [-74.14010422764461, 40.535441781195594], [-74.14005434565807, 40.53545178265631], [-74.14000321007146, 40.53545691231959], [-74.13995163489804, 40.53545708852768], [-74.13948154212022, 40.53536354307274], [-74.13927563086473, 40.53532288029777], [-74.13906788388647, 40.53528807515453], [-74.13885858732235, 40.53525917558082], [-74.13864802944258, 40.535236221380764], [-74.138436500253, 40.5352192441696], [-74.13822429109703, 40.53520826733011], [-74.13801169425406, 40.535203305981184], [-74.1350750932569, 40.535241170622974], [-74.13490721847988, 40.535270096020255], [-74.1347411038744, 40.53530443123399], [-74.13457704876932, 40.53534411439487], [-74.13441534878356, 40.53538907399659], [-74.13425629529412, 40.53543922902523], [-74.13410017491086, 40.53549448910474], [-74.13395313244355, 40.535566460988804], [-74.13381005153853, 40.53564293988989], [-74.13367116937623, 40.53572379903906], [-74.13353671617952, 40.53580890440635], [-74.13340691483263, 40.5358981149225], [-74.13327383658317, 40.53600815233875], [-74.13314683122164, 40.53612229116586], [-74.1330261157118, 40.536240336448444], [-74.13320696251593, 40.536290635640874], [-74.1332156758573, 40.53627869385781], [-74.13338312919123, 40.5363244213335], [-74.13336905396339, 40.536356195225366], [-74.13320656709256, 40.536311368808654], [-74.13319580101377, 40.53630674205062], [-74.13301207668793, 40.536256065042814], [-74.1329712040364, 40.53629468857738], [-74.1331723597528, 40.53635519822346], [-74.13316120141164, 40.53637400270103], [-74.13309062611808, 40.5363527309569], [-74.13307585702779, 40.53639748804031], [-74.13294266453627, 40.53632203308197], [-74.13119545689379, 40.53787240877454], [-74.13128337090322, 40.537933490833574], [-74.13133279192023, 40.537891329942426], [-74.13132101349746, 40.53788461808034], [-74.13151307747667, 40.53771070573342], [-74.13152564883184, 40.53771854263834], [-74.13135412336848, 40.53787151485735], [-74.13135956321447, 40.53787521524048], [-74.13137656776155, 40.53788749200204], [-74.13129269799776, 40.53795670187077], [-74.13136153935845, 40.538001720428944], [-74.13173356771327, 40.53825184035868], [-74.13181387201769, 40.538307358685685], [-74.13211244495555, 40.538052296677684], [-74.13202912079947, 40.53799648064422], [-74.1320160616874, 40.537987672177294], [-74.13194001141805, 40.5379363722046], [-74.13192776126479, 40.53792810861168], [-74.13185028741444, 40.53787584892666], [-74.13183905268437, 40.53786826969194], [-74.13176571749533, 40.53781744629256], [-74.13169530373547, 40.53787902910877], [-74.13168409320599, 40.53787135316617], [-74.13169767602062, 40.53785712331574], [-74.13172118332932, 40.53783735726568], [-74.13175700174476, 40.537807236781916], [-74.1317653008701, 40.537800259339875], [-74.13181654200083, 40.53775717054616], [-74.13182665649255, 40.53774866481481], [-74.131846271476, 40.5377346437995], [-74.13185661324943, 40.53774261876041], [-74.13178520730719, 40.537802291251616], [-74.13185740371571, 40.53785188895487], [-74.13203209392418, 40.53796798435104], [-74.13203758048124, 40.53797163087127], [-74.13204941374808, 40.5379794943672], [-74.13213560512452, 40.53803312718265], [-74.13221581581413, 40.53796492609293], [-74.13221608018897, 40.537964700125244], [-74.13243402654851, 40.53777653227275], [-74.13235106393256, 40.537719399099394], [-74.13199817051284, 40.53748098494119], [-74.13192190951362, 40.537539662442484], [-74.13191182124638, 40.537532840976915], [-74.13193499515837, 40.53751379882872], [-74.13197768070874, 40.53747872154013], [-74.13198946627011, 40.53746903747591], [-74.13199934255547, 40.53746092067324], [-74.13204846767975, 40.53742055322291], [-74.13205832728966, 40.5374124501718], [-74.13207952279657, 40.53739503414038], [-74.13209260140769, 40.53740465225158], [-74.13201821176241, 40.53746460223468], [-74.1320867758657, 40.53751489134775], [-74.13213200952625, 40.537545070134954], [-74.13217569640288, 40.53757421752609], [-74.13227873270836, 40.537642960834354], [-74.132358112262, 40.53769592115892], [-74.13236894558239, 40.53770314949493], [-74.13245302764156, 40.53776028136392], [-74.13275480490104, 40.53750453647995], [-74.13266909657912, 40.537442608810935], [-74.13257069197421, 40.5373751275351], [-74.13254236849414, 40.53735570494654], [-74.1324810728012, 40.53731367131547], [-74.13243400364236, 40.53728139278021], [-74.13239380632373, 40.537253827974666], [-74.13232205524741, 40.53720522204013], [-74.13224717384058, 40.53727022595487], [-74.13223596269246, 40.537262550736585], [-74.13225925203326, 40.53724190143914], [-74.13226527980805, 40.53723655530608], [-74.13231462298528, 40.537192805891365], [-74.13239801584261, 40.53711886493268], [-74.13240922528998, 40.53712568439255], [-74.13239022671648, 40.53714278902637], [-74.13239010549317, 40.537142893370074], [-74.1323376970766, 40.537188120784265], [-74.13241056494718, 40.53723672391339], [-74.1326908356868, 40.537426880575886], [-74.1327760371171, 40.53748401091587], [-74.13307445571547, 40.53722912330401], [-74.13299485796335, 40.537174549236404], [-74.13298060828508, 40.53716485133182], [-74.13291262916057, 40.53711858816072], [-74.13289785640718, 40.53710853401196], [-74.13286528600719, 40.537086368922076], [-74.13280917293142, 40.53704817981583], [-74.1327292463779, 40.53699378771441], [-74.13271794955475, 40.53698609823457], [-74.13264694004168, 40.53693703881057], [-74.13256683070924, 40.536997376724116], [-74.13255786139064, 40.5369914077573], [-74.13258133450967, 40.53697173488311], [-74.13261491537888, 40.53694581083255], [-74.13264197295432, 40.536924925074736], [-74.13265090290378, 40.536918029599235], [-74.13266830698964, 40.53690459450426], [-74.13270876886787, 40.5368733589692], [-74.132735602333, 40.53685453524314], [-74.132743451717, 40.53686135972777], [-74.13266726879559, 40.53692018958826], [-74.13274096742529, 40.536970514597385], [-74.13275151405895, 40.53697771575411], [-74.1328316896651, 40.537032462213524], [-74.13284007641691, 40.53703818975167], [-74.13291844847332, 40.53709170588447], [-74.13292737131076, 40.537097797844105], [-74.13300283995096, 40.53714933104183], [-74.13301721920324, 40.53715914908029], [-74.1330945751156, 40.537212017927736], [-74.13317604148284, 40.5371419844368], [-74.13317616358326, 40.53714188009126], [-74.13339187377338, 40.53695798434753], [-74.13330732067607, 40.53689864197504], [-74.13329656721403, 40.53689136421298], [-74.13321806916443, 40.53683823665119], [-74.13320844392692, 40.536831723352684], [-74.13312853241104, 40.536777638426365], [-74.13311938058938, 40.536771444554155], [-74.13304340527381, 40.536720023929306], [-74.13303088228227, 40.53671154817349], [-74.13295577183409, 40.53666123872185], [-74.13288633976343, 40.53672604241201], [-74.13287438280999, 40.53671963230237], [-74.13289203626539, 40.53670145879365], [-74.13289925552112, 40.536695070884875], [-74.13294850623681, 40.53665150139699], [-74.13301385446526, 40.53659369293556], [-74.1330350883952, 40.53657487562588], [-74.1330440563831, 40.53658084455711], [-74.13297252934821, 40.53664328251968], [-74.13304651887665, 40.53669273852], [-74.13329285686525, 40.53685798057631], [-74.13331686814696, 40.53687408670027], [-74.13332786488384, 40.53688146302078], [-74.13341004365357, 40.53693972187094], [-74.13368134221429, 40.53670908436055], [-74.13368957732476, 40.536703424813574], [-74.13366576406797, 40.536687643042846], [-74.13380823263125, 40.53656217617863], [-74.13386054140379, 40.53659796249381], [-74.13359543790433, 40.53681717084161], [-74.13321060585267, 40.537146089540805], [-74.13277513113499, 40.537519835125615], [-74.13246512857398, 40.53778519473216], [-74.13220259291258, 40.53801008686732], [-74.13193731463402, 40.5382371143752], [-74.13117706403472, 40.538889413059714], [-74.13104786423696, 40.53899765026269], [-74.13062872480312, 40.53936604352793], [-74.13008136262059, 40.539848874054925], [-74.13007620892891, 40.53984511951034], [-74.13007172337215, 40.53985113574353], [-74.13008325390014, 40.539859439550796], [-74.12996934203085, 40.53996776961415], [-74.12993843189791, 40.53994713555815], [-74.1300492851502, 40.53983850810548], [-74.13005836130274, 40.53984343795619], [-74.13006392754764, 40.53983813468063], [-74.13005667868796, 40.53983267085222], [-74.13020869361581, 40.5397026638173], [-74.13012468138237, 40.53964658701432], [-74.12992916823943, 40.539521154057276], [-74.12984771335016, 40.53946889602442], [-74.12976405619064, 40.53941522294951], [-74.1296942734552, 40.53947824038521], [-74.12968375508696, 40.53947171959242], [-74.12984555503498, 40.539326856961516], [-74.12985819958884, 40.53933524178914], [-74.12978384857315, 40.53939867623573], [-74.13011984290469, 40.53961580881986], [-74.13012883708546, 40.53962162097209], [-74.13014051454833, 40.539629166365444], [-74.13022754297842, 40.539683717556336], [-74.13052164283053, 40.539425558119696], [-74.1304304813722, 40.53936292030345], [-74.13015622961893, 40.539168776673066], [-74.13008440018643, 40.53911792687624], [-74.13001148713674, 40.53918004171496], [-74.13000272814944, 40.539172393658546], [-74.13016935798473, 40.5390328780233], [-74.13017804327282, 40.53904000479672], [-74.13010505586573, 40.539101735967925], [-74.130448398878, 40.5393451134319], [-74.13054176521159, 40.53940930625567], [-74.1308390685189, 40.5391475928925], [-74.13075187600008, 40.53908999447238], [-74.1307392231052, 40.53908108387502], [-74.13066327325774, 40.53902759483427], [-74.13065254269095, 40.53902003735925], [-74.13057878861137, 40.53896809441335], [-74.1305630482502, 40.53895701008492], [-74.1305024944511, 40.53891436357641], [-74.13047733304663, 40.53889664368449], [-74.13040743526518, 40.53884741626671], [-74.13033366098274, 40.538909000381715], [-74.13032244899301, 40.538900471244446], [-74.13034313706332, 40.5388824873475], [-74.13035121393887, 40.53887546569789], [-74.13039930643454, 40.538833658032196], [-74.13044784914226, 40.53879145848406], [-74.13046292342244, 40.5387783547327], [-74.13047277242568, 40.53876979179294], [-74.13048528294742, 40.538758916255986], [-74.13049601788298, 40.538765930776215], [-74.13042531514306, 40.53883031223013], [-74.13065501375159, 40.5389909516347], [-74.13067307625626, 40.539003584135074], [-74.1307244736066, 40.53903952770754], [-74.13075959354838, 40.5390640878557], [-74.13077148681744, 40.53907240592385], [-74.13085918854094, 40.53913133980232], [-74.13115328715367, 40.5388735746063], [-74.13107294890185, 40.53881805636092], [-74.13103332596896, 40.53879154117598], [-74.13098497193288, 40.538759182264165], [-74.1309722667305, 40.538750680693354], [-74.13089371476259, 40.538698110994915], [-74.130881040585, 40.53868962964966], [-74.13080381066602, 40.53863794662793], [-74.13071925808853, 40.538581361914886], [-74.13064774721937, 40.538642142694705], [-74.13063765737984, 40.53863532178967], [-74.13065428041827, 40.538617311132974], [-74.1306632570612, 40.53860970559962], [-74.13071230442803, 40.538568145832265], [-74.13072134858022, 40.538560483592335], [-74.1307248577209, 40.53855750911845], [-74.13077736811273, 40.53851301527632], [-74.130799718963, 40.53849249031404], [-74.13081093015272, 40.53850016583937], [-74.13073782459357, 40.53856443328906], [-74.13080965715949, 40.538612221563156], [-74.13109054543966, 40.53880156737978], [-74.13117436985067, 40.53885516139858], [-74.13147424215319, 40.538598314882684], [-74.13139059484412, 40.53854384369088], [-74.13138154351878, 40.538537652414064], [-74.13130326515676, 40.53848411268165], [-74.13129293456258, 40.538477046677485], [-74.13121290254978, 40.53842230719425], [-74.13120198444494, 40.538414838943154], [-74.131125475332, 40.53836250914887], [-74.13111517489904, 40.53835546353301], [-74.13103945369103, 40.53830497996996], [-74.13096668803547, 40.53836767222718], [-74.13095678068017, 40.538360738059005], [-74.13097509115991, 40.538341749902095], [-74.1309848364326, 40.53833354605366], [-74.13103263828731, 40.53829330686974], [-74.13104306533829, 40.53828452861476], [-74.13106488337263, 40.538266162835484], [-74.13110213313608, 40.53823480522071], [-74.1311196228268, 40.53821889282785], [-74.13113064695301, 40.538226921521776], [-74.13105906666092, 40.53828898106287], [-74.1311352951365, 40.53833835682635], [-74.13114477760698, 40.53834473308612], [-74.13122513527071, 40.538398766272486], [-74.13141556350926, 40.53852680878589], [-74.13149740072586, 40.538581381378116], [-74.13179350815649, 40.53832599199171], [-74.1317117617114, 40.538268528100936], [-74.13170084809326, 40.53826112355357], [-74.13162168805269, 40.538207420868716], [-74.13161107195381, 40.538200218013216], [-74.13153338702135, 40.5381475165802], [-74.1315233806718, 40.53814072748779], [-74.13144461494203, 40.53808729160741], [-74.13143339816786, 40.53807968148695], [-74.1313886951325, 40.53804935472692], [-74.13134558736793, 40.53802010971008], [-74.13128102022999, 40.53797630604305], [-74.13125546600635, 40.53795896991141], [-74.13116466312675, 40.5379001358691], [-74.13086625286705, 40.53816441458347], [-74.12881604599956, 40.5399857423965], [-74.1287873750151, 40.53996308041394], [-74.1286137155024, 40.54010385504027], [-74.12844667307078, 40.54024921571889], [-74.12828645569175, 40.54039898151023], [-74.12813326284557, 40.54055296599051], [-74.1279872852721, 40.54071097748305], [-74.12788702673318, 40.54085416819418], [-74.12779434220613, 40.54100029873628], [-74.12770937956203, 40.541149136048986], [-74.12763227435845, 40.5413004427534], [-74.12756314962428, 40.54145397753053], [-74.12750211566271, 40.54160949550625], [-74.12746105532581, 40.54174455708288], [-74.12742903240955, 40.54188103431571], [-74.1274061294962, 40.54201857548663], [-74.12739240566611, 40.54215682613376], [-74.12738789634466, 40.54229542996431], [-74.12739261321042, 40.54243402977327], [-74.1274065441645, 40.54257226836384], [-74.12744321814655, 40.54273687808907], [-74.12748797280015, 40.54290032049877], [-74.12754074499249, 40.5430623648181], [-74.1276014602687, 40.543222782245074], [-74.12767003295674, 40.5433813462729], [-74.12770001341886, 40.54337412153509], [-74.1277314342735, 40.54342308480361], [-74.12789824961531, 40.54336617156212], [-74.12786756770616, 40.54331518006602], [-74.12788626375576, 40.543308077074705], [-74.12792830507608, 40.54337324705599], [-74.12769175450491, 40.54345570819183], [-74.12756660762842, 40.54349933393445], [-74.12762994038083, 40.543608893115554], [-74.12775375608553, 40.54356602518414], [-74.12798968390734, 40.54348434363108], [-74.12803068497496, 40.54355055827895], [-74.12801054879323, 40.54355725060719], [-74.12798005253295, 40.54350903021883], [-74.12781893446551, 40.54356439964454], [-74.12794808054065, 40.543764270981825], [-74.12808505077827, 40.54396108784557], [-74.12822972124094, 40.54415467204766], [-74.12838196101643, 40.544344848325316], [-74.1285416323362, 40.544531444499434], [-74.1287085906994, 40.54471429163112], [-74.12886010299626, 40.54488860547736], [-74.12901864630722, 40.54505924176275], [-74.12918406740488, 40.545226035515675], [-74.12935620640944, 40.54538882547804], [-74.12944814577739, 40.54548399654139], [-74.12954591589445, 40.545575723517445], [-74.12964929541047, 40.545663798715815], [-74.12975805027173, 40.54574802271361], [-74.12987193425054, 40.54582820480734], [-74.12999068950306, 40.5459041634447], [-74.13011404715266, 40.54597572663567], [-74.13026052930192, 40.5460537683542], [-74.13041166382703, 40.54612646491236], [-74.1305671158249, 40.546193655212655], [-74.13072654082103, 40.54625519035942], [-74.13088958553416, 40.54631093398849], [-74.13110690077004, 40.546387296218136], [-74.13132719587999, 40.54645851709803], [-74.13155026188892, 40.54652452906437], [-74.13177588719115, 40.54658526949438], [-74.13200385775187, 40.546640680766075], [-74.13257816042828, 40.546754026076755], [-74.13274571808144, 40.54678761708334], [-74.1329153709079, 40.54681439842986], [-74.13308664365856, 40.546834295093575], [-74.13325905654519, 40.546847251337255], [-74.1334321265842, 40.546853230866425], [-74.13346993318864, 40.54685516378519], [-74.13364262288287, 40.54685496293632], [-74.13382177095173, 40.5468457418508], [-74.134000373591, 40.546831650391844], [-74.13417820332243, 40.5468127065072], [-74.13435503365305, 40.54678893432459], [-74.13453063936326, 40.54676036412176], [-74.13475634475162, 40.54672134775148], [-74.13498045519505, 40.54667730545007], [-74.13520277840065, 40.54662827500777], [-74.13542312360997, 40.546574298494534], [-74.13564130176358, 40.54651542222416], [-74.13585712583111, 40.546451696357344], [-74.13700629799997, 40.54696777099985], [-74.13744805299993, 40.54730532899985], [-74.13820137199997, 40.54803495999985], [-74.13861839799995, 40.54845247299989], [-74.13861798499991, 40.548492383999886], [-74.13769731599993, 40.54938354599991], [-74.137389882, 40.549592369999885], [-74.13607343399998, 40.550433299999874], [-74.13528075999987, 40.550863238999895], [-74.13455868199995, 40.55125507799986], [-74.13316825999992, 40.552356178999936], [-74.13177828099997, 40.55348997099988], [-74.131086652, 40.5539737589999], [-74.13042459599988, 40.55443704199991], [-74.12975602399997, 40.55503262499995], [-74.12896141599992, 40.555769871999864], [-74.12811697199996, 40.556534435999886], [-74.12730057799996, 40.55729517199993], [-74.1272522789999, 40.557265490999896], [-74.12572675099993, 40.55632805699984], [-74.12402047099992, 40.555230827999914], [-74.12380031599992, 40.55508465799988], [-74.12269895799986, 40.55435850199992], [-74.12076288199997, 40.55308194999988], [-74.1194585929999, 40.5522219059999], [-74.11936210999988, 40.55214080699988], [-74.11852157499992, 40.55143427699991], [-74.11876291999992, 40.55121611399988], [-74.11813477599993, 40.550592692999906]]]}}, {\"id\": \"110\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 111, \"Shape_Leng\": 0.0599308800658, \"Shape_Area\": 0.000208683276503, \"zone\": \"Green-Wood Cemetery\", \"LocationID\": 111, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99516890099986, 40.6596695219999], [-73.99296401999993, 40.65833740999993], [-73.99238339499986, 40.65889901599996], [-73.990188415, 40.657600817999906], [-73.98959238099985, 40.658134060999934], [-73.9890128659999, 40.658690795999895], [-73.98843135299977, 40.65925121199996], [-73.98622761299997, 40.65792048299986], [-73.98402397199989, 40.65659116999997], [-73.98179568699996, 40.65524609699991], [-73.9814529379999, 40.65341633599998], [-73.98129697399993, 40.65263750699989], [-73.98110911899991, 40.65159173199992], [-73.98105378299987, 40.6513038189999], [-73.98071174699994, 40.64952408699987], [-73.9803633469999, 40.64769374399988], [-73.98029115499993, 40.647297160999855], [-73.98132887599996, 40.64718167299989], [-73.98180533899996, 40.64712779899987], [-73.982309398, 40.646927403999875], [-73.98307340099996, 40.646625133999905], [-73.98528412799996, 40.64576793799989], [-73.98834986699991, 40.64456224399992], [-73.98905872499985, 40.64411924099995], [-73.99180823499984, 40.64578396899985], [-73.99392923899993, 40.64706931099991], [-73.9960456529999, 40.64834065499993], [-73.99830602299986, 40.649740014999935], [-73.99779466000003, 40.65028888999992], [-74.00214474599989, 40.65295610699996], [-74.00159439599989, 40.6534880839999], [-74.00099519199993, 40.65406723399986], [-74.00041318199995, 40.6546287019999], [-73.99983786599982, 40.655186157999935], [-73.99925656999994, 40.655745718999846], [-73.99867041399985, 40.656306860999905], [-73.99809100200001, 40.65686468799991], [-73.99750144599993, 40.65742847199989], [-73.99691824199996, 40.657989632999914], [-73.99633560799998, 40.65854721599988], [-73.99575278499995, 40.65910782999988], [-73.99516890099986, 40.6596695219999]]]}}, {\"id\": \"111\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 112, \"Shape_Leng\": 0.107267875765, \"Shape_Area\": 0.000462068671429, \"zone\": \"Greenpoint\", \"LocationID\": 112, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.95439555417066, 40.73911477252253], [-73.95428317027596, 40.73910737037338], [-73.9542326293958, 40.73910735029926], [-73.95419351267796, 40.739106957142994], [-73.95415627885323, 40.739106914853046], [-73.95412186600322, 40.73910076884593], [-73.95402778116019, 40.739083144574764], [-73.95397240398297, 40.739074945827895], [-73.953815514917, 40.73905312586354], [-73.95373783285332, 40.7390453336898], [-73.95358089064729, 40.73903297728494], [-73.95351461853663, 40.73902692697508], [-73.95336744598872, 40.739010514220354], [-73.95323299037699, 40.738992683138186], [-73.95292355724176, 40.73893494239519], [-73.95286232074187, 40.738921560300376], [-73.95278707574712, 40.738904174916264], [-73.95271781388718, 40.7388910848032], [-73.95265702902608, 40.73887887647444], [-73.95255527571982, 40.73885768188004], [-73.95238448777513, 40.73882039306002], [-73.95218669424783, 40.7387852540356], [-73.95217731880568, 40.73878358409279], [-73.95214506084804, 40.73877825742584], [-73.95210541738054, 40.73877098437249], [-73.95203337473411, 40.738757256928324], [-73.95201751319439, 40.73875376510164], [-73.95200514191774, 40.73875151743825], [-73.9519879435442, 40.73874762097677], [-73.95197731225186, 40.738731289086516], [-73.95197285125762, 40.73872743361655], [-73.95195469190358, 40.7387089390017], [-73.95194927258065, 40.73870482128728], [-73.95194368602378, 40.7387017118441], [-73.9519358866919, 40.73869852089705], [-73.95192759029223, 40.73869589681883], [-73.95191056675478, 40.73869269364296], [-73.95190236352236, 40.738692204390674], [-73.95189451325636, 40.738692452066324], [-73.95188714964519, 40.73869343974128], [-73.95188035286634, 40.738695156727694], [-73.95188004444059, 40.73869526482079], [-73.95187976088395, 40.738695399728655], [-73.95187948788579, 40.73869555273364], [-73.9518792503409, 40.738695715135], [-73.95187902576566, 40.738695912219725], [-73.95187881197509, 40.73869611952807], [-73.95187846958474, 40.73869659700273], [-73.9518782086343, 40.73869714638034], [-73.95187803065264, 40.738697785251695], [-73.95187789894662, 40.73869930700642], [-73.95187825200313, 40.73870205341412], [-73.95188040058603, 40.73870989773035], [-73.95188068446178, 40.73871145634694], [-73.95188069530279, 40.73871260859812], [-73.95188060027678, 40.73871305049302], [-73.95188044628392, 40.738713409940274], [-73.95188033957237, 40.73871356418746], [-73.95188022032096, 40.73871369866136], [-73.95188008963592, 40.73871380615867], [-73.95187993583103, 40.73871390543759], [-73.95187975825947, 40.73871397672964], [-73.95187956968144, 40.73871403980829], [-73.95187936768745, 40.738714075909655], [-73.95187914324185, 40.73871409340613], [-73.95187864527348, 40.73871407560779], [-73.95187806586495, 40.738713985906536], [-73.9518578844407, 40.73870829531153], [-73.9518126933132, 40.738694183756], [-73.9517171496975, 40.73866521896622], [-73.95154202956655, 40.73861125820223], [-73.95064521733299, 40.73836447215223], [-73.95050973889533, 40.738322685026716], [-73.95046458280514, 40.73830958079499], [-73.95038748197699, 40.738284837125704], [-73.9502194406015, 40.738221531600374], [-73.95004848537417, 40.738158052683225], [-73.94996516469928, 40.73812448107607], [-73.94964771678872, 40.73799773111222], [-73.94948430551214, 40.73793385945839], [-73.94939701000155, 40.73789966433983], [-73.94925743288589, 40.73784498780436], [-73.94884377400736, 40.73768589241637], [-73.94861200384757, 40.737595450101686], [-73.94858408326095, 40.73758455026463], [-73.94840521772191, 40.73751383459341], [-73.9481229165706, 40.737397154608416], [-73.94810675303027, 40.737433707959426], [-73.94809172682254, 40.73746770486471], [-73.94754211492793, 40.73727891654832], [-73.94720149585581, 40.73716644881961], [-73.94718139793846, 40.73716125315256], [-73.94716178589233, 40.73715493175969], [-73.94713634142643, 40.73714465473251], [-73.94709557208321, 40.737124860017104], [-73.94704934942234, 40.73709995757922], [-73.94703617987669, 40.7370917931531], [-73.94703400243661, 40.73709027972293], [-73.94700570033963, 40.73707061752642], [-73.94698861180811, 40.73706106400497], [-73.94696827978046, 40.73705259857922], [-73.94691047650039, 40.73703173433757], [-73.94685990836236, 40.73700738818559], [-73.9468397774883, 40.736999149493236], [-73.9468215743273, 40.73699376466284], [-73.94677824012722, 40.73698401889661], [-73.94675925460247, 40.73697872365636], [-73.9466991905293, 40.736957750711255], [-73.94661589520477, 40.73692455496032], [-73.94658050648984, 40.7369442689772], [-73.94652352854791, 40.73692685395814], [-73.94645392267336, 40.73680959046114], [-73.94662099934824, 40.736470082694524], [-73.94663121313155, 40.73644959458711], [-73.94680539926374, 40.736100225202975], [-73.94687945162181, 40.7359393498804], [-73.94719983367108, 40.73535517811412], [-73.94718691118442, 40.735230782100416], [-73.94715775859925, 40.735032297251614], [-73.94708643463501, 40.73454591548952], [-73.94706532915383, 40.734401992819265], [-73.94659806707347, 40.73444222434728], [-73.94662828221189, 40.734584789334775], [-73.94673692340704, 40.735294943719914], [-73.94675678399388, 40.73525766653474], [-73.94692902737461, 40.73531314068372], [-73.94662663786625, 40.73586558516736], [-73.94644310074177, 40.735813184455786], [-73.94643051256956, 40.735835358937784], [-73.9464005067585, 40.73588906853683], [-73.94630720090619, 40.73605751950615], [-73.94624748296931, 40.736166165226955], [-73.94617195881506, 40.736202888774216], [-73.9458731056289, 40.73610183125813], [-73.94557048457217, 40.736612303910405], [-73.9455325990523, 40.736605081020336], [-73.94551998044125, 40.736590662918225], [-73.94554528485209, 40.736537832854566], [-73.94541584986361, 40.736501744510726], [-73.94540003127128, 40.73653776407872], [-73.9453589717991, 40.736549754575876], [-73.94520743701781, 40.73650885051612], [-73.94520113938772, 40.73648723040826], [-73.94522012115175, 40.7364440057188], [-73.94507806672308, 40.736393497684865], [-73.9450590833402, 40.73643912568502], [-73.94503381424778, 40.73644872026402], [-73.94499908889478, 40.73643669371263], [-73.94499910475585, 40.73641747877059], [-73.94525492279553, 40.735892789003714], [-73.94518507181313, 40.7354791526778], [-73.94503299031923, 40.735428615683325], [-73.9450278576709, 40.735450523519155], [-73.94502179930069, 40.73547490611652], [-73.94500642212093, 40.73553144180913], [-73.94499058532952, 40.735570308764515], [-73.94496124483574, 40.73563424915079], [-73.94492309227375, 40.735702786597], [-73.94488540424965, 40.73576637049456], [-73.94486214046046, 40.735810538937095], [-73.94483653414815, 40.73586778119121], [-73.944785512709, 40.73596454331514], [-73.94477272080769, 40.735985761277085], [-73.94475921021491, 40.736004107824215], [-73.94473954067765, 40.736024243277555], [-73.94469052058979, 40.736064138286196], [-73.94467094742069, 40.73608275263356], [-73.94463300629042, 40.7361238502514], [-73.94461272994495, 40.736147398742844], [-73.9445772611247, 40.736191181304946], [-73.9445676624131, 40.73620282161703], [-73.94454962530499, 40.736224657881706], [-73.94454903299513, 40.73622528833371], [-73.94454837002635, 40.736225873184395], [-73.94454679443378, 40.73622692583219], [-73.944544887742, 40.73622780761089], [-73.94454261380297, 40.736228553683965], [-73.94453916861191, 40.736229317788215], [-73.94452715112095, 40.736231013733864], [-73.94452429787943, 40.73623158010659], [-73.94452191718351, 40.73623225409138], [-73.9445200696358, 40.7362329914904], [-73.94451850588766, 40.736233837248214], [-73.94451721533652, 40.7362348280481], [-73.94451617245115, 40.736235944109566], [-73.94451565068118, 40.73623676289501], [-73.94451534256572, 40.7362375559844], [-73.94451527153959, 40.73623795114505], [-73.94451524743039, 40.73623833929227], [-73.94451528167916, 40.73623873450372], [-73.94451537650774, 40.73623911282414], [-73.9445155184605, 40.73623949987866], [-73.944515707992, 40.736239878747334], [-73.9445162630204, 40.73624064427562], [-73.94451704402161, 40.73624140220654], [-73.94451805009484, 40.736242176160964], [-73.94452008496624, 40.73624341114128], [-73.94452280603181, 40.736244735408626], [-73.94453691364184, 40.7362506676148], [-73.94453998943426, 40.736252353239685], [-73.94454119610867, 40.73625317285769], [-73.9445422028042, 40.73625399355208], [-73.9445429831162, 40.73625478599291], [-73.94454363410208, 40.73625563281771], [-73.94454415356005, 40.736256533354975], [-73.94454454389673, 40.7362575063692], [-73.94454479148013, 40.73625852471414], [-73.9445449211989, 40.736259586391874], [-73.94454475257284, 40.73626187438715], [-73.94454404114089, 40.73626432344887], [-73.94454280763553, 40.73626689740155], [-73.94454106488915, 40.73626952605764], [-73.9445388600858, 40.73627214628246], [-73.94453322105807, 40.73627716919283], [-73.94453001123387, 40.73627933711444], [-73.94452667130409, 40.73628117226581], [-73.94452008735732, 40.736283654181655], [-73.94451542117358, 40.73628439139539], [-73.94449242081755, 40.73628307377263], [-73.94448138831775, 40.736281492532946], [-73.94447036793825, 40.73627990342435], [-73.94444758298177, 40.736274884901825], [-73.94442362731286, 40.73626792082761], [-73.94438048917405, 40.73625197878863], [-73.94431566023265, 40.7362244835064], [-73.94329968017576, 40.73582361273334], [-73.94321347652942, 40.73579053001607], [-73.94316788320036, 40.73576750865239], [-73.94310507092837, 40.735735797710205], [-73.94300691110804, 40.73568865195108], [-73.94294414311412, 40.73566019179398], [-73.94288044032209, 40.73563294704598], [-73.94284189605364, 40.73561723236554], [-73.94239443542429, 40.73542574994758], [-73.94212245962963, 40.73523376140624], [-73.94210536596876, 40.73521903812462], [-73.94196892130114, 40.73511897640343], [-73.94177231077718, 40.734982081389916], [-73.94173819589838, 40.734950717917634], [-73.94179189160984, 40.73491289714997], [-73.94178616641555, 40.734894936923226], [-73.94176306503515, 40.73485938231849], [-73.94169368832134, 40.73476937861393], [-73.94160658196857, 40.734687702163804], [-73.9415456683245, 40.73473433593477], [-73.94129801856792, 40.734376293433634], [-73.94114746701946, 40.734152927537586], [-73.94099746599917, 40.733790485555836], [-73.94115615809403, 40.73374890960982], [-73.9411353395535, 40.733702811249984], [-73.94103843274786, 40.73348826020298], [-73.94094207548785, 40.73350865207841], [-73.94092914681453, 40.733511391877165], [-73.94091758051687, 40.733484047121365], [-73.94081817225852, 40.73324890944027], [-73.94075490700678, 40.733277809914455], [-73.94072562055, 40.733291185407055], [-73.9406442348714, 40.733170511479614], [-73.94059112746399, 40.73309176622027], [-73.94056868930231, 40.73305849425773], [-73.94068491601338, 40.73301707712448], [-73.94036658952942, 40.73221258691483], [-73.93988785146199, 40.731295621033695], [-73.9398132262544, 40.731152688912026], [-73.93974607754137, 40.73106547547206], [-73.93891503477053, 40.7303837947332], [-73.93856120412507, 40.73013364321876], [-73.9381033496066, 40.729728519521366], [-73.93808900912045, 40.729746846260504], [-73.93729511262924, 40.72930055678739], [-73.93711874507939, 40.729189932064926], [-73.9369312479987, 40.7291080912655], [-73.93684001378081, 40.72908791393823], [-73.93655894374989, 40.729017826266926], [-73.93608245486588, 40.7288827118521], [-73.93563591233656, 40.728751952652814], [-73.93561453981158, 40.72874598563435], [-73.93527438167969, 40.728651019073425], [-73.93501193202125, 40.728572427731145], [-73.93492926391862, 40.72854391575297], [-73.93480845721055, 40.72852861070382], [-73.93425572782597, 40.72843291447266], [-73.93423032768017, 40.72843131516514], [-73.93423524033687, 40.72839549557832], [-73.93422871008791, 40.728390826901084], [-73.93415421562172, 40.72838165362064], [-73.93376676075171, 40.728333939440496], [-73.93352761652707, 40.72830739759007], [-73.93349948194367, 40.7283041392015], [-73.93348555981649, 40.728294414781345], [-73.93348271999301, 40.72828156384591], [-73.93325729805613, 40.72824070281476], [-73.93301658610119, 40.72821034223369], [-73.93291116235257, 40.72823331594163], [-73.93262682661627, 40.72818061356344], [-73.93240789866175, 40.7281459238374], [-73.93238700367864, 40.72814769455616], [-73.932360906291, 40.728167770367016], [-73.9323459884982, 40.728182393884644], [-73.93226852175046, 40.72817465801565], [-73.93255438599991, 40.727862889999905], [-73.93412337399995, 40.726855470999894], [-73.93539740399997, 40.725906866999956], [-73.93676382399988, 40.724838918999836], [-73.93765314499996, 40.7242655549999], [-73.9384885709999, 40.723808905999924], [-73.93891183799992, 40.72357882499994], [-73.93931199899998, 40.72334789399991], [-73.94022123699996, 40.722891504999936], [-73.9410941879999, 40.72230924499994], [-73.94150380799985, 40.7219909789999], [-73.94195633599989, 40.721652318999894], [-73.94281683699987, 40.72114362099989], [-73.94367547999985, 40.7206599749999], [-73.94440536699992, 40.72024041199988], [-73.94560358899994, 40.71956728599989], [-73.94679211599993, 40.7188216079999], [-73.94721341799986, 40.718560985999865], [-73.94841638599995, 40.71845553299995], [-73.94853260499984, 40.71916717599985], [-73.94887459699996, 40.721292406999886], [-73.94907816799986, 40.72254028699993], [-73.94986599500001, 40.72217948399988], [-73.95064594299997, 40.72170569299989], [-73.95164632299998, 40.723485856999936], [-73.95249594800003, 40.72321773399991], [-73.95332036799981, 40.72293269999991], [-73.95382461699991, 40.722759734999904], [-73.95418330299992, 40.72263642799991], [-73.95477662900004, 40.72245950499994], [-73.95627311499997, 40.723389875999914], [-73.95771989199996, 40.72428495699989], [-73.95814292503528, 40.724404284201356], [-73.95803916741664, 40.72442744087862], [-73.95794183489491, 40.724649202699844], [-73.95795096213246, 40.72503643916838], [-73.95827588223725, 40.72516745704099], [-73.9583517707884, 40.72519805730171], [-73.95842611843923, 40.725400535832165], [-73.96012497133233, 40.72531963663684], [-73.96028544944221, 40.725948122413], [-73.9610634351171, 40.72584489918542], [-73.9616151883593, 40.725865563073995], [-73.96165280828099, 40.72620392623105], [-73.96161716963752, 40.72666142199101], [-73.96149920105307, 40.72675109112526], [-73.96141826820885, 40.72735617970476], [-73.96114159312455, 40.728103725404615], [-73.96105130889325, 40.72846638151088], [-73.96101148031258, 40.72851935234766], [-73.96099640841605, 40.72876912608592], [-73.96106107684327, 40.728776716825294], [-73.96107076838537, 40.729219509694424], [-73.96114030095012, 40.7294163293656], [-73.96119502412036, 40.729416347820276], [-73.96121771697015, 40.72952970731533], [-73.96134881243212, 40.73016195102422], [-73.96129090052489, 40.730161931570954], [-73.96128413823038, 40.73016192929752], [-73.96130143673095, 40.730252942924714], [-73.96143187677845, 40.730965171647846], [-73.96171614512016, 40.73156991744345], [-73.96182557485885, 40.7316077990263], [-73.96190947717899, 40.73165137399744], [-73.96212128302159, 40.73237372333121], [-73.96226340423281, 40.732915551623485], [-73.96236012830968, 40.73291262777684], [-73.96284830750984, 40.73287157545573], [-73.96332595813487, 40.7328308015024], [-73.9633368536688, 40.73283852928469], [-73.96352136166972, 40.732827188371246], [-73.96353117193021, 40.73285022101445], [-73.9635572631168, 40.73284899745191], [-73.96356590964024, 40.732894298019346], [-73.96357459572387, 40.73289504894067], [-73.96357806468741, 40.732912125681175], [-73.9635885915715, 40.73291184522025], [-73.96358665220083, 40.73292157806546], [-73.96367726013192, 40.73292007347841], [-73.96366122635746, 40.73279850598194], [-73.96403099136643, 40.732757938759605], [-73.96404925210085, 40.73276153790588], [-73.96406507049178, 40.732779040653625], [-73.96408821900232, 40.732919099545036], [-73.9640789145577, 40.732932960467735], [-73.96406047861366, 40.73294502382845], [-73.96372106318971, 40.7329800783661], [-73.96365189579522, 40.73298729001453], [-73.96365321450872, 40.73299217114834], [-73.96357720532276, 40.73299759710496], [-73.9635744914258, 40.732994796040735], [-73.9635387832987, 40.73299665450352], [-73.96353736821686, 40.73300047416273], [-73.96275541965417, 40.73307273927299], [-73.96235748813149, 40.73311001754874], [-73.96235652228026, 40.73310793337061], [-73.96230408011061, 40.73311409141064], [-73.96234062194117, 40.73338922663284], [-73.96230837696692, 40.733532325268946], [-73.96235505955215, 40.73371326112401], [-73.96232467828575, 40.73390776034564], [-73.96228520236923, 40.734047695876654], [-73.96218552877323, 40.73424901663273], [-73.9620903444991, 40.734507997873685], [-73.96198477870479, 40.73463803314813], [-73.96190964676319, 40.73475080674919], [-73.96176208458849, 40.73489161477911], [-73.96157494306726, 40.7351796149286], [-73.96119003134037, 40.73560279658299], [-73.96051916717968, 40.736342192300086], [-73.96051155046175, 40.736350608897524], [-73.9604674814225, 40.736399645855464], [-73.9604226223433, 40.7364445560135], [-73.96029557044754, 40.7365629197999], [-73.96023863089073, 40.736613761013814], [-73.96022288450308, 40.736636655602425], [-73.96021577508543, 40.73664518107986], [-73.96018765307628, 40.736675472872975], [-73.96014731842976, 40.73671569374603], [-73.96009521419664, 40.73675854873325], [-73.96007713554904, 40.73677856103871], [-73.96003383184511, 40.7368343138929], [-73.96000106579093, 40.73686913511551], [-73.95991659213416, 40.736954050356296], [-73.9598950195283, 40.73697905906495], [-73.95985081417777, 40.73703685585992], [-73.95982931147853, 40.73706090063316], [-73.95981583252653, 40.73707273787133], [-73.95979907338204, 40.73708439336929], [-73.95977974555224, 40.73709566098169], [-73.95972243919357, 40.73712631202641], [-73.95970585909082, 40.73713685201491], [-73.9596922503639, 40.737147257009646], [-73.95966543236098, 40.73717216449541], [-73.95960239868396, 40.737236951506105], [-73.95958311452335, 40.73725416310573], [-73.95956431785012, 40.737268671834386], [-73.9594701001917, 40.73733478183132], [-73.9594013832884, 40.73738101602729], [-73.95936872914949, 40.73740432887948], [-73.95930574127573, 40.737452068588844], [-73.95919107556782, 40.737539116286506], [-73.95916372962881, 40.73755700896849], [-73.95910990410896, 40.73758603978782], [-73.95908953404918, 40.737598009016814], [-73.95902797086988, 40.737637357848634], [-73.95898856589714, 40.737664727690294], [-73.95888819766095, 40.73774116402611], [-73.95883332045653, 40.73778546722045], [-73.95882474477474, 40.73779080322135], [-73.95881432450622, 40.73779583349642], [-73.95877797055266, 40.73780944591503], [-73.9587654299909, 40.737814808469125], [-73.95873036313797, 40.73783374297751], [-73.95867800305545, 40.73786649270285], [-73.95864031018328, 40.737899942069745], [-73.95861342305075, 40.7379212020659], [-73.95848746915823, 40.73801668347339], [-73.95845868680824, 40.73803920305101], [-73.95843189518179, 40.73805864437206], [-73.95837507902779, 40.73809694936863], [-73.95831235513249, 40.73813702647084], [-73.95824213550935, 40.73817991962026], [-73.95815588076832, 40.73823083041091], [-73.95795987117343, 40.738348391373066], [-73.9578942469991, 40.73838640527256], [-73.95785444148449, 40.7384101097698], [-73.95776488076119, 40.73846435936516], [-73.95771673677052, 40.738493320247436], [-73.95763409607012, 40.7385381177614], [-73.9575470386147, 40.73858224599516], [-73.95741823516057, 40.738649241332396], [-73.95715479533308, 40.73877632162018], [-73.95709329518277, 40.73880598925817], [-73.95695851073039, 40.73887387325899], [-73.95694640793143, 40.73887918200204], [-73.9569300286511, 40.738887073071545], [-73.95691820009638, 40.738888914788], [-73.95690347291868, 40.738889962683686], [-73.95677816914431, 40.73890559317934], [-73.95564004576131, 40.73904431620742], [-73.95563515530745, 40.73904470181219], [-73.95563004111821, 40.739044699841955], [-73.95561454319123, 40.73904581043332], [-73.95558336322831, 40.7390400711876], [-73.95553067150925, 40.739032693262196], [-73.95519660913385, 40.739084541693586], [-73.95492280493156, 40.739103300664986], [-73.95480757566851, 40.73911884350023], [-73.95476314020331, 40.73912353605884], [-73.95471533253463, 40.73912654195908], [-73.9546632150002, 40.73912795313434], [-73.95460353380118, 40.73912784837409], [-73.95454644819083, 40.7391249169295], [-73.95444125113258, 40.739117653969956], [-73.95439555417066, 40.73911477252253]]], [[[-73.96236596889429, 40.72420906161418], [-73.96200744849497, 40.7239919013038], [-73.96207271921921, 40.72388030020147], [-73.96246790011047, 40.72413157960129], [-73.96247195112899, 40.724174989273585], [-73.96244340911423, 40.72419978548133], [-73.96239043044622, 40.72419356718676], [-73.96236596889429, 40.72420906161418]]]]}}, {\"id\": \"112\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 113, \"Shape_Leng\": 0.032745350566, \"Shape_Area\": 5.79634566998e-05, \"zone\": \"Greenwich Village North\", \"LocationID\": 113, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99136388099988, 40.72755978299991], [-73.99154832699986, 40.72708947099994], [-73.99314656999991, 40.727867202999896], [-73.99425278499996, 40.728412456999905], [-73.99506103499998, 40.728810771999875], [-73.99581237099986, 40.7291805329999], [-73.99657214499979, 40.729552233999904], [-73.99606314799985, 40.73015526099988], [-73.99556245499996, 40.73071521599994], [-73.99697435499994, 40.73140883199991], [-73.99861225599987, 40.732215429999904], [-74.000066577, 40.7329292359999], [-73.99975771599985, 40.73334132199987], [-73.99955589199998, 40.73361059599991], [-73.999187803, 40.73413483399989], [-73.99871443800002, 40.734783860999855], [-73.99853021599989, 40.73503789199989], [-73.99827418999986, 40.73539113999986], [-73.99780442999989, 40.73603215799988], [-73.99734132799996, 40.736667888999925], [-73.99683993899998, 40.73736088899995], [-73.99361688399992, 40.73600449199991], [-73.99171498799984, 40.73521530099993], [-73.99119899299986, 40.73501236899993], [-73.9907539679999, 40.73483915499987], [-73.98990295999991, 40.7344347899999], [-73.98986852099996, 40.73352711499991], [-73.99002687699983, 40.73275628399994], [-73.99021190999987, 40.73199633399989], [-73.99039009999987, 40.731288523999865], [-73.99050558599987, 40.730564955999945], [-73.99072530399991, 40.729878686999896], [-73.99075728499993, 40.729740830999916], [-73.99093053599996, 40.7291422959999], [-73.99112256699988, 40.72856796599996], [-73.99135869199992, 40.72778246999995], [-73.99136388099988, 40.72755978299991]]]}}, {\"id\": \"113\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 114, \"Shape_Leng\": 0.0317270347711, \"Shape_Area\": 4.70207704945e-05, \"zone\": \"Greenwich Village South\", \"LocationID\": 114, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99257987399992, 40.724201554999915], [-73.99260322299989, 40.724136449999925], [-73.99267182399991, 40.72416025399983], [-73.99339891300001, 40.72441250499989], [-73.99345769099995, 40.724432895999925], [-73.99417930499997, 40.724692406999914], [-73.99490827399991, 40.72495122899995], [-73.99532907899994, 40.72508373999996], [-73.99584577599984, 40.72520835299988], [-73.9967717579999, 40.7254319439999], [-73.99761019599993, 40.72579713799992], [-73.998361193, 40.726167467999865], [-73.9991182469999, 40.72654231299988], [-73.99991579199988, 40.72693471299986], [-74.00063239199994, 40.72728661199988], [-74.00139033499991, 40.72766000199989], [-74.00214891499996, 40.728035366999904], [-74.00269315099997, 40.72831405399993], [-74.00281726599992, 40.72836312899993], [-74.00250642399995, 40.729016384999966], [-74.00220337299997, 40.72965325599993], [-74.00213899299997, 40.72979574399987], [-74.00168465499992, 40.73058156199995], [-74.00141242499994, 40.7310652559999], [-74.00095442299988, 40.73170609799994], [-74.00051066399996, 40.73231522299996], [-74.000066577, 40.7329292359999], [-73.99861225599987, 40.732215429999904], [-73.99697435499994, 40.73140883199991], [-73.99556245499996, 40.73071521599994], [-73.99606314799985, 40.73015526099988], [-73.99657214499979, 40.729552233999904], [-73.9958123719999, 40.729180532999926], [-73.99506103499998, 40.728810771999875], [-73.99425278499996, 40.728412456999905], [-73.99314656999991, 40.727867202999896], [-73.99154832699986, 40.72708947099994], [-73.99179379299984, 40.72639145899992], [-73.9920616679999, 40.72564244299991], [-73.99220971099992, 40.72523128999992], [-73.99232642199998, 40.72490823199993], [-73.99257987399992, 40.724201554999915]]]}}, {\"id\": \"114\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 115, \"Shape_Leng\": 0.116169413964, \"Shape_Area\": 0.000373168991958, \"zone\": \"Grymes Hill/Clifton\", \"LocationID\": 115, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08069514799999, 40.62753689699988], [-74.08076183699995, 40.62745285899991], [-74.08084371499999, 40.62737949099987], [-74.08099625499995, 40.627339813999896], [-74.0816522029999, 40.62735102899992], [-74.08183352499998, 40.62740084399991], [-74.08202428299997, 40.62743512499987], [-74.08222094199995, 40.62745254499992], [-74.08241960099991, 40.62745240299991], [-74.08261621699997, 40.62743470099987], [-74.08280689099995, 40.62740014699991], [-74.08298808899993, 40.62735007299989], [-74.0830624689999, 40.62731250399988], [-74.08326107499997, 40.627194742999954], [-74.08344837699995, 40.62706443699991], [-74.08362236799998, 40.62692257799988], [-74.08378128799998, 40.626770399999906], [-74.08392362399994, 40.62660935499987], [-74.08404821100001, 40.626441020999906], [-74.08415419899998, 40.626267114999905], [-74.08496211899991, 40.624625232999925], [-74.0852499159999, 40.6228008709999], [-74.08537203099993, 40.62220532899992], [-74.08548866899994, 40.62165317899995], [-74.0855941459999, 40.62119087199993], [-74.08560513399996, 40.62114273099995], [-74.0858126919999, 40.6203755489999], [-74.08589883599991, 40.619909428999925], [-74.08600629799997, 40.61909415099995], [-74.08604124499995, 40.61882017799994], [-74.08608006299993, 40.618515303999864], [-74.08626756599996, 40.61783201099995], [-74.0865715779999, 40.61716938599994], [-74.08698538899989, 40.61653298199997], [-74.08741339199996, 40.6159293259999], [-74.08769173799999, 40.61553515299991], [-74.08783375499992, 40.61531764999988], [-74.08785144099994, 40.61524323599987], [-74.08785147699994, 40.615243072999924], [-74.08785147599997, 40.61524292099995], [-74.08784843099988, 40.615164874999884], [-74.08784843099991, 40.61516472999992], [-74.0878483839999, 40.615164568999894], [-74.08782341799991, 40.6150879359999], [-74.08782337099996, 40.61508779999988], [-74.08782330099994, 40.61508769299994], [-74.08777810199999, 40.61501762099987], [-74.087778008, 40.61501746799991], [-74.08777786599995, 40.61501732399994], [-74.08771657499992, 40.61495785599988], [-74.087617514, 40.614832668999924], [-74.08753556499991, 40.614699614999886], [-74.08747218499994, 40.614560802999925], [-74.08742825199987, 40.61441850099989], [-74.08720982799997, 40.61330697399994], [-74.0869297369999, 40.61349961799993], [-74.08628168999991, 40.613975617999884], [-74.08617531899993, 40.614061606999904], [-74.08546427599995, 40.61458243099993], [-74.08476230899998, 40.6151046959999], [-74.08464853199992, 40.61520502499992], [-74.08404976799996, 40.61562731399987], [-74.08379332799989, 40.61580983899993], [-74.08331319599992, 40.6161634259999], [-74.08323479899997, 40.616219476999945], [-74.08274330499992, 40.616581570999905], [-74.08173992099994, 40.61731079299991], [-74.08110332499999, 40.61778874499986], [-74.08000094899994, 40.618552951999916], [-74.07938896199995, 40.618885794999926], [-74.07805406299991, 40.61958534199986], [-74.07769140399995, 40.61977633599981], [-74.07695604999999, 40.620160559999874], [-74.07626873999996, 40.62051818499985], [-74.0759248129999, 40.6206971299999], [-74.07558011999997, 40.620876468999924], [-74.07461777599994, 40.62138010099991], [-74.0744973829999, 40.62144277399996], [-74.07381368799996, 40.621797074999975], [-74.07253551699989, 40.622464021999896], [-74.0720638059999, 40.6218592669999], [-74.07174504899992, 40.621458933999925], [-74.07134330099989, 40.62093170099999], [-74.07127025999996, 40.6208358429999], [-74.07116568699992, 40.620698785999885], [-74.07114283699993, 40.620549106999874], [-74.071134623, 40.62035994799992], [-74.07114727799993, 40.62016924599988], [-74.07115583099988, 40.62012115699988], [-74.07118111099999, 40.61997902699989], [-74.07123598299995, 40.61979131499989], [-74.07125950599996, 40.61973410799993], [-74.0713113039999, 40.61960813799994], [-74.0714060829999, 40.61943137899991], [-74.07151888099999, 40.61926275799987], [-74.07164799799989, 40.619103771999896], [-74.0718329299999, 40.61890075099991], [-74.0719773749999, 40.6187633899999], [-74.07203694399993, 40.618706741999915], [-74.072258849, 40.61852327399987], [-74.07249719099994, 40.61835175599991], [-74.07255048299993, 40.61831840599996], [-74.07275022399995, 40.61819340999991], [-74.07301605899991, 40.61804928399993], [-74.07329261599995, 40.6179201889999], [-74.0735311079999, 40.61778201399987], [-74.07375937799999, 40.61763293099988], [-74.07397615999992, 40.617473641999915], [-74.07418030999995, 40.61730493399995], [-74.07437079999995, 40.61712770499991], [-74.07454676799989, 40.61694292999989], [-74.07502569899994, 40.61637873899986], [-74.07527690499992, 40.61597807099987], [-74.07538235299992, 40.61580988099994], [-74.07620479299995, 40.61441868999987], [-74.07661374799996, 40.6137287289999], [-74.0768105549999, 40.61339668199991], [-74.07721874699989, 40.612707982999936], [-74.07755680899999, 40.61213759399992], [-74.07784324799991, 40.611655287999895], [-74.07841920399994, 40.61070060799994], [-74.07915097299991, 40.60948761699993], [-74.07939423999989, 40.60908436399989], [-74.07962112599992, 40.6087629369999], [-74.07985980899997, 40.60844643599988], [-74.08011002799992, 40.60813523899987], [-74.08069498099995, 40.607388260999876], [-74.08129735699994, 40.60661901599992], [-74.0813674559999, 40.6065294999999], [-74.08148289099995, 40.60638202899987], [-74.0815547009999, 40.60629029699987], [-74.08280949799995, 40.60641097899988], [-74.08341933399991, 40.60652250899992], [-74.08355131599996, 40.60655198599987], [-74.08406781599992, 40.60668148499991], [-74.084900025, 40.60689013199992], [-74.08542756199994, 40.60702238999989], [-74.08635759599996, 40.60740005899988], [-74.0871642159999, 40.60772760199992], [-74.08796268699999, 40.608051827999915], [-74.0887463859999, 40.60837004599991], [-74.08949370099997, 40.60868693299991], [-74.09045495899996, 40.60909452599991], [-74.09081541099992, 40.60923658599989], [-74.09305685599998, 40.61011993599989], [-74.09410959099992, 40.610406551999894], [-74.09446404899988, 40.6104852189999], [-74.094824891, 40.61054805699994], [-74.09519027099995, 40.61059466399995], [-74.09555829799993, 40.61062477799989], [-74.09592704599994, 40.6106383469999], [-74.09721750699994, 40.61062801799988], [-74.09730455699996, 40.61062732099988], [-74.0977696879999, 40.61062359299988], [-74.09787758299997, 40.61072932199992], [-74.09810312099995, 40.61096151899992], [-74.0984706099999, 40.61135390299989], [-74.0986215139999, 40.61151891199989], [-74.09935042999992, 40.61241907699984], [-74.09987733699994, 40.612918652999866], [-74.10055168499994, 40.61346054699993], [-74.10068375899988, 40.61355314999987], [-74.10127614399993, 40.6139854019999], [-74.10209434299992, 40.6146004909999], [-74.10256802199996, 40.61518744299984], [-74.10302593099988, 40.615781107999936], [-74.10350925299994, 40.61641607499991], [-74.10254940599991, 40.61685368499989], [-74.10189835099996, 40.61745221099992], [-74.10134946699999, 40.61802367099989], [-74.10104903199996, 40.61837105999995], [-74.10073089199992, 40.61870975499989], [-74.10039572599996, 40.619038962999866], [-74.10004426999996, 40.61935795499992], [-74.09967734799997, 40.61966605399991], [-74.09814034099995, 40.62091145599987], [-74.09751734399991, 40.621418887999866], [-74.09639991999997, 40.62232899699995], [-74.09533507499991, 40.62320160699987], [-74.09395327099995, 40.624333906999894], [-74.0922397139999, 40.62574291499989], [-74.0915809599999, 40.6265459219999], [-74.08941665299993, 40.629813551999895], [-74.08934045699996, 40.62992328499993], [-74.08924974899998, 40.63002750999987], [-74.08914567599997, 40.630124621999904], [-74.08902975899994, 40.6302131689999], [-74.08890387699992, 40.63029199099984], [-74.08870635899996, 40.63042749999993], [-74.08852313599994, 40.63057561499994], [-74.08835621399997, 40.63073496399992], [-74.08820728099995, 40.63090392499991], [-74.08807764899994, 40.63108071699992], [-74.08761781699995, 40.63164751199986], [-74.08714129599997, 40.63223951599987], [-74.08574928999992, 40.633955556999865], [-74.08520082199986, 40.63376239799988], [-74.08491321699994, 40.63363992399991], [-74.08344172599995, 40.632275245999914], [-74.08260312199998, 40.63146480799991], [-74.08201443099996, 40.63089152599993], [-74.08140615499994, 40.630301235999866], [-74.0817313539999, 40.63001222599995], [-74.08075821999996, 40.629494042999944], [-74.07972492199995, 40.62895068799986], [-74.08037988999992, 40.62826190099992], [-74.08045421199994, 40.628184620999946], [-74.0805160199999, 40.628099883999866], [-74.08056350699997, 40.62800951999989], [-74.08059556699995, 40.6279157079999], [-74.08061179899994, 40.62782070099993], [-74.08061902199996, 40.62772455599993], [-74.08064690599996, 40.62762867599987], [-74.08069514799999, 40.62753689699988]]]}}, {\"id\": \"115\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 116, \"Shape_Leng\": 0.0681164844265, \"Shape_Area\": 0.000260415337217, \"zone\": \"Hamilton Heights\", \"LocationID\": 116, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93924885699987, 40.82829635099989], [-73.93968952899993, 40.827656573999896], [-73.94013953799991, 40.827038689999874], [-73.94059472699999, 40.826413603999875], [-73.94104840800001, 40.825792629999924], [-73.94150312699996, 40.82517136699994], [-73.94195569699997, 40.82455443999991], [-73.94242953999998, 40.823909156999896], [-73.94292366900005, 40.8232605869999], [-73.9435059449999, 40.82262775299988], [-73.94404986399995, 40.82205159199989], [-73.94460364599989, 40.82146198099991], [-73.94515456999994, 40.82087626499993], [-73.94607828700005, 40.821263215999906], [-73.94723844699998, 40.8217539859999], [-73.94808289799983, 40.82210802199994], [-73.94854029799983, 40.821487622999925], [-73.94992194399984, 40.8220723139999], [-73.95037710599996, 40.82144561799986], [-73.95082880000002, 40.820827936999876], [-73.9529102909999, 40.82169694699989], [-73.95357262899982, 40.82197589199988], [-73.95367103499984, 40.82201753399993], [-73.953769274, 40.822058690999896], [-73.9556966079999, 40.82287647699992], [-73.95573074999992, 40.822891256999874], [-73.95576937299988, 40.822904320999896], [-73.95634010399982, 40.82309232799988], [-73.95651468499986, 40.82312012799985], [-73.95682824599993, 40.82316507899991], [-73.95708264199992, 40.82320154799986], [-73.95715187399986, 40.82321376599985], [-73.95722203099996, 40.82322595899991], [-73.95950897919589, 40.823595788951415], [-73.95953101411843, 40.823619182180664], [-73.95953800225304, 40.823629334087286], [-73.95954284439091, 40.823639168452445], [-73.95954472552476, 40.82364494182928], [-73.95954522180324, 40.82364733759965], [-73.95953351105088, 40.823685018397306], [-73.9595557549387, 40.82369379776886], [-73.95901612941893, 40.824504188499226], [-73.95899582803376, 40.82449642742513], [-73.95725896977147, 40.82711730987312], [-73.95694866420463, 40.827000553935086], [-73.95631784510306, 40.82794421538855], [-73.95620054528379, 40.82790122659748], [-73.95620373779566, 40.827896986099844], [-73.95622394360247, 40.82787008739143], [-73.955039129784, 40.827413431911815], [-73.95469736619506, 40.8279183855142], [-73.95454188062122, 40.8278614763655], [-73.95446469977213, 40.827974169698734], [-73.95446458034706, 40.82797433214995], [-73.95436068010201, 40.82811788301515], [-73.95427220906045, 40.82823654314243], [-73.9542165362577, 40.828307470855506], [-73.95416950644189, 40.82836086924021], [-73.95413649135837, 40.82839340956269], [-73.95406111804229, 40.828462132451484], [-73.95403271717643, 40.82849042349063], [-73.95402106678668, 40.82850298030449], [-73.95398614911781, 40.82854351625173], [-73.95395406407437, 40.82858535816411], [-73.95359128626804, 40.82909772064874], [-73.95297619362613, 40.83009020327107], [-73.95234497932466, 40.83101985864506], [-73.95219759081193, 40.83122045489184], [-73.95214990407374, 40.83127471725009], [-73.95169902461552, 40.8319159967756], [-73.9515984038694, 40.83204708164592], [-73.95140314324226, 40.83232213003093], [-73.95131641118994, 40.83245051197062], [-73.95119348573833, 40.83264713466519], [-73.95109620887555, 40.8328050967018], [-73.95105949632541, 40.83289766190666], [-73.95105224571309, 40.83292243763382], [-73.95099727522978, 40.83311027766788], [-73.95096613500134, 40.833216703289736], [-73.95093576231477, 40.83329399798564], [-73.95066373627958, 40.83363599766629], [-73.95042306202323, 40.83404708748847], [-73.95024424006596, 40.83426741467557], [-73.95015521048293, 40.83439675940791], [-73.94997855399998, 40.83443366399996], [-73.94985590499991, 40.834459286999945], [-73.94977616199992, 40.83445596499997], [-73.94970807799987, 40.83444928999993], [-73.94964119799997, 40.83443432999993], [-73.94957431799986, 40.834419361999934], [-73.94955175300004, 40.83441071599988], [-73.94941570599993, 40.83435858999992], [-73.94919448899992, 40.834210658999886], [-73.94890543299992, 40.83406529899992], [-73.94829108099988, 40.83386118099985], [-73.94592558100007, 40.83280648999992], [-73.94582395199988, 40.8327654609999], [-73.94571933699994, 40.832721412999874], [-73.942979662, 40.8315679799999], [-73.94136799599993, 40.83089083799994], [-73.94034643199987, 40.83045784199987], [-73.94017089799992, 40.830384390999875], [-73.93962569899988, 40.830155836999936], [-73.93899743399992, 40.82988885299995], [-73.93864026699997, 40.82973887399988], [-73.93896034899993, 40.828986329999914], [-73.93924885699987, 40.82829635099989]]]}}, {\"id\": \"116\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 117, \"Shape_Leng\": 0.169885857987, \"Shape_Area\": 0.00090429427072, \"zone\": \"Hammels/Arverne\", \"LocationID\": 117, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7763584369479, 40.609655838242304], [-73.7762653832992, 40.60964903889423], [-73.7761407621011, 40.60965826032536], [-73.77595171894995, 40.609678117323156], [-73.77583557010675, 40.60969112854047], [-73.77547924055104, 40.60974052018876], [-73.77545862875209, 40.60974177710561], [-73.77543801698695, 40.609743023631964], [-73.77539529253983, 40.60974197668977], [-73.77535164745696, 40.60973740696431], [-73.77526446193362, 40.60971801904944], [-73.7752225507697, 40.60970354714687], [-73.77514648392243, 40.60966655761685], [-73.77505226664096, 40.609609001436844], [-73.7749233665848, 40.609520785202946], [-73.77480417881385, 40.60941856747317], [-73.77473640418869, 40.60935819942156], [-73.77465245646789, 40.60929002641901], [-73.77453359848452, 40.6091916818316], [-73.77441360213419, 40.609080924632025], [-73.77426957108304, 40.60894965218348], [-73.77422234411402, 40.6088932316352], [-73.77420239758109, 40.60885865673876], [-73.77417221920565, 40.60880955501728], [-73.77412897394224, 40.60874659586809], [-73.77412946283731, 40.60870682899959], [-73.7741336102761, 40.60869621091815], [-73.77416577139441, 40.60865140098891], [-73.77422269363439, 40.608601128640224], [-73.77427052824257, 40.60856521948805], [-73.7743300451969, 40.608523300849534], [-73.77442552706036, 40.608465016780436], [-73.77445904722425, 40.60844792753528], [-73.7744908031617, 40.60843557121446], [-73.77451194308915, 40.608431821160444], [-73.77453648247197, 40.608432445601345], [-73.77456033784036, 40.60843617603362], [-73.77458795569864, 40.60844210177574], [-73.77461557246363, 40.60844802717402], [-73.77464085996316, 40.60845123703547], [-73.77466166934929, 40.608450917813535], [-73.7746799841398, 40.60844717240926], [-73.77477056234717, 40.6084166322238], [-73.77479725102974, 40.6084108315392], [-73.77480327193534, 40.60840898895146], [-73.77481026628094, 40.60840568858566], [-73.77481441008918, 40.60839956311232], [-73.77481379409207, 40.60839319561505], [-73.77480685406117, 40.608380133811366], [-73.77479766202062, 40.60836608486643], [-73.77475339012696, 40.60826703036626], [-73.77470784811686, 40.60818430112374], [-73.7746963915617, 40.6081687523627], [-73.77468082407886, 40.60815284590764], [-73.77466217243813, 40.608137104116345], [-73.77439705423373, 40.60793574072559], [-73.77437661952841, 40.60791987845193], [-73.77434996612125, 40.60789760031595], [-73.77432745803145, 40.607876367320884], [-73.77430904775353, 40.607855925233544], [-73.77429437865158, 40.60783662516765], [-73.77428088383343, 40.60781602134547], [-73.77426287133888, 40.607782351729895], [-73.77424702088246, 40.60774578625362], [-73.77423322429085, 40.60770256132093], [-73.77421737753347, 40.607636638971414], [-73.77421620346279, 40.60761682411601], [-73.77422245171752, 40.60758264377292], [-73.77423017881631, 40.607565450146836], [-73.77424665012255, 40.607533783895676], [-73.77425539589512, 40.6075193035672], [-73.77427548255106, 40.60749490162597], [-73.77429834320833, 40.60747487429979], [-73.77432541481345, 40.60745694349326], [-73.77435307582272, 40.60744256228131], [-73.77438085592443, 40.607431964281616], [-73.77440850401456, 40.60742523779212], [-73.77443548153934, 40.607421084574284], [-73.77446245971632, 40.60741693252402], [-73.7744829412198, 40.60741538738047], [-73.77450343459158, 40.60741384259156], [-73.77455109867324, 40.60741400863545], [-73.77457386389516, 40.60741537677115], [-73.77458879604322, 40.60741627067692], [-73.77460294599074, 40.607417648882524], [-73.7746211805568, 40.607424105881], [-73.77463059951616, 40.60743066185656], [-73.77463657538321, 40.60743822949263], [-73.7746394828631, 40.60744494408254], [-73.77464146767248, 40.60745208004296], [-73.77464280843147, 40.607468004179324], [-73.77464131316107, 40.60748378722352], [-73.77463415783882, 40.60754541435735], [-73.77464611907904, 40.60760045904278], [-73.77466711273037, 40.60765019097922], [-73.77470777471912, 40.60771575682316], [-73.77473181602394, 40.60774841223412], [-73.77474504280364, 40.60776464731212], [-73.7747638620594, 40.607786584997385], [-73.77478645925184, 40.60780261444577], [-73.77481299320901, 40.60782149692494], [-73.77482414248267, 40.60783689898707], [-73.77482661058232, 40.60785840825176], [-73.77482677634974, 40.60787925595491], [-73.77482689131568, 40.60789411487932], [-73.77482802473858, 40.6079189977522], [-73.77483488501822, 40.60799049571916], [-73.77484441731436, 40.608015944374486], [-73.77484886546486, 40.60802483315967], [-73.77488854943083, 40.60808251659783], [-73.77493877830533, 40.60815984441243], [-73.77497330783932, 40.60820776549682], [-73.77499455012945, 40.60823303083537], [-73.77504484438212, 40.60828776452203], [-73.77505485789824, 40.60829999518134], [-73.7750648720794, 40.60831222533854], [-73.77523851986173, 40.60855094300021], [-73.77526040812863, 40.60857753303507], [-73.77527385543803, 40.60859129323246], [-73.77528731484858, 40.60860505177669], [-73.77531972207748, 40.60863415738225], [-73.77536481707952, 40.60867131076613], [-73.77544786519, 40.60873668961531], [-73.77550407441576, 40.60878228469155], [-73.77552127293168, 40.60879804159957], [-73.77553555532769, 40.60881294578076], [-73.77554297676075, 40.608823253288854], [-73.7755486543443, 40.60883559237239], [-73.77555991525668, 40.60888118139148], [-73.7755665331226, 40.60889839440418], [-73.7755866554815, 40.60893724626726], [-73.77560428285237, 40.60895891142637], [-73.77561965324978, 40.60897033293901], [-73.77565785343113, 40.60899155124125], [-73.77567326241287, 40.60900202745267], [-73.77574188419324, 40.60906012892616], [-73.77583121674111, 40.60914344826377], [-73.7758451335811, 40.60915511937793], [-73.77587386659512, 40.609177274397666], [-73.77590421500724, 40.609196902361134], [-73.77591974593487, 40.609206045409465], [-73.77595185825965, 40.609221470635234], [-73.77595575350477, 40.60922264858929], [-73.77596014736233, 40.609223089212506], [-73.77596253397127, 40.60922302249701], [-73.77596761840643, 40.60922224903923], [-73.77597575572115, 40.60921990491122], [-73.7759813765442, 40.609217421351666], [-73.77599218651525, 40.609210734202485], [-73.7759969268967, 40.60920682542785], [-73.77600096931879, 40.60920272765983], [-73.7760041734712, 40.60919859089884], [-73.77600645602092, 40.60919455235683], [-73.77600772152827, 40.60919075424813], [-73.77600798764608, 40.60918894544779], [-73.77600799341488, 40.609187226103494], [-73.77600774966821, 40.609185577640446], [-73.77600724708519, 40.609184026342746], [-73.77600650791886, 40.60918255700849], [-73.77600550834056, 40.609181195893385], [-73.77596856303008, 40.60914140163102], [-73.77593092580906, 40.60910696494236], [-73.7757849079194, 40.608982344966435], [-73.77572596269108, 40.60892450565263], [-73.7757061698235, 40.6089033676901], [-73.77568626097025, 40.608878807874376], [-73.77562418082805, 40.60879036278403], [-73.77559808590682, 40.60875663253898], [-73.77557032764808, 40.60872590569287], [-73.77553727918631, 40.608693621540894], [-73.77549750814698, 40.608658287943236], [-73.77537471211414, 40.608554153910156], [-73.7753380915165, 40.60851959219094], [-73.77530770397058, 40.60848724029768], [-73.77512849738142, 40.60826077770732], [-73.7750735213425, 40.60819313914054], [-73.77504814770323, 40.60816652365162], [-73.77500343026048, 40.60812609299651], [-73.77497445618741, 40.608098525073515], [-73.77494598484249, 40.60806542027058], [-73.77493177205682, 40.60804058265339], [-73.7749269655384, 40.6080258413621], [-73.77492519696118, 40.60800006414668], [-73.77493055726652, 40.60793791146084], [-73.77492977398721, 40.60792156624607], [-73.7749267509852, 40.60790717808122], [-73.77490855452615, 40.607850905287044], [-73.77489995781174, 40.60783209431492], [-73.7748901347523, 40.607815757697125], [-73.77487268409385, 40.607794191613536], [-73.77482533208136, 40.60774691180897], [-73.77480631263747, 40.60772449611469], [-73.77476028131473, 40.60765729876772], [-73.77474669742094, 40.6076314007238], [-73.77473738535232, 40.60760709537761], [-73.7747310501998, 40.60757961682421], [-73.77472761598476, 40.607550504679345], [-73.77472723631891, 40.60753006231274], [-73.7747299843843, 40.60751034628585], [-73.77473660705792, 40.60748062377299], [-73.7747373966932, 40.60746392965906], [-73.77473443925516, 40.60745064999259], [-73.77472758702018, 40.607436553021984], [-73.77471733297283, 40.60742185364809], [-73.77470197610445, 40.60740331756045], [-73.77468903799695, 40.60738910981529], [-73.77467746725007, 40.60737949623219], [-73.77463538087116, 40.60735104727599], [-73.77462135640792, 40.607340501097255], [-73.77459009469509, 40.60729765709537], [-73.77454551613688, 40.60712210253725], [-73.77458166277762, 40.60693211666177], [-73.77462489159208, 40.60681135146315], [-73.77462803843727, 40.60677873175094], [-73.77462647119515, 40.60674219365964], [-73.77460149071791, 40.60659615185787], [-73.77458447675636, 40.606525967320735], [-73.77460443182908, 40.606477260875565], [-73.77468845087068, 40.60637623456068], [-73.77471597176782, 40.606330001256225], [-73.77473691648497, 40.606272048423904], [-73.77474541830286, 40.60623500033817], [-73.77476161442432, 40.60616947406585], [-73.77477666357314, 40.60612491806986], [-73.77481362381549, 40.60602657104515], [-73.77482760366993, 40.60598385962577], [-73.77483262353243, 40.6059562150156], [-73.77483419700594, 40.60588580675087], [-73.77483934867541, 40.60585394634876], [-73.77484871856116, 40.60582943435121], [-73.77488097814064, 40.60576901801865], [-73.7748923469087, 40.605740412493816], [-73.77489782948433, 40.60571927010758], [-73.77490332458554, 40.60569812774546], [-73.77490994287606, 40.605662327067925], [-73.77491282982935, 40.60562960749049], [-73.77491198709524, 40.60559928566352], [-73.77490728653336, 40.60556725334269], [-73.77489353184065, 40.605504567988696], [-73.77487934789126, 40.60539742312145], [-73.77487577051181, 40.605326625978776], [-73.7748780089713, 40.60529359821665], [-73.77488547725292, 40.60525798800983], [-73.77489825231649, 40.60521900225543], [-73.77490819791973, 40.605195032667474], [-73.77495922863096, 40.60507968442977], [-73.77515942093704, 40.60462937267351], [-73.77517084548859, 40.60459801203555], [-73.77517990205578, 40.60456443168788], [-73.77518635992388, 40.60452057903637], [-73.77518711261327, 40.60450025631843], [-73.77517968909105, 40.604448821765295], [-73.77516723801882, 40.60442360882837], [-73.77514902078904, 40.60440550964398], [-73.775131682203, 40.604396721963944], [-73.77511128119933, 40.60439207129643], [-73.77508384952773, 40.60439080168692], [-73.77505358255998, 40.60439319190189], [-73.77502292796397, 40.60439841896687], [-73.77492667490617, 40.60441799699597], [-73.77489092243015, 40.604421042858455], [-73.77483359721172, 40.604420534189295], [-73.77479068451456, 40.60441700921837], [-73.77475403318755, 40.604409778518], [-73.77472318614407, 40.60439861520081], [-73.77470340449662, 40.604385717019085], [-73.77469424733953, 40.604374983371166], [-73.77468662049279, 40.604355776729086], [-73.77468798964632, 40.60432464838919], [-73.77469363854065, 40.60430696481727], [-73.77470889679802, 40.60427758356298], [-73.77473132437919, 40.60424535244026], [-73.77477178552539, 40.6042015582438], [-73.77481959506719, 40.604161370768445], [-73.77487848447853, 40.60412594254677], [-73.77491373899659, 40.604112513232366], [-73.77496888359173, 40.60410086080547], [-73.77499969776939, 40.60409761527278], [-73.77503248889928, 40.60409612210815], [-73.77506714247416, 40.60409637874052], [-73.77510340854614, 40.60409839473083], [-73.77512871591884, 40.60410233529263], [-73.77515729231767, 40.60411055860527], [-73.77518662764915, 40.60412173744511], [-73.77526249036806, 40.60415396270995], [-73.77528421727511, 40.60416158741617], [-73.77530321052191, 40.60416664976927], [-73.77533426151207, 40.604169915815966], [-73.77536062624208, 40.60416508654276], [-73.7753760831684, 40.60415730886782], [-73.77540696318981, 40.60413383850895], [-73.77544367157964, 40.60409578217126], [-73.77546430298095, 40.60407040120036], [-73.77548575925312, 40.60403914958894], [-73.77555704898475, 40.603930928325845], [-73.77559061167692, 40.603876161419], [-73.77567242794629, 40.60372599580607], [-73.77570186438697, 40.603682387834354], [-73.77577763204935, 40.60358478331456], [-73.7758547544727, 40.60350560663005], [-73.7758814058216, 40.603471681753405], [-73.7758950323549, 40.60344572766287], [-73.77590501110117, 40.6034155619671], [-73.7759111492324, 40.60338546065215], [-73.77592245363391, 40.60330904545656], [-73.77593307894469, 40.60326907366642], [-73.77594998950397, 40.60322961939753], [-73.77598887611845, 40.60315477125693], [-73.77600346476142, 40.603123794982096], [-73.77601557930303, 40.60309098462737], [-73.77602243778801, 40.603060840314285], [-73.77602601326558, 40.60302704114226], [-73.77603171304683, 40.60293396491081], [-73.77605391761067, 40.6027528944022], [-73.77604574101473, 40.60273907369198], [-73.77602911188818, 40.60273393498305], [-73.77600401563208, 40.60273422850361], [-73.7759507985543, 40.602745525162575], [-73.77591015473361, 40.602759629315685], [-73.77587304594194, 40.60277655065808], [-73.77584590981601, 40.60279337333101], [-73.77583413094258, 40.60280384156571], [-73.77581534137109, 40.602829541823716], [-73.77580804561218, 40.60284516045268], [-73.77579852667681, 40.602875669469306], [-73.77578265111752, 40.602948581740016], [-73.77577095277304, 40.602980742867786], [-73.77575502513491, 40.60300566533067], [-73.77572546382092, 40.60303001185737], [-73.7756891989242, 40.60304220643062], [-73.77565936175343, 40.60304698393531], [-73.77563285731395, 40.60304806704136], [-73.7756175817329, 40.60304742506565], [-73.77560642181831, 40.603045836539785], [-73.77559683616948, 40.603043449122254], [-73.77558895434596, 40.603040299587875], [-73.77558293018475, 40.60303646044293], [-73.7755806102369, 40.60303428590139], [-73.77557740048579, 40.603029444393314], [-73.7755761648483, 40.60302104891316], [-73.77558060876527, 40.602992546224506], [-73.77559843811993, 40.602936056706156], [-73.77561158536422, 40.60290883133475], [-73.77564545195459, 40.60285830634876], [-73.77568814362705, 40.60281474067054], [-73.77572593568114, 40.60278406218195], [-73.77577486968538, 40.602746749801895], [-73.7758674827302, 40.60267423096543], [-73.77590052116638, 40.60264474869203], [-73.77593843827076, 40.60260871194167], [-73.7759763667971, 40.60257266531682], [-73.77600933049506, 40.602544596984025], [-73.77601699635197, 40.602538074917675], [-73.77603607413987, 40.60252495565892], [-73.77605793555121, 40.60251331742217], [-73.77608197645208, 40.60250345940946], [-73.77610751118806, 40.602495673795495], [-73.77613377079997, 40.60249018759194], [-73.77616006838659, 40.602487141023076], [-73.77617290098054, 40.602486868483275], [-73.77618574390479, 40.60248659596206], [-73.77621025347722, 40.602488581003094], [-73.77622492564387, 40.60249261587597], [-73.77623203834975, 40.60249616007703], [-73.77623565675155, 40.602498502987906], [-73.776255921844, 40.602509123909435], [-73.77627825768181, 40.6025169696893], [-73.77630201637034, 40.60252181274498], [-73.77632650874143, 40.60252351259417], [-73.77635102434505, 40.60252201992921], [-73.77637485205688, 40.60251737804779], [-73.77639730070666, 40.60250972159692], [-73.77641617364749, 40.602500205457986], [-73.77643286433131, 40.60248855887899], [-73.77644696287877, 40.60247506787019], [-73.77647924254995, 40.60244817664342], [-73.77652001305698, 40.60240753106775], [-73.77655507698655, 40.602363902337125], [-73.77658406174028, 40.60231775407642], [-73.77660893153029, 40.60226367841057], [-73.77662545802484, 40.602207778288054], [-73.77661833448919, 40.60220195088371], [-73.77661006915427, 40.602197077395], [-73.77660087925568, 40.60219328591058], [-73.77659100632879, 40.602190676080745], [-73.77658070986051, 40.60218931649889], [-73.77657026047001, 40.60218924289839], [-73.77656852620001, 40.602194348208926], [-73.7765657167677, 40.60219916756192], [-73.77656190827142, 40.60220357041622], [-73.7765572038716, 40.60220743751217], [-73.77655173099527, 40.602210664102216], [-73.77654563788592, 40.602213162788054], [-73.77653908958678, 40.60221486588792], [-73.776532263471, 40.602215727270355], [-73.77652534443668, 40.6022157236029], [-73.77651851989935, 40.60221485498513], [-73.77651197471418, 40.60221314494505], [-73.7765056466287, 40.60221051998804], [-73.77649999949183, 40.602207110046834], [-73.77649519915138, 40.60220301526591], [-73.77649138658535, 40.602198355902964], [-73.77648867376318, 40.602193268796206], [-73.77648714035558, 40.602187903346156], [-73.77648683139624, 40.60218241712754], [-73.7764877559585, 40.602176971262125], [-73.77648988688918, 40.60217172568651], [-73.77649883959204, 40.60216217866229], [-73.77651289165587, 40.60215025609919], [-73.77652982761255, 40.60213815971758], [-73.7765455725766, 40.602128725744386], [-73.77655976662355, 40.602120215862925], [-73.77660496626646, 40.60209312573129], [-73.7766352964549, 40.602071509529836], [-73.77665458669364, 40.602054859726444], [-73.77667042929866, 40.602041193959494], [-73.77670556145134, 40.602010878377094], [-73.77672850763598, 40.60198927460886], [-73.77676002871895, 40.60195776286881], [-73.77678810455096, 40.60192490385656], [-73.77684680922381, 40.60184460932285], [-73.77687575407889, 40.601809635239206], [-73.77692438118366, 40.60175726487803], [-73.77696014499182, 40.60172188183386], [-73.77699505209678, 40.60169160073948], [-73.77702968356833, 40.601666182988], [-73.77733632025031, 40.60147266976104], [-73.77736657905773, 40.60145068432779], [-73.77740272820665, 40.601416686357126], [-73.77742563726459, 40.60139184905503], [-73.7774543929935, 40.601357085211205], [-73.77746362163673, 40.601339060011156], [-73.77747583241022, 40.60132209639769], [-73.77749082004738, 40.601306479530905], [-73.77751446064458, 40.60128834417351], [-73.77754181531016, 40.60127355034078], [-73.77757208129815, 40.601262532169315], [-73.7776043704283, 40.601255612995764], [-73.77762215646955, 40.60124166312658], [-73.77763689332879, 40.60122578195794], [-73.77764822536693, 40.60120835274635], [-73.77765587911384, 40.601189796106425], [-73.77765961418032, 40.60115214903175], [-73.77767142267312, 40.6011137956808], [-73.77769101593326, 40.6010773407748], [-73.77770514895494, 40.60105817691085], [-73.77772144347017, 40.601040037332275], [-73.77775981349949, 40.60100173391062], [-73.77780302364842, 40.600966552196674], [-73.77785063938293, 40.60093484598243], [-73.77790720295133, 40.600897492664764], [-73.77796667277282, 40.600862860979], [-73.77802882379063, 40.600831081982044], [-73.7780398057688, 40.600820882962886], [-73.77805276670358, 40.60081213695798], [-73.77806738001723, 40.600805064340626], [-73.77808327749685, 40.6007998433199], [-73.77810005857332, 40.60079660544993], [-73.77811730041341, 40.60079543231548], [-73.77813456857402, 40.60079635347585], [-73.77815142794938, 40.600799345720645], [-73.778167453733, 40.60080433365425], [-73.7783571476974, 40.600845100435365], [-73.77835997267935, 40.60084449143691], [-73.77836261812291, 40.60084352113981], [-73.77836500516919, 40.600842218467655], [-73.77836706266203, 40.60084062225229], [-73.77836872926964, 40.600838780075726], [-73.77836995531133, 40.600836746851705], [-73.77837070423969, 40.60083458318917], [-73.7783709537299, 40.60083235358523], [-73.77837069634516, 40.60083012450277], [-73.77836993975761, 40.6008279623889], [-73.77836870652085, 40.600825931694644], [-73.7783670333964, 40.60082409295341], [-73.77836497025906, 40.6008225009767], [-73.77836257860959, 40.600821203219844], [-73.77835992974104, 40.60082023836825], [-73.77815341199033, 40.6007549801212], [-73.77813381914017, 40.60074807937766], [-73.77811584657964, 40.600738969936295], [-73.77809991955218, 40.60072786733307], [-73.77808641490137, 40.600715034263494], [-73.77807565215487, 40.60070077436665], [-73.77807854866104, 40.60067913110666], [-73.77808016555346, 40.60065470270799], [-73.77808041391145, 40.600576277525946], [-73.77807471168266, 40.60053260761391], [-73.77807640267658, 40.60051418663679], [-73.77809077931204, 40.6004828494937], [-73.77812145153689, 40.60043988994484], [-73.7781382332667, 40.60042034482642], [-73.77815821309494, 40.600402085303], [-73.77818086978304, 40.600385541085764], [-73.77820562116898, 40.60037115450053], [-73.77821809226897, 40.60036577456361], [-73.77823203524598, 40.60036173971335], [-73.77826431381608, 40.600357686965275], [-73.77830243560074, 40.600358985823554], [-73.77834715405771, 40.60036563656279], [-73.77836673971015, 40.60037015876991], [-73.77838774899787, 40.600376458025636], [-73.77847044329695, 40.600407405892824], [-73.77850756206627, 40.60041861698767], [-73.77871486318202, 40.600467609314826], [-73.7788669679822, 40.60049874430103], [-73.77898728659612, 40.6005201382651], [-73.77910890171088, 40.60053958019397], [-73.77913417165485, 40.60053988072603], [-73.77918989225739, 40.600545940797375], [-73.77924725759344, 40.600555055553784], [-73.77930686918678, 40.60056730924248], [-73.77936988436021, 40.60058293728319], [-73.77942452180807, 40.60059833292931], [-73.7794547259279, 40.60060761290144], [-73.77952429814611, 40.600629934858055], [-73.77958486504491, 40.60064998874053], [-73.77968480144577, 40.600683086206224], [-73.77983460397324, 40.60072841664939], [-73.77986876620307, 40.60073691251733], [-73.77990294207429, 40.600745406726], [-73.77993422228535, 40.600750598396104], [-73.77996753324919, 40.600752499977816], [-73.78000237945831, 40.60075107131856], [-73.7800740054335, 40.6007383945677], [-73.7801090008511, 40.600727429033746], [-73.78017210547576, 40.60069783284526], [-73.78025665342628, 40.600650374903964], [-73.78031928742601, 40.60060939459747], [-73.78042062181392, 40.60050748412532], [-73.78061842937426, 40.600308551696564], [-73.78069401326346, 40.60021683274359], [-73.78072266528144, 40.60016579288055], [-73.78073981985993, 40.60011544960967], [-73.78073270457962, 40.6000550020549], [-73.7807552715078, 40.600004183271594], [-73.78081083231571, 40.59994775402446], [-73.78087073851444, 40.599916908449245], [-73.78101073372724, 40.59986097311825], [-73.78110356332841, 40.5998165825566], [-73.78116628337473, 40.59977094663728], [-73.78121212475874, 40.59972363005979], [-73.78131649934976, 40.599631407796394], [-73.78137308926598, 40.59959197377427], [-73.78143367080311, 40.59954626997921], [-73.78147774223326, 40.59948736072947], [-73.78152003742214, 40.599407978473316], [-73.78154063755586, 40.59935143783998], [-73.78156560000286, 40.59931942664422], [-73.78158957195356, 40.59929725676646], [-73.78160801552015, 40.59928255054644], [-73.78163488736334, 40.5992625908602], [-73.78165622085076, 40.59925637265338], [-73.78165884725445, 40.59925639523394], [-73.78171657592806, 40.599237290590494], [-73.78177096037318, 40.59921314893261], [-73.7818212474255, 40.59918430459706], [-73.78186674066829, 40.599151157047686], [-73.78190681007644, 40.59911416534232], [-73.78194090074186, 40.59907384177559], [-73.78197067852172, 40.59902678668573], [-73.78199231614101, 40.59897721506158], [-73.78200218336956, 40.598942525597096], [-73.78200805391967, 40.598907313339055], [-73.78200967919355, 40.59886215896594], [-73.78201130686442, 40.59881694294629], [-73.7819913320746, 40.59868213228944], [-73.78193601357016, 40.5985311442433], [-73.78190559187668, 40.59847002881388], [-73.78189405690782, 40.59843640959525], [-73.7818778837792, 40.59840391984822], [-73.7818572569845, 40.59837293018506], [-73.78183226542883, 40.59834364177731], [-73.78180869810592, 40.59832117980023], [-73.78178262043288, 40.598300392040336], [-73.78159013310601, 40.59813269856015], [-73.78151695888268, 40.598041560857666], [-73.78149557370479, 40.59801681040708], [-73.78145229184497, 40.59798997277521], [-73.7814425498352, 40.59799193545415], [-73.78141927498908, 40.597999546456386], [-73.78141092338406, 40.59799941313283], [-73.78138790044059, 40.59799123811753], [-73.7813770762337, 40.5979850038498], [-73.781358744408, 40.597969371289565], [-73.78134045293935, 40.597912693023375], [-73.78134197798609, 40.597778878399126], [-73.7813297399812, 40.597753037446914], [-73.78110523555996, 40.597598484223674], [-73.78093054006914, 40.59748733362211], [-73.78061773744298, 40.59732607457592], [-73.7805956871336, 40.59731339156522], [-73.78057109963153, 40.5973037943738], [-73.78054471043457, 40.59729757008672], [-73.78051730893495, 40.597294904893886], [-73.78049749982736, 40.597296880877465], [-73.78047753762309, 40.59729748645169], [-73.78045026422004, 40.597296087476344], [-73.78042342543422, 40.597292135036035], [-73.78039361984408, 40.597284509030494], [-73.78036553370627, 40.597273739041945], [-73.7802367475551, 40.59723281645072], [-73.78018334426079, 40.597212569703295], [-73.78016742223146, 40.59720085941007], [-73.78015218615221, 40.5971778667985], [-73.78014720242385, 40.59716323342814], [-73.78014410214381, 40.59714675698096], [-73.78015037078755, 40.59705837242387], [-73.78015983766757, 40.59698901447003], [-73.78017115365934, 40.59692870146343], [-73.78017065145609, 40.59691234692223], [-73.78016845573181, 40.596897627246925], [-73.78016314495001, 40.596885207826446], [-73.78015594130338, 40.596880565702286], [-73.78014537451874, 40.596878474819675], [-73.7801013698854, 40.59687809309132], [-73.78009116767264, 40.596876416867374], [-73.78008143556367, 40.596871940612694], [-73.78006730392137, 40.59685824331076], [-73.78005493695551, 40.596861344908895], [-73.78004769901891, 40.596870985123076], [-73.78004458246923, 40.596884612056115], [-73.78004452317522, 40.59688866766952], [-73.78003922252748, 40.59694067031956], [-73.78002556782963, 40.59699178065659], [-73.7800037665822, 40.59704122192325], [-73.77997415009615, 40.5970882427268], [-73.77993716845845, 40.5971321284582], [-73.77989338369284, 40.59717221215324], [-73.77988394639937, 40.597178767440646], [-73.77981453577355, 40.59724086137078], [-73.77979724995491, 40.59725237288961], [-73.77975178148083, 40.597276130929465], [-73.77972702901246, 40.59729462584321], [-73.77972417563883, 40.59733937660601], [-73.7797184031111, 40.59735254828653], [-73.77968112274064, 40.597372450884386], [-73.77959238264599, 40.59741005749713], [-73.7795467449914, 40.59743287964549], [-73.77944343306667, 40.59748454291393], [-73.77937174240404, 40.59751761689244], [-73.77931746547935, 40.597540070048304], [-73.77915495057023, 40.59760103954861], [-73.77906643551208, 40.597638006057025], [-73.77904364405249, 40.597649399323544], [-73.77902413723443, 40.59766418484445], [-73.77901338422856, 40.597675700095444], [-73.77900398445954, 40.59768866657711], [-73.77898263163226, 40.59771873110344], [-73.77898142408172, 40.59773041853677], [-73.7789828491783, 40.597742091725465], [-73.77898686516694, 40.59775340863895], [-73.7789933543786, 40.59776403768566], [-73.77899559921607, 40.597765588777115], [-73.77899748054092, 40.59776739912647], [-73.77899894756466, 40.59776941986163], [-73.77899996068378, 40.59777159643082], [-73.77900050836284, 40.59777400681979], [-73.77900049949349, 40.597776453083824], [-73.77899993434409, 40.59777886112848], [-73.77899883003234, 40.59778115801702], [-73.77899722000666, 40.597783274179385], [-73.77899545189929, 40.597784914299424], [-73.77899337992558, 40.597786332755604], [-73.77876825301163, 40.59796552577206], [-73.77876452167372, 40.597966343462474], [-73.77876068004736, 40.59796677165174], [-73.77875679777884, 40.59796680257679], [-73.77875205386624, 40.597966292568785], [-73.77874748621518, 40.597965190882256], [-73.7787432209162, 40.59796352792929], [-73.77873937571361, 40.59796134961628], [-73.77872496402216, 40.59795521299957], [-73.77871154174142, 40.59794788778284], [-73.77868535376176, 40.59795736443183], [-73.77863586418795, 40.59797155202583], [-73.77861593382974, 40.59797866329838], [-73.77860198346538, 40.59798548084939], [-73.77858816631479, 40.5979944775467], [-73.77858115374502, 40.59800014630816], [-73.77855757266263, 40.59802139842021], [-73.77854903824989, 40.59803337676645], [-73.77853670204476, 40.59806270101494], [-73.77853278398769, 40.598068583306144], [-73.77852819473152, 40.59807346398388], [-73.77851179227486, 40.59808582353919], [-73.77849136266016, 40.59809713178735], [-73.77846656618118, 40.59810762897902], [-73.77843466744606, 40.598118490252], [-73.77840574320462, 40.59812625979506], [-73.77834122816843, 40.598140211891916], [-73.77831588651237, 40.59814722153905], [-73.77828844723506, 40.598156543231674], [-73.7782659814658, 40.59816546993], [-73.77824654067494, 40.59817477018871], [-73.77822973160433, 40.59818466137417], [-73.77820861784828, 40.59819956997295], [-73.7781889408424, 40.59821559675375], [-73.77817093945065, 40.59823255622041], [-73.7781547647767, 40.598250282810874], [-73.7781483232691, 40.59825915581627], [-73.7781100087146, 40.59831192707847], [-73.77809083914629, 40.59833555631479], [-73.77807747589547, 40.59835011866123], [-73.77806376669655, 40.59836236641773], [-73.77804964009295, 40.5983721094675], [-73.77803567464659, 40.59837952098092], [-73.77801915791676, 40.59838660305958], [-73.7779610831601, 40.598406635644004], [-73.7779399558411, 40.59841494332945], [-73.77786310256693, 40.59844737564203], [-73.77784156447136, 40.59845818473893], [-73.77781414448636, 40.59847591701703], [-73.77780489089355, 40.59848408479449], [-73.77773188259499, 40.5985681612952], [-73.77767796085797, 40.5986146769899], [-73.77762405113403, 40.59866119268243], [-73.77752065324304, 40.59872484063874], [-73.77747337927345, 40.59876188575304], [-73.7774217481076, 40.598812404361425], [-73.77740175443988, 40.59883839137449], [-73.7773860641275, 40.59886697069832], [-73.77737230109213, 40.59891276307381], [-73.77736943709169, 40.598928237101994], [-73.7773692598685, 40.59893063243328], [-73.77735188319646, 40.59916508706544], [-73.77731478794612, 40.59924620544075], [-73.77730957913344, 40.59927415722556], [-73.77730277625054, 40.599351183073196], [-73.77729516944352, 40.59937579804465], [-73.7771985062932, 40.599576959253014], [-73.77718611055549, 40.599595143687054], [-73.77714992395396, 40.59963759607435], [-73.7771302938322, 40.59965359646174], [-73.77706440746492, 40.59969139973547], [-73.77703664882237, 40.599718252658896], [-73.77701942427383, 40.59973918407221], [-73.77698813377805, 40.599787094308326], [-73.77694848707488, 40.59987827614478], [-73.77694612101355, 40.59988281914574], [-73.77693419349195, 40.5999057502418], [-73.77677517850812, 40.60016512656038], [-73.77659982572908, 40.60049065852063], [-73.77654724474655, 40.6005664167483], [-73.77649688161841, 40.60063645142544], [-73.77631885136726, 40.600920643647555], [-73.77631782727785, 40.60094080502258], [-73.77631290592502, 40.600977852503604], [-73.77628516667868, 40.601033892467534], [-73.7762692735167, 40.6010558062395], [-73.77624760401717, 40.60107755781903], [-73.77623171270285, 40.60108474790965], [-73.77619005997714, 40.60109354620841], [-73.77616879063572, 40.60110161020604], [-73.77616693234631, 40.60110230820097], [-73.77615613306466, 40.601116758882455], [-73.77615621270874, 40.60112132472349], [-73.7761566439831, 40.601147603261595], [-73.77615216857947, 40.6011574829831], [-73.7761000802084, 40.601209322580054], [-73.77607314799916, 40.60123613298599], [-73.7760333108812, 40.60128149560171], [-73.7760108211145, 40.60131151999539], [-73.77600256916033, 40.60132342683074], [-73.77600206164578, 40.6013341075328], [-73.77600848335044, 40.60135293128455], [-73.77601411201164, 40.60136940338431], [-73.77601141422332, 40.60137772803619], [-73.77599890906717, 40.601393345907695], [-73.77598284829236, 40.60140875899457], [-73.77596308986715, 40.60142411059151], [-73.77593904118956, 40.60143986778643], [-73.77590955359184, 40.60145662273234], [-73.77580714550358, 40.601512453299506], [-73.77574683385451, 40.601561063642194], [-73.77567846230677, 40.60160655925803], [-73.77559846587381, 40.601648052803284], [-73.77556183167897, 40.60165759847418], [-73.77551653577505, 40.601657321201564], [-73.77545258302268, 40.6016500010345], [-73.77541205144493, 40.601648480912594], [-73.77538628853989, 40.60165052971063], [-73.77535990528, 40.601657961023626], [-73.77529277155317, 40.60168286457615], [-73.77527009492586, 40.60169127545776], [-73.77524796667733, 40.601697860831585], [-73.77522740650352, 40.60170227758095], [-73.77511292814287, 40.60172105484165], [-73.7749751364952, 40.60174817941184], [-73.77497335885174, 40.601736026847725], [-73.77496574936545, 40.60170788757113], [-73.7749507282859, 40.60167596699738], [-73.77492950702356, 40.601646161332646], [-73.77490258024206, 40.601619165329964], [-73.77487057559526, 40.601595608249845], [-73.77485941843604, 40.60157362382938], [-73.77485327577887, 40.60155052161247], [-73.77485211519904, 40.60153792815272], [-73.77485245260944, 40.60152530628266], [-73.77485721562444, 40.60150157247189], [-73.77487610311192, 40.601464290997676], [-73.77488743205231, 40.60142527056756], [-73.77489094557998, 40.6013853959533], [-73.77488656404168, 40.60134557129298], [-73.77487438680039, 40.60130669959067], [-73.77485468998128, 40.6012696622409], [-73.77482792021073, 40.60123529904385], [-73.77481555470831, 40.601222664689914], [-73.77478343828261, 40.60118050240465], [-73.77475569154595, 40.60113657488152], [-73.77473248035847, 40.601091144716946], [-73.7747124639902, 40.6010400982236], [-73.77471039107347, 40.60103372401186], [-73.77471009679886, 40.6010298065911], [-73.77471054782121, 40.60102570182557], [-73.77471176740971, 40.601021353135664], [-73.77471611047916, 40.60101232091053], [-73.7747324443184, 40.60098586816184], [-73.77473418477746, 40.60098153154965], [-73.7747421272563, 40.60096342643765], [-73.77461657884193, 40.60097065110737], [-73.77461980021181, 40.60099674347047], [-73.77460398507942, 40.600997727516564], [-73.77460211334092, 40.60098020375983], [-73.77448133964737, 40.60098770385082], [-73.7744795424717, 40.60097089006147], [-73.77452504422452, 40.60096806382276], [-73.77452342169408, 40.60095290104663], [-73.77460096410347, 40.60094808556443], [-73.77459908825217, 40.600930543706205], [-73.77461755485552, 40.60092939800124], [-73.77462062960322, 40.6009581651908], [-73.77474724721814, 40.60094979034324], [-73.77475698466166, 40.600915562651494], [-73.77476252178143, 40.60088079472821], [-73.77476381099754, 40.600845785370645], [-73.77464113375633, 40.600850796763226], [-73.7746403362974, 40.600837656349505], [-73.77476313172296, 40.60083174488498], [-73.7747564130924, 40.600787305587104], [-73.77474285023148, 40.600743783106346], [-73.77472263330482, 40.60070178762814], [-73.77469604576548, 40.60066190792817], [-73.77466346038042, 40.60062470311798], [-73.77463690395614, 40.600597317739656], [-73.77461553550671, 40.60056794358526], [-73.7745993150754, 40.60053671437618], [-73.77459299222583, 40.600519869765186], [-73.77458859552836, 40.6004962729264], [-73.77458914133618, 40.60047244293528], [-73.7745930697114, 40.600453818324034], [-73.77459228166806, 40.600440679944526], [-73.77458861595133, 40.60042782805817], [-73.7745821762568, 40.60041562621356], [-73.77457314474957, 40.600404419570985], [-73.77456177690955, 40.60039452513893], [-73.77454839430636, 40.60038622280626], [-73.77453337550058, 40.60037974742551], [-73.77449368157664, 40.600360688598315], [-73.77448414163507, 40.60035522614953], [-73.77447534678542, 40.60034908000434], [-73.77446427155027, 40.600339263844155], [-73.77445509082497, 40.60032837109524], [-73.77444798230009, 40.6003166125852], [-73.7744430835592, 40.6003042158982], [-73.77444333608145, 40.60029027291197], [-73.77444042075112, 40.60027650679024], [-73.77443502469914, 40.600264373402375], [-73.77442716608316, 40.60025304839728], [-73.7744270542206, 40.60025291482401], [-73.77441643984454, 40.60024230431037], [-73.77440366709074, 40.60023317982313], [-73.77438317080079, 40.6002234910496], [-73.77437018193261, 40.60021536680437], [-73.77435842277049, 40.60020621890213], [-73.7743125982149, 40.60017550972069], [-73.77426438292592, 40.60014700647956], [-73.77426028243998, 40.6001447454859], [-73.77425173334203, 40.60013871898036], [-73.77424462061073, 40.60013169128236], [-73.77423914296601, 40.60012385873662], [-73.77423545344563, 40.600115440173965], [-73.77423365512885, 40.600106670797814], [-73.7742337982574, 40.60009779561279], [-73.77423587883177, 40.60008906257942], [-73.77423983872315, 40.600080715686495], [-73.77424077337095, 40.60006393016101], [-73.77423820633749, 40.60004724385151], [-73.77423220250277, 40.60003107847646], [-73.77422291360617, 40.60001584258892], [-73.77421057441094, 40.600001921249756], [-73.77418396402193, 40.59998249666926], [-73.77415850996134, 40.59995729156492], [-73.77414111420349, 40.599933421066744], [-73.7741281796629, 40.59990795710509], [-73.77411995097059, 40.59988138128994], [-73.77410967950233, 40.5998406212787], [-73.77410594774378, 40.59979921375171], [-73.77410881041702, 40.59975776603781], [-73.77411822552256, 40.59971688605384], [-73.7741340549569, 40.59967717338902], [-73.77415219771076, 40.599645056515634], [-73.7741745395031, 40.5996145159247], [-73.77417488660657, 40.599613264992406], [-73.77417494940711, 40.5996121128465], [-73.77417488005402, 40.599611563212804], [-73.77417472840716, 40.59961104089229], [-73.77417450615069, 40.599610535353555], [-73.77417421236633, 40.59961005832185], [-73.77417384731483, 40.59960959756784], [-73.77417341161646, 40.59960916465262], [-73.77417231607093, 40.59960835282883], [-73.77417091211092, 40.599607620478054], [-73.77416922544775, 40.59960696882382], [-73.7741649638547, 40.59960588842678], [-73.77415934273652, 40.59960503956458], [-73.77413094827763, 40.59960295866264], [-73.77411967068737, 40.59960125181844], [-73.77411709754821, 40.599600553858046], [-73.77411490366404, 40.5995997562928], [-73.7741130859808, 40.59959885241541], [-73.77411161370193, 40.599597849871834], [-73.77411101198963, 40.599597308573415], [-73.77411049523876, 40.599596731088006], [-73.77411007173556, 40.59959613686533], [-73.77410973033992, 40.59959550544495], [-73.7741093214267, 40.599594171774925], [-73.77410924166621, 40.599593450235304], [-73.7741092441623, 40.599592712439396], [-73.77411412164395, 40.59956539089597], [-73.7741348041225, 40.59950379033323], [-73.77414082969473, 40.59947584079814], [-73.774141689022, 40.59945941703737], [-73.77414097253451, 40.5994089238521], [-73.77414328371775, 40.599393106218976], [-73.77414785481132, 40.59937929182412], [-73.77415246595795, 40.59937111471465], [-73.77416455981803, 40.59935488374408], [-73.77416909783773, 40.59934766174467], [-73.77417762359391, 40.59932786760801], [-73.77418426304358, 40.59930361045845], [-73.77418446850707, 40.59930206238519], [-73.77418439064002, 40.599300585960385], [-73.77418401798765, 40.59929918870057], [-73.77418336140357, 40.599297845832496], [-73.77418241995224, 40.59929657410738], [-73.7741811840228, 40.59929535574826], [-73.77417965160431, 40.59929420046784], [-73.77417781177027, 40.59929308964871], [-73.7741733892158, 40.59929106289546], [-73.77416761842694, 40.59928911541469], [-73.77413993625021, 40.59928221708981], [-73.77413389337684, 40.59928039589266], [-73.7741290093149, 40.59927855000033], [-73.77412426793614, 40.59927610764756], [-73.77412239338659, 40.59927479907428], [-73.7741208394749, 40.59927343567882], [-73.77411960342737, 40.599271992829], [-73.77411868741596, 40.599270478067844], [-73.77411806616315, 40.59926889201569], [-73.77411776434845, 40.599267215957894], [-73.77411815160367, 40.59925045805245], [-73.77412213675348, 40.599232285394066], [-73.77412969411935, 40.599212361364714], [-73.77414205349845, 40.59918747767375], [-73.77418482152235, 40.599117618568314], [-73.77420518646193, 40.59907998885753], [-73.77421643846336, 40.59905413951923], [-73.7742228395081, 40.59903030574869], [-73.77422411746566, 40.59901915882223], [-73.77422411780901, 40.59900839755201], [-73.7742238871052, 40.59900675078433], [-73.7742233845751, 40.59900521874304], [-73.77422263379135, 40.599003785056546], [-73.77422161057754, 40.59900244967746], [-73.77422031496025, 40.599001204731586], [-73.77421874732259, 40.59900006697305], [-73.77421476133972, 40.59899802415852], [-73.7742102064766, 40.59899645668075], [-73.77420449275506, 40.598995059313836], [-73.77417887099674, 40.59899075840607], [-73.77416837527039, 40.59898827894795], [-73.77416344285437, 40.59898650432967], [-73.77415942170818, 40.5989844614441], [-73.7741577364582, 40.598983331662495], [-73.77415626354211, 40.59898213143316], [-73.77415499093769, 40.598980841131294], [-73.77415392103025, 40.59897947047851], [-73.77415118559153, 40.598974584292236], [-73.77414910150786, 40.59896881717438], [-73.77414522234422, 40.59894923135865], [-73.77414330739184, 40.59894245621191], [-73.77414040909038, 40.598936336183854], [-73.77413858788456, 40.59893368496501], [-73.77413648082494, 40.59893126705587], [-73.77412070066742, 40.59891614295413], [-73.77410258240812, 40.598900825783346], [-73.77405725561275, 40.598867948431845], [-73.7740085544746, 40.598837405486954], [-73.77395500546103, 40.59880805099072], [-73.77394465922117, 40.5988033661098], [-73.7739324708673, 40.59879902841154], [-73.773879145257, 40.598783876483346], [-73.7738680182614, 40.59877960452776], [-73.77385846405322, 40.598774946997125], [-73.77383620507035, 40.59876126041149], [-73.77382748458088, 40.59875476822587], [-73.77381454646792, 40.59874513426274], [-73.77379337887521, 40.598726479545164], [-73.77377245684356, 40.59870508653122], [-73.77374432839089, 40.59867015398153], [-73.77372538853857, 40.59862849443554], [-73.77371225023631, 40.598409325955075], [-73.77371011299202, 40.59838456571208], [-73.77370141351436, 40.59832306115278], [-73.77370149938109, 40.59831491083325], [-73.77370324567592, 40.59830541449905], [-73.77371648942128, 40.59826692432115], [-73.77371858470356, 40.59825873356289], [-73.77371949436639, 40.598251738307574], [-73.77371901244801, 40.59824412933134], [-73.77371810271825, 40.598240767898915], [-73.77371671887805, 40.59823767559087], [-73.77371486164249, 40.59823483599077], [-73.77371253123867, 40.59823224692098], [-73.77370971541684, 40.5982298912694], [-73.77367405492255, 40.598189766315755], [-73.77363244061033, 40.598153123466695], [-73.77358545219488, 40.5981204731734], [-73.77353374425259, 40.59809227026811], [-73.77348642674832, 40.59807746915618], [-73.7734369219436, 40.59806766990313], [-73.77338613523759, 40.59806305172887], [-73.77333499547422, 40.59806369909571], [-73.77329921604125, 40.59806165278117], [-73.77326423243323, 40.59805557765264], [-73.77323082101059, 40.59804560852997], [-73.77319972324258, 40.59803196664905], [-73.77317162925218, 40.5980149547521], [-73.7731471625014, 40.59799495036838], [-73.77312844612996, 40.597974438100294], [-73.77311355698103, 40.597952178698215], [-73.77310277082675, 40.59792858445023], [-73.7730730681773, 40.597890776494424], [-73.77307026659959, 40.597887861975956], [-73.77306706257957, 40.597885081859154], [-73.77305945949531, 40.59787999706022], [-73.77305043595175, 40.5978756216245], [-73.77304022999472, 40.59787207932362], [-73.77302907446409, 40.59786947331306], [-73.77301747820698, 40.59786791935512], [-73.77300592490842, 40.59786749965713], [-73.77299489981293, 40.59786822556462], [-73.77296826720625, 40.59787288305163], [-73.77291269298549, 40.597886704182535], [-73.77288812728872, 40.59789159960686], [-73.772815286146, 40.5979008220502], [-73.77275691437465, 40.597903687496824], [-73.7727570748028, 40.597904983321584], [-73.77273734104323, 40.5979063941005], [-73.77271339239246, 40.59790647305511], [-73.77268672376206, 40.597904529064], [-73.77265976439314, 40.59790070279755], [-73.77260315523904, 40.59789267585408], [-73.77257530877226, 40.59788957623183], [-73.77255072726562, 40.59788833024757], [-73.77252829048805, 40.59788893417865], [-73.77248573531337, 40.59789269440952], [-73.77246641203124, 40.59789477289049], [-73.77235384103724, 40.59790689605607], [-73.77224040842336, 40.59791528886193], [-73.77215537297705, 40.59792157722971], [-73.77211321788315, 40.597922277012565], [-73.772036000558, 40.597918855358934], [-73.77200600657046, 40.597918336322444], [-73.7719756750917, 40.597919590916014], [-73.77194871454776, 40.59792300411046], [-73.77190455979193, 40.597931634559586], [-73.77179430028144, 40.597955603437235], [-73.77171188221453, 40.59796997462358], [-73.77167614196895, 40.5979778009719], [-73.77164670369835, 40.59798720656996], [-73.77161581115801, 40.598000644223994], [-73.77158534673, 40.598016720811636], [-73.77152710619445, 40.5980504287897], [-73.77145271133203, 40.598092245217465], [-73.77140901159491, 40.598119615587756], [-73.77130116654475, 40.59819358603526], [-73.7711972790028, 40.59826388128698], [-73.77119144202022, 40.598267832581584], [-73.77117966319527, 40.598274778868785], [-73.77116651606542, 40.5982808670233], [-73.7711522271575, 40.59828598994161], [-73.77113704375897, 40.59829007524053], [-73.77112127437523, 40.59829301648246], [-73.7711052481878, 40.59829476657703], [-73.77108933187579, 40.59829527524618], [-73.77105367950502, 40.59829765617103], [-73.7710179559255, 40.59829599827974], [-73.7709829456769, 40.59829033824227], [-73.7709494149873, 40.598280800095154], [-73.77091809986165, 40.598267592381674], [-73.77088117884767, 40.59823698598354], [-73.77083827197097, 40.59821131200746], [-73.77079050002219, 40.59819124121038], [-73.77073911081987, 40.59817729817922], [-73.77068544592018, 40.59816984698546], [-73.7706309070899, 40.59816908193106], [-73.77060989596227, 40.59816140998561], [-73.77059090361828, 40.59815112513521], [-73.77057448291983, 40.598138527803066], [-73.77056111373088, 40.59812398412913], [-73.77054516579517, 40.59811274383423], [-73.77052691955656, 40.59810376113528], [-73.77050691790143, 40.598097304876525], [-73.77048575786431, 40.59809356626493], [-73.77046406974573, 40.598092658002564], [-73.7704424992162, 40.598094606127304], [-73.7703705912474, 40.598089358941095], [-73.77029837514523, 40.598090670887245], [-73.77022687646752, 40.59809852314532], [-73.77015711217174, 40.598112804927666], [-73.77000536315805, 40.598118760749955], [-73.76988103186068, 40.59811849375457], [-73.76985203600657, 40.59811954282386], [-73.76982616733864, 40.598121724591586], [-73.76981453611454, 40.59812350224781], [-73.7698027471798, 40.59812625183336], [-73.76977813280078, 40.59813482990247], [-73.7697565611052, 40.59814487210707], [-73.76969872152439, 40.59817498728073], [-73.76966704117271, 40.59819061136191], [-73.76961275094072, 40.59821925592996], [-73.76959619242577, 40.59822799426069], [-73.76956240787261, 40.59824413597682], [-73.7695477270407, 40.59824963583339], [-73.76953162147454, 40.59825380818306], [-73.76951429278408, 40.59825659198217], [-73.76949607048141, 40.59825791512722], [-73.7694772510458, 40.598257769727475], [-73.76945836563297, 40.59825613817835], [-73.76943991058556, 40.59825305690714], [-73.7694223817573, 40.59824862708657], [-73.76937573689459, 40.59823235193847], [-73.76933970822688, 40.5982173301098], [-73.76928763545249, 40.598195631887506], [-73.76925038618148, 40.59818149986785], [-73.76918555500956, 40.59815972894328], [-73.76910769216117, 40.59813461921067], [-73.76902982881, 40.59810950000841], [-73.76895212674998, 40.598082607123395], [-73.7688975492035, 40.59806084021625], [-73.76877753897563, 40.59800692701061], [-73.76872294114966, 40.59798446667334], [-73.7686400308551, 40.59795308757885], [-73.76858501067476, 40.59793333600989], [-73.76853401985419, 40.597916645184704], [-73.76848685686616, 40.59790309777968], [-73.76846375279293, 40.59789845792449], [-73.76840476505055, 40.597891917869944], [-73.76837842432509, 40.59788695776965], [-73.76834488487442, 40.59787660505765], [-73.7682732797337, 40.59784910390578], [-73.76824081682878, 40.59783840210951], [-73.76818980308668, 40.59782479178571], [-73.76817114818597, 40.59782142213814], [-73.76815495718161, 40.59781988601652], [-73.76814045974687, 40.59782020804662], [-73.76813404220754, 40.597821104416326], [-73.76812817714196, 40.59782248792586], [-73.76812285388164, 40.59782435923732], [-73.7681180614672, 40.597826727194125], [-73.76811381057426, 40.597829591555964], [-73.76811008919918, 40.59783295222738], [-73.76809925802587, 40.59784597005487], [-73.76795322999999, 40.5974260569999], [-73.76776273800007, 40.59654905699993], [-73.76771811799993, 40.596261309999925], [-73.76742090699994, 40.59630520399985], [-73.76733661099998, 40.59637765199988], [-73.76726813399988, 40.59645316599989], [-73.76721443599983, 40.596535878999845], [-73.76717720099997, 40.5966231809999], [-73.76715706599988, 40.59671223899986], [-73.767127717, 40.59684901399989], [-73.76711717700006, 40.5969881329999], [-73.76712577699985, 40.59712732699992], [-73.76715321499994, 40.59726433399986], [-73.76716449199986, 40.597421597999904], [-73.76715168699985, 40.59758390099991], [-73.76711296699986, 40.5977480419999], [-73.76704742299998, 40.59791042699988], [-73.76695532899997, 40.598067273999895], [-73.76683818399978, 40.598214847999884], [-73.76669864799996, 40.598349814999935], [-73.76654034499992, 40.598469425999895], [-73.76636755699995, 40.59857174399987], [-73.76618488799977, 40.59865569699989], [-73.76599689399987, 40.59872106199988], [-73.76654204999997, 40.599434441999904], [-73.76444582499997, 40.6004352409999], [-73.76386778499985, 40.59972731299991], [-73.76336096499998, 40.599105042999945], [-73.76165797299986, 40.599973069999876], [-73.76158779199984, 40.59990109899993], [-73.76154124799989, 40.599846560999886], [-73.76128753399995, 40.59954927099993], [-73.7618148629999, 40.59895137099994], [-73.76255289900001, 40.59831811799991], [-73.7633344969999, 40.59782287399992], [-73.76319972899987, 40.59714459099989], [-73.76301370999988, 40.596081549999916], [-73.76299716399983, 40.595987023999875], [-73.762981598, 40.59589946999991], [-73.76277845599988, 40.59475654399989], [-73.76362979899989, 40.59466694999985], [-73.76440183099987, 40.59457168299989], [-73.76429039299992, 40.5939396449999], [-73.76414300299994, 40.593205171999884], [-73.7640494709999, 40.59273904699989], [-73.763912182, 40.592039344999925], [-73.76385620499985, 40.59175407799996], [-73.76376223423699, 40.591439709016804], [-73.76409879358965, 40.59139387155286], [-73.76428206816591, 40.59136497918443], [-73.76463923166975, 40.59129877037863], [-73.7649902320434, 40.591233701134755], [-73.76517139989306, 40.59120547114956], [-73.76541994963503, 40.59117360484565], [-73.76551723507008, 40.591155606480676], [-73.76567285203018, 40.59112681628276], [-73.76604736542077, 40.59104682933076], [-73.76622772170013, 40.59099390262785], [-73.76632501255979, 40.59097365115132], [-73.76642597471904, 40.59096093333518], [-73.76665068484326, 40.590938785749806], [-73.76674971845058, 40.59092708264958], [-73.76704832460794, 40.59088297651934], [-73.76719481614633, 40.59087299025633], [-73.7673542988742, 40.590863371033244], [-73.76745345525089, 40.59084341823688], [-73.7674668729756, 40.590838754554], [-73.76762172035325, 40.59078493572174], [-73.767762798651, 40.590758583689066], [-73.76796973006509, 40.590729482964356], [-73.76807147481031, 40.59072282634708], [-73.76813345500128, 40.590736899968306], [-73.7682199338214, 40.590798815059145], [-73.76828636409597, 40.590834347763334], [-73.76844393841534, 40.590857674630136], [-73.76858695488026, 40.59086859720194], [-73.76872708525138, 40.59087006895558], [-73.7688338763409, 40.59086377966345], [-73.76886283140857, 40.590862074658574], [-73.76914741260788, 40.59083210085482], [-73.76945313604804, 40.590792470296336], [-73.76978825596215, 40.59075090723356], [-73.76995620052728, 40.590730072328796], [-73.76998923560154, 40.59072466308158], [-73.77002485586418, 40.59071653127758], [-73.77017538907467, 40.59067149100075], [-73.77021137888435, 40.59066247649311], [-73.7702442519966, 40.59065615682651], [-73.7705747448231, 40.59060997196553], [-73.77069663752131, 40.59059042245807], [-73.77072132001807, 40.59058540091686], [-73.77074679602244, 40.590579751878856], [-73.77082321393041, 40.590562155867616], [-73.77085305962974, 40.590556955789495], [-73.77088020413456, 40.59055383907996], [-73.77088579106663, 40.5905539133788], [-73.77091290683522, 40.59056256741985], [-73.77095214359046, 40.590567734022976], [-73.770982527501, 40.59057085669498], [-73.77101650427375, 40.59057321135812], [-73.77111441651954, 40.59057796341298], [-73.77133707453225, 40.5905854575432], [-73.7714284595353, 40.59058518920861], [-73.77153385684235, 40.590580634936146], [-73.7716369311758, 40.5905716732657], [-73.77168745194966, 40.59056555059407], [-73.77178997907545, 40.59055066116292], [-73.7720906007932, 40.59050297169141], [-73.77224111424952, 40.590482178451175], [-73.77228081745474, 40.59047669280408], [-73.77242217935688, 40.590459952836035], [-73.7724707070396, 40.59045613158627], [-73.77252453190594, 40.59045434707629], [-73.77274072126971, 40.59045705359362], [-73.77278993524673, 40.590456331193415], [-73.77283395784046, 40.59045418505977], [-73.77286232906431, 40.59045153972608], [-73.77289315118053, 40.59044730501539], [-73.77303452122213, 40.59042092899833], [-73.77306871342176, 40.59041582689397], [-73.77309963866246, 40.590412601363944], [-73.77315678948564, 40.59040902184443], [-73.77348503908469, 40.59040324931067], [-73.77353992912971, 40.59040098929999], [-73.77360853280666, 40.590395369164234], [-73.77364313739031, 40.59038690973319], [-73.77368113064308, 40.590375602424984], [-73.7737060326092, 40.59036817678946], [-73.77374135977975, 40.59035925050591], [-73.77376174702637, 40.59035635485481], [-73.77379843010951, 40.59035488663678], [-73.77383881018656, 40.59035725307069], [-73.77388174249228, 40.59036294796987], [-73.77396801908459, 40.590378130233624], [-73.77400459636709, 40.59038364070015], [-73.77404303478913, 40.59038733611079], [-73.77407763164722, 40.590387963017875], [-73.77409639395029, 40.59038676638325], [-73.77411550495816, 40.590384156311714], [-73.77415547523118, 40.59037462586904], [-73.77420071437766, 40.59035848800225], [-73.77429526133092, 40.59031727595606], [-73.77434868621285, 40.59029769512071], [-73.77438246523175, 40.59028884634951], [-73.77441435006315, 40.59028444260473], [-73.77442481913887, 40.590283544758606], [-73.774675873645, 40.590274031949086], [-73.77478603527494, 40.59027136640258], [-73.77521117634113, 40.59026821837303], [-73.77529299783197, 40.590262922297356], [-73.77547648519086, 40.590245422625664], [-73.77559905541047, 40.59022739979776], [-73.77579676083829, 40.59019371819814], [-73.77586029605193, 40.59018468496815], [-73.77591759770544, 40.59017829422469], [-73.7759259477778, 40.59017858994795], [-73.7759304282186, 40.590181408327894], [-73.77592993689225, 40.59019043002462], [-73.77592039829484, 40.59020593725191], [-73.77590847858993, 40.59022046581634], [-73.77584876326614, 40.59028427873138], [-73.77582466487753, 40.59031622740646], [-73.7758237310679, 40.59031964671146], [-73.77582424055556, 40.59032276359304], [-73.77582658548931, 40.59032463243996], [-73.77582980872609, 40.590325214695696], [-73.77583688595261, 40.5903245987577], [-73.77592417813304, 40.59030830669686], [-73.77608710709404, 40.590274314192655], [-73.7763506460391, 40.59021225018254], [-73.7764548908938, 40.59018123250739], [-73.77646844928364, 40.59017495574101], [-73.77648111993174, 40.59016582974159], [-73.77651625337222, 40.59012613068266], [-73.77653951423233, 40.59010444696549], [-73.77657809244383, 40.59008743829471], [-73.77662543841134, 40.59007310468421], [-73.7766624348081, 40.59006633216663], [-73.77669882905153, 40.59006324194554], [-73.77675237473706, 40.59006713769311], [-73.77686267144291, 40.59009159350015], [-73.77691114218703, 40.59009768501643], [-73.7770131564979, 40.590101215869105], [-73.77710926069993, 40.59010242007234], [-73.77720080214564, 40.59010129103926], [-73.77728844028084, 40.59009784111593], [-73.77735319926097, 40.5900938153919], [-73.7774555913146, 40.59008337811344], [-73.77750026725015, 40.59007629686677], [-73.77761352076865, 40.59005347120861], [-73.77775882321265, 40.59001959651591], [-73.77783509043982, 40.590000499009875], [-73.77791587910747, 40.58997897012016], [-73.77810144460149, 40.58992581087995], [-73.77814946692536, 40.5899067311578], [-73.77824956631362, 40.58985379363007], [-73.77827350862977, 40.5898439612682], [-73.77838680010395, 40.58981645229559], [-73.77854450219125, 40.58978726338208], [-73.77865504053747, 40.589763604203256], [-73.77870305555749, 40.58975055782302], [-73.77884539331758, 40.589705679221986], [-73.77897465802009, 40.58965958734476], [-73.77912834079974, 40.589605940762596], [-73.77918495356101, 40.58959310889531], [-73.7792490976237, 40.58958558675223], [-73.77928116131307, 40.589584514170056], [-73.77936059540092, 40.589593285465625], [-73.77939398598417, 40.58959101588608], [-73.77942325796694, 40.589580203248275], [-73.77951586431837, 40.58958863902988], [-73.77980375731784, 40.58964376225575], [-73.77985959718424, 40.5896471020727], [-73.77992945455958, 40.5896443543739], [-73.78008826023272, 40.589623721038066], [-73.78014903250914, 40.58961425629994], [-73.78022286069903, 40.58961102029941], [-73.78031021036223, 40.5896087735682], [-73.7803723678408, 40.58960535332382], [-73.78043036101892, 40.589596621429024], [-73.78048449570841, 40.5895831363487], [-73.78055284379498, 40.58956872330119], [-73.78057452813783, 40.589562559239596], [-73.78064367908992, 40.58953746814707], [-73.78067604037903, 40.58952847091273], [-73.7807538436896, 40.58950493553284], [-73.78084602797253, 40.589469369569876], [-73.78086936029906, 40.589458022039786], [-73.78097447293345, 40.58942078706918], [-73.78102983980423, 40.58940253149113], [-73.7810878441754, 40.58937957901906], [-73.78114865778065, 40.58935719074782], [-73.78121208011379, 40.58934257024145], [-73.78135755523371, 40.58929116570024], [-73.78142243554618, 40.5892713063463], [-73.78148525449765, 40.58926058396593], [-73.78161081064151, 40.58924953919869], [-73.78168370364948, 40.58925399144242], [-73.78178622344916, 40.589250998424106], [-73.78190890137239, 40.58922427779513], [-73.78206226147378, 40.58918551287879], [-73.78211351753535, 40.58917062621591], [-73.78214454631176, 40.58916027457861], [-73.78218351388836, 40.58914671437307], [-73.7822239170208, 40.5891343279088], [-73.78228054765907, 40.58911220116923], [-73.78234015973189, 40.58909592483009], [-73.78240601796996, 40.589091357053796], [-73.78243514633095, 40.589091628491374], [-73.78246331446263, 40.58908957496756], [-73.78252737843965, 40.5890738554523], [-73.7825569715204, 40.58906584271833], [-73.7827088491647, 40.589039205136864], [-73.78277341608162, 40.589024693488234], [-73.78288961869187, 40.58899867107045], [-73.7829760232428, 40.58896705489506], [-73.78310566264246, 40.58892437200097], [-73.78330722135844, 40.58884745134519], [-73.78350063414018, 40.588757871344185], [-73.78370053961723, 40.58870244196633], [-73.7838100709537, 40.588678910463756], [-73.78393636438304, 40.58864763816582], [-73.7840617124231, 40.58860933172051], [-73.78438543099911, 40.58851831957554], [-73.7845331104848, 40.58847805521786], [-73.78465258073332, 40.588452506565005], [-73.78474624813403, 40.5884360215682], [-73.78484776439457, 40.588417769344844], [-73.78491351080592, 40.58840713115515], [-73.78497718722853, 40.58840096427861], [-73.78505919649476, 40.58839153601536], [-73.78513382316405, 40.58838182450065], [-73.78517136143141, 40.588374843862226], [-73.78524236737745, 40.588363503818826], [-73.78537587818671, 40.58833948306335], [-73.7855107314079, 40.588334881533314], [-73.78563095491499, 40.58831017929579], [-73.78577545794695, 40.58828019977907], [-73.78602063181066, 40.58821892589794], [-73.78610751824587, 40.58819477306146], [-73.7863055521708, 40.588147107883046], [-73.78639182672046, 40.58813302223642], [-73.78645905573437, 40.58812814944798], [-73.78653419682246, 40.58811629426871], [-73.78660941617714, 40.588105726485026], [-73.78670594394542, 40.58809180288192], [-73.78684206289357, 40.58807553117461], [-73.78692504410577, 40.58806459058936], [-73.78708881251839, 40.58803827501504], [-73.78720687841518, 40.58802304044104], [-73.78737986814394, 40.5879936613258], [-73.7875018580916, 40.587966520161615], [-73.78768370715751, 40.58792293771202], [-73.78818270539261, 40.587748355103614], [-73.78842295061148, 40.587645903597306], [-73.78844687378574, 40.587633619629], [-73.7884747250357, 40.58761563313569], [-73.78849742602627, 40.58759770081677], [-73.7885370655059, 40.5875648047434], [-73.78857477507556, 40.587526224503904], [-73.78858894789693, 40.58750816774745], [-73.78860298459861, 40.58748444605279], [-73.78878406873122, 40.58745028886727], [-73.78887538807133, 40.58747315028718], [-73.78898283996136, 40.58749323982922], [-73.78901325881863, 40.587503761110014], [-73.789033888718, 40.587516639145285], [-73.78907590827097, 40.58754940537708], [-73.7891046960101, 40.58756749598723], [-73.78912690923757, 40.587576758770204], [-73.78917735702238, 40.587600318920565], [-73.78923024711176, 40.58762075825999], [-73.78926228451019, 40.587623555800576], [-73.78929582562273, 40.5876290828052], [-73.78932570533473, 40.587630893138964], [-73.78934393075197, 40.587631323432134], [-73.78936256861488, 40.587631762184955], [-73.78941470793565, 40.58762814730407], [-73.7894506559178, 40.58762324290702], [-73.7894798745329, 40.58761734360713], [-73.78953791751881, 40.58760331243083], [-73.78957437415306, 40.58759835347445], [-73.78959754044311, 40.587593758839766], [-73.78963535896544, 40.58758765059032], [-73.78968379436958, 40.58757481667646], [-73.78971718205132, 40.58756891606819], [-73.78975824801644, 40.587563227687674], [-73.78983704829267, 40.58755569077077], [-73.78989469426458, 40.58754627446977], [-73.78989734537727, 40.587545841227], [-73.7899629857302, 40.58753478548764], [-73.79009685734076, 40.587507393733866], [-73.79015229641777, 40.58749835423499], [-73.79030455195283, 40.58747718972551], [-73.79040001810121, 40.58745866959412], [-73.79050360269464, 40.587435228422265], [-73.79057603970988, 40.587414990688536], [-73.7906377095661, 40.58740429622775], [-73.7908451648303, 40.58735652295986], [-73.79088356288405, 40.58735003774263], [-73.79091570414633, 40.58734252271362], [-73.7909970316632, 40.58731214238663], [-73.79103431331967, 40.58730370018289], [-73.79105759554307, 40.587296331506266], [-73.7910866172636, 40.58728535294129], [-73.79111283049583, 40.587273487051164], [-73.79114367230208, 40.58726949006762], [-73.79128113976063, 40.58727429683877], [-73.79129641503224, 40.58728496892727], [-73.79132452803208, 40.587296357735376], [-73.79134615079208, 40.58730196234346], [-73.79138171999988, 40.58730488096742], [-73.79141754078475, 40.58730302795018], [-73.79147720790623, 40.58730159698543], [-73.79153984145417, 40.5872926236508], [-73.79166457516907, 40.587269957099906], [-73.79172064744127, 40.587259767900726], [-73.79187055097091, 40.58723094345442], [-73.79197454551789, 40.58721593046026], [-73.79200188355018, 40.58721073844644], [-73.79209625186265, 40.5871879100734], [-73.79218465863211, 40.587160037742436], [-73.79230828344397, 40.58712628438011], [-73.7923418903835, 40.58711530607998], [-73.79239225941785, 40.58709885185527], [-73.79243962937679, 40.58707876584057], [-73.79251797208626, 40.58704316501592], [-73.79261297557468, 40.58702484039965], [-73.79274578763986, 40.587003395481474], [-73.79281029204658, 40.58699607282039], [-73.79293381050245, 40.586981607976604], [-73.79301601821497, 40.58697159926604], [-73.79307866251777, 40.58696246260878], [-73.7931376215086, 40.586953788701045], [-73.79319946350594, 40.58694819077354], [-73.79326405221966, 40.58694802873628], [-73.79332342680792, 40.58694600004854], [-73.79338308107955, 40.58693745408561], [-73.79347727539468, 40.58690957206976], [-73.7934991539871, 40.586900966351365], [-73.79352137558897, 40.58688859752055], [-73.79358367520715, 40.58683828784179], [-73.79376524530197, 40.586786545451425], [-73.79379355003898, 40.58678572342175], [-73.7938518333594, 40.58680037160348], [-73.79388370976301, 40.58680542653921], [-73.793909486807, 40.58680871498136], [-73.79394033243052, 40.58681104787106], [-73.79396902115447, 40.58681183829104], [-73.79400669001211, 40.58680816790469], [-73.7940302958186, 40.586802807888745], [-73.79405336467549, 40.586794915697695], [-73.79410918649302, 40.58677676118774], [-73.79414149029743, 40.58677369480513], [-73.79419885193848, 40.586773194286415], [-73.79434460522953, 40.58675667898859], [-73.79440078001352, 40.586742397253], [-73.79444991319518, 40.586732822878], [-73.79451058376749, 40.586720035790826], [-73.79453736495408, 40.586711429260994], [-73.79468886939398, 40.586660703166345], [-73.79471998303308, 40.58664917842727], [-73.79474172935485, 40.58663755569442], [-73.79479420340178, 40.5865976027612], [-73.79482352521471, 40.58658098691356], [-73.7948552134114, 40.58656660745071], [-73.79488456535024, 40.58655162103625], [-73.79494319554186, 40.58653396786084], [-73.7949761236387, 40.58652763234345], [-73.79500512248676, 40.58652378533054], [-73.79503399443716, 40.586522548533615], [-73.79506190351425, 40.58651971730596], [-73.79522576667671, 40.58648314117326], [-73.79525645329633, 40.58647981862521], [-73.79529022754973, 40.586478726533535], [-73.79534267191754, 40.58647907192307], [-73.79538200168786, 40.58647658445956], [-73.79542826118839, 40.58647179523145], [-73.79544358593455, 40.58647027336698], [-73.79554878114587, 40.5864711989017], [-73.79559625132377, 40.586468456294], [-73.79573362789837, 40.58644836700528], [-73.79576511591748, 40.586441443520215], [-73.79583321779106, 40.58642068145796], [-73.79588349292341, 40.586404083471976], [-73.79590852396218, 40.586392026085434], [-73.79595625878476, 40.58635658482831], [-73.79598297100075, 40.58633541544899], [-73.79602046510774, 40.58631904717583], [-73.79614753501264, 40.586336005065384], [-73.79622173814558, 40.58634764565386], [-73.7962517833779, 40.58635312824201], [-73.79633505857086, 40.58637636567698], [-73.79643078427875, 40.586379831296576], [-73.79645157575762, 40.586379271389966], [-73.79652371849949, 40.58637732967471], [-73.79659741256897, 40.586377433686245], [-73.79663442436521, 40.58637609446339], [-73.79670174417555, 40.58636727152463], [-73.79673529350114, 40.58635868583969], [-73.79675983252, 40.586353073274196], [-73.7968225160063, 40.58633828034796], [-73.79685841157989, 40.586327223020525], [-73.79691111161142, 40.58630574856179], [-73.7969761667721, 40.58628436727837], [-73.79704230538812, 40.58626018928657], [-73.79706738813755, 40.58625040830158], [-73.79712944995846, 40.586230426382116], [-73.79721710027918, 40.586201323701616], [-73.79726406299116, 40.5861827385613], [-73.79758258715597, 40.58607732581422], [-73.79761231706715, 40.58606590540913], [-73.797641879923, 40.586051161915876], [-73.79771707099138, 40.586024999005154], [-73.79781087949239, 40.58600652307735], [-73.79787491546294, 40.58599744170836], [-73.79796497730955, 40.58599143199313], [-73.79803790681721, 40.585982770755514], [-73.79808945022879, 40.58597652251826], [-73.79812309463986, 40.585971177627265], [-73.79818740957423, 40.58595599982967], [-73.7982150827617, 40.58594515212624], [-73.798277244352, 40.58592714276106], [-73.79831147819522, 40.58591855781807], [-73.79833821336442, 40.58590927517722], [-73.79835935100574, 40.58589939685526], [-73.79841218458479, 40.58586820538177], [-73.79846640545998, 40.58584368065295], [-73.79860416989281, 40.58585500681993], [-73.79862937227614, 40.58587618655242], [-73.79869486678675, 40.58589644661037], [-73.79881542788075, 40.58589401901037], [-73.79893359561238, 40.58588564380834], [-73.7990820019512, 40.585869892424], [-73.7992838155624, 40.58584414752272], [-73.79949174993197, 40.58581418097826], [-73.79963642261946, 40.585790705604765], [-73.7996884641193, 40.585779912603186], [-73.79974084641992, 40.58576904830181], [-73.80007589595284, 40.58568487426214], [-73.80014577318751, 40.58566735276141], [-73.80028251237498, 40.585633066317115], [-73.80045350238458, 40.58558144809817], [-73.80054546083578, 40.58553705958264], [-73.80061781447155, 40.585507557681986], [-73.80071108578056, 40.58547432779182], [-73.80075861454394, 40.585455246485665], [-73.80079451624748, 40.58543770353159], [-73.80083956676282, 40.585429947433184], [-73.80091511175826, 40.5854070609331], [-73.80116064145146, 40.58539678798376], [-73.80119138109775, 40.58540675592561], [-73.80122412486946, 40.58541057680467], [-73.80126813622081, 40.58541032910153], [-73.80133785981273, 40.58540717178863], [-73.80143187846257, 40.58540075072497], [-73.80156339177964, 40.58538874038294], [-73.80169093145422, 40.585374651358855], [-73.80177161574133, 40.58536373177829], [-73.80184018400166, 40.58535171084623], [-73.80190170596248, 40.585337688308385], [-73.80195743621874, 40.58532133965559], [-73.80203251846667, 40.585295083772266], [-73.80222781866699, 40.58522258551991], [-73.80241963073168, 40.58515512425481], [-73.8025397086005, 40.58511541017158], [-73.80265525068585, 40.585083856758146], [-73.80286516391384, 40.58503028390364], [-73.80304240416451, 40.58497667426328], [-73.80323770130614, 40.58490057139446], [-73.80330229481847, 40.58488615648929], [-73.80338852227408, 40.584885629431014], [-73.80349876666156, 40.58492428756064], [-73.80351488810318, 40.584944793638485], [-73.80354468490675, 40.58497428275867], [-73.80357247869416, 40.58499043123193], [-73.8036112870732, 40.58500425779923], [-73.80366299243434, 40.58501453985067], [-73.80371873405733, 40.585018084192], [-73.80377557037241, 40.58501469648513], [-73.80407247759545, 40.58495669707044], [-73.80421036759756, 40.58493290516727], [-73.80431422436612, 40.58490993835772], [-73.80440924673873, 40.58487933799817], [-73.80455361072326, 40.58482670556006], [-73.80468776810687, 40.584774468384396], [-73.80476634753872, 40.58474763131192], [-73.80484109780319, 40.5847257224393], [-73.80494831722095, 40.584696115699174], [-73.80519393375725, 40.58463024471364], [-73.80524718089742, 40.58461403466702], [-73.80540383998208, 40.58456159321666], [-73.80562312964219, 40.5845111837831], [-73.80571961087702, 40.58450608736719], [-73.80586233123243, 40.58452681532272], [-73.80587704621392, 40.58453571088795], [-73.8059267115608, 40.5845450606945], [-73.80597104297532, 40.584544415858616], [-73.8060599290281, 40.584535191296176], [-73.8061974730921, 40.584516025274255], [-73.8063360357249, 40.584496124678736], [-73.80643157748595, 40.584480795707194], [-73.80651754277206, 40.58446468676176], [-73.80659491176878, 40.5844474456911], [-73.806637364536, 40.58443489159597], [-73.80668208768715, 40.58441723533228], [-73.80672634779604, 40.58439624727743], [-73.80688379979635, 40.58431803956488], [-73.8069189892024, 40.5843050131207], [-73.80718262664281, 40.58421883524597], [-73.80743073987672, 40.58413374616213], [-73.80746677141387, 40.58412351351577], [-73.80758995887524, 40.58409352531212], [-73.80783502965463, 40.58401495035292], [-73.80792558260914, 40.58400509638657], [-73.8082897849443, 40.58402471459722], [-73.80834370637812, 40.5840283168177], [-73.8084997147581, 40.58404475982289], [-73.80854592324482, 40.58404450361386], [-73.8086334832092, 40.58403590560176], [-73.80866370337569, 40.58403408345098], [-73.80875382070491, 40.58403182792798], [-73.80881507288822, 40.58402882269197], [-73.80887034203347, 40.58402397084169], [-73.80892008844539, 40.58401705754021], [-73.80897688682197, 40.58400618263784], [-73.80903284025594, 40.583992859027944], [-73.80908759716272, 40.583977162520895], [-73.80914080102232, 40.58395920477134], [-73.80919306534241, 40.58393898495752], [-73.80930913443461, 40.58389096291579], [-73.80936210702265, 40.58387099616531], [-73.80940719882081, 40.583856059634705], [-73.80951491959115, 40.58382343175396], [-73.80962143091784, 40.58381744017788], [-73.81132481815267, 40.58372152471486], [-73.81216750375401, 40.58362614107536], [-73.81269595844536, 40.583566322154645], [-73.81320376356805, 40.583508838115605], [-73.81341809223822, 40.58348912300502], [-73.81352448727559, 40.58347314904673], [-73.8136458227654, 40.58342805825606], [-73.81398801425166, 40.583331094686095], [-73.81417259932803, 40.583273454375394], [-73.81425389459, 40.58324543395957], [-73.81453624637712, 40.58314811233034], [-73.81471054401754, 40.58309434392088], [-73.8148522431602, 40.58306547678401], [-73.81501580175153, 40.58301698588703], [-73.81523945963075, 40.5829246535502], [-73.81539400218122, 40.58286969235861], [-73.81544351379941, 40.5828520849442], [-73.815896308824, 40.58273930861986], [-73.81652734669423, 40.5825756970909], [-73.81655438991994, 40.58256868458874], [-73.81684582937446, 40.58246342820497], [-73.81702595651316, 40.58241017939203], [-73.81730795916116, 40.58233475897], [-73.81751059199993, 40.58278527099993], [-73.81670273099988, 40.5830153519999], [-73.81567618499987, 40.58330770499987], [-73.81544517899997, 40.58337349199993], [-73.81550882099985, 40.583475793999924], [-73.81563939099986, 40.58372220799987], [-73.8156731639999, 40.58377646399992], [-73.81570076199996, 40.58382964799994], [-73.81576138799994, 40.58395921499993], [-73.81637844599986, 40.58491100499989], [-73.81693289499991, 40.58582331099997], [-73.81733678299989, 40.586483192999964], [-73.81739092999989, 40.58656943199988], [-73.81771610099989, 40.587150957999924], [-73.81788322399981, 40.58744846899986], [-73.81800188699995, 40.58740172099992], [-73.81812861199995, 40.58737005199992], [-73.81829590099994, 40.58735573699989], [-73.81849901800001, 40.5873623529999], [-73.81880351800001, 40.58749592199988], [-73.81890104000004, 40.58757409799995], [-73.81893219199988, 40.58759907299989], [-73.81897507799992, 40.58764405899989], [-73.81904410199994, 40.587733741999934], [-73.81909407399996, 40.58783196099988], [-73.81912265299988, 40.58793502599987], [-73.81912652299997, 40.58799676499997], [-73.81912916381579, 40.588038892972584], [-73.81883722364618, 40.58825143495896], [-73.81876839992097, 40.58828335015362], [-73.81869746373584, 40.5883124533212], [-73.8186248236585, 40.58833859329379], [-73.81855048071942, 40.58836178455365], [-73.81847464070607, 40.58838196290506], [-73.81837503223598, 40.58841883613845], [-73.8182786497713, 40.588460394601796], [-73.81818587202191, 40.58850647500418], [-73.81809706353732, 40.588556896287045], [-73.81801257327248, 40.58861146033523], [-73.81800184906037, 40.58861614188699], [-73.81791292627506, 40.588661799767884], [-73.81761886059695, 40.58881278574863], [-73.81741586147443, 40.588917014109676], [-73.8174387363485, 40.58897998214495], [-73.81746593567614, 40.58904194281439], [-73.81758962988171, 40.58921724695753], [-73.8176380045051, 40.58928579948519], [-73.81762330439474, 40.58929182682335], [-73.81760761896984, 40.58926958598405], [-73.81753265580444, 40.58930032794415], [-73.81750208580694, 40.58931286053868], [-73.81751220913848, 40.589330131000146], [-73.81751021655829, 40.5893330450298], [-73.8174204510319, 40.589359333252396], [-73.8173679813751, 40.58925520305159], [-73.81721886184167, 40.58931889409227], [-73.81720706565159, 40.58933901122138], [-73.81716590847275, 40.589365798736665], [-73.8171296961493, 40.589367974281366], [-73.81708178953154, 40.58935746176736], [-73.81700298780069, 40.58934015378189], [-73.81691558269159, 40.58931169365621], [-73.81682951096518, 40.58930484708518], [-73.81678888161393, 40.589311491706496], [-73.81674197582348, 40.58931916984591], [-73.81669191177083, 40.58935459790421], [-73.81662649621312, 40.5894008878561], [-73.81659271775965, 40.58942478851081], [-73.81657902384649, 40.58943447499689], [-73.81653859470231, 40.589423720552396], [-73.81650433814303, 40.58942814090482], [-73.81647294031325, 40.589432189373845], [-73.81644364346825, 40.58944147163449], [-73.81641916119337, 40.58945261646849], [-73.8163966008141, 40.589473464001316], [-73.81636027213152, 40.589518161667186], [-73.81631809499207, 40.589554637500314], [-73.81629654402978, 40.589565049193325], [-73.8162760181166, 40.589559046456046], [-73.8162594080273, 40.58955081680619], [-73.81624423864828, 40.589551538558084], [-73.81615706220046, 40.58959018541568], [-73.81605618762214, 40.589626567224975], [-73.81600226087951, 40.58966601319696], [-73.81596626679338, 40.589692350189154], [-73.81594103102285, 40.58969799204], [-73.81589307115703, 40.58970239092926], [-73.81584523000214, 40.58970677239824], [-73.81582455452242, 40.589711971379764], [-73.81526955469286, 40.58997885168365], [-73.81524832374934, 40.58998905679544], [-73.81521320297809, 40.59001530432234], [-73.81518965919054, 40.590036896046136], [-73.81516608506067, 40.590069682718905], [-73.81513268300009, 40.590118852217245], [-73.81508349102971, 40.59021872243718], [-73.81504418972436, 40.59027832738786], [-73.81500343530371, 40.590334202617754], [-73.81496908451777, 40.590372185924764], [-73.81492392915133, 40.590425811487336], [-73.81488172581524, 40.5904712469067], [-73.81484403460317, 40.590512852816836], [-73.8148439398326, 40.59051295150662], [-73.8147478467245, 40.5905881261723], [-73.81470067461719, 40.59061058988816], [-73.81459418101943, 40.59065371581371], [-73.81455024927243, 40.59066641369609], [-73.81451879513504, 40.5906735499502], [-73.81448107260579, 40.590679081360825], [-73.81444754631244, 40.590679818805036], [-73.81441717033375, 40.59067737575093], [-73.81438262208175, 40.59067173534666], [-73.81435121456343, 40.59066209537734], [-73.8143261088946, 40.590650877903805], [-73.81429696620411, 40.59063785446438], [-73.81426069690679, 40.59062681008563], [-73.81421986803537, 40.59061955705884], [-73.81418215917384, 40.590616308833766], [-73.81414340154399, 40.59061624615179], [-73.8140931002443, 40.59062095612935], [-73.81405803164166, 40.59062423925842], [-73.81402182783657, 40.590636014628444], [-73.81390504662012, 40.59067398931834], [-73.81381576194369, 40.59071314429661], [-73.8137648345345, 40.590735474723935], [-73.81370144041946, 40.5907632701736], [-73.81363221410687, 40.59079363240033], [-73.81357398073838, 40.590826154455165], [-73.81352240790815, 40.59085495054058], [-73.81345814115188, 40.59089083997578], [-73.81340983344633, 40.59091782311177], [-73.81337671660636, 40.590940831799564], [-73.81335361365774, 40.590960758282726], [-73.81329441415484, 40.59100931490095], [-73.81325671566422, 40.591046302490895], [-73.81358856612626, 40.5916976858236], [-73.81355405036116, 40.59170889619408], [-73.81323474432587, 40.59107074001757], [-73.81319209060828, 40.59112657773379], [-73.81315702925521, 40.591185405844314], [-73.81312991669729, 40.591246626341714], [-73.81305738293007, 40.59125608117274], [-73.81298550919139, 40.59126188974403], [-73.81290225557298, 40.59126860685094], [-73.81281452639965, 40.59128381789963], [-73.81274321153676, 40.591296184005664], [-73.81262404792933, 40.59133237049375], [-73.81258705762329, 40.591363405517114], [-73.81254622831922, 40.59138110560623], [-73.81249376357003, 40.59138693715447], [-73.81244327084264, 40.59138389198795], [-73.81238893874787, 40.591368998361766], [-73.81233849596734, 40.59134818671009], [-73.81228502224809, 40.59132612550706], [-73.8122433822537, 40.591325823102245], [-73.81219867626373, 40.591340554976725], [-73.8121436008147, 40.59135870088933], [-73.81204110190585, 40.59142913463002], [-73.81200998568714, 40.591450516300334], [-73.8119428948392, 40.59147339695089], [-73.81183397866033, 40.59152355729164], [-73.81170344836342, 40.59159374663789], [-73.81157348240845, 40.591686117017986], [-73.81151303182712, 40.591783490316764], [-73.81151254025703, 40.59195764194907], [-73.81153473098952, 40.5920893176199], [-73.8115671565879, 40.592148616318184], [-73.81150436968763, 40.59219055825777], [-73.81153174691062, 40.59225941328688], [-73.81151655218235, 40.59231481588405], [-73.81152858438593, 40.592492571675734], [-73.81156051309381, 40.59272770565675], [-73.81157420358255, 40.592828532702185], [-73.81156191658249, 40.59284995400254], [-73.81152689347525, 40.5928824422123], [-73.81149423585859, 40.592901326696776], [-73.81148025711856, 40.59290248300566], [-73.81147327231092, 40.59285357282747], [-73.81147916226398, 40.5927421153034], [-73.81146628908357, 40.59262788118394], [-73.81143557275092, 40.59250770046838], [-73.81142968258294, 40.59225500399127], [-73.81139868245076, 40.59210108919213], [-73.8113613719154, 40.591981482975385], [-73.81132554567534, 40.59174649614576], [-73.81117019530946, 40.59175817259397], [-73.8110076658511, 40.59176829725974], [-73.81094534713289, 40.591824261227686], [-73.81093416761482, 40.59185107823741], [-73.81089019538996, 40.59195661942398], [-73.81087995835122, 40.591996955109494], [-73.81090890307112, 40.59205844575268], [-73.81082116336776, 40.59207994851496], [-73.81082476546278, 40.592087810078276], [-73.81081791874632, 40.59209026869603], [-73.81083023597257, 40.592129350188245], [-73.81083807023013, 40.592129094702535], [-73.81086040071834, 40.59220609190293], [-73.81088386616315, 40.592285870450745], [-73.81089235942197, 40.59231335225499], [-73.8109501895272, 40.592503059144576], [-73.81095747722595, 40.59250180963575], [-73.81095422671093, 40.59248851717666], [-73.81109769692013, 40.59246860684041], [-73.81110011941593, 40.59247808093863], [-73.81110742207213, 40.592476125474995], [-73.81111636738369, 40.59247459321044], [-73.81125929809448, 40.59245448962914], [-73.81138593152033, 40.592442318454566], [-73.81138707024905, 40.59245986017511], [-73.81137968396601, 40.59246108207444], [-73.81139249922137, 40.59264088905654], [-73.81140179703912, 40.59276603941133], [-73.81140751227157, 40.592871257238706], [-73.81137658047176, 40.592872166581486], [-73.8113792293374, 40.59287722194816], [-73.81107287021051, 40.593022330572076], [-73.811060600922, 40.593006664973764], [-73.81104740380232, 40.593012737332295], [-73.8110282125314, 40.592987425551215], [-73.81109624136856, 40.592957729358226], [-73.81109997856512, 40.5929601229667], [-73.81116539848063, 40.59293648735208], [-73.81117286836944, 40.592943334836534], [-73.81121929327969, 40.59292661224749], [-73.81122673155643, 40.59293142787692], [-73.81137383935804, 40.59286110160759], [-73.81137331289641, 40.592703054284186], [-73.81129377931236, 40.59271699686204], [-73.81128202539524, 40.592719056434596], [-73.81121682748493, 40.59273048576794], [-73.81120557268075, 40.59273245836711], [-73.8111116956163, 40.592748913986036], [-73.81108958088781, 40.5927527917948], [-73.81101743525888, 40.59276543758535], [-73.81100438932782, 40.59276772452018], [-73.81092243628765, 40.59278209106758], [-73.81091016761651, 40.59278424089124], [-73.81075729308176, 40.592811038230685], [-73.81074974913311, 40.592792113552555], [-73.81078919953887, 40.592785307121936], [-73.81082523967643, 40.5927790891237], [-73.81083602321985, 40.59277722886852], [-73.81085983237149, 40.592773120696314], [-73.81137347255543, 40.59268449963313], [-73.8113582931578, 40.59246258404489], [-73.81123958938298, 40.59247169404629], [-73.81124288390839, 40.592491050305036], [-73.81117549363144, 40.59249835059593], [-73.81117764467437, 40.59250216995335], [-73.81097892144219, 40.59252896193116], [-73.8109756520523, 40.59251363998759], [-73.810953210548, 40.59251676016602], [-73.81096369026606, 40.592541316479156], [-73.81093965894601, 40.59254684780672], [-73.81091502170727, 40.5924606489484], [-73.8109087864633, 40.59244141348822], [-73.81089585366696, 40.59240152115437], [-73.81089303386865, 40.59239282206006], [-73.81087632857833, 40.59234129377452], [-73.81087408982033, 40.592334388538475], [-73.81085814227838, 40.592285199540804], [-73.81085513382493, 40.5922759171319], [-73.81083836170998, 40.5922241823336], [-73.8108356597854, 40.592215848478865], [-73.81081637692152, 40.592133226122165], [-73.81081984088483, 40.59213267512395], [-73.81080537904415, 40.59209209573514], [-73.81079464766287, 40.59209357946655], [-73.81079225824638, 40.592087032493986], [-73.81059009992056, 40.59213657587351], [-73.81020220578183, 40.59222619110156], [-73.8101838004068, 40.592223851456055], [-73.81016604131385, 40.59221948613128], [-73.81014778263885, 40.592212493302604], [-73.81013122661898, 40.59220336385751], [-73.8100780877514, 40.59220538577175], [-73.81002586904063, 40.59221315913202], [-73.8099756361539, 40.5922265253016], [-73.80995316806148, 40.59223460344844], [-73.80991130015991, 40.59225371768247], [-73.81006184504064, 40.59307761089721], [-73.80999821494291, 40.593081742697045], [-73.80998357571092, 40.593081594351275], [-73.80990340300123, 40.59308077605166], [-73.80977831849519, 40.5923990277862], [-73.80973138393709, 40.5924041860533], [-73.80976380498812, 40.59264631840473], [-73.80980266288222, 40.59264924614467], [-73.8098074971652, 40.592710409504704], [-73.80978743592274, 40.5927113397621], [-73.80978598807614, 40.59269298499477], [-73.80974319444822, 40.59269495851413], [-73.80970982436389, 40.59243960833941], [-73.80966779372882, 40.59247033269036], [-73.80963383143487, 40.592495166312446], [-73.80961620385169, 40.59253713813281], [-73.80950556866756, 40.59277519735679], [-73.80950264563126, 40.5928761596151], [-73.8095281973081, 40.593008777227794], [-73.80970128863605, 40.59353404259564], [-73.80961679374357, 40.59357572691456], [-73.80961196378863, 40.593576984263684], [-73.80959170493088, 40.593580688618054], [-73.80958983855328, 40.593580874331316], [-73.80957210064267, 40.593582663640035], [-73.8095581604756, 40.59358279366014], [-73.80955380210722, 40.59358283150243], [-73.8095374918281, 40.593581228351184], [-73.80952333760344, 40.593577891991664], [-73.80951716532438, 40.593575593640644], [-73.80951162078615, 40.593572883034334], [-73.8095067153687, 40.59356977711205], [-73.80950247217092, 40.59356626770331], [-73.80948844806677, 40.593550602379075], [-73.80947603911868, 40.593531526817294], [-73.80946639485778, 40.59351238279516], [-73.8094462888366, 40.593466846060814], [-73.80944057712485, 40.59345633666909], [-73.80943487840625, 40.59344581841944], [-73.80942207563055, 40.59342821830208], [-73.80940742101458, 40.593413722284], [-73.809376154314, 40.5933835758989], [-73.8093619281368, 40.59337235897331], [-73.80935376557314, 40.59336777857332], [-73.80934565917377, 40.593364694808294], [-73.80934158659272, 40.59336370649519], [-73.80933747652837, 40.59336306892663], [-73.80933333205287, 40.593362783783284], [-73.80932913773749, 40.5933628667872], [-73.80931797010604, 40.59336427008053], [-73.80930616021514, 40.59336716817911], [-73.8092538289374, 40.593387407193426], [-73.8092408596564, 40.59339103244979], [-73.80922880342266, 40.59339292110057], [-73.8091972679537, 40.5933953369568], [-73.80917407641935, 40.59339569415159], [-73.80915365567306, 40.59339407490559], [-73.80914442218545, 40.593392492833026], [-73.80913573458386, 40.59339038026238], [-73.80910194254544, 40.593380102979054], [-73.80907143592952, 40.59336933659729], [-73.8090436597832, 40.59335786274377], [-73.8090184143207, 40.59334560972089], [-73.80895786981318, 40.59331320819313], [-73.8089361547232, 40.59330158071348], [-73.80862030990554, 40.59313435927286], [-73.80851768249428, 40.593074303827834], [-73.80844038658456, 40.59303387624539], [-73.80832111121644, 40.59296684022191], [-73.80830432387019, 40.59295915812369], [-73.80827521057529, 40.592948735781924], [-73.80826417479943, 40.592945105764386], [-73.8082083398357, 40.59292675905855], [-73.8080442864002, 40.592878432933716], [-73.8080010094562, 40.59286385323473], [-73.8079628467709, 40.59284926447565], [-73.80793775175624, 40.59283817261359], [-73.80791189469635, 40.59282513596059], [-73.80788476930209, 40.59280990689864], [-73.80785481534912, 40.59279160328719], [-73.80780755431262, 40.59276160864791], [-73.80770160074354, 40.59269665671573], [-73.80767265848256, 40.59268018333559], [-73.80763927500527, 40.59266331384481], [-73.8075126476875, 40.59260657528282], [-73.80746208913058, 40.592581681951], [-73.80744766643767, 40.59257316587315], [-73.80743263373071, 40.59256264947223], [-73.80737138295157, 40.59251367455597], [-73.80735515859567, 40.59250319283622], [-73.80733923448065, 40.592494908099006], [-73.80731054866757, 40.59248390992642], [-73.80724077071318, 40.59246255851192], [-73.80722392069639, 40.59245596576283], [-73.80720944534748, 40.592448961521825], [-73.80719049471517, 40.59243783507385], [-73.80714453181291, 40.59240841931032], [-73.8071181939608, 40.592394408993485], [-73.80704392152812, 40.592362630072174], [-73.8069352650588, 40.59230947887315], [-73.80692764644432, 40.59230514296949], [-73.80691980581196, 40.59229961002788], [-73.80689522432588, 40.59227864007779], [-73.8068854411234, 40.59227123021747], [-73.806874567465, 40.59226495336959], [-73.80686910550764, 40.59226263898151], [-73.80686354627534, 40.592260828526435], [-73.80685667553333, 40.59225930401776], [-73.80684941288627, 40.59225840909638], [-73.80683328821836, 40.5922584907184], [-73.80681753486085, 40.5922604278507], [-73.80679788045367, 40.59226387959141], [-73.80679513037921, 40.59226704412631], [-73.80678289293694, 40.59226643836913], [-73.80677822541055, 40.59226732261611], [-73.80676740146946, 40.592268474777406], [-73.80675768885948, 40.59226879986979], [-73.80672963104533, 40.592269365338055], [-73.80670307474279, 40.59226990585022], [-73.80667911877451, 40.59226920900125], [-73.80665586847148, 40.59226600843095], [-73.80664571748989, 40.59226335242344], [-73.80663638290012, 40.59225995077403], [-73.80661166636489, 40.59224861700997], [-73.80653538283988, 40.59221045936758], [-73.80648079157945, 40.59218678295148], [-73.80645838139473, 40.59217875839882], [-73.80640686405575, 40.59216265896346], [-73.80638475066843, 40.592154373550095], [-73.80629859387237, 40.59211511871476], [-73.80623135421918, 40.592082343262355], [-73.80620694259615, 40.59207223356183], [-73.80616779799563, 40.592058542349946], [-73.80614774525729, 40.5920526560412], [-73.80609921205132, 40.59204185728225], [-73.80607784880968, 40.592035914470095], [-73.80606163594105, 40.59202945664892], [-73.80604514697389, 40.59202060419283], [-73.80602807459857, 40.59200930900447], [-73.80600567884328, 40.59199242307285], [-73.80599072502842, 40.59197961028604], [-73.80597757081334, 40.591965683447995], [-73.8059669094547, 40.59195150935579], [-73.80596648473352, 40.59195095093233], [-73.80595770522844, 40.59193571670593], [-73.80594676932562, 40.5919102220002], [-73.80593841309583, 40.59188341804981], [-73.80593270594095, 40.59185554588023], [-73.80592971712133, 40.59182689493249], [-73.80592952782479, 40.591806732415776], [-73.80593155065063, 40.591760943604136], [-73.80593086680973, 40.59174005066042], [-73.80592045908095, 40.591655185664344], [-73.8059225448708, 40.59160406599659], [-73.80592216636714, 40.59158813562979], [-73.80591806873274, 40.591550756522004], [-73.80591156706421, 40.59149139128078], [-73.80590551083067, 40.591408902501016], [-73.80589854177514, 40.59135580442444], [-73.80588536526888, 40.591292592313096], [-73.80587981268424, 40.59127666308931], [-73.80587085612899, 40.59126121227514], [-73.80585893055505, 40.59124698225888], [-73.8058447658738, 40.59123507929866], [-73.8058235929902, 40.59118780215949], [-73.80579625655068, 40.59112734877598], [-73.80579163929991, 40.59111467153945], [-73.80578861982288, 40.59110082546802], [-73.80578716401085, 40.59108567497266], [-73.80578724799275, 40.59106888897451], [-73.8057919964334, 40.59100796829336], [-73.80579442179297, 40.5909419005238], [-73.80579578692459, 40.59093133036707], [-73.80579754704986, 40.59092330096305], [-73.80580107848306, 40.590907250382834], [-73.80580204455653, 40.59089591427579], [-73.8058006675252, 40.59086977044736], [-73.80579592013918, 40.59082461853175], [-73.80579246867174, 40.59078869148326], [-73.8057874034242, 40.590767375490245], [-73.80578669939553, 40.59076583503742], [-73.80578572285154, 40.590764374203175], [-73.80578446296637, 40.59076301290543], [-73.80578293207833, 40.590761740107745], [-73.805781115628, 40.59076057488397], [-73.80577902839048, 40.59075949966858], [-73.80577408293284, 40.59075767260706], [-73.80576802802213, 40.59075625813848], [-73.80576583170394, 40.590755965772274], [-73.80576112051773, 40.59075534565509], [-73.80575824909194, 40.590755205269645], [-73.80574947643382, 40.590758009797206], [-73.80574626997947, 40.59075571742871], [-73.80574117844195, 40.59075583396588], [-73.80573192380764, 40.59075774574914], [-73.80572770393, 40.59075873826445], [-73.80572399023131, 40.590760235564765], [-73.80572071231869, 40.59076195926416], [-73.80571790763142, 40.59076388195132], [-73.805715573999, 40.5907659942409], [-73.80571180498656, 40.59077048989375], [-73.80570857722209, 40.5907754464982], [-73.80570373711976, 40.59078673098863], [-73.80570106413039, 40.5907998103706], [-73.80570056008588, 40.59081465951825], [-73.80570748366095, 40.590875384657444], [-73.80570722859026, 40.590926192297346], [-73.80570895373269, 40.59094235060543], [-73.80571761728746, 40.59099337084207], [-73.80571725830877, 40.591007381916114], [-73.8057125726178, 40.59103847872931], [-73.80571268723433, 40.59105180658029], [-73.80572448497985, 40.591130271468565], [-73.8057325340218, 40.59117301452977], [-73.80573576915708, 40.591190156946375], [-73.80575243265774, 40.59128896414003], [-73.80579662785689, 40.59151971733705], [-73.8058091059093, 40.59159925488233], [-73.80581938310792, 40.59169275599108], [-73.80582787272633, 40.591735031542335], [-73.8058298721609, 40.591750064343806], [-73.80582955757521, 40.59176090750589], [-73.80582548309859, 40.59178888829706], [-73.80582589879396, 40.59180418864761], [-73.80582979202732, 40.59183876598117], [-73.80582967597115, 40.59185441774794], [-73.80582944725386, 40.59185575843655], [-73.80582900629432, 40.59185708184603], [-73.80582835307048, 40.59185839551511], [-73.80582749818397, 40.591859681368675], [-73.80582515259432, 40.59186219051787], [-73.80582201534749, 40.59186457100658], [-73.80581805080675, 40.59186683483684], [-73.8058134261234, 40.59186886150206], [-73.80580830610067, 40.59187058259356], [-73.80580302092355, 40.59187188843459], [-73.80578937244614, 40.5918738547828], [-73.80577447345892, 40.59187447794066], [-73.80575828067309, 40.59187375666177], [-73.80574055429527, 40.591871701597036], [-73.8057136358894, 40.59186675601148], [-73.80568650889998, 40.591859857508474], [-73.80565937714404, 40.5918510496552], [-73.80563248630229, 40.591840414119204], [-73.8055281962188, 40.59179303249294], [-73.80540889704737, 40.59173592572224], [-73.80535394294343, 40.59171199568678], [-73.80533561774497, 40.59170467006639], [-73.80527935890794, 40.59168218808927], [-73.80525481375378, 40.59167342958878], [-73.80523372006142, 40.59166796403305], [-73.80522630359638, 40.59166325024416], [-73.8051702731144, 40.59164327132655], [-73.80509962356163, 40.59161387587225], [-73.80503545939035, 40.59158399515519], [-73.80488383522498, 40.591508508756846], [-73.80480575004528, 40.59146895958783], [-73.80476400054194, 40.591449716730416], [-73.80471983452654, 40.59143258548361], [-73.80470994408527, 40.59142950656728], [-73.8046985518073, 40.591426795673215], [-73.80465046401255, 40.59141805130664], [-73.80463232266452, 40.591412913991526], [-73.80455694652761, 40.591383806693514], [-73.80448669213679, 40.59136051664996], [-73.80436184785303, 40.59131367428318], [-73.8043598772318, 40.59131307753788], [-73.80435826055427, 40.59131258777654], [-73.8043544708548, 40.59131172608509], [-73.80434913397227, 40.59131103297447], [-73.8043395424287, 40.59131058408128], [-73.80433496306853, 40.59131334085864], [-73.80432874341584, 40.5913113671547], [-73.8043197954532, 40.59131305368404], [-73.80431121286246, 40.591315596744046], [-73.80430333508215, 40.59131887880457], [-73.80429647253338, 40.59132278496651], [-73.80428263197656, 40.5913333244758], [-73.80427544085889, 40.591340461892074], [-73.8042704176468, 40.59134544287283], [-73.80426018397826, 40.591358780070514], [-73.80425229907365, 40.5913728863764], [-73.80423021792889, 40.59142574558964], [-73.80422320774693, 40.59144799515944], [-73.80421838603534, 40.591468870525716], [-73.80420358554483, 40.59155299999552], [-73.80419003434086, 40.591622001650265], [-73.80417218532634, 40.59168615956413], [-73.80416741727521, 40.591709033815995], [-73.80416445531056, 40.59174027741094], [-73.80416331320505, 40.59178298782879], [-73.804163140454, 40.59178958770142], [-73.80416282661457, 40.59182024174899], [-73.80416389309873, 40.59184333284309], [-73.8041641010062, 40.59184482806478], [-73.80416440502263, 40.59184599894534], [-73.80416569103024, 40.59185470983277], [-73.80417596602165, 40.591859463240155], [-73.80417813448955, 40.591860944212456], [-73.80418269897227, 40.59186350078752], [-73.8041875828177, 40.59186574228067], [-73.80421458092256, 40.59187607209371], [-73.80427957013401, 40.59189833559109], [-73.8043084328775, 40.591909146520095], [-73.80438421369706, 40.5919416126328], [-73.8043997140324, 40.59194885234437], [-73.80442009939036, 40.59195838798625], [-73.80450016953385, 40.59199848911672], [-73.80453792098572, 40.59201560082394], [-73.80462483579102, 40.59204956349492], [-73.80464277339345, 40.59205596966904], [-73.80481573709523, 40.59211776859751], [-73.80491764890516, 40.592156114803736], [-73.80496227091965, 40.59217290532089], [-73.80508551958042, 40.592220409079275], [-73.80511450986728, 40.59223248989636], [-73.80514561892481, 40.59224690564197], [-73.80527174695679, 40.592311759099765], [-73.80532893956504, 40.59233819636061], [-73.80535062910116, 40.59234635594698], [-73.8053846249805, 40.59235915717559], [-73.80551114077319, 40.59240073998178], [-73.80556324806898, 40.592421019509224], [-73.80562173943827, 40.592447925894064], [-73.80574710995971, 40.59250950904214], [-73.80579868944848, 40.592532991239956], [-73.80600051162031, 40.5926159282937], [-73.80606952150035, 40.59264551911171], [-73.8062101515529, 40.59271267474632], [-73.80625780301824, 40.592734188172784], [-73.80630706180499, 40.59275497419454], [-73.80634574672236, 40.592771292452774], [-73.80650324063359, 40.59283772896337], [-73.80669255942881, 40.592915177751046], [-73.80688541131894, 40.59299251472488], [-73.80701238174966, 40.59304869567286], [-73.80703934800665, 40.593058422497], [-73.80706410061508, 40.593065082318134], [-73.80708731483065, 40.59306885804147], [-73.80710917781217, 40.59306977745653], [-73.80711067889034, 40.59306968063114], [-73.80711216797057, 40.59306947673415], [-73.80711364593867, 40.59306916358911], [-73.80711509962897, 40.5930687339703], [-73.80711802121199, 40.5930675229441], [-73.80712097936073, 40.59306582614309], [-73.80712480334842, 40.5930630048285], [-73.80713670677899, 40.59305223606514], [-73.80713982037585, 40.59304982919891], [-73.80714273144689, 40.59304799611679], [-73.80714541698477, 40.593046713326146], [-73.8071480889948, 40.59304585318993], [-73.80714943722005, 40.59304556730158], [-73.80715078362202, 40.59304538008511], [-73.80715215498121, 40.59304530230739], [-73.80715353619463, 40.59304531283429], [-73.80720102715897, 40.59304876975507], [-73.80724914926088, 40.59305482978539], [-73.80729558299446, 40.59306062561803], [-73.80730837885626, 40.59306395293842], [-73.80731386523037, 40.5930660591069], [-73.80731885591649, 40.59306850821481], [-73.8073424062986, 40.5930825236951], [-73.80737427670415, 40.59310354195218], [-73.80742595444788, 40.593138271857214], [-73.80747119169197, 40.59316911799778], [-73.80755723037738, 40.59322596872831], [-73.80761324036321, 40.59326986414689], [-73.80765636672551, 40.59329558278543], [-73.80772997404883, 40.59333319608832], [-73.80773892789193, 40.59333731671861], [-73.8077494426433, 40.593341324196636], [-73.80779266428885, 40.59335474174367], [-73.80781002253813, 40.59336123641584], [-73.80781870474202, 40.59336543783739], [-73.80782611143395, 40.59336993482544], [-73.80783240566706, 40.593374809072124], [-73.80783763517655, 40.59338012130346], [-73.80787257345793, 40.59342724985865], [-73.80792061763908, 40.5934890883047], [-73.80793174413228, 40.59350574862727], [-73.80795507620142, 40.59354694161515], [-73.80796983416096, 40.59356651698752], [-73.8080228385052, 40.59362446442616], [-73.80802332090913, 40.5936249966361], [-73.80804688683963, 40.59365031287615], [-73.80802985692688, 40.59366908745205], [-73.80800324329972, 40.59367278911918], [-73.80800171159981, 40.593700117680825], [-73.80798065192212, 40.59372333426592], [-73.80795000725702, 40.59373222561915], [-73.80793600951917, 40.59374410704813], [-73.80793051874187, 40.59374724943838], [-73.80791316053892, 40.5937569726593], [-73.8078992259712, 40.59376759436648], [-73.80789358565342, 40.593773213430694], [-73.80788884474839, 40.59377902280416], [-73.8078578695703, 40.59382394300738], [-73.80785236547301, 40.59383193987792], [-73.80782939829089, 40.593865319446685], [-73.80781782542283, 40.59388439076516], [-73.80781540169887, 40.5938895375779], [-73.8078135430893, 40.593895037315626], [-73.80781142831069, 40.59390737987553], [-73.80781234323437, 40.59394654598187], [-73.80781177664778, 40.593958539821365], [-73.80780972642636, 40.59396899191706], [-73.8078019702852, 40.59399585823754], [-73.80780083771035, 40.594003754863095], [-73.80780083641612, 40.59400420451047], [-73.80780090001183, 40.594010895247976], [-73.80780131750262, 40.594013517282434], [-73.80780206552055, 40.59401606632431], [-73.80780315809604, 40.59401855429146], [-73.80780456820631, 40.5940209781233], [-73.80780844063125, 40.5940256946958], [-73.8078137442953, 40.59403030493491], [-73.80782594807337, 40.59403798882024], [-73.80786098223899, 40.594056112124704], [-73.80786953030969, 40.59406189881814], [-73.80787295925681, 40.59406477718192], [-73.8078758819427, 40.59406769976455], [-73.80788026060581, 40.59407314677242], [-73.80788434031027, 40.59407961102293], [-73.80790059974898, 40.594111138132085], [-73.80790676774026, 40.59411897391959], [-73.80791018390353, 40.59412224863597], [-73.80791388441594, 40.59412515392202], [-73.80791650320944, 40.59412676122308], [-73.80791955906287, 40.59412816235546], [-73.80792307563598, 40.59412938366103], [-73.80792712430092, 40.5941304438545], [-73.80795585477743, 40.59413459919688], [-73.80796095498302, 40.59413560686697], [-73.80796515805487, 40.59413677604408], [-73.80796926358724, 40.59413846657757], [-73.80797095038997, 40.59413944157088], [-73.80797242438499, 40.594140516726235], [-73.80797368470348, 40.5941416886918], [-73.80797473241036, 40.594142969028724], [-73.8079755668741, 40.5941443481869], [-73.80797618914225, 40.59414584409365], [-73.80797721674685, 40.59414983519391], [-73.80797768884928, 40.59415395084572], [-73.8079775709933, 40.59415786866439], [-73.80797687598293, 40.59416141594815], [-73.80797565026897, 40.59416441234536], [-73.80797484437612, 40.594165707678016], [-73.8079739069837, 40.59416687647356], [-73.80797286407389, 40.59416790185512], [-73.80797168046269, 40.594168790799685], [-73.8079703913573, 40.59416952812132], [-73.80796899539685, 40.5941701285602], [-73.80795183804565, 40.59417561965375], [-73.80794340991892, 40.59417771359337], [-73.80793410535998, 40.59418003005471], [-73.80792795958978, 40.594181045733926], [-73.80790787186201, 40.59418276887226], [-73.80790290865251, 40.5941834901671], [-73.80789855837253, 40.59418444568758], [-73.80789674906913, 40.59418502901666], [-73.80789508124975, 40.594185782122224], [-73.80789353092956, 40.59418671551855], [-73.80789210975456, 40.59418782972779], [-73.80789081840769, 40.59418911654196], [-73.80788965623967, 40.594190572609335], [-73.8078877096954, 40.594194017952724], [-73.80788627986357, 40.5941981409796], [-73.8078853449169, 40.59420296594524], [-73.80788489525852, 40.5942151131023], [-73.80788671364523, 40.5942281018668], [-73.80789459806354, 40.594255112950734], [-73.80789688004687, 40.594266923248874], [-73.80789919253861, 40.594284171609885], [-73.80789918943552, 40.594293500659845], [-73.80789837337375, 40.59429779492294], [-73.80789694444839, 40.59430160232551], [-73.80789489176581, 40.59430496623955], [-73.80789362345557, 40.59430649517257], [-73.80789218972515, 40.594307923981425], [-73.80785053280607, 40.59434142593154], [-73.80779319455692, 40.59438541110755], [-73.80778114829683, 40.5943962790439], [-73.80776973533688, 40.594409236295306], [-73.80775866888293, 40.594424508547014], [-73.80774538902782, 40.59444560377509], [-73.80774339247085, 40.59444969787449], [-73.80774194896011, 40.594454521317964], [-73.80773961922722, 40.59447159120238], [-73.80773816031667, 40.59447779372389], [-73.80773694668152, 40.59448086368424], [-73.80773542660151, 40.59448358014689], [-73.80773356514105, 40.59448601659886], [-73.80773136248801, 40.594488184097465], [-73.80772715937749, 40.594491219905656], [-73.80772188266538, 40.594494030097664], [-73.80770022809358, 40.59450238625926], [-73.80769570915837, 40.59450443294537], [-73.80769192235334, 40.59450653295833], [-73.80762339890732, 40.5945535795301], [-73.80761706906092, 40.594556990055665], [-73.80760947586984, 40.5945602019536], [-73.80757608116296, 40.59457161830038], [-73.80756932662158, 40.59457467831049], [-73.80756359977514, 40.59457795481397], [-73.80755241222474, 40.59458618590764], [-73.80754226042313, 40.59459565108296], [-73.80753338160471, 40.59460616360671], [-73.80752600137625, 40.59461745329685], [-73.80752076979171, 40.59462930429164], [-73.80751187239565, 40.5946584567991], [-73.80750844940488, 40.594665781324274], [-73.80750437881883, 40.594672087524536], [-73.807471499583, 40.59471377982106], [-73.8074431545658, 40.594752159101155], [-73.80744093502183, 40.594755909721], [-73.8074392216071, 40.59475996860559], [-73.80743725731814, 40.594769096181366], [-73.80743722690852, 40.59477964111512], [-73.80743911524135, 40.5947925849379], [-73.80744929111644, 40.59481579009958], [-73.80745390122084, 40.594831143537846], [-73.8074548738101, 40.59483827723872], [-73.80745484170708, 40.59484491353606], [-73.80745383192628, 40.59485113489949], [-73.80745202143406, 40.59485639095589], [-73.807451818213, 40.594856976298914], [-73.807446737245, 40.59486581921311], [-73.80743943494979, 40.5948745327575], [-73.80742998350355, 40.59488300933113], [-73.80741856029397, 40.59489108572201], [-73.80740443278832, 40.59489861796651], [-73.80736974006633, 40.594913553550455], [-73.80736180027915, 40.5949178177747], [-73.8073551262955, 40.59492219167442], [-73.80733773154965, 40.59493591401392], [-73.80732310339948, 40.594949532262824], [-73.80731110851839, 40.594963173018584], [-73.80730171210539, 40.59497686319602], [-73.80727482562536, 40.59502556319677], [-73.80723502281842, 40.595091811361954], [-73.80720750224992, 40.59514353603544], [-73.80720081171643, 40.59515309762062], [-73.80719157996631, 40.59516287122246], [-73.80715261007045, 40.59519475590738], [-73.80713865341457, 40.595208465893094], [-73.8071360094414, 40.595211991807396], [-73.80713375349453, 40.595215931836016], [-73.80713030001894, 40.595225264308375], [-73.80712512047312, 40.59525991699092], [-73.80712362480558, 40.59526641697525], [-73.80712169472778, 40.59527189781788], [-73.80711866673239, 40.59527750263232], [-73.80711685230982, 40.59527992156264], [-73.80711480139865, 40.59528211560621], [-73.80711253952029, 40.595284075424296], [-73.80711002967402, 40.59528582792702], [-73.80710728510584, 40.59528735269798], [-73.80710428091606, 40.595288672312], [-73.80709037585275, 40.595293034057946], [-73.80708510301046, 40.595294169266026], [-73.80705494713384, 40.595300637550224], [-73.80703942357032, 40.59530525021256], [-73.80698841354582, 40.59532481459914], [-73.80696619886479, 40.59533498804796], [-73.80693313000738, 40.59535233999461], [-73.80692963899834, 40.59535442288865], [-73.80692633587272, 40.595356894264455], [-73.80692013891135, 40.59536310573507], [-73.80690468005695, 40.595385917062785], [-73.80689889446067, 40.59539305817521], [-73.80689255705727, 40.59539858538485], [-73.80688913728866, 40.59540073155611], [-73.80688549372952, 40.59540251783259], [-73.80686762450455, 40.595408710857335], [-73.80684657127279, 40.59541342125137], [-73.80677388004736, 40.59542224981925], [-73.80673869879953, 40.59542617116167], [-73.80666648810744, 40.595432145932634], [-73.80665375101495, 40.59543319602255], [-73.80661100431212, 40.59543896836372], [-73.80655072109633, 40.59545054773365], [-73.80648696866679, 40.59546471391846], [-73.80641841167686, 40.59548176169289], [-73.80640943777072, 40.595484186812016], [-73.8063864892022, 40.5954903886281], [-73.80637505690609, 40.595493485249335], [-73.80634058103588, 40.5955028017772], [-73.8062655672258, 40.59552618845428], [-73.80623717982269, 40.59553405552983], [-73.80618258993553, 40.59554603129968], [-73.80613590532047, 40.5955527333131], [-73.80609327973673, 40.595557623455214], [-73.80607074853644, 40.595558656451075], [-73.8060600346754, 40.59555811685038], [-73.80605033944218, 40.59555675773775], [-73.80604149807385, 40.595554555213546], [-73.80603347280245, 40.595551497654746], [-73.80599855087524, 40.59553549139253], [-73.8059373439846, 40.59550736302133], [-73.80591381395588, 40.595498174322636], [-73.80586588105912, 40.59548346902585], [-73.80584647879436, 40.595476655485385], [-73.80583652226585, 40.59547204550335], [-73.80582632109676, 40.59546585898852], [-73.80578720765428, 40.5954368687348], [-73.80577826735704, 40.595431737940395], [-73.80576945390293, 40.59542780553139], [-73.80571002345427, 40.59540594045551], [-73.8056496334033, 40.59538546857757], [-73.80554970079176, 40.59535332146851], [-73.80547628368342, 40.595327397149816], [-73.80546775369838, 40.595323565212816], [-73.8054590623756, 40.595318307994866], [-73.80542613930707, 40.59529341615489], [-73.80541867347472, 40.59528913469108], [-73.80541126393751, 40.59528589820453], [-73.80540982355363, 40.595285436734265], [-73.80540832497651, 40.5952851098593], [-73.80540675419151, 40.595284900132874], [-73.80540512414001, 40.59528481612062], [-73.8054016385436, 40.59528502649887], [-73.80539779744554, 40.59528573199521], [-73.80539048986736, 40.59528800656691], [-73.8053736005161, 40.5952946511712], [-73.80536955611655, 40.595295887223614], [-73.80536600938741, 40.59529663711013], [-73.80536263102323, 40.595296928920774], [-73.80536107050737, 40.59529689059373], [-73.80535958247123, 40.59529671736072], [-73.80535817834652, 40.595296408068165], [-73.8053568321431, 40.595295982105604], [-73.80535555841846, 40.595295421571855], [-73.80535434332423, 40.595294727281306], [-73.8053532710781, 40.59529395968715], [-73.80535228099234, 40.59529312907355], [-73.80535057338746, 40.59529122622769], [-73.80534919793054, 40.595288998936866], [-73.80534811804199, 40.59528640492175], [-73.80534694039684, 40.595281232795124], [-73.80534545308215, 40.59526874965364], [-73.80534389739496, 40.595263361108465], [-73.80534275776517, 40.595261206757336], [-73.80534135686115, 40.59525934061723], [-73.80533967275775, 40.59525774472517], [-73.80533768021503, 40.5952564073115], [-73.80531166853439, 40.595243414378174], [-73.80528024608601, 40.59523091619975], [-73.80521822973037, 40.59520864843651], [-73.80517819688355, 40.59519549667705], [-73.80517162088512, 40.59519387322306], [-73.80515328334947, 40.59519090684529], [-73.80514501976303, 40.59518888415321], [-73.80510965013248, 40.5951767124514], [-73.8050581169696, 40.59515714620452], [-73.80501528397882, 40.59513972148564], [-73.80496564817823, 40.59511760141171], [-73.80490619437862, 40.59508789063374], [-73.8048529847726, 40.595056336043314], [-73.8048366440257, 40.59504461924752], [-73.80480294935424, 40.595017437797395], [-73.80478846021549, 40.595007228890964], [-73.8047320298805, 40.594973812712], [-73.80470909685009, 40.59495840172421], [-73.80467204999985, 40.5949302616301], [-73.80464046899166, 40.5949070726052], [-73.80463907835683, 40.59490583722306], [-73.80463794885962, 40.5949045111486], [-73.804637055003, 40.59490309551121], [-73.80463639744458, 40.59490159081495], [-73.80463598876811, 40.59489997781494], [-73.80463581660986, 40.59489827575613], [-73.80463616986167, 40.59489449454271], [-73.80463712460106, 40.59489110084267], [-73.80463873183612, 40.59488718622956], [-73.80464772722588, 40.594869299304925], [-73.80464923932395, 40.594864924996116], [-73.80464992395905, 40.59486108068474], [-73.804651670683, 40.594841713542394], [-73.80465086460816, 40.59483050075372], [-73.80464939025944, 40.59482565312272], [-73.80464474472976, 40.59482262868672], [-73.80464408404325, 40.59481806187834], [-73.80464226996291, 40.59481658151806], [-73.80464022928463, 40.594815255067296], [-73.80462863472779, 40.594809147328355], [-73.80461501951793, 40.59480320753547], [-73.8045662681746, 40.59478561879295], [-73.80454797933899, 40.594778150377074], [-73.80451749247611, 40.594764797168786], [-73.80450888258939, 40.59476022504577], [-73.8045017005284, 40.59475559386781], [-73.80449229719711, 40.59474781555164], [-73.80448329943941, 40.594738280706395], [-73.80446136708883, 40.594708040139665], [-73.80443848374023, 40.59466783864812], [-73.80442801227665, 40.59465324976916], [-73.80437012156551, 40.5945864590204], [-73.80431816425566, 40.59452639444167], [-73.80430854967291, 40.59451385187752], [-73.80428600162014, 40.594479954563504], [-73.8042607906567, 40.5944482125013], [-73.80423662846725, 40.59442134515491], [-73.80421505847822, 40.59440071247972], [-73.80417646705253, 40.59436728252363], [-73.80413054129566, 40.59433344367824], [-73.80411459954651, 40.594319170453375], [-73.80410298118355, 40.59430496708519], [-73.80409224487823, 40.59428818073962], [-73.80406481365787, 40.59423196814821], [-73.80404215171487, 40.59418887555853], [-73.80403735959106, 40.5941754683651], [-73.8040350912002, 40.59416315408304], [-73.80402722367894, 40.59407086466389], [-73.80401462360645, 40.59392399388458], [-73.80401191331644, 40.59388163743026], [-73.80400956805103, 40.593815102042996], [-73.80400883285976, 40.59376772490985], [-73.80401015818572, 40.59372688008922], [-73.80401102960299, 40.59371998220158], [-73.80401285848261, 40.593712511822794], [-73.80402625236006, 40.59367717987675], [-73.80402859239526, 40.59366852074118], [-73.80402950979618, 40.59366212853634], [-73.80402972598941, 40.593660651628454], [-73.80402978935507, 40.59364347860633], [-73.80402608266859, 40.5935900259677], [-73.8040270202504, 40.59358072598496], [-73.80402929189894, 40.59357103022671], [-73.80403789382356, 40.59355046873564], [-73.80403903338974, 40.59354865147445], [-73.80404047940992, 40.59354695167133], [-73.80404223259575, 40.593545351569794], [-73.80404429226036, 40.59354386038237], [-73.80404935685054, 40.59354116743554], [-73.80405576668511, 40.59353883764103], [-73.80406973970598, 40.593535601171745], [-73.80410356478987, 40.59353018346805], [-73.80411074089258, 40.59352847584183], [-73.80411633350981, 40.59352655794666], [-73.804118889223, 40.59352535625706], [-73.80412103093943, 40.59352406272572], [-73.8041227714809, 40.59352265157481], [-73.80412412334559, 40.5935211327098], [-73.80412464566962, 40.59352033213826], [-73.80412508524886, 40.593519494569165], [-73.8041256686377, 40.59351773063855], [-73.8041258636516, 40.59351583101718], [-73.80412568015268, 40.59351380510313], [-73.80405944647693, 40.593355280659665], [-73.80405265595694, 40.59325817374548], [-73.80403055905919, 40.59321189486048], [-73.80400505652692, 40.59317906245953], [-73.80394974576262, 40.59312704463244], [-73.80392744479627, 40.59310606972992], [-73.80390545276721, 40.59308856153904], [-73.80389765765679, 40.59308382054931], [-73.80388812529323, 40.593079373624285], [-73.8038457614602, 40.59306379426807], [-73.80383748195098, 40.59305966409821], [-73.8038305837224, 40.593055222670245], [-73.8037856886146, 40.59301661168232], [-73.80378278856868, 40.59301466171095], [-73.80377921433735, 40.5930127650354], [-73.80376605945018, 40.59300723101289], [-73.80376135214232, 40.59300494423552], [-73.80375720216702, 40.59300220876016], [-73.80375556435362, 40.59300074744986], [-73.80375419863284, 40.59299920468211], [-73.80375259802916, 40.59299689741053], [-73.80375130673907, 40.59299448143771], [-73.8037503456632, 40.592991949093125], [-73.80374969084525, 40.59298930083849], [-73.80374931853149, 40.59298362681031], [-73.80375018745143, 40.592977360171616], [-73.80375446436813, 40.592965093246924], [-73.80376827508375, 40.59293673278252], [-73.80376946328023, 40.5929342952444], [-73.80377115951049, 40.59293173175404], [-73.80377612618894, 40.592926074381786], [-73.80379981215998, 40.59290579062862], [-73.80380392365605, 40.59290179955293], [-73.80380676878129, 40.5928983553097], [-73.80380799040506, 40.592896358096326], [-73.8038087764684, 40.59289450387974], [-73.80380911207457, 40.59289277638417], [-73.80380912656838, 40.59289195735643], [-73.80380902231202, 40.592891163925344], [-73.80380873853996, 40.59289118153407], [-73.8038081147172, 40.59289054971977], [-73.80380729893427, 40.59289070094645], [-73.80380627268089, 40.592890230948925], [-73.80380503347197, 40.59288976109022], [-73.80380317955768, 40.59288959542119], [-73.80379927199998, 40.59288865225863], [-73.80379329848462, 40.59288738273863], [-73.80378816362871, 40.59288580555809], [-73.80378116338818, 40.592884226867405], [-73.80377066646705, 40.59288263299105], [-73.80374204069533, 40.592883277342125], [-73.80370931670697, 40.59288361782297], [-73.80369772652469, 40.592884463147406], [-73.8036916381217, 40.592885569331635], [-73.80368635419893, 40.59288715451708], [-73.80368177512447, 40.59288924483574], [-73.80367972852247, 40.59289048306917], [-73.80367785840275, 40.59289186768995], [-73.80363926449387, 40.592923643609275], [-73.80361739455434, 40.592941337743774], [-73.80328496755737, 40.59276191647635], [-73.80307523673538, 40.5926133837814], [-73.80305837855087, 40.59259351652782], [-73.8030519011422, 40.592586661503674], [-73.80304019179809, 40.592579788800194], [-73.80303153999354, 40.59257718949006], [-73.80302130187022, 40.592575722138456], [-73.80298178777937, 40.59257565439694], [-73.8029750456643, 40.59257859621791], [-73.80295487812259, 40.592599219554394], [-73.80294773193391, 40.59260325096413], [-73.80276364574277, 40.59261552449286], [-73.8024384629031, 40.59264006383509], [-73.80249228040066, 40.59319232243856], [-73.80249285356626, 40.5932863110879], [-73.80237735357416, 40.59328671578617], [-73.8024964394105, 40.593875206143494], [-73.80250965087274, 40.59394812804151], [-73.8026241357365, 40.59393071351859], [-73.80263394474773, 40.5939282732088], [-73.80270714100797, 40.593915957814346], [-73.80271741822474, 40.593914228124426], [-73.80279429173713, 40.59390129309683], [-73.80280727640996, 40.593902134432064], [-73.80288322111704, 40.593889355396556], [-73.80289671587268, 40.59388766689349], [-73.8029718298389, 40.59387502743457], [-73.80298475753881, 40.59387439573126], [-73.8030596018963, 40.59386180299612], [-73.80307025008247, 40.593860292369015], [-73.80314197343122, 40.59384822446308], [-73.80311672501837, 40.593761023733585], [-73.80313503440718, 40.59375794307124], [-73.80316549464914, 40.593863132844724], [-73.8031574006413, 40.59386449523641], [-73.80318082190897, 40.59394537813094], [-73.80316616820349, 40.59394784319243], [-73.80314275347314, 40.59386698493287], [-73.80305458988154, 40.593881818891376], [-73.80304191109988, 40.593882701821116], [-73.802945278346, 40.59389635014725], [-73.80293400176858, 40.593898773726906], [-73.80284963278892, 40.59391177673786], [-73.8028385380614, 40.59391304818359], [-73.80276170058654, 40.59392471845378], [-73.80274941413883, 40.59392796369013], [-73.80267008927831, 40.5939403712832], [-73.80265877687386, 40.59394197237121], [-73.80251304282993, 40.59396685179244], [-73.80256837601054, 40.594272266994125], [-73.8027275047448, 40.59425057915393], [-73.80273723095492, 40.59424940019141], [-73.80279473322307, 40.594242424946906], [-73.80289861369401, 40.59422982719239], [-73.80294044341211, 40.59422475409602], [-73.80323967749763, 40.59418846225912], [-73.80322027459164, 40.594089013252265], [-73.80324253170998, 40.59408723986537], [-73.80328793893025, 40.59428211589177], [-73.80325775448772, 40.59428585206265], [-73.8032402429577, 40.594204054877146], [-73.80281972468117, 40.5942608645989], [-73.80282824885298, 40.59428830098164], [-73.8027220186593, 40.594303215092175], [-73.80271634477465, 40.59427361576115], [-73.80260442850847, 40.59428775204504], [-73.80260727634301, 40.59429859845107], [-73.80270640618437, 40.59428597779938], [-73.80270983859059, 40.5943064330321], [-73.80257740915879, 40.59432212325605], [-73.80258875504278, 40.594384747406714], [-73.80233663086597, 40.59440486335874], [-73.80234513845657, 40.59449597525689], [-73.8027737033389, 40.594481825418214], [-73.80277681567533, 40.59455581805204], [-73.80304215715385, 40.594536398598], [-73.80309776811409, 40.59456301417784], [-73.80309543789551, 40.594601309137836], [-73.80311851898679, 40.59458951790862], [-73.8031857440441, 40.59466300284165], [-73.80322769006415, 40.59463957914481], [-73.8034328702208, 40.59452424437818], [-73.80347149929422, 40.594502672654755], [-73.8034177410978, 40.594446734360325], [-73.80344017041448, 40.59443420917679], [-73.80350896976357, 40.594505800011135], [-73.80348620935123, 40.59451851059976], [-73.80348397998934, 40.59451618968468], [-73.80348190694238, 40.59451638868293], [-73.80319675364962, 40.594678199883234], [-73.80335031368843, 40.59484029875073], [-73.80337754303827, 40.59486904065903], [-73.80365519313493, 40.59473064203005], [-73.80372454267375, 40.59481711197995], [-73.80371008333042, 40.594823976771146], [-73.80365167080726, 40.59475314278156], [-73.80359397066675, 40.59478296363171], [-73.80358444964375, 40.59478758608685], [-73.80352315030714, 40.59481737109676], [-73.80351665446385, 40.59482221834983], [-73.80345448960435, 40.594851701127496], [-73.80344626568802, 40.594856765387355], [-73.8033846803619, 40.59488526035868], [-73.8033610256465, 40.59488688764136], [-73.80334489744955, 40.59487108511581], [-73.80320010623193, 40.5948728464242], [-73.80320303445049, 40.594937293391084], [-73.80338350994701, 40.59493448643871], [-73.80338472349477, 40.59494957958508], [-73.8032037365531, 40.59495275556909], [-73.80320704407237, 40.595009683617015], [-73.80339097327905, 40.59500651249416], [-73.80339061688487, 40.59502044297907], [-73.80320801748975, 40.59502645047776], [-73.80321494251235, 40.59514564845063], [-73.80331306803032, 40.59514906685277], [-73.80331384838637, 40.59516528003146], [-73.80335137111726, 40.595163695756774], [-73.80335292788996, 40.59515574997719], [-73.80336665301185, 40.59515465252251], [-73.80337768921211, 40.595153768924085], [-73.80339410194205, 40.5951524554281], [-73.80345112529838, 40.59514789330489], [-73.80351421895661, 40.59514284313191], [-73.8035262195315, 40.595141883266855], [-73.80359362442029, 40.59513648961429], [-73.80360538450552, 40.59513554792576], [-73.80363981182515, 40.595132792762975], [-73.80367181134925, 40.59511491674045], [-73.80368318999163, 40.5951085616773], [-73.80375440681247, 40.5950687765873], [-73.80376521683073, 40.595062737846085], [-73.80383512197926, 40.595023686591446], [-73.80386069871201, 40.59500939792123], [-73.80389507006726, 40.594990875367245], [-73.80390842264872, 40.5950048133322], [-73.80367723202455, 40.5951343038194], [-73.8036515862445, 40.59514954741729], [-73.80331443412837, 40.59517742793796], [-73.80331453729063, 40.59517956193947], [-73.8033288308292, 40.59518814029877], [-73.80342683886519, 40.59519132453132], [-73.80349792164738, 40.59517513809814], [-73.80354001503517, 40.59516683509507], [-73.80357135342217, 40.59516370962484], [-73.80358534121645, 40.59516364357043], [-73.80359840436348, 40.59516456670431], [-73.80362294026634, 40.5951689124724], [-73.803646332867, 40.59517618453232], [-73.80366722697441, 40.59518595437375], [-73.80368468695728, 40.59519779018381], [-73.80427909056901, 40.59587879998554], [-73.80428103560175, 40.59589634532498], [-73.80429363815999, 40.595941689581856], [-73.80429973539147, 40.59595333523267], [-73.80432894848398, 40.59599038707798], [-73.80439804267058, 40.5960362518511], [-73.80449519927369, 40.59616928994784], [-73.8045703785302, 40.596263521888986], [-73.80459353615714, 40.59628696666379], [-73.80467800508394, 40.596355207199416], [-73.80469297945253, 40.596369507345806], [-73.80470623094206, 40.59638667501458], [-73.8047180211975, 40.59640697417502], [-73.80473137309313, 40.596438587419804], [-73.80473367046876, 40.596468867609694], [-73.80473460190171, 40.59648179175649], [-73.80473523397653, 40.59648758265381], [-73.80475097698772, 40.59652968178393], [-73.80477270313175, 40.59656221943952], [-73.80477863809365, 40.59657258452709], [-73.80486202466062, 40.59673011000001], [-73.80486462646938, 40.5967336283515], [-73.80495031715071, 40.59674437643974], [-73.8050096526091, 40.59676326859498], [-73.8050163175305, 40.59676392003429], [-73.80501624964808, 40.596760297258975], [-73.80509431912361, 40.59676740814035], [-73.80509981825776, 40.59670649264582], [-73.80510759382422, 40.59669046906144], [-73.80509892506618, 40.596609062110815], [-73.80513406252756, 40.59660758897049], [-73.80514809966373, 40.59672153809763], [-73.80515426905582, 40.59672110829035], [-73.80515688401542, 40.59673375752353], [-73.80521293968133, 40.59673277935769], [-73.80521738621708, 40.59676677965622], [-73.80522451261183, 40.59676670110157], [-73.80522538542246, 40.596787700908955], [-73.80520595406547, 40.596790450639794], [-73.80520916717902, 40.59680961376632], [-73.80520407924881, 40.59689225938943], [-73.80524338143303, 40.59689472284927], [-73.80524832538248, 40.596895032613546], [-73.80524737229516, 40.59694503138051], [-73.80521848902949, 40.596944450521725], [-73.80520078229813, 40.5969440949982], [-73.80520109057889, 40.596960391962625], [-73.80521063000919, 40.59696015013703], [-73.80524481796691, 40.59695928365311], [-73.80525143538655, 40.59703274470248], [-73.8052079241033, 40.59703502156488], [-73.8052217803191, 40.597188839232295], [-73.80519243494223, 40.59719037599562], [-73.80519068397012, 40.597170930044896], [-73.80518866803234, 40.597157856303404], [-73.80518573178918, 40.597137201627575], [-73.80518463880799, 40.59712891761403], [-73.80518167318017, 40.59710424887901], [-73.80518073771371, 40.597095612147584], [-73.80517870389916, 40.5970748821053], [-73.805177951163, 40.59706628957624], [-73.80517643161772, 40.597046771979734], [-73.80517553345443, 40.59703318481173], [-73.80517425241224, 40.597008983826875], [-73.80517300904616, 40.59696735128349], [-73.80517284271625, 40.59694639606395], [-73.80517362267638, 40.59689827886409], [-73.80517470455591, 40.59687255300976], [-73.8051751750359, 40.596863839587606], [-73.80517691041392, 40.59683770590141], [-73.80517761958872, 40.59682868663975], [-73.80518115407425, 40.59679148514447], [-73.80516310470956, 40.59679204575619], [-73.80515744715044, 40.596769287821786], [-73.80516503589428, 40.59676847996068], [-73.80516341154969, 40.59675799203129], [-73.80514962023199, 40.59675741965358], [-73.8051370606517, 40.59682203580175], [-73.80510231687563, 40.596819158531865], [-73.8051026419087, 40.59681118484153], [-73.80501035319556, 40.596806042102045], [-73.80501106896821, 40.59679371749341], [-73.8049736528738, 40.59678016384713], [-73.80497148353813, 40.59678712102054], [-73.80494572745903, 40.59678109683486], [-73.80495249836117, 40.59676487939935], [-73.80493010902488, 40.59675715430032], [-73.80487565214911, 40.596748535931646], [-73.80494117146355, 40.59685653723154], [-73.80508458771831, 40.59710107810031], [-73.80511185479908, 40.597151047998054], [-73.8050845721135, 40.59717903581115], [-73.80505481227492, 40.59720551630663], [-73.80502098047307, 40.597228880888075], [-73.8050089049317, 40.5972378550633], [-73.80496957533927, 40.597269743253925], [-73.80493314568685, 40.597303573832285], [-73.80484578876006, 40.59741506485055], [-73.80476515501009, 40.597529461048694], [-73.80475076026305, 40.5975512759179], [-73.8047298146228, 40.59762533029136], [-73.80472959936768, 40.597635676144975], [-73.80472761310365, 40.59764591193186], [-73.80472388986286, 40.59765586226717], [-73.80471849344073, 40.59766535665723], [-73.80471112228027, 40.59767466319516], [-73.80470214568697, 40.59768311083398], [-73.80469173344079, 40.597690539797384], [-73.80468018577389, 40.59769585634601], [-73.80466801290432, 40.59770028907284], [-73.80466667714964, 40.59770070897699], [-73.8046446977668, 40.597705946068814], [-73.80462188261247, 40.597708328269334], [-73.80459886525753, 40.59770778942537], [-73.80458138264058, 40.59770656186115], [-73.80456298434068, 40.597708134669546], [-73.80454475876998, 40.597709338868626], [-73.80451363490388, 40.59771386970599], [-73.80449561721734, 40.59771801121211], [-73.80448509590953, 40.597720051916085], [-73.80447756524427, 40.59772215198696], [-73.80446414395617, 40.59773047396787], [-73.8044563632081, 40.597731887410625], [-73.80444431269628, 40.597727530736826], [-73.80443910360154, 40.597723103279556], [-73.8044358272784, 40.597723480844266], [-73.80442940256619, 40.59772835624638], [-73.80442416889065, 40.59773254630136], [-73.80442085683887, 40.597735967314875], [-73.80441857054221, 40.59773832944421], [-73.80440580135847, 40.59774574552294], [-73.8043898349653, 40.59775016441469], [-73.80437866087794, 40.59775423512657], [-73.80436267120022, 40.59776727488013], [-73.80435822020667, 40.59777467713658], [-73.80435118081824, 40.597780428672834], [-73.8043484398126, 40.597782181224154], [-73.80433914461089, 40.597783565275876], [-73.80432404105332, 40.59778889196269], [-73.80431282695541, 40.59779408739367], [-73.80430791850921, 40.59779784057721], [-73.80429952498113, 40.59779903283201], [-73.80429452398953, 40.59779754218128], [-73.8042900591187, 40.59779745332579], [-73.80427924903044, 40.597799932111606], [-73.80427294307228, 40.59780140300344], [-73.80426238305888, 40.59780456958749], [-73.80424670540191, 40.597809675654936], [-73.80424035951756, 40.59781224496843], [-73.80423360865741, 40.59781755839648], [-73.80423190049063, 40.597823182303024], [-73.80423207114204, 40.59782614702989], [-73.8042326047061, 40.5978286451348], [-73.80422806444808, 40.597829680439574], [-73.80421769770996, 40.59782831835391], [-73.80420966950642, 40.5978279198609], [-73.80420311288294, 40.59782867781846], [-73.80419463977586, 40.59783212236537], [-73.8041847304747, 40.597834410018024], [-73.8041708939004, 40.59783795465341], [-73.80415718796982, 40.59784558873946], [-73.80414434347911, 40.597854159777334], [-73.80412543979935, 40.597866012107666], [-73.80411123700334, 40.597871147475296], [-73.80409956610482, 40.59787269213958], [-73.80408649461421, 40.59787195785554], [-73.80407482429868, 40.59787352831791], [-73.804065864218, 40.59787806884137], [-73.80405722417916, 40.5978824469034], [-73.80404748778118, 40.59788767397824], [-73.80403554415484, 40.59789602355816], [-73.80398037247112, 40.597921814126465], [-73.803955819145, 40.597933080825435], [-73.80393338922408, 40.597943498583916], [-73.80392075541613, 40.59795388044847], [-73.80391880964353, 40.597966311174716], [-73.8039144548259, 40.59800253620614], [-73.80390637589971, 40.59801940236728], [-73.8038984136408, 40.59803283788846], [-73.80388859776922, 40.598040342881525], [-73.80388685014029, 40.598047118481375], [-73.8038896446305, 40.5980517082], [-73.80388273645939, 40.59806154966371], [-73.80387647055754, 40.59806186783111], [-73.80386762022486, 40.59805941098298], [-73.80386126093074, 40.598054486319036], [-73.8038565878277, 40.59805143165019], [-73.80385241060453, 40.598052027794516], [-73.80384868845704, 40.59805624854796], [-73.80384964249717, 40.59806239901485], [-73.80385182215109, 40.598067919316996], [-73.80385593800585, 40.59807706642114], [-73.80385746539623, 40.598083464972774], [-73.80385748008983, 40.59809095775862], [-73.80385331601502, 40.59809904785898], [-73.80384570372395, 40.598103400184435], [-73.80384062878422, 40.59810306165635], [-73.80381966983131, 40.59809103059309], [-73.80381154878697, 40.5980854172205], [-73.8038059751034, 40.598082553671006], [-73.80380093554906, 40.598082215704096], [-73.80379282787145, 40.598084096286485], [-73.80378259246703, 40.598086795638054], [-73.80376990111107, 40.598091987991204], [-73.80371151197171, 40.59811096738205], [-73.80370714924571, 40.598112385607], [-73.80369294305646, 40.59811864638088], [-73.80369221364158, 40.59811982487824], [-73.80368751590298, 40.598127393044074], [-73.80368332624671, 40.598144101319036], [-73.80368483048022, 40.598159118392964], [-73.80368599280031, 40.5981682056756], [-73.80368138795558, 40.59817901234667], [-73.80367896997076, 40.59818032633877], [-73.80367170913654, 40.598175620258175], [-73.803663626871, 40.598168853667694], [-73.80365698225997, 40.59816326994627], [-73.80365227068292, 40.598161341002246], [-73.8036495681198, 40.598161967636116], [-73.8036472836057, 40.59816737158798], [-73.80364610804297, 40.598175494764384], [-73.80364456970692, 40.598184109690415], [-73.80364023682611, 40.59818808228747], [-73.80363138229842, 40.59818675020758], [-73.80362454370491, 40.59818569548285], [-73.80361532449221, 40.598185953639636], [-73.80360361343291, 40.598188650956175], [-73.80359355082508, 40.59819431552485], [-73.80358075867733, 40.5982092528809], [-73.80357503810254, 40.59821956359604], [-73.80357453476084, 40.598232051563755], [-73.80357782268702, 40.59824029159776], [-73.80358067118338, 40.59825124821056], [-73.80357553086318, 40.59826065408488], [-73.80357063634332, 40.59827190002148], [-73.80356578073959, 40.598282020058065], [-73.8035585485594, 40.59829230071363], [-73.80354913666106, 40.59829708803306], [-73.80354200774552, 40.59829647307497], [-73.80353204588022, 40.59829239261442], [-73.8035270441307, 40.59829092873383], [-73.80352283004476, 40.59829149799846], [-73.80351914721821, 40.598294593847285], [-73.80351665263262, 40.59829818526974], [-73.80351417389636, 40.59830811670883], [-73.80351090999115, 40.59831598870457], [-73.80349615803692, 40.5983236599576], [-73.80349417737577, 40.59832468922324], [-73.80348529924201, 40.59833084985262], [-73.80345463435872, 40.59836263590645], [-73.8034480956325, 40.598369761818645], [-73.8034457717393, 40.59837629049267], [-73.80344747125586, 40.59838565310981], [-73.80344679241954, 40.59839630180452], [-73.80344033314654, 40.59840114911476], [-73.80343103518419, 40.59840365905473], [-73.80342415691189, 40.59840375752377], [-73.80341407622026, 40.59840308155364], [-73.80340842307415, 40.59840249491124], [-73.80339966262281, 40.59840637699786], [-73.80339557838491, 40.59841106418959], [-73.80339402411326, 40.59841328481571], [-73.80339685940257, 40.598416776124424], [-73.80340383015348, 40.59842192082107], [-73.80341482571609, 40.59842987215662], [-73.80343045341482, 40.59844203087837], [-73.80343320999688, 40.59844777113629], [-73.80343092465714, 40.59845314760732], [-73.8034235253518, 40.59845933873882], [-73.8034162397084, 40.598463278801674], [-73.80339972746978, 40.59848243621005], [-73.8033935514441, 40.598487970705904], [-73.8033890618123, 40.59849649932738], [-73.80338940193096, 40.598503553072256], [-73.80338610107484, 40.598511426006255], [-73.80337927223748, 40.59851899001058], [-73.8033723521624, 40.598521338159856], [-73.80335830742624, 40.598521917402685], [-73.8033464635038, 40.598520524733566], [-73.8033378983732, 40.598518752858745], [-73.803326306063, 40.598518046819265], [-73.80331394331519, 40.598521623023885], [-73.80329818618019, 40.59852900588811], [-73.80328484470323, 40.59853504963294], [-73.80326373338173, 40.598550028165164], [-73.80324373585117, 40.59856659946133], [-73.80320358175042, 40.59860432605042], [-73.80319234268016, 40.598618141228876], [-73.80316610486285, 40.59865001788153], [-73.80313217792676, 40.59868855221686], [-73.80312408419121, 40.59869789755894], [-73.80311386319177, 40.598708118116306], [-73.80310166848568, 40.598715810117646], [-73.80308620057197, 40.598722728553724], [-73.80307007829715, 40.598731676492505], [-73.80305665751189, 40.598739997639534], [-73.8030434844952, 40.59875015717671], [-73.8030272996374, 40.59876772423149], [-73.8030172458891, 40.598782035971695], [-73.80301149067553, 40.598792345426205], [-73.80300615192698, 40.59880743270039], [-73.80300191229124, 40.59881664910897], [-73.80299435528552, 40.598827395239155], [-73.80297943325748, 40.598844304553005], [-73.80295721424766, 40.59885675528617], [-73.8029215526787, 40.59887297197992], [-73.80289207302586, 40.598880414181956], [-73.80286443010492, 40.598891494138385], [-73.80283474315364, 40.59889888883744], [-73.8028005997594, 40.598907239736526], [-73.80270532053349, 40.59892217815673], [-73.80267116793497, 40.59893318769594], [-73.80265662904353, 40.598935693432466], [-73.80262990538368, 40.598935655748505], [-73.8025709047583, 40.59893139380663], [-73.80242371672547, 40.59892943908573], [-73.80235841779329, 40.598925949574614], [-73.80233534244981, 40.598922848954544], [-73.80227866522567, 40.59891220530125], [-73.8022009589097, 40.59890135588739], [-73.80219289253607, 40.59890022541899], [-73.80218256438735, 40.598897443068104], [-73.80216066276246, 40.598888841588995], [-73.80215058177286, 40.59888579781207], [-73.80209436587793, 40.5988751022577], [-73.8020758667759, 40.59887051342681], [-73.80202769328517, 40.598856436368564], [-73.80200819959828, 40.59885275600165], [-73.80198680899797, 40.59885105306522], [-73.80196349903323, 40.59885133790645], [-73.80193825657729, 40.59885358956036], [-73.80188884923457, 40.59886166308926], [-73.80176573815994, 40.598887413356366], [-73.80174596688464, 40.59888972935516], [-73.80168808986745, 40.598893177437], [-73.80167092680178, 40.598896309269506], [-73.80165643944441, 40.59890113759857], [-73.80162583428714, 40.59891525023699], [-73.80154617023564, 40.598954969993926], [-73.80149524423824, 40.59897716984119], [-73.80147213607032, 40.59898873821648], [-73.80136360351148, 40.59905041583156], [-73.80125412491208, 40.59911234534786], [-73.80124572498376, 40.59911649962689], [-73.8012349066254, 40.59912290165509], [-73.80102288887112, 40.599142301743676], [-73.80094867862815, 40.59915166461994], [-73.80095155254565, 40.59918124108186], [-73.80091083450364, 40.59918389943921], [-73.80089078534671, 40.59920057125718], [-73.8009032734675, 40.599311739891085], [-73.80089306997968, 40.5993124045616], [-73.80089596964322, 40.59933820579021], [-73.80090900567352, 40.59933735454029], [-73.80091500623944, 40.59939076113572], [-73.80088285514184, 40.59939286070252], [-73.80087685441282, 40.59933944355061], [-73.80088646326415, 40.599338816215564], [-73.80088341549074, 40.599311680189615], [-73.80087055313173, 40.59931252034548], [-73.80085916792893, 40.59921116465404], [-73.8008229985188, 40.59918872956965], [-73.80077984407792, 40.59919154616162], [-73.80077654707628, 40.59916236047825], [-73.80056517505572, 40.59917549389558], [-73.80056802039566, 40.59919942306312], [-73.80049526371978, 40.599204368068506], [-73.8005098546783, 40.59932910744505], [-73.8004737922779, 40.59933155912371], [-73.80046729855626, 40.59927604898415], [-73.80046234043184, 40.599276385654], [-73.80045131861326, 40.59918256888035], [-73.79997353177183, 40.59921225227235], [-73.79918457376515, 40.59925814415448], [-73.79917478119656, 40.59925788429216], [-73.7991604000919, 40.599258137594596], [-73.79916724875041, 40.59932103899795], [-73.79919148970744, 40.599319509092666], [-73.79919690825373, 40.59936939496672], [-73.79918842607492, 40.59936993131793], [-73.79918997630604, 40.59941675603338], [-73.79914327503613, 40.599418798862885], [-73.79913853303405, 40.599373370623425], [-73.79912877642295, 40.599373353570385], [-73.79912265687948, 40.59932234552719], [-73.79914566217283, 40.599320526996095], [-73.79914024196671, 40.59925849421589], [-73.79878847850674, 40.59926470134389], [-73.79868820989631, 40.59926289919994], [-73.79865412394807, 40.59923643959254], [-73.79861531632372, 40.59921407570161], [-73.79857262596838, 40.599196290988324], [-73.79852697576115, 40.59918346992112], [-73.79847935256511, 40.5991758896647], [-73.79843078589377, 40.59917371408809], [-73.79838232565622, 40.59917699022275], [-73.79831033792354, 40.599185811299456], [-73.79823995746294, 40.59920032882015], [-73.79817195788713, 40.59922038321027], [-73.79810708663994, 40.59924575403499], [-73.79811229286895, 40.599301040240505], [-73.79782248234716, 40.599321828289945], [-73.797690034392, 40.59932850328669], [-73.79768899035615, 40.599320536589296], [-73.79677270989202, 40.59937714163005], [-73.79676951663444, 40.599377338526345], [-73.79660826359405, 40.59938695425], [-73.79567352802896, 40.599443325709686], [-73.79561756986661, 40.59944482979853], [-73.79561833118453, 40.59946221670444], [-73.79554315704534, 40.59946730371283], [-73.7955463762112, 40.59949676214905], [-73.79546504863077, 40.59949853112459], [-73.79546353545742, 40.599472676268576], [-73.79531360293953, 40.59948278577274], [-73.79519333656766, 40.59945483905391], [-73.79507068550593, 40.59943375695542], [-73.79494632315618, 40.599419655226036], [-73.79482093231367, 40.59941261128927], [-73.79469712002859, 40.599412364832276], [-73.7945736159583, 40.59941902430574], [-73.79445108219672, 40.59943255400854], [-73.79433017563669, 40.59945288140938], [-73.79421154444945, 40.59947989753474], [-73.79409582461015, 40.599513457553456], [-73.79398363648939, 40.599553381552646], [-73.79387558152638, 40.5995994555023], [-73.79377223900744, 40.59965143240249], [-73.79367416295854, 40.59970903360729], [-73.79358187917697, 40.59977195031823], [-73.79355194144343, 40.59978384190545], [-73.79351553895094, 40.59981721206467], [-73.79348180624321, 40.59984894069674], [-73.79346903770168, 40.59985846693826], [-73.79345627135446, 40.599867993182286], [-73.79343111451105, 40.59987957988232], [-73.79341155585969, 40.59990114197143], [-73.7933853835357, 40.59992040790082], [-73.7933774308123, 40.59992652384601], [-73.7933590537064, 40.599949157894514], [-73.79334683895578, 40.59995173967587], [-73.79333658528581, 40.59994967201541], [-73.793318747224, 40.59996565861445], [-73.79330907387678, 40.59996846559873], [-73.79328887679628, 40.5999674098546], [-73.79325781619306, 40.599965755414665], [-73.79318874258787, 40.5999579659839], [-73.79316543249615, 40.59996113323081], [-73.79314362456479, 40.59997575784589], [-73.79312325093758, 40.599985147393355], [-73.79310013027344, 40.59998661873064], [-73.79303939811115, 40.59997793018574], [-73.79302610417544, 40.59997987153428], [-73.79298202592064, 40.59998632311619], [-73.79291325690718, 40.599987832966825], [-73.79289534205127, 40.59997816086266], [-73.79287566137774, 40.599958009276875], [-73.79285499477507, 40.5999478333037], [-73.79284525604315, 40.599949252315966], [-73.7928139093852, 40.59995599633718], [-73.79280540330505, 40.59995222398298], [-73.79278322675351, 40.599942400939916], [-73.79277201560392, 40.59994156552569], [-73.79275541652942, 40.59994843764551], [-73.79274275792098, 40.59999232399194], [-73.79273589624465, 40.599997921513406], [-73.79270311156472, 40.60000675502064], [-73.7926669528276, 40.60000738805587], [-73.79263370220625, 40.60000871035043], [-73.79262432704724, 40.600012023419616], [-73.7926189008937, 40.60001993124491], [-73.79261285952832, 40.60004319908422], [-73.7925965027121, 40.600052235598206], [-73.79258554038339, 40.60005131886049], [-73.79255899397361, 40.6000443848093], [-73.79253803879685, 40.600048005326535], [-73.79252458170205, 40.600036663570386], [-73.7925078288936, 40.600020212240125], [-73.79248706249774, 40.600018749068], [-73.79245722965986, 40.60001065511517], [-73.79242878573592, 40.599995830145794], [-73.79237472275223, 40.59996234081977], [-73.79235534608962, 40.599953324214106], [-73.79233256449508, 40.59995439142583], [-73.7922867200118, 40.59999268592288], [-73.79228420646572, 40.60001861864865], [-73.79226718128294, 40.60002071349122], [-73.7922606445012, 40.60001427440201], [-73.79224190801924, 40.60001015465224], [-73.79221194321207, 40.60001255428417], [-73.79219461864305, 40.600020661379865], [-73.79219580084778, 40.60003536742371], [-73.79219277190971, 40.60004984186571], [-73.79215987900972, 40.60005471646542], [-73.79213992689124, 40.60006399954949], [-73.79212903284483, 40.600078850610736], [-73.79210604552307, 40.60009049204167], [-73.79209106359569, 40.6001129547833], [-73.79208109552215, 40.60013549628636], [-73.79205879155799, 40.6001577772417], [-73.79205236431127, 40.6001698751547], [-73.79204593467385, 40.6001819636813], [-73.79203992547666, 40.60020570197197], [-73.79203025769763, 40.60022200504383], [-73.7919309831019, 40.60025276538813], [-73.7919135636537, 40.60026540277281], [-73.79188654338597, 40.600278555848426], [-73.79185432754552, 40.600287303363515], [-73.79182306911324, 40.60029456996683], [-73.79179857672624, 40.60030008540291], [-73.791766981757, 40.6003053860915], [-73.79174667556799, 40.600309408203785], [-73.79171766890654, 40.60031568189015], [-73.79169154494598, 40.6003187438909], [-73.79166252001741, 40.60032255702297], [-73.79158933627791, 40.60033492254135], [-73.79157418998489, 40.60033842978014], [-73.79154070576021, 40.60035038092102], [-73.79150983810042, 40.60036699462479], [-73.79147321564986, 40.60039126202272], [-73.79143789580179, 40.60041724442809], [-73.7914170305427, 40.60043332408108], [-73.79138201374109, 40.600456353137254], [-73.7913476245363, 40.60047716606795], [-73.79125345715042, 40.60054367745674], [-73.79116394278277, 40.60061382242475], [-73.79107932245793, 40.60068741211294], [-73.79098636367527, 40.60077810789886], [-73.79090083628613, 40.600872956907025], [-73.7908848836045, 40.600898858614926], [-73.79086987166565, 40.60092081939088], [-73.79084517544963, 40.60094282017078], [-73.79082205893033, 40.600959894094636], [-73.79078960263361, 40.6009802064472], [-73.79076000197976, 40.600993616409404], [-73.79073654738588, 40.601008723908244], [-73.79071831565926, 40.60103192823717], [-73.79070491695126, 40.60105388204501], [-73.79069791123399, 40.60106719693587], [-73.79068511656347, 40.601083243910075], [-73.79067067838217, 40.601095606594036], [-73.79064880280328, 40.60110578672244], [-73.7906204704403, 40.60111599244255], [-73.79058890078466, 40.601124982358186], [-73.79056344142451, 40.601130748403925], [-73.79053507391464, 40.601136035872635], [-73.79050959633192, 40.60113934086352], [-73.79047958817972, 40.60114094285974], [-73.79045247839791, 40.60114179541207], [-73.79043249059018, 40.601145323325554], [-73.79040350209186, 40.6011540577235], [-73.79037646383706, 40.601164749401704], [-73.79035235258114, 40.601178382704184], [-73.79032695536986, 40.60119276150756], [-73.79029900492381, 40.60121108532431], [-73.79027427221618, 40.601228165570554], [-73.79025461686722, 40.60124199646003], [-73.79008673003658, 40.60137030374593], [-73.78999783810993, 40.601431278092136], [-73.78997570071967, 40.60145775187738], [-73.78995755466649, 40.60147833272847], [-73.78993929914898, 40.60150526928182], [-73.78990757356118, 40.60153969759161], [-73.78984978803163, 40.601602254207705], [-73.78979682206725, 40.60164324816433], [-73.78974621092634, 40.60167663888602], [-73.78969738402583, 40.60170326750279], [-73.78960830868348, 40.601774832767255], [-73.78948529810154, 40.60187995698547], [-73.789399318344, 40.60196553557684], [-73.7893334495707, 40.6020131805561], [-73.78924402338541, 40.60207287693086], [-73.78907194280562, 40.60218640697253], [-73.78901099030347, 40.60223918417946], [-73.78895787564191, 40.60228865229501], [-73.78888972550826, 40.602339662855506], [-73.78885201574484, 40.60236640317209], [-73.78881266616573, 40.6023914326664], [-73.78875014623017, 40.60243826441257], [-73.78865330410798, 40.60250890229268], [-73.78857420265727, 40.60258226157871], [-73.78851556653747, 40.6026295548494], [-73.78846327872503, 40.60266335085979], [-73.78841829858403, 40.60269256060772], [-73.78836540496295, 40.60272931939144], [-73.78832266934042, 40.60275727994631], [-73.78827700707446, 40.60279368528578], [-73.78824663875106, 40.6028137213267], [-73.78820778807214, 40.602842144677105], [-73.78816768514831, 40.60287860628837], [-73.78814055596504, 40.60290418177349], [-73.78810890201653, 40.602934372893976], [-73.78807903186681, 40.60295780228287], [-73.78804089550735, 40.60297690990407], [-73.7879792848659, 40.60301620725914], [-73.7879432123765, 40.60304465809412], [-73.78789694353753, 40.60308402492366], [-73.7878575563353, 40.60311117193005], [-73.7878234143171, 40.603124388805696], [-73.7877915552983, 40.60313423818632], [-73.78776928578714, 40.603136132701636], [-73.78774716058876, 40.603129553163186], [-73.78773333842466, 40.60312517823237], [-73.78769792465073, 40.60311549997262], [-73.78766902829128, 40.60311478501596], [-73.78761995942553, 40.603123190110765], [-73.78757584187358, 40.60313461149207], [-73.78752715891446, 40.60315276731443], [-73.78747569794746, 40.603170893854724], [-73.78743204173527, 40.60318782809225], [-73.78738115026472, 40.60320511296994], [-73.7873431252405, 40.60321786633621], [-73.78730232089374, 40.60323059209747], [-73.78727495168467, 40.603237944741856], [-73.78663897364007, 40.60325551557714], [-73.78650335648598, 40.603250206178984], [-73.78635332811808, 40.60319465903842], [-73.78630485944038, 40.60315872781616], [-73.78630354951883, 40.60315059416467], [-73.78631438792186, 40.603138375778244], [-73.78631473411522, 40.60312250986259], [-73.78628810615058, 40.603114094258245], [-73.78625649061964, 40.60311372905261], [-73.78622201957894, 40.603104804131334], [-73.7861688184278, 40.603074726410995], [-73.78616632596842, 40.603067149436335], [-73.78616855124802, 40.603047449751955], [-73.78617884041063, 40.603032063377015], [-73.78620120622986, 40.6030097754698], [-73.78624389553218, 40.60297567213896], [-73.78626804341448, 40.60295174993144], [-73.7862878110677, 40.602930442584594], [-73.78634019785913, 40.60290424073625], [-73.78638878260494, 40.602874081925734], [-73.78643305290161, 40.60284028422953], [-73.78647254185027, 40.6028032041017], [-73.78650683298203, 40.60276323261459], [-73.78646693278718, 40.6027230415177], [-73.78658983800632, 40.60264611723691], [-73.78662553843012, 40.6026771163546], [-73.78676630785824, 40.60259542356703], [-73.78690284497017, 40.60250965936481], [-73.78703494719684, 40.60241995099959], [-73.7871624185521, 40.602326431574866], [-73.78716533441545, 40.60232270508558], [-73.78717833739034, 40.602306053437005], [-73.78719464253105, 40.602283084329386], [-73.78721056286666, 40.602259747282055], [-73.78721960231034, 40.60224216051117], [-73.78723151311956, 40.60221590822633], [-73.78724040001623, 40.602198319998735], [-73.78724998481145, 40.6021775453092], [-73.78726421495581, 40.602148243417204], [-73.78727070288187, 40.60213440357697], [-73.78728010373015, 40.602113270533565], [-73.78729172089956, 40.60208982884531], [-73.7873046490465, 40.60206497278436], [-73.78731850516144, 40.60204025782366], [-73.78733315550726, 40.60201219574944], [-73.78734339509619, 40.60199070858736], [-73.7873523026373, 40.60197434905111], [-73.78736942151794, 40.60194731749087], [-73.78738061112072, 40.601929880603265], [-73.78739495122065, 40.60190604823695], [-73.78741155786297, 40.60187952350493], [-73.78741930529874, 40.60186460826789], [-73.78743058075541, 40.601840286570294], [-73.78744253850779, 40.60181304843787], [-73.7874551094419, 40.60178748070311], [-73.78746708758952, 40.601764393821774], [-73.78747574429947, 40.60175120514712], [-73.78748824832351, 40.6017282827449], [-73.78750016406788, 40.601698829452225], [-73.7875120146236, 40.601676457681535], [-73.78752185315028, 40.60166078588676], [-73.78753506455548, 40.60163962535567], [-73.78754626169652, 40.601617786457766], [-73.78755333656756, 40.60160215746474], [-73.78756566424069, 40.60157357658296], [-73.78757451437741, 40.60155705609453], [-73.78758714219506, 40.60153078030234], [-73.78759671822294, 40.60151249019823], [-73.78760873824098, 40.60148885438453], [-73.78761975946907, 40.601467023865744], [-73.78763043154159, 40.601441980823054], [-73.78764094701762, 40.60141821104985], [-73.78764964021822, 40.601396219251065], [-73.7876635101432, 40.60137046458449], [-73.78767448766595, 40.60135303498229], [-73.78768902814275, 40.60133095195749], [-73.78788971616285, 40.601041052185266], [-73.78792994192655, 40.60099530305105], [-73.78800227052682, 40.60093739150589], [-73.78807155049908, 40.60085446865581], [-73.78835483974417, 40.60064281780449], [-73.78850007513702, 40.600538857801006], [-73.78862536682287, 40.60047958313282], [-73.78876061567749, 40.600380115166615], [-73.78886763442691, 40.600323471371816], [-73.78898515769973, 40.600239857576376], [-73.78908852968223, 40.60018551074722], [-73.78913278615575, 40.600152020364064], [-73.78918828231555, 40.60012826864019], [-73.78921850456057, 40.60011322982359], [-73.78923630845084, 40.60009857498711], [-73.78924924498638, 40.60008345235839], [-73.78926404974409, 40.60006066967891], [-73.78927079438951, 40.600042454823715], [-73.7892835779177, 40.60001988471624], [-73.78930418649621, 40.60000746818144], [-73.78935965392378, 40.60000419212756], [-73.78939493881202, 40.599994288782526], [-73.78946269696266, 40.59999749212843], [-73.78951637510197, 40.59998109297374], [-73.7895346213066, 40.59997181492665], [-73.78954114938688, 40.59994369480517], [-73.78954721422738, 40.59992726237672], [-73.78955983471057, 40.59991161644167], [-73.789577510785, 40.599892494475945], [-73.7896023117366, 40.599872907924585], [-73.78962635344539, 40.59985404136063], [-73.78965485679561, 40.599837019752776], [-73.78967615916372, 40.599817823659684], [-73.78970098614833, 40.59980139808361], [-73.78975070067237, 40.599760263188934], [-73.78976540612769, 40.59974667361867], [-73.7897729885848, 40.59972862394277], [-73.78978096149109, 40.59972164031226], [-73.78980075832338, 40.59971560726004], [-73.78980602655167, 40.599700713125365], [-73.78983482716296, 40.59965673968303], [-73.78987511294126, 40.59958962492249], [-73.78988889381985, 40.59956914533011], [-73.78990943888154, 40.599542887105386], [-73.78992643537947, 40.59951810955082], [-73.7899485203326, 40.59949413287183], [-73.78998038603889, 40.59945949287673], [-73.78999622917712, 40.599437045719036], [-73.79001376410697, 40.5994136634844], [-73.79003227104491, 40.599393111258145], [-73.79005223846883, 40.599373957721404], [-73.7901352995466, 40.599346031132235], [-73.79017301455475, 40.599333420686726], [-73.7902060998847, 40.59932046906359], [-73.79024080346245, 40.59930403392536], [-73.79028210290387, 40.599278777478744], [-73.79030062101957, 40.59926610498575], [-73.79030531748317, 40.599260151845975], [-73.7903277629682, 40.59921186284681], [-73.79034006550985, 40.59919167636005], [-73.79033913622047, 40.59914520730624], [-73.79033746838687, 40.5991159748663], [-73.79033423877563, 40.59908715069454], [-73.79032939368571, 40.599072616868334], [-73.7903245001797, 40.59904683534915], [-73.79032383949522, 40.599027868616645], [-73.79032748499574, 40.59900292172906], [-73.79033669416049, 40.598948591917704], [-73.7903516789119, 40.59889089541175], [-73.79037103389119, 40.59884839008622], [-73.79038145956571, 40.598831641062084], [-73.79038743050691, 40.598811065911356], [-73.7903914329734, 40.598785957671225], [-73.79039668745611, 40.598764174664815], [-73.7904042964466, 40.59873772254561], [-73.79040940067183, 40.59871843311063], [-73.79041555276622, 40.59869706620852], [-73.79043233771182, 40.598649089022764], [-73.79066455579289, 40.59811095337753], [-73.79066404130111, 40.59809405054189], [-73.79067875876665, 40.59805002258146], [-73.79069745099893, 40.59800396804827], [-73.79071634863192, 40.59796039081182], [-73.79073876433837, 40.59789472047222], [-73.79075868097767, 40.59785039572392], [-73.79079111150178, 40.59778570807599], [-73.79080339747308, 40.597752302580325], [-73.79081330022179, 40.59771380604236], [-73.79083507961937, 40.597647809516985], [-73.79085314627787, 40.59758248137466], [-73.79087622688401, 40.597546323235164], [-73.79088855613779, 40.597521175847845], [-73.79090029665446, 40.597495865386705], [-73.7909062908479, 40.597479215804086], [-73.79091651274328, 40.59745963108719], [-73.7909229886426, 40.597436399804856], [-73.79092749195192, 40.597412688479075], [-73.79093675560797, 40.597397305838506], [-73.79095604358122, 40.597391109157925], [-73.79096864748604, 40.5973802895651], [-73.79100150128556, 40.597339205208776], [-73.79103242414756, 40.59729583809075], [-73.791055729084, 40.5972482608309], [-73.79107079436692, 40.597198362372886], [-73.79107981022912, 40.59714864289385], [-73.79107551858681, 40.59708169040688], [-73.79106821763796, 40.59704390857649], [-73.79106882093532, 40.59701335561357], [-73.79108998321672, 40.596970798497146], [-73.79109553348212, 40.596945117242946], [-73.79109463123835, 40.59692384553428], [-73.79107264031752, 40.596895205226495], [-73.79106730800333, 40.596874058816844], [-73.79106696389054, 40.596847979853464], [-73.79107940608634, 40.596828325444775], [-73.79110071587499, 40.596806319625486], [-73.79110296641785, 40.59678974447834], [-73.79108785355278, 40.59674116087698], [-73.79107684455165, 40.596703291722974], [-73.79107312725631, 40.59668296947974], [-73.79107086341469, 40.59663636346874], [-73.791056112662, 40.596517871897476], [-73.79104972413373, 40.59649798573421], [-73.7910361609896, 40.59648305729745], [-73.79102080130005, 40.59646464080734], [-73.79100822160366, 40.59643780061812], [-73.79099965333589, 40.596423457730246], [-73.79099262550773, 40.596392754685276], [-73.79099089428213, 40.5963457896187], [-73.79098674490581, 40.59630153849022], [-73.79097923693209, 40.59623943515399], [-73.79097335401144, 40.59620906690359], [-73.79096682814968, 40.59618778496309], [-73.79096043520504, 40.59616202703847], [-73.79094400255747, 40.59612054561262], [-73.79094119971516, 40.59607218256965], [-73.79094283512603, 40.59605546306748], [-73.79093996379385, 40.59602946779227], [-73.79093525469249, 40.596005082651715], [-73.79093189738232, 40.59598317572389], [-73.79093142251077, 40.59596118340643], [-73.790932475687, 40.59591183761246], [-73.7909327566756, 40.59588239076157], [-73.79094780252512, 40.59581244634658], [-73.79096604102041, 40.5957635720813], [-73.79098882223289, 40.59569842487067], [-73.79101069642412, 40.59563558121417], [-73.79102320111208, 40.59557335059413], [-73.79102536652763, 40.595527157215926], [-73.79101345609467, 40.59547866067372], [-73.79100581941853, 40.59545383697884], [-73.79098572888587, 40.595406973545835], [-73.79097620965142, 40.59537989516071], [-73.79095521567369, 40.595331273362156], [-73.7909435156542, 40.59528719847054], [-73.79093122170288, 40.59524421278011], [-73.79092260879435, 40.59519954022791], [-73.79090664934786, 40.59514634312917], [-73.7909002713192, 40.59512344093681], [-73.79089223495866, 40.59510163371589], [-73.79087767200046, 40.59506296582114], [-73.79086693228183, 40.59504071223796], [-73.7908553733466, 40.595019690683834], [-73.79083826830144, 40.59499687651807], [-73.79082506252028, 40.59498107537174], [-73.79079527272947, 40.59494950193799], [-73.79077176258703, 40.594927396981525], [-73.79074977196845, 40.59491399242646], [-73.7907238134791, 40.594900329849146], [-73.79070327272055, 40.594883983926046], [-73.79068741160566, 40.594863648929596], [-73.79068028754959, 40.59484537253556], [-73.79067461362513, 40.59483380817941], [-73.79066531446506, 40.59480915398734], [-73.7906508004326, 40.59478270208199], [-73.79057472642864, 40.594765134514], [-73.79049786957147, 40.594757003940195], [-73.79042048230902, 40.59475269500828], [-73.79042117097245, 40.59476274268536], [-73.79042269746881, 40.59477130271896], [-73.79043261017847, 40.594825986911104], [-73.79043709310866, 40.59482599508647], [-73.79045684084385, 40.59496013627619], [-73.79057004196036, 40.59495180029974], [-73.79057552409152, 40.594990247969385], [-73.79057713478595, 40.595003012646934], [-73.79058403455441, 40.595057692506316], [-73.79060301364446, 40.595208097389424], [-73.79060878545663, 40.59525384133497], [-73.79061012490762, 40.59526445024794], [-73.79061174677369, 40.59527730105524], [-73.79061362295641, 40.59529217306905], [-73.790669593295, 40.59573571724391], [-73.79067071979917, 40.59574464427231], [-73.79070673101859, 40.59603000671374], [-73.79071439952162, 40.596085541523266], [-73.79080405847378, 40.59608143327048], [-73.79080626141678, 40.59609339588424], [-73.79059442974041, 40.59610838477282], [-73.79059334326142, 40.59609727842324], [-73.79068189277959, 40.59608889858272], [-73.79067422496254, 40.596033363772264], [-73.79067270897326, 40.59602143256255], [-73.79066544464762, 40.59596422687496], [-73.7906638296012, 40.59595150558215], [-73.7906568850775, 40.595896820290086], [-73.79064645458273, 40.59581468544173], [-73.79064137096427, 40.59577464311962], [-73.79063777408551, 40.59574632555772], [-73.79063349603543, 40.59571263793082], [-73.79062909576751, 40.595677977236136], [-73.79062786162403, 40.59566826496695], [-73.7906204146483, 40.595609616846055], [-73.79061775872772, 40.595588704302024], [-73.7906029478242, 40.59547206308198], [-73.79060063981308, 40.59545388609338], [-73.79057680046407, 40.59526615444707], [-73.79057536323519, 40.59525483938538], [-73.79056985972822, 40.59521149813542], [-73.79056812031568, 40.595197798910334], [-73.79056783356431, 40.59519553891475], [-73.7905577786379, 40.595116355437426], [-73.79055239375442, 40.59507394988801], [-73.79055065349051, 40.595060244127374], [-73.79053857666516, 40.59498078405602], [-73.79042537549705, 40.594989119498905], [-73.79040674421687, 40.59485583557948], [-73.79031932474746, 40.59486165540376], [-73.79031712329405, 40.59484883921797], [-73.79040454942977, 40.594841310437], [-73.790402351078, 40.594827641197135], [-73.79041243624242, 40.59482765959148], [-73.7904047210201, 40.59478664463586], [-73.7903834231324, 40.5947891694952], [-73.79038123820116, 40.594771227099315], [-73.79038036102303, 40.594765085368174], [-73.79037896334765, 40.59475196591303], [-73.79029571148867, 40.59475382143422], [-73.79021283170947, 40.59476009535141], [-73.7901526364882, 40.5947682022728], [-73.79016082074588, 40.594817827453475], [-73.79014961159086, 40.59481957844358], [-73.79015777357735, 40.594869158518165], [-73.79016562708783, 40.59486912946874], [-73.79016964649367, 40.59489816920264], [-73.79017141401827, 40.59491093685303], [-73.7901809999016, 40.59498018667439], [-73.7901827807702, 40.59499304548499], [-73.79019224695952, 40.59506143699918], [-73.7901994322676, 40.595113343949095], [-73.7902038447646, 40.59514521866608], [-73.79020562561757, 40.595158085182426], [-73.79021521095275, 40.59522732226598], [-73.79021697825911, 40.59524009494063], [-73.79022055426886, 40.59526589551324], [-73.79022458719228, 40.59529890253937], [-73.79022732688355, 40.59532129972688], [-73.79023589224633, 40.59539131831443], [-73.7902374656807, 40.595404172891776], [-73.79024612515725, 40.595474952067036], [-73.79024831933687, 40.59549288459687], [-73.79025595436414, 40.59555529731096], [-73.79025761755744, 40.595568895380936], [-73.79026609106864, 40.59563815957806], [-73.79026766387427, 40.59565100812256], [-73.79027612537864, 40.59572018333823], [-73.79027768616686, 40.595732940724226], [-73.79028604655542, 40.59580128849247], [-73.79028761938284, 40.59581413268065], [-73.79029628728671, 40.59588499144111], [-73.79029775614858, 40.59589700047142], [-73.79030621838635, 40.59596617183124], [-73.7903077798403, 40.595978930893175], [-73.79031625476225, 40.59604819893949], [-73.7904238115932, 40.59604197775186], [-73.79042481472277, 40.59605393215388], [-73.79025042399027, 40.59606657324087], [-73.79025001049114, 40.59606660381418], [-73.79017830001315, 40.59607131970442], [-73.79017616944475, 40.59606021094181], [-73.7902505477276, 40.596053769005664], [-73.79028151177364, 40.59605141610619], [-73.79027434427995, 40.59598223940168], [-73.79018744892942, 40.59531159480487], [-73.79018263760119, 40.59527445373374], [-73.79018161945703, 40.595266595737236], [-73.79015546104264, 40.59506471355596], [-73.79014793843845, 40.59500664659013], [-73.7901343095824, 40.594901463811155], [-73.7901299201233, 40.59487241401846], [-73.7901388858189, 40.59487157565576], [-73.79013119491599, 40.594822874798425], [-73.79011998666475, 40.59482370856218], [-73.79011344054702, 40.59477348142072], [-73.79006564361859, 40.59477991738313], [-73.78994705523382, 40.594807310743484], [-73.78990643704788, 40.59480862313334], [-73.78990069186753, 40.594802426427655], [-73.7898665049066, 40.59471554399451], [-73.7898580733858, 40.5947171864501], [-73.78986144770661, 40.59472643272148], [-73.78986855134775, 40.594745883345425], [-73.789895889111, 40.59482073601437], [-73.78988243242814, 40.59482412801082], [-73.7898546191521, 40.59474499492269], [-73.78984868911779, 40.59472812489389], [-73.78984569560428, 40.59471959684828], [-73.7897851316379, 40.59473138972904], [-73.78978687941984, 40.59473616232949], [-73.78978971292146, 40.594743855786], [-73.7898230113407, 40.594834270153264], [-73.78981067236668, 40.59483851791692], [-73.78977984173702, 40.59474728733255], [-73.78977687995943, 40.59473852429906], [-73.78977512688962, 40.59473333722033], [-73.78975503447937, 40.59473725030926], [-73.78973901889819, 40.59474366855456], [-73.78972206998854, 40.594764934420226], [-73.78970621834631, 40.594777989176116], [-73.7896931691552, 40.59479273670557], [-73.78968322877788, 40.59480883077525], [-73.78967663059062, 40.594825893538285], [-73.78967352950468, 40.59484352440503], [-73.78968737611616, 40.59484256383519], [-73.789712057399, 40.594840900032956], [-73.78971317198065, 40.594842611043035], [-73.78972631819998, 40.594938300925335], [-73.78971619942226, 40.594948532043944], [-73.78972052634776, 40.594998081509914], [-73.78973173375076, 40.59499724895631], [-73.78973825297507, 40.595052721927196], [-73.78973975187641, 40.59506548306161], [-73.78974959344472, 40.59514922805597], [-73.78975402689434, 40.59518694492447], [-73.78976094448967, 40.59524580623983], [-73.78976253995151, 40.59525938274981], [-73.78976484966432, 40.59527903871139], [-73.78977218658352, 40.595341469007586], [-73.78977369748347, 40.595354332356266], [-73.7897834381252, 40.59543721253982], [-73.78978611238573, 40.595459963608434], [-73.78979476954494, 40.59553363303614], [-73.78979619274124, 40.59554573530402], [-73.78980603231723, 40.59562946001463], [-73.78980753381967, 40.595642240920654], [-73.78981216446039, 40.59568163796591], [-73.78981867672636, 40.595737059818966], [-73.78982861675051, 40.595821637268536], [-73.78986386560985, 40.59612156629507], [-73.78987360648384, 40.59620444378454], [-73.78999689158506, 40.5961969820469], [-73.78999909439095, 40.59620894350328], [-73.78969872360817, 40.59622803991133], [-73.78969763458414, 40.59621778828498], [-73.78984110232868, 40.59620694687976], [-73.78983128043242, 40.59612322003144], [-73.78982984108444, 40.59611128576754], [-73.78982485495767, 40.59606995411195], [-73.78981719085897, 40.5960064131736], [-73.78980819728906, 40.59593185656867], [-73.78980278325271, 40.595886973504456], [-73.7897966602605, 40.595836213541794], [-73.78979511829786, 40.595823438593], [-73.78978501167339, 40.59573964537727], [-73.78978367321078, 40.595728551625896], [-73.78977347472866, 40.59564399681793], [-73.78977203516642, 40.59563206741061], [-73.78976192792679, 40.59554827854598], [-73.7897604890394, 40.59553634545402], [-73.78975038247809, 40.59545255977184], [-73.78974875210814, 40.595439040828296], [-73.78973884562713, 40.59535691288157], [-73.78973729471367, 40.59534405616042], [-73.78972957937049, 40.59528009126765], [-73.78972709480868, 40.59525950184503], [-73.78972565680681, 40.595247570764336], [-73.7897170605364, 40.595176306496874], [-73.7897141101677, 40.59515184427499], [-73.7896958667097, 40.59500060009031], [-73.78970595371298, 40.59499976381352], [-73.78969826580318, 40.5949502081961], [-73.78968593487112, 40.59495189342681], [-73.78967500189778, 40.59486560299156], [-73.78967436006263, 40.594862632520375], [-73.78967389675836, 40.594860475900155], [-73.78946035244326, 40.59499746624783], [-73.78942472254533, 40.59495613940296], [-73.7894137054954, 40.59494678965259], [-73.78940242709974, 40.59494978552369], [-73.78939514308192, 40.594962893435806], [-73.78940128760799, 40.59497764597044], [-73.78941219682909, 40.59500223251819], [-73.78942455099664, 40.595021687932665], [-73.78942487963515, 40.595044940819704], [-73.78938382544042, 40.59511301633009], [-73.78937810584742, 40.59513606061433], [-73.78937936325215, 40.59516049444521], [-73.78937617071956, 40.59517654346756], [-73.7893531151334, 40.59519376497541], [-73.78933124413726, 40.59519872262826], [-73.7893042029443, 40.59518744353603], [-73.78927385022848, 40.59518478580918], [-73.78924406645662, 40.59518527951386], [-73.78921094671546, 40.59518354447518], [-73.78918073556748, 40.59518477635302], [-73.78917366224985, 40.59519113084521], [-73.78918045239367, 40.59519580732923], [-73.78918865986205, 40.595207935118445], [-73.78918922141126, 40.59523222276166], [-73.78918593810931, 40.59528055659536], [-73.78918207200937, 40.595296435339016], [-73.78914183196389, 40.59533825331586], [-73.78912861078479, 40.595384012408864], [-73.78912042464385, 40.59542483626864], [-73.78911266330519, 40.59547345891198], [-73.78911118815091, 40.59552140906726], [-73.7891150378258, 40.595544560359336], [-73.78912313581736, 40.59556902433601], [-73.78914168951475, 40.59558950940148], [-73.78916410295061, 40.59560693005603], [-73.7891853582447, 40.59562502439616], [-73.78920272046062, 40.59564498574594], [-73.78921284672033, 40.59566690514239], [-73.78921553350067, 40.595691340422974], [-73.78920999076962, 40.59571471723351], [-73.78918997886312, 40.5957370226188], [-73.78916661360014, 40.59575855606989], [-73.78915195151714, 40.59579261475969], [-73.78914988006115, 40.595808639529736], [-73.78913005053164, 40.59582954988415], [-73.78911655249819, 40.59585027249395], [-73.78910667303099, 40.5958774873199], [-73.78911174911906, 40.595901323713036], [-73.78912428120124, 40.59591325204032], [-73.78913823241385, 40.5959285146331], [-73.78912087264571, 40.595952716115], [-73.78910870574009, 40.59597449526334], [-73.78910696253806, 40.5959917278535], [-73.78911779697158, 40.59601718762182], [-73.78913143742209, 40.596029810717305], [-73.78914969171474, 40.59605188944355], [-73.78916261774464, 40.596077298765444], [-73.78916999432185, 40.59610196010387], [-73.78917203908622, 40.59612425166959], [-73.78917376347391, 40.59615063137689], [-73.78917605194492, 40.5961780836387], [-73.78918028632464, 40.59619502966316], [-73.78919140114502, 40.59621815792506], [-73.78920034771825, 40.596239588818584], [-73.7892122341178, 40.59628390681026], [-73.78920609254766, 40.59630961604179], [-73.78920558571939, 40.59633517143004], [-73.78921176727629, 40.59636056837389], [-73.78922841127229, 40.59638358062375], [-73.78924554412214, 40.59640532288754], [-73.78926249847342, 40.596430632030874], [-73.78926858096267, 40.59645372374887], [-73.78926786684315, 40.596478045235756], [-73.78926243909658, 40.59650606031712], [-73.78925362474264, 40.596528439520206], [-73.78924244828703, 40.59657752532247], [-73.78923305136935, 40.5966016864738], [-73.78922877308254, 40.596620778685406], [-73.78923712086214, 40.59664470299366], [-73.78923757533869, 40.59666156233858], [-73.78922956489518, 40.59668819424699], [-73.78923191354295, 40.59671100839003], [-73.78922928197579, 40.5967329137645], [-73.7892239473967, 40.59675716361741], [-73.78920081200863, 40.596773510441814], [-73.7891987448845, 40.59678063872335], [-73.78920315252981, 40.59678789699024], [-73.78922257366165, 40.59680697822421], [-73.7892363961816, 40.59682951705122], [-73.789243478524, 40.596846189358494], [-73.78924970828197, 40.59686733632711], [-73.7892548556676, 40.59689097582707], [-73.78925996090993, 40.59691675778654], [-73.78925638990108, 40.59694043507512], [-73.78924787332302, 40.596969767471904], [-73.78923987553891, 40.59699196053738], [-73.78923548177478, 40.59701796782671], [-73.78922272649518, 40.59704311475999], [-73.7892167419102, 40.59706766983139], [-73.78922113376595, 40.5970911459456], [-73.78922065193284, 40.597101194329], [-73.78922017232334, 40.59711123450751], [-73.78921222323429, 40.597140342279616], [-73.78919799854089, 40.59716323592125], [-73.7891863750342, 40.597185339907064], [-73.78917081216848, 40.59720896771843], [-73.78915880590186, 40.59725824900197], [-73.78915978633384, 40.597280944871976], [-73.78915616499374, 40.59729771448121], [-73.78914947665892, 40.59730958944472], [-73.78914279977994, 40.59732145504719], [-73.78912107702604, 40.59733895721868], [-73.78913441572564, 40.59735375043998], [-73.7891339679721, 40.597361214907785], [-73.7891181658512, 40.59738166440244], [-73.78910195861314, 40.59740386869476], [-73.78910066668473, 40.59741251135709], [-73.78909938750327, 40.59742115404281], [-73.7891038727293, 40.597437190842136], [-73.78911067519103, 40.59744917181233], [-73.78911747897803, 40.59746115161184], [-73.7891318136952, 40.59748208896854], [-73.78914765407926, 40.59750882712001], [-73.78915664160107, 40.597524927127566], [-73.78918554149362, 40.5975987330701], [-73.78919199856605, 40.597648976506015], [-73.78919207479233, 40.597718532782544], [-73.78919080249129, 40.597788276563584], [-73.7891815464319, 40.5978353565264], [-73.78916558855904, 40.59788275807967], [-73.78913926451678, 40.59792849262147], [-73.7890682912148, 40.598019927382616], [-73.78897703662628, 40.59809167597658], [-73.78889576026538, 40.59816135406835], [-73.7888599973617, 40.59820677317101], [-73.7888443043442, 40.59822633994588], [-73.78883199925075, 40.598247037905765], [-73.78881408327982, 40.5982639717166], [-73.78879716014761, 40.598276781931496], [-73.78877241067005, 40.59829111752411], [-73.78876415584486, 40.5983051870716], [-73.78874406562063, 40.5983146499108], [-73.78872539542698, 40.59833078085937], [-73.7887047635189, 40.598332363441244], [-73.78867553924202, 40.598326951269264], [-73.78864132611197, 40.598327005286194], [-73.78860652033906, 40.5983348660897], [-73.78858394436504, 40.59834555952459], [-73.78855367211774, 40.59836181233864], [-73.78853627903102, 40.59837399090139], [-73.78852376561038, 40.5983858276339], [-73.7885112521853, 40.598397664365194], [-73.78850339346606, 40.59841329197849], [-73.78849232596227, 40.598427293171284], [-73.78846760804294, 40.59844300668786], [-73.78842878612564, 40.59846589063771], [-73.78841627774574, 40.59847244917879], [-73.7883667765844, 40.598490828844234], [-73.78834045693753, 40.59850137092641], [-73.78828847173249, 40.59852085167889], [-73.78826186810186, 40.598538669702705], [-73.78822073050371, 40.59855368647922], [-73.7882101727132, 40.59856744482851], [-73.78820372858448, 40.59859120624723], [-73.78819122459387, 40.5986152815045], [-73.7881752184711, 40.59863306476014], [-73.78815240499182, 40.598651852034955], [-73.78810693083862, 40.59868212574317], [-73.788092046576, 40.59869634384689], [-73.7880719963165, 40.598718838438586], [-73.7880581860508, 40.598733914930165], [-73.78803815735542, 40.59874960952733], [-73.78801808483836, 40.59876477380874], [-73.78800284383935, 40.59877954074016], [-73.78798201164625, 40.59879555885279], [-73.78796656719273, 40.598804347276136], [-73.78792811711247, 40.59882477173756], [-73.78791566941302, 40.59884574927773], [-73.78789223767818, 40.59886552629757], [-73.78787414051945, 40.59887232743411], [-73.78785750117561, 40.59888494973109], [-73.78783559420927, 40.59891575216709], [-73.7878231440424, 40.598929814403846], [-73.78780551211833, 40.59894658440242], [-73.78778210998513, 40.59896809069548], [-73.78775051756722, 40.59897220166182], [-73.78774204023071, 40.59898890026985], [-73.78772349722136, 40.59900975780416], [-73.78768875002692, 40.59903250461987], [-73.78765582919964, 40.599056649149034], [-73.78765085932115, 40.59908126029], [-73.78764162877339, 40.599097461778555], [-73.78762264350814, 40.599119418646346], [-73.78759278945736, 40.59913398663168], [-73.7875940545273, 40.59914132040214], [-73.78758292335176, 40.59915686065165], [-73.78756221247788, 40.599179028422746], [-73.78754027623143, 40.59919312636744], [-73.78753410819982, 40.59919709546847], [-73.78751457283121, 40.59920293159925], [-73.78750156253538, 40.59921484855332], [-73.78747494755444, 40.59922882910726], [-73.78744993620624, 40.59924741516291], [-73.78743157906484, 40.59925416096674], [-73.78740174482711, 40.59925860880858], [-73.7873804101433, 40.5992656476337], [-73.78735136126865, 40.59928332488849], [-73.78732492543952, 40.599296451505474], [-73.78728350798691, 40.59933238568244], [-73.78724385944058, 40.59936939614651], [-73.78721550733746, 40.59940593246331], [-73.78719087856445, 40.59942691316934], [-73.78716919221348, 40.59943985923696], [-73.78714983363757, 40.5994457752097], [-73.78712482297082, 40.599441964660706], [-73.78710231226577, 40.59944330089239], [-73.78709695541514, 40.59945209900691], [-73.78709826803978, 40.59947048999461], [-73.78709163412174, 40.599483382177745], [-73.78706556464131, 40.599504199359316], [-73.78703790554069, 40.599519185735836], [-73.78700832765766, 40.599532657934624], [-73.78697627601906, 40.599580577675724], [-73.78696895024865, 40.59959544206845], [-73.78697631058081, 40.59960673849804], [-73.78698368124559, 40.59961803494638], [-73.78699255784456, 40.59962797552006], [-73.78700143510532, 40.599637916094224], [-73.78702449400724, 40.59965732894118], [-73.7870231292598, 40.59966658409975], [-73.78700008160382, 40.59968064302487], [-73.78694898907159, 40.59968996800667], [-73.78693257025918, 40.59971126191214], [-73.78692146983113, 40.59973575359784], [-73.78690879051676, 40.59975883745109], [-73.78689334053554, 40.599783429293], [-73.78687267364991, 40.59980683204932], [-73.786857947441, 40.59982714702436], [-73.78684530718954, 40.59984899809233], [-73.78681970674815, 40.59987036441243], [-73.78679571731941, 40.59988341436123], [-73.78675556871694, 40.59991783874602], [-73.78669090075941, 40.59996095327426], [-73.78666800277085, 40.599979983349826], [-73.78664367471623, 40.59999890421692], [-73.78662409463627, 40.60001863317834], [-73.78660540137093, 40.60003039506851], [-73.78658480977553, 40.600040983068446], [-73.78656003666514, 40.60005156329871], [-73.78653086817748, 40.60006234292852], [-73.78650050573333, 40.600073272282394], [-73.78647084246816, 40.60008336695432], [-73.78646687478977, 40.6000975243803], [-73.78644433473032, 40.60010796499426], [-73.78642400877713, 40.60011320490792], [-73.78640244096495, 40.60011151582979], [-73.7863951501854, 40.6001155908396], [-73.78633700069804, 40.6001372485292], [-73.78629157279269, 40.6001634861503], [-73.78623873377425, 40.600190863410134], [-73.78620893972155, 40.60020463957006], [-73.78616546425769, 40.600233789255185], [-73.78612739135649, 40.60024684904553], [-73.78607800558748, 40.60026187600753], [-73.78605715435523, 40.60027257215632], [-73.78602374866986, 40.60028173196699], [-73.78599381598204, 40.6002949847854], [-73.78597109766216, 40.60031649181468], [-73.78595291643495, 40.60033808870927], [-73.7859251043756, 40.60036803375622], [-73.78588400526695, 40.60042214129993], [-73.78585126197785, 40.600460344561036], [-73.78582371060969, 40.60050464317005], [-73.78579378813265, 40.60056892899841], [-73.7860650818483, 40.60066726577754], [-73.7860887345344, 40.600629725706334], [-73.78614358698643, 40.60064861875177], [-73.78610528242092, 40.600711757064076], [-73.78578006551935, 40.60059326189304], [-73.78577809498513, 40.60059664583663], [-73.78575323544425, 40.6006414545477], [-73.78573810856423, 40.60066461562998], [-73.78572362900528, 40.600685093422435], [-73.78570450753243, 40.6007090210393], [-73.78568891607705, 40.6007296855599], [-73.78566778500591, 40.600761281107125], [-73.78564205273258, 40.60079719452088], [-73.78598359117792, 40.60091823794192], [-73.78594303730421, 40.600983932868175], [-73.78588596247505, 40.6009599105216], [-73.7859185896004, 40.600919824672765], [-73.78563276373285, 40.60081638362843], [-73.78561954620693, 40.60084602160113], [-73.78560190733319, 40.600867979472376], [-73.78558774342773, 40.60089287039912], [-73.78558236941569, 40.60091813841519], [-73.78556742279173, 40.60098826992372], [-73.78556455167376, 40.60101774760224], [-73.78556344393961, 40.601046480886154], [-73.7855645941961, 40.601078281662204], [-73.785565682718, 40.601092349145134], [-73.78557493300859, 40.601136834461705], [-73.78557852749977, 40.60116175851862], [-73.7855819807166, 40.60118652969231], [-73.78557921545466, 40.60121617007184], [-73.78557197568765, 40.60124112769142], [-73.78556135556887, 40.601262766607505], [-73.78554956347622, 40.60128851669036], [-73.78554609407448, 40.601317029788326], [-73.78554281915132, 40.60133604297962], [-73.78553432310275, 40.601358369880046], [-73.78551703221468, 40.60138224593091], [-73.78549716081618, 40.60140389337458], [-73.78546138101116, 40.60144250364772], [-73.7854472113968, 40.60145817304529], [-73.78541561873325, 40.60149511682999], [-73.7853935808703, 40.601518021708465], [-73.78536819795482, 40.601537767595985], [-73.78531548692725, 40.601569178447484], [-73.78528368751248, 40.60158193287872], [-73.78525301558588, 40.6015896477717], [-73.7852205056038, 40.60159910488], [-73.78519419789826, 40.60160512614742], [-73.78517278165994, 40.60161145240967], [-73.7851434486721, 40.60161801750758], [-73.78508307760494, 40.6016356887331], [-73.78505429032063, 40.601637552942606], [-73.7850259143311, 40.6016366007363], [-73.78499779144987, 40.60163772669872], [-73.78496309533641, 40.60164369486099], [-73.78494233197902, 40.6016489606668], [-73.78491612549156, 40.601667624012265], [-73.78488951641785, 40.60167929893805], [-73.78487942718564, 40.6016793350031], [-73.78487058620505, 40.60167295098844], [-73.78485964586187, 40.601672732321354], [-73.78481371970436, 40.60169927483625], [-73.78479582997821, 40.601707121093725], [-73.78476801116503, 40.60170193622614], [-73.78473444345646, 40.601713760587565], [-73.7846953711013, 40.60172505143188], [-73.78464772967796, 40.60173703820713], [-73.78462516662738, 40.60174358017457], [-73.78459545855709, 40.60175252854367], [-73.78456651170444, 40.60176370379556], [-73.78453737153366, 40.601776274199324], [-73.78451003112923, 40.60178739046276], [-73.78448233170379, 40.601800098582906], [-73.78442865140248, 40.601831101453676], [-73.78438419864632, 40.60185854736744], [-73.78433250516588, 40.601885348927574], [-73.78428141234191, 40.60191643900972], [-73.78422268038925, 40.601949368441986], [-73.7841695299335, 40.601981003605616], [-73.7841009961769, 40.60202712595284], [-73.78403910487113, 40.6020690276031], [-73.7839886112991, 40.60210442251684], [-73.78394152998196, 40.60214078728609], [-73.78389262255426, 40.60217544616631], [-73.78385119349083, 40.602202708753694], [-73.78382888415051, 40.602218119556994], [-73.78380343380033, 40.60223681204847], [-73.78376189431727, 40.60227656372816], [-73.78373615075938, 40.6022979565654], [-73.78364491488625, 40.60239324975153], [-73.78359527397528, 40.602453644697775], [-73.7835567842201, 40.60249636480659], [-73.78352073287924, 40.60254098123988], [-73.78350560905308, 40.60256343874268], [-73.78349609995502, 40.60258469139354], [-73.78346945172416, 40.60265182910564], [-73.783470791035, 40.60268345837027], [-73.78346832091061, 40.60271646710019], [-73.78346427862317, 40.60273515377761], [-73.78345786484006, 40.60276009589777], [-73.78344576340848, 40.60280464870275], [-73.78342932788466, 40.602852264826986], [-73.78340637324834, 40.602902866437525], [-73.78339598604265, 40.60292856684337], [-73.78337317693969, 40.60297792548421], [-73.78337125913842, 40.60303734729875], [-73.78336431251098, 40.60305909998535], [-73.78335762780752, 40.603076404919356], [-73.78334197405361, 40.60312704892585], [-73.78332456130755, 40.60316634970285], [-73.78328989623121, 40.60322815067817], [-73.78324859465145, 40.60329257721532], [-73.78322836742842, 40.60333601684888], [-73.78320987069085, 40.60338527586794], [-73.78317278644302, 40.60347158433601], [-73.78315109265385, 40.60354112698989], [-73.78313663833141, 40.603589602046114], [-73.78313211669624, 40.60361750109593], [-73.78313333402781, 40.6036431964009], [-73.78313980170076, 40.60367054792616], [-73.7831363378543, 40.6037149281873], [-73.78312802405742, 40.60376099285311], [-73.78313432465106, 40.60381474689355], [-73.7831474349527, 40.60383772586492], [-73.7831644438236, 40.60385762378231], [-73.7831847587032, 40.60387781558215], [-73.783204316904, 40.60389851976425], [-73.7832266271072, 40.603915691076395], [-73.78325548762324, 40.60393146715378], [-73.78327556725573, 40.603937133511245], [-73.78329312325309, 40.60393035955327], [-73.78332575086432, 40.60393209536047], [-73.78335044081527, 40.60394754139181], [-73.78337341425866, 40.6039609755029], [-73.78339119484156, 40.603972238523156], [-73.78345546639713, 40.60402033005832], [-73.78346451772407, 40.604027244807426], [-73.78348138271578, 40.60404783581104], [-73.78348417724956, 40.60406425763439], [-73.78348890697673, 40.60407029961985], [-73.78349844663508, 40.60409438958107], [-73.78350383731244, 40.60411893879855], [-73.78351427600708, 40.6041424797821], [-73.7835298312738, 40.604166309841176], [-73.78355968611207, 40.604199793412896], [-73.78357890384383, 40.6042160843274], [-73.78359541366827, 40.60423316288997], [-73.78360749285513, 40.60425011499985], [-73.78361728511553, 40.60427244871115], [-73.78362855440238, 40.60429527974954], [-73.78363588044574, 40.60431341203277], [-73.78363792382835, 40.60432790517752], [-73.7836050067433, 40.60436089394235], [-73.78360617356066, 40.60436893672121], [-73.78362666254831, 40.60439036396483], [-73.78362721014985, 40.60440363719123], [-73.7836277707412, 40.60441690390853], [-73.78363675613298, 40.60442241428663], [-73.78363998576066, 40.604435702446274], [-73.78365127613456, 40.60445183351137], [-73.78368085978576, 40.60445971517304], [-73.78370918509917, 40.60447294300802], [-73.78374492984211, 40.604511796550256], [-73.78376440568522, 40.604536254817845], [-73.78377702304792, 40.604554974018534], [-73.78380616108427, 40.6045870601519], [-73.78387077837405, 40.60467501877713], [-73.78388089200148, 40.604685988365674], [-73.78389101793894, 40.604696958646514], [-73.78389001884796, 40.60471348104693], [-73.78388690725316, 40.604729305567595], [-73.78390153621177, 40.604750990626904], [-73.78392399606521, 40.604776806608314], [-73.78393645837984, 40.604799451052095], [-73.78394764971192, 40.60482442511683], [-73.78395674485067, 40.60486526297045], [-73.78394980798916, 40.604880397954055], [-73.78393049962523, 40.604902891534806], [-73.78393244742743, 40.604925345498565], [-73.7839173477141, 40.60493961760182], [-73.7838997853996, 40.60495962162028], [-73.78387413209273, 40.604971298107024], [-73.78385371955581, 40.604980696325406], [-73.78382786824373, 40.6050135354341], [-73.78380917955153, 40.605045090158704], [-73.783817659506, 40.60504991504556], [-73.78382062196515, 40.6050583674584], [-73.78379507186638, 40.605074403408324], [-73.78378379511754, 40.6050868721936], [-73.78376301359896, 40.6051117053729], [-73.78373194492409, 40.60515394494663], [-73.78371775172984, 40.60517672805842], [-73.78370892902625, 40.60519743251664], [-73.78369369911347, 40.60521935960868], [-73.78368064738753, 40.60523643512147], [-73.78367444264207, 40.6052478962025], [-73.78366824932449, 40.60525935730473], [-73.78366474741357, 40.605286194979705], [-73.78366993164084, 40.60530877297986], [-73.78366696292244, 40.60532443693426], [-73.78362945445033, 40.60536284518687], [-73.78360001082207, 40.60538074573848], [-73.78357072106958, 40.60539518038593], [-73.7835581114659, 40.60540712394665], [-73.7835540305273, 40.605433961030286], [-73.78351610591254, 40.605476682180374], [-73.78349084244981, 40.60549576261445], [-73.7834718473759, 40.605498005567284], [-73.78344368371363, 40.605510990827774], [-73.78340119872027, 40.60550624624426], [-73.78338424455795, 40.605517029497385], [-73.78337962432077, 40.605531816832475], [-73.78339077689193, 40.60554648965174], [-73.78340456530181, 40.60555774514357], [-73.78340388940889, 40.60557670938839], [-73.78338792822798, 40.60559084359666], [-73.78335880755237, 40.605596687257936], [-73.78333790302254, 40.605590443562825], [-73.78331677485, 40.60558727846798], [-73.78330237302306, 40.60559787754757], [-73.78329484997334, 40.605607850637675], [-73.78328732604626, 40.60561782255298], [-73.78326977128401, 40.60563498066413], [-73.78325205219865, 40.605655695282614], [-73.78323984389571, 40.60567825758837], [-73.7832328529063, 40.60570267590857], [-73.78325148966285, 40.605719622327435], [-73.78325298186276, 40.60573678016784], [-73.78324158710831, 40.60575276143667], [-73.78319733341529, 40.60578361079231], [-73.78319201769607, 40.605812256025004], [-73.7831765582718, 40.605835585674804], [-73.78315553330835, 40.60585530477965], [-73.78311089073352, 40.60588525256186], [-73.78307003702224, 40.60592412256307], [-73.78303641072941, 40.605952821904445], [-73.78298147504987, 40.60601269326453], [-73.78293629446303, 40.60605202329308], [-73.78290564931923, 40.60606160188001], [-73.7828931300397, 40.60608185734915], [-73.78288504089629, 40.60608769622132], [-73.78284369490656, 40.6061034127956], [-73.78282806847018, 40.606108966192174], [-73.78280460652957, 40.6061258343007], [-73.78279970949166, 40.60615639717901], [-73.78279335180751, 40.6061751344013], [-73.7827745192134, 40.60618560810544], [-73.78274680234489, 40.60619173320182], [-73.78273951892068, 40.606215430226285], [-73.78273827896189, 40.60623319489354], [-73.7827129263491, 40.60625364353648], [-73.78268130761911, 40.60625407947574], [-73.78265122934431, 40.60627821034834], [-73.78265821587696, 40.606298863367904], [-73.78263012680134, 40.606310661790616], [-73.78260173572208, 40.606324746580995], [-73.78260283324343, 40.606332104711875], [-73.78259735072109, 40.60635416753223], [-73.7825863267078, 40.60636874352882], [-73.78255966900377, 40.60637986832512], [-73.78253617144011, 40.6063860281578], [-73.78250922760172, 40.60638721069508], [-73.78248626610716, 40.60639594847147], [-73.7824627541849, 40.60641379679842], [-73.78237872847606, 40.606483987074036], [-73.78233124579783, 40.60651892741941], [-73.78228947950905, 40.606550772072936], [-73.78225374776285, 40.60658388025983], [-73.7822083011732, 40.60663940993958], [-73.78212243600952, 40.60674097013422], [-73.7820356078126, 40.60684043917709], [-73.78198561886217, 40.60690498446752], [-73.78195272531626, 40.60694405814234], [-73.78188075607422, 40.607023744217315], [-73.78183306559185, 40.60707472240228], [-73.78180699983872, 40.60710045488062], [-73.78178620290736, 40.60711875904475], [-73.78174774113893, 40.60715900266933], [-73.78173055395993, 40.60717563050875], [-73.7816923823052, 40.60721727839561], [-73.78165603780594, 40.60725338484819], [-73.78163667536029, 40.60727025933283], [-73.78161663663873, 40.607284413524795], [-73.78157907651074, 40.607327297228416], [-73.78155885431538, 40.60734336860472], [-73.78153852900552, 40.6073691838018], [-73.78150652174156, 40.60740495535023], [-73.7814658668753, 40.6074434837317], [-73.78141950854503, 40.60748841193749], [-73.78136750281018, 40.60754362356929], [-73.78130436722184, 40.607599111030616], [-73.78127010687184, 40.607636813191746], [-73.78102182419175, 40.60786652366035], [-73.7809873307122, 40.60790705402877], [-73.7804083226787, 40.608425974827945], [-73.78024564879574, 40.608563353754725], [-73.77940143922393, 40.60922626068232], [-73.77927015444938, 40.60930736180752], [-73.77881487685055, 40.609550268000085], [-73.77869322136964, 40.609602597384516], [-73.77856806939954, 40.609643663429], [-73.77828921242825, 40.60970618063097], [-73.77798083000845, 40.609760562693744], [-73.77768759425464, 40.60980458137117], [-73.77739041698314, 40.60983973857036], [-73.77727898274847, 40.60984911309185], [-73.77713093483726, 40.609853221092216], [-73.77703113969525, 40.60984827188615], [-73.7769279804783, 40.6098352029453], [-73.77680331562756, 40.609808224394556], [-73.77666726353627, 40.60977291227631], [-73.77658082259629, 40.60974596268235], [-73.776474930084, 40.60969521951114], [-73.7763584369479, 40.609655838242304]]]}}, {\"id\": \"117\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 118, \"Shape_Leng\": 0.243966217692, \"Shape_Area\": 0.00182693922711, \"zone\": \"Heartland Village/Todt Hill\", \"LocationID\": 118, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.10834417899996, 40.610234766999945], [-74.10778645199997, 40.61023505699993], [-74.10722963899997, 40.610252444999915], [-74.10667509699992, 40.610286802999916], [-74.10549388999996, 40.610343616999955], [-74.10429401399988, 40.610383201999944], [-74.10337596999989, 40.6104134719999], [-74.10290182399991, 40.610429102999895], [-74.10239596999999, 40.61044802799987], [-74.10181225499998, 40.61046986299992], [-74.101654781, 40.61047575399986], [-74.10138596799997, 40.61048580699988], [-74.10082806899996, 40.61050667099992], [-74.10003616599987, 40.61053628199992], [-74.09946989799994, 40.61055810099989], [-74.09938452999991, 40.61056138999991], [-74.0977696879999, 40.61062359299988], [-74.09730455699996, 40.61062732099988], [-74.09721750699994, 40.61062801799988], [-74.09592704599994, 40.6106383469999], [-74.09555829799993, 40.61062477799989], [-74.09519027099998, 40.61059466299988], [-74.094824891, 40.61054805699994], [-74.09446404899988, 40.6104852189999], [-74.09410959099992, 40.610406551999894], [-74.09305685599998, 40.61011993599989], [-74.09081541099992, 40.60923658599989], [-74.09045495899996, 40.60909452599991], [-74.09055711299992, 40.60900115099989], [-74.09080114099994, 40.60877808899989], [-74.09096036099994, 40.608629682999876], [-74.0914097369999, 40.60822629799987], [-74.09149702999991, 40.60805196799984], [-74.09160388599999, 40.60788297299994], [-74.09172937299998, 40.60772119199991], [-74.09187212199998, 40.60756836799993], [-74.09203043499996, 40.607426039999915], [-74.09239912899989, 40.607072288999916], [-74.09249749399994, 40.606943849999894], [-74.09258024999994, 40.60680678599988], [-74.092645556, 40.60666283899991], [-74.09264557999991, 40.606662785999916], [-74.09264560299997, 40.60666272199991], [-74.09269213899994, 40.60651384699985], [-74.09271908099993, 40.60636222199992], [-74.09271909299996, 40.60636212299986], [-74.092719093, 40.6063620239999], [-74.09272620699986, 40.60620985699994], [-74.092713803, 40.60605936299993], [-74.09271379199996, 40.606059272999886], [-74.09271376799991, 40.606059182999914], [-74.09268272599991, 40.60591244899991], [-74.09262561899992, 40.60545055399992], [-74.09258665899993, 40.60514053599992], [-74.09244145299986, 40.604119585999875], [-74.09260921099994, 40.60323012299987], [-74.09272060299999, 40.60249806099995], [-74.09282191499993, 40.60180956599985], [-74.09285575199992, 40.60172180699987], [-74.09288063499999, 40.6016323259999], [-74.09289641899991, 40.60154165099987], [-74.09290300799996, 40.601450319999834], [-74.09290036399996, 40.60135887199993], [-74.092888504, 40.601267850999896], [-74.0928674959999, 40.60117779299992], [-74.09283746599989, 40.60108923399986], [-74.09263886099998, 40.60084710199989], [-74.0924525679999, 40.60059897899986], [-74.09227904299993, 40.60034540999989], [-74.09211869899993, 40.600086968999946], [-74.09197184899992, 40.59982426299994], [-74.091798925, 40.599318209999886], [-74.09172051499995, 40.59922085699985], [-74.09171640399988, 40.59913660299988], [-74.09171809699998, 40.5990523009999], [-74.09172558899995, 40.59896818299989], [-74.09173885999999, 40.59888447899991], [-74.09175787199997, 40.59880142199993], [-74.09178257499993, 40.59871923899987], [-74.09192827899989, 40.598504050999914], [-74.09228640999991, 40.59808239999993], [-74.09272305199994, 40.59746426399992], [-74.09336426799996, 40.5970330689999], [-74.09375453799998, 40.59674488999988], [-74.09427668499993, 40.59644927099987], [-74.09490220899987, 40.59629562299991], [-74.09586192699999, 40.59595088899989], [-74.09722072899997, 40.59536515499987], [-74.09798651199996, 40.594929686999876], [-74.0980683909999, 40.59488151999991], [-74.0987378739999, 40.59448771499993], [-74.09908179799994, 40.59428707099988], [-74.09949192299995, 40.5940477809999], [-74.09979588999995, 40.59387136599988], [-74.10041498999992, 40.59334309299988], [-74.10053851599993, 40.5931832049999], [-74.10064720699992, 40.59301685499988], [-74.10064730199993, 40.59301670199992], [-74.1006473839999, 40.59301653999994], [-74.10074036799995, 40.592844988999914], [-74.10074047299992, 40.592844782999926], [-74.10074056699987, 40.592844566999894], [-74.100817394, 40.592668734999904], [-74.10083052299994, 40.592632305999864], [-74.10097579699995, 40.591832233999845], [-74.10090957399993, 40.59102161399986], [-74.10089056999995, 40.59042994899993], [-74.10088251699995, 40.59017921599992], [-74.10149482299994, 40.58934426099992], [-74.10207990099998, 40.58874175099988], [-74.10246256599999, 40.588396335999924], [-74.1025675829999, 40.588301540999936], [-74.10282423899991, 40.58806986599989], [-74.10292541199989, 40.58797853999988], [-74.10317352499999, 40.58775457099988], [-74.10328237999992, 40.58765630999993], [-74.10344974399993, 40.587505230999945], [-74.10357323099993, 40.587393759999884], [-74.10392537999996, 40.58707587399993], [-74.10454864599998, 40.58650064899995], [-74.10525775499997, 40.58592143599986], [-74.10588393199993, 40.58540720699986], [-74.10649465099986, 40.58489534299985], [-74.10729030999991, 40.58422229799991], [-74.10790851599995, 40.58368563799992], [-74.10924835599995, 40.5826784129999], [-74.11028714299995, 40.58219874799994], [-74.11127706499995, 40.58179072899986], [-74.11220998099996, 40.581266237999905], [-74.11306954499995, 40.58076852199988], [-74.11367299799996, 40.580312936999874], [-74.11394959699996, 40.58004141599992], [-74.11422746699996, 40.579733907999916], [-74.11481941499994, 40.579176874999895], [-74.1154228399999, 40.578630866999895], [-74.11551544900001, 40.5785498789999], [-74.11562578099988, 40.57847975299992], [-74.11575021599991, 40.5784236329999], [-74.11588397299995, 40.57838369499988], [-74.11602165599997, 40.57836082599986], [-74.11622655900001, 40.57835373999991], [-74.1164310249999, 40.578333594999904], [-74.1166329409999, 40.57830047599992], [-74.11683019199994, 40.578254717999904], [-74.11702078499995, 40.57819692599987], [-74.1175761019999, 40.57770348399992], [-74.118033584, 40.577170408999955], [-74.11844855599999, 40.57665868099991], [-74.11910954899994, 40.57615725199988], [-74.11978567499995, 40.5758155559999], [-74.12112080199991, 40.57530684499993], [-74.1212571389999, 40.575254893999904], [-74.1215372899999, 40.57515325299986], [-74.12164470299997, 40.57512727699991], [-74.12164481999999, 40.575127248999905], [-74.121644926, 40.57512723099993], [-74.12175709299996, 40.57511182099988], [-74.1217572469999, 40.5751118029999], [-74.12175739999992, 40.575111793999895], [-74.12187169099991, 40.57510757499995], [-74.12187183299987, 40.57510756699992], [-74.12187198599999, 40.57510757499989], [-74.12198554099993, 40.575114630999884], [-74.12198569499998, 40.57511464099983], [-74.12198585999995, 40.575114666999845], [-74.12209602099993, 40.575132586999864], [-74.12264375299996, 40.57548301999988], [-74.12276057999999, 40.575566625999954], [-74.122760698, 40.5755667069999], [-74.12276083999994, 40.5755667879999], [-74.12289120499989, 40.57564049999994], [-74.12289127699994, 40.57564054599992], [-74.12289135999994, 40.57564058099989], [-74.12303304099993, 40.57570264699992], [-74.12303319599995, 40.57570271899991], [-74.12303336099995, 40.57570277299987], [-74.12318352299998, 40.57575177099994], [-74.12318366499987, 40.57575181599987], [-74.1231838069999, 40.57575185199992], [-74.12333936499992, 40.57578694099992], [-74.12333955299995, 40.57578698599991], [-74.12333972999994, 40.57578701199989], [-74.1234975639999, 40.57580793399994], [-74.12408417799992, 40.57583401999993], [-74.12513333899997, 40.575879987999905], [-74.12563664099993, 40.575973443999935], [-74.12665688899989, 40.576284932999926], [-74.12708814699995, 40.57636910599994], [-74.12714220299996, 40.57617076799988], [-74.1271422509999, 40.57617061599995], [-74.12714229799994, 40.57617048099992], [-74.12721289299994, 40.57597456599993], [-74.12721297499998, 40.57597434899988], [-74.12721306899995, 40.57597415099987], [-74.12730002599994, 40.5757815399999], [-74.12730004999996, 40.57578149499989], [-74.12730007399989, 40.57578144999996], [-74.12740288899992, 40.57559339599988], [-74.12740297099995, 40.5755932429999], [-74.12740306499991, 40.57559308999995], [-74.12752099599996, 40.57541088199988], [-74.12752110199995, 40.57541070999993], [-74.1275212319999, 40.57541053899995], [-74.12765358199992, 40.57523494299987], [-74.12784711399996, 40.57516757599994], [-74.12814797599997, 40.575056538999924], [-74.12914464399998, 40.57473005399986], [-74.12930379600002, 40.57467541099988], [-74.1309404869999, 40.574106010999834], [-74.13194427599997, 40.57373938299985], [-74.13227350399993, 40.57362850199991], [-74.13261022999991, 40.573530162999894], [-74.13295322899991, 40.573444774999885], [-74.1333012009999, 40.573372680999924], [-74.13378794099991, 40.573296974999934], [-74.13461855599995, 40.57319771299989], [-74.1357919339999, 40.57305194999985], [-74.1363534319999, 40.57299130999997], [-74.13704258199989, 40.57290845199991], [-74.13813873699996, 40.572779642999905], [-74.13846596999996, 40.572739886999955], [-74.14024008799997, 40.57252737199985], [-74.14110631299992, 40.57242255299991], [-74.141847646, 40.57233261399994], [-74.14418588199999, 40.57203740999988], [-74.1440387979999, 40.5712994959999], [-74.14387453599993, 40.570606593999905], [-74.14402153299991, 40.57033011999988], [-74.14438744999991, 40.56970610099991], [-74.14489975699995, 40.569915169999945], [-74.14559994099997, 40.57021110699987], [-74.14629083699998, 40.570496730999885], [-74.146347254, 40.57051823399995], [-74.14640641799993, 40.5705348979999], [-74.1464676189999, 40.57054652199994], [-74.14653012099988, 40.5705529679999], [-74.146593173, 40.5705541569999], [-74.14665601599995, 40.570550074999886], [-74.14671789599998, 40.57054077199993], [-74.14677806799996, 40.57052635799992], [-74.14684848799997, 40.57051367099988], [-74.14692022299994, 40.57050644499987], [-74.14699254699998, 40.57050475399989], [-74.14706472699994, 40.5705086149999], [-74.14713603399994, 40.57051798899991], [-74.14720574299992, 40.57053278099995], [-74.14789716899996, 40.57088728599995], [-74.14983771799996, 40.56935826299985], [-74.15033299399995, 40.56893102999989], [-74.15055139699999, 40.56874813699989], [-74.1507746989999, 40.56856871799991], [-74.1510028049999, 40.56839284899993], [-74.15123561899996, 40.56822060499991], [-74.15147304199998, 40.56805205899985], [-74.15171497199997, 40.56788728399988], [-74.15228114599995, 40.567392945999856], [-74.15295627699996, 40.56677601699993], [-74.15357985599994, 40.56625318599991], [-74.15416347399992, 40.56583667599993], [-74.15434186999994, 40.565742096999884], [-74.15450973599995, 40.5656339799999], [-74.154664477, 40.5655134429999], [-74.15480382999996, 40.56538193999996], [-74.15492589699993, 40.56524125699989], [-74.15502927899996, 40.565093413999904], [-74.15511310499993, 40.56494054499992], [-74.15585090399995, 40.56377444399988], [-74.15675091099989, 40.56288697499989], [-74.1568588409999, 40.562789093999896], [-74.15685888799997, 40.562789056999904], [-74.15685893499987, 40.562789021999926], [-74.15698080699997, 40.562699864999956], [-74.15698100799993, 40.562699720999944], [-74.15698121999995, 40.56269959399989], [-74.15711531399992, 40.562620740999876], [-74.15711540799994, 40.562620686999885], [-74.15711551399995, 40.562620641999935], [-74.15725961299992, 40.562553382999916], [-74.15725974199995, 40.562553318999896], [-74.15725989499994, 40.56255326399991], [-74.15741136699988, 40.562498565999874], [-74.15825784199991, 40.56220197699982], [-74.15915791699992, 40.56193583399991], [-74.1600564059999, 40.56170549999991], [-74.16096801899997, 40.56146020199987], [-74.1615313009999, 40.56126935399991], [-74.16193293199993, 40.56111113299994], [-74.16272123599997, 40.56077419499996], [-74.16377941599998, 40.560316978999886], [-74.1649414939999, 40.56011179699991], [-74.16589404599996, 40.560283361999915], [-74.16688971999996, 40.560476604999955], [-74.16982582400001, 40.56109042099991], [-74.16976493899996, 40.56288426399993], [-74.16974974799987, 40.56332951299986], [-74.16974968899994, 40.56333125299989], [-74.16974554099995, 40.56345282199995], [-74.16973775199993, 40.56383841899987], [-74.16972693399991, 40.56437399799987], [-74.16972490199993, 40.56447115899986], [-74.16969147299992, 40.56607039999989], [-74.16967801699991, 40.56647024499989], [-74.16966171099988, 40.566903382999904], [-74.16963069799989, 40.567651638999905], [-74.16962643399991, 40.568298409999926], [-74.16964473399992, 40.568945264999904], [-74.16968558799996, 40.56959152099988], [-74.16974893299987, 40.57023648399991], [-74.16983467899996, 40.57087948699992], [-74.16989846899996, 40.57139322399989], [-74.16994243599997, 40.5719086199999], [-74.16996647099997, 40.5724249999999], [-74.16997053699991, 40.57294169899994], [-74.16995465599994, 40.57345803799992], [-74.16991886099999, 40.57397335499995], [-74.16980912399991, 40.57579489399991], [-74.16975526299997, 40.57668893399995], [-74.16973086899992, 40.57696575699988], [-74.16970918699998, 40.5774539449999], [-74.16967337599986, 40.577941821999964], [-74.16962344999997, 40.57842904399988], [-74.16955946599997, 40.57891524199987], [-74.16948145799996, 40.57940007499996], [-74.16913807999991, 40.58180581199991], [-74.16909417599999, 40.58211339699994], [-74.16908500799998, 40.58217764599992], [-74.16907820699998, 40.58222502299985], [-74.16905498199999, 40.58238696299988], [-74.1690483779999, 40.582432934999915], [-74.16903999799996, 40.582491778999945], [-74.16901204299992, 40.58268826899994], [-74.16895369199989, 40.5830983639999], [-74.16842916199994, 40.58649792299988], [-74.16801418199992, 40.5880698669999], [-74.1679917399999, 40.58818579699995], [-74.16796126199993, 40.588300656999905], [-74.16792283599987, 40.58841412299991], [-74.1678765679999, 40.588525874999895], [-74.16782312399997, 40.588630568999854], [-74.16776136599995, 40.58873256899993], [-74.1676915249999, 40.58883149299986], [-74.16761386399995, 40.58892696699992], [-74.16727661599998, 40.58929921399985], [-74.16692477999996, 40.589663906999945], [-74.16655880399995, 40.59002054099989], [-74.16617917199996, 40.590368657999896], [-74.16578641299996, 40.59070780399986], [-74.16369888299994, 40.592258338999905], [-74.16345157200001, 40.59241614799988], [-74.16321702399988, 40.59258595999993], [-74.16299668999994, 40.59276682799989], [-74.16279187799998, 40.59295768699991], [-74.16260370599998, 40.5931573659999], [-74.16244891299996, 40.59337908599984], [-74.16231073399989, 40.593608032999875], [-74.16219000599993, 40.59384310599989], [-74.16208738699994, 40.594083113999865], [-74.16200333499995, 40.59432685099992], [-74.16194448999997, 40.594535495999885], [-74.161908184, 40.59468531299995], [-74.16188841099995, 40.59483745799984], [-74.16188546199997, 40.5949903349999], [-74.16188699199998, 40.59500712699986], [-74.16189940599996, 40.59529877099993], [-74.16180204399988, 40.59530144799985], [-74.16025756099997, 40.59534393799988], [-74.15989227899996, 40.595170140999855], [-74.15891672599996, 40.59462968499985], [-74.1584322029999, 40.59435961999991], [-74.15800268899991, 40.59412020399985], [-74.15795821599993, 40.59408704499986], [-74.15636568699993, 40.5931951279999], [-74.1561030049999, 40.59304747599991], [-74.15537259699995, 40.59259733799994], [-74.15450856599989, 40.59213609399994], [-74.152352083, 40.590936961999894], [-74.152162103, 40.590836192999845], [-74.15096968899994, 40.59017657399992], [-74.1497277719999, 40.5894925039999], [-74.14869312199986, 40.5888423539999], [-74.14858798599987, 40.58876582799994], [-74.14846882599996, 40.588698540999886], [-74.14833787899993, 40.588642607999915], [-74.1481980769999, 40.588599680999934], [-74.14805282699994, 40.588570806999904], [-74.14790576499993, 40.58855634599993], [-74.1477604, 40.58855595599992], [-74.14684720299991, 40.58857108499991], [-74.14645464799997, 40.58855885299993], [-74.14636402099994, 40.58855786899987], [-74.14583447999993, 40.58856291899987], [-74.14387326699992, 40.588615298999926], [-74.14230337999994, 40.588725478999955], [-74.14168277399988, 40.58874118499987], [-74.13864458599996, 40.58906080499989], [-74.13849744199997, 40.58931370499991], [-74.137288794, 40.59078631499988], [-74.13704557999992, 40.591073817999856], [-74.13681568999996, 40.59136787899989], [-74.13659957099995, 40.59166794799989], [-74.13639761399997, 40.591973456999895], [-74.13549571999992, 40.59370046799996], [-74.13544073999995, 40.59381694299984], [-74.13510176799998, 40.594535032999936], [-74.13385485799996, 40.596357845999904], [-74.1328130649999, 40.59724097399986], [-74.13205430799998, 40.59781870599989], [-74.13123935899998, 40.598309748999924], [-74.13114463199999, 40.59840588899992], [-74.13106374499998, 40.598510423999905], [-74.13099847699996, 40.59862154199993], [-74.13095007699992, 40.59873715199991], [-74.13091909399999, 40.598855047999905], [-74.13080319399995, 40.598842085999934], [-74.12982988699997, 40.59867157699988], [-74.12888881099992, 40.59854783299987], [-74.12798105699996, 40.59842483599991], [-74.12714563699993, 40.59833036199988], [-74.12545223499997, 40.59813734999989], [-74.12535321399994, 40.59814001699987], [-74.12525469399999, 40.59814805499983], [-74.12515717599992, 40.5981614239999], [-74.12506115499997, 40.59818005599987], [-74.12496712099988, 40.5982038559999], [-74.12487554999991, 40.59823270299993], [-74.12478690899994, 40.59826644999988], [-74.12470164799997, 40.5983049259999], [-74.12462019999995, 40.59834793499992], [-74.12454297999987, 40.598395257999876], [-74.12399136699995, 40.598714810999894], [-74.12322499699997, 40.5990969529999], [-74.123185791, 40.59919511599987], [-74.12264348699986, 40.5994185849999], [-74.12229425099991, 40.59968717799993], [-74.12070424299995, 40.600705980999884], [-74.12064234399995, 40.60078271599989], [-74.1206422609999, 40.60078282499992], [-74.12064219099997, 40.600782950999914], [-74.12059587499996, 40.60086676699992], [-74.12059578099999, 40.600866936999886], [-74.12059572199999, 40.6008671179999], [-74.12056678099997, 40.60095524599989], [-74.12056671099998, 40.60095547199994], [-74.12056668799994, 40.60095567899986], [-74.12055576799993, 40.60104529199988], [-74.12059546299996, 40.601809757999874], [-74.11957594399998, 40.60181702899996], [-74.11790155199996, 40.601856453999865], [-74.11777623899994, 40.60186024799992], [-74.11684368799999, 40.60189480699987], [-74.11500030999999, 40.60207428599988], [-74.11487058399989, 40.602136659999935], [-74.11511342599988, 40.60253381699985], [-74.11530272499996, 40.603868255999906], [-74.11537754999995, 40.60398418699989], [-74.11544265199991, 40.60469494899995], [-74.11554728399992, 40.60539592799987], [-74.11567482099998, 40.60613313799993], [-74.11579141499993, 40.60686116499993], [-74.1158562429999, 40.60715118399992], [-74.11587716399993, 40.60725747099985], [-74.1160100819999, 40.6073743499999], [-74.11615857899993, 40.60748094799987], [-74.11625356799993, 40.60753644099988], [-74.1163204779999, 40.6075755289999], [-74.116493189, 40.60765675199984], [-74.11706965299993, 40.60820624999994], [-74.11742466500002, 40.609289008999916], [-74.11834442099989, 40.609144125999904], [-74.11839688799994, 40.60913837299989], [-74.11848942399992, 40.60912822499994], [-74.1186902039999, 40.60910620799989], [-74.11883362299994, 40.60909048099991], [-74.11897246899993, 40.60907525399993], [-74.11913216599991, 40.6090577409999], [-74.120254056, 40.6089347039999], [-74.1205232869999, 40.60890517599993], [-74.12066342799993, 40.60888980599989], [-74.12079903099993, 40.60887493299986], [-74.12116879899992, 40.60883437599989], [-74.1212725339999, 40.60936216999991], [-74.1213358189999, 40.609666497999875], [-74.12135179599994, 40.609765267999926], [-74.1204407989999, 40.610018066999906], [-74.11993967899998, 40.610146266999884], [-74.11832650899994, 40.61058077399987], [-74.11738923899999, 40.61083321499991], [-74.11687458199992, 40.610971825999904], [-74.11677996799996, 40.61099204899987], [-74.11651105799997, 40.611050927999905], [-74.115668338, 40.61117538399991], [-74.11508745399988, 40.61120943099993], [-74.11458547799997, 40.6112022959999], [-74.11425629899999, 40.61119614899993], [-74.11392707099994, 40.6111766829999], [-74.11359920399994, 40.61114391299989], [-74.11327407999993, 40.6110979379999], [-74.1129530689999, 40.6110389719999], [-74.11263747399991, 40.61096736599992], [-74.11113638099991, 40.610586059999925], [-74.11082654399989, 40.6105069149999], [-74.11047956499992, 40.61041865599992], [-74.11000934099998, 40.61033620599996], [-74.10945696499992, 40.61028543899993], [-74.10890146199993, 40.61025158499995], [-74.10885268299998, 40.610250108999914], [-74.10834417899996, 40.610234766999945]]]}}, {\"id\": \"118\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 119, \"Shape_Leng\": 0.0835070355744, \"Shape_Area\": 0.000185779447534, \"zone\": \"Highbridge\", \"LocationID\": 119, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9247831879999, 40.84475266499989], [-73.92387743499995, 40.84474050799987], [-73.9235400199999, 40.844758943999885], [-73.92273460999992, 40.844796838999876], [-73.92262808499991, 40.84480091099986], [-73.92246996799993, 40.8448133119999], [-73.92150714599991, 40.844873573999976], [-73.92043297999987, 40.84494079399986], [-73.91930035199984, 40.844993234999876], [-73.91877179199986, 40.845034466999955], [-73.91838402199984, 40.845055012999914], [-73.91729224999992, 40.84510993999991], [-73.91732494799987, 40.845057482999955], [-73.91740919699994, 40.84492513499992], [-73.91746161499991, 40.84480576699992], [-73.9174967979999, 40.8446811909999], [-73.91751358199987, 40.844553629999936], [-73.9175114879999, 40.8444254339999], [-73.91749072699996, 40.844299016999884], [-73.91745219900001, 40.844176629999865], [-73.917444646, 40.8440326819999], [-73.91745769299985, 40.84388636899992], [-73.91749203599986, 40.84374026199993], [-73.91754763699993, 40.84359702599993], [-73.91762366200004, 40.843459320999905], [-73.91771856499989, 40.84332957399993], [-73.91783013899986, 40.84320987999994], [-73.91795571399985, 40.84310189999992], [-73.91794965199988, 40.842229570999976], [-73.91768745899992, 40.84223933299993], [-73.91618344399991, 40.84233489999991], [-73.91689176299988, 40.841347256999946], [-73.91729851599982, 40.84076611799985], [-73.91768361899989, 40.84022549599991], [-73.91813727499989, 40.83960685399992], [-73.91859196099995, 40.83898208999995], [-73.91901668400004, 40.83838109299989], [-73.919162653, 40.838253711999926], [-73.91930863299986, 40.83812632299989], [-73.91938436399987, 40.838076523999845], [-73.91969564899983, 40.83787180699988], [-73.92001799499995, 40.83765612799989], [-73.92036502999983, 40.83742570699993], [-73.921882894, 40.8364257009999], [-73.92237223199994, 40.83610874299993], [-73.92340016399989, 40.83543276399989], [-73.92399049499993, 40.83505229799987], [-73.92414918399993, 40.83482856699992], [-73.92439355099992, 40.83448402799995], [-73.92502907499991, 40.83327258399991], [-73.92537540099983, 40.83262809599993], [-73.92642918799987, 40.83142328399986], [-73.92726316799988, 40.83056486699991], [-73.92742893599986, 40.83034231499989], [-73.92755002799989, 40.83019959099992], [-73.927612063, 40.830126482999916], [-73.92781170699992, 40.829918619999894], [-73.92781180199987, 40.829918521999915], [-73.92781189699987, 40.829918431999936], [-73.92802724899998, 40.82971953899987], [-73.92825730199993, 40.829530513999906], [-73.92850073899989, 40.829352349999965], [-73.92861034599984, 40.8292809099999], [-73.92875620599996, 40.829185834999855], [-73.92889437100001, 40.82910412899992], [-73.9291719679999, 40.828939958999854], [-73.9294498339999, 40.82881040399989], [-73.929737931, 40.828692722999904], [-73.93003491599993, 40.828587589999906], [-73.93033933099994, 40.82849555499994], [-73.93064968200004, 40.82841702999988], [-73.93095939299991, 40.828337088999916], [-73.9311413199999, 40.82830380799988], [-73.93119978199987, 40.82829838699997], [-73.93126199199995, 40.82829117399994], [-73.93146631099984, 40.82828175999986], [-73.93208873599988, 40.828232744999916], [-73.93250790099984, 40.82820039299993], [-73.93294190900001, 40.8281677059999], [-73.93303721899991, 40.828159845999906], [-73.93307251084072, 40.828156152518105], [-73.93308540348333, 40.82822567381785], [-73.93332562839899, 40.830028684268136], [-73.93341936721266, 40.83077435096889], [-73.93347556906217, 40.831221410575075], [-73.9335460674697, 40.83150945350143], [-73.93360592036706, 40.83278953052076], [-73.93357044736123, 40.83289219258061], [-73.9335639574196, 40.83294273602123], [-73.93356390857384, 40.8329916533309], [-73.93353807924747, 40.833058492578786], [-73.93350048262067, 40.833152580516746], [-73.9334623563697, 40.83364382307132], [-73.9331993131559, 40.83477039637639], [-73.93317633641155, 40.83494357382704], [-73.93314306709256, 40.83519412761651], [-73.93299639001587, 40.83567878918864], [-73.9326562035398, 40.83647347652592], [-73.93235564218675, 40.83699333562838], [-73.9318666017282, 40.837640055747805], [-73.93150395434233, 40.83832068951], [-73.93095819766826, 40.839254146912985], [-73.93086414351596, 40.83935336886757], [-73.93081468528484, 40.83942834514667], [-73.93076944989443, 40.83957017766081], [-73.93075007893717, 40.83961582213049], [-73.9307157038901, 40.83963862978712], [-73.9306598019076, 40.83971523229951], [-73.93059094734627, 40.83985868154704], [-73.93055441793419, 40.83988800989454], [-73.93051572319956, 40.839933642183894], [-73.9305070562785, 40.84001027406932], [-73.93043183972385, 40.840231705387474], [-73.93048324521037, 40.84046006056709], [-73.93039907621815, 40.84061025903065], [-73.93036686093104, 40.84062002230456], [-73.92985714678215, 40.84156744895765], [-73.92958939508082, 40.842083549079724], [-73.92957870884518, 40.84227666960251], [-73.92938012588226, 40.842467145624376], [-73.92919408007222, 40.84281841523691], [-73.92918458673105, 40.84320375764766], [-73.92861574822331, 40.844676865029925], [-73.92827454706787, 40.84497576115691], [-73.92811925349231, 40.84531720381841], [-73.92808209386837, 40.84539159691672], [-73.92796538299997, 40.845362455999904], [-73.927784594, 40.8453197489999], [-73.92740093599986, 40.84522543499989], [-73.92706435800001, 40.845139449999884], [-73.92696149299991, 40.84511449499996], [-73.92676979799985, 40.84507093399992], [-73.92668910099992, 40.845051611999914], [-73.92661872999987, 40.84503564699987], [-73.92596341899986, 40.84488974399987], [-73.92526006699997, 40.844793802999966], [-73.92511859399998, 40.8447796449999], [-73.9247831879999, 40.84475266499989]]]}}, {\"id\": \"119\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 120, \"Shape_Leng\": 0.0832612702158, \"Shape_Area\": 9.31462821313e-05, \"zone\": \"Highbridge Park\", \"LocationID\": 120, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92295449499989, 40.85885076199988], [-73.92282845999992, 40.8568514089999], [-73.92277921199988, 40.856826197999915], [-73.92263334899997, 40.856831752999895], [-73.92204492207946, 40.85685415979202], [-73.9220746492405, 40.85677949112044], [-73.92215747201129, 40.85676624723816], [-73.92216865123653, 40.85673844640938], [-73.92214637179917, 40.85672634039559], [-73.92210975280419, 40.856720271081564], [-73.92216714714418, 40.856663482235554], [-73.92221972224736, 40.85664175431943], [-73.92224682591336, 40.85661396435389], [-73.92229299113583, 40.85662729499239], [-73.92233282511857, 40.85660797677498], [-73.92234437219699, 40.85660230078274], [-73.92235458615987, 40.85659530485318], [-73.92236320797554, 40.856587166207184], [-73.92237001880542, 40.85657809205398], [-73.9223748457777, 40.856568311972886], [-73.92237756617774, 40.85655807349529], [-73.92237142612484, 40.85628034182682], [-73.92234465243865, 40.85625312276868], [-73.92237468673567, 40.85621505323653], [-73.92237580882698, 40.85597834922766], [-73.92234459387058, 40.85568233590737], [-73.92229611007241, 40.855663835114505], [-73.92227873064479, 40.85565498884376], [-73.92240035875274, 40.855510055974335], [-73.922669296953, 40.855179774960675], [-73.92281985673772, 40.85498867651118], [-73.92290114139594, 40.85488550530869], [-73.923132985165, 40.85458947031775], [-73.92341584183609, 40.85422580636223], [-73.92360595220674, 40.853977198034684], [-73.92400939220502, 40.85343695391786], [-73.92444065139927, 40.85284047270442], [-73.92467021184008, 40.852506639577925], [-73.92497396914037, 40.85207090936956], [-73.92502191229713, 40.8520217104241], [-73.92503326859799, 40.85199378494464], [-73.92507294866871, 40.85194222070442], [-73.9251098027101, 40.851892817510986], [-73.92514666528864, 40.85183695594672], [-73.92515518101455, 40.85181547591902], [-73.92517217729034, 40.851802592329285], [-73.9252175403104, 40.85173813710844], [-73.92527707938275, 40.8516479117049], [-73.92529694152745, 40.8516070872074], [-73.9253479660683, 40.85153834833878], [-73.92542451604857, 40.851422334244376], [-73.92548973032295, 40.8513267343892], [-73.92553224547352, 40.851273030364204], [-73.9255435930326, 40.85124295131125], [-73.92563999034672, 40.85109901600742], [-73.9256626741845, 40.85106893704496], [-73.92573638681277, 40.85095507157289], [-73.92575544274924, 40.85092631771205], [-73.92585363309088, 40.850778147042426], [-73.92587970419625, 40.850752743447956], [-73.9258946157337, 40.850716021158725], [-73.92602877508632, 40.850514127209394], [-73.92623817249392, 40.850207175479795], [-73.92625607778919, 40.85018092828886], [-73.92649455759486, 40.84983926318877], [-73.9267143909258, 40.849532892486465], [-73.92692303519823, 40.849250532110666], [-73.92711679645474, 40.84896816076986], [-73.92725466369204, 40.84876344005083], [-73.9274036912767, 40.848557312118054], [-73.92752293332553, 40.84837941376697], [-73.92765334858233, 40.848187403075315], [-73.92797377830429, 40.84772713626884], [-73.92813798976522, 40.84749163370049], [-73.92821693262097, 40.847380137446535], [-73.928391510366, 40.84713222929974], [-73.92842200947663, 40.84708268101525], [-73.92845501928952, 40.84703147378574], [-73.92848994827723, 40.84698195788152], [-73.92928544019536, 40.845854221201265], [-73.92932522842061, 40.8457950120814], [-73.92938078797037, 40.845713088024276], [-73.92942845919356, 40.84564416196216], [-73.92961778841168, 40.84537197685638], [-73.92967631216537, 40.84528959996813], [-73.92971850477531, 40.8452317782637], [-73.92976558965158, 40.845155498584724], [-73.9298142904901, 40.84507738364217], [-73.92986949219897, 40.844983883071116], [-73.92992306713143, 40.84489407370508], [-73.92998963816926, 40.844769196937264], [-73.93004648072018, 40.844660929727574], [-73.93009844615624, 40.84455511760717], [-73.93015203635413, 40.84444930594286], [-73.93022352213245, 40.844284441595754], [-73.93027389095322, 40.84416349199061], [-73.93031939529575, 40.844044140613825], [-73.93039902075682, 40.843828808692926], [-73.93045266598449, 40.8436811598807], [-73.9304997882218, 40.84355565022295], [-73.93054530049922, 40.843428546088674], [-73.93056966892286, 40.84337194730109], [-73.93057942780858, 40.8433374999176], [-73.93062818153543, 40.84320584088484], [-73.93067529907346, 40.84308525801471], [-73.93069805828584, 40.84302311939349], [-73.9307273630005, 40.84293970550288], [-73.93077282921934, 40.84281025121564], [-73.93078580763273, 40.84279303321152], [-73.9308020456184, 40.8427659647572], [-73.93081019415833, 40.842717972930494], [-73.93089465201413, 40.8425383401419], [-73.93097488039662, 40.84240043256798], [-73.9312222988684, 40.84193119781963], [-73.93152929047892, 40.84138536615741], [-73.93183459772912, 40.84083694824564], [-73.93216198837912, 40.84025502632082], [-73.9327955073118, 40.83912367646816], [-73.93295967475893, 40.838785698899954], [-73.93309911785128, 40.83854338462381], [-73.93316799815203, 40.838421587995605], [-73.93326662858152, 40.83825017256957], [-73.93335677225114, 40.83808776694384], [-73.93341799707589, 40.83797177345045], [-73.93346646603968, 40.83789121675665], [-73.93353109087465, 40.83776619294514], [-73.93363822412816, 40.837579303046795], [-73.93377509137271, 40.8373711607887], [-73.93390943779711, 40.837132723576715], [-73.93401488559304, 40.83693422612265], [-73.93410248267041, 40.836766027122664], [-73.93419813061857, 40.836572646434874], [-73.93428437111211, 40.836383690973754], [-73.9343123461022, 40.83631834930896], [-73.93436121591056, 40.83620061961779], [-73.93460027799988, 40.83609686799992], [-73.93467393099985, 40.836064902999915], [-73.93474569399996, 40.836092093999916], [-73.93479886299984, 40.8361101159999], [-73.93485740300004, 40.83598502599991], [-73.93496274099988, 40.83575993399989], [-73.93506336299993, 40.83567765899992], [-73.93530787999994, 40.83547772399988], [-73.93572624299988, 40.835156346999966], [-73.93649557099988, 40.83456879499993], [-73.93700641599989, 40.83419310999991], [-73.93705905900006, 40.834165988999885], [-73.93729666499993, 40.833977743999945], [-73.93729719799981, 40.8339773209999], [-73.93795718299988, 40.83345441799985], [-73.93838508899996, 40.8331135029999], [-73.93857474699988, 40.83291508499991], [-73.93862894999997, 40.83285794199987], [-73.93868316299985, 40.832800788999904], [-73.9387736019999, 40.832663211999886], [-73.93890834499989, 40.832480356999945], [-73.93894859799995, 40.832417335999914], [-73.9390289729999, 40.832291496999915], [-73.93913480799989, 40.832097594999915], [-73.93922536499997, 40.831899659999934], [-73.93927114599988, 40.83176179199989], [-73.93927116899984, 40.83176171999991], [-73.9392711939998, 40.83176165699991], [-73.93933313199983, 40.83162629999987], [-73.93941078699996, 40.83149507599991], [-73.93941083399996, 40.8314949949999], [-73.93950338699985, 40.83136945999995], [-73.93960960099984, 40.83125111899991], [-73.93960970799996, 40.83125100199988], [-73.93960982599997, 40.83125089499988], [-73.93972831400002, 40.831140897999916], [-73.93980468099991, 40.83107011299992], [-73.93980484699988, 40.83106995999986], [-73.93980498999994, 40.83106978999994], [-73.93987048999986, 40.830991976999925], [-73.9398705739998, 40.83099188599994], [-73.93987063299984, 40.830991795999864], [-73.93992383199988, 40.8309082759999], [-73.93992389199987, 40.83090817699982], [-73.93992395099987, 40.83090805999994], [-73.93996366299984, 40.8308206059999], [-73.93996368699995, 40.83082053399987], [-73.93998939900004, 40.830730965999855], [-73.94007905799997, 40.8304639569999], [-73.9401708989999, 40.83038439099996], [-73.94034643299983, 40.8304578419999], [-73.94037033200001, 40.83133125399993], [-73.9403683079999, 40.83146049199992], [-73.94034688799985, 40.831590404999865], [-73.94030591799994, 40.83171845099985], [-73.94024600800003, 40.83184212099991], [-73.94016848499993, 40.83195905599989], [-73.94007535599997, 40.83206722799986], [-73.93967601899988, 40.83268537099987], [-73.93962608699985, 40.83279875399988], [-73.93962606299988, 40.83279880799989], [-73.93962603899988, 40.832798852999865], [-73.93955995000002, 40.83290869699989], [-73.93955986799995, 40.83290884099993], [-73.93955977199981, 40.83290896699987], [-73.9394782749999, 40.833013327999886], [-73.93947814499981, 40.833013488999924], [-73.93947800199997, 40.83301364299994], [-73.93938222499987, 40.833110835999946], [-73.93938216599997, 40.83311089899993], [-73.93938209499981, 40.83311096299988], [-73.93927389999998, 40.83319934299993], [-73.93927371099988, 40.83319949599997], [-73.93927348499993, 40.83319964899987], [-73.9391548789999, 40.83327799299992], [-73.93849196699979, 40.83384379999993], [-73.93840360699998, 40.83390946299992], [-73.93832027799995, 40.833978817999906], [-73.93824224799982, 40.83405164199993], [-73.93816976599996, 40.83412770199994], [-73.93810306499991, 40.83420675399989], [-73.93804235799993, 40.83428854499991], [-73.93798783999989, 40.834372813999934], [-73.93793968599998, 40.834459289999856], [-73.93789804799981, 40.83454769599993], [-73.93786306199998, 40.83463774999996], [-73.937751162, 40.834890575999964], [-73.93738281099989, 40.83569072899993], [-73.93702353899988, 40.836468946999894], [-73.93669078700003, 40.83719324899993], [-73.93638316899992, 40.837850124999875], [-73.936230758, 40.838077855999884], [-73.93610200500004, 40.83822716899989], [-73.93595950299988, 40.838369897999904], [-73.93580416699986, 40.83850497699989], [-73.93563711299996, 40.838631419999956], [-73.93545962199988, 40.83874840399992], [-73.93533073299984, 40.83887479599988], [-73.93533061499988, 40.83887491299987], [-73.93533051899982, 40.83887503999995], [-73.93521776599987, 40.83901582199993], [-73.93521772999993, 40.83901586699992], [-73.93521769499985, 40.83901592099991], [-73.93512487199997, 40.83916933099994], [-73.93512477699991, 40.839169484999886], [-73.93512471799994, 40.839169636999905], [-73.93505508799984, 40.8393332359999], [-73.93501119299988, 40.83950397199993], [-73.93499463699986, 40.83967805499986], [-73.93499462399987, 40.83967813599992], [-73.93499462399996, 40.839678216999864], [-73.93500574300005, 40.83985180199987], [-73.93500575299987, 40.83985191099993], [-73.93500577799999, 40.83985200999991], [-73.93504373300003, 40.84002151299995], [-73.93504374400003, 40.84002156699993], [-73.93504376899986, 40.84002162099989], [-73.93510666600004, 40.84018372699994], [-73.93519161599997, 40.840335399999915], [-73.93519169999996, 40.840335543999885], [-73.93519180599998, 40.84033568799992], [-73.93529543199988, 40.840474817999954], [-73.93543637099998, 40.84075133099985], [-73.93559506299997, 40.841173399999924], [-73.93562818399985, 40.84127909999994], [-73.93562826699996, 40.84127936199993], [-73.93562830199987, 40.841279613999916], [-73.9356424249998, 40.8413891299999], [-73.93564244799997, 40.84138928299995], [-73.93564243699991, 40.84138944599988], [-73.935636661, 40.84150013999992], [-73.93563664999986, 40.84150019499987], [-73.93561111099991, 40.84160880699991], [-73.93561108599988, 40.84160889599986], [-73.93561105099988, 40.84160899499991], [-73.93556697199989, 40.841712527999924], [-73.93514017399997, 40.8423116739999], [-73.93467106399996, 40.84295724299987], [-73.93421198500002, 40.843568902999934], [-73.933715538, 40.844246374999926], [-73.93323248400002, 40.8449099219999], [-73.93315044100001, 40.84502194299996], [-73.93277020399985, 40.8455411059999], [-73.93232515199996, 40.84615390099989], [-73.93224030199994, 40.84626460299991], [-73.93218663399995, 40.84634306699996], [-73.93210093699993, 40.84645772999987], [-73.93207258099989, 40.84649567899991], [-73.93204002799989, 40.84653350699988], [-73.93201002999993, 40.84657596599987], [-73.93198004399996, 40.846618414999895], [-73.93191507399995, 40.84671052399989], [-73.93185010299997, 40.846802632999925], [-73.93139355399987, 40.84742823099985], [-73.93116151700002, 40.84774060099989], [-73.93088430099981, 40.84811893799988], [-73.9306210789998, 40.848480976999895], [-73.92970731599995, 40.84812261199988], [-73.92898242699985, 40.8492561219999], [-73.92886675399986, 40.849392312999925], [-73.92873382199987, 40.849520488999886], [-73.92858520000006, 40.849638801999966], [-73.92842287999989, 40.84974567999992], [-73.92824918899996, 40.84983985399993], [-73.92747680599985, 40.85069436999992], [-73.92707982300001, 40.851432342999885], [-73.92703315699981, 40.852200774999886], [-73.92769565699983, 40.8524877819999], [-73.92723934899986, 40.85311537199989], [-73.92679937899987, 40.85371714499991], [-73.92469199999985, 40.8566419969999], [-73.92467094699998, 40.8568108889999], [-73.92466977999999, 40.856982610999886], [-73.92468915299989, 40.85715512199993], [-73.9247291889999, 40.857326286999886], [-73.92478952099987, 40.857493996999885], [-73.92486929999991, 40.85765619099987], [-73.92496717399985, 40.8578110039999], [-73.92508143699999, 40.85795678799993], [-73.92518924999993, 40.85802753699993], [-73.92531748399992, 40.858090806999954], [-73.925465479, 40.858142410999974], [-73.92563075799984, 40.85817795199988], [-73.92580872899991, 40.858193419999964], [-73.92599285799992, 40.8581858659999], [-73.9261752689999, 40.858154095999936], [-73.92634775499995, 40.858098970999926], [-73.92650299499998, 40.858023418999906], [-73.92663553999984, 40.85793192399994], [-73.92674232299986, 40.85782976799987], [-73.92682273599989, 40.85772223799992], [-73.92687811100001, 40.85761400599994], [-73.92691117499997, 40.857508759999874], [-73.92712222099993, 40.858109610999854], [-73.92709951599986, 40.85827552099992], [-73.92705253299991, 40.85844209099991], [-73.92698078699978, 40.85860624099995], [-73.92688470899988, 40.85876482899985], [-73.92676563099991, 40.85891478399985], [-73.9266257519999, 40.85905332599989], [-73.92646803499991, 40.859178123999854], [-73.92629594799986, 40.85928741399988], [-73.92611328799987, 40.85938011099994], [-73.9259203249999, 40.85950140899994], [-73.92573839899995, 40.85963767999988], [-73.9255703789999, 40.859788079999895], [-73.92541898499994, 40.85995133099993], [-73.92528662599987, 40.86012579699997], [-73.92517535599984, 40.86030947899993], [-73.92508669599982, 40.86050012699992], [-73.92502162200003, 40.860695310999894], [-73.92498050399995, 40.8608925189999], [-73.92496310699987, 40.861089282999934], [-73.9249677899999, 40.861253467999894], [-73.92496864099992, 40.86128326999995], [-73.92499818399988, 40.86146373799991], [-73.92474153399992, 40.86159363799992], [-73.924677287, 40.861496644999946], [-73.92461430599997, 40.86140156099989], [-73.92411747399994, 40.86065512499987], [-73.92295846599977, 40.85891378099996], [-73.92295449499989, 40.85885076199988]]]}}, {\"id\": \"120\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 121, \"Shape_Leng\": 0.0969153373445, \"Shape_Area\": 0.000384563286473, \"zone\": \"Hillcrest/Pomonok\", \"LocationID\": 121, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79911195699995, 40.738206139999875], [-73.79822058899992, 40.7381746309999], [-73.79729260599996, 40.73818542899995], [-73.79684795799999, 40.73823554099991], [-73.79640218700004, 40.73828235799987], [-73.79639000899994, 40.73823357999985], [-73.79636108899999, 40.73812415299987], [-73.79607605199982, 40.738144297999945], [-73.7965035009998, 40.73800383599993], [-73.79651029099988, 40.7379086369999], [-73.79584803299987, 40.7380746799999], [-73.79594852799994, 40.73781371899989], [-73.79516450899986, 40.73641140399995], [-73.79458633199991, 40.73505225199988], [-73.79459579399989, 40.7345442459999], [-73.794626052, 40.73359608899993], [-73.79459486499994, 40.73340935799989], [-73.79454395199986, 40.73322422999991], [-73.79447360099982, 40.73304250799991], [-73.79438449799987, 40.73286592899996], [-73.7942776749999, 40.73269614499986], [-73.79397708699996, 40.73229685299988], [-73.79392707899997, 40.73199969499991], [-73.793909425, 40.73171777599989], [-73.79381937199997, 40.731657387999874], [-73.79384983499995, 40.73124678299997], [-73.79378938699996, 40.73070118199996], [-73.79352516299998, 40.73005381599991], [-73.79322750899982, 40.72934789399992], [-73.79283910399985, 40.72842900799989], [-73.79229974199997, 40.72714071899992], [-73.79204513299999, 40.72655107399986], [-73.79191210299997, 40.72620816199992], [-73.79176169599994, 40.7258396519999], [-73.79174472699992, 40.725788849999944], [-73.7917255479998, 40.72572175499991], [-73.79165577199997, 40.72552430799988], [-73.79157252999995, 40.725329611999896], [-73.79147613699985, 40.725138464999894], [-73.79136702000001, 40.724951617999906], [-73.79088077399986, 40.724188858999916], [-73.79045149300003, 40.723554531999845], [-73.79035703899997, 40.72300529999988], [-73.79029346799997, 40.72137614899989], [-73.790302674, 40.7208914649999], [-73.79037661699995, 40.720103723999934], [-73.79050800399992, 40.71980649699987], [-73.79053210899994, 40.71975195999992], [-73.79070834499987, 40.71926865199993], [-73.79073248899982, 40.71920875699987], [-73.791417731, 40.71920231099991], [-73.79155692199994, 40.71920099499993], [-73.79176986799989, 40.719178606999876], [-73.79211141199988, 40.71913886399993], [-73.79249599699979, 40.719067435999904], [-73.79261566399995, 40.71904520999994], [-73.79309042899993, 40.71893343699986], [-73.79315493399996, 40.718917578999864], [-73.79399335099998, 40.71864513999992], [-73.7944373599998, 40.7185008579999], [-73.79480794899995, 40.71837515899988], [-73.79563775099989, 40.71807616099995], [-73.79653978099992, 40.71778674899987], [-73.79741025099996, 40.717532907999946], [-73.79802996199992, 40.71735242599987], [-73.79912568399995, 40.71703434599991], [-73.79959535799996, 40.71689904299997], [-73.80004693799998, 40.716781906999906], [-73.8002167269999, 40.71673785899987], [-73.8009273429999, 40.7166039329999], [-73.80188321, 40.716408810999916], [-73.80308484099997, 40.71622016199989], [-73.80313952600001, 40.71621390699995], [-73.80319837999987, 40.7162025249999], [-73.80394838899987, 40.716102911999926], [-73.80464476399993, 40.716034396999866], [-73.80494148499987, 40.715996230999956], [-73.80579237899985, 40.71594264899989], [-73.80670677799996, 40.71588523999989], [-73.80780927699995, 40.715853895999906], [-73.80782231800004, 40.715911465999916], [-73.80784387599995, 40.715999733999894], [-73.80809856399999, 40.71680213799988], [-73.80823529499995, 40.71723813199987], [-73.80861283999984, 40.71844559999989], [-73.80902609799993, 40.719862791999894], [-73.80904032699983, 40.71991160499987], [-73.80808460099985, 40.72022476199991], [-73.80786105999998, 40.72030243799987], [-73.80722012300005, 40.72052512499991], [-73.80636387699998, 40.72082022699992], [-73.80578893899984, 40.7210187299999], [-73.80550258899997, 40.72111680899993], [-73.80475913799987, 40.72138373999996], [-73.80459931299991, 40.72144112099992], [-73.80452680199981, 40.721462545999884], [-73.80453391999993, 40.72151758799987], [-73.8045963239999, 40.72220220899997], [-73.80466720799983, 40.72290600699992], [-73.80473910599993, 40.72363153599986], [-73.80480892999992, 40.724354387999924], [-73.80487958299983, 40.725049592999895], [-73.80497988299993, 40.72503869699988], [-73.80652142399995, 40.7248730639999], [-73.80665182000001, 40.72555305099993], [-73.80678542899983, 40.72623219899988], [-73.80691166299997, 40.72691266599991], [-73.80704045699993, 40.727596503999905], [-73.80717042299993, 40.72826913299987], [-73.80806424099987, 40.7281700029999], [-73.80895751599999, 40.72807345899992], [-73.80985637299995, 40.7279736039999], [-73.8111179729999, 40.72781641599994], [-73.81101008099996, 40.72743995099989], [-73.81088635099985, 40.72717185599986], [-73.81080551900001, 40.72704547999992], [-73.81080542499983, 40.727045334999914], [-73.8108053549999, 40.7270451729999], [-73.81074444399991, 40.72691102299994], [-73.81074438399986, 40.726910887999985], [-73.81074434999987, 40.72691076199987], [-73.81070503300002, 40.7267713519999], [-73.81070500999995, 40.72677128899985], [-73.81070499699992, 40.72677121699989], [-73.81068816499996, 40.726629718999966], [-73.81068815299979, 40.72662962799992], [-73.81068815299999, 40.72662953899987], [-73.810693702, 40.72648898699987], [-73.81074426599986, 40.72626511299991], [-73.81151613699981, 40.72665799399994], [-73.81246807499996, 40.727133044999924], [-73.81280168299986, 40.72729564399989], [-73.81394369400002, 40.72785224499995], [-73.81443976499985, 40.7280940229999], [-73.81459667899993, 40.72815359199994], [-73.81459693999996, 40.728153690999875], [-73.81459716399995, 40.728153808999885], [-73.8147455559999, 40.72822995299986], [-73.81474579299991, 40.7282300699999], [-73.81474600600002, 40.72823021399993], [-73.81488170099985, 40.728321896999944], [-73.81488183099988, 40.72832198699994], [-73.81488194899997, 40.72832209599993], [-73.81500091699995, 40.728427258999936], [-73.81500104699994, 40.72842737799991], [-73.81500114199996, 40.72842749499995], [-73.81510007599998, 40.728543224999875], [-73.81510024099985, 40.72854342299991], [-73.81510037099989, 40.728543638999895], [-73.81517723199983, 40.728666627999864], [-73.81517732599985, 40.72866678199994], [-73.81517739599994, 40.728666952999916], [-73.81523144899997, 40.72879391199994], [-73.81514620799996, 40.729685964999945], [-73.81504493700004, 40.73039883499994], [-73.81502127700006, 40.730465669999866], [-73.81502002699983, 40.73049387199991], [-73.81497572599984, 40.73115749299992], [-73.81497202699998, 40.731212908999915], [-73.81496454700003, 40.73132495999994], [-73.81496058399995, 40.7313843389999], [-73.8149426019999, 40.73165369899988], [-73.81493851899984, 40.7317148709999], [-73.81492830299987, 40.73186790399991], [-73.81492410799987, 40.731930747999854], [-73.81492083199987, 40.731979832999855], [-73.81491646899985, 40.73203762099993], [-73.81491341999994, 40.73207858899988], [-73.81489836700001, 40.73230923099991], [-73.81489282099993, 40.732394229999976], [-73.81488551400004, 40.7325062089999], [-73.81487971599992, 40.73259505099986], [-73.81487391699997, 40.732683920999904], [-73.81486810699997, 40.7327729559999], [-73.81486255999994, 40.73285795599985], [-73.81485676300004, 40.73294679699994], [-73.81485170599991, 40.733024276999856], [-73.81484566899994, 40.7331167979999], [-73.81483811099991, 40.73323261899989], [-73.81483204899995, 40.73332552499987], [-73.8148269949999, 40.73340297599987], [-73.81481996299976, 40.733510727999935], [-73.81481416399997, 40.73359959799995], [-73.81480147499998, 40.73379915999988], [-73.81477621499985, 40.734246805999874], [-73.81475908299986, 40.73455042099991], [-73.81475217299983, 40.73470101899988], [-73.81474261099987, 40.73490936899987], [-73.81472294099993, 40.73533802999991], [-73.81469163099986, 40.73594999399995], [-73.8146939199999, 40.737140479999866], [-73.81483959999986, 40.737914413999924], [-73.81497208299996, 40.738560715999874], [-73.815038629, 40.7388936679999], [-73.81504998799986, 40.73894313399992], [-73.81499755799993, 40.73893258599988], [-73.81489497099994, 40.73891574299995], [-73.81407382, 40.73878090299996], [-73.813224018, 40.7387022539999], [-73.81227737799998, 40.7386644509999], [-73.81139729099982, 40.738629306999904], [-73.81040435599981, 40.7385896489999], [-73.809411422, 40.73854997199989], [-73.80893016799997, 40.73853237999986], [-73.80844891400001, 40.73851478699987], [-73.80805258499988, 40.738500293999905], [-73.80752082000002, 40.738480847999924], [-73.8065788389999, 40.73844721099994], [-73.80564550999999, 40.7384186529999], [-73.80471291600003, 40.738393948999914], [-73.80466543299995, 40.73839298499992], [-73.80461553500005, 40.738391577999906], [-73.80363600499999, 40.738360851999886], [-73.80268632099998, 40.73833105199989], [-73.80180834299986, 40.738302654999885], [-73.80132155499992, 40.738284201999946], [-73.80089151300002, 40.7382690159999], [-73.80000292499997, 40.73823762399993], [-73.79911195699995, 40.738206139999875]]]}}, {\"id\": \"121\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 122, \"Shape_Leng\": 0.0665318439446, \"Shape_Area\": 0.000226597104976, \"zone\": \"Hollis\", \"LocationID\": 122, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.75725671499985, 40.71813860199992], [-73.75588877999999, 40.7162986439999], [-73.75470078399988, 40.714710052999905], [-73.75407540999991, 40.7139046949999], [-73.7540478399999, 40.713877538999945], [-73.75401024499985, 40.71383490099993], [-73.75370834599993, 40.71332463699988], [-73.75360661900005, 40.71307112199987], [-73.7535751219999, 40.712992296999886], [-73.7531839939999, 40.71205517599985], [-73.75245965599989, 40.71031656399992], [-73.7518426989999, 40.708822920999886], [-73.75139828700003, 40.70774871999987], [-73.75121246999993, 40.707306579999965], [-73.7507743589998, 40.706321836999905], [-73.75047058199985, 40.70563993099994], [-73.75052149099987, 40.70562662299987], [-73.75205066499986, 40.70522802799995], [-73.752937772, 40.70499820599988], [-73.75382395799984, 40.704765834999876], [-73.75471127099998, 40.70453789099988], [-73.75561453899982, 40.70430046399987], [-73.75651887399991, 40.70406425799995], [-73.75740580399986, 40.70383535499992], [-73.75829554399995, 40.7036033809999], [-73.75918228200003, 40.70337203399986], [-73.76006675299989, 40.70314071099992], [-73.76095546600006, 40.702909577999954], [-73.76126359299998, 40.703598939999914], [-73.76262946699983, 40.70324155099991], [-73.76278023599987, 40.70320597499994], [-73.76366846199983, 40.70297297499996], [-73.76455359299997, 40.702742996999866], [-73.76544147799987, 40.702512314999886], [-73.76612489399999, 40.702334485999884], [-73.766369252, 40.70256339699992], [-73.766572825, 40.70275162099996], [-73.76665413499987, 40.70333761699992], [-73.76679155699992, 40.704008526999885], [-73.76701441799999, 40.7052681309999], [-73.76705298500002, 40.705470301999895], [-73.76725075899985, 40.70631255799994], [-73.76734431899995, 40.70663634599993], [-73.7675558879999, 40.707321957999895], [-73.767762698, 40.70799573499992], [-73.76789085799992, 40.70840640299988], [-73.76825969099993, 40.709526367999956], [-73.76844546499984, 40.709637026999886], [-73.76865794899992, 40.70973774499992], [-73.76881217299984, 40.709808123999885], [-73.77084362899988, 40.7101716259999], [-73.77115938700003, 40.71020522199994], [-73.77147819699991, 40.71022223599994], [-73.77165155299986, 40.710222371999905], [-73.77169204899995, 40.710263984999884], [-73.771724447, 40.71031681899991], [-73.77158686799989, 40.71035023399988], [-73.77158671399987, 40.71035026999991], [-73.77158658399989, 40.71035031499988], [-73.77144162599984, 40.71039590799989], [-73.7714414249999, 40.71039596999992], [-73.77144123499994, 40.71039604199991], [-73.77130105399985, 40.71044965099991], [-73.7713009589999, 40.710449686999866], [-73.77130086399985, 40.710449730999855], [-73.77118657799993, 40.710499509999885], [-73.77102494399993, 40.71057651599992], [-73.77102481399993, 40.710576578999884], [-73.77102469499992, 40.710576640999946], [-73.77089074799977, 40.71065160299992], [-73.770765846, 40.71073519299988], [-73.77076577600003, 40.710735237999884], [-73.77076570499999, 40.71073529199989], [-73.7706583899999, 40.71081884299989], [-73.7700680079999, 40.71135386799989], [-73.76962552399995, 40.71178631999993], [-73.76943987099992, 40.711976892999864], [-73.76946291199997, 40.712011247999946], [-73.77039451900005, 40.7134001979999], [-73.77113435099984, 40.714501936999966], [-73.77192122100001, 40.71568504499992], [-73.77110410799999, 40.71597506299987], [-73.77007984799988, 40.716327201999896], [-73.76922535799989, 40.716626587999876], [-73.769139141, 40.71665679899991], [-73.76906210699998, 40.71668376799988], [-73.76817892099992, 40.71699299399994], [-73.76733228200001, 40.717289796999886], [-73.76664095199988, 40.717646613999875], [-73.765714472, 40.71639337899988], [-73.76437291199994, 40.71696662599988], [-73.76335501799994, 40.717411382999934], [-73.76247897300004, 40.717785614999904], [-73.76166150099981, 40.71814054599984], [-73.76084486799982, 40.71849351999986], [-73.7600295979999, 40.71884681499988], [-73.75921639499985, 40.71919899199987], [-73.75878322199996, 40.71938563599986], [-73.75839264999983, 40.71955392299988], [-73.75832728699997, 40.71958208899985], [-73.75725671499985, 40.71813860199992]]]}}, {\"id\": \"122\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 123, \"Shape_Leng\": 0.0834211389387, \"Shape_Area\": 0.000296445173366, \"zone\": \"Homecrest\", \"LocationID\": 123, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96285973999984, 40.61254948599989], [-73.96237947799978, 40.609986373999966], [-73.96137268399995, 40.61009887499992], [-73.9604436189999, 40.61020150699987], [-73.95952069599996, 40.61030205299987], [-73.95859278499987, 40.61040303099988], [-73.95828056999996, 40.60874735399996], [-73.95772401299996, 40.60579983299993], [-73.95753057699993, 40.604823308999904], [-73.957283474, 40.60346964199992], [-73.95683001599991, 40.601087348999954], [-73.95775745999987, 40.600985888999936], [-73.95733246399982, 40.598730913999866], [-73.956927197, 40.596612075999865], [-73.95655540499982, 40.59464772499994], [-73.95607720399984, 40.59210143199992], [-73.95700330399997, 40.592002255999894], [-73.9567157929999, 40.590498733999866], [-73.95665683399993, 40.59010165299992], [-73.95658790599994, 40.58988411499994], [-73.95639200299993, 40.58892867499993], [-73.95622721599997, 40.587995914999944], [-73.95619313699986, 40.58776226899994], [-73.95712566399992, 40.58766096099986], [-73.9581954319999, 40.587544331999865], [-73.95934736799988, 40.58741753699991], [-73.95978030299993, 40.58736947499985], [-73.9603495359999, 40.58730628599994], [-73.96040710199983, 40.58803435199996], [-73.96045968399986, 40.58872381999986], [-73.96051667199988, 40.589452911999885], [-73.96057230499996, 40.590181426999905], [-73.9606241039998, 40.59086268099987], [-73.9606798409999, 40.59159758199995], [-73.96283785899992, 40.59136011199992], [-73.96377366299998, 40.59125722899988], [-73.96483485600002, 40.59113696899989], [-73.96514385199995, 40.5911019159999], [-73.9652882789999, 40.59108660999993], [-73.96542523899983, 40.59107142799992], [-73.9665195109999, 40.590952872999885], [-73.96744527499996, 40.59085120999989], [-73.96787702199993, 40.59080375599996], [-73.96837099999982, 40.59074946599986], [-73.96929959299999, 40.590647588999964], [-73.97022337699987, 40.59054017499992], [-73.97115101799994, 40.59044244399989], [-73.97128748899985, 40.59115950199986], [-73.97141847599984, 40.59183850999992], [-73.97155530499995, 40.59256045799992], [-73.97178220299999, 40.593756083999914], [-73.97189684799982, 40.5943525109999], [-73.97176509099998, 40.59511141099989], [-73.97169554999981, 40.59548854199992], [-73.9715332449999, 40.59636870899992], [-73.97132951099988, 40.59635243099994], [-73.97148350899987, 40.59717158499996], [-73.97163038199993, 40.5979440169999], [-73.97176105299991, 40.59865193199989], [-73.9719108049999, 40.599423058999896], [-73.97291116900001, 40.599315803999964], [-73.9725653369999, 40.601794372999905], [-73.972452617, 40.60261705399988], [-73.97235152099995, 40.60334811899986], [-73.97217742800002, 40.604194711999845], [-73.97239735499981, 40.60565188999986], [-73.97258982399998, 40.60667153299988], [-73.97299433999983, 40.6088141419999], [-73.97191331299986, 40.60893592099993], [-73.9708953839999, 40.60904971899987], [-73.96996969299995, 40.60915048299991], [-73.96904086599989, 40.60925319299989], [-73.96792168899997, 40.60938918999989], [-73.96768635599992, 40.609417786999906], [-73.96739979599994, 40.60945960699995], [-73.96629835499999, 40.609555212999936], [-73.96522378099998, 40.60967440099992], [-73.96429649899991, 40.60977642599992], [-73.96478112799986, 40.61233092199988], [-73.96385332599989, 40.61243251899986], [-73.96285973999984, 40.61254948599989]]]}}, {\"id\": \"123\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 124, \"Shape_Leng\": 0.127724410062, \"Shape_Area\": 0.000754837821479, \"zone\": \"Howard Beach\", \"LocationID\": 124, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.85096453700001, 40.671719510999885], [-73.84847765299996, 40.67091673899989], [-73.84787000899989, 40.67077949999991], [-73.84741378199982, 40.67062999499992], [-73.84593839099993, 40.67017897699987], [-73.84556670099984, 40.67008464499985], [-73.84322150199992, 40.66934041099991], [-73.84305613800001, 40.66930593799987], [-73.84288589399995, 40.66928507599986], [-73.84271367699986, 40.66927829299993], [-73.84258049099998, 40.66928053299986], [-73.84243585399989, 40.66928855699992], [-73.84229542199994, 40.66881391799989], [-73.84227237899998, 40.66873604599992], [-73.84215802199992, 40.66870144799986], [-73.84117399199987, 40.6685074429999], [-73.84013623800001, 40.66826802499991], [-73.84008991099994, 40.66843655599991], [-73.83927850999994, 40.66820184899992], [-73.83834358199996, 40.66798902599986], [-73.83744908699992, 40.667798349999885], [-73.83730072099985, 40.66770735899994], [-73.83667829499991, 40.667586491999906], [-73.83570108799996, 40.667260221999854], [-73.83528941499992, 40.667235978999955], [-73.83487654300001, 40.6672116649999], [-73.83464965599987, 40.667115307999936], [-73.83334819299995, 40.66677318699994], [-73.83300277599999, 40.66592051799994], [-73.83295614299986, 40.66582706599994], [-73.83291947199984, 40.665765048999916], [-73.832705339, 40.665405772999854], [-73.83247768299998, 40.6649682129999], [-73.83185737999993, 40.663653930999914], [-73.83107771099985, 40.66205622399986], [-73.83030563699988, 40.660424996999936], [-73.83003131999997, 40.659894007999895], [-73.82991335799981, 40.65966566799994], [-73.82931419799984, 40.65889865199993], [-73.82859942399985, 40.657196972999884], [-73.82827925599996, 40.656033532999885], [-73.82807181199998, 40.65526770199994], [-73.82782507599995, 40.65424036199991], [-73.82770683599992, 40.653748035999875], [-73.82759641799996, 40.653288265999926], [-73.827478071, 40.65279547799985], [-73.82736078999979, 40.652307118999914], [-73.82724192599984, 40.65181216199989], [-73.82712509999998, 40.65132568599991], [-73.82701324499997, 40.65085989899995], [-73.82689623399989, 40.65034970899996], [-73.82671943599986, 40.64957881899986], [-73.82639242872236, 40.64832434880783], [-73.82642966960194, 40.648330939921514], [-73.82646514835756, 40.64834179333344], [-73.82649802934516, 40.648356653302045], [-73.82652753836285, 40.64837516994182], [-73.82655298060223, 40.648396907268534], [-73.8265567128021, 40.648438746575316], [-73.8265661906541, 40.6484800563238], [-73.82658130944175, 40.64852038005436], [-73.82660190211631, 40.64855927220128], [-73.82664941531884, 40.64860232700511], [-73.82669139111327, 40.64864860003752], [-73.82672745601468, 40.64869767959131], [-73.82675728912727, 40.64874912898789], [-73.82681157114656, 40.648776515307446], [-73.82686957464297, 40.64879904518614], [-73.82693055388556, 40.6488164289636], [-73.82699372488173, 40.64882844314125], [-73.82705666059681, 40.64886108019405], [-73.827123174935, 40.64888930136809], [-73.82719273528124, 40.64891288067976], [-73.82726478462722, 40.64893162931573], [-73.82733874603294, 40.648945397144374], [-73.82741402724488, 40.648954073918354], [-73.82743265697913, 40.6489593510468], [-73.82746167904786, 40.64896468928232], [-73.82749140376714, 40.64896682240293], [-73.827521224185, 40.648965706852245], [-73.8275505313945, 40.648961365408745], [-73.82757872696895, 40.648953886720946], [-73.82758261602079, 40.64894768992991], [-73.82758781280077, 40.64894207679325], [-73.82759417049097, 40.648937205890874], [-73.82760150947664, 40.64893321483335], [-73.82760962241967, 40.648930216374445], [-73.82761828011664, 40.64892829522538], [-73.82762723797369, 40.648927505661504], [-73.82763624291756, 40.64892786998926], [-73.82764504054491, 40.6489293779159], [-73.82765338230843, 40.648931986840054], [-73.82766103254055, 40.64893562305563], [-73.82766777510973, 40.648940183833595], [-73.82771464774395, 40.64896759750319], [-73.82776578804481, 40.648990155262155], [-73.82782033413827, 40.649007476939914], [-73.82787736674904, 40.64901927061066], [-73.82793592469449, 40.6490253375128], [-73.82799502108342, 40.64902557539942], [-73.82805365995087, 40.649019980261265], [-73.82811085304165, 40.649008646394705], [-73.82819827957856, 40.64902300983798], [-73.82828395313024, 40.64904256346106], [-73.82836734344619, 40.64906718624298], [-73.82844793440582, 40.64909672578908], [-73.82852522721237, 40.64913099927314], [-73.82859874347989, 40.64916979456999], [-73.82866802819375, 40.64921287156725], [-73.82872209037984, 40.64926061807522], [-73.82878155878606, 40.649304477755244], [-73.82884595377621, 40.649344096854094], [-73.82891475597495, 40.64937915582064], [-73.82900455239407, 40.64954665058593], [-73.82907978220996, 40.64966181132405], [-73.82911349712793, 40.64976828127217], [-73.82912090236671, 40.64982581589166], [-73.82913955292905, 40.649833843592724], [-73.8292314845031, 40.649880869870955], [-73.82933909113267, 40.64994732500823], [-73.82938618067728, 40.650034409045695], [-73.82945096002805, 40.65004416733566], [-73.82968053496424, 40.65022886619794], [-73.82986948766401, 40.6504296306443], [-73.83001172288489, 40.65091735424785], [-73.83038140837137, 40.65114791992982], [-73.83053125515515, 40.651329520124385], [-73.8306991637799, 40.65140431258623], [-73.83081972914304, 40.65151378578286], [-73.83093651353481, 40.65162612925582], [-73.8309944186776, 40.65176986504941], [-73.83110356276802, 40.65225669541116], [-73.83133151403268, 40.65274755744263], [-73.83129886631285, 40.65323377210722], [-73.83129214295737, 40.6534709244684], [-73.83139918925785, 40.653514049479845], [-73.83156040928802, 40.65360326778682], [-73.83156068724121, 40.653603650326346], [-73.83159216342204, 40.65364699650286], [-73.83161314793685, 40.653748953236196], [-73.83158322059658, 40.65381378633368], [-73.83118201843119, 40.65417083010364], [-73.83120160215553, 40.65418976108502], [-73.8311318184884, 40.65424809794608], [-73.83042516703101, 40.65483084527054], [-73.83042047893507, 40.65482827198997], [-73.83015719582366, 40.65500135673469], [-73.83013498681484, 40.65502863509911], [-73.83014785427359, 40.655034787654046], [-73.83016885409619, 40.65500906266849], [-73.83020905999666, 40.65502997144834], [-73.83018320665524, 40.6550642732668], [-73.8301301265926, 40.65503966637229], [-73.83011226681153, 40.65509728218521], [-73.83008155069426, 40.655143841054596], [-73.82990080019458, 40.65529687902322], [-73.82934489242977, 40.65571002208626], [-73.82917306859065, 40.6558648655424], [-73.8290406635642, 40.65602238094798], [-73.82902428985523, 40.65604926151257], [-73.82901417557953, 40.65610788445241], [-73.82903950091571, 40.65612309967281], [-73.82907029897899, 40.65612314544521], [-73.82910836312278, 40.65611561380376], [-73.82915279961652, 40.65609636311554], [-73.8291881885571, 40.65607227030177], [-73.82921364262403, 40.6560371254486], [-73.82934890224662, 40.65592418720412], [-73.8293897294643, 40.65589872179445], [-73.82945602175384, 40.65583397285637], [-73.82943341421314, 40.65581876175753], [-73.82966677652134, 40.655599040016355], [-73.82985054455618, 40.65564484306467], [-73.83002267408831, 40.655636819818895], [-73.83002356082699, 40.655644408828344], [-73.83006519242146, 40.655658957870024], [-73.8300860160229, 40.65566312775938], [-73.83015032686718, 40.65566460284434], [-73.83020017064874, 40.65565570813665], [-73.830308055358, 40.65561999347818], [-73.83036977490912, 40.655571794439446], [-73.83044328071922, 40.65551947244447], [-73.83053411626767, 40.655420264784226], [-73.83055410998001, 40.655394769436384], [-73.83058237017302, 40.65532375453788], [-73.83057157846139, 40.6551565164177], [-73.83055658589424, 40.655154737125905], [-73.83055620208107, 40.65530585075516], [-73.83055145115408, 40.65535943678201], [-73.83053069151869, 40.655357648990694], [-73.83053540141074, 40.65531987725622], [-73.83053759090538, 40.65491222374297], [-73.83055028060309, 40.65491224243858], [-73.8305577778015, 40.6551398033839], [-73.83056930687597, 40.65514245592541], [-73.8305583178127, 40.654927189274744], [-73.8305560975616, 40.654892921740554], [-73.83056886758, 40.65486131285202], [-73.83060245974755, 40.65480688977757], [-73.83097845384094, 40.654494997287955], [-73.83135734463538, 40.65418069958862], [-73.83134353287538, 40.654168379395145], [-73.83144752173938, 40.65410263768664], [-73.83146913310613, 40.6541158575324], [-73.8317081441153, 40.65393117429689], [-73.83195297309997, 40.65403133434726], [-73.8319968847036, 40.654001526994875], [-73.83205445130537, 40.65404641806764], [-73.8321233098165, 40.654273161676066], [-73.83200052870741, 40.65444081896505], [-73.83205201431502, 40.65446642666777], [-73.83216050229879, 40.6545095086727], [-73.83219775627535, 40.65445407709594], [-73.83221698518575, 40.654460386292925], [-73.83216594711062, 40.65453045482919], [-73.83204647773077, 40.654481076319556], [-73.83190474303174, 40.654630519939104], [-73.83185835751884, 40.65461183972555], [-73.8316022021337, 40.65484199906216], [-73.83154966356706, 40.654917034858364], [-73.83149330103005, 40.65497982053967], [-73.83141211466815, 40.65519373895365], [-73.83138106348437, 40.65535179641271], [-73.83135764938115, 40.655470975839776], [-73.83130844752576, 40.65559932463885], [-73.83116100809627, 40.65591923373715], [-73.83110341009044, 40.65594681934605], [-73.83092856003488, 40.65603056223288], [-73.83088868963759, 40.65603595147719], [-73.83083321606247, 40.65604345157029], [-73.83075005477025, 40.6560712470092], [-73.83060324919234, 40.65613989430177], [-73.83047191680427, 40.656215670937975], [-73.83046931981686, 40.65621703514758], [-73.83040507534943, 40.656251972145085], [-73.83038132783888, 40.65626527842323], [-73.83038004883461, 40.65626599456392], [-73.83034959412645, 40.65628306117194], [-73.83028389469517, 40.656306722631484], [-73.83024550384125, 40.656324569073654], [-73.83021213161007, 40.65633719250549], [-73.83007856308653, 40.65646906446877], [-73.83008724952313, 40.65648669174412], [-73.83008780413877, 40.65650636274809], [-73.83007188433658, 40.65653203038682], [-73.83005514080463, 40.6565542055135], [-73.83003301456556, 40.656567168336075], [-73.82998526032287, 40.65658261675722], [-73.82992739066431, 40.65657155373612], [-73.82988890772087, 40.65655640318574], [-73.82984486541207, 40.6565228412311], [-73.82979956416511, 40.65649197276191], [-73.82971177079273, 40.656460699950216], [-73.82965828569365, 40.656446948422015], [-73.8296089716734, 40.656431618228844], [-73.82960562793441, 40.65644448515605], [-73.82955286969737, 40.656437956806926], [-73.82955413715317, 40.65642377086847], [-73.82953233393798, 40.656426860449734], [-73.82947893001378, 40.65644927767983], [-73.82937603410723, 40.65650633521598], [-73.82930775204434, 40.656553628037784], [-73.82925819722638, 40.65660651123434], [-73.82924163157504, 40.65663788649891], [-73.8292365379905, 40.656664527519055], [-73.82924646374745, 40.65668501339268], [-73.829258470225, 40.656706295098665], [-73.8292606922071, 40.6567258116075], [-73.82925959351662, 40.656741673244774], [-73.82926720321493, 40.65676960885122], [-73.82928204674597, 40.65681342281772], [-73.82930287477964, 40.65687263733883], [-73.82930974695338, 40.656932457535156], [-73.82928525714213, 40.65696556283856], [-73.8292662361284, 40.656984085030096], [-73.82923845041701, 40.65700718833912], [-73.82920543743198, 40.65703583493437], [-73.82915801556825, 40.65707444514942], [-73.82913482607461, 40.65709375102419], [-73.82911536880955, 40.65711766545049], [-73.82909990801164, 40.65713080100087], [-73.82908591602636, 40.65714092489513], [-73.82905921013833, 40.65715292547795], [-73.82903021278634, 40.65716524162264], [-73.8290043505531, 40.657174389432235], [-73.82898285487585, 40.657185608348904], [-73.82896778740296, 40.65720524792007], [-73.82896084087689, 40.657226015639814], [-73.82896508276986, 40.657262985923694], [-73.82898079995603, 40.65729522216301], [-73.8290132530184, 40.65742521182773], [-73.82907748488758, 40.65758128480352], [-73.82913225563826, 40.657702754762674], [-73.8291791973457, 40.65779977854427], [-73.82920828304648, 40.65782204739614], [-73.82925795356903, 40.65785419395176], [-73.8293238942211, 40.65788066313418], [-73.8293876196137, 40.65788523554142], [-73.829417622329, 40.657883235187946], [-73.82945589724294, 40.65789870070532], [-73.82951375096921, 40.65791452442429], [-73.82952765623907, 40.65792898916625], [-73.82953241448706, 40.657938357706115], [-73.82953964021831, 40.65795661560979], [-73.8295541464757, 40.65797806131063], [-73.82965499496454, 40.65808820416821], [-73.8296890823881, 40.65810921378971], [-73.82972820317205, 40.65812103421233], [-73.82975314668518, 40.65813615539201], [-73.82977638969626, 40.65816095049101], [-73.82979192787604, 40.65818477793193], [-73.82980019620072, 40.65820303702627], [-73.82982243247803, 40.658218312164244], [-73.82986405418082, 40.658229503318054], [-73.82991710261359, 40.65824944025393], [-73.82993039585925, 40.658260095953196], [-73.82993139548506, 40.658271994963805], [-73.82993571633205, 40.658287551171746], [-73.82995461939598, 40.65830234272708], [-73.82997749698586, 40.658312700818946], [-73.83000933019638, 40.658323712267645], [-73.830053020508, 40.65833887197489], [-73.83009440305685, 40.658359102859706], [-73.83011807881135, 40.6583792993705], [-73.83013028740389, 40.65840264372116], [-73.83013062197735, 40.658425804682416], [-73.83012345473455, 40.65845006243003], [-73.8301136277782, 40.65846019477694], [-73.83007584536904, 40.65848264406377], [-73.83005061350448, 40.65849004839812], [-73.83002290290227, 40.65849173668062], [-73.82998894883302, 40.65849182605149], [-73.82995500685273, 40.65848794953018], [-73.8299167073457, 40.65847962236206], [-73.82988279238083, 40.65846860680395], [-73.82985300549767, 40.658467911061095], [-73.82981318489234, 40.65847766646732], [-73.82977169416301, 40.65848836843674], [-73.8297320793274, 40.65849859839243], [-73.82970706838907, 40.658502513864484], [-73.82966825159824, 40.65852258057251], [-73.82962667738495, 40.65855707860324], [-73.82960652783132, 40.658600186315454], [-73.82959986903019, 40.65865680691539], [-73.82961343158013, 40.65870966027128], [-73.82967103985062, 40.65879528408579], [-73.8296925320881, 40.65883615062187], [-73.82972318498385, 40.65889390400776], [-73.82975486375837, 40.65894854086477], [-73.82981399218815, 40.65901687524389], [-73.82982393761212, 40.65903097943117], [-73.82987347914914, 40.659101233040296], [-73.82991536797653, 40.659155414532115], [-73.82992819287433, 40.65918098140742], [-73.82994255948424, 40.65924208618034], [-73.8299597282835, 40.659296555440676], [-73.82996192425614, 40.659303519491544], [-73.829965808183, 40.65932446628158], [-73.82997176218308, 40.65934938454383], [-73.82999639203986, 40.65945334232529], [-73.83000997071117, 40.65950143734391], [-73.83001426888887, 40.659523338315715], [-73.83001290827576, 40.659554586714876], [-73.83002344734399, 40.65957887999991], [-73.83005013033102, 40.65963239448661], [-73.83008151753675, 40.65971114418245], [-73.83010115287577, 40.65975480863344], [-73.83011665938497, 40.65978815376841], [-73.83013138340058, 40.659806903175934], [-73.83014950104759, 40.6598086849573], [-73.83016243281726, 40.6598041109262], [-73.83017248495642, 40.65978890212878], [-73.8301773206398, 40.65977606369466], [-73.83017592521581, 40.659758451051225], [-73.83016641944104, 40.65973654021963], [-73.83015376710541, 40.65972096853834], [-73.83014068468839, 40.659709202632996], [-73.83013365723922, 40.65969380058542], [-73.83012437380432, 40.65966760604267], [-73.83011550258621, 40.65964315775379], [-73.83010937351253, 40.65960903757745], [-73.83009786351585, 40.65956491303398], [-73.83007338502352, 40.65947650297142], [-73.8300689875143, 40.659423984367145], [-73.8300341461414, 40.65925979142264], [-73.83002780027702, 40.65922988643684], [-73.8300216692606, 40.65919656117337], [-73.83001214184989, 40.659163163878794], [-73.82994729215727, 40.659105694910686], [-73.82989716548171, 40.65904868626837], [-73.82988152586422, 40.65903089943323], [-73.82982589468608, 40.658961269237615], [-73.82978551303394, 40.65888007029169], [-73.8297638699683, 40.65882457547287], [-73.82972481464317, 40.65874402629943], [-73.82971373370535, 40.65870140373724], [-73.82969520083391, 40.65865592869146], [-73.8297065289459, 40.658601976355435], [-73.82979240260045, 40.658568017626614], [-73.82987072192734, 40.658570973769045], [-73.82994527545966, 40.658588127013736], [-73.83000120757706, 40.658593890244056], [-73.83005721360321, 40.65857124945066], [-73.83013346980562, 40.65853149250714], [-73.8301580575766, 40.65851742995981], [-73.83019910654943, 40.658508969064115], [-73.8302440680071, 40.65842950198766], [-73.83024427039376, 40.658349969348166], [-73.83016976043841, 40.658315773528294], [-73.8301027678112, 40.658258865210236], [-73.83000600274035, 40.65817634852307], [-73.82991415861346, 40.658144597291], [-73.82989047899356, 40.65813641184888], [-73.8298271228172, 40.65811643485667], [-73.82977140702343, 40.65802545719266], [-73.82970079467543, 40.657925935443245], [-73.82963013197599, 40.65784629785772], [-73.82951463014486, 40.65779783883258], [-73.82945866947031, 40.65780343760186], [-73.82937290582937, 40.65779478840809], [-73.82922393902857, 40.657706512383925], [-73.82910499991122, 40.657544429058845], [-73.82906423436305, 40.657442112650095], [-73.82909434825203, 40.65733421866331], [-73.82917290744729, 40.65724344063423], [-73.82924393205002, 40.657181056146825], [-73.82934715237306, 40.65710305815867], [-73.82942711948475, 40.6570137395408], [-73.8294982617395, 40.65690590795538], [-73.829494655654, 40.65685761471543], [-73.82946131261423, 40.65676951032551], [-73.82946151593032, 40.65668997716951], [-73.82950265119837, 40.65664743198321], [-73.82958468477634, 40.65665607388188], [-73.82968157040276, 40.65669030322539], [-73.82977849236663, 40.656710330484664], [-73.829891865403, 40.656728820795706], [-73.82990152392094, 40.65673039577452], [-73.83000967244328, 40.656736236211835], [-73.83009172569676, 40.656736357448665], [-73.83018507808814, 40.656693887968906], [-73.83024118364905, 40.65663148083697], [-73.83032728545408, 40.65650662730798], [-73.83039087156523, 40.65643570913236], [-73.83050532057368, 40.656342367316164], [-73.83106670806474, 40.65613845468092], [-73.83110547238412, 40.656118761668594], [-73.83114230124039, 40.6560970206366], [-73.83120986304854, 40.65605265626985], [-73.83127112568108, 40.656003271494676], [-73.83132544915816, 40.65594938222129], [-73.83137226599476, 40.65589155141575], [-73.8314110871302, 40.65583038321861], [-73.83144150703528, 40.65576651663348], [-73.83146320794879, 40.655700618851284], [-73.8314759631957, 40.65563337828009], [-73.83146993367257, 40.65555345217365], [-73.83151638832737, 40.65546449041526], [-73.8315472294415, 40.65541574309183], [-73.83155930253487, 40.65535534364431], [-73.83155309177049, 40.6552429728747], [-73.83161418598198, 40.65519443568756], [-73.83168728328506, 40.655091260206675], [-73.83175312959285, 40.65496210950518], [-73.8318523847929, 40.65488503826691], [-73.83191670520524, 40.65488260712334], [-73.83200725288442, 40.6549005207142], [-73.83206411482192, 40.654889396766535], [-73.83212857266406, 40.65487544577758], [-73.8321818606802, 40.65489475195476], [-73.83224117996961, 40.6550865424841], [-73.83248465579328, 40.65587372706237], [-73.83275352381291, 40.65674298580427], [-73.83285224112385, 40.65706213877172], [-73.83290309524602, 40.657483771623006], [-73.83278194361473, 40.657548460383495], [-73.83208497100838, 40.656131028511304], [-73.83194394578256, 40.65617142725338], [-73.832600363706, 40.65782659162732], [-73.83201491707895, 40.65807753263678], [-73.83096907996668, 40.65823029417234], [-73.83128446183628, 40.65840668263044], [-73.83194576717005, 40.658274346859095], [-73.83284428358573, 40.65813812869346], [-73.83338439472544, 40.65805185920798], [-73.83306069956936, 40.656687879277946], [-73.83280439716816, 40.65582273525583], [-73.83256888625472, 40.65502775193516], [-73.83238192807039, 40.65423766665479], [-73.83217106968996, 40.653346552404486], [-73.83197428225198, 40.65264323892533], [-73.8318330050207, 40.65213831057452], [-73.83170565136656, 40.65168313457724], [-73.83156681412632, 40.651167714415784], [-73.8314392955408, 40.65069429983084], [-73.83127055094019, 40.65006783182466], [-73.83111241541593, 40.649419922041375], [-73.83104647225137, 40.64908920770505], [-73.83103675075473, 40.64878844359568], [-73.83103729946343, 40.64871144237756], [-73.83104672459606, 40.64863477538528], [-73.83106495328707, 40.64855903507983], [-73.83109184464118, 40.64848480675928], [-73.83112719082222, 40.64841266403528], [-73.8311707186609, 40.6483431644014], [-73.83122209176555, 40.64827684492427], [-73.83128091312282, 40.648214218094076], [-73.83134672816617, 40.64815576786409], [-73.83141902828879, 40.64810194591172], [-73.83149725477422, 40.64805316814803], [-73.83158080311367, 40.64800981150383], [-73.83166902767807, 40.647972211017645], [-73.83420327568595, 40.648096289815804], [-73.83434221467297, 40.648099766847906], [-73.83448069356174, 40.64810904803825], [-73.83461829473161, 40.64812410539682], [-73.83475460320872, 40.648144893514186], [-73.83488920791645, 40.64817134969802], [-73.835021702915, 40.6482033941623], [-73.83515168862532, 40.6482409302681], [-73.83527877303453, 40.648283844814614], [-73.83540257287673, 40.64833200838043], [-73.8354627270177, 40.64836934008047], [-73.83551775585408, 40.64841100708057], [-73.8355671231816, 40.64845660338249], [-73.83561034795801, 40.64850568470053], [-73.8356470089908, 40.648557772790504], [-73.83567674904124, 40.648612360108984], [-73.83569927830658, 40.64866891475942], [-73.83571437724277, 40.64872688567386], [-73.83572189870628, 40.648785707983265], [-73.83570725223217, 40.64884020954196], [-73.83570065481867, 40.648895613671996], [-73.83570218834342, 40.648951232947006], [-73.83571183379932, 40.649006377269814], [-73.83572947152959, 40.6490603624343], [-73.83575488271195, 40.64911251861524], [-73.83578775207206, 40.649162198678994], [-73.83572275412187, 40.64937822992839], [-73.83622362056934, 40.65107375730866], [-73.836538981054, 40.6526984633564], [-73.83704202978238, 40.65440955028222], [-73.83754994165396, 40.656028284424565], [-73.8379647452176, 40.657764771744134], [-73.8382680405472, 40.65937790027199], [-73.83878517503126, 40.661088448434775], [-73.83805800948616, 40.662006063339035], [-73.8387360855594, 40.662454060021076], [-73.83934944817479, 40.66218467836681], [-73.83958258784455, 40.66208228458937], [-73.83997949975627, 40.66190427386222], [-73.83972540353096, 40.66095065887474], [-73.83978752612934, 40.660485998048], [-73.83942312759972, 40.65923064416343], [-73.83895864143369, 40.65755139365543], [-73.83855449359646, 40.6558920626626], [-73.83814021702017, 40.654224792560896], [-73.83762971693615, 40.6525077107773], [-73.83723913520092, 40.650896922762705], [-73.83681334575157, 40.649249415001925], [-73.83592320073285, 40.645499790674954], [-73.83626109724382, 40.645351677048296], [-73.83633074787276, 40.64532114928048], [-73.83640735456112, 40.64531741349842], [-73.83970469104592, 40.64515639707752], [-73.84000646464365, 40.645136068888796], [-73.84009313481012, 40.645056203562476], [-73.84030326670076, 40.645009444971855], [-73.84058394688778, 40.644863026022605], [-73.84089258685123, 40.644969036711764], [-73.845996083007, 40.64456831949446], [-73.84950699084165, 40.64413376567876], [-73.85183280737658, 40.64586005671262], [-73.85192932673164, 40.6459731546442], [-73.85201779804854, 40.646090008962794], [-73.8520979699638, 40.64621028771974], [-73.85216961468622, 40.646333649237064], [-73.85223252864492, 40.64645974307781], [-73.85228653306682, 40.64658821104078], [-73.85233147448616, 40.64671868817892], [-73.85236722518137, 40.646850803835086], [-73.85239368353616, 40.64698418269542], [-73.85241077433211, 40.64711844585524], [-73.85241844896015, 40.6472532118956], [-73.85241668556101, 40.64738809796636], [-73.85239926161105, 40.6474855292713], [-73.85237453097487, 40.64758203920727], [-73.85234257501176, 40.64767731014961], [-73.85230349886227, 40.647771028550196], [-73.85225743110122, 40.64786288596904], [-73.85220452331654, 40.64795258008958], [-73.8521449496086, 40.6480398157142], [-73.85207890601953, 40.648124305734925], [-73.85203846598957, 40.64824676738453], [-73.85200205819733, 40.648369960827395], [-73.85198704979906, 40.648440729640605], [-73.85196495171064, 40.648510411831545], [-73.85193589437841, 40.648578595993264], [-73.851900049338, 40.6486448795617], [-73.85185762820169, 40.64870887119298], [-73.85180888140944, 40.648770193073794], [-73.85175409675192, 40.648828483152535], [-73.85174846522098, 40.64883872545535], [-73.85174616040437, 40.648866909827035], [-73.85172587398122, 40.648912154569786], [-73.851697757682, 40.64894799320826], [-73.85167303933729, 40.6489693146833], [-73.85165839001101, 40.649000900377935], [-73.85163030849162, 40.64902136277526], [-73.85161119411089, 40.649043546699254], [-73.8515887121938, 40.64906743355764], [-73.85158643770166, 40.64908195165877], [-73.85159089132347, 40.649096478442125], [-73.8515717997111, 40.64910841232451], [-73.85155498639377, 40.64910411981999], [-73.8515449128939, 40.649095566046405], [-73.85154042061026, 40.64909812226115], [-73.8515336867929, 40.64910067608463], [-73.8515213251907, 40.649111762743246], [-73.85150896387735, 40.64912370580646], [-73.85150335327252, 40.64912540752068], [-73.85149886566872, 40.649126254781436], [-73.85145630361569, 40.649102283949176], [-73.85143723187643, 40.64910567656139], [-73.85139792829229, 40.64912868809394], [-73.85138556695787, 40.64914062946871], [-73.85137657351407, 40.64915001435361], [-73.85138773436582, 40.649174799373874], [-73.85137202165079, 40.64917990359876], [-73.85133725114797, 40.649182421175006], [-73.85132719860135, 40.64916447157302], [-73.85132502492941, 40.649133718905006], [-73.85132394460764, 40.64911492617057], [-73.85130828776491, 40.64909526036337], [-73.85129485181847, 40.649085846484944], [-73.85124297023212, 40.64907516355653], [-73.85118955811286, 40.64907062099602], [-73.85113586061982, 40.6490723247009], [-73.85108312956335, 40.649080234953836], [-73.85103259422581, 40.64909416734892], [-73.85098543270321, 40.64911379709003], [-73.85094274444295, 40.64913866656226], [-73.85090552461405, 40.64916819600036], [-73.8508976017595, 40.64920064337293], [-73.850898688717, 40.64921602004624], [-73.85091541738015, 40.64925789588271], [-73.85093211912952, 40.64931172852641], [-73.85093423750284, 40.64936725172731], [-73.850927408209, 40.64941251427143], [-73.85091158726522, 40.64946630643265], [-73.85090369682953, 40.64948423230819], [-73.85088799001969, 40.64948762919146], [-73.85088238186796, 40.649487621920244], [-73.85087901364395, 40.64948932533866], [-73.8508621464434, 40.649509803763024], [-73.85083072973137, 40.64951659533433], [-73.85082731757902, 40.64953794476203], [-73.8508093154364, 40.64956440096624], [-73.85080586316285, 40.649603688120564], [-73.85080470275665, 40.6496207701668], [-73.85079562923032, 40.64966688284576], [-73.85079213977107, 40.649722398269354], [-73.85079096686978, 40.649745458385965], [-73.8507707816691, 40.649745432196724], [-73.85076965946267, 40.64974543074063], [-73.85076965564022, 40.649747139694014], [-73.85077185391279, 40.649766788110924], [-73.85078640830984, 40.64977791128623], [-73.85078185996018, 40.649806092222754], [-73.85070897081077, 40.6498034361198], [-73.85070336667691, 40.64980171988691], [-73.85068206689586, 40.64979827666417], [-73.85066077351658, 40.64979226925465], [-73.85061030166267, 40.64979476622227], [-73.85059680888422, 40.64981012328831], [-73.85058099294764, 40.649862205947784], [-73.85058538497059, 40.649904065801174], [-73.85058537349428, 40.649909190315405], [-73.85059992406472, 40.649922022467436], [-73.85058867347907, 40.64993738194251], [-73.85057629463016, 40.6499570119271], [-73.85057504998849, 40.6500116760312], [-73.85058726080118, 40.65006806707686], [-73.85059386659752, 40.6501227413788], [-73.85060609083524, 40.65017315317966], [-73.85060708653099, 40.650228674420646], [-73.85060247877834, 40.650283335830125], [-73.85058665633537, 40.65033798099494], [-73.85057653598565, 40.65034992519283], [-73.85055520793269, 40.6503584392563], [-73.8505383729429, 40.65036439663913], [-73.85052714912491, 40.65036865235395], [-73.8505203895992, 40.65038231154805], [-73.85049799039878, 40.65036861445438], [-73.85047002729478, 40.65033612046203], [-73.85042751079845, 40.65029164901825], [-73.85039615267061, 40.65027281689653], [-73.85037708349644, 40.6502744998756], [-73.85037704901447, 40.6502898755957], [-73.85039827326455, 40.65031137131142], [-73.85042180513697, 40.65033520302619], [-73.85043630981514, 40.6503685342596], [-73.85043618918141, 40.650422346766355], [-73.8504450863127, 40.65045567070605], [-73.85042261012721, 40.650476994825375], [-73.85039790929824, 40.65048977594706], [-73.8503698265407, 40.65051024021072], [-73.85033840378038, 40.65051959365265], [-73.8503148248562, 40.65053152198848], [-73.85026883423848, 40.65053658715473], [-73.85024190234327, 40.65054338440373], [-73.85021831217075, 40.65056129146181], [-73.85019692461215, 40.65059713849151], [-73.85017554722222, 40.650627859493774], [-73.85017217294566, 40.650632126404396], [-73.85015529624562, 40.65065602145418], [-73.85011037314206, 40.65068500381192], [-73.85006096796688, 40.65071141779318], [-73.85005311362572, 40.65071311651133], [-73.85004638150967, 40.65071481551951], [-73.85003739282374, 40.65072249251934], [-73.85002053333528, 40.65073869935453], [-73.84997000591888, 40.650765966563554], [-73.84991497447258, 40.650800914414745], [-73.8498543445891, 40.650832439862995], [-73.84978812176738, 40.65085712398512], [-73.8497286297765, 40.65088096213344], [-73.8496893831687, 40.65087834833244], [-73.84968598790128, 40.650892010198575], [-73.84968367444216, 40.65092361226076], [-73.84961078670443, 40.65092009973005], [-73.84957152059873, 40.65092602763811], [-73.84954570943403, 40.65093368092026], [-73.84951990723373, 40.65093706440981], [-73.84948962372066, 40.65093873256865], [-73.84946832622983, 40.650934433389544], [-73.8494526399356, 40.650927579873404], [-73.84942572156427, 40.65092925242015], [-73.84937404260533, 40.65096933110812], [-73.84930217800292, 40.651009382823084], [-73.84925726840814, 40.65103153255095], [-73.84922351138029, 40.65108102948895], [-73.84921448394407, 40.651105788247456], [-73.84920545733438, 40.65112969177341], [-73.84921658357277, 40.651168997987604], [-73.84923897186141, 40.65118781867134], [-73.84925128119134, 40.65119893909617], [-73.84928261075792, 40.65123143779344], [-73.84929598318138, 40.65126903915758], [-73.8493149601344, 40.651281836058025], [-73.84931501987279, 40.65128187617559], [-73.84933405656909, 40.65129471436317], [-73.84934183808984, 40.651324619678086], [-73.84934739432025, 40.65134768875856], [-73.84935406865914, 40.65137161386604], [-73.84936074751849, 40.65139383119351], [-73.84939766406322, 40.65143487895031], [-73.84942653543145, 40.65146180228387], [-73.84946074188646, 40.651484816509644], [-73.8494993834858, 40.651503316139454], [-73.84954338481424, 40.65151729434486], [-73.84958985302212, 40.651525414081334], [-73.84963745408, 40.65152744224391], [-73.84992253316281, 40.651657963603704], [-73.84997165682728, 40.65165085434766], [-73.8500216521738, 40.65165020427447], [-73.85007106583203, 40.651656032281764], [-73.85011846133993, 40.65166816894875], [-73.85024113552014, 40.65157691801331], [-73.85029008960439, 40.65159190568471], [-73.85034262272379, 40.65159941087342], [-73.850396026226, 40.651601096646615], [-73.85044919461023, 40.65159692810691], [-73.8505010272444, 40.65158699154691], [-73.8505504511477, 40.65157149266227], [-73.85059644320425, 40.65155075229338], [-73.85063805134276, 40.651525199783364], [-73.85065070643299, 40.65151685777235], [-73.85066506411285, 40.65151030717369], [-73.8506806922369, 40.651505745151205], [-73.85069712042095, 40.65150330901478], [-73.85071385420143, 40.65150307208859], [-73.8507303899161, 40.65150504150387], [-73.85074622986458, 40.651509157983945], [-73.8507608972872, 40.65151529762897], [-73.8507739507164, 40.65152327564461], [-73.8507849972619, 40.65153285190431], [-73.85089682314013, 40.65150012580615], [-73.85100570443234, 40.65146207349548], [-73.8511112043237, 40.651418847633906], [-73.85121289956777, 40.651370621638634], [-73.85131038218424, 40.65131758898634], [-73.8514032610962, 40.651259962436924], [-73.85149116369915, 40.65119797317987], [-73.85176596171377, 40.65082762385985], [-73.85178606566313, 40.65081563951879], [-73.8518083735549, 40.65080618429004], [-73.85183234172547, 40.650799488605834], [-73.8518573860499, 40.650795715645444], [-73.8518828961762, 40.65079495735937], [-73.8519083205149, 40.65079724246117], [-73.85193296507735, 40.65080252166269], [-73.85195622589887, 40.65081066558662], [-73.85197753292577, 40.65082147464948], [-73.85202131328329, 40.65081555671428], [-73.8520630898557, 40.650803960543776], [-73.8521016033448, 40.650787035688936], [-73.85213569281673, 40.65076529232678], [-73.85216433069581, 40.650739385880875], [-73.85218665374042, 40.650710097264614], [-73.85219704272762, 40.65069056992688], [-73.8522276747456, 40.65065671385671], [-73.85226445114697, 40.65062661455847], [-73.85230658847337, 40.650600913240076], [-73.85235318906456, 40.65058015741842], [-73.85240326018318, 40.65056478925558], [-73.8524557351611, 40.65055513613993], [-73.85250949612202, 40.650551403711816], [-73.85256339779653, 40.65055367148329], [-73.85261629191655, 40.65056189114387], [-73.85266705167781, 40.65057588759025], [-73.8528487636828, 40.65060697497252], [-73.85303212709414, 40.65063180860759], [-73.85321677107093, 40.65065033827065], [-73.85340232218083, 40.650662526486464], [-73.85358840515592, 40.650668348605116], [-73.85361016035154, 40.65066612555268], [-73.85368048940352, 40.6506561022797], [-73.853749560563, 40.650641883112854], [-73.85382105364425, 40.65063807819402], [-73.8538926809404, 40.650639904213115], [-73.85396367995506, 40.650647341731684], [-73.85403329487946, 40.65066031157476], [-73.85410078463791, 40.650678675673944], [-73.85419248918079, 40.650727177009394], [-73.85428814586837, 40.650771027212656], [-73.8543873486555, 40.65081004014438], [-73.85448967644179, 40.65084405019863], [-73.8545946948596, 40.650872913005834], [-73.8547019581179, 40.650896506045825], [-73.85478828709621, 40.65091700060927], [-73.85491085294494, 40.65095207423868], [-73.8550300218924, 40.65099338414661], [-73.8551452476776, 40.65104074097392], [-73.85524875978942, 40.651090200298626], [-73.8553479640749, 40.651144540009774], [-73.85544246443914, 40.65120354314979], [-73.85544068316777, 40.651215071567044], [-73.85544139399369, 40.65122666690516], [-73.85544457788902, 40.65123801875736], [-73.85545014962247, 40.651248823234674], [-73.85545796003967, 40.65125879110121], [-73.85546780005674, 40.65126765551702], [-73.85547940625679, 40.65127517918143], [-73.855492467942, 40.65128116068574], [-73.85550663545077, 40.65128543990492], [-73.8555215295185, 40.65128790228396], [-73.85553675143038, 40.65128848190484], [-73.8555518936946, 40.65128716325115], [-73.85558158624221, 40.65127215170911], [-73.8556139177695, 40.6512607325139], [-73.8556481446975, 40.65125316829065], [-73.85568347985614, 40.651249633005186], [-73.85571911058845, 40.65125020796399], [-73.85575421744053, 40.65125487994403], [-73.85578799300526, 40.65126354149639], [-73.85583421027903, 40.65127447840715], [-73.8558782271612, 40.65128980133864], [-73.85591932944749, 40.65130926166608], [-73.8559568502225, 40.65133254363233], [-73.85599018068156, 40.651359269471584], [-73.85605277956542, 40.65139889146354], [-73.85611005993886, 40.651442934081814], [-73.85616148594845, 40.651490985319235], [-73.85620657650314, 40.651542595668346], [-73.85624490977578, 40.651597282325675], [-73.85627612714923, 40.65165453370846], [-73.85629993657182, 40.65171381424032], [-73.85631611529033, 40.65177456936082], [-73.8563245119338, 40.651836230713705], [-73.85632504793078, 40.651898221463334], [-73.85631771824539, 40.651959961690814], [-73.85631557540161, 40.65197067296015], [-73.85631053684067, 40.65202347805094], [-73.85631458119158, 40.65207633271785], [-73.85632763942613, 40.65212833456416], [-73.85634948861879, 40.652178595751586], [-73.85637975575189, 40.65222625815806], [-73.85641792408416, 40.65227050803005], [-73.8564633419696, 40.65231058987507], [-73.85651523398519, 40.652345819361805], [-73.85651776435327, 40.65241568407999], [-73.85652917856396, 40.65248503261862], [-73.85654936904841, 40.65255321126237], [-73.85657814550649, 40.65261957732213], [-73.85661523669977, 40.65268350519321], [-73.856660293008, 40.65274439225287], [-73.85671288972269, 40.65280166454157], [-73.85677253105183, 40.652854782173215], [-73.85683865479179, 40.65290324442496], [-73.85691063762755, 40.65294659445796], [-73.85695671321744, 40.652959281879056], [-73.85700489642797, 40.652966044760376], [-73.8570538834724, 40.6529667001055], [-73.85710234881229, 40.65296123018148], [-73.8571489810268, 40.65294978299878], [-73.85719251829887, 40.65293266830593], [-73.8572317825592, 40.65291034920802], [-73.857265711365, 40.65288342963558], [-73.8572656367711, 40.652876406753755], [-73.85726691218123, 40.652869451141804], [-73.8572695101429, 40.652862712516615], [-73.85727337473517, 40.652856335924405], [-73.8572784227743, 40.65285045861886], [-73.85728454560274, 40.65284520710652], [-73.857292864355, 40.65284002416054], [-73.8573022208929, 40.65283599639789], [-73.85731234163377, 40.65283324158901], [-73.8573229306507, 40.6528318402835], [-73.85733367832401, 40.652831833455224], [-73.8573338217152, 40.652831845695125], [-73.85739480830867, 40.65283376808535], [-73.85745561922818, 40.65282975475945], [-73.85751526257104, 40.65281987117969], [-73.8575727654805, 40.65280427855977], [-73.85762719001433, 40.65278323123447], [-73.8576776484451, 40.65275707251168], [-73.85772331773968, 40.652726229071625], [-73.8578960147709, 40.65280300339614], [-73.8581470689611, 40.65291904009401], [-73.85828132383823, 40.65299487215183], [-73.85839791574583, 40.653132998353556], [-73.85852026658151, 40.653311189403766], [-73.85856069416674, 40.65353824112123], [-73.85857198690208, 40.653725197940965], [-73.85868853267029, 40.653885577914785], [-73.85876419753458, 40.65403255472361], [-73.85876338041551, 40.654049771785886], [-73.85875807991593, 40.65416162733512], [-73.85885729088807, 40.65422851438058], [-73.85892499802527, 40.654255434322025], [-73.85900327710367, 40.65428655623113], [-73.85915498446028, 40.654402468871005], [-73.85926552902157, 40.65463850910966], [-73.85927772471581, 40.65470428193611], [-73.85929442176521, 40.654794329098245], [-73.85929417768489, 40.6549100555591], [-73.85930573554418, 40.654972383384326], [-73.85939916950612, 40.65500810629567], [-73.85951590784107, 40.655079466255074], [-73.85961223713458, 40.65514717084673], [-73.85967017937695, 40.65518789516082], [-73.85971498705463, 40.65523000429859], [-73.85975838110188, 40.65527594725711], [-73.8599590652879, 40.65549029704331], [-73.86072952202075, 40.65604681266761], [-73.86084030699911, 40.656171574779144], [-73.86106223001468, 40.656251961649616], [-73.86125486605461, 40.6563634693526], [-73.86135193244354, 40.65639462119119], [-73.86136381950391, 40.65641475416531], [-73.86136875139077, 40.65643518131159], [-73.86135850209601, 40.65645937797578], [-73.86135294244752, 40.65648094641138], [-73.86136322406594, 40.65651084904303], [-73.86138863839427, 40.65654285630874], [-73.8614013099616, 40.65656773931727], [-73.86140125106603, 40.65657808763923], [-73.86139225922217, 40.65658721647978], [-73.86136038361256, 40.65659420520712], [-73.86130453077845, 40.6565889764917], [-73.86123700169337, 40.65658368055685], [-73.86117420316046, 40.65657281662431], [-73.86108773357645, 40.656559509821406], [-73.86101561574247, 40.65654210479475], [-73.86098888249673, 40.65653696605765], [-73.86095970776479, 40.656545741612355], [-73.86093087921417, 40.65656129711496], [-73.86090869860645, 40.656568296936086], [-73.86086488851105, 40.65656105262832], [-73.86083111892167, 40.65656062746426], [-73.8607879936244, 40.65656993558813], [-73.86072304455804, 40.656588631862135], [-73.86067414586064, 40.656584591665585], [-73.86064784053592, 40.65651297627976], [-73.86063722236801, 40.656471847101834], [-73.86062404578773, 40.65646529853594], [-73.86061669329668, 40.65646465720695], [-73.86061429246945, 40.65647560596028], [-73.86060742785907, 40.65651814179852], [-73.86062302068669, 40.65662787031268], [-73.86061557040897, 40.65664054217458], [-73.8606030661496, 40.656656474211104], [-73.86058123554955, 40.65666879928139], [-73.8604949922559, 40.656684449245326], [-73.86045768505309, 40.656686381328], [-73.86044288674648, 40.65669698517322], [-73.86044003211457, 40.65671737543661], [-73.86044847473897, 40.656731000372886], [-73.86048143216688, 40.65673998720494], [-73.86050971089193, 40.65674718601957], [-73.86051434648235, 40.65675339538305], [-73.86051503888413, 40.6567670097428], [-73.86047711306266, 40.65680648870485], [-73.86044742487603, 40.65683653491432], [-73.86035088822705, 40.65687934621476], [-73.8603057625893, 40.65689748985388], [-73.86028077947775, 40.65691752337259], [-73.86024448470059, 40.65695316147395], [-73.86022643272057, 40.65698470455705], [-73.86020839501049, 40.65701037444651], [-73.86019280975664, 40.657017710835355], [-73.86015948294516, 40.65701226368072], [-73.86015025570852, 40.65699537213987], [-73.86014272235784, 40.656961355566395], [-73.86011860091382, 40.65684538540262], [-73.86011027831917, 40.656809583354615], [-73.86009989868663, 40.656792992796326], [-73.86007981013411, 40.65678226363547], [-73.86006777819524, 40.65677986104951], [-73.86004168871449, 40.65679426689902], [-73.86003298298654, 40.65682052391104], [-73.86006870315983, 40.65693949825014], [-73.86006234511073, 40.65696372700656], [-73.86005569574436, 40.65697225515543], [-73.86004870310448, 40.656972549706204], [-73.86000758602333, 40.6569700282924], [-73.85996142495827, 40.65696629518921], [-73.85993091334905, 40.656965833688346], [-73.85988145175494, 40.65696512678623], [-73.85984572805579, 40.65696766219373], [-73.85983363521494, 40.65697706322045], [-73.85984591953108, 40.65699662103968], [-73.85988581709219, 40.657014209361556], [-73.8599341705406, 40.65703803700531], [-73.85996243295547, 40.65705261874036], [-73.86000010014942, 40.65705011196406], [-73.86000819218508, 40.657049744706775], [-73.86002338719183, 40.65704871413688], [-73.8600544693938, 40.65704144973319], [-73.86006924302438, 40.65704267592144], [-73.8600749979822, 40.65704861107995], [-73.86007068925896, 40.657058047902595], [-73.86005194367061, 40.657077486192584], [-73.86004275639077, 40.65708727391358], [-73.86002730651316, 40.657104300786], [-73.85999101706163, 40.65713697389694], [-73.85996128243944, 40.65717146595805], [-73.85994397479166, 40.65720924111883], [-73.85993882823615, 40.657222822069706], [-73.859915202642, 40.65721444921534], [-73.85988237954868, 40.65727360319275], [-73.85981027609661, 40.65732241149373], [-73.85980008978002, 40.65738015765183], [-73.85979094247071, 40.6574320146118], [-73.85980671376048, 40.65745659918364], [-73.8599233637411, 40.657670448972134], [-73.8599371302826, 40.65770538048095], [-73.85993897279535, 40.657720177150324], [-73.85992218331278, 40.657735197565096], [-73.85987781547442, 40.657752189195975], [-73.85980115657043, 40.657785608986785], [-73.85973220186875, 40.65782348532852], [-73.85970571828646, 40.65783698479454], [-73.85966622177314, 40.657886205519254], [-73.85965275827455, 40.65790113881692], [-73.85961945958066, 40.6579435366632], [-73.8595924482289, 40.65798845320094], [-73.85957204817527, 40.658035349751735], [-73.85955850409009, 40.65808366389066], [-73.85955197842098, 40.65813281619025], [-73.85955515249182, 40.658199410351564], [-73.85956518319344, 40.65826560908615], [-73.85958200860286, 40.65833100359451], [-73.85960552483706, 40.65839519004295], [-73.8596355866937, 40.65845777205655], [-73.85967919564028, 40.65849227645346], [-73.85972750407433, 40.658522936426], [-73.85977993090417, 40.658549383168015], [-73.8598358454955, 40.65857129855368], [-73.85989457525865, 40.658588418964136], [-73.85996520362455, 40.658589698040444], [-73.8600352653625, 40.6585966168402], [-73.86010398613425, 40.658609098895106], [-73.86013583432293, 40.65860496877396], [-73.86016814087985, 40.65860502745556], [-73.86019996283548, 40.65860927322729], [-73.86023037136584, 40.65861758216259], [-73.86025847890078, 40.65862971173905], [-73.86028735677134, 40.65870650880664], [-73.86031051842248, 40.658784420991545], [-73.8602702573463, 40.658805414746475], [-73.86023343179158, 40.65882979359743], [-73.86020052991832, 40.658857234380406], [-73.86019761204383, 40.658860858644715], [-73.86019553423012, 40.65886480442198], [-73.86019435541495, 40.65886895978968], [-73.8601941090356, 40.65887320688011], [-73.86019480208077, 40.65887742522407], [-73.86019641489253, 40.65888149516746], [-73.86019890172327, 40.65888530126566], [-73.86020219203384, 40.65888873555805], [-73.86020619249393, 40.658891700630384], [-73.86021078963006, 40.65889411237782], [-73.86021585304373, 40.65889590239074], [-73.86022123911003, 40.65889701989502], [-73.86022679505248, 40.658897433192514], [-73.86023236327559, 40.65889713056004], [-73.86029526049886, 40.65887855938868], [-73.8603544076603, 40.65885385483544], [-73.8604087789173, 40.65882344537561], [-73.86041955595196, 40.658820522853624], [-73.86042952324041, 40.658816246999066], [-73.86043839032605, 40.65881074241427], [-73.86044589881381, 40.65880416950846], [-73.8604518298994, 40.658796719822305], [-73.86045601074608, 40.65878861044688], [-73.86045831952022, 40.65878007769706], [-73.86045868894304, 40.658771370225125], [-73.86045710824996, 40.658762741774886], [-73.86045362350445, 40.65875444378722], [-73.86042536587483, 40.658711918528546], [-73.86040490009731, 40.65866687883854], [-73.86039260036225, 40.658620148243934], [-73.8603943204936, 40.65861234068068], [-73.86039762697337, 40.65860483511534], [-73.86040243799279, 40.658597817247745], [-73.86040863451935, 40.658591460711435], [-73.8604160632406, 40.65858592277726], [-73.8604245403572, 40.65858134046279], [-73.86043385613202, 40.6585778271418], [-73.8604437800774, 40.65857546973935], [-73.86045469169552, 40.658576440331096], [-73.86046566831932, 40.65857609849028], [-73.86047643943894, 40.65857445264147], [-73.86048673960897, 40.6585715433451], [-73.86049631499077, 40.65856744229853], [-73.86050492960698, 40.65856225056831], [-73.86051237115804, 40.658556096100504], [-73.86051845625364, 40.65854913056663], [-73.86052303493194, 40.65854152562638], [-73.86051785422997, 40.65853995847909], [-73.86051308166277, 40.65853776487685], [-73.86050884674245, 40.658535004346554], [-73.86050526439084, 40.658531751800155], [-73.86050243182072, 40.658528095500934], [-73.8605004258989, 40.658524134668866], [-73.86049930105918, 40.658519976787744], [-73.86049908782564, 40.658515734688855], [-73.8604997919849, 40.65851152348857], [-73.86050139442817, 40.658507457465085], [-73.86051208590054, 40.658497672301536], [-73.86052466844858, 40.65848929233575], [-73.86053882769906, 40.65848252693921], [-73.86055420988649, 40.658477545143434], [-73.86057043069079, 40.65847447141754], [-73.86057677057049, 40.65847834256678], [-73.8605821667946, 40.658482969523305], [-73.86058646610162, 40.65848822087385], [-73.86058954638308, 40.65849394747118], [-73.86059630759262, 40.658497080093504], [-73.86060367023326, 40.65849929071967], [-73.86061141948765, 40.6585005148507], [-73.86061932925698, 40.65850071677058], [-73.86062716875976, 40.658499890587755], [-73.86063470926462, 40.65849806040769], [-73.860641730764, 40.65849527962917], [-73.86064802839297, 40.65849162938626], [-73.86065341840745, 40.65848721618129], [-73.86071997976644, 40.65841629578989], [-73.86082324998215, 40.65838623488364], [-73.86092445810512, 40.65835234987692], [-73.8609407764822, 40.65835672306757], [-73.86095588562512, 40.658363139073494], [-73.86096934842652, 40.65837141227997], [-73.86098077540692, 40.65838130334327], [-73.86098983598349, 40.65839252611519], [-73.86099626803218, 40.65840475592056], [-73.8609998854719, 40.65841763895132], [-73.86100058364913, 40.658430802500874], [-73.86099834236325, 40.65844386574699], [-73.86099352033254, 40.65848373292757], [-73.86098115499654, 40.65852264566346], [-73.8609615079497, 40.658559780695214], [-73.86093499484525, 40.6585943523723], [-73.86090217660049, 40.65862562927499], [-73.8608910415375, 40.65863846316638], [-73.86088284859116, 40.658652523323696], [-73.86087782365375, 40.65866742208985], [-73.86087732297413, 40.65868134176507], [-73.86087372289653, 40.658694994099044], [-73.86086712818317, 40.658707981802635], [-73.86085773074154, 40.658719926927574], [-73.86084580404027, 40.65873048186486], [-73.86083169515129, 40.65873933946055], [-73.86077245596431, 40.65875445177121], [-73.86071626304346, 40.65877524396901], [-73.86066405086045, 40.65880137028717], [-73.86061668769061, 40.65883239625404], [-73.86057496117598, 40.6588678059176], [-73.86053956522544, 40.658907010425516], [-73.86051108847565, 40.65894935781752], [-73.86049000450191, 40.658994143866344], [-73.86047666394151, 40.65904062378934], [-73.86047128866177, 40.65908802463314], [-73.86047396806993, 40.65913555812848], [-73.86048465762494, 40.659182433798016], [-73.86053093212242, 40.659228474545245], [-73.86057345610458, 40.659276567127264], [-73.86061207311211, 40.65932653460514], [-73.86064944052751, 40.65932152674169], [-73.86068516151484, 40.65931178683697], [-73.86071821072426, 40.659297594469116], [-73.86074763950079, 40.6592793570216], [-73.86077260311248, 40.65925759798893], [-73.86079238499961, 40.65923294195022], [-73.86080641734287, 40.65920609664029], [-73.86081402183073, 40.65916870217401], [-73.86081724695877, 40.6591309417317], [-73.86081606738044, 40.65909311216202], [-73.86081668201687, 40.65907383103328], [-73.86082105398334, 40.65905483377392], [-73.86082908610129, 40.65903654263418], [-73.86084059983905, 40.65901936416935], [-73.86085533927911, 40.659003680203085], [-73.86087297680847, 40.658989839340656], [-73.86094803804941, 40.65895065725018], [-73.86102487572501, 40.65891352586272], [-73.86116835978298, 40.6588088220647], [-73.86121224376127, 40.658766656119575], [-73.86124943398261, 40.65872091007865], [-73.86127943196229, 40.65867219712342], [-73.86130183562334, 40.658621170202494], [-73.86131634468514, 40.65856851327989], [-73.86132276468814, 40.65851493216581], [-73.86131463955402, 40.65848898311093], [-73.86130941531205, 40.65846260485655], [-73.86130281819504, 40.658430417916534], [-73.86130016093611, 40.65839790415194], [-73.8612988615183, 40.6583528596514], [-73.86128968762509, 40.658308349239846], [-73.8612728030787, 40.658265167684704], [-73.86124850937952, 40.658224086024504], [-73.86113319136574, 40.65815924192561], [-73.86101981543375, 40.65809244126645], [-73.86100193454507, 40.65807938835124], [-73.86098144498347, 40.65806878993202], [-73.860958920957, 40.6580609430229], [-73.86093499368765, 40.65805606752872], [-73.86090375083607, 40.65805664022474], [-73.86081572030625, 40.658061544025244], [-73.86072753647886, 40.65806443903921], [-73.86066308604214, 40.658068945629424], [-73.86059952811799, 40.65807824993888], [-73.86053747698874, 40.658092262041954], [-73.86050949236048, 40.65813359870286], [-73.86047483240216, 40.6581718896318], [-73.86043405988346, 40.65820651309113], [-73.86038783682788, 40.65823690689206], [-73.86020789424994, 40.65829470976332], [-73.8601783281795, 40.65830916491898], [-73.86014617638143, 40.658319953733056], [-73.86011221667275, 40.65832681520192], [-73.86007727060799, 40.658329583332666], [-73.86004218360436, 40.65832819115835], [-73.85989401628483, 40.658309592716414], [-73.85983524905286, 40.658297272790186], [-73.85982492317088, 40.65829897492685], [-73.85981436865637, 40.658299333503514], [-73.85980388476865, 40.65829833835319], [-73.8597937687644, 40.658296017692074], [-73.8597843074698, 40.65829243731947], [-73.85977576914713, 40.658287698752105], [-73.85976839588899, 40.65828193634574], [-73.8597748765548, 40.65826388696597], [-73.8597777544119, 40.65824530378508], [-73.85977727198093, 40.65824458373749], [-73.85977694750098, 40.65824381407119], [-73.8597767893799, 40.658243014728804], [-73.85977680171472, 40.65824220642171], [-73.85977698418533, 40.65824141009362], [-73.85977733206455, 40.65824064637783], [-73.8597778363381, 40.65823993506261], [-73.85977848394018, 40.658239294578664], [-73.85977925809077, 40.65823874152121], [-73.85978013873165, 40.658238290220254], [-73.85978110304457, 40.65823795236933], [-73.85978327686513, 40.658240310606224], [-73.85978595149133, 40.65824234851229], [-73.85978904700261, 40.65824400519239], [-73.85979247090133, 40.65824523114304], [-73.85979612087758, 40.65824598973154], [-73.85979988786572, 40.65824625829025], [-73.85980365930405, 40.658246028794395], [-73.85980732249737, 40.65824530810159], [-73.85981076798527, 40.658244117746854], [-73.85981389281268, 40.65824249329964], [-73.85981660360636, 40.658240483299906], [-73.85981881936422, 40.65823814780911], [-73.85982047387745, 40.65823555661405], [-73.85982151770705, 40.65823278714298], [-73.85983888003487, 40.65821144559328], [-73.85995685463872, 40.658130104356246], [-73.85997204122235, 40.65812936285185], [-73.85998683717858, 40.65812665317485], [-73.8600008060138, 40.65812205526313], [-73.86001353563562, 40.658115704759005], [-73.86002465050994, 40.658107789007914], [-73.86003382273883, 40.65809854153106], [-73.86004078173501, 40.65808823513673], [-73.86004532220305, 40.658077173872094], [-73.86004731019706, 40.658065684053284], [-73.86004364414178, 40.658063991304914], [-73.86004040323813, 40.65806185390814], [-73.86003767946562, 40.658059332524516], [-73.86003555012763, 40.65805649871297], [-73.8600340756564, 40.65805343289953], [-73.86003329789838, 40.65805022209471], [-73.8600332389273, 40.65804695742388], [-73.86003390041654, 40.658043731541326], [-73.86003526359225, 40.65804063600036], [-73.8600372897666, 40.65803775865489], [-73.86003992143455, 40.658035181166596], [-73.86004308390716, 40.65803297668691], [-73.86004668743104, 40.658031207780475], [-73.86005062973462, 40.65802992465072], [-73.860183216439, 40.658000250104614], [-73.86019595405857, 40.65799055133859], [-73.86020644859818, 40.65797940284286], [-73.86021442008946, 40.657967102032565], [-73.86021965587379, 40.65795397706383], [-73.86022201627462, 40.65794037807924], [-73.86021816254974, 40.65793403249064], [-73.86021545296794, 40.657927352545244], [-73.86021393581149, 40.65792045727575], [-73.8602136381153, 40.657913469551566], [-73.86024198953398, 40.657901889098895], [-73.86027223300464, 40.65789353392311], [-73.86030373983284, 40.65788857770955], [-73.86033585506264, 40.657887123486745], [-73.86036791109113, 40.65788920148472], [-73.8603838782098, 40.65787869881579], [-73.86039799754259, 40.65786675945564], [-73.86041004790772, 40.657853570437744], [-73.86044971765398, 40.657842199173004], [-73.86048734107469, 40.65784589781524], [-73.8605252694779, 40.65784533533559], [-73.86056267529403, 40.65784052400714], [-73.86056480806081, 40.65784066145086], [-73.8605669404149, 40.657840520341445], [-73.8605690100849, 40.65784010479974], [-73.86057095663061, 40.65783942696082], [-73.86057272320707, 40.6578385066195], [-73.86057425822476, 40.657837370652636], [-73.86057551685668, 40.65783605223369], [-73.86057646234767, 40.65783458986454], [-73.8605770670858, 40.65783302625066], [-73.8605773134111, 40.65783140705412], [-73.86057719413046, 40.657829779560366], [-73.8605767127272, 40.65782819129701], [-73.86057588325953, 40.65782668864608], [-73.86057472995071, 40.65782531548933], [-73.86052898667043, 40.657815879958825], [-73.86048529822187, 40.65780189396694], [-73.86044444123876, 40.65778360613783], [-73.8604071420188, 40.65776134156808], [-73.86040537631257, 40.65760582054201], [-73.8603868461818, 40.65755532113634], [-73.86035941846357, 40.657507227160686], [-73.86032360895182, 40.657462443016705], [-73.86031029516808, 40.657418070541944], [-73.8603163571442, 40.65738435445212], [-73.86034164704868, 40.657247905698014], [-73.86035030672473, 40.65722606716767], [-73.86037373148025, 40.65720987283221], [-73.86039455362588, 40.65721291443623], [-73.86041574418222, 40.657213438364245], [-73.8604367815357, 40.65721143171957], [-73.86045714784515, 40.657206943896405], [-73.8604763417868, 40.657200085364195], [-73.86053263950032, 40.65718076941999], [-73.86058792205708, 40.657159821871936], [-73.86060930392513, 40.65715545670058], [-73.86061974209726, 40.657161699393086], [-73.86061013643769, 40.65720541164122], [-73.86062372337487, 40.65720545608297], [-73.8606580646268, 40.65717426203342], [-73.86066287940999, 40.657157439737254], [-73.86066429380234, 40.6571402561819], [-73.86066227518654, 40.65712310766844], [-73.86066039598722, 40.657120401401976], [-73.86065916030824, 40.65711748825223], [-73.86065860545966, 40.65711445617963], [-73.86065874819455, 40.65711139673541], [-73.8606595842032, 40.65710840229702], [-73.86066108824288, 40.657105563279615], [-73.86066321489993, 40.65710296540492], [-73.86066589996193, 40.657100687113875], [-73.86066906235502, 40.6570987971978], [-73.86067260659318, 40.657097352721244], [-73.86067642566083, 40.657096397299135], [-73.86068040424409, 40.657095959779575], [-73.8606844222127, 40.65709605337329], [-73.86070268132822, 40.65709518094141], [-73.86072054730728, 40.65709218126326], [-73.86073759543031, 40.657087125648566], [-73.86075342042025, 40.65708013428231], [-73.86084264927302, 40.656931296013134], [-73.8608442617313, 40.65692686614719], [-73.86084678228117, 40.65692268919323], [-73.86085014624553, 40.65691887233131], [-73.86085426730544, 40.65691551350191], [-73.86085903971427, 40.656912698892185], [-73.86086434101308, 40.656910500724614], [-73.86087003517075, 40.65690897540414], [-73.86087597607579, 40.65690816207006], [-73.86088201128541, 40.65690808159261], [-73.86088798593681, 40.65690873603678], [-73.86127520118559, 40.65700090635928], [-73.86149375847893, 40.65699993405751], [-73.86161371163614, 40.65699651025286], [-73.86162917655315, 40.657047859102434], [-73.86163823380747, 40.657100087800636], [-73.86164080314583, 40.65715273365753], [-73.86163686179145, 40.65720533028669], [-73.86162644464528, 40.65725741173658], [-73.861617028336, 40.65740064660212], [-73.86161656428084, 40.65754406042941], [-73.86162505358661, 40.65768732889235], [-73.86164247711287, 40.6578301279915], [-73.86166879551332, 40.65797213478675], [-73.86166935768706, 40.657982341840516], [-73.86167213666835, 40.657992336254836], [-73.86167705536486, 40.65800184076984], [-73.8616839773257, 40.65801059171595], [-73.86169271052574, 40.65801834632875], [-73.86170301269323, 40.658024889483436], [-73.86171459803093, 40.65803003966288], [-73.86172714514397, 40.65803365399324], [-73.86269762139094, 40.657618376333446], [-73.86269778299766, 40.65761830900614], [-73.86317083299996, 40.65827651199994], [-73.86174758999981, 40.65893835699988], [-73.86122244999984, 40.659220936999894], [-73.8603597819999, 40.65964582899991], [-73.859386918, 40.6597727149999], [-73.85851436399994, 40.65989894199987], [-73.85761052799985, 40.660108355999924], [-73.85792219800001, 40.6613188559999], [-73.8580250429998, 40.66172104999989], [-73.85811779299999, 40.66203657099987], [-73.85842950899992, 40.66345335999987], [-73.85753563299991, 40.66359786299995], [-73.85659075099987, 40.66374715999991], [-73.85611710199991, 40.663808401999844], [-73.85568461199988, 40.66386749199993], [-73.85610973799986, 40.66550601099989], [-73.85646311899993, 40.66704176299991], [-73.85684676199999, 40.668602934999925], [-73.85722693699994, 40.670096211999876], [-73.85724675799997, 40.670174050999925], [-73.85726744599997, 40.670255293999965], [-73.8572848669999, 40.6703219889999], [-73.85730947799988, 40.670416232999855], [-73.85763323099987, 40.67165619399989], [-73.85855480399997, 40.67153293399986], [-73.85947416499987, 40.671380531999944], [-73.86038937900003, 40.67126877499985], [-73.86077512799977, 40.67280562199989], [-73.86107192599995, 40.674019813999905], [-73.86106294999989, 40.67434293499984], [-73.86131863599985, 40.6750212689999], [-73.86029565599982, 40.67477189399992], [-73.85959262199998, 40.674556409999894], [-73.85928317999998, 40.674452715999934], [-73.85825153699983, 40.6741356109999], [-73.85711349099991, 40.673575054999944], [-73.85648739299987, 40.6733767739999], [-73.85283803899996, 40.6722791889999], [-73.85163460999999, 40.671930332999835], [-73.85096453700001, 40.671719510999885]]], [[[-73.83032725337058, 40.65513280580328], [-73.8303778971816, 40.65512833511791], [-73.83041325059034, 40.65515782233629], [-73.83041203905732, 40.65521585576173], [-73.83040011516177, 40.65525714450149], [-73.83035808100335, 40.65532208288756], [-73.83033058333683, 40.65536987237486], [-73.83026601851493, 40.65542864678473], [-73.83019020357145, 40.655478819385834], [-73.83012570678862, 40.65551061084527], [-73.83000321318903, 40.65555212870698], [-73.82992594068641, 40.65554220350465], [-73.82988919856028, 40.655522822584075], [-73.82994556435834, 40.655424494353], [-73.83004306625594, 40.65534942254124], [-73.83032725337058, 40.65513280580328]]], [[[-73.85975533686633, 40.65410417609493], [-73.86042178299996, 40.65461220199988], [-73.86048308199986, 40.65465892799988], [-73.86052767799987, 40.6547059619999], [-73.86056716699993, 40.654747451999874], [-73.86059601899994, 40.65477969699992], [-73.86098917376947, 40.65520976705256], [-73.86094500963966, 40.65522068001973], [-73.86089920007385, 40.655226442960796], [-73.86085277407571, 40.65522692644851], [-73.86080677472893, 40.65522211962026], [-73.86076223553128, 40.65521213047425], [-73.86072015717329, 40.65519718344291], [-73.86045463170862, 40.655067107366136], [-73.86030402711816, 40.65492500272203], [-73.86025748919218, 40.65488405050393], [-73.8602513031107, 40.65487860669235], [-73.86024486376577, 40.65487225456878], [-73.86021221902251, 40.65484005263404], [-73.86015870367471, 40.654793663305895], [-73.85979242750741, 40.65447615737696], [-73.85975533686633, 40.65410417609493]]], [[[-73.85898002989568, 40.65267545791771], [-73.85975533686633, 40.65410417609493], [-73.85948691913471, 40.65393234893363], [-73.85941022942576, 40.65386473792502], [-73.85940989064696, 40.65385786181041], [-73.8594447402705, 40.653637548104285], [-73.85934240802169, 40.65356106691007], [-73.85924955873766, 40.65325429542834], [-73.8591261328236, 40.653172994731754], [-73.85904529229784, 40.6531079760839], [-73.85900715381493, 40.65300405837214], [-73.85898002989568, 40.65267545791771]]], [[[-73.85898002989568, 40.65267545791771], [-73.85868859934205, 40.65248755609561], [-73.85854822276487, 40.65236079018085], [-73.85837385611718, 40.652185292449964], [-73.85805076446096, 40.65180186819084], [-73.85793978941108, 40.651675741576426], [-73.85783392607334, 40.65155815255726], [-73.85774883373948, 40.65148988169545], [-73.85752745156287, 40.651379243112665], [-73.85731941515513, 40.65123709900783], [-73.85730224535344, 40.65106410457844], [-73.85718757766341, 40.65087569460869], [-73.85702601785158, 40.65069371930442], [-73.85695870295893, 40.65058737348713], [-73.85898002989568, 40.65267545791771]]]]}}, {\"id\": \"124\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 125, \"Shape_Leng\": 0.0497322121757, \"Shape_Area\": 7.64293209892e-05, \"zone\": \"Hudson Sq\", \"LocationID\": 125, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00269315099992, 40.72831405499989], [-74.00214891499996, 40.728035366999904], [-74.00221100999997, 40.727961316999874], [-74.00251324499995, 40.72760092699987], [-74.00312206399995, 40.72687712799991], [-74.003730175, 40.726250058999945], [-74.00403067899997, 40.7255124959999], [-74.00427457099987, 40.72491210899993], [-74.004574744, 40.72417312999992], [-74.00478574899996, 40.72362483799989], [-74.00505887099996, 40.722977863999894], [-74.00536435999994, 40.722183951999966], [-74.00538616799996, 40.722112360999894], [-74.00539186699997, 40.72203736699992], [-74.00538048599999, 40.721962173999934], [-74.00535249999994, 40.72189013499991], [-74.0053098609999, 40.721824191999865], [-74.00591289899992, 40.7222529489999], [-74.00629841699995, 40.72254950399988], [-74.00762904199998, 40.72349770399995], [-74.00791205999997, 40.72369976699988], [-74.00807835299996, 40.72382686299987], [-74.00830216199988, 40.723977808999905], [-74.00857090899997, 40.72416935499997], [-74.00864380599998, 40.72422131699989], [-74.00931760399993, 40.7247015519999], [-74.010040787, 40.72520616399988], [-74.0108123829999, 40.72578980299994], [-74.01092277299992, 40.72579415099991], [-74.01102457999994, 40.72579386199991], [-74.01103681999997, 40.72572537699997], [-74.011176661, 40.7251694249999], [-74.01135428299995, 40.72427718299988], [-74.01143933, 40.72428446899995], [-74.01159122599994, 40.724297603999936], [-74.0119643202455, 40.72432986461987], [-74.01178072408688, 40.72523995904116], [-74.0116541510404, 40.72586737457335], [-74.01165326569942, 40.725871761851344], [-74.0119583307484, 40.72597759972094], [-74.01231588417461, 40.72603384328542], [-74.01411595444266, 40.72621586380551], [-74.01477821548454, 40.72628170647329], [-74.01516324118418, 40.72631971193679], [-74.01520426531133, 40.726361949379644], [-74.01518864981979, 40.72645118250504], [-74.01517084019778, 40.72657939015507], [-74.01514325523299, 40.726710526279305], [-74.01512356549452, 40.72679394256457], [-74.01430671596631, 40.72670812119669], [-74.01341677520719, 40.72662103121345], [-74.01234454766038, 40.7265194304825], [-74.011542018976, 40.72644734692706], [-74.01138323602366, 40.728229272183555], [-74.01100472899994, 40.7281932389999], [-74.01087083299997, 40.72818049399996], [-74.01076791499999, 40.72817395799991], [-74.01067264999996, 40.72911266599987], [-74.01054441199987, 40.7291000979999], [-74.00934961999998, 40.728987555999865], [-74.00859272099993, 40.728915277999924], [-74.0070902079999, 40.72877225299993], [-74.00531369199997, 40.72860449399992], [-74.00312767499993, 40.72839380099989], [-74.00281726599992, 40.72836312899993], [-74.00269315099992, 40.72831405499989]]]}}, {\"id\": \"125\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 126, \"Shape_Leng\": 0.12994872606, \"Shape_Area\": 0.0007032738273, \"zone\": \"Hunts Point\", \"LocationID\": 126, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88615037899993, 40.82730567799994], [-73.88635032500002, 40.826012622999926], [-73.88647552799978, 40.825285423999915], [-73.88710697199997, 40.823305870999874], [-73.88706248999988, 40.823190274999945], [-73.88753429599987, 40.82250933899995], [-73.88719405099995, 40.82259352799986], [-73.88709505500002, 40.82260272399986], [-73.88698003699992, 40.82261725799987], [-73.8865801009999, 40.82266811799986], [-73.88508162399985, 40.822868444999834], [-73.88456358400003, 40.82293706699989], [-73.8843985489999, 40.822965575999866], [-73.88436128599993, 40.82287380099997], [-73.88433212399998, 40.82281014299991], [-73.88429488199988, 40.82271820499992], [-73.88423986499997, 40.82262562999994], [-73.88378786100002, 40.82193341399996], [-73.88244779599998, 40.81926832299992], [-73.8809802929999, 40.818237845999924], [-73.8780628329999, 40.81618263199993], [-73.87663459981849, 40.81571150804957], [-73.87673185506821, 40.81538649648655], [-73.87642408558143, 40.815284123784465], [-73.87520539092391, 40.81487874500281], [-73.87446381927738, 40.81464523186163], [-73.87250110885978, 40.813888128234055], [-73.87239437900097, 40.81397433560582], [-73.87232977098004, 40.81392182334129], [-73.8724366723506, 40.81385093400157], [-73.87218961012046, 40.813673670767166], [-73.87209716960993, 40.813738021521004], [-73.87203687539917, 40.8136855133727], [-73.87212931147367, 40.813621168692585], [-73.87208038785018, 40.81358405461941], [-73.87180463640202, 40.81331489053922], [-73.87165777683394, 40.81325426197599], [-73.87095753684095, 40.812146180609446], [-73.87056806927927, 40.81208731762241], [-73.87047093165313, 40.81145829499172], [-73.87055342191628, 40.811243380722885], [-73.86811219140058, 40.806751832831786], [-73.8684634695915, 40.805936548939], [-73.86978968017556, 40.804125105863825], [-73.8715185503029, 40.801874148888096], [-73.87161142592561, 40.80191439620694], [-73.87169881930807, 40.80180527192038], [-73.87160336192525, 40.801751811446536], [-73.8717450937391, 40.80154736611613], [-73.87238060453703, 40.80129011463271], [-73.87246949337055, 40.80078919284294], [-73.87223657260759, 40.800761622649624], [-73.87223731051014, 40.80075769467135], [-73.872345653064, 40.8001811333312], [-73.87234684998835, 40.800174763010446], [-73.87297065553919, 40.80024973054026], [-73.87297862944645, 40.80025068902641], [-73.8728771106369, 40.800840089046176], [-73.87287336804405, 40.80083936740926], [-73.8726270581177, 40.80079186150162], [-73.8725988302891, 40.800891858140254], [-73.87258566151068, 40.8009611639185], [-73.87257566696651, 40.80101380496975], [-73.8725854704256, 40.80106099335556], [-73.87261058260187, 40.80110410183526], [-73.87266088531166, 40.80114791108709], [-73.87270503758596, 40.80116717681056], [-73.87317223327184, 40.80099783970593], [-73.87355096572502, 40.80085330277184], [-73.87467716313955, 40.80097942336672], [-73.87473060452689, 40.80125663762865], [-73.87526517037327, 40.80139815254847], [-73.87585788581308, 40.80145598758635], [-73.87592998071509, 40.801084223593705], [-73.87814218819929, 40.801319290306495], [-73.8780768548655, 40.80166664403499], [-73.87798629509406, 40.801656715506894], [-73.87796880375369, 40.801711115128164], [-73.8780330023647, 40.80205990686082], [-73.87806294853425, 40.802194319344984], [-73.8779064649521, 40.80227180102014], [-73.87811438894289, 40.80236808487932], [-73.87816581140154, 40.80255168374491], [-73.87823464921779, 40.80264680576819], [-73.87843295513449, 40.80270273469493], [-73.87874344166545, 40.80273911424301], [-73.8789245383548, 40.80277208050827], [-73.87921348407693, 40.802795326835025], [-73.87934292184536, 40.80277251917869], [-73.87950995961023, 40.80276949081517], [-73.87962764836531, 40.80274331871939], [-73.8797430082038, 40.80274343912096], [-73.87975704576816, 40.80274345376413], [-73.87991660603436, 40.80276000771109], [-73.88005462984884, 40.80276015140756], [-73.88017115118777, 40.80272421955093], [-73.88027898796521, 40.80272105380306], [-73.88041836510055, 40.80266794028656], [-73.88056943638324, 40.802632847479146], [-73.88077149808389, 40.802587427211876], [-73.88092030545262, 40.8025585434273], [-73.88106910311686, 40.802535882321486], [-73.88117965667224, 40.802529773446125], [-73.88132025761237, 40.80251125198685], [-73.88143082957599, 40.80249477276176], [-73.8815332076576, 40.80248139645061], [-73.88170931957762, 40.802447354975726], [-73.88182125479855, 40.802431914256914], [-73.88193863933247, 40.80242166390699], [-73.88203490143374, 40.80240779338885], [-73.8820601312739, 40.802404158282236], [-73.88206539258177, 40.80240288628746], [-73.88219256706495, 40.8023721458367], [-73.8823208879866, 40.802351534972935], [-73.88237010115716, 40.80235384695559], [-73.88239465208328, 40.802354999950545], [-73.88244775081927, 40.802351212093384], [-73.88243021774045, 40.80226557239003], [-73.88251894912045, 40.80225218128442], [-73.88254113402193, 40.80233568156294], [-73.88259526944276, 40.80232570632336], [-73.88276028400804, 40.8023062088511], [-73.88288323972418, 40.802289127183336], [-73.88297059068802, 40.80228429903961], [-73.88307091642712, 40.80225982008288], [-73.88320360331396, 40.80222799770891], [-73.88354011849329, 40.802179174631775], [-73.88369542422481, 40.80216212360288], [-73.88380941028178, 40.802122195736544], [-73.88383849993177, 40.80210718399523], [-73.88385182690112, 40.80209553165792], [-73.8838774507143, 40.802075334868626], [-73.8838999931241, 40.802061357851024], [-73.88391536772113, 40.80204892757249], [-73.88393791667498, 40.80203106129535], [-73.88396462519694, 40.80196514835965], [-73.88378353970126, 40.80192901592477], [-73.88377737963155, 40.80193989916992], [-73.8837384573708, 40.801956193769755], [-73.88369036996542, 40.80194525580517], [-73.88366893088867, 40.80191334596797], [-73.88368024716245, 40.801880690192576], [-73.88372326665917, 40.80186284394103], [-73.88377442409538, 40.80187378498754], [-73.88379279040652, 40.80190724801428], [-73.88378457957894, 40.801919683823925], [-73.88396774707935, 40.801957444852604], [-73.8839894532262, 40.8019038796612], [-73.8850517233671, 40.80213090679797], [-73.8851457697332, 40.802259589487285], [-73.88526714505164, 40.80228439531952], [-73.88527262213964, 40.80227402896786], [-73.88531087058676, 40.802253326772586], [-73.88536544598874, 40.80226063974465], [-73.88539676408833, 40.80230215307005], [-73.88538033197243, 40.80233428576994], [-73.88533116885945, 40.80235290267008], [-73.88528342105936, 40.802342485538915], [-73.88525755169408, 40.8023082374665], [-73.88526166505221, 40.80229683344886], [-73.88515517126592, 40.80227245404643], [-73.8852555367023, 40.80240978102536], [-73.88559978112261, 40.802565497333326], [-73.88578073880988, 40.802680392136644], [-73.88591424672595, 40.80279851699306], [-73.88600887424614, 40.80295265729101], [-73.88607771246615, 40.803054329694945], [-73.88609972664483, 40.80308537287127], [-73.88616377056557, 40.80317568497681], [-73.88627576454361, 40.80326428936847], [-73.8864243555294, 40.803345162755825], [-73.88648691823535, 40.80337921283246], [-73.88672389248723, 40.80353021498235], [-73.88684879825252, 40.803635219116686], [-73.8870857353168, 40.80380916368018], [-73.88727097347115, 40.803947002336685], [-73.88738095624211, 40.80411068128396], [-73.8872473845825, 40.80418614111941], [-73.88733552633306, 40.80429318940804], [-73.88740234683179, 40.80437434258937], [-73.88752886022138, 40.80440172674034], [-73.88771124600427, 40.80461478122007], [-73.88763129537637, 40.804671325375985], [-73.88795897223945, 40.8049979259206], [-73.88859217706313, 40.80478927560033], [-73.88882576006077, 40.80485322537337], [-73.88907297129742, 40.805013860005374], [-73.88927892586686, 40.80518182876005], [-73.88938265718072, 40.80531020813596], [-73.88944105902301, 40.80544009355648], [-73.88940685507217, 40.80558202078195], [-73.88922416091161, 40.80584535119195], [-73.88946981686438, 40.80553923652497], [-73.88947004066108, 40.80553895898339], [-73.88977893617728, 40.80515563359968], [-73.88979493978445, 40.80513577207837], [-73.889995124756, 40.804887350890446], [-73.88999598170764, 40.804886286752776], [-73.890150985016, 40.80495818226255], [-73.8893743925369, 40.80592527311521], [-73.88954478026055, 40.80580707619275], [-73.88965874963506, 40.80584774526955], [-73.88975332987161, 40.80587549108881], [-73.88981346825864, 40.80587739135758], [-73.88981398349773, 40.805877391850046], [-73.88985041274157, 40.80585714644435], [-73.88987229258508, 40.805831356464886], [-73.88989905762257, 40.805785290927766], [-73.88992823364853, 40.80574844674448], [-73.88996089069998, 40.805714769917444], [-73.88999139115282, 40.80570426037567], [-73.89005932692345, 40.80570432518393], [-73.89013594081199, 40.8057146909474], [-73.89023295689198, 40.805736907528235], [-73.89033596715151, 40.80576663651646], [-73.89035259769872, 40.8057714358389], [-73.89043667322538, 40.80579363876647], [-73.89051422588408, 40.80584779297371], [-73.89058213079707, 40.80586752284705], [-73.89072122618852, 40.8058750287227], [-73.89086354177184, 40.80589237135277], [-73.89101234323621, 40.80589988606528], [-73.89127112984032, 40.80591242189428], [-73.89149717186729, 40.80592719896933], [-73.89149903729991, 40.80593123874312], [-73.89144104277274, 40.80566189856482], [-73.89178618135598, 40.80562289208621], [-73.89197929534278, 40.80638470897619], [-73.89245494636062, 40.806370106605264], [-73.89270641330583, 40.806343990441896], [-73.89270681066346, 40.806343949098085], [-73.89253177139425, 40.805656551568276], [-73.89227315843527, 40.80570243312479], [-73.89216364547643, 40.80530259256541], [-73.8918131200952, 40.80518695546602], [-73.89180973582788, 40.8051937849576], [-73.89167604184013, 40.805150953142494], [-73.891686993125, 40.80513011010927], [-73.89156417608577, 40.80509393995633], [-73.89156004966382, 40.805099981040684], [-73.89149997938128, 40.80508158808914], [-73.89150751395869, 40.80507031104261], [-73.89130372999014, 40.80500678832565], [-73.89129856973848, 40.805016497722505], [-73.89126710622891, 40.805010489424156], [-73.89128402413917, 40.80497628189916], [-73.89131376917373, 40.80498493831311], [-73.89131118790088, 40.80498982140615], [-73.89146488860796, 40.80504398744601], [-73.89150645665745, 40.80505769308249], [-73.89169561834646, 40.80511584342212], [-73.89173240995615, 40.80504936342029], [-73.89186610809385, 40.80509048742664], [-73.89182888997564, 40.805166472094164], [-73.89219739671567, 40.80528468791936], [-73.89230020327585, 40.80566060567011], [-73.89233730507779, 40.805656370118335], [-73.89233507004711, 40.805648679479724], [-73.8923980350899, 40.805638488931876], [-73.89240027239099, 40.80564532485975], [-73.89256667738418, 40.80562071000472], [-73.8927481403741, 40.80634026130199], [-73.89289770901877, 40.80632691879631], [-73.89304654394503, 40.80631476551206], [-73.89315654170535, 40.80631240992103], [-73.89330516489227, 40.80634399783534], [-73.8933765511915, 40.806297864564264], [-73.89345420126807, 40.80629302000554], [-73.89354804655919, 40.80627589888845], [-73.89362245442146, 40.80627596757254], [-73.89371954313471, 40.806253934131405], [-73.89383602227304, 40.80624420825316], [-73.89395248091247, 40.80624677353253], [-73.89403658607417, 40.806251767219436], [-73.89410449503039, 40.8062690374305], [-73.8942209637422, 40.80626668559508], [-73.89452510142179, 40.80624238250855], [-73.89462863918334, 40.80623264499358], [-73.8946325235109, 40.80623233192205], [-73.8945739391341, 40.80604577300727], [-73.89472601909603, 40.8060262465835], [-73.89477679720956, 40.806213634245196], [-73.89480337371349, 40.80620822282261], [-73.89485360535866, 40.80621490815875], [-73.89472970526674, 40.80574110146166], [-73.89484295983623, 40.80572399701221], [-73.89496909528714, 40.80623113049386], [-73.89509836487399, 40.80686363917353], [-73.89528550564228, 40.80684271107136], [-73.89537269057816, 40.80670351733481], [-73.89545911867938, 40.80660308388955], [-73.89550633968572, 40.806595305303645], [-73.89541794877962, 40.80626777768829], [-73.89539221099777, 40.80617680177723], [-73.8953020614291, 40.80590140416171], [-73.89531181635847, 40.80586945615722], [-73.8952796287401, 40.805766184067906], [-73.89531197487513, 40.80576867103503], [-73.89534416707072, 40.80586948553051], [-73.8954054906102, 40.80596049374926], [-73.89546655698209, 40.806216200015356], [-73.8955116391718, 40.80634898300486], [-73.89559109091819, 40.8065813433475], [-73.89572273778744, 40.80655965730823], [-73.89569912530925, 40.80644502168963], [-73.89567025138753, 40.80629013045689], [-73.89574790898853, 40.80628036801016], [-73.89583174887831, 40.806454974302504], [-73.89583838073091, 40.80654060757189], [-73.89597794687755, 40.80651761583953], [-73.89594184174582, 40.80639116118041], [-73.89593553363193, 40.80628791146807], [-73.8958776634502, 40.80605679059504], [-73.89579066958078, 40.80583055920915], [-73.89587481599304, 40.8058085115824], [-73.895948888355, 40.80602244045197], [-73.89601320589148, 40.8062683161559], [-73.89610824517877, 40.80649615095125], [-73.89622424119065, 40.8064770418422], [-73.89620708243055, 40.80641584533903], [-73.89619572352184, 40.80632406352671], [-73.89619585301467, 40.8062410313225], [-73.89623040737013, 40.80621047190215], [-73.8962764250295, 40.80620614259313], [-73.89635118493798, 40.806210580554584], [-73.89678682323746, 40.80615925851491], [-73.89708748307676, 40.80612383765668], [-73.89711438014326, 40.80611086592895], [-73.89715079498718, 40.806093303628536], [-73.89717387408169, 40.80604525364753], [-73.89726014999152, 40.80604095984719], [-73.8973129614691, 40.80608152729469], [-73.89738639439209, 40.80622461713665], [-73.89749681953876, 40.806214400202535], [-73.89747588735356, 40.805999226821534], [-73.89741472698567, 40.805800059478074], [-73.8974859174091, 40.8057878325167], [-73.89754706596734, 40.805994373750714], [-73.89763818741136, 40.80620132083884], [-73.89776126648744, 40.806189933501514], [-73.89775065693242, 40.80613712932528], [-73.89772827251059, 40.80596749413728], [-73.89779945195382, 40.80596264091157], [-73.89786715835055, 40.80611264967438], [-73.89787360834964, 40.806179539994936], [-73.8980134089615, 40.8061666043618], [-73.89800306596389, 40.80609064751374], [-73.89799037051527, 40.80593085364808], [-73.89787127990651, 40.805534981759635], [-73.89793600296244, 40.8055202900288], [-73.89806803798805, 40.805913715708385], [-73.89816110752683, 40.806138070099855], [-73.89817641121758, 40.80613353615754], [-73.89822049087707, 40.80612047397424], [-73.89826085644889, 40.80605058777354], [-73.89824374203175, 40.805958800410465], [-73.89823813224358, 40.80586702336686], [-73.89819792651059, 40.80583202779013], [-73.8981059864958, 40.80577950527198], [-73.89804856839497, 40.80571827344633], [-73.8980744627235, 40.80566924860035], [-73.89808317604229, 40.805652752263114], [-73.8981151559596, 40.80563396548572], [-73.89831343224124, 40.80551748247637], [-73.89853926244915, 40.80541173173145], [-73.89904988651375, 40.805321475042476], [-73.89919723524343, 40.80529093940124], [-73.899469855831, 40.80523444066137], [-73.89985531773284, 40.80514737360609], [-73.90022352372337, 40.80506029150024], [-73.90049967830153, 40.80499497839508], [-73.90063777921122, 40.80494702693435], [-73.90081615675385, 40.804885999475765], [-73.90097148274116, 40.804855541470076], [-73.90115552085723, 40.80485569941628], [-73.9013222978677, 40.80486021234714], [-73.90152939058154, 40.804825429345016], [-73.9016732158928, 40.80479496040219], [-73.90178827694882, 40.804768838170574], [-73.90191482832148, 40.80475146587337], [-73.9019953447206, 40.80475153435198], [-73.90205261120369, 40.804766689202694], [-73.9020685374805, 40.80476910021592], [-73.90208285109311, 40.8047807796404], [-73.90209715811166, 40.80479634613557], [-73.90210122978193, 40.804811128297494], [-73.90210234487554, 40.804819692275316], [-73.90210325482057, 40.804826685031244], [-73.90211039482898, 40.804843802748856], [-73.90211549422733, 40.80485625165956], [-73.90213183502179, 40.804881154694804], [-73.90214921089174, 40.80489828059668], [-73.90216556590887, 40.80491307267905], [-73.90217680960626, 40.80492397049571], [-73.90218395580109, 40.80493719845514], [-73.90219315158322, 40.80494809570518], [-73.90221054985811, 40.804950444085605], [-73.90222284749719, 40.80494811309474], [-73.90288034499996, 40.80874479299996], [-73.90293929200004, 40.80906270099992], [-73.90298795399984, 40.80936430099996], [-73.90299766999989, 40.809455582999924], [-73.9030087789999, 40.809550072999876], [-73.90302018099997, 40.80962342899996], [-73.90302967699985, 40.80970212099986], [-73.90307568299997, 40.80976898599992], [-73.90307457199998, 40.80988061199988], [-73.90279087799998, 40.81008304699985], [-73.90259424199986, 40.81022335899995], [-73.90090498099991, 40.81142871299987], [-73.89912449199987, 40.81269524099988], [-73.89844723499995, 40.81317650299993], [-73.89815051799985, 40.81336635399987], [-73.89786486599995, 40.81356666999989], [-73.89775779499988, 40.81364891599991], [-73.89759123799989, 40.8137768729999], [-73.89733049099995, 40.81399633599995], [-73.89708341799985, 40.814224345999904], [-73.89685073199996, 40.81446016799987], [-73.89542858799986, 40.81584467199988], [-73.89580071599984, 40.816065648999874], [-73.8962135219999, 40.81631897099989], [-73.8970446669999, 40.8168018599999], [-73.89567610799998, 40.8181293679999], [-73.894930762, 40.81884671999994], [-73.89389462500002, 40.8198602899999], [-73.89290156199988, 40.82081122299989], [-73.89283653100001, 40.820974054999965], [-73.89387380299988, 40.820979334999926], [-73.89490478799985, 40.82098751999995], [-73.8958813589999, 40.82099257499987], [-73.89586019599982, 40.82263104599987], [-73.89488228899991, 40.82313435499987], [-73.89433874699982, 40.82341635299983], [-73.89390164399987, 40.823643120999904], [-73.89320004999998, 40.82400818299992], [-73.89389702299998, 40.82398399799995], [-73.89487088500002, 40.82393727199992], [-73.89483522899992, 40.82624684099993], [-73.89386719099983, 40.82651496799989], [-73.89293226099981, 40.82678703699992], [-73.89191271899998, 40.82679444999991], [-73.89090560099991, 40.82678778499992], [-73.88997942599995, 40.82680999199989], [-73.88983599199996, 40.826804733999936], [-73.88980564199993, 40.82680362199993], [-73.88969004999984, 40.826799383999834], [-73.88963895299987, 40.82679750999989], [-73.88885046199992, 40.82676859499989], [-73.88789510999993, 40.82674318499987], [-73.88639957199996, 40.82750337899987], [-73.88609624199987, 40.82765577699989], [-73.88615037899993, 40.82730567799994]]]}}, {\"id\": \"126\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 127, \"Shape_Leng\": 0.0600188855815, \"Shape_Area\": 0.000163195647796, \"zone\": \"Inwood\", \"LocationID\": 127, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.911224294065, 40.87302771883775], [-73.91119073224566, 40.87298467599476], [-73.91118400043388, 40.87298831795382], [-73.91117439655734, 40.872984663421995], [-73.91118017602021, 40.8729744565152], [-73.91116864796129, 40.87297226012855], [-73.91116674737339, 40.8729562208655], [-73.91115332406393, 40.872943080425706], [-73.91110344780436, 40.872884709050886], [-73.91109962664044, 40.872867938744974], [-73.91110445375335, 40.87285117510123], [-73.91108430768067, 40.8728263601674], [-73.91105552733826, 40.87280081839454], [-73.91103635995363, 40.87277163688558], [-73.91103927429148, 40.872748307554204], [-73.91101240352127, 40.87272203768021], [-73.91099227963159, 40.872689938350334], [-73.91098078938774, 40.87266003201782], [-73.91096737028889, 40.8726439838647], [-73.91096164062033, 40.872617001897765], [-73.91094435651671, 40.8726082351562], [-73.91091462693844, 40.87257393920995], [-73.91083208878284, 40.87251773000828], [-73.91076301659224, 40.872452058077684], [-73.91069663181787, 40.87236327354684], [-73.91066886250454, 40.872318082527606], [-73.91064877277671, 40.87229624946579], [-73.91064402620876, 40.87226205286497], [-73.91060004692918, 40.87218492862953], [-73.91056469338129, 40.87211653582043], [-73.91054082062271, 40.87205832757491], [-73.91053797212396, 40.87203359781723], [-73.91055333016271, 40.87201833783434], [-73.910544749547, 40.87198632794731], [-73.91051802379027, 40.87190629880136], [-73.91049514192561, 40.87182555244866], [-73.91048468158361, 40.87176153752579], [-73.91047920100607, 40.871729725536305], [-73.91047327677899, 40.87169534306595], [-73.91046185402595, 40.8716415112769], [-73.91045522870999, 40.871575320019616], [-73.91045053766994, 40.87149967593997], [-73.91044577257622, 40.871488037442056], [-73.91042949267288, 40.871482927922784], [-73.91043239004443, 40.87146329950374], [-73.9104477186584, 40.87146112338387], [-73.91045350986802, 40.87143349263079], [-73.91045545985004, 40.871403679088836], [-73.91045068458797, 40.871390582631264], [-73.91043824727869, 40.871377478551786], [-73.91045264466976, 40.8713629380159], [-73.91046990523188, 40.87135349691886], [-73.91047856276914, 40.871328775580324], [-73.910499658267, 40.87131715831588], [-73.91051786233251, 40.87131208375768], [-73.91052747017778, 40.87129536119792], [-73.9105504764678, 40.871283014168284], [-73.91056680858036, 40.871258300446634], [-73.9105582087164, 40.871240832534276], [-73.91056588782178, 40.871228475295815], [-73.91058704261461, 40.8711732190149], [-73.9105985494958, 40.87116449378853], [-73.91061103840086, 40.871139046347885], [-73.91062831291488, 40.87112014990978], [-73.91066096630357, 40.87106126526175], [-73.91066290216686, 40.87105108170268], [-73.910672492682, 40.87103800475036], [-73.91069358025322, 40.87103219563057], [-73.91070605869155, 40.871015482982855], [-73.91071567489408, 40.87099148305042], [-73.91072912439586, 40.87096749563152], [-73.9107368187509, 40.87094422427525], [-73.91072726861505, 40.870918039087265], [-73.91071292770269, 40.87089983835427], [-73.91069092913638, 40.87086854714837], [-73.91069192441397, 40.87084309096916], [-73.91070344390971, 40.870824919459565], [-73.91070443211592, 40.870804560661924], [-73.91072840254334, 40.870789297265276], [-73.91073130565934, 40.87076530151838], [-73.9107389951068, 40.870745668085036], [-73.91074959513284, 40.870705676166885], [-73.91075731220944, 40.87066495362825], [-73.9107372219756, 40.87064384795903], [-73.9107612369148, 40.87059586166744], [-73.91076412223113, 40.87058494994799], [-73.91076797207829, 40.870571139989956], [-73.91076321449745, 40.870554404624095], [-73.910771854801, 40.87053332217968], [-73.91077478077005, 40.87050132069525], [-73.91075950216383, 40.870456212546955], [-73.91076623112325, 40.87044531301325], [-73.91078255041363, 40.87042204781847], [-73.91076915471747, 40.87040603533581], [-73.91077305792284, 40.87035294548002], [-73.9107778959443, 40.870318766837286], [-73.91077411735417, 40.87027948514924], [-73.91075785297797, 40.87026202120585], [-73.91077897510436, 40.87023076285047], [-73.9107818820916, 40.870203850026016], [-73.91078863932837, 40.87017185166904], [-73.91078961731874, 40.87015003373561], [-73.91081268992717, 40.8700969591967], [-73.91080986642974, 40.870052589569745], [-73.91080989861791, 40.87002859159873], [-73.91081758861237, 40.870017684092836], [-73.91081282552356, 40.86999586172145], [-73.91082053654102, 40.86996022499773], [-73.9108234376424, 40.8699376796571], [-73.91081674002805, 40.86993331586905], [-73.91083402520907, 40.86990569346967], [-73.9108407482736, 40.869890416719144], [-73.9108407706945, 40.86987369493129], [-73.91085708459481, 40.86985333858792], [-73.91085711578782, 40.869830070177755], [-73.91084850117832, 40.86982351689344], [-73.91084755376188, 40.869813331101454], [-73.91086099421386, 40.86979588140576], [-73.91086678151244, 40.8697704381356], [-73.91088311464894, 40.86973553880109], [-73.91086874328361, 40.86973116088064], [-73.91087931262922, 40.86971371717354], [-73.91088893770015, 40.86968390016123], [-73.91088991976903, 40.86965917352723], [-73.91085928331742, 40.86964751556358], [-73.91088227999221, 40.86964170687792], [-73.91090242290286, 40.86962281260925], [-73.9109149158664, 40.869595185825965], [-73.91092931612711, 40.86957846642498], [-73.91092070672006, 40.86956755453764], [-73.91092169441232, 40.869547186351866], [-73.91093033139222, 40.86953701733963], [-73.91092363633767, 40.86952318935024], [-73.9109457240888, 40.86948829494994], [-73.91095152625854, 40.86945120817017], [-73.91092950706621, 40.869435919113364], [-73.91096019309789, 40.869410486341906], [-73.91099086451533, 40.86940469176468], [-73.91101288698583, 40.869418523362796], [-73.9110435360088, 40.86942072512756], [-73.91106176168185, 40.86940765580079], [-73.91113645047578, 40.869440437276346], [-73.91119295273501, 40.86945721129196], [-73.91125998364576, 40.86948562818122], [-73.91130786590509, 40.869508213248125], [-73.91135000667573, 40.86952205974031], [-73.91138545202726, 40.869520637221726], [-73.911442974879, 40.869492316562216], [-73.91146694984837, 40.86947415352528], [-73.91147752414803, 40.86945234244303], [-73.91147470849903, 40.86941088220467], [-73.91146613195859, 40.8693752336082], [-73.91142686201691, 40.86936211004542], [-73.91141921168855, 40.86935264984734], [-73.91140292777331, 40.86934972847731], [-73.91139623561234, 40.86934244714696], [-73.91136078970678, 40.86933442355124], [-73.91135794914872, 40.869311873114164], [-73.91112525892672, 40.869213505408], [-73.91111955311597, 40.86917858855775], [-73.91109943435848, 40.869178573059706], [-73.91106400513746, 40.869166911456745], [-73.91103335740765, 40.86915525302693], [-73.91105736488242, 40.86911162543575], [-73.91107752901732, 40.869078187477406], [-73.91108138261409, 40.869061459425666], [-73.91107757570234, 40.86904327606211], [-73.91109483782506, 40.86903165573075], [-73.91109389484104, 40.869019290133835], [-73.91112043564152, 40.869006199729476], [-73.91113515331632, 40.86897277565782], [-73.91112177734426, 40.8689523975784], [-73.91114059177055, 40.86893395120445], [-73.91113013545254, 40.86892030091883], [-73.91114364443597, 40.86889530382795], [-73.91114965452034, 40.868880531589426], [-73.91116612307725, 40.86887713149072], [-73.91120057610416, 40.86886465979005], [-73.91125149289988, 40.868861286183474], [-73.91128294662256, 40.86884767634568], [-73.91130393727757, 40.8688249554264], [-73.9113281775882, 40.86880243953577], [-73.91133483134577, 40.86878534909652], [-73.91133742099721, 40.868774273964696], [-73.9113297074864, 40.86875863646886], [-73.91133402644324, 40.86874039636797], [-73.91132547908168, 40.86871823689665], [-73.91133149891803, 40.86870521125036], [-73.91132553554417, 40.868675895507614], [-73.91133500160275, 40.86865765936163], [-73.91135046077642, 40.868646595289825], [-73.91134705123666, 40.868623792457946], [-73.91135909150219, 40.868606215312695], [-73.91136427055301, 40.86858340919442], [-73.91139004972909, 40.868563240128715], [-73.9114098549045, 40.86851308940594], [-73.91141932680448, 40.86849159309228], [-73.91144165907038, 40.8684766253026], [-73.91148208144206, 40.86840825517563], [-73.91149242138631, 40.868375035257806], [-73.91149244825745, 40.86835484641313], [-73.91150538755127, 40.868303385066504], [-73.91151485871004, 40.8682812392571], [-73.91152432337303, 40.868264966127796], [-73.91153547484777, 40.86826431698232], [-73.91154065941288, 40.868245428719284], [-73.91158019244979, 40.86820376706928], [-73.91160166210436, 40.868186197122384], [-73.91162485562174, 40.868174480740095], [-73.91165319817412, 40.86815430536876], [-73.91166781354046, 40.868135424822256], [-73.91171163037997, 40.86810028506055], [-73.91172833168767, 40.868074488910274], [-73.9117319206429, 40.86806062376697], [-73.91173006189779, 40.86804016393433], [-73.9117338348191, 40.86803017993704], [-73.91172569759559, 40.868030173712334], [-73.91172571085134, 40.8680201880116], [-73.91172510380609, 40.86800353710151], [-73.91173640142546, 40.86799117266264], [-73.91173516231117, 40.86797738507466], [-73.9117552250194, 40.86796598305693], [-73.9117527627056, 40.867935057768385], [-73.91176907763841, 40.86791318788755], [-73.91178038282324, 40.86789560192557], [-73.91179668040857, 40.86789561438441], [-73.91180859603989, 40.86788230457578], [-73.9117942026245, 40.86786802111718], [-73.91179676558265, 40.867831400035136], [-73.91180556034618, 40.86781047911382], [-73.91182376049142, 40.86778908877634], [-73.91184571092025, 40.86777625586334], [-73.91187101089474, 40.86775948971885], [-73.91188960227053, 40.86774775335029], [-73.91189225260952, 40.86772591741011], [-73.91191159865868, 40.86770020567247], [-73.91184774646815, 40.86767252047246], [-73.91199296785679, 40.86750583452552], [-73.9120975526718, 40.86737319112113], [-73.91222962953469, 40.86721546219405], [-73.91245215298915, 40.86694055877056], [-73.9126175467601, 40.86673955984179], [-73.91273173919515, 40.866605212711306], [-73.91255219352705, 40.866528624292194], [-73.91254775130287, 40.866514943004354], [-73.91282859693649, 40.86615711135718], [-73.91312015100736, 40.86578124232253], [-73.91351997708159, 40.86526747116957], [-73.9138448505671, 40.864848338592154], [-73.9139186313323, 40.86474558440355], [-73.91393526952008, 40.86473928454506], [-73.91403866833014, 40.86471050057713], [-73.91416821692901, 40.86466640191282], [-73.91425257819594, 40.86464572510955], [-73.91431200712502, 40.864625022695805], [-73.91438922696749, 40.86462417974642], [-73.91492387927859, 40.864580378919115], [-73.9149286371918, 40.86456956663371], [-73.91454037626016, 40.864412351623365], [-73.91447397606235, 40.8643112859949], [-73.91441113392617, 40.864173627375536], [-73.9150440759955, 40.864436721607696], [-73.9150740494132, 40.86444650490859], [-73.91508583854264, 40.86444814391995], [-73.91516833371625, 40.864439254176325], [-73.91525511928769, 40.86443362651924], [-73.91543189718274, 40.86441830355375], [-73.9154286973113, 40.86440772916068], [-73.91543408398833, 40.8643874009978], [-73.91545872397363, 40.86438416877071], [-73.91545342726465, 40.864333737086355], [-73.91563236347605, 40.86430947317944], [-73.91563874300478, 40.864348514579284], [-73.9156644439211, 40.86435015381803], [-73.9157115910561, 40.86434286717414], [-73.91575872905088, 40.864342091289565], [-73.91581658566604, 40.86433481225365], [-73.91586052063045, 40.8643259020842], [-73.91589911645785, 40.86430803808506], [-73.91592270819694, 40.8642804014022], [-73.91593561439066, 40.864242184211996], [-73.9159313804981, 40.86420396508695], [-73.91592605810229, 40.864173055876684], [-73.9158982423084, 40.86414619304341], [-73.91587148065646, 40.864129091035586], [-73.91585434906487, 40.8641225769717], [-73.91584686790445, 40.8641103696352], [-73.91579009737727, 40.86410300812718], [-73.91574189992429, 40.86409647019708], [-73.91572596025568, 40.86409562529626], [-73.9156672828416, 40.86408664891993], [-73.91561569079903, 40.864075356830625], [-73.91558377690811, 40.864070651048074], [-73.9155387298444, 40.864066818623186], [-73.91549340827902, 40.86406619394988], [-73.9154398263038, 40.864078357077574], [-73.91540554211757, 40.86408158273295], [-73.91538444080764, 40.86408030195847], [-73.9153745421635, 40.86407970133206], [-73.91535981968394, 40.86407729022841], [-73.91534579103364, 40.86407312961483], [-73.91533281441582, 40.86406732572734], [-73.91532122117134, 40.86406002676066], [-73.91531130731792, 40.86405141908448], [-73.91530571233524, 40.86404293955746], [-73.91529828280513, 40.86403531519251], [-73.91528923936099, 40.86402877240845], [-73.91527885056384, 40.86402350550435], [-73.91526742492745, 40.86401967088994], [-73.91524067177384, 40.86399606817472], [-73.91521584698249, 40.863975952691064], [-73.91519574800094, 40.863945224735815], [-73.91518803106224, 40.86389774401877], [-73.9151850595422, 40.86387946104222], [-73.91518192994755, 40.86386020413749], [-73.91518119486989, 40.86384311506673], [-73.91517880018044, 40.863787451908316], [-73.91517693846924, 40.863744186521814], [-73.91517656557124, 40.8637355194391], [-73.91517539779056, 40.863708386201246], [-73.91517556429709, 40.86368043375087], [-73.915200710796, 40.86365218622607], [-73.9152181857278, 40.86364108139574], [-73.91525443919686, 40.86361804378021], [-73.91529676393407, 40.8635986440192], [-73.91530810246218, 40.86359047369598], [-73.91534455813895, 40.863564205684476], [-73.91536190015128, 40.86354723256714], [-73.91537404288228, 40.86352873067105], [-73.91541657363635, 40.86344790631755], [-73.91542702548192, 40.863429170988994], [-73.91546863429258, 40.863362497623086], [-73.91547682444238, 40.863346107296714], [-73.9154851646404, 40.86330071832315], [-73.91547306854308, 40.8632803306086], [-73.91545987478226, 40.86325707392488], [-73.91545061536698, 40.86323298451724], [-73.9154549655414, 40.86320845729237], [-73.91542295700886, 40.86319418063079], [-73.91541083335659, 40.86319548646495], [-73.91534506689356, 40.86316454277737], [-73.91541096127442, 40.86309491906979], [-73.91538673051149, 40.863084384200725], [-73.91544970533424, 40.863006510494785], [-73.91549965939643, 40.86293812779098], [-73.91552015328232, 40.86291007496551], [-73.91556578044927, 40.86293107187448], [-73.91557595266735, 40.86291610425661], [-73.91556299694936, 40.862909251121025], [-73.91557991520047, 40.862895998348016], [-73.91558892372463, 40.86290113918815], [-73.91559964505785, 40.86289215924693], [-73.91559401542858, 40.86288915679958], [-73.91560361016737, 40.86287975337308], [-73.91561827252399, 40.86287890822119], [-73.91567199779999, 40.86290242787801], [-73.91568204906035, 40.86289390578285], [-73.91569323361948, 40.86287981889348], [-73.91571907457929, 40.86285286403296], [-73.91578014796721, 40.86278915736011], [-73.91581868929809, 40.86275125154459], [-73.91583883849175, 40.8627314341833], [-73.91585768842067, 40.86271494020015], [-73.91586450572528, 40.8627089754841], [-73.91586962339923, 40.86270449596135], [-73.91593363168722, 40.86264848769425], [-73.91595185469565, 40.86263145582462], [-73.91595260030206, 40.86263075695855], [-73.9161051792775, 40.86248814480873], [-73.91616935439103, 40.862441260343836], [-73.9162048494689, 40.86242128604049], [-73.91622522141265, 40.8624054177958], [-73.91627635012742, 40.86236559143216], [-73.91639175003213, 40.86227570227624], [-73.91640019183043, 40.86226912657566], [-73.91642605482255, 40.86224898054915], [-73.91649196679447, 40.86219763869996], [-73.91652512043825, 40.86218050396302], [-73.91653438522638, 40.86217571564995], [-73.91656892746099, 40.862157863900705], [-73.91659639029284, 40.86214637960494], [-73.91660191778598, 40.86214195529302], [-73.91661720458566, 40.86212972006312], [-73.91663823024982, 40.862112890466776], [-73.91664696334843, 40.8621059005061], [-73.91666173979016, 40.86209363474733], [-73.91672483714147, 40.86204125761551], [-73.91673206785232, 40.862034730435695], [-73.91674689045284, 40.86202134865992], [-73.91675431445412, 40.86201464538368], [-73.91676153411372, 40.862008128077925], [-73.91678113004234, 40.861990437862325], [-73.91678584948338, 40.86198617595878], [-73.91679534822482, 40.8619776012672], [-73.91685100780661, 40.86196360881367], [-73.91687889367468, 40.861953580029], [-73.91692039865875, 40.861931727543194], [-73.9169581550374, 40.8619332555635], [-73.91698437353891, 40.86192504083441], [-73.91700967075877, 40.86191278695554], [-73.91701546163455, 40.86190998158957], [-73.91705129801854, 40.861892622976505], [-73.91706909419574, 40.86187826898196], [-73.91709135961965, 40.86186030823941], [-73.91715278759457, 40.86181076114889], [-73.91723056919918, 40.86173654373122], [-73.91727034392623, 40.86169469967212], [-73.91729467741676, 40.86167941823067], [-73.91734277917362, 40.86161732807046], [-73.91737841920231, 40.861570977988144], [-73.91740810708951, 40.86153273761919], [-73.91740994469338, 40.86148637270622], [-73.91746019820957, 40.86142946988304], [-73.91748061393716, 40.86139863458938], [-73.91750401182631, 40.86136332486962], [-73.91751560442543, 40.861349924267], [-73.9175329272891, 40.86133573604431], [-73.91754749618079, 40.861323131290504], [-73.91756413342189, 40.861306835801585], [-73.91757176600544, 40.861292116971654], [-73.91759256564323, 40.86127636471438], [-73.91761405307706, 40.86126112556544], [-73.9176563633175, 40.86122119177241], [-73.9176882562444, 40.861192821694], [-73.91772917654819, 40.86114499922021], [-73.91778534143992, 40.861093504262975], [-73.9178082344227, 40.86106355285196], [-73.91782702380144, 40.86104278280521], [-73.91782221945564, 40.86102443659887], [-73.91783553764998, 40.86099693592736], [-73.91782468139061, 40.86098042213537], [-73.91782136420318, 40.860948901791126], [-73.91784103780006, 40.86093144631544], [-73.91786115746794, 40.86090065580482], [-73.91786536368912, 40.8608844678496], [-73.91787707054957, 40.86088383607668], [-73.91788211332285, 40.86087367385555], [-73.91790472454431, 40.86084987200048], [-73.91790211673528, 40.86083961320201], [-73.91792190583864, 40.860825426172156], [-73.91792234646313, 40.86081399922542], [-73.91788387387606, 40.860802221270625], [-73.91792828566328, 40.86074989768184], [-73.91795061771704, 40.860731984729355], [-73.91805221056595, 40.86062109843893], [-73.91826264968334, 40.86040482431119], [-73.9184343925178, 40.86022153339531], [-73.9183607452798, 40.860195809044086], [-73.9183764802733, 40.86017013802981], [-73.91838738720058, 40.86013529739785], [-73.91839903543183, 40.86009619750837], [-73.9183931121764, 40.8600804694219], [-73.91841071714774, 40.86004867742144], [-73.91842828384382, 40.860056892616996], [-73.91843100240942, 40.860045612185054], [-73.91843508111558, 40.86001791560625], [-73.91845133777615, 40.859991245841286], [-73.91846755096878, 40.85999228321194], [-73.91849190290432, 40.85997076061676], [-73.91850002964881, 40.85995024476261], [-73.91852845077686, 40.85991845921059], [-73.91855953955259, 40.8599071983025], [-73.9185879382561, 40.85989387361192], [-73.91861094928478, 40.8598631103016], [-73.91860422359034, 40.8598354076015], [-73.91860966058975, 40.859811808084224], [-73.91862051432182, 40.85978206428477], [-73.91864215241455, 40.85976771710835], [-73.91866650284997, 40.859739008230505], [-73.91867866057468, 40.85974824666026], [-73.918694889946, 40.859734921573725], [-73.91869354554858, 40.85972877034594], [-73.91870302556357, 40.859716458205334], [-73.91868681416331, 40.85970516392412], [-73.91873839393087, 40.85964830753148], [-73.91872079932321, 40.85964222493354], [-73.91880140046692, 40.859558095732645], [-73.91877796600042, 40.8595467513165], [-73.9188857796496, 40.859439163497186], [-73.91890338201662, 40.859418041641725], [-73.91900350576367, 40.85931346590101], [-73.91913611131905, 40.85917299233999], [-73.9192321799137, 40.85907661750321], [-73.91933772085359, 40.858963833604314], [-73.91940131157716, 40.85889924908136], [-73.91943853541449, 40.85885542134399], [-73.91948096295053, 40.85881393735841], [-73.91950213660932, 40.85879113379047], [-73.91952379146707, 40.85878347750467], [-73.91954642281223, 40.858791579133346], [-73.9195470792652, 40.85879866699752], [-73.91981340987077, 40.858912646598796], [-73.92009437526832, 40.85902815900783], [-73.92029013424884, 40.85910617748469], [-73.92028080078518, 40.859115275254396], [-73.92036070029987, 40.85914871145067], [-73.92041198649034, 40.85914773801526], [-73.9204346715058, 40.859120946591695], [-73.92044466588774, 40.85912449207713], [-73.92046134307469, 40.85910326112518], [-73.92032551794716, 40.859043492109855], [-73.92026958923736, 40.85901867159204], [-73.92024228428149, 40.85901106140345], [-73.92020833439094, 40.85899586467842], [-73.92019834633355, 40.858987258634485], [-73.9200964799355, 40.85894521618592], [-73.9200691729343, 40.858938614947505], [-73.9199566565892, 40.85888644357646], [-73.91993534975472, 40.85888086367001], [-73.91990871589499, 40.85886719354676], [-73.91984679689212, 40.85884388206952], [-73.91977223409496, 40.858801858298555], [-73.91977424610867, 40.858794772291134], [-73.91984149127128, 40.85882719172704], [-73.91991539174748, 40.858858598542184], [-73.91996200375141, 40.8588783602684], [-73.92007317233622, 40.858926343959574], [-73.92047868273168, 40.85910409209883], [-73.92043089498743, 40.85915500079302], [-73.92044055434963, 40.85916101282756], [-73.92043350742907, 40.859172355154946], [-73.92050906689407, 40.85920510375513], [-73.92050818037727, 40.85921245071382], [-73.92061976769563, 40.85925924508325], [-73.92065316209771, 40.85926326599644], [-73.92067070680027, 40.859289977915196], [-73.92069792257463, 40.859324035337025], [-73.92069348890548, 40.8593513980483], [-73.92074793277061, 40.859398152496794], [-73.92081547121266, 40.85952299932421], [-73.9208248103987, 40.85953762397836], [-73.92096638988286, 40.85975936332763], [-73.92107166676892, 40.85993630055117], [-73.92120245201015, 40.86009856883875], [-73.92127091296308, 40.86018804281668], [-73.92132187073445, 40.86021411051681], [-73.92134032212977, 40.86021678989969], [-73.92134560571333, 40.8602127856764], [-73.92136406183813, 40.86021146544214], [-73.92139397626858, 40.86019080173543], [-73.92143091729879, 40.86017480665244], [-73.92145906374984, 40.86016281389905], [-73.92151623522282, 40.860130812815925], [-73.921516243884, 40.86012347346225], [-73.92154265042431, 40.86009880067782], [-73.92155671740983, 40.86009947665001], [-73.92159014848224, 40.86007280002688], [-73.92163335462683, 40.859964040976585], [-73.92166068449596, 40.859890650861324], [-73.92168799950892, 40.85983994521568], [-73.92170654327566, 40.85976520860327], [-73.92171803927064, 40.85971182591403], [-73.92170136023316, 40.85969379563055], [-73.92171017514295, 40.85967244240074], [-73.92173654090277, 40.85967246028849], [-73.92173832622478, 40.85964710317339], [-73.92172252011017, 40.85963373783402], [-73.92170406731475, 40.85963106018624], [-73.92168648450186, 40.85963572198345], [-73.92169705737561, 40.85961169543938], [-73.92168123846953, 40.85960901907275], [-73.92166455789794, 40.859592321597304], [-73.92167072779478, 40.85958097977293], [-73.92166897017356, 40.85957230640456], [-73.92164788416521, 40.85957094989075], [-73.92164878482414, 40.859550933989105], [-73.92169009920316, 40.859547620619786], [-73.92173052181315, 40.85955566202522], [-73.92171996443375, 40.85956766728357], [-73.92175425140597, 40.85956902335616], [-73.92174017250268, 40.8595796940834], [-73.9217586285431, 40.85957903969074], [-73.92176920814246, 40.85955901979739], [-73.9218034849839, 40.85955904303371], [-73.92180349832363, 40.85954769753205], [-73.92181669341468, 40.859542365833605], [-73.92181579426178, 40.85956039306925], [-73.92183337804909, 40.85955506534812], [-73.92183602852785, 40.85954038021452], [-73.9218395729169, 40.85952169724188], [-73.92182640001612, 40.859508343081615], [-73.92180092709991, 40.85949497170502], [-73.92178421372903, 40.859496968979514], [-73.92178423413667, 40.85947961643427], [-73.92178951542853, 40.859475612188575], [-73.92179128111253, 40.8594669330016], [-73.92178337916735, 40.859460255864654], [-73.92177813999356, 40.85943689270538], [-73.92175531523264, 40.859410853164086], [-73.92174475909617, 40.85941151174202], [-73.92172807855657, 40.859394815447835], [-73.92173862702768, 40.859391481185064], [-73.92174216515188, 40.85937813885152], [-73.92178084462995, 40.85937616585848], [-73.9218010581483, 40.85938351774832], [-73.92181422123235, 40.85939487385553], [-73.921845891252, 40.85938288171889], [-73.92182743857511, 40.85937152320405], [-73.92179318314086, 40.859352814615036], [-73.92180638758411, 40.85932879922491], [-73.9218362834468, 40.85932347934413], [-73.92184158900768, 40.85930012500567], [-73.92185654400586, 40.85929078717129], [-73.92185568980089, 40.859270760194725], [-73.92186363617589, 40.85924073418782], [-73.92187159464771, 40.85920002908451], [-73.92186633783652, 40.85918134014976], [-73.92184878326474, 40.85916264288542], [-73.92181013137586, 40.85914125631136], [-73.92182860052971, 40.85912859049983], [-73.92181807326534, 40.859115238129995], [-73.92179523134078, 40.85910387546236], [-73.92175392059862, 40.85909383425407], [-73.92171348569521, 40.85909647245439], [-73.92170031211961, 40.85908378468559], [-73.92168362346396, 40.859074434775295], [-73.92167133304646, 40.85906707886248], [-73.92165637797629, 40.85907574909283], [-73.92164144199684, 40.85906840025916], [-73.92162299102328, 40.85905570001837], [-73.92157640358245, 40.8590550019536], [-73.92156323476758, 40.859038306848646], [-73.92153688128289, 40.85902828444114], [-73.92151667805072, 40.85901158404418], [-73.92149647545965, 40.85900556492629], [-73.92148241816876, 40.85899687380335], [-73.92145693841896, 40.858980171480056], [-73.92143674687564, 40.85896481260308], [-73.92141478438363, 40.85895210943488], [-73.92140688831238, 40.85893942522685], [-73.92139545451528, 40.85894008384499], [-73.92138491884843, 40.8589320720708], [-73.9213620792983, 40.8589193770061], [-73.92134098678977, 40.8589126898417], [-73.92132253891833, 40.85890733762705], [-73.92130672674394, 40.85889931286579], [-73.92129619437269, 40.85887995389809], [-73.92131378782533, 40.85887529166445], [-73.92134542016208, 40.858885328108634], [-73.92137001739772, 40.85889669038683], [-73.92138847158147, 40.85889670296219], [-73.92137003870113, 40.85887867143289], [-73.92133313153455, 40.858866633349265], [-73.92130940178662, 40.858863943160955], [-73.92127775129002, 40.85886926170835], [-73.9212558013679, 40.85884655352702], [-73.9212391392654, 40.858814512042784], [-73.92123916377011, 40.85879381975475], [-73.92124971927129, 40.85878448837616], [-73.92125502240955, 40.85876313160802], [-73.92126384154848, 40.85873710468035], [-73.9212568347567, 40.858715082986905], [-73.92123928232215, 40.858693711790785], [-73.92121027761029, 40.858689017753186], [-73.92118832732727, 40.858666311233264], [-73.92117253002358, 40.8586456158418], [-73.92113298285777, 40.85863890664739], [-73.92110400043417, 40.85861619463397], [-73.92110930043289, 40.858597512881154], [-73.92112073232282, 40.85858818278334], [-73.92110317429248, 40.858572151533615], [-73.92108911428319, 40.85856680177905], [-73.92108561693321, 40.85855611977917], [-73.92107155243102, 40.85855344452817], [-73.92110236769079, 40.85851275393425], [-73.92111383643037, 40.85847271718303], [-73.92112177944212, 40.85844535685077], [-73.92109189655196, 40.85844132909271], [-73.92108224017782, 40.85843265148383], [-73.92106819230459, 40.85841728803167], [-73.92106558355202, 40.85839726855678], [-73.92105857633999, 40.858374571228296], [-73.92107355499465, 40.8583452163224], [-73.92107885124753, 40.85832986610281], [-73.92105864446829, 40.858317173447546], [-73.92107887262824, 40.85831184831971], [-73.92108767646545, 40.858299841248005], [-73.9210639508839, 40.85829381914129], [-73.92099712882886, 40.85830377722352], [-73.92098748015539, 40.85828975896982], [-73.92094165735571, 40.858284414913996], [-73.92092950082889, 40.85825501412292], [-73.92091459857794, 40.85821896663772], [-73.92093869016941, 40.85812067446154], [-73.92081205174519, 40.85808225560556], [-73.92066414958995, 40.858036084760066], [-73.9206600093851, 40.85802724891318], [-73.92062427494882, 40.85801681477228], [-73.92060931660876, 40.858018696153046], [-73.9204843854752, 40.85823254806245], [-73.92035142875443, 40.858195220619145], [-73.92036477249754, 40.85816493709913], [-73.92047278308782, 40.85819530433254], [-73.9204827854302, 40.85817196149781], [-73.92042547044454, 40.85814731130444], [-73.92043129405509, 40.8581372208709], [-73.92049359805799, 40.858165341058694], [-73.92050776371386, 40.85814200109049], [-73.92050777201234, 40.8581350580941], [-73.92047369719569, 40.858125570886756], [-73.92047870523538, 40.85811294928194], [-73.92051858770813, 40.8581262325458], [-73.92055690666027, 40.85805810091989], [-73.9205244898293, 40.858049874644045], [-73.92053115561134, 40.85803978528795], [-73.92055940739407, 40.85804926796663], [-73.92058272320153, 40.85801141923442], [-73.92054366782155, 40.85800066666606], [-73.92056616182882, 40.85796439326244], [-73.92053125929229, 40.857953013828954], [-73.9205362614933, 40.85794544354678], [-73.92056783382262, 40.857955559738315], [-73.92059865529569, 40.85789878704372], [-73.92058121435349, 40.85788362925946], [-73.92056957361469, 40.857888664366676], [-73.92056458747452, 40.85788297887912], [-73.9205828871062, 40.85787415680171], [-73.92061195368183, 40.85789563653846], [-73.92061194313827, 40.857904469531334], [-73.92077315152532, 40.85794118585792], [-73.92077568813255, 40.85790332113988], [-73.92072417703403, 40.85788498852888], [-73.92075084244313, 40.857831995110466], [-73.92075835110651, 40.857810540535226], [-73.92071431024056, 40.85779473427547], [-73.92071848663913, 40.85778338174714], [-73.92070601599154, 40.857778321847256], [-73.92068020565321, 40.85781995106413], [-73.92067272424282, 40.85781742469497], [-73.92069520768324, 40.85778083514134], [-73.92063119159812, 40.85779593807325], [-73.92062871453668, 40.85778458097081], [-73.92069356299953, 40.85776789384213], [-73.92069440518277, 40.857757169747714], [-73.92067862502522, 40.857752747676464], [-73.9206744727128, 40.85774390193742], [-73.92068280608277, 40.857727500938644], [-73.92070610816093, 40.85770101410192], [-73.92071524126256, 40.85770985505421], [-73.92072604222552, 40.85771238319908], [-73.92074102474629, 40.8576997783191], [-73.92074018031833, 40.857712392912646], [-73.92071186501481, 40.85775592096779], [-73.92070770216483, 40.857766643785325], [-73.9207176690908, 40.85777233268963], [-73.9207209967533, 40.85776665292151], [-73.92074591736552, 40.857774872820706], [-73.92075009406679, 40.85776289023625], [-73.92085894403364, 40.8577957794152], [-73.92085476658923, 40.857808401104336], [-73.92082485064823, 40.85780143807836], [-73.92081902487526, 40.85781341970117], [-73.92085806645605, 40.85782607154625], [-73.92090874735563, 40.857843772803285], [-73.92091792340256, 40.857817277915004], [-73.92095947032028, 40.85782740083621], [-73.92095530904551, 40.8578349717129], [-73.92100848746315, 40.857847632689], [-73.92101432159707, 40.85783880318057], [-73.92100188046803, 40.85781796615766], [-73.92098278238336, 40.85780154635137], [-73.92098779381654, 40.85777630838017], [-73.92096035984791, 40.857787014262115], [-73.92095035684837, 40.85781035713936], [-73.92082407985954, 40.85775220748162], [-73.92083991421826, 40.857721294888435], [-73.92096286020838, 40.857778813197406], [-73.92097118010598, 40.85777313684196], [-73.92098531723391, 40.857764312351854], [-73.92098948520304, 40.857749799648595], [-73.92097620645463, 40.8577359050516], [-73.9209629044585, 40.857741577492064], [-73.92092883675433, 40.8577264083683], [-73.92093134355565, 40.85772262741055], [-73.92092137657248, 40.8577156785819], [-73.92091720674985, 40.85772134789474], [-73.92091056810698, 40.85771945284374], [-73.92091472483578, 40.85771440436354], [-73.92090475572954, 40.857709985133646], [-73.92090225899011, 40.85771439581783], [-73.92089228746032, 40.85771312803338], [-73.92089395343788, 40.8577074558319], [-73.92087484340281, 40.857701129947436], [-73.92087151368248, 40.85770744044379], [-73.92086403145021, 40.85770616565296], [-73.92086987022243, 40.85769418520922], [-73.9208972809938, 40.85770240678379], [-73.92090394377362, 40.85769420857401], [-73.92091142601693, 40.857695473478444], [-73.92090725459605, 40.85770304384414], [-73.92091722324945, 40.857707471784636], [-73.92092388660173, 40.85769990379672], [-73.92093219665317, 40.85770306094022], [-73.92092553330006, 40.857710630101245], [-73.92093550328049, 40.85771505753855], [-73.9209588081255, 40.857685411498565], [-73.92088985451419, 40.857663906180285], [-73.9209140083115, 40.857619113275405], [-73.9208915753442, 40.85761341584083], [-73.92088491030914, 40.857623515605454], [-73.92081427830648, 40.85760706042049], [-73.92078761914492, 40.857654370632254], [-73.92077266841045, 40.857649939761984], [-73.92079765383627, 40.85760388767714], [-73.92077605235171, 40.85759756056828], [-73.9207885591364, 40.857571698701406], [-73.9207835691488, 40.85756980360235], [-73.92071949545583, 40.85763286578186], [-73.92069706927529, 40.857621486761744], [-73.92075532130517, 40.85755715914579], [-73.92077195797644, 40.85754896662086], [-73.92078857040964, 40.85756223330813], [-73.92079356302033, 40.857553403232686], [-73.92090075662524, 40.857582499179266], [-73.92089408405236, 40.85759764155769], [-73.92091900484338, 40.857605861420005], [-73.92092983808878, 40.85759199222074], [-73.92098217660369, 40.85761221698062], [-73.92100215706549, 40.85758509875878], [-73.92094565691801, 40.85756423272509], [-73.92095148909628, 40.85755666349581], [-73.92101011202337, 40.85757846860787], [-73.92101505160244, 40.85757358148218], [-73.92101050029305, 40.85757127391592], [-73.92101657172314, 40.857561776988994], [-73.92102453923798, 40.85756407868504], [-73.9210319869216, 40.85755504439358], [-73.92090011378151, 40.85749428733865], [-73.92090943072884, 40.85747877738354], [-73.92093729165354, 40.85748726226913], [-73.92094474783687, 40.85747033747435], [-73.92099676230846, 40.85748447487976], [-73.92103583613053, 40.8574393594625], [-73.9210135498486, 40.85742946602829], [-73.92104891513684, 40.85738152111153], [-73.92106933540855, 40.857394233182], [-73.92109350791351, 40.857382956301265], [-73.92116405945595, 40.85743662069778], [-73.92113244274998, 40.85745494138236], [-73.92107291328591, 40.85751698028255], [-73.92112119690319, 40.8575367708331], [-73.92118439137633, 40.85752410721182], [-73.92122323658643, 40.85752233035374], [-73.92130475209376, 40.85742065143315], [-73.92136378026812, 40.85739996536387], [-73.92140169782738, 40.85738959755385], [-73.92145963506641, 40.85736725288179], [-73.9215196726864, 40.85734890638002], [-73.92155550463495, 40.85732175059622], [-73.92158397059998, 40.85728979192766], [-73.92159242598983, 40.85726341560259], [-73.92158614447966, 40.85723223373973], [-73.92157985204594, 40.857211443841805], [-73.92155849622625, 40.857152702522846], [-73.92155857155751, 40.857088830629124], [-73.92159160704567, 40.85706515423337], [-73.92161794209801, 40.85705477965232], [-73.92164638409345, 40.857043606572454], [-73.92166323981648, 40.85703562380639], [-73.9217032680549, 40.85702126106141], [-73.92174434054141, 40.85701329471569], [-73.92180121504491, 40.856998943362186], [-73.92183702160173, 40.85699257192147], [-73.92186334597841, 40.856991790499244], [-73.9218865189657, 40.8569846112325], [-73.92190021806313, 40.85697502778716], [-73.92191814231168, 40.85695425312496], [-73.92194132896722, 40.856934283619346], [-73.92196978492625, 40.85691111831288], [-73.92204492207946, 40.85685415979202], [-73.92263334899997, 40.856831752999895], [-73.92277921199988, 40.856826197999915], [-73.92282845999992, 40.8568514089999], [-73.92295449499989, 40.85885076199988], [-73.92295846599977, 40.85891378099996], [-73.92411747399994, 40.86065512499987], [-73.92461430599997, 40.86140156099989], [-73.924677287, 40.861496644999946], [-73.92474153399992, 40.86159363799992], [-73.92537318199992, 40.8625808149999], [-73.92601092899994, 40.8635736649999], [-73.92661948999994, 40.86452429399989], [-73.92726915099985, 40.86553455999994], [-73.928057704, 40.866199363999954], [-73.92872203199994, 40.86674266299988], [-73.92700250499988, 40.8679512599999], [-73.92623313399999, 40.868506137999915], [-73.92382143399999, 40.86901838199991], [-73.92234360699992, 40.86945904099991], [-73.92214424199994, 40.86915018199993], [-73.92187285799986, 40.86927147499992], [-73.92187269199987, 40.86927154799987], [-73.92187252499993, 40.86927162799988], [-73.92161072100001, 40.86940564999989], [-73.92161064999992, 40.869405685999936], [-73.92135975499993, 40.869551853999894], [-73.92112156199988, 40.869709123999854], [-73.92112131299997, 40.869709285999875], [-73.92112107599998, 40.869709465999904], [-73.92089662999997, 40.8698770189999], [-73.91925481799977, 40.87100784199991], [-73.91969887499998, 40.872064775999924], [-73.91945895199984, 40.8722492809999], [-73.91845930899993, 40.873022052999936], [-73.9171766959999, 40.87249897299991], [-73.91627785499992, 40.87211564499995], [-73.91505692599985, 40.87160289599989], [-73.91415003100002, 40.87121985099985], [-73.9140098549999, 40.871160593999946], [-73.91366488599988, 40.87150058499987], [-73.91334258899985, 40.871756027999915], [-73.91267976599985, 40.87229176099988], [-73.91206172199999, 40.87280609199992], [-73.91176119999992, 40.873063789999925], [-73.91154281116535, 40.873253706161705], [-73.91142580718564, 40.87318173085131], [-73.91135863952978, 40.87312772184548], [-73.91130490090681, 40.87309194960776], [-73.91127418900135, 40.87306348765566], [-73.911224294065, 40.87302771883775]]], [[[-73.92151894059094, 40.85660117062315], [-73.92159078705224, 40.856496661134166], [-73.9215969651289, 40.85649911033576], [-73.9216718169156, 40.85639401600263], [-73.92168970374024, 40.8564006434736], [-73.92175965629767, 40.856304681808545], [-73.921823250446, 40.85632912586351], [-73.9220295673179, 40.856040055047735], [-73.92204171700534, 40.856045054280955], [-73.92212909531288, 40.85592262927441], [-73.92212701759904, 40.85591940891104], [-73.92212563579955, 40.855915977100395], [-73.92212498555783, 40.8559124189683], [-73.92212508375181, 40.85590882951213], [-73.92212592520553, 40.85590529500261], [-73.92212749215773, 40.855901909575984], [-73.92212974214682, 40.85589875797033], [-73.92213261947896, 40.855895920224334], [-73.92213604839007, 40.85589346932677], [-73.92213994408347, 40.85589146720413], [-73.9221442056687, 40.85588996488286], [-73.9221487254288, 40.8558890018259], [-73.92215338838639, 40.855888600403496], [-73.92215807516381, 40.85588877209427], [-73.92218505751369, 40.855847415549796], [-73.9222189508469, 40.85579546579659], [-73.92223557106672, 40.85577118594845], [-73.92225931710543, 40.85572561566736], [-73.92227219913848, 40.8557008923733], [-73.9222811377406, 40.855686931638864], [-73.92228832283236, 40.85567571342004], [-73.92228837057658, 40.855675638904344], [-73.92229611007241, 40.855663835114505], [-73.92234459387058, 40.85568233590737], [-73.92234217898557, 40.855685461012676], [-73.92233636262432, 40.85569298743926], [-73.92232971276387, 40.85570159265731], [-73.92232888081013, 40.85570267178515], [-73.92232785330081, 40.85570400173132], [-73.92219572192109, 40.85587499055044], [-73.92217874229385, 40.85589696519633], [-73.92218117594763, 40.85589995310889], [-73.9221829564415, 40.85590319471384], [-73.92218404128785, 40.85590660873356], [-73.92218440300836, 40.85591010854001], [-73.92218403221776, 40.85591360667985], [-73.92218293740308, 40.85591701537637], [-73.92218114624346, 40.855920249043855], [-73.92217870362472, 40.85592322545872], [-73.92217567339657, 40.85592587095337], [-73.92217212888958, 40.85592811806574], [-73.92216815974786, 40.85592991040063], [-73.92216386729518, 40.85593120413541], [-73.92215935791673, 40.85593196633968], [-73.92215474438092, 40.85593217732165], [-73.92215014297037, 40.855931831128885], [-73.92206335132519, 40.85605391178446], [-73.92207704565449, 40.856059135365925], [-73.92195201468988, 40.85623699044721], [-73.92187322153111, 40.856349662056054], [-73.92188760113714, 40.85635513372336], [-73.9218175009566, 40.85645422436558], [-73.92183549330599, 40.85646214131242], [-73.92176508454392, 40.8565658685902], [-73.92177019744629, 40.856567894901275], [-73.92169841904696, 40.85667230185494], [-73.92151894059094, 40.85660117062315]]]]}}, {\"id\": \"127\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 128, \"Shape_Leng\": 0.0702090290681, \"Shape_Area\": 0.000207405982869, \"zone\": \"Inwood Hill Park\", \"LocationID\": 128, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.92359742020389, 40.878898712992644], [-73.9236171919141, 40.87885275853596], [-73.92362423150792, 40.87885490128636], [-73.92363695289211, 40.87881535593209], [-73.92367919311681, 40.878826074498626], [-73.92366647926978, 40.878863496512956], [-73.92367773890643, 40.87886562613649], [-73.92366219321842, 40.87891158339879], [-73.92365233486956, 40.878911576877584], [-73.92362973975199, 40.87896394312155], [-73.92359172576555, 40.87895215916629], [-73.92361150251907, 40.878900860405935], [-73.92359742020389, 40.878898712992644]]], [[[-73.9264055692112, 40.87762147653741], [-73.92635080268249, 40.87759232046259], [-73.92629520344381, 40.87756537764219], [-73.92630825734184, 40.87755054598148], [-73.92629847830014, 40.877533241262185], [-73.92629265898066, 40.877514937239674], [-73.9262909640355, 40.87749615181873], [-73.92629344141879, 40.877477416524314], [-73.92630002103081, 40.8774592614629], [-73.92631051670104, 40.8774422003228], [-73.92632463145766, 40.87742671584045], [-73.92634196592844, 40.87741324614079], [-73.9263534010335, 40.87740645871844], [-73.92636692938808, 40.8774000162195], [-73.92637867418443, 40.87739558168086], [-73.9263910032764, 40.87738349591492], [-73.92640802970632, 40.87737059295209], [-73.92642110544408, 40.87733596882139], [-73.92641753339738, 40.87729419342119], [-73.92641611724353, 40.87726616044383], [-73.92640746178061, 40.87723743282434], [-73.92639977123254, 40.8772210887617], [-73.9263991893142, 40.877220013564994], [-73.92637952771739, 40.87719099712255], [-73.92635386246528, 40.877164821268956], [-73.92632864338461, 40.8771458606087], [-73.92630027044785, 40.8771296702047], [-73.92626926759239, 40.877116549029495], [-73.9262453844698, 40.877107196388664], [-73.92615199405294, 40.87708622922075], [-73.92605688918461, 40.87707030956458], [-73.92596054274807, 40.8770595165764], [-73.92586343379948, 40.877053903921215], [-73.92575170680726, 40.877069382332614], [-73.92564159048314, 40.87709048786865], [-73.9255906871655, 40.87710228685281], [-73.92549403426749, 40.87712842248997], [-73.92539963430987, 40.877158935721575], [-73.92530783107327, 40.87719371542732], [-73.92524634149107, 40.877228149550874], [-73.92519060669277, 40.877267827799535], [-73.92514139843446, 40.87731220075884], [-73.92510748792195, 40.877350298819394], [-73.92507840203368, 40.87739062951776], [-73.92504660587467, 40.87740930387462], [-73.92501165764392, 40.87742438784907], [-73.92497425597504, 40.87743557990353], [-73.92493514854823, 40.877442656302506], [-73.92489511714379, 40.87744547558453], [-73.9248275546625, 40.87744477528718], [-73.92476021287621, 40.87744057495241], [-73.92469340614272, 40.87743289418772], [-73.92466562648586, 40.877428645942], [-73.92463117456374, 40.877422495132706], [-73.92455187350787, 40.87740427039568], [-73.92447065956256, 40.87739181072038], [-73.92438825846597, 40.87738522744807], [-73.92428280917072, 40.87736652931362], [-73.9241788599333, 40.877343494966674], [-73.92407672001595, 40.877316192937386], [-73.92397669329614, 40.87728470445305], [-73.92397527867065, 40.87728422323365], [-73.92391006772809, 40.87726096010417], [-73.9238717096996, 40.87724224949382], [-73.9238136130879, 40.87722978627896], [-73.92375369171315, 40.87722419516896], [-73.92369334793713, 40.877225607014616], [-73.92363399400763, 40.87723398877401], [-73.92357701900715, 40.87724914428568], [-73.92353366131377, 40.877264724938115], [-73.92348842591906, 40.877276845938525], [-73.9234293956286, 40.8772870102061], [-73.92336912887104, 40.877291291674275], [-73.9233149012421, 40.87729209093475], [-73.92326088083085, 40.87728840831927], [-73.92320770869343, 40.87728028752906], [-73.92315569321521, 40.87726773224347], [-73.92310580095484, 40.87725093071908], [-73.92305863152065, 40.877230084878235], [-73.92303506229437, 40.87720731455437], [-73.92300876211289, 40.87718468513292], [-73.92294412537231, 40.87713691678709], [-73.92294354888257, 40.87713253970405], [-73.92292227478717, 40.87709403012095], [-73.92288254092276, 40.8770651332168], [-73.92286583578245, 40.8770554958633], [-73.9228243580552, 40.8770375304725], [-73.92279786519552, 40.877028327483224], [-73.92277768846607, 40.877025253009215], [-73.92276904248315, 40.877032685901696], [-73.92276213709975, 40.8770234953405], [-73.9227771360302, 40.87701081720804], [-73.92278406427518, 40.87700032286081], [-73.92276677289159, 40.87699550255101], [-73.92274949451486, 40.876989367026646], [-73.92272934926771, 40.87696966040841], [-73.92269767779175, 40.876944704783746], [-73.92265706349428, 40.87692871165308], [-73.92263113766302, 40.87692257098523], [-73.92260748875238, 40.876935675171914], [-73.9226075060566, 40.87692080888346], [-73.92259771145176, 40.87691729840976], [-73.92258387170895, 40.87692253928837], [-73.92257466338114, 40.87691422163936], [-73.92257120834557, 40.87690634910445], [-73.92257641535174, 40.87689715895161], [-73.92257007003339, 40.87689584818489], [-73.92257758831418, 40.87687923028559], [-73.92258912035129, 40.87686829828595], [-73.92256839578593, 40.87685735168327], [-73.92253610173903, 40.876868262710666], [-73.92249578378302, 40.8768419865078], [-73.9224837033913, 40.8768240489412], [-73.92243706188535, 40.87678901507315], [-73.92243362687847, 40.87677326378681], [-73.92241576943685, 40.876765822988446], [-73.92240887383943, 40.876748312229026], [-73.92239907976202, 40.87674305715715], [-73.922400623303, 40.87673234126452], [-73.92240533581122, 40.876699431748], [-73.92241886633713, 40.87663307471047], [-73.92242299211522, 40.87655445689456], [-73.92240693330263, 40.87647889402186], [-73.92241108552582, 40.87638802009181], [-73.92243750433738, 40.87622621072902], [-73.92244171582213, 40.87607407695334], [-73.9224269873826, 40.876017912260686], [-73.922414963897, 40.875941326184744], [-73.92241369514767, 40.875879038367216], [-73.92241845190593, 40.87583973536945], [-73.92241443957168, 40.875811142403386], [-73.92242252812991, 40.8758029801022], [-73.92242659071222, 40.8757784809567], [-73.92242259483537, 40.87574580815281], [-73.92242396062542, 40.8757243604639], [-73.92242265440576, 40.875694751448364], [-73.92243077867309, 40.87564575148044], [-73.92244696222404, 40.875613085202474], [-73.92247388452017, 40.87558962682106], [-73.92247527129064, 40.87556001090333], [-73.92249078070564, 40.87552683740952], [-73.92252715989451, 40.875470708211054], [-73.92254872142617, 40.87543499058307], [-73.92255413930911, 40.875407422151675], [-73.92257168317542, 40.875358428492376], [-73.9225838173596, 40.87532881978236], [-73.92259326010884, 40.875309429326634], [-73.9226094679212, 40.875256348549414], [-73.92264179478191, 40.87521655023125], [-73.92265124753175, 40.87518796562525], [-73.92264992358426, 40.87516346285896], [-73.92265534021038, 40.875136919827476], [-73.92265672015898, 40.87510322472124], [-73.92265343019594, 40.87504553672262], [-73.92265347533191, 40.87500673563258], [-73.92265621964499, 40.87495160036858], [-73.92266297766196, 40.8749250582346], [-73.92265628020954, 40.87489953416529], [-73.9226563300901, 40.8748566527251], [-73.92265370811916, 40.87480661962215], [-73.92265104634217, 40.87477904576762], [-73.92264706229452, 40.87473616162276], [-73.92265246019703, 40.87471676794623], [-73.92263505079865, 40.87465039011858], [-73.92263911310297, 40.87462486404581], [-73.92265258491385, 40.87460955596584], [-73.92263913448598, 40.8746064854712], [-73.92264185921482, 40.87457994063308], [-73.92262241329665, 40.87453041004044], [-73.9226076645474, 40.874492615722225], [-73.92258487775169, 40.87441806653691], [-73.9225674567449, 40.8743629189126], [-73.92253526184382, 40.87428938947979], [-73.92253663536354, 40.87426181832856], [-73.92253531894417, 40.87424037754327], [-73.92250844284314, 40.87421483823994], [-73.92248291316665, 40.87419541609168], [-73.92245467613134, 40.874187230049856], [-73.92244396077807, 40.874149446459604], [-73.92241305509722, 40.874120835910325], [-73.92239695345737, 40.8740830488613], [-73.92237683031455, 40.87404525123125], [-73.922341230702, 40.874002859089806], [-73.92232781274556, 40.87397324191267], [-73.92227950313357, 40.87387416555273], [-73.92226069249847, 40.87385986117744], [-73.92226204657453, 40.87384863303612], [-73.92224324758365, 40.87382513568446], [-73.9222184093859, 40.87378172534674], [-73.92220362984004, 40.87376945924827], [-73.92218619484476, 40.87372554881244], [-73.92216067057093, 40.87369081785797], [-73.92211364251733, 40.87365300102316], [-73.92208944494341, 40.87363971185712], [-73.92206527090744, 40.87360804377443], [-73.92203570759096, 40.87358760930834], [-73.9219926791775, 40.873568175225685], [-73.9219859680907, 40.8735559244366], [-73.92196579224448, 40.8735528389296], [-73.92194296317747, 40.87352627732294], [-73.92191875466726, 40.8735221817653], [-73.92189592800635, 40.87348439059406], [-73.92186904474784, 40.87347517824162], [-73.92183005389812, 40.87345269423069], [-73.92178168307726, 40.873416921132666], [-73.92172791772019, 40.87337809225714], [-73.9216701428844, 40.873330056596004], [-73.9216183807434, 40.87330297087042], [-73.9215901530671, 40.8732876340777], [-73.92149841910096, 40.87324779384943], [-73.92141123301121, 40.8732024497342], [-73.92132916660044, 40.873151899122355], [-73.92125275808976, 40.873096473550774], [-73.921169296781, 40.87306939228805], [-73.92108198156424, 40.87305347540086], [-73.92104702078319, 40.87304425733171], [-73.92100265196893, 40.87303606039706], [-73.92097844127738, 40.873033999037375], [-73.92092598584571, 40.873031918824836], [-73.92089505217784, 40.8730278184436], [-73.92084932060317, 40.873027787086194], [-73.92081299828271, 40.87302571905911], [-73.92079015106097, 40.873015491522644], [-73.9207740068131, 40.873014463241574], [-73.92075652672875, 40.87300934497615], [-73.92071319024366, 40.87301051299764], [-73.92069726314591, 40.8730063510054], [-73.92069129315439, 40.872998781906695], [-73.92066636738936, 40.87300027767254], [-73.92064442871197, 40.87299344641935], [-73.92060953713961, 40.8729858585973], [-73.92057463023556, 40.872990373776126], [-73.9205606635804, 40.87299414465014], [-73.92052973842686, 40.8730130524324], [-73.9204938100222, 40.87302892057545], [-73.92046088686524, 40.87304176665674], [-73.92042797810988, 40.8730523417989], [-73.92038905696707, 40.87307123464822], [-73.92035813253032, 40.87307878651478], [-73.9203292096807, 40.873089364882055], [-73.92030226117289, 40.87310599670292], [-73.92029230572703, 40.87309841662261], [-73.92027634052175, 40.87310597879865], [-73.92025736613591, 40.87312110321887], [-73.92022292951731, 40.873149462307765], [-73.92019300382896, 40.873165335009034], [-73.92016706828103, 40.87316759571542], [-73.92014910357753, 40.87318120741502], [-73.92013213760734, 40.87319179517842], [-73.92012314974757, 40.87320768235057], [-73.92010118828748, 40.87322053491071], [-73.92008222781284, 40.87323338937726], [-73.92006027075733, 40.8732522938619], [-73.9200462721765, 40.873281811066754], [-73.92001431202269, 40.87332190544188], [-73.9199853538589, 40.87335215975609], [-73.9199758550652, 40.87337902433251], [-73.9199658574108, 40.87339642369028], [-73.91995189299274, 40.8734077693715], [-73.91994884190137, 40.873448640142286], [-73.9199597904169, 40.873469845068826], [-73.91995279557034, 40.87348573411186], [-73.91995376124532, 40.87351147064185], [-73.9199577235154, 40.87353115832614], [-73.91994374992086, 40.87353946949144], [-73.91994074372579, 40.87355385728064], [-73.9199357337655, 40.873576554151356], [-73.91991976382094, 40.873588655304864], [-73.91991776524846, 40.87359320132476], [-73.91990280475248, 40.87359394680504], [-73.91989579909402, 40.87360907896863], [-73.91986389068703, 40.87360603101838], [-73.91981902154345, 40.87360826978589], [-73.91978312944534, 40.87360369743557], [-73.91978308103732, 40.87364382313614], [-73.91982295820223, 40.87364990126987], [-73.9198628465619, 40.873646147316194], [-73.91989077111431, 40.87364616670429], [-73.91991668692258, 40.873651479752745], [-73.91993362050847, 40.873666629031426], [-73.91994956492047, 40.87367648256047], [-73.91994555163063, 40.87369843297073], [-73.91994054414333, 40.873719617949426], [-73.9199465131935, 40.873727951659475], [-73.91995149021679, 40.873742336274944], [-73.91995446542515, 40.87375369370475], [-73.9199594277758, 40.87377110343666], [-73.91996637212853, 40.87379608824889], [-73.91997433025303, 40.87381804746398], [-73.91996733096013, 40.873827885077546], [-73.91997828639676, 40.87384302919795], [-73.9199917496795, 40.87384417366738], [-73.92012428446928, 40.87392222958632], [-73.92013926594399, 40.873904076629884], [-73.92016516941842, 40.87391847571556], [-73.92015018861089, 40.873936627503014], [-73.92019105255761, 40.87396012286744], [-73.9202090220016, 40.873942729004156], [-73.9202369138315, 40.87396166747344], [-73.92021794909964, 40.873977547754535], [-73.9202877068482, 40.874015444218784], [-73.92034849897662, 40.8740503081534], [-73.92040330261301, 40.87408365502706], [-73.92043917329222, 40.874106379939406], [-73.9204720447048, 40.874136687033314], [-73.92049844862419, 40.874159793050396], [-73.92054333350322, 40.874144678227346], [-73.92057619910042, 40.87418027917513], [-73.92060608624132, 40.874206791627785], [-73.92062699165683, 40.87423481112878], [-73.92076966714161, 40.87417738586543], [-73.92078761461626, 40.874178911084414], [-73.92080255630347, 40.87419405886663], [-73.92080751306929, 40.87421526009998], [-73.92066084429216, 40.874278743471514], [-73.92066980301328, 40.87429691178115], [-73.92068572694191, 40.87432341511571], [-73.92069665918827, 40.87434840144937], [-73.92071107412367, 40.874387393739674], [-73.92070010598974, 40.874391933609424], [-73.92071604571977, 40.87440632505257], [-73.92072498444587, 40.87443055482081], [-73.92074290853479, 40.8744517555819], [-73.92075785117514, 40.87446614650183], [-73.92075483429564, 40.874489612027126], [-73.92076677770551, 40.87451081688575], [-73.92078070500605, 40.8745312585311], [-73.92078169241671, 40.8745395887767], [-73.92079265694505, 40.8745471695053], [-73.92078865109977, 40.87456306014473], [-73.92080359467634, 40.87457669452618], [-73.92079957959791, 40.8745910734341], [-73.92080954431921, 40.87460092273833], [-73.92081650482736, 40.87461379460436], [-73.92081649581894, 40.87462135908938], [-73.92082945415412, 40.87462894101476], [-73.92084240240087, 40.87464333105765], [-73.92083940266566, 40.87465241443042], [-73.92084337428213, 40.87466452838157], [-73.92085235584491, 40.874663778679036], [-73.92086879559815, 40.87467627994868], [-73.92088273221307, 40.87468839252236], [-73.92088671102694, 40.87470429684725], [-73.92089167392983, 40.87472095067331], [-73.9209056154877, 40.87473912355146], [-73.92091956358946, 40.87475124433697], [-73.9209225407913, 40.87476184470561], [-73.92091453517739, 40.874780010750236], [-73.9209155100288, 40.87479817524273], [-73.92092349157225, 40.87479969361022], [-73.92092746229659, 40.874813320455736], [-73.9209384181303, 40.874827708452024], [-73.9209473691218, 40.87484285160678], [-73.92097230770959, 40.8748413557891], [-73.92098723992625, 40.87486407674106], [-73.92098721924138, 40.87488148351575], [-73.92100015808852, 40.874895124706924], [-73.92101610565861, 40.87490270061613], [-73.92103202290917, 40.87493450766608], [-73.92105493692358, 40.874960259196556], [-73.92108133530783, 40.874978061804434], [-73.92111420160488, 40.87501366259345], [-73.92114009384979, 40.87504849286096], [-73.92115600330543, 40.875077274761516], [-73.92116695696885, 40.87509468853005], [-73.92117492688537, 40.8751060493373], [-73.92117491342296, 40.875117405193684], [-73.92118186508976, 40.87513784085071], [-73.9211698884575, 40.87514389297375], [-73.92116987769634, 40.87515296968485], [-73.92117284868006, 40.8751681191225], [-73.92116885342608, 40.87517568021374], [-73.9211698417931, 40.87518325291774], [-73.92115787665249, 40.87518853980346], [-73.92115986015386, 40.87519611486262], [-73.92114389543798, 40.87520366727274], [-73.92114588498157, 40.8752067048147], [-73.92113391038936, 40.875209721255395], [-73.9211319044239, 40.875221831110125], [-73.92112690618701, 40.87523469595512], [-73.9211109446201, 40.87523846720637], [-73.92110394084429, 40.87525284290744], [-73.9211129036416, 40.87526723019337], [-73.92110691408503, 40.87527631152839], [-73.92110491037751, 40.875285396760574], [-73.9210799642233, 40.87529294352374], [-73.92106199576551, 40.87530883400249], [-73.92105700109649, 40.87531942020209], [-73.92103602209794, 40.875343628802725], [-73.92102403531777, 40.87536784422347], [-73.92101105768876, 40.875376920770776], [-73.92099010857669, 40.87538523549415], [-73.92097812109449, 40.87540036481026], [-73.92096913044006, 40.875418521973565], [-73.920955153647, 40.875429110728255], [-73.9209302080565, 40.87543742320585], [-73.92092920057632, 40.875446499233476], [-73.92091823046628, 40.87545330838505], [-73.92091322183165, 40.87546466031515], [-73.92090325515422, 40.87546616638258], [-73.92089428451298, 40.875457831169996], [-73.92087559199673, 40.87546395002319], [-73.92084158803193, 40.875457335018744], [-73.92079579222155, 40.875450352408], [-73.92076221925713, 40.87543951476236], [-73.9207327312122, 40.87541631556056], [-73.92067983077874, 40.87539773829669], [-73.92057606461886, 40.875365222461205], [-73.92053640420096, 40.875336604217345], [-73.92047030646286, 40.875292524993], [-73.92039300443471, 40.87525384948977], [-73.92034213453783, 40.87523372489072], [-73.9203024613251, 40.87521593053747], [-73.92025056805598, 40.87520816837106], [-73.92023398573666, 40.875202823991025], [-73.92018482308941, 40.875189096689034], [-73.92009707040329, 40.875147034489174], [-73.9198189520158, 40.87503434871183], [-73.91972463724571, 40.87499613519409], [-73.91924742219437, 40.87483104494457], [-73.91897439539596, 40.87470861588605], [-73.91896727626954, 40.874709376634], [-73.91894286506647, 40.87469700569053], [-73.91890319914951, 40.87467379886836], [-73.918857942435, 40.87464324876078], [-73.9188589780202, 40.874630895688746], [-73.91884881993983, 40.87461388732878], [-73.91885699002506, 40.87458762583615], [-73.91887126163864, 40.874568329706634], [-73.91889469405842, 40.874556756448385], [-73.91894154174601, 40.87452665921787], [-73.91899146424903, 40.87448652348068], [-73.91901591932367, 40.874463361689216], [-73.91903833950522, 40.87443479585484], [-73.91908625963019, 40.87436298815266], [-73.9191173711259, 40.87430931388154], [-73.91916221498532, 40.87424985963383], [-73.91921118168392, 40.874155648781866], [-73.91922652147316, 40.8740969478044], [-73.91923368469071, 40.87406837125045], [-73.91924749339371, 40.87401082192224], [-73.91925978641387, 40.87393975445136], [-73.91926195569941, 40.87383392226377], [-73.91925904468509, 40.87372035852512], [-73.91926009034685, 40.873689455653626], [-73.91927030351344, 40.8736608812292], [-73.91928102175844, 40.87363655754588], [-73.91926375452918, 40.87361259137962], [-73.91921896474608, 40.873617972027276], [-73.91893896566567, 40.873302966470234], [-73.91862104668414, 40.873197294299786], [-73.91855365026461, 40.873180588115396], [-73.91853666560324, 40.873196857670536], [-73.91853998447118, 40.8732290426561], [-73.91854552710514, 40.873256142715604], [-73.9185622051726, 40.87329596461076], [-73.91857669301552, 40.87331205769525], [-73.91860122675237, 40.87332139429759], [-73.91862350877972, 40.873347659666095], [-73.91864021482168, 40.873374776470676], [-73.91862843705421, 40.87342091852606], [-73.9186272845728, 40.87345140736928], [-73.91860268946238, 40.873492884923806], [-73.918551335354, 40.87352079132853], [-73.91850556824869, 40.873539389397614], [-73.9184731967847, 40.873557151592706], [-73.91845086074116, 40.87357492036813], [-73.91843599403886, 40.873567021044586], [-73.918414036594, 40.873585051918646], [-73.91842852157109, 40.87359522794529], [-73.91839947532085, 40.873628237548765], [-73.91838048803518, 40.873642613995955], [-73.91835591726749, 40.87366462284993], [-73.91833357491785, 40.87368747088964], [-73.91830232945689, 40.873695066371134], [-73.91828781665394, 40.873708608859026], [-73.91826659956283, 40.8737221454243], [-73.91824427174542, 40.87373313367925], [-73.91821079469307, 40.87374243042434], [-73.91818288978932, 40.87375510703017], [-73.91817615884986, 40.87377711961341], [-73.91815939815253, 40.87380505857412], [-73.91813594089376, 40.87382705878905], [-73.918106915921, 40.87384312976647], [-73.91807901550598, 40.87385157522655], [-73.91805893427565, 40.873849867984006], [-73.91805165797422, 40.87386254546654], [-73.91803715672657, 40.87389725829061], [-73.91794108366345, 40.87398314781029], [-73.91792902917979, 40.873978893205056], [-73.91775468095318, 40.87424853456616], [-73.91776990325637, 40.8742543038425], [-73.9175772988473, 40.87454867854727], [-73.91749921000535, 40.87452007854011], [-73.91769066058586, 40.87422522218328], [-73.91770931109484, 40.87423249407923], [-73.91789032098379, 40.87396247775167], [-73.91788506630628, 40.87396025014748], [-73.91787144917501, 40.87390499528864], [-73.91784685200199, 40.87393101283527], [-73.91781448198516, 40.87394623603753], [-73.91780777752835, 40.87395724404439], [-73.91779327441755, 40.873953001417846], [-73.91777764915211, 40.87396314669117], [-73.91778097440154, 40.873980088296435], [-73.91776812456737, 40.87399151458243], [-73.91774021567048, 40.874006731058564], [-73.91770785270614, 40.874016027949004], [-73.9176434714112, 40.87402630142591], [-73.9175903009546, 40.87403575514311], [-73.91755115955162, 40.87406777530254], [-73.91749486066541, 40.87408434930755], [-73.91742914539073, 40.87411754000315], [-73.91739625419048, 40.87416024365767], [-73.91726640898548, 40.87420999319872], [-73.91718819114, 40.87424198521311], [-73.91711624585014, 40.874257358727355], [-73.9170372269262, 40.874303002166734], [-73.91698090690281, 40.87432669780203], [-73.91685267654526, 40.87433728573168], [-73.91671349901951, 40.87435498785026], [-73.9166556014652, 40.87438699433422], [-73.91658048205615, 40.87443560224685], [-73.91644828715275, 40.874495443085415], [-73.91634348667563, 40.874523849889655], [-73.91621216322862, 40.874503583365296], [-73.91609852720278, 40.87450015889178], [-73.91601675124143, 40.87451485046097], [-73.91589033125027, 40.87451963732432], [-73.91576604987803, 40.87452767756272], [-73.91568463084475, 40.874527618105084], [-73.91557751037357, 40.87451940890471], [-73.91540183736201, 40.874503016307244], [-73.91522402146052, 40.87448662186813], [-73.91506980141547, 40.8744491021406], [-73.91498198606968, 40.87442464213988], [-73.91463283278519, 40.874326621096806], [-73.91422167905834, 40.874199603645565], [-73.91419784904603, 40.87418935531245], [-73.91390244516936, 40.874096565043786], [-73.91358433902973, 40.874029105900824], [-73.91349623201457, 40.87399286752708], [-73.91334350989779, 40.87395015089037], [-73.91329209830897, 40.873941980883494], [-73.9132471240974, 40.87392730960938], [-73.91319197797986, 40.87389272024491], [-73.91314007075697, 40.87386867985808], [-73.9130929525688, 40.873854006909966], [-73.91303726119472, 40.87384257993784], [-73.91297760373487, 40.87384462648886], [-73.9128872888601, 40.873835961915], [-73.91272877461886, 40.87380656837369], [-73.91260884231676, 40.8737674436236], [-73.91250176411435, 40.87372832966794], [-73.91229017490778, 40.87363544449129], [-73.91208146441699, 40.87352884099816], [-73.91154281116535, 40.873253706161705], [-73.91176119999992, 40.873063789999925], [-73.91206172199999, 40.87280609199992], [-73.91267976699986, 40.87229176099989], [-73.91334258899985, 40.871756027999915], [-73.91366488699992, 40.87150058499987], [-73.9140098549999, 40.871160593999946], [-73.91415003100002, 40.87121985099985], [-73.91505692599985, 40.87160289599989], [-73.91627785499992, 40.87211564499995], [-73.9171766959999, 40.87249897299991], [-73.91845930899993, 40.873022052999936], [-73.91945895199984, 40.8722492809999], [-73.91969887499998, 40.872064775999924], [-73.91925481799977, 40.87100784199991], [-73.92089662999997, 40.8698770189999], [-73.92112107599998, 40.869709465999904], [-73.92112131299997, 40.869709285999875], [-73.92112156199988, 40.869709123999854], [-73.92135975499993, 40.869551853999894], [-73.92161064999992, 40.869405685999936], [-73.92161072100001, 40.86940564999989], [-73.92187252499993, 40.86927162799988], [-73.92187269199987, 40.86927154799987], [-73.92187285799986, 40.86927147499992], [-73.92214424199994, 40.86915018199993], [-73.92234360699992, 40.86945904099991], [-73.92382143399996, 40.869018380999925], [-73.92623313399999, 40.868506137999915], [-73.92700250499988, 40.8679512599999], [-73.92872203199994, 40.86674266299988], [-73.92934104099999, 40.86725723999993], [-73.92997974899997, 40.86779614899995], [-73.93091942499993, 40.868571000999935], [-73.93117877500005, 40.8687688669999], [-73.93127671599983, 40.8688435939999], [-73.93141437399996, 40.868961927999905], [-73.93156536999994, 40.86958215799991], [-73.9321985510756, 40.8698448708717], [-73.93220600531866, 40.870096445104515], [-73.93212328527157, 40.8703789431052], [-73.93204740481397, 40.87058290396658], [-73.9320315692666, 40.87075720253468], [-73.93191377136804, 40.87112949372269], [-73.9317844012482, 40.87133442292377], [-73.93160149587301, 40.871583875469994], [-73.93146821139372, 40.87179771620653], [-73.93137655898947, 40.8721086273606], [-73.93122107559076, 40.87233532285591], [-73.93106254199596, 40.87266680652866], [-73.93084511434465, 40.87288717030784], [-73.93068381818158, 40.873156876900445], [-73.92997686743148, 40.87433896233042], [-73.92986215062545, 40.87441844152513], [-73.9293191881939, 40.87529443856099], [-73.92917968255965, 40.87555554896632], [-73.9290134580973, 40.87573866059271], [-73.92859002632834, 40.876096010295036], [-73.9283297696842, 40.87635346958729], [-73.92814623603574, 40.87652252894397], [-73.92803638576686, 40.87658244300851], [-73.92796594643464, 40.876617685376495], [-73.92753082127467, 40.87671053481295], [-73.9268994050243, 40.877216633931994], [-73.92660824434302, 40.87750763121183], [-73.92653096278354, 40.87758369178244], [-73.92648185718433, 40.877618203017974], [-73.9264159145478, 40.87761004770086], [-73.9264055692112, 40.87762147653741]]]]}}, {\"id\": \"128\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 129, \"Shape_Leng\": 0.0932727273403, \"Shape_Area\": 0.00047398833179, \"zone\": \"Jackson Heights\", \"LocationID\": 129, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87586503899996, 40.76605165399989], [-73.875535095, 40.76423051299991], [-73.87532637000001, 40.76305660499986], [-73.87532089499993, 40.763008007999886], [-73.87531552900002, 40.762944974999876], [-73.87499575099986, 40.761264321999924], [-73.87549091099996, 40.76135159499989], [-73.87601371899997, 40.76144373199991], [-73.875806546, 40.76030783599991], [-73.87551080399989, 40.758679635999975], [-73.87513874300004, 40.756633815999905], [-73.87420803099994, 40.75673156299993], [-73.87365721799983, 40.756803485999946], [-73.87271067399989, 40.75506352099989], [-73.872692407, 40.75502993899988], [-73.87266966999984, 40.75498679899989], [-73.87214232699986, 40.753986047999874], [-73.87207047299985, 40.75386198799987], [-73.87179244199989, 40.753325668999906], [-73.87084215599985, 40.75154469299985], [-73.86968551, 40.74961236199989], [-73.86942458599987, 40.74915685199991], [-73.87037881299995, 40.74906407799989], [-73.871356485, 40.74895695099991], [-73.87184095900001, 40.7489055769999], [-73.87237256399987, 40.74885155099986], [-73.87277356899986, 40.74881295799992], [-73.87330310099995, 40.748758124999895], [-73.8737009829999, 40.748717318999866], [-73.87398293699992, 40.74868566899991], [-73.87462946399991, 40.74861392799995], [-73.87543236199986, 40.74852735999987], [-73.87555643099991, 40.74851360799994], [-73.87625180199986, 40.748438849999914], [-73.87647952999997, 40.74841640199987], [-73.87692458299989, 40.74836898899989], [-73.87741612499993, 40.748319373999955], [-73.87834312599989, 40.7482226599999], [-73.87845779299981, 40.74821481099991], [-73.87927779999994, 40.74812409299993], [-73.87983894900005, 40.74806172499992], [-73.88020647999986, 40.74802497999991], [-73.88113747599988, 40.74792729299991], [-73.88207112100002, 40.74782992699986], [-73.88256876999999, 40.74777784499986], [-73.8829991829999, 40.747734401999914], [-73.88393579499991, 40.747630242999904], [-73.88485538499994, 40.747534811999856], [-73.88578740499986, 40.74743975399987], [-73.88671656299988, 40.7473414629999], [-73.88764663699996, 40.747244838999926], [-73.88778326899985, 40.7472289269999], [-73.88867766199998, 40.747134277999926], [-73.88957128399994, 40.7470390539999], [-73.8905012919998, 40.74694193899997], [-73.89143128599993, 40.74684473699989], [-73.89175048799984, 40.74681172799989], [-73.89240906699999, 40.74713532899987], [-73.89379708799999, 40.747691499999874], [-73.89521297399997, 40.748290552999926], [-73.89560209500003, 40.748449719999904], [-73.89583955299999, 40.74854685699991], [-73.89614389199978, 40.74867878599984], [-73.89624245599988, 40.74871954499987], [-73.89648281799981, 40.74907123399993], [-73.8970765, 40.75011887599991], [-73.8972114379999, 40.75046637499993], [-73.89728972099992, 40.750872780999906], [-73.89733942799997, 40.751419567999875], [-73.89748654099999, 40.752014347999896], [-73.89757880899995, 40.75238393499996], [-73.89762297800004, 40.752523183999905], [-73.89770744899984, 40.75273485899988], [-73.89790235299994, 40.75308771499985], [-73.89826519799995, 40.75366541399989], [-73.89855812999998, 40.7541370959999], [-73.89897117099996, 40.75471530599986], [-73.89905513000004, 40.754949824999876], [-73.89912259299984, 40.75516929399989], [-73.89923583699982, 40.75553001699993], [-73.89959808799998, 40.75623543799993], [-73.89969372699987, 40.75644660899995], [-73.89972650399997, 40.756541866999875], [-73.89976862099998, 40.75666423699991], [-73.89979518199993, 40.75677539099996], [-73.89982174199989, 40.75688653699989], [-73.89985245499989, 40.757111608999885], [-73.89986047499984, 40.75733746299991], [-73.89984592699983, 40.75756215299985], [-73.89980933199992, 40.75778375299994], [-73.89978921699989, 40.7578984789999], [-73.89976111400001, 40.757985758999915], [-73.8997166869999, 40.75816257899992], [-73.89967747499992, 40.758296863999945], [-73.89964864399998, 40.75837966699987], [-73.89960731900003, 40.75845463599988], [-73.89956743900002, 40.758545262999846], [-73.89948479499999, 40.75868286799992], [-73.89941035199995, 40.75880883999987], [-73.89906072899988, 40.75932801599991], [-73.89866137899992, 40.75992101699997], [-73.89825240299986, 40.76051448699995], [-73.89791145199985, 40.76102034899989], [-73.897895281, 40.761044505999905], [-73.89735543399992, 40.76185091399985], [-73.8972310879999, 40.762029524999924], [-73.89720062399994, 40.7620732819999], [-73.89700714599998, 40.76235119299988], [-73.89668188499988, 40.76277758899986], [-73.89641138399999, 40.7631500829999], [-73.89624744699991, 40.76333274799986], [-73.89608492499991, 40.76347375399991], [-73.89598406499984, 40.76354310999991], [-73.89585991899992, 40.763628473999916], [-73.89557704399981, 40.76382796799991], [-73.89542447799998, 40.76394724499989], [-73.89527479499989, 40.76408559799993], [-73.8951683709999, 40.7641973619999], [-73.89506657699998, 40.76433132799987], [-73.89505758700001, 40.76434653799987], [-73.89499458899998, 40.76445315399994], [-73.89483535199999, 40.76465531699995], [-73.89477219899987, 40.76488921799994], [-73.89470543599994, 40.76506903299987], [-73.89461716499984, 40.765256256999855], [-73.89453879999992, 40.7653572669999], [-73.89446044299983, 40.76544639099992], [-73.89437624799997, 40.7655265849999], [-73.89428031700004, 40.76560677599987], [-73.8941491849999, 40.765694356999866], [-73.89403763099993, 40.76575891899991], [-73.89390458899999, 40.765818256999836], [-73.89384202100001, 40.765842335999935], [-73.89377741500005, 40.765867198999956], [-73.89349605899986, 40.765950929999924], [-73.89325764999997, 40.76621185499989], [-73.89325108799979, 40.766271445999976], [-73.89299530699995, 40.76624861399995], [-73.89277464899999, 40.766230073999914], [-73.89232583099981, 40.76621747199987], [-73.89206947799994, 40.76621963599993], [-73.89183535799988, 40.766227079999936], [-73.89165463899995, 40.76615653599992], [-73.89144489699993, 40.7660746539999], [-73.89089043699997, 40.765858197999925], [-73.89072002699994, 40.76588687999986], [-73.89042403199987, 40.76596206099988], [-73.88994886800005, 40.7661044099999], [-73.88918520499999, 40.76629769199996], [-73.88877310599993, 40.766511326999854], [-73.88835471199987, 40.76669771399991], [-73.88777877399983, 40.766823295999856], [-73.88770661799987, 40.76683859799995], [-73.88762452899995, 40.766865200999895], [-73.88713429399999, 40.76698296599988], [-73.88671945999997, 40.767091312999874], [-73.88667620399993, 40.76710325899991], [-73.88602554199989, 40.767282957999875], [-73.88564823400002, 40.767407690999946], [-73.88506266099994, 40.76759894299993], [-73.88439729399991, 40.767842169999874], [-73.88426804199986, 40.767894764999916], [-73.8835659359999, 40.768180444999885], [-73.88277241799986, 40.76852944099993], [-73.88159926399999, 40.7691258519999], [-73.88068583599984, 40.76963245199993], [-73.88056341499993, 40.76970115299986], [-73.88020161599998, 40.769904189999856], [-73.87961158799999, 40.77023244799991], [-73.87897365099987, 40.770556420999874], [-73.87854915699987, 40.77077199599996], [-73.87818646999993, 40.77092753499992], [-73.87781416299985, 40.77107045699987], [-73.87743339799992, 40.77120025499991], [-73.87704537199986, 40.77131652899986], [-73.87665135299991, 40.771418945999926], [-73.87615314099999, 40.77152345599989], [-73.87608949699984, 40.77153690299987], [-73.87604778699992, 40.77144359299993], [-73.87600801500002, 40.7713084929999], [-73.87598378999998, 40.771176146999935], [-73.8759736269999, 40.77104276199989], [-73.87597756999998, 40.77090960899993], [-73.87600937699986, 40.77048155299986], [-73.87602197499997, 40.770312000999915], [-73.87601415599997, 40.769816975999944], [-73.87638792400003, 40.76899955799995], [-73.87625062099983, 40.76816607499988], [-73.87624365599994, 40.76812831899984], [-73.87623300399999, 40.768085937999885], [-73.87613975699989, 40.76769550599995], [-73.87586503899996, 40.76605165399989]]]}}, {\"id\": \"129\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 130, \"Shape_Leng\": 0.142028320665, \"Shape_Area\": 0.000468322514327, \"zone\": \"Jamaica\", \"LocationID\": 130, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76962552399995, 40.71178631999993], [-73.7700680079999, 40.71135386799989], [-73.7706583899999, 40.71081884299989], [-73.77076570499999, 40.71073529199989], [-73.77076577600003, 40.710735237999884], [-73.770765846, 40.71073519299988], [-73.77089074799977, 40.71065160299992], [-73.77102469499992, 40.710576640999946], [-73.77102481399993, 40.710576578999884], [-73.77102494399993, 40.71057651599992], [-73.77118657799993, 40.710499509999885], [-73.77130086399985, 40.710449730999855], [-73.7713009589999, 40.710449686999866], [-73.77130105399985, 40.71044965099991], [-73.77144123499994, 40.71039604199991], [-73.7714414249999, 40.71039596999992], [-73.77144162599984, 40.71039590799989], [-73.77158658399989, 40.71035031499988], [-73.77158671399987, 40.71035026999991], [-73.77158686799989, 40.71035023399988], [-73.771724447, 40.71031681899991], [-73.77169204899995, 40.710263984999884], [-73.77165155299986, 40.710222371999905], [-73.77147819699984, 40.71022223499992], [-73.77115938700003, 40.71020522199994], [-73.77084362899981, 40.710171624999894], [-73.76881217299984, 40.709808123999885], [-73.76865794899992, 40.70973774499992], [-73.76844546499984, 40.709637026999886], [-73.76918606499989, 40.70933634199989], [-73.77093585099995, 40.70862589699986], [-73.77230141699995, 40.70807142299988], [-73.77265189, 40.707992923999875], [-73.77278262899993, 40.707957893999925], [-73.77299790100001, 40.70790021499987], [-73.77333799499985, 40.70779351599987], [-73.77367069099986, 40.70767319499985], [-73.77373764399991, 40.707645606999904], [-73.77399462499987, 40.707539717999914], [-73.77430843599988, 40.70739366799994], [-73.77461091499988, 40.70723572399992], [-73.7749009329999, 40.707066661999875], [-73.77505158100001, 40.70692403599988], [-73.77518799499988, 40.706770116999955], [-73.77530808899986, 40.70660617899991], [-73.77541010599997, 40.70643377599989], [-73.77549258399985, 40.7062546719999], [-73.77555451099995, 40.70607083499987], [-73.77559530099992, 40.705884318999956], [-73.77561486299999, 40.705697221999834], [-73.77561355999983, 40.70551157799985], [-73.7755921879999, 40.70532930799993], [-73.77507462299982, 40.70523998399989], [-73.7743477909998, 40.7039863969999], [-73.77394353599978, 40.70331150399988], [-73.7735339129999, 40.702620427999854], [-73.7733174479998, 40.70226169599991], [-73.77312771699984, 40.70194727699994], [-73.77272432599987, 40.70126221599989], [-73.77230768699997, 40.70057150699989], [-73.77274622199978, 40.70037864699992], [-73.77293506699988, 40.700294919999905], [-73.77318719099986, 40.70018646399989], [-73.7734383289999, 40.70007532199988], [-73.77359394899996, 40.69999438299987], [-73.77395659499993, 40.69980578099988], [-73.77441230099998, 40.70028087699991], [-73.77488318599994, 40.7007477519999], [-73.77536886199987, 40.701205982999966], [-73.77586892899987, 40.70165517099991], [-73.77638291300003, 40.7020949279999], [-73.77691037799988, 40.70252488499991], [-73.77791103899986, 40.70328238899989], [-73.77849131099994, 40.70383322499985], [-73.77916009800003, 40.70438896699998], [-73.7798182799998, 40.70492789799992], [-73.78034491200003, 40.70541241699986], [-73.78147887399984, 40.705455250999854], [-73.78256582599995, 40.70542486399989], [-73.78442027299995, 40.7047624669999], [-73.7848487629999, 40.70458556999992], [-73.78597976500005, 40.704186615999916], [-73.78751689999984, 40.703715081999896], [-73.78804161499981, 40.70355559599991], [-73.78835413999998, 40.70346023099993], [-73.78919309, 40.70320438499993], [-73.78966391499986, 40.703039488999934], [-73.78977597299989, 40.70297886399991], [-73.78999405799986, 40.702871415999944], [-73.79055566899994, 40.702567795999904], [-73.79115569699985, 40.70229407599992], [-73.79131126499983, 40.70225465399992], [-73.7914732489998, 40.70222990499984], [-73.791637999, 40.7022205499999], [-73.79180176799989, 40.702226635999864], [-73.79200046599996, 40.70222080899988], [-73.79219799199994, 40.702200687999905], [-73.79239170299988, 40.7021665299999], [-73.79235282099985, 40.70209793999988], [-73.79186916599991, 40.70136613699988], [-73.79194372099994, 40.701271725999945], [-73.7920169119999, 40.70111772699987], [-73.79210582699996, 40.700967788999954], [-73.7921059579999, 40.70096757299994], [-73.7921061119999, 40.70096735699988], [-73.79221016699994, 40.700822850999934], [-73.7922102969999, 40.7008226709999], [-73.79221043999986, 40.700822508999885], [-73.79232894499991, 40.70068447699989], [-73.79232905199986, 40.70068434099988], [-73.79232916999992, 40.70068422399992], [-73.79246106899983, 40.70055382499993], [-73.79246111600004, 40.70055378899984], [-73.79260506799987, 40.70043209999988], [-73.79284356499997, 40.700243682999876], [-73.79357708699986, 40.699756471999876], [-73.79449920899998, 40.699385776999854], [-73.79572173799994, 40.69901473499993], [-73.7964760649998, 40.698693084999924], [-73.79722374699978, 40.6983594569999], [-73.79753672799987, 40.69819892499987], [-73.79786193399997, 40.69799340399991], [-73.79854640199993, 40.69746653299995], [-73.79876814999977, 40.69725955099994], [-73.79918226899999, 40.6968714759999], [-73.79985927599985, 40.69634707999995], [-73.80053204200006, 40.69585669199992], [-73.80119617899997, 40.69533648899994], [-73.80211252599995, 40.69454449899996], [-73.80295940099998, 40.69381747099987], [-73.80381770599995, 40.69307963199988], [-73.80468792999994, 40.6923455719999], [-73.80555849799987, 40.691608704999965], [-73.80642479399984, 40.690873857999904], [-73.80723957200003, 40.69143198799988], [-73.80803640399986, 40.69198289499991], [-73.80828132399985, 40.69214738599988], [-73.80890216900004, 40.69256434899993], [-73.80971017199981, 40.692199383999885], [-73.81026957999993, 40.692045248999925], [-73.81054216699988, 40.69197245899991], [-73.81063298299996, 40.69194738299988], [-73.81075433199987, 40.69215870099987], [-73.81102158699986, 40.69262288699985], [-73.81139791299987, 40.69332231399992], [-73.81172663199997, 40.6939192989999], [-73.81176990199997, 40.69399390499992], [-73.81202952199988, 40.69444762999992], [-73.8120913559999, 40.694559620999875], [-73.81224188600005, 40.69483223599984], [-73.81245207999991, 40.69519812499991], [-73.81299801799996, 40.6961120629999], [-73.81315381599988, 40.69636935899991], [-73.81338900499996, 40.6967577539999], [-73.81361850799995, 40.69713673899985], [-73.81365634399988, 40.69720105099985], [-73.81369419799996, 40.697266795999894], [-73.8140705039998, 40.69788712999989], [-73.81415191799988, 40.69803996999988], [-73.81430922799986, 40.69833533199988], [-73.81441548799988, 40.698518972999885], [-73.81466945899993, 40.69897071199993], [-73.81492343299988, 40.69942244099986], [-73.81498727099982, 40.699548830999966], [-73.81513940199982, 40.69985000899991], [-73.81537190699999, 40.70031031099995], [-73.81540728100003, 40.70038033799993], [-73.81546252299988, 40.70049286499988], [-73.81551777599985, 40.70060539199991], [-73.81593703000003, 40.701505116999904], [-73.81613853900002, 40.70193756999987], [-73.81637824499992, 40.70245196399987], [-73.81628916400003, 40.7024540649999], [-73.8159402059999, 40.70245890199989], [-73.815315933, 40.702487330999915], [-73.81496204599993, 40.70249848699989], [-73.81429413399988, 40.70253110899994], [-73.81398666699995, 40.70254206699991], [-73.81356470599994, 40.70252849799989], [-73.81314101299999, 40.702499551999914], [-73.81301882900003, 40.70249515699991], [-73.81193002299993, 40.70234083599994], [-73.81152856899989, 40.70227287499991], [-73.81163157099994, 40.70305215999991], [-73.81171289, 40.70366220199985], [-73.81177296799999, 40.7041809509999], [-73.81180202699996, 40.70436635199991], [-73.81190436299985, 40.70512394099994], [-73.81201248599983, 40.70594494499993], [-73.80934800399993, 40.70668543899991], [-73.80694856299993, 40.70736709999989], [-73.80658872800001, 40.70663390999988], [-73.80613695999992, 40.70569149199989], [-73.8056477309999, 40.70467172799994], [-73.80357251599989, 40.705233230999966], [-73.80265548599988, 40.70548042999997], [-73.80165725599993, 40.705761474999846], [-73.80222783399978, 40.70674931299991], [-73.80278721399985, 40.707715802999864], [-73.80302414799988, 40.70813965699997], [-73.80278213499996, 40.70827267099985], [-73.80255069099992, 40.70841720299988], [-73.80233119999987, 40.708572470999904], [-73.80212493399993, 40.7087376039999], [-73.80193297099994, 40.70891159599995], [-73.80159590599997, 40.70954842499991], [-73.80151152599983, 40.70962160799997], [-73.80151145499995, 40.70962166199995], [-73.80151137199988, 40.7096217159999], [-73.80141430599987, 40.7096867729999], [-73.8014141629999, 40.7096868729999], [-73.80141399799987, 40.70968695199988], [-73.80130625, 40.709742148999965], [-73.80130615599984, 40.709742192999926], [-73.80130606099976, 40.709742228999914], [-73.80119037199992, 40.70978616299991], [-73.80119015899984, 40.70978624399993], [-73.80118994599997, 40.70978629699991], [-73.80106920299994, 40.709818210999856], [-73.79985783199996, 40.710252562999905], [-73.79870150699992, 40.71066337699988], [-73.79807226500003, 40.70963987799992], [-73.79776810199988, 40.7091431879999], [-73.79731400799989, 40.70927019899988], [-73.79639418099987, 40.70953157899987], [-73.79561668999996, 40.709753078999874], [-73.79473052399995, 40.71001246199995], [-73.79389609999994, 40.71035232899985], [-73.79303800099989, 40.71071107499988], [-73.79202854699994, 40.71106937099994], [-73.79101410700002, 40.71143258599991], [-73.79021544199992, 40.71171704499989], [-73.78941412099978, 40.71199753199987], [-73.78928335099997, 40.712031728999946], [-73.78914879699997, 40.71205688599986], [-73.78901209599994, 40.7120727089999], [-73.78900157599992, 40.71207263699995], [-73.78899105499988, 40.71207249999985], [-73.78898053499987, 40.712072309999854], [-73.78897000299989, 40.712072074999874], [-73.78895945899993, 40.712071775999895], [-73.78894892899983, 40.71207142299989], [-73.7889383849999, 40.7120710159999], [-73.78892784199994, 40.71207055599987], [-73.78891729899986, 40.71207004099992], [-73.78890676899992, 40.712069472999914], [-73.78889622599996, 40.71206884199988], [-73.78888568399994, 40.71206816399993], [-73.78887515399994, 40.7120674249999], [-73.78886461199977, 40.71206663999991], [-73.78885408199986, 40.712065791999905], [-73.7888435639998, 40.7120648899999], [-73.78883304599998, 40.712063934999954], [-73.78882252899999, 40.71206292599988], [-73.78881202299988, 40.71206185199992], [-73.78880153, 40.71206073299987], [-73.78879103599988, 40.712059553999936], [-73.7887805549999, 40.71205832699993], [-73.78877008599987, 40.71205703799991], [-73.78875961699991, 40.712055695999936], [-73.78874917199984, 40.71205430699993], [-73.7887387379998, 40.712052855999964], [-73.78872830499998, 40.71205135099987], [-73.7887178959999, 40.712049792999885], [-73.78870749899997, 40.712048178999964], [-73.78869712499983, 40.712046502999904], [-73.78868675199999, 40.712044781999914], [-73.78867640200002, 40.71204300799991], [-73.78866607699987, 40.71204117799994], [-73.78865576299997, 40.712039294999904], [-73.78864546199993, 40.71203735799987], [-73.78863519600003, 40.71203536699988], [-73.78862492999987, 40.71203333199993], [-73.78861469999994, 40.71203123299989], [-73.78860448199985, 40.712029079999866], [-73.78859429999986, 40.71202688199987], [-73.78858412899999, 40.71202462999992], [-73.78857398199985, 40.71202232299989], [-73.78856385999993, 40.71201996399993], [-73.78855376099986, 40.71201755899993], [-73.78854369699997, 40.7120150909999], [-73.78853364699978, 40.712012578999925], [-73.78852363099992, 40.71201001999984], [-73.78851365099997, 40.71200739899987], [-73.78850368299993, 40.712004733999954], [-73.7884937509998, 40.712002022999854], [-73.788483855, 40.71199925699994], [-73.78847398199989, 40.71199643899991], [-73.78846414499994, 40.71199357499989], [-73.78845433099994, 40.71199065799992], [-73.78844455400004, 40.711987694999884], [-73.78843481199984, 40.71198468799988], [-73.78784678499987, 40.711810832999916], [-73.7874204069999, 40.71184767799992], [-73.78699598699988, 40.71190160899989], [-73.78657528699986, 40.71197249299993], [-73.78616005899983, 40.71206007099989], [-73.78575198499988, 40.7121639529999], [-73.78535272199989, 40.71228361699985], [-73.78446144599992, 40.71249390499992], [-73.78356075700003, 40.71271078799989], [-73.78262547499989, 40.712937547999914], [-73.782470433, 40.71297658099992], [-73.7823665, 40.7130027439999], [-73.78227126599995, 40.713025130999895], [-73.7817827399998, 40.71313999399994], [-73.78058892899989, 40.713428175999894], [-73.78042034199986, 40.71346870999984], [-73.7799800519999, 40.71357457099991], [-73.77972553599984, 40.71363576999988], [-73.77886957499987, 40.71384929699984], [-73.77800119699988, 40.714051123999894], [-73.77730633899989, 40.714209316999906], [-73.77712346999995, 40.71426199399986], [-73.77618628399998, 40.71448557399985], [-73.775714584, 40.714598454999916], [-73.77529927899995, 40.7146963139999], [-73.7744423819999, 40.71490504299985], [-73.77357135199996, 40.71512009499986], [-73.77275859199995, 40.71539722099993], [-73.77192121999992, 40.715685043999905], [-73.77113434999984, 40.71450193699991], [-73.77039451899988, 40.713400198999885], [-73.76946291199997, 40.712011247999946], [-73.76943987099992, 40.711976892999864], [-73.76962552399995, 40.71178631999993]]]}}, {\"id\": \"130\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 131, \"Shape_Leng\": 0.116547120922, \"Shape_Area\": 0.000423696152789, \"zone\": \"Jamaica Estates\", \"LocationID\": 131, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.77732562899998, 40.722392700999926], [-73.77727808000003, 40.72234633599988], [-73.77600685799993, 40.723303568999924], [-73.77583631399985, 40.72343173999993], [-73.776032805, 40.72336247699992], [-73.7760563109999, 40.72335593999985], [-73.77587312899986, 40.7234728209999], [-73.77548473600002, 40.723720613999866], [-73.77524876600002, 40.723843161999895], [-73.77496975299988, 40.72398806099991], [-73.774371087, 40.72426113199986], [-73.77423835499987, 40.72431652299992], [-73.77372529499999, 40.724530629999926], [-73.77348555899981, 40.7246249519999], [-73.77286784099987, 40.72486798499995], [-73.77224702999986, 40.7250770729999], [-73.77165390499994, 40.72526758999993], [-73.77163268999986, 40.72527440499992], [-73.771395781, 40.72535050099993], [-73.76991329299986, 40.72574839799991], [-73.7697527519999, 40.72578523199992], [-73.76924074899993, 40.72590270099988], [-73.76872630299988, 40.72599314299992], [-73.76871602799994, 40.72598984399985], [-73.7687058369998, 40.725986266999875], [-73.76869575399984, 40.72598241999985], [-73.76868582499995, 40.72597829299986], [-73.76867607499993, 40.725973915999894], [-73.76866656199985, 40.7259692859999], [-73.7686572999999, 40.72596439499989], [-73.76864834499995, 40.72595927999993], [-73.76863972199986, 40.72595394099988], [-73.76863146699995, 40.72594840399995], [-73.76862361599991, 40.725942669999874], [-73.7686161789998, 40.72593676599994], [-73.76860920399997, 40.725930700999896], [-73.76860271599992, 40.725924518999896], [-73.76859671300002, 40.72591821299991], [-73.76859122999998, 40.7259118259999], [-73.76858628199994, 40.72590537699995], [-73.76858186599988, 40.72589887499988], [-73.76857799399986, 40.72589235699989], [-73.76857466699992, 40.72588583999987], [-73.768571885, 40.72587934999989], [-73.7685696449999, 40.72587288899995], [-73.76856792799998, 40.72586650099986], [-73.76856674299998, 40.72586018599985], [-73.76856605399993, 40.72585397099989], [-73.76745788000002, 40.72613928099987], [-73.76717559599999, 40.72618267299988], [-73.76660888099983, 40.726267875999966], [-73.76629387899985, 40.726318367999944], [-73.76618933199987, 40.72632947499991], [-73.76610679099996, 40.72633693499997], [-73.76568253199987, 40.726377523999886], [-73.7654693229998, 40.72638204199991], [-73.76545422899993, 40.72638236199994], [-73.76475781199981, 40.72639712399995], [-73.76386689499986, 40.7262841589999], [-73.76376765199994, 40.72627157299995], [-73.76368265699995, 40.72625991699991], [-73.76241922899995, 40.726086592999934], [-73.7620440879999, 40.72603512499993], [-73.76181677799993, 40.72600393899992], [-73.76102756099999, 40.72589417899992], [-73.76083706099992, 40.72587704299985], [-73.76017281699988, 40.7258140209999], [-73.75994854100003, 40.72580527799991], [-73.75955973699996, 40.7258022329999], [-73.75922708600002, 40.72580597799991], [-73.75879801899991, 40.72584059599993], [-73.75835433199994, 40.72588850899994], [-73.75796018499997, 40.72595204999991], [-73.75755793699985, 40.72603243899989], [-73.75721902199986, 40.72610785399996], [-73.75698318299987, 40.72616032499988], [-73.75683174999993, 40.72619401699993], [-73.75677678599992, 40.72620833599996], [-73.75669771299987, 40.7262287109999], [-73.75670987299993, 40.72617907299986], [-73.75603126999992, 40.72559271799984], [-73.75581422999997, 40.725430010999965], [-73.75566637499988, 40.725319169999906], [-73.75548073699997, 40.72511980699992], [-73.75524449499997, 40.724783126999924], [-73.75508625099991, 40.72449005999985], [-73.75496919499987, 40.72415964699993], [-73.755035606, 40.72411532099988], [-73.75577120999998, 40.72364021599991], [-73.75546163099992, 40.723352762999916], [-73.75491106099987, 40.72284152399987], [-73.7546271559998, 40.72258094099987], [-73.75433064399982, 40.72230878599986], [-73.7532266449999, 40.721283649999876], [-73.75301295399987, 40.721099278999915], [-73.75281552300001, 40.720903624999885], [-73.75263582400002, 40.7206979659999], [-73.75247511899987, 40.72048373099989], [-73.7523343839998, 40.72026246099993], [-73.75313314299989, 40.71991531799997], [-73.75393063599992, 40.71957218199985], [-73.75474912799989, 40.71921669499994], [-73.75556553099986, 40.7188668699999], [-73.75638211900001, 40.718515726999954], [-73.75721025899989, 40.7181586389999], [-73.75725671499985, 40.71813860199992], [-73.75832728699997, 40.71958208899985], [-73.75839264999983, 40.71955392299988], [-73.75878322199986, 40.71938563499985], [-73.75921639499985, 40.71919899199987], [-73.7600295979999, 40.71884681499988], [-73.76084486799982, 40.71849351999986], [-73.76166150099981, 40.71814054599984], [-73.76247897300004, 40.717785614999904], [-73.763355018, 40.71741138199993], [-73.76437291199994, 40.71696662599988], [-73.765714472, 40.71639337899988], [-73.76664095199988, 40.717646613999875], [-73.76733228200001, 40.717289796999886], [-73.76817892099992, 40.71699299399994], [-73.76870434999991, 40.71680902899992], [-73.76906210699998, 40.71668376799988], [-73.769139141, 40.71665679899991], [-73.76922535799989, 40.716626587999876], [-73.77007984799988, 40.716327201999896], [-73.77110410799999, 40.71597506299987], [-73.77192122099994, 40.71568504399992], [-73.7727585929999, 40.715397220999904], [-73.77357135199996, 40.71512009499986], [-73.7744423819999, 40.71490504299985], [-73.77529927899995, 40.7146963139999], [-73.775714584, 40.714598454999916], [-73.77618628399998, 40.71448557399985], [-73.77712346999995, 40.71426199399986], [-73.77730633899989, 40.714209316999906], [-73.77800119599992, 40.714051123999894], [-73.77886957500004, 40.713849297999914], [-73.77972553599984, 40.71363576999988], [-73.7799800519999, 40.71357457099991], [-73.78042034099985, 40.71346870999985], [-73.7805889279999, 40.713428175999894], [-73.78178274, 40.71313999299988], [-73.78227126599994, 40.7130251299999], [-73.78236649999998, 40.71300274299994], [-73.782470433, 40.71297658099992], [-73.78262547499989, 40.712937547999914], [-73.78356075700003, 40.71271078799989], [-73.78446144599992, 40.71249390499992], [-73.78535272199989, 40.71228361699985], [-73.78575198499988, 40.7121639529999], [-73.78616005899983, 40.71206007099989], [-73.78657528699986, 40.71197249299993], [-73.78699598699988, 40.71190160899989], [-73.7874204069999, 40.71184767799992], [-73.78784678499987, 40.711810832999916], [-73.78843481199984, 40.71198468799988], [-73.78844455400004, 40.711987694999884], [-73.78845433099994, 40.71199065799992], [-73.78846414499994, 40.71199357499989], [-73.78847398199989, 40.71199643899991], [-73.788483855, 40.71199925699994], [-73.7884937509998, 40.712002022999854], [-73.78850368299993, 40.712004733999954], [-73.78851365099997, 40.71200739899987], [-73.78852363099992, 40.71201001999984], [-73.78853364699988, 40.712012577999914], [-73.78854369699997, 40.7120150909999], [-73.78855376099986, 40.71201755899993], [-73.78856385999993, 40.71201996399993], [-73.78857398199985, 40.71202232299989], [-73.78858412899999, 40.71202462999992], [-73.78859429999986, 40.71202688199987], [-73.78860448199985, 40.712029079999866], [-73.78861469999994, 40.71203123299989], [-73.78862492999987, 40.71203333199993], [-73.78863519600003, 40.71203536699988], [-73.78864546199993, 40.71203735799987], [-73.78865576299997, 40.712039294999904], [-73.78866607699987, 40.71204117799994], [-73.78867640200002, 40.71204300799991], [-73.78868675199999, 40.712044781999914], [-73.78869712499983, 40.712046502999904], [-73.78870749899997, 40.712048178999964], [-73.7887178959999, 40.712049792999885], [-73.78872830499998, 40.71205135099987], [-73.7887387379998, 40.712052855999964], [-73.78874917199984, 40.71205430699993], [-73.78875961699991, 40.712055695999936], [-73.78877008599987, 40.71205703799991], [-73.7887805549999, 40.71205832699993], [-73.78879103599994, 40.712059552999925], [-73.78880153, 40.71206073299987], [-73.78881202299988, 40.71206185199992], [-73.78882252899987, 40.7120629249999], [-73.78883304599987, 40.71206393399992], [-73.7888435639998, 40.7120648899999], [-73.78885408199986, 40.712065791999905], [-73.78886461199977, 40.71206663999991], [-73.78887515399994, 40.7120674249999], [-73.78888568399994, 40.71206816399993], [-73.78889622599979, 40.71206884099995], [-73.78890676899992, 40.712069472999914], [-73.78891729899986, 40.71207004099992], [-73.78892784199994, 40.71207055599987], [-73.7889383849999, 40.7120710159999], [-73.78894892899983, 40.71207142299989], [-73.78895945899997, 40.71207177499989], [-73.78897000299993, 40.71207207399986], [-73.78898053499987, 40.712072309999854], [-73.78899105499988, 40.71207249999985], [-73.78900157599998, 40.712072635999895], [-73.78901209599994, 40.7120727089999], [-73.78914879699997, 40.71205688599986], [-73.78928335099997, 40.712031728999946], [-73.78941412099978, 40.71199753199987], [-73.79021544199992, 40.71171704499989], [-73.79101410700002, 40.71143258599991], [-73.79202854699994, 40.71106937099994], [-73.79303800099989, 40.71071107499988], [-73.79300721999992, 40.7115494699999], [-73.79300604199986, 40.711765221999904], [-73.79302654199988, 40.711981363999875], [-73.79306872799985, 40.71219599599993], [-73.79313224999994, 40.71240723599992], [-73.79321635699988, 40.71261327999988], [-73.79350163599989, 40.71321339799988], [-73.79356328499998, 40.71335799599989], [-73.79360704399981, 40.713508469999894], [-73.79360705599986, 40.713508522999916], [-73.79360706699994, 40.713508577999924], [-73.79363166700001, 40.71366304199988], [-73.79363635499982, 40.71381920899989], [-73.79363636699993, 40.7138193969999], [-73.79363634199994, 40.71381958599994], [-73.79362091299997, 40.71397534699987], [-73.79358570299998, 40.714128677999874], [-73.7935316789999, 40.714277165999924], [-73.7935316439998, 40.71427725599991], [-73.7935315959999, 40.71427734599991], [-73.79346012399989, 40.714419191999895], [-73.79314903799985, 40.714933429999846], [-73.79268721599992, 40.71568003299994], [-73.79240413399991, 40.716193040999926], [-73.79234087599984, 40.71646287399992], [-73.79226766099997, 40.717283926999876], [-73.79224175599992, 40.71742717899984], [-73.7921849289999, 40.71761139399988], [-73.79210873799991, 40.71779211499996], [-73.79210870199984, 40.71779218799989], [-73.79201376599997, 40.71796768699994], [-73.7919010649999, 40.718136366999936], [-73.79173290199986, 40.7183199289999], [-73.79162293699987, 40.71842251499991], [-73.79150283700001, 40.71851882299991], [-73.79137357500004, 40.71860799199986], [-73.79123627699988, 40.71868931099988], [-73.79089607099996, 40.7188759379999], [-73.79076914099981, 40.71913418899989], [-73.79073248899982, 40.71920875699987], [-73.7907083439998, 40.71926865199988], [-73.79053210899994, 40.71975195999992], [-73.79050800399992, 40.71980649699987], [-73.79037661699995, 40.720103723999934], [-73.790302674, 40.7208914649999], [-73.79029346799997, 40.72137614899989], [-73.79035703899997, 40.72300529999988], [-73.79045149300003, 40.723554531999845], [-73.79088077299993, 40.72418885899991], [-73.79136702000001, 40.724951617999906], [-73.79147613699985, 40.725138464999894], [-73.79157252999995, 40.725329611999896], [-73.79165577199997, 40.72552430799988], [-73.7917255479998, 40.72572175499991], [-73.79174472699992, 40.725788849999944], [-73.7916922309998, 40.7257923849999], [-73.79085659799983, 40.72602748899991], [-73.78998042299995, 40.726317244999954], [-73.78964444799998, 40.72642499899994], [-73.78911280699997, 40.7266001679999], [-73.78825003899982, 40.726886972999935], [-73.78738339599995, 40.72717198099985], [-73.78655584699987, 40.727441503999906], [-73.78619017499992, 40.72756357399987], [-73.78568492799992, 40.72773019299991], [-73.78482734399994, 40.72801171399991], [-73.78397695399983, 40.728291396999964], [-73.78360140099988, 40.728412819999924], [-73.78292253099984, 40.7286359479999], [-73.78205851399987, 40.72891889699988], [-73.78116054999998, 40.72921275899992], [-73.78024774699992, 40.729459822999914], [-73.77936529399982, 40.72970186799997], [-73.77848636099995, 40.72994389499989], [-73.77757506899988, 40.73019327299987], [-73.7775479359998, 40.730136776999885], [-73.77672825799984, 40.72843008099995], [-73.77596177499996, 40.72681284899995], [-73.77556110500004, 40.72596986399986], [-73.7752338719998, 40.725237532999884], [-73.77634188999991, 40.72446564699988], [-73.7764331279998, 40.724303181999886], [-73.77654571399995, 40.72414504599983], [-73.77667910999996, 40.72399361299994], [-73.77683220599998, 40.7238512149999], [-73.7770032789999, 40.723720008999905], [-73.77719009799996, 40.723601881999905], [-73.77738999299997, 40.72349836699989], [-73.77759997899993, 40.72341057599986], [-73.77781693000003, 40.723339142999876], [-73.77803765999981, 40.723284268999855], [-73.7778760189999, 40.72303670299993], [-73.77774895399996, 40.72284210999989], [-73.77757868099984, 40.72264221799984], [-73.77738048899985, 40.72244906299987], [-73.77732562899998, 40.722392700999926]]]}}, {\"id\": \"131\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 132, \"Shape_Leng\": 0.24547851707, \"Shape_Area\": 0.00203830095472, \"zone\": \"JFK Airport\", \"LocationID\": 132, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.8250346749999, 40.663580133999844], [-73.82486060999993, 40.663576789999865], [-73.82427005699986, 40.663576563999904], [-73.82306169099988, 40.66353701599996], [-73.81791759599996, 40.662956015999875], [-73.81496922499986, 40.66247266799991], [-73.81442762999983, 40.662514101999896], [-73.81388758999984, 40.662569925999925], [-73.81335011199994, 40.662640078999836], [-73.81281616300001, 40.66272443599994], [-73.81228672799995, 40.66282283799994], [-73.81176694899993, 40.6629345349999], [-73.81116608799995, 40.663177473999895], [-73.81061044899995, 40.6634808539999], [-73.81006836999995, 40.66379225199996], [-73.80945926899989, 40.664044088999944], [-73.80884594199996, 40.66429766899991], [-73.8084501609999, 40.664461300999946], [-73.80779777599999, 40.66458900799989], [-73.80764810699988, 40.6646183059999], [-73.80614396, 40.664912729999855], [-73.8057239779999, 40.664960640999894], [-73.8043205059999, 40.66512073299992], [-73.80377524499997, 40.66516018399992], [-73.80299527199993, 40.6652166109999], [-73.80214866500003, 40.66524995299992], [-73.80203207699995, 40.66524802499994], [-73.80188667499992, 40.665248916999886], [-73.80180049599981, 40.66525044099989], [-73.80072889200001, 40.66526937699988], [-73.79983020599992, 40.665221051999936], [-73.79785485699999, 40.665114804999895], [-73.79762708599996, 40.66510255299991], [-73.79702409499983, 40.66506588399988], [-73.79642246799989, 40.66501610199989], [-73.79589506299997, 40.66496081399986], [-73.79582296499997, 40.66495325499988], [-73.7952263039999, 40.66487742499987], [-73.79394317799995, 40.664702387999924], [-73.79256433099995, 40.66451427599985], [-73.79199464699985, 40.664439276999865], [-73.79118919099993, 40.66433323399988], [-73.79057832499984, 40.6642432999999], [-73.79005979999995, 40.66416690799991], [-73.7899673679998, 40.664153288999934], [-73.78990759999994, 40.66414453799993], [-73.78984347199989, 40.6641359919999], [-73.78811679799986, 40.6639058809999], [-73.78751177999993, 40.663825242999906], [-73.78618470399978, 40.66360955299993], [-73.78502513799997, 40.66342107199989], [-73.78481455499993, 40.66349858199991], [-73.784349205, 40.663402598999916], [-73.78321833499997, 40.663166283999864], [-73.78211253799985, 40.66283540499988], [-73.78140184699991, 40.66262274199985], [-73.78128320799988, 40.66258308299995], [-73.780755286, 40.66240660199992], [-73.77932933099991, 40.66183951999991], [-73.77876727900004, 40.66161599499987], [-73.77871548099994, 40.66159539999994], [-73.77864409099983, 40.66156701299997], [-73.77849144899989, 40.661506302999875], [-73.776792695, 40.66074400799986], [-73.77670850699995, 40.660706229999946], [-73.77644444499985, 40.66062997399993], [-73.77560556199992, 40.66038771299989], [-73.77552237699994, 40.66036546999993], [-73.7748492269999, 40.66014868699996], [-73.7745200739998, 40.660012414999905], [-73.77434521599992, 40.659932140999906], [-73.77418923599993, 40.6598605239999], [-73.77388075599985, 40.6596788949999], [-73.77382964799997, 40.65965115799987], [-73.7737837049999, 40.65962415999994], [-73.77331158599986, 40.65934671699995], [-73.77305501099993, 40.6591997509999], [-73.77291920299982, 40.65912940399993], [-73.7704997779999, 40.657876039999884], [-73.76976145199995, 40.657482235999915], [-73.76877799799986, 40.65694558799989], [-73.7687316869999, 40.65692006399991], [-73.768070047, 40.6565553269999], [-73.76786196699993, 40.65644061399993], [-73.7677137589999, 40.656358979999915], [-73.76740737399983, 40.65619021499988], [-73.76735274299986, 40.65616012599987], [-73.76729260999991, 40.65612444299994], [-73.76683051599994, 40.65587605299992], [-73.76669593900003, 40.655802847999894], [-73.766280083, 40.65557821499993], [-73.76566897099984, 40.65525346899986], [-73.76475455399994, 40.654746896999875], [-73.76417770199993, 40.65442150099993], [-73.76406290099993, 40.65435626599989], [-73.7579393749999, 40.6508533029999], [-73.75465758099988, 40.64901209699987], [-73.7541108669999, 40.648697664999915], [-73.75416443199985, 40.648654094999856], [-73.7545491879999, 40.64834992799991], [-73.75483677999976, 40.64807090299996], [-73.75502113300001, 40.6478138189999], [-73.75522386899995, 40.64744446899989], [-73.75460006764922, 40.6472007730313], [-73.75379289369529, 40.64670560418293], [-73.7536600374918, 40.64687113919082], [-73.75376706809165, 40.646930780708104], [-73.7537377082634, 40.64696042357786], [-73.75340189507166, 40.646807461209285], [-73.75343611609503, 40.64678154246174], [-73.75361134104453, 40.646856182256506], [-73.75376146361373, 40.646686322658], [-73.75057032811948, 40.64472855667385], [-73.75048503032512, 40.64461845248783], [-73.75032661179766, 40.64446083625524], [-73.75014428585494, 40.644350168634816], [-73.7490763129122, 40.64363287005161], [-73.74900238843753, 40.6435592661201], [-73.74892145185547, 40.64349007456676], [-73.74883395552305, 40.64342568208787], [-73.74874038845483, 40.64336644855764], [-73.74870930098776, 40.64334179222194], [-73.74863060917853, 40.64319138890303], [-73.7485454659819, 40.64304303842062], [-73.74845396356635, 40.64289690129079], [-73.74835620097792, 40.642753135633406], [-73.7482522840352, 40.64261189700159], [-73.74814232521368, 40.64247333821328], [-73.7480264435227, 40.642337609186114], [-73.74777740949662, 40.64208880289097], [-73.74771698986176, 40.6419549062034], [-73.74765147633876, 40.64182240679818], [-73.74758092454391, 40.64169141709597], [-73.74757643258339, 40.641683429164935], [-73.74757626864935, 40.64168199877661], [-73.74756505759332, 40.64162283863933], [-73.74754538897116, 40.641564973459914], [-73.74751750052548, 40.64150910261509], [-73.74748172934405, 40.641455901375764], [-73.74743850778465, 40.64140601274523], [-73.7474246205621, 40.6413764680137], [-73.74741689624642, 40.64134564309854], [-73.7474155205471, 40.641314279133795], [-73.74742052653178, 40.64128313021307], [-73.7474317938311, 40.64125294525846], [-73.74744905153455, 40.64122445001427], [-73.74747188470438, 40.641198329597636], [-73.747499744352, 40.64117521202666], [-73.74751181131329, 40.64117317701244], [-73.74752327225228, 40.6411696535371], [-73.74753381313113, 40.64116473814625], [-73.74754314512361, 40.64115856552464], [-73.74755101252694, 40.64115130480637], [-73.74755719976996, 40.64114315493975], [-73.74756153731882, 40.641134339236174], [-73.74756390632288, 40.64112509925161], [-73.74756424187052, 40.64111568816709], [-73.74756253476885, 40.64110636385221], [-73.74755883179367, 40.64109738179864], [-73.7475532344098, 40.64108898811972], [-73.74754589598956, 40.641081412807104], [-73.74753701760997, 40.64107486342859], [-73.74752684254419, 40.641069519440876], [-73.74751079261682, 40.64042050436142], [-73.74750206113116, 40.64039207545248], [-73.74748761968836, 40.64036503061832], [-73.74746782126103, 40.64034003086601], [-73.74744289570371, 40.6403171311831], [-73.74744927347035, 40.64024137008906], [-73.74746824072419, 40.640162967688774], [-73.7474802076455, 40.640083767972826], [-73.74748511952103, 40.6400041332127], [-73.74748295390837, 40.639924427668234], [-73.7474874644855, 40.63988641791955], [-73.74748529534024, 40.639848288971436], [-73.74747648506289, 40.63981071897193], [-73.74746119036125, 40.63977437612688], [-73.7474524320164, 40.63975912719788], [-73.74744708612687, 40.6397429884568], [-73.74744529835942, 40.63972639966524], [-73.7474471174258, 40.639709812848274], [-73.74745249375627, 40.63969367997683], [-73.74746128085008, 40.63967844065174], [-73.74747323926697, 40.639664510125726], [-73.74748804315348, 40.639652267988176], [-73.74754928202161, 40.63952932464898], [-73.74757191274122, 40.63948389251257], [-73.74755806224371, 40.639441934755645], [-73.74754979183439, 40.63939729798184], [-73.74754790379832, 40.63935224174217], [-73.74755242024823, 40.63930729403572], [-73.74756328824401, 40.63926298158842], [-73.7475803804156, 40.63921982368134], [-73.74762024328325, 40.639074740467834], [-73.74763143903947, 40.63901419286407], [-73.74761811035013, 40.63894040320511], [-73.74763057301057, 40.638906263824076], [-73.74766449288337, 40.638832026182584], [-73.74767582401284, 40.638800446336695], [-73.74768040032282, 40.63877654041844], [-73.7477147538997, 40.638734489749936], [-73.74774229930892, 40.638701131128784], [-73.74772606071858, 40.6386447015041], [-73.74772609744177, 40.63855285020455], [-73.74777500158162, 40.63837016658211], [-73.74777952874075, 40.63835907267909], [-73.74778518436705, 40.63834627295628], [-73.74781225191403, 40.638304478008266], [-73.74782128750516, 40.63828741420521], [-73.74782695771786, 40.63827034352352], [-73.74782811200754, 40.63826180424532], [-73.7478292624093, 40.63825411918972], [-73.74782815720052, 40.638249845274956], [-73.74782705455601, 40.63824471830709], [-73.74781703473738, 40.638225904921654], [-73.74781593209414, 40.63822077795377], [-73.74781593855258, 40.63821906900296], [-73.74781594823676, 40.63821650649802], [-73.74781949759755, 40.6381678276257], [-73.74782995639367, 40.63807047557322], [-73.74783570405538, 40.63803290418112], [-73.7478470717812, 40.63799193120316], [-73.74786067163458, 40.63795352229415], [-73.7478957798794, 40.63786134933289], [-73.74791053393895, 40.63781440398595], [-73.74792988195635, 40.63773842534734], [-73.74794246705504, 40.637671827922034], [-73.74794595925108, 40.63763766883913], [-73.74794514917606, 40.637555667897225], [-73.74794747163065, 40.63753431904881], [-73.74795090962952, 40.637514680940555], [-73.7479702100006, 40.637451515490376], [-73.74799513204617, 40.63738409086547], [-73.74802679707516, 40.637312410691884], [-73.74802908555866, 40.63730045713858], [-73.74803137572476, 40.63728765002759], [-73.7480315660138, 40.63723725431574], [-73.74803388651996, 40.63721676069721], [-73.74803843870359, 40.63719883231716], [-73.74804523977623, 40.63717920157467], [-73.74808031511408, 40.637095569786695], [-73.74809279075087, 40.637058013788106], [-73.74809966015668, 40.63702044751698], [-73.74811006443797, 40.63693761472843], [-73.74811692671953, 40.6369017552278], [-73.748124885848, 40.63687273164474], [-73.74812714762473, 40.63686761204869], [-73.74814391801104, 40.63686544577316], [-73.74816342022596, 40.63686285291793], [-73.748181958921, 40.63685800836739], [-73.74821198773496, 40.6368446257198], [-73.74826747018606, 40.63680744481219], [-73.7483634697079, 40.63671743343141], [-73.74839815536178, 40.63665363731989], [-73.74840924794756, 40.636483319414666], [-73.74842848625192, 40.63630184797762], [-73.74845687452745, 40.63612135833602], [-73.74848582725184, 40.635982095170284], [-73.74846407433594, 40.63595078378952], [-73.74846487933117, 40.635861662526075], [-73.74849297657127, 40.63567172844184], [-73.74854241848169, 40.63555460734049], [-73.74857577209558, 40.635500250370015], [-73.74860324379861, 40.635457326293164], [-73.74861889305822, 40.635444679550616], [-73.7486942175569, 40.63542939097165], [-73.7487349724278, 40.635418254406595], [-73.74876275470102, 40.63538847916552], [-73.74877979914012, 40.63533040956812], [-73.74880661033536, 40.63529097044979], [-73.74884578400585, 40.63525995690306], [-73.74900204451106, 40.63519236434149], [-73.74908166657815, 40.63514576731353], [-73.74915016047596, 40.635125988992634], [-73.74922842905895, 40.635103734088574], [-73.7492926711699, 40.63508419401817], [-73.74933539830448, 40.63505642857014], [-73.74936511437703, 40.63502989573867], [-73.74939809033248, 40.63500833858589], [-73.749477009468, 40.63498506936848], [-73.74955459566455, 40.63498120310817], [-73.74969316885283, 40.63494162334695], [-73.74980600508579, 40.63491519006314], [-73.74991001947673, 40.63488722670299], [-73.74995536558899, 40.634871653633915], [-73.74997722438475, 40.63485578019461], [-73.75000235066122, 40.63483247773605], [-73.75010474833118, 40.634803275492075], [-73.75017390727064, 40.634788713694405], [-73.75030816891771, 40.63479408263555], [-73.7504457445215, 40.63477088583465], [-73.75052267929836, 40.634758070442544], [-73.75062836415057, 40.63471646949397], [-73.75073275582511, 40.63467439740074], [-73.75078821189561, 40.63464342016499], [-73.75080781850349, 40.634622327172316], [-73.75081926856258, 40.634604949888946], [-73.75082743535218, 40.63458882933], [-73.75086407034361, 40.63456154197828], [-73.75099646021218, 40.634479209709596], [-73.75112682317871, 40.634399866480955], [-73.75118128290848, 40.63437525367938], [-73.75125688143288, 40.63433410688712], [-73.7513205615511, 40.63432002726201], [-73.75133389361284, 40.63431969858779], [-73.75137625183444, 40.6342840535226], [-73.75143514452971, 40.63424937732433], [-73.75152087297442, 40.63420660594698], [-73.75154329427396, 40.63417492453081], [-73.75155838145801, 40.634138478815466], [-73.751574773625, 40.63410005957541], [-73.75159882339658, 40.63408523527375], [-73.75166205100028, 40.63407661425942], [-73.75186213641547, 40.634004283261596], [-73.75193817967717, 40.63398885589988], [-73.7520461881115, 40.63395181495419], [-73.75210021112169, 40.63391830765483], [-73.75214715284747, 40.63388074962219], [-73.75218878170351, 40.633847325965625], [-73.75221291779746, 40.63382879513763], [-73.75228483426119, 40.63380863851779], [-73.75235508487731, 40.63379092234884], [-73.75240933538956, 40.633745035822315], [-73.75245502470679, 40.6337045100348], [-73.75251235529672, 40.63368292316824], [-73.75260211764352, 40.63364984888782], [-73.75265631341018, 40.63363764186706], [-73.75275631212412, 40.63364230349722], [-73.75296000940244, 40.633645045856746], [-73.7530689994549, 40.63362458525472], [-73.75320617450369, 40.63359158484089], [-73.75341601415487, 40.633550589152925], [-73.75349765803799, 40.633530535060075], [-73.75351332531253, 40.63351239859552], [-73.7535198771118, 40.63349440676723], [-73.75353303999954, 40.63347154228246], [-73.75353932880935, 40.63346592946119], [-73.75356385552718, 40.63345879067186], [-73.75361241275372, 40.6334512098324], [-73.753690614823, 40.63342667419207], [-73.75375784378504, 40.63339777766772], [-73.75377694706003, 40.63338538405601], [-73.75377660737931, 40.63337028693528], [-73.75376232723436, 40.633354007192665], [-73.75376740764654, 40.63334416534382], [-73.75379664693634, 40.633328937731505], [-73.75387867256455, 40.63329280020785], [-73.753957421808, 40.63325660016266], [-73.75410574377158, 40.63319029409778], [-73.75411400945772, 40.63318598265151], [-73.75414992118378, 40.633168987377346], [-73.7541805178403, 40.63315713979144], [-73.75422279870031, 40.633141886175004], [-73.75427674329964, 40.63312901912947], [-73.75430506875782, 40.6331177156499], [-73.75434172916353, 40.63310245165502], [-73.75435569788613, 40.633095646976315], [-73.75438552708675, 40.63308635033907], [-73.75441789324101, 40.63308309720109], [-73.75445135571276, 40.63308531069578], [-73.75450787357656, 40.63307845965082], [-73.75455160892948, 40.633069550000094], [-73.75458063586557, 40.633062832387445], [-73.75460136335478, 40.63305000401671], [-73.75460998461537, 40.633037367761354], [-73.75462602651683, 40.63301525262286], [-73.75464388538201, 40.632979277813355], [-73.75467219593482, 40.63295183595382], [-73.7547257494144, 40.632918298945256], [-73.75476459598178, 40.63289647886701], [-73.75479180499259, 40.63288465136806], [-73.75483158903131, 40.63287268599521], [-73.75490324828398, 40.63286314904655], [-73.75492844934085, 40.632858510269784], [-73.75497423365873, 40.632850400924596], [-73.75498836788584, 40.63284700004566], [-73.75499944126739, 40.63284433429466], [-73.75504634040247, 40.632837378389645], [-73.75507496607389, 40.63283192347043], [-73.75510630943813, 40.63282246519053], [-73.75512626252157, 40.63281430126728], [-73.75514208750556, 40.63280233970864], [-73.75516622495809, 40.632782931583726], [-73.75517970307233, 40.63277195310988], [-73.75520250803841, 40.63274309885272], [-73.75521515399498, 40.63273327153914], [-73.75523192286363, 40.63272883370477], [-73.75526060382325, 40.63272837323832], [-73.7552825272304, 40.63273344225974], [-73.7552940433308, 40.63273656840546], [-73.75531029075492, 40.63273712450136], [-73.75532756649883, 40.632741799981346], [-73.75533891369847, 40.63275230978138], [-73.75535700715443, 40.63276996933914], [-73.75538503946642, 40.63277897705101], [-73.7554190855173, 40.632779186683756], [-73.75546104350131, 40.63277364912333], [-73.75550007253388, 40.6327609973894], [-73.75552439566636, 40.63274987725503], [-73.75555446417502, 40.6327246890014], [-73.7555974037463, 40.63269645393822], [-73.7556138803065, 40.632674009050696], [-73.75561450693363, 40.632650433157], [-73.75561676373152, 40.6326244990687], [-73.75564077717657, 40.63260953546596], [-73.75566085845692, 40.632595992023965], [-73.75568013691804, 40.6325850805996], [-73.75569749684557, 40.63258681924811], [-73.75575077877197, 40.63257762871056], [-73.75577188570304, 40.63256971305933], [-73.75578297837113, 40.63256073427027], [-73.75579550007, 40.63255532504481], [-73.75581905849747, 40.63255658329881], [-73.75584751064262, 40.63254934212479], [-73.75585893932731, 40.63255688644429], [-73.75587483581369, 40.63255508173109], [-73.75589579130344, 40.63253890503703], [-73.7559253565285, 40.63253287493039], [-73.75597648108734, 40.63253248822788], [-73.75600263133546, 40.63252433680228], [-73.75603621696222, 40.63251246876385], [-73.75605957824654, 40.63250840113113], [-73.75607599598159, 40.632501573964625], [-73.75610493061326, 40.63249082083037], [-73.75613095727087, 40.632487115674245], [-73.75615773263061, 40.63248516931406], [-73.7561745005988, 40.63248070286042], [-73.75620434134115, 40.63247851538164], [-73.75622523370636, 40.63247941055959], [-73.75626139108071, 40.63247377748211], [-73.75628789837182, 40.63246653308169], [-73.75631528510678, 40.63245517283825], [-73.75633067386859, 40.63244419811154], [-73.7563395049462, 40.63243282651429], [-73.75634600028775, 40.63242054413628], [-73.75635669373776, 40.63241216771853], [-73.75637032315481, 40.632409423822445], [-73.7563867619205, 40.63241675924196], [-73.75639362639575, 40.632421000715716], [-73.75639978351029, 40.6324219750433], [-73.75642748681958, 40.63241297593068], [-73.75644827666468, 40.63240267202473], [-73.75648885226455, 40.63239062600158], [-73.75655350205051, 40.63237629775954], [-73.75660686882948, 40.63236414207318], [-73.7566443505458, 40.63235085308022], [-73.75666563107961, 40.632341473092126], [-73.75670392809458, 40.63233266369388], [-73.75673983597723, 40.63229150017247], [-73.75679202591738, 40.632264457943805], [-73.75686057294176, 40.632262514161894], [-73.75694995943626, 40.6322565300303], [-73.75697491342238, 40.63225013256656], [-73.75700052169095, 40.632232839541636], [-73.75701324998789, 40.63222004710484], [-73.75703757806313, 40.632207445535144], [-73.75706992673652, 40.632198813423784], [-73.75711856482872, 40.63218925392883], [-73.75713166934715, 40.632191504890415], [-73.75714083769348, 40.63219665818928], [-73.75714799946306, 40.63218912511935], [-73.75715865681255, 40.63218072107617], [-73.75716688504453, 40.6321773074557], [-73.75717249301164, 40.63217062097838], [-73.75717969544705, 40.632161579391656], [-73.75719252775905, 40.63216001446085], [-73.75721744562729, 40.63215361569562], [-73.7572517803673, 40.63214353189771], [-73.7573012565386, 40.63213134020055], [-73.75734568131885, 40.63212081210866], [-73.75738780818773, 40.63210791698879], [-73.75742398540433, 40.632086639373746], [-73.75745388137142, 40.632068833775385], [-73.75747297457133, 40.63204966045865], [-73.75748358593373, 40.6320144402201], [-73.75749827965645, 40.63198606194012], [-73.75752947030263, 40.63194937549313], [-73.75754868671227, 40.63192581014227], [-73.75757274652175, 40.631907882474536], [-73.75763936245248, 40.63187796733063], [-73.75766174651459, 40.631865334056464], [-73.75768102421527, 40.63185442296525], [-73.75770658938339, 40.63183861249353], [-73.75773524825536, 40.63182401508024], [-73.75776671718441, 40.63181947014023], [-73.75781407545027, 40.63181460182421], [-73.75786233850928, 40.631818299153224], [-73.75790341524566, 40.63181687604509], [-73.75795975654852, 40.631808897472354], [-73.75799870111251, 40.631799180495484], [-73.75804316756229, 40.631787196393134], [-73.75808862169123, 40.631780814853755], [-73.75814094243319, 40.6317787016428], [-73.75818551546642, 40.631776434907515], [-73.75822016631851, 40.63176871197242], [-73.75826666624022, 40.63175231464911], [-73.75831639327441, 40.63172980089709], [-73.75837390268825, 40.63170738757813], [-73.75840800529826, 40.63169200651097], [-73.75843471865126, 40.63167737685817], [-73.75845963000812, 40.63164268061586], [-73.75848512889073, 40.63161564302819], [-73.75851596563963, 40.631576594413254], [-73.75854646556999, 40.63155082931299], [-73.75856752919167, 40.63154437005996], [-73.75862267398684, 40.63153814536175], [-73.75868393990363, 40.63153291989644], [-73.758741704338, 40.631529966956684], [-73.75878043724416, 40.631529088113346], [-73.75884997041852, 40.63151896844933], [-73.75897247001858, 40.631497266918664], [-73.75900906659348, 40.631489575299604], [-73.75904134603263, 40.63146974316016], [-73.75907180593221, 40.63144543366575], [-73.75910055263272, 40.6314264170106], [-73.7591293759085, 40.631405919382615], [-73.75915871325151, 40.631393107222486], [-73.75918672601648, 40.63138795138194], [-73.75922323824433, 40.63138319697572], [-73.75925753087604, 40.63137459462253], [-73.75928928927192, 40.63135975687647], [-73.75931051807694, 40.63134739594609], [-73.7593225377309, 40.631331363430185], [-73.759332330336, 40.63131293895996], [-73.75934852999643, 40.63128665017219], [-73.75937603869771, 40.631241061223434], [-73.75940636732578, 40.6311928992238], [-73.759428283806, 40.63116964109625], [-73.75945154587143, 40.63115289160992], [-73.75948286372973, 40.63114011199057], [-73.75953158196724, 40.6311275883868], [-73.75955940732992, 40.63111416851668], [-73.75958992784098, 40.631102541142184], [-73.75961840113031, 40.631079680497585], [-73.75968491183559, 40.631038564620276], [-73.75980279920832, 40.63098671522968], [-73.75983248858583, 40.630976264119965], [-73.75986776277361, 40.6309747452635], [-73.75988793308467, 40.630986563687856], [-73.75990440716355, 40.63099387237703], [-73.7599302662337, 40.63099606622411], [-73.75994663679194, 40.63099217586444], [-73.75997858596594, 40.63098411739689], [-73.76001141454829, 40.63097194473035], [-73.76006058703396, 40.63094322696733], [-73.7600900076537, 40.63092745081629], [-73.76010910058119, 40.630910620627645], [-73.76012253442501, 40.630898779544374], [-73.7601621667395, 40.630878139526786], [-73.76019214398727, 40.63085737012922], [-73.7602137700301, 40.63084445868934], [-73.76025371792642, 40.63082617928269], [-73.76027525726276, 40.63081768841192], [-73.76031691743549, 40.6307941425862], [-73.76034398715748, 40.630780419566065], [-73.76037384025314, 40.630764068446034], [-73.76041692849454, 40.63074409376548], [-73.76044221810595, 40.63072443932604], [-73.76045987972935, 40.630701693152545], [-73.76050547757657, 40.63067527395926], [-73.76053091824248, 40.63066388118418], [-73.76056937547406, 40.63065915787683], [-73.76061342699172, 40.630661911474704], [-73.7606793594872, 40.63067118946043], [-73.76069898019995, 40.63067531951988], [-73.7607192411912, 40.63067149092445], [-73.76075328608108, 40.630671727625064], [-73.7607663903239, 40.63067400614984], [-73.76077763490368, 40.63067328789428], [-73.7607882515007, 40.630666365058374], [-73.76079871707456, 40.63065120868321], [-73.76082264621186, 40.63063918245398], [-73.76085325393692, 40.630623160466435], [-73.76088465534657, 40.63060741520441], [-73.7609120400946, 40.63059605436556], [-73.76094128812467, 40.630587661153925], [-73.7609679127443, 40.63057745098741], [-73.760998356, 40.63056730303197], [-73.7610258651621, 40.630551523338426], [-73.76105643152373, 40.6305369570399], [-73.76109585282921, 40.630525181406725], [-73.76112982898906, 40.63051418937378], [-73.76116741261265, 40.63051212709648], [-73.7612079670496, 40.63051569778118], [-73.7612312271107, 40.63051947897371], [-73.76125670903987, 40.6305263121318], [-73.76128694367084, 40.63052412247826], [-73.76131227686686, 40.630512291218324], [-73.76134619270204, 40.630498417995994], [-73.76138921621218, 40.63048596303698], [-73.76143644071634, 40.63048833976526], [-73.76149481082305, 40.63047652055118], [-73.76151721749568, 40.63046682285876], [-73.76154609974837, 40.63044986514292], [-73.76158384549086, 40.630442971522584], [-73.76162817878301, 40.630447455725665], [-73.76167726317362, 40.63044291876145], [-73.76170878035607, 40.630434639287344], [-73.76173894571001, 40.63041161861906], [-73.76176520181093, 40.63038373149479], [-73.7617851400165, 40.63035884948161], [-73.76180536131753, 40.630346073713945], [-73.76184133173781, 40.630341782918606], [-73.76188743880134, 40.6303436905783], [-73.76191993967197, 40.63034238548273], [-73.76196692560318, 40.63033043305897], [-73.76198301778457, 40.63032368730292], [-73.76202016617357, 40.630292500899024], [-73.76202356486533, 40.63026868309858], [-73.76204329334408, 40.63025244612014], [-73.76206325149165, 40.6302527070551], [-73.76208184709208, 40.63026120006665], [-73.76209586454806, 40.63027006741182], [-73.76211449776119, 40.630278559825754], [-73.7621476220663, 40.63027420926925], [-73.76219292080067, 40.630260496048656], [-73.76220794059698, 40.63025114045051], [-73.76222090635459, 40.63023222758297], [-73.76225562918488, 40.63018363415494], [-73.76227486437197, 40.63016393881594], [-73.76229285501617, 40.63015027949212], [-73.76233939251507, 40.63013267221251], [-73.76238520005512, 40.630097249184296], [-73.76243265991498, 40.630063586263475], [-73.76252560314214, 40.6300348761254], [-73.76259133473147, 40.62999971262329], [-73.7626484031195, 40.629968786777525], [-73.76269869800606, 40.62993735338237], [-73.76275193784853, 40.6298994203793], [-73.7628074786225, 40.62986239725879], [-73.76285765139592, 40.62983527289683], [-73.76289946189033, 40.629799374850414], [-73.76295388724613, 40.62976191076733], [-73.76302107334124, 40.62971201096331], [-73.76308661984865, 40.629660374819025], [-73.76310569745239, 40.6296552895763], [-73.76312253435842, 40.62964934397451], [-73.76312545290956, 40.62964861101601], [-73.76315467374808, 40.62967816842226], [-73.76315924098999, 40.62967942944156], [-73.7631639573132, 40.629680319690515], [-73.7631687699776, 40.62968082984631], [-73.76317051492023, 40.62968091955061], [-73.76320641608257, 40.62967569488079], [-73.76321681421055, 40.629673585826204], [-73.76322696470938, 40.62967086628676], [-73.7632368103359, 40.62966755658323], [-73.76324628813578, 40.62966367518182], [-73.76325534239558, 40.6296592450868], [-73.76326391914132, 40.62965429466728], [-73.76331144955105, 40.629630595665496], [-73.76335594386991, 40.629610073283146], [-73.76336834851364, 40.62960398282405], [-73.76338129973229, 40.62959859342582], [-73.76339472970923, 40.62959393074778], [-73.76340591299954, 40.62959071709631], [-73.76341748134074, 40.62958842648459], [-73.7634293033663, 40.629587085111666], [-73.76344008585049, 40.62958577929286], [-73.76345097300862, 40.62958515385705], [-73.76346189115502, 40.629585215353835], [-73.76347276641425, 40.62958596178847], [-73.76348352315705, 40.62958738982226], [-73.76350122621838, 40.629586700501484], [-73.76351895188444, 40.62958689226078], [-73.76353662363248, 40.629587965612906], [-73.76357056393655, 40.6295884597631], [-73.76357653988008, 40.62958826733082], [-73.7635885734355, 40.62958568448226], [-73.76360036889726, 40.62958252970071], [-73.76361187960619, 40.629578814952374], [-73.76361783278999, 40.629576729055785], [-73.76362339454302, 40.629574092355696], [-73.76362847896975, 40.62957094404483], [-73.76363085242582, 40.629568653773795], [-73.7636325543882, 40.62956603979584], [-73.76363351189114, 40.62956321571355], [-73.7636336855933, 40.62956030055934], [-73.76363306624754, 40.62955742163632], [-73.76361749961579, 40.62952167440964], [-73.76361645089808, 40.62951871385463], [-73.76361731235069, 40.6295157540438], [-73.76361890955845, 40.629512978352516], [-73.76362117926836, 40.62951048968119], [-73.76362403802558, 40.62950838318238], [-73.76362737759139, 40.62950673603195], [-73.76363107151938, 40.629505613306435], [-73.76363498092074, 40.629505054256896], [-73.76367871368474, 40.62949586672036], [-73.76373256767408, 40.62948730441171], [-73.76379272175978, 40.62946983442432], [-73.76380605564347, 40.629462982228844], [-73.76381987889455, 40.62945671789198], [-73.76383414592328, 40.62945106226117], [-73.76386912947355, 40.629440758693676], [-73.76392372816082, 40.62942517064356], [-73.76398294861052, 40.62940698379244], [-73.7640411873824, 40.62939099187456], [-73.76406406519877, 40.62938459079084], [-73.76408655232912, 40.62937742630796], [-73.7641086018728, 40.629369516590415], [-73.76413017113231, 40.62936087244031], [-73.76416593843608, 40.629347535263314], [-73.76420201687426, 40.62933469392969], [-73.7642383939009, 40.62932235209914], [-73.76427505806674, 40.629310514271104], [-73.76431199924436, 40.62929918427782], [-73.76432562415123, 40.62929514007378], [-73.76433891715781, 40.629290497946045], [-73.76435183378379, 40.629285275729266], [-73.76436432867723, 40.6292794889109], [-73.76437636152289, 40.629273158851916], [-73.76441300752685, 40.62925075403588], [-73.76441782243766, 40.62924800536531], [-73.76442309937791, 40.629245804456716], [-73.76442873242357, 40.6292441959906], [-73.76443460887138, 40.629243214414835], [-73.76444060991837, 40.62924287757899], [-73.764446611526, 40.62924319176268], [-73.76445623866347, 40.629244123347604], [-73.76446594001445, 40.62924433805846], [-73.76447562277858, 40.62924383805078], [-73.76448519635906, 40.629242624144126], [-73.76449457143279, 40.62924070989279], [-73.7644978859462, 40.629236805376806], [-73.76450184476276, 40.62923326555135], [-73.76450637708541, 40.62923015074945], [-73.76451140773813, 40.62922751610168], [-73.76451684574849, 40.62922540665455], [-73.76452260238051, 40.62922385673713], [-73.76452857462218, 40.62922289612643], [-73.76453466017286, 40.629222539690275], [-73.76454167049532, 40.62922469775129], [-73.76454827217987, 40.62922750381578], [-73.76455436224664, 40.62923091595819], [-73.76455984721007, 40.62923488138283], [-73.76456463956274, 40.62923933557967], [-73.76456866544093, 40.62924421088292], [-73.76457727785248, 40.62925092415933], [-73.76458656107263, 40.629257096516405], [-73.76459645873051, 40.6292626881341], [-73.76460690541754, 40.6292676655403], [-73.76461783858255, 40.62927199560319], [-73.76462918685293, 40.62927565237709], [-73.76469601826966, 40.62930546044355], [-73.76471169033192, 40.629312262875686], [-73.764726844702, 40.62931971141195], [-73.76474143813013, 40.629327786865254], [-73.76475542541905, 40.62933646099893], [-73.76478875600937, 40.629357999067466], [-73.76482281837222, 40.62937886113491], [-73.76485759145058, 40.62939903241494], [-73.76487361499012, 40.629406068241884], [-73.76489002235603, 40.62941257127074], [-73.76490678237407, 40.62941852719776], [-73.76492386517279, 40.629423926580145], [-73.76493408716709, 40.62942584960647], [-73.76494449524783, 40.629427068719515], [-73.76495500237463, 40.62942757385679], [-73.76496552940306, 40.629427360501], [-73.76497598902309, 40.629426432661795], [-73.76498629546414, 40.62942479418455], [-73.76499389199198, 40.62942172181311], [-73.76500106538414, 40.6294181072894], [-73.7650077554797, 40.62941398784963], [-73.76501389332667, 40.62940939953896], [-73.76501942886131, 40.62940438464041], [-73.76502431003084, 40.62939898844781], [-73.76502848828177, 40.62939326145599], [-73.76503192978599, 40.62938725636768], [-73.76503423343196, 40.629382482304045], [-73.76503574897981, 40.62937752603335], [-73.76503645570155, 40.629372466754475], [-73.76503634101641, 40.62936738033252], [-73.76503540620031, 40.629362341655685], [-73.76503366680842, 40.629357429326824], [-73.7650311478655, 40.62935271794774], [-73.76502789002711, 40.62934827679081], [-73.76502394010535, 40.629344175141384], [-73.76501776724778, 40.62933809981014], [-73.76501099847133, 40.62933240589728], [-73.76500367346122, 40.62932712581705], [-73.76499583652024, 40.629322292160474], [-73.76498753350214, 40.62931793400401], [-73.76497881357875, 40.62931407506937], [-73.76496973075218, 40.62931074093152], [-73.7649603348704, 40.629307950455434], [-73.76494850993906, 40.62930414615166], [-73.76493707425986, 40.62929970368582], [-73.76492609175202, 40.62929464228731], [-73.7649156135491, 40.629288990708346], [-73.76490570001685, 40.62928277822219], [-73.7648610124392, 40.62925630986229], [-73.76485148051162, 40.62924957278026], [-73.76484141524753, 40.629243303856], [-73.76483085393808, 40.629237528630135], [-73.76482225027127, 40.62923194621304], [-73.76481302413637, 40.629226978527065], [-73.76480324664354, 40.62922266341147], [-73.76479299967764, 40.62921903856024], [-73.76478236428618, 40.62921612977116], [-73.76477217068526, 40.629212524795776], [-73.76476247394531, 40.629208195601926], [-73.76475336672318, 40.62920318091118], [-73.76474492713982, 40.629197525781215], [-73.7647417478057, 40.62919496227704], [-73.76473917644952, 40.62919202776671], [-73.76473728929172, 40.62918880751127], [-73.76473613460911, 40.629185391740236], [-73.76473574724314, 40.62918187702178], [-73.76473613849734, 40.62917836289142], [-73.764737297029, 40.62917494800066], [-73.76474154826337, 40.62916979093531], [-73.76474640438602, 40.6291649519055], [-73.76475182126312, 40.62916047521576], [-73.76475775699419, 40.62915639545896], [-73.76476415802367, 40.629152747539145], [-73.76477097322164, 40.629149564354776], [-73.7647781418146, 40.629146869905675], [-73.76479765522052, 40.629142147137195], [-73.76481677891066, 40.629136571347004], [-73.76483544663138, 40.629130161498], [-73.76490513522714, 40.62910540866846], [-73.76493251039375, 40.62909227757688], [-73.76496037396085, 40.62907975895889], [-73.76498870588472, 40.62906786265747], [-73.76501747731787, 40.62905660084299], [-73.7650466693259, 40.629045980512736], [-73.7650762539453, 40.62903601266585], [-73.76509971002196, 40.62902918586088], [-73.76512287799476, 40.629021810139434], [-73.76514573716072, 40.629013894840824], [-73.76516826528541, 40.62900544729139], [-73.76519044277647, 40.62899647364975], [-73.76521224540274, 40.62898698609622], [-73.7652336529205, 40.628976988777886], [-73.76525464835133, 40.628966494895224], [-73.76529320125475, 40.62894919847576], [-73.76533137805642, 40.628931423147215], [-73.76536916817881, 40.628913175422134], [-73.76546947233055, 40.628853737819995], [-73.76553631026275, 40.62882027665417], [-73.76559055507889, 40.628782729703964], [-73.76564671200506, 40.628743730796536], [-73.76568068639081, 40.62872249987193], [-73.76572766467147, 40.62869166233603], [-73.76577466863773, 40.62866370616869], [-73.76581093689565, 40.628646267714835], [-73.76584951207545, 40.62862823117769], [-73.76589420336144, 40.628612512116014], [-73.76593928289284, 40.628599127509844], [-73.76597213038873, 40.62859126136338], [-73.76607218802029, 40.62855663347984], [-73.7661195748294, 40.628532492758964], [-73.76619212915514, 40.62849174392619], [-73.76639800533182, 40.62835121586794], [-73.76646100724825, 40.6282915615494], [-73.76649501288001, 40.62825139294141], [-73.76652939526993, 40.62820628230533], [-73.76667032231768, 40.628001725690716], [-73.76668403182535, 40.62798474895866], [-73.76669708885986, 40.62796747822715], [-73.7667094839294, 40.62794992403177], [-73.76672120706945, 40.62793210662342], [-73.76673224725123, 40.62791403586424], [-73.76674259495724, 40.627895729828246], [-73.76675224266138, 40.627877202907804], [-73.7667611841475, 40.62785847234611], [-73.76676940794098, 40.627839549846996], [-73.76677690671515, 40.62782045583435], [-73.76684488734944, 40.62766592316767], [-73.76684945267611, 40.62765783087837], [-73.76685325865539, 40.62764951306259], [-73.7668562884253, 40.62764101307601], [-73.76685852623642, 40.62763237042382], [-73.76685996115991, 40.6276236294791], [-73.76686058490878, 40.627614833615034], [-73.76686039447092, 40.62760602688592], [-73.76685939410102, 40.62759725034499], [-73.76686061161894, 40.627591702737064], [-73.76686253499463, 40.6275862708154], [-73.76686514424892, 40.62758100881906], [-73.76686841568339, 40.62757596561828], [-73.76687231812464, 40.627571189901], [-73.76687681315445, 40.627566727156896], [-73.76688185950474, 40.62756262019008], [-73.76688740602556, 40.627558908433535], [-73.76689340268125, 40.62755562679931], [-73.7669138107104, 40.62755326815109], [-73.76693406193907, 40.62755022348407], [-73.76695411655649, 40.62754649606844], [-73.76697393385558, 40.627542094030474], [-73.76699347444435, 40.627537026672194], [-73.7669995971799, 40.62752836441569], [-73.76700489774849, 40.62751939475185], [-73.76700935027542, 40.62751016034818], [-73.76701293392289, 40.62750070957858], [-73.76701563050011, 40.62749108847694], [-73.76701742532335, 40.627481346434436], [-73.7670183092109, 40.62747153117857], [-73.76701569515575, 40.62744982883178], [-73.76701407736509, 40.62742806970115], [-73.76701346102085, 40.627406280769236], [-73.76701384602596, 40.627384489510824], [-73.76701522987504, 40.62736272054744], [-73.76701462915281, 40.62735503777835], [-73.7670148554514, 40.62734734177521], [-73.76701590397035, 40.627339684797235], [-73.7670177727647, 40.62733212011472], [-73.76702044297221, 40.627324696943255], [-73.76702390210798, 40.627317464343356], [-73.76702812493608, 40.627310470847334], [-73.76703308314185, 40.62730376498135], [-73.7670387440383, 40.6272973882261], [-73.76704506722791, 40.627291386067775], [-73.76705201585456, 40.62728579746587], [-73.76705953921358, 40.62728066017907], [-73.76706874383656, 40.62726782162728], [-73.7670771412838, 40.62725466464171], [-73.7670847160451, 40.627241223366795], [-73.76709144868353, 40.62722752289316], [-73.76709732545605, 40.62721359502321], [-73.76710233327698, 40.62719947223159], [-73.76712314969501, 40.627140345407184], [-73.76714467160132, 40.627081365255556], [-73.76716689546122, 40.62702253595712], [-73.76721208650336, 40.626949124504065], [-73.76725810051867, 40.62687601055565], [-73.76730493067119, 40.62680319912286], [-73.76735257584865, 40.626730693385426], [-73.76740103118654, 40.626658500536415], [-73.76741265731177, 40.6266321871153], [-73.7674250514114, 40.62660607845947], [-73.76743820794681, 40.626580186452124], [-73.76745211836821, 40.626554523904375], [-73.76746677717723, 40.62652910437782], [-73.76748217581074, 40.62650394175316], [-73.76749830998838, 40.62647904614118], [-73.76751516687348, 40.626454430224086], [-73.76772095788968, 40.62619063724526], [-73.76775752719107, 40.62611336296656], [-73.7677721639386, 40.62601240617546], [-73.76777222017238, 40.62599617778423], [-73.76776898701145, 40.62595858700786], [-73.76776904921671, 40.62594065022581], [-73.767773587144, 40.625925284225175], [-73.76777697413748, 40.62591845813745], [-73.7677814822052, 40.625911632928016], [-73.7678510332581, 40.62582695479091], [-73.76793346888846, 40.62572658684279], [-73.7681417801081, 40.625463070142395], [-73.76815415926434, 40.625449429024044], [-73.76816878309884, 40.625434937716875], [-73.76823286329876, 40.62538296275592], [-73.76825648311167, 40.625360801969826], [-73.76826773786703, 40.62534801224389], [-73.7682767508157, 40.62533521875825], [-73.76828239779898, 40.62532327141245], [-73.76828692619806, 40.62531046815623], [-73.76829038175273, 40.62528399579813], [-73.76829278930508, 40.62523616724308], [-73.76829397897339, 40.625216523777766], [-73.768300915086, 40.625155891881505], [-73.7683054618712, 40.62513796411724], [-73.76831112016495, 40.625122600151066], [-73.7683257825466, 40.62509700578745], [-73.7683460561622, 40.62506971384352], [-73.7683663209338, 40.6250449839017], [-73.76841808484274, 40.62498871281703], [-73.76844172050497, 40.62496142775179], [-73.76845860808363, 40.62494096106141], [-73.76850023949468, 40.624896628043146], [-73.76851422621908, 40.624879678831185], [-73.76856340725853, 40.62480123612889], [-73.76860408710382, 40.62473271820608], [-73.7686171489849, 40.62466282801152], [-73.76862359415976, 40.6245853219183], [-73.7686217709152, 40.62457367644523], [-73.76862069515812, 40.624561977247375], [-73.7686203694395, 40.624550251815776], [-73.76862079691871, 40.6245385278925], [-73.76863549892782, 40.62449292988524], [-73.76865793092446, 40.62444919712085], [-73.76868770421753, 40.624408087171425], [-73.76872430356536, 40.62437031140524], [-73.76873772440149, 40.62436567922352], [-73.76875001912778, 40.62435949299941], [-73.76876088001904, 40.62435190708572], [-73.76877003527649, 40.62434311183306], [-73.76877725642362, 40.624333326216785], [-73.76878236316831, 40.624322796087746], [-73.7687852271618, 40.6243117841123], [-73.76878577709435, 40.62430056597931], [-73.76878399909147, 40.62428942195739], [-73.76878110232373, 40.62422270203291], [-73.76877796216628, 40.62421377116687], [-73.76877666945663, 40.62420457810707], [-73.7687772562009, 40.62419534315398], [-73.76877970736207, 40.6241862881099], [-73.76878396502812, 40.62417763011904], [-73.76878992611906, 40.6241695770614], [-73.76879744811067, 40.62416232205966], [-73.76880635075322, 40.624156039160276], [-73.76881633621885, 40.624143088171685], [-73.76882362601532, 40.62412913216011], [-73.76882805014026, 40.62411449490773], [-73.7688295063075, 40.624099515096596], [-73.76882796113601, 40.624084540874634], [-73.76882345004064, 40.62406991882952], [-73.76881607781758, 40.6240559879643], [-73.76880882615856, 40.62404169513365], [-73.7688048757524, 40.62402667022669], [-73.76880434081289, 40.624011352231115], [-73.7688072371525, 40.62399618891719], [-73.76881348095439, 40.62398162280006], [-73.768822889009, 40.62396807916799], [-73.76883518687913, 40.62395595486489], [-73.76883841708924, 40.6239522061632], [-73.76884079414899, 40.62394810403742], [-73.76884225317274, 40.623943759876326], [-73.76884275478271, 40.623939291866684], [-73.76884228415045, 40.62393482188033], [-73.76884085518273, 40.62393047181582], [-73.76883850684732, 40.623926359938956], [-73.76883530214276, 40.62392259876787], [-73.76875748130531, 40.62381998823741], [-73.76875291702562, 40.623808354139896], [-73.76875103420382, 40.62379629619005], [-73.7687518891432, 40.62378417096829], [-73.76875545621765, 40.6237723360525], [-73.76876163024419, 40.62376114122831], [-73.76877022903915, 40.62375091782677], [-73.76866759912812, 40.623588691870225], [-73.76863983616859, 40.62347903492009], [-73.76864312823386, 40.62343138820987], [-73.76861015682157, 40.62337337002994], [-73.76858406027465, 40.62331335403188], [-73.76856504520188, 40.62325181221814], [-73.76855326086714, 40.62318922909007], [-73.76855308910385, 40.62318720888311], [-73.7685525281643, 40.623171102853874], [-73.76855277292091, 40.623154991868525], [-73.76855382388167, 40.62313889999024], [-73.76855567951644, 40.623122850776326], [-73.76855833596699, 40.62310686586485], [-73.76856445685358, 40.623086228944665], [-73.76857134278468, 40.62306573198391], [-73.76857899217852, 40.623045393779336], [-73.76858739678129, 40.6230252287764], [-73.76859655119661, 40.6230052529287], [-73.768606446225, 40.6229854844399], [-73.76861707399567, 40.62296593800331], [-73.76862696197361, 40.62292614294331], [-73.76863612400902, 40.62288624781495], [-73.76864456009086, 40.62284625905276], [-73.76865226893555, 40.62280618584328], [-73.7686592480413, 40.62276603503795], [-73.76866313206479, 40.62274893999481], [-73.76866781708627, 40.62273196481158], [-73.76867329605709, 40.622715126192546], [-73.76867956296898, 40.62269844877488], [-73.76868661087502, 40.62268195310872], [-73.7686944291862, 40.62266565889768], [-73.76870300681249, 40.622649590194165], [-73.76871233509627, 40.6226337650346], [-73.76872240107345, 40.62261820487838], [-73.76873319111314, 40.62260292984728], [-73.76874469313805, 40.62258795722453], [-73.7687517382295, 40.62257678208443], [-73.76875948088907, 40.62256587683613], [-73.76876790122411, 40.62255526903333], [-73.76877698185318, 40.622544980795055], [-73.768786699849, 40.62253503624782], [-73.76879703317857, 40.62252545992884], [-73.76880795788237, 40.62251627195097], [-73.76881945081844, 40.62250749510021], [-73.7688314838828, 40.62249914814575], [-73.76884402603831, 40.62249125277827], [-73.76885705482441, 40.62248382393223], [-73.76887053516842, 40.62247688120623], [-73.7688844388137, 40.62247043994167], [-73.76911304704414, 40.62230020764196], [-73.7691499749121, 40.622276654806186], [-73.7691862890147, 40.62225255377323], [-73.76922197518535, 40.62222791597613], [-73.76925702092481, 40.62220274975544], [-73.7692914139998, 40.62217706404205], [-73.76932513919682, 40.6221508700141], [-73.76933924483245, 40.622140155087955], [-73.76935277893075, 40.62212901883874], [-73.76936571819242, 40.62211748196483], [-73.76937804391703, 40.62210556006948], [-73.76938973517646, 40.6220932731006], [-73.7694007730429, 40.622080640839044], [-73.76941114283879, 40.62206768115219], [-73.76942082423848, 40.62205441908239], [-73.76942980406625, 40.62204087191082], [-73.76943806796594, 40.62202706293444], [-73.76943909504818, 40.62202376430043], [-73.76944088895686, 40.62202066101584], [-73.76944339413559, 40.62201785863213], [-73.769446523782, 40.622015448216004], [-73.76945017624116, 40.62201351011977], [-73.7694542288964, 40.62201211014105], [-73.769458547051, 40.622011293929994], [-73.76946298798225, 40.62201108808022], [-73.76946740590363, 40.62201149996388], [-73.76947165086744, 40.62201251405936], [-73.76947558477929, 40.62201410015046], [-73.76947907599535, 40.62201620313077], [-73.76948200710078, 40.622018754038656], [-73.76948958785172, 40.62208660989106], [-73.76949122522518, 40.62209387380998], [-73.76949364722697, 40.62210101001646], [-73.76949683802994, 40.6221079679464], [-73.76950077677074, 40.62211470104368], [-73.76950543380194, 40.62212116115477], [-73.76951077775935, 40.62212730421765], [-73.76951488202643, 40.62212479008758], [-73.76951843276731, 40.62212182875304], [-73.76952135011707, 40.622118488229816], [-73.76952356310456, 40.622114847145085], [-73.76952502101452, 40.622110992589775], [-73.769525688104, 40.622107012174965], [-73.76952555122534, 40.62210300121163], [-73.7695246119857, 40.62209905377691], [-73.76952289183043, 40.622095260044205], [-73.76952647084717, 40.62207524215706], [-73.76953094211447, 40.62205533001469], [-73.76953630587634, 40.62203554408718], [-73.76954255124781, 40.622015911602155], [-73.76954967604951, 40.621996451767686], [-73.76955766697007, 40.62197718805461], [-73.76956651706179, 40.62195814493155], [-73.76957621684247, 40.621939342023985], [-73.76957614588838, 40.62192982119035], [-73.76957690782679, 40.62192031731395], [-73.76957849882942, 40.621910870994554], [-73.76958091284605, 40.62190152884764], [-73.769584135165, 40.621892329040996], [-73.76958815698806, 40.62188331317089], [-73.76959295597149, 40.62187452257053], [-73.76959851486356, 40.62186599407178], [-73.76960480513839, 40.621857766909976], [-73.7696118017596, 40.621849876738246], [-73.76962955995242, 40.62184025299341], [-73.76964674485195, 40.62183004291749], [-73.76966332105941, 40.62181926410372], [-73.76967925648758, 40.62180793882556], [-73.7696945200498, 40.621796087772665], [-73.7697090820228, 40.621783734979], [-73.7697146400569, 40.62177668277605], [-73.76971945081117, 40.621769318775804], [-73.76972348458568, 40.62176168823273], [-73.76972671714512, 40.62175384074788], [-73.76972913095265, 40.62174582284251], [-73.76973070596648, 40.621737685461994], [-73.76973143591486, 40.62172947889159], [-73.76973131423883, 40.62172125401086], [-73.76973034323274, 40.62171306103218], [-73.76972852877415, 40.621704952763416], [-73.76972588182295, 40.62169697876197], [-73.76972974597128, 40.62166282317235], [-73.76974531116765, 40.621633699697135], [-73.7697899512098, 40.62160538342686], [-73.7697940909038, 40.621601102776964], [-73.76979760714596, 40.62159651204383], [-73.76980046012629, 40.62159166055969], [-73.76980261813513, 40.62158660477591], [-73.76980405604107, 40.6215814028983], [-73.76980475721858, 40.62157611314453], [-73.76980471388426, 40.62157079783143], [-73.76980392714529, 40.621565514530175], [-73.76980240607485, 40.62156032598794], [-73.76980016504409, 40.62155529095859], [-73.76981332097934, 40.621534454247495], [-73.76980516277663, 40.621515640596584], [-73.76980523479268, 40.62149475397529], [-73.7698025430636, 40.621480127802386], [-73.76980535702252, 40.62145924592373], [-73.76981912694133, 40.62144047489153], [-73.76982199110606, 40.62140497283596], [-73.7698155809872, 40.62137735099985], [-73.76974528683073, 40.62112588175847], [-73.76975585615149, 40.621106620196336], [-73.76977817279169, 40.621087255234855], [-73.76980738698879, 40.621068721114185], [-73.7698350719695, 40.621062065049706], [-73.76986000090845, 40.62104568820952], [-73.76988324717225, 40.621036890620935], [-73.76991315213583, 40.62102052400654], [-73.7699471447698, 40.621005327050966], [-73.76996645723942, 40.62099130876381], [-73.76998637178036, 40.62097778509849], [-73.77000686475955, 40.6209647748278], [-73.77002791372077, 40.62095229079746], [-73.77006954778791, 40.62092847181169], [-73.77008467078387, 40.62091292716777], [-73.7700953110428, 40.62088849793653], [-73.77009538383358, 40.62086914843996], [-73.77009088724725, 40.62085971075152], [-73.77007920110442, 40.620841729494316], [-73.77008243996472, 40.620823231002696], [-73.77009366424308, 40.62079245192023], [-73.77011033520135, 40.62077048110149], [-73.77013119683447, 40.62076056097611], [-73.77014161704857, 40.62074716718853], [-73.77014141903945, 40.62073182298345], [-73.77014121422373, 40.62070154406916], [-73.77014985520455, 40.620663607486684], [-73.77023670585152, 40.62048289488686], [-73.77031005582694, 40.62037058378929], [-73.7703666831723, 40.62029867382738], [-73.77041082116838, 40.6202595191995], [-73.77044322286108, 40.62022301322432], [-73.77046155012042, 40.62019777912231], [-73.77049319005005, 40.620177032209995], [-73.77053739283372, 40.620149427108274], [-73.77058952001528, 40.62011463920457], [-73.7706709380609, 40.620066785965015], [-73.77077016777163, 40.62003118215498], [-73.7709437441232, 40.62000134175697], [-73.77109809419906, 40.62000538117142], [-73.77128160285079, 40.620084812874886], [-73.77138995416946, 40.62015301702537], [-73.77148534799002, 40.620232974991936], [-73.77160377191542, 40.620343247899825], [-73.77165454611215, 40.62041642715899], [-73.77171003007592, 40.620669263211866], [-73.77171699423648, 40.620961678898944], [-73.77156382711908, 40.621360527393755], [-73.7714760930544, 40.621598272863764], [-73.77141068794693, 40.62168232692856], [-73.77130563307693, 40.62177610778632], [-73.7712382622036, 40.62189031307071], [-73.77114355388717, 40.62204241575438], [-73.77107118086863, 40.62213676315836], [-73.77104111915503, 40.62221150832388], [-73.77102862072397, 40.62226790907029], [-73.77097308310023, 40.62233796390867], [-73.7709149992296, 40.62240388857771], [-73.77090364093927, 40.62244748380958], [-73.7709249541415, 40.62250703317263], [-73.77090735485449, 40.622571409475164], [-73.77095099724231, 40.62265230601103], [-73.770928432952, 40.62270219684788], [-73.77089021348017, 40.622755565035], [-73.77086739903149, 40.62282634306178], [-73.7708327022184, 40.622911551791326], [-73.77081128924291, 40.622949747147885], [-73.77083934993313, 40.62298885323997], [-73.77089044820052, 40.62300419899276], [-73.77091145703879, 40.62307807936677], [-73.77088576897397, 40.62315869629415], [-73.77083838283006, 40.623203191955625], [-73.77082069703974, 40.62328299797239], [-73.77086001275374, 40.623330538186465], [-73.77092465895865, 40.62336690114086], [-73.77096875214411, 40.623340605806035], [-73.77098537788825, 40.62332377402937], [-73.7710179250475, 40.623320312199176], [-73.77106841006369, 40.62333608481959], [-73.77109204879525, 40.62332852410139], [-73.77110277385513, 40.62330302315218], [-73.77108646010738, 40.62328159298681], [-73.77108337112618, 40.62325564010892], [-73.77109995894352, 40.62323053406613], [-73.77114623114845, 40.62322466820464], [-73.77117239021177, 40.623244064377566], [-73.7712519499769, 40.62328477803945], [-73.77129441699431, 40.62328434386562], [-73.77135751084269, 40.623297657236606], [-73.7714187722017, 40.623281264211805], [-73.77143883682004, 40.623284616852466], [-73.77145620797471, 40.62329297296164], [-73.77155510482564, 40.623362218098244], [-73.77160292680045, 40.623406511065454], [-73.7716552250971, 40.623481700825046], [-73.77172760292548, 40.62358502801516], [-73.77173897590608, 40.6236057260367], [-73.77186033009272, 40.623764606242396], [-73.77255536221206, 40.624448926834454], [-73.77266148887917, 40.62457090378069], [-73.77300150699934, 40.624839462974855], [-73.7732353722024, 40.62499416223845], [-73.77336936296258, 40.62506619419173], [-73.77343972988045, 40.62502755085904], [-73.77348668402476, 40.62505922810143], [-73.77348616522504, 40.625085118163526], [-73.77348560409436, 40.62511314810922], [-73.77349461607285, 40.62514256918544], [-73.77357897310141, 40.62516133097599], [-73.77364341420143, 40.62520859101682], [-73.77362449683453, 40.625236842888626], [-73.77365915317526, 40.62525912903939], [-73.77368363678958, 40.62533150208693], [-73.77407526407741, 40.62578348292003], [-73.7742137292342, 40.62594597387122], [-73.77435982099605, 40.626104540966075], [-73.7745133472214, 40.6262589768667], [-73.77467410789808, 40.62640907894024], [-73.77484189177467, 40.62655465089074], [-73.77501647899835, 40.62669550188855], [-73.77519764175813, 40.62683144719504], [-73.77538514193589, 40.62696230978304], [-73.77557873475992, 40.627087916954245], [-73.7757781659672, 40.627208104951144], [-73.77598317496873, 40.627322716064306], [-73.77619349302059, 40.627431600236655], [-73.77640884490044, 40.62753461516092], [-73.77656320283751, 40.627577027966574], [-73.77672027120764, 40.627613204921], [-73.77687961719566, 40.62764304608586], [-73.77704080084247, 40.62766646976843], [-73.77720337615169, 40.62768341013487], [-73.77736689520371, 40.6276938218004], [-73.77753090677732, 40.627697674893184], [-73.77769495697099, 40.62769495909415], [-73.77785859382915, 40.62768568215203], [-73.77802136496797, 40.627669869872605], [-73.77818282119773, 40.62764756508357], [-73.77834251714502, 40.6276188300741], [-73.77850001186737, 40.62758374400991], [-73.7786548699618, 40.62754240382372], [-73.77880666416571, 40.627494923081464], [-73.77895497594699, 40.627441433824835], [-73.77909939508251, 40.62738208288988], [-73.77923952322412, 40.62731703420362], [-73.77943815178199, 40.6271672019052], [-73.77963027601938, 40.627012524938706], [-73.77981569324412, 40.62685316719321], [-73.77999420781757, 40.62668929803533], [-73.78016562975726, 40.626521090219555], [-73.78032977783832, 40.62634872279653], [-73.78048647818801, 40.626172377018705], [-73.78063556587868, 40.625992240243676], [-73.78077688201479, 40.625808503834996], [-73.78091027781731, 40.62562136106153], [-73.78103561120142, 40.62543101099438], [-73.78115275085243, 40.6252376549018], [-73.7812615717945, 40.6250414971426], [-73.78136195895802, 40.62484274605734], [-73.78419483298926, 40.6208926699551], [-73.78949653224412, 40.62274664610792], [-73.78925824323107, 40.622886053919174], [-73.78919528322274, 40.622990922805585], [-73.78918594317895, 40.62300669798104], [-73.78911343183823, 40.62312915883662], [-73.78896262786533, 40.62328635703602], [-73.78880577116429, 40.623376736173], [-73.78843028253671, 40.62363939007823], [-73.7882285620579, 40.62376512877669], [-73.78487888720439, 40.62663147085399], [-73.78346558387602, 40.628118140878556], [-73.78284335187118, 40.62940570608821], [-73.78264543417806, 40.630247968544396], [-73.78377728687502, 40.63063411582445], [-73.78434764221069, 40.63137332021671], [-73.78501190495508, 40.631728189090815], [-73.78570293720223, 40.632094989240336], [-73.78570771785074, 40.63209452405763], [-73.78593974683073, 40.632071906585665], [-73.78649505508018, 40.632229771423916], [-73.78690666692243, 40.632261900864485], [-73.78743060093569, 40.632576536786054], [-73.78779972389135, 40.63302419014572], [-73.78832339420084, 40.6334250795327], [-73.7892270198415, 40.63394725414592], [-73.78943616699877, 40.6340492577919], [-73.78945016698914, 40.63406366184333], [-73.78946071511484, 40.6340796820937], [-73.78946749894477, 40.634096843960826], [-73.78947031687677, 40.634114637938026], [-73.78946908512059, 40.63413253664237], [-73.7894638409377, 40.63415000864211], [-73.7894261901355, 40.63425419441592], [-73.78948519498256, 40.63428214317577], [-73.78949544725019, 40.634276111137304], [-73.7895042910262, 40.63426890002691], [-73.78951149487399, 40.634260697974334], [-73.78951687076218, 40.634251719043725], [-73.78952027802049, 40.634242197937894], [-73.78952162816965, 40.63423238293962], [-73.78952088608465, 40.6342225301659], [-73.78951807095376, 40.63421289722416], [-73.78954048577843, 40.63417268694484], [-73.78957094985863, 40.634135675205904], [-73.78960869986732, 40.6341027908895], [-73.7896527891609, 40.63407485715359], [-73.78970211314223, 40.634052575100604], [-73.78979065407431, 40.63403025692456], [-73.78988117200988, 40.63401314992332], [-73.78997313912214, 40.63400135395755], [-73.79006601986006, 40.63399493790799], [-73.7901592720115, 40.63399393897522], [-73.79018338575843, 40.63399473291488], [-73.79020735106536, 40.63399255191643], [-73.79023054082906, 40.63398745306502], [-73.79025234611903, 40.63397957018629], [-73.7902721961424, 40.63396911009475], [-73.79028957023738, 40.633956346885576], [-73.79038485091743, 40.63395856627045], [-73.79038792704422, 40.63393720424558], [-73.79049314921534, 40.63396500109682], [-73.79053304224058, 40.63395690282931], [-73.79057102300065, 40.6339445728838], [-73.79060632203662, 40.63392826107031], [-73.79063822503424, 40.633908298003504], [-73.79066608605616, 40.633885087760575], [-73.79067138886722, 40.63387822489607], [-73.79067793184666, 40.63387201467103], [-73.79068557898376, 40.63386658513973], [-73.79069417178903, 40.63386205023438], [-73.79070408384901, 40.633858333163005], [-73.79071462205353, 40.633855806876504], [-73.79072554203712, 40.63385452921195], [-73.79073659000414, 40.633854531000736], [-73.79074750917302, 40.633855811224635], [-73.79075804675547, 40.63385834101588], [-73.79076956011139, 40.63386374895698], [-73.79077973419163, 40.63387053905073], [-73.790788286062, 40.63387852290436], [-73.79079497784153, 40.63388747803396], [-73.79079962300845, 40.63389715594929], [-73.79080209298559, 40.633907287115875], [-73.79080231893697, 40.63391759019431], [-73.79080029422181, 40.63392777895709], [-73.79079607598177, 40.63393756976981], [-73.79076604011865, 40.63395567894366], [-73.79075676815526, 40.63396126718425], [-73.7907587809813, 40.63398578260416], [-73.79073165223517, 40.634027689068475], [-73.79068301089686, 40.634043230777095], [-73.79067758390491, 40.63405227012879], [-73.79069485088321, 40.634055592064016], [-73.79075209788766, 40.63405076003948], [-73.79076283988275, 40.634068878896414], [-73.79072501421285, 40.63407868196885], [-73.79070763314095, 40.63411155676027], [-73.7906881908633, 40.63411316696722], [-73.79068489718905, 40.6341304368499], [-73.79065571819211, 40.6341377871781], [-73.79066451992368, 40.63416791982652], [-73.79067962789144, 40.63419656301452], [-73.79070064104286, 40.63422295700536], [-73.79072700193291, 40.6342464009223], [-73.79075801012759, 40.63426627205102], [-73.7907928429953, 40.63428204310243], [-73.7908111127525, 40.63431035575525], [-73.79108132786324, 40.63423263418287], [-73.79120937878695, 40.634292570055926], [-73.7913087779411, 40.634401300827356], [-73.79130837076706, 40.634531559043296], [-73.79125088480927, 40.6346888518818], [-73.79122898504875, 40.634857064034925], [-73.79174707915567, 40.635493020877604], [-73.79188217210884, 40.635580105086916], [-73.79197460306241, 40.63563997503727], [-73.79222318018806, 40.63589008813999], [-73.79286950387751, 40.63600972090083], [-73.79308469922691, 40.63606532197151], [-73.79322527289374, 40.63608285421273], [-73.79330547380057, 40.63609285693105], [-73.79343373004328, 40.636087659116605], [-73.79359019918722, 40.63617478001646], [-73.79373951221346, 40.63627274349697], [-73.7939102137638, 40.63636531615685], [-73.7939452517824, 40.63638816277205], [-73.79410219077344, 40.63649049296779], [-73.79425869505887, 40.63656675767252], [-73.79442956693237, 40.6366050559177], [-73.79444917094999, 40.63660184517127], [-73.79459351375628, 40.63657821123562], [-73.79474296180884, 40.63663275318419], [-73.7948708189186, 40.63675781386124], [-73.79495701378538, 40.636817189054675], [-73.79515520090551, 40.636953710524736], [-73.79543259302967, 40.637106174120326], [-73.79558220892679, 40.63710644015766], [-73.7957956970553, 40.637188231169866], [-73.79590925887318, 40.63732954788929], [-73.79607999980418, 40.63741126294715], [-73.7963432952073, 40.637514851228474], [-73.79661356807392, 40.63766730017196], [-73.79668466495421, 40.63771627324537], [-73.79681287577591, 40.63772735482356], [-73.79690544683324, 40.637743800160635], [-73.79694783097634, 40.63786328070894], [-73.79706159379626, 40.63793946591554], [-73.79725372923231, 40.6380157900091], [-73.7974958850722, 40.638043353949286], [-73.79764548701372, 40.63804904518251], [-73.79785901313491, 40.638119978036706], [-73.7980154439074, 40.63822337519173], [-73.79822890601164, 40.63831601776468], [-73.79837852383066, 40.63831627998194], [-73.79849301895358, 40.63831648083057], [-73.79852101882554, 40.638316529862045], [-73.79854807076897, 40.63833982104324], [-73.79863471817262, 40.638414423101985], [-73.79874153985106, 40.63843089292629], [-73.79876716487976, 40.63843271275026], [-73.79889825186386, 40.638442021780094], [-73.79908367407819, 40.63838264387314], [-73.79923314593111, 40.63843175211149], [-73.79973096309048, 40.638736558976184], [-73.80015081910298, 40.638905542768526], [-73.80042438422666, 40.63913531815223], [-73.80044220412391, 40.63915028513429], [-73.80176556112649, 40.639776735115646], [-73.80296815476305, 40.64028898605274], [-73.80465071082949, 40.640926963131896], [-73.80565288923972, 40.64162087077651], [-73.80579685288461, 40.641588236033876], [-73.80607729812384, 40.641621587018705], [-73.80621256020946, 40.64167009492619], [-73.80641503675442, 40.64174270506609], [-73.80669543516173, 40.64179249305545], [-73.80746397407664, 40.64219378697276], [-73.80864189012154, 40.642825898775506], [-73.80918082188222, 40.643029536203535], [-73.8100216378948, 40.64332134400354], [-73.81011433589615, 40.64334366400593], [-73.81075488263984, 40.64349789690536], [-73.81114276103064, 40.643701277467756], [-73.81147284678906, 40.64399223391306], [-73.81203306986673, 40.64430548288073], [-73.81278138102108, 40.64450580716119], [-73.81329120817746, 40.644576029430496], [-73.81350685093165, 40.6446366538269], [-73.8139162641244, 40.644861977076886], [-73.81413907180028, 40.64493357127085], [-73.81495103596714, 40.645269131959644], [-73.81589171154134, 40.64568147759156], [-73.81595404342984, 40.64573809159133], [-73.81602149065631, 40.64579119515375], [-73.81609371342545, 40.6458405207369], [-73.81617034788053, 40.645885819832664], [-73.81625100793559, 40.64592686421838], [-73.81633528722062, 40.64596344710643], [-73.81643071612778, 40.645998005645694], [-73.81652936516893, 40.646026845082275], [-73.81663064461708, 40.64604979301203], [-73.81673394901814, 40.64606671225031], [-73.81683866081133, 40.646077501652904], [-73.8169441540211, 40.64608209671973], [-73.81704979799966, 40.64608046998095], [-73.81715496119858, 40.64607263116144], [-73.81727126904765, 40.646083523887455], [-73.81738678602395, 40.64609852475084], [-73.81750126216694, 40.6461176012922], [-73.81761254067318, 40.64614028701629], [-73.81772234126825, 40.64616682397743], [-73.81783043434912, 40.64619715668482], [-73.81795416894467, 40.646237286461904], [-73.81807491562687, 40.646282388775674], [-73.81819233025855, 40.646332335083514], [-73.81830607819735, 40.64638698303703], [-73.81841583524769, 40.64644617688754], [-73.81853647403962, 40.64651949339181], [-73.81865099214541, 40.64659829931739], [-73.81875895970646, 40.646682298871156], [-73.81885997144619, 40.646771176765014], [-73.81895364819273, 40.64686459939905], [-73.81903963830126, 40.646962216113344], [-73.81911761897327, 40.64706366050395], [-73.8192471979852, 40.647244110277356], [-73.81938312471513, 40.64742183030758], [-73.81952530016113, 40.64759669108742], [-73.81967362076583, 40.64776856519206], [-73.8198279784907, 40.64793732737236], [-73.81998826089563, 40.64810285464576], [-73.82015435122005, 40.64826502638614], [-73.82032612846861, 40.648423724412154], [-73.82047623546282, 40.64851093704876], [-73.82062072622644, 40.64860348931486], [-73.82075927363805, 40.64870117169109], [-73.82089156402674, 40.648803763043496], [-73.8210172978826, 40.648911031124314], [-73.82113619053361, 40.64902273309735], [-73.82124797279177, 40.64913861608748], [-73.82135239156109, 40.64925841775329], [-73.821449210412, 40.64938186688043], [-73.82153821011678, 40.64950868399607], [-73.82162113047113, 40.649601478825744], [-73.82169676096179, 40.6496978188099], [-73.82176484106242, 40.649797372124084], [-73.82182513624882, 40.649899795874894], [-73.82187743880839, 40.65000473728069], [-73.82192156855426, 40.650111834886324], [-73.82195737344777, 40.65022071980834], [-73.82198473012227, 40.65033101700498], [-73.82198322623395, 40.6504091828282], [-73.82198790363265, 40.65048727582654], [-73.82199874537184, 40.65056501269861], [-73.82201571214154, 40.65064211143375], [-73.82204199276579, 40.65072759338713], [-73.82207580081815, 40.65081152658687], [-73.82211698163465, 40.65089352698888], [-73.82216534681433, 40.65097321939108], [-73.8221902479438, 40.65100962319341], [-73.82221144247737, 40.651047363752205], [-73.82223097577003, 40.65109186980007], [-73.8222453562692, 40.651137500480665], [-73.8222544751245, 40.651183910435485], [-73.82224699214011, 40.651276502232356], [-73.82224652345762, 40.65136926857901], [-73.82225307066219, 40.65146190149238], [-73.82239196285094, 40.651782323627835], [-73.82243483427497, 40.651865834613645], [-73.82247190960453, 40.65195093972748], [-73.82250308723727, 40.652037405796534], [-73.82253086731384, 40.65213544095627], [-73.82255106707531, 40.65223454628074], [-73.82256361725032, 40.65233438207507], [-73.8225684993983, 40.6524372355868], [-73.82256526265546, 40.65254012676072], [-73.82257768536014, 40.65265594752302], [-73.82259836379991, 40.65277108194031], [-73.82262723675585, 40.65288518892124], [-73.82266421873223, 40.65299793041589], [-73.82270920020741, 40.65310897241871], [-73.82276204795913, 40.65321798595695], [-73.82285571192203, 40.653374139149534], [-73.82294137953392, 40.65353293343764], [-73.82301892265205, 40.653694131402226], [-73.82308822527926, 40.653857492028585], [-73.82314918373586, 40.65402277106661], [-73.82320066970243, 40.654186123846564], [-73.82321692539362, 40.65423769869672], [-73.82348381110747, 40.65536991271862], [-73.82340406982628, 40.65608024209355], [-73.82285416151373, 40.657696987497985], [-73.82246765596551, 40.659159866398866], [-73.82243101125562, 40.65923011406355], [-73.82238733204298, 40.659297989284234], [-73.82233688072874, 40.659363084269756], [-73.82227996040434, 40.659425007931176], [-73.8222169130284, 40.65948338823206], [-73.8221481173748, 40.659537874423144], [-73.821713048647, 40.65987010482566], [-73.8216632417594, 40.65990368229462], [-73.82161125510115, 40.659935285598245], [-73.82164220805245, 40.65998214203958], [-73.82166962703728, 40.65997661925753], [-73.82169328351475, 40.660011437843366], [-73.82167498880229, 40.660021147909184], [-73.82169139439411, 40.66003508674308], [-73.82160726243185, 40.66006973820904], [-73.82157269913499, 40.660014033506634], [-73.82159281364113, 40.6600071077536], [-73.82156893575569, 40.65995869387322], [-73.82148333467894, 40.660000411094025], [-73.82139373444609, 40.66003693505062], [-73.82130067723702, 40.66006804473084], [-73.81970074120684, 40.660724086808685], [-73.819741832896, 40.66078890177405], [-73.81976926718504, 40.66077781467784], [-73.81979109615273, 40.660812630789344], [-73.81977097679966, 40.660820946705336], [-73.81978369873667, 40.66084601012296], [-73.8197141990923, 40.6608737261759], [-73.81967417049665, 40.66081244754959], [-73.81969794346361, 40.66080413687485], [-73.81965725287024, 40.660741919045414], [-73.8190119215481, 40.661006522767046], [-73.81893611080119, 40.66103701246259], [-73.81885784899167, 40.66106367002582], [-73.81877746971344, 40.66108638182685], [-73.8186953155868, 40.661105051055245], [-73.81862106189631, 40.66110284405483], [-73.81854750471477, 40.661094808317145], [-73.81847543090659, 40.66108102980283], [-73.81840561146835, 40.66106165590523], [-73.81833879327824, 40.66103689387313], [-73.81764402333809, 40.66094903243002], [-73.81763843488952, 40.66099288686265], [-73.81768045496287, 40.66099295347147], [-73.81768043594998, 40.66099990989073], [-73.81769505282463, 40.660999933057305], [-73.81769314953247, 40.661027755503554], [-73.81767487969634, 40.661027726546756], [-73.81767296498694, 40.66105972277706], [-73.81750491431545, 40.661048326201495], [-73.81751421355285, 40.66098851672879], [-73.81756353412058, 40.66099137710596], [-73.81756727119134, 40.66093673426202], [-73.81717968323512, 40.66087661147585], [-73.81698527812745, 40.6608281598642], [-73.8168833989935, 40.66080006192711], [-73.81677960495773, 40.66077637452772], [-73.81667422498099, 40.6607571727397], [-73.81656759304919, 40.660742517420054], [-73.8164600471155, 40.660732455016365], [-73.81613954700371, 40.66071525433433], [-73.81613358334366, 40.66078458246752], [-73.81617376586361, 40.660788819822514], [-73.8161754844337, 40.660827778788665], [-73.81616087340889, 40.6608263649524], [-73.81616082356737, 40.66084445023504], [-73.81598180388714, 40.66083581654858], [-73.81599658826129, 40.66077462431111], [-73.8160422583988, 40.66077608901588], [-73.81605203034718, 40.66071157420517], [-73.8154613461441, 40.6606635852798], [-73.81535709770534, 40.660644065666865], [-73.81525403832491, 40.66062117545136], [-73.81515235583697, 40.66059495635309], [-73.81468148791375, 40.66051521476173], [-73.81467075459373, 40.660579108725585], [-73.81470180006029, 40.6605833318736], [-73.8146980430019, 40.660620890384784], [-73.8146779499051, 40.6606194675287], [-73.8146760376376, 40.66065007155565], [-73.81457011758248, 40.66063459763916], [-73.8145885643114, 40.66057062883586], [-73.81461414120095, 40.66057067005973], [-73.81462533969825, 40.66050667365313], [-73.814089401262, 40.660419891420446], [-73.81388342232948, 40.66036953568016], [-73.81376113561484, 40.66035099875185], [-73.81374960325887, 40.66034738827007], [-73.81358233645567, 40.66031040195066], [-73.81325179850951, 40.66023191665837], [-73.81309228037946, 40.660194038556426], [-73.81262125517684, 40.6601570498984], [-73.81260077144056, 40.66016005549476], [-73.81230741828408, 40.6602031101358], [-73.81200162977873, 40.660206060348365], [-73.81199051239548, 40.66020636936097], [-73.81193258523973, 40.66019214833288], [-73.81157245411485, 40.660217980880745], [-73.81153691191429, 40.660231303485325], [-73.81149705738905, 40.660225267131636], [-73.81143455044243, 40.660252885947564], [-73.81141753006645, 40.66030673267926], [-73.81128488574544, 40.6603734089116], [-73.81122870505939, 40.6605463973961], [-73.81128449891122, 40.66063246839242], [-73.81130060352015, 40.660634562813165], [-73.81131639075859, 40.66063776535484], [-73.81134249000172, 40.660645809051], [-73.81136661265082, 40.660656866568395], [-73.81138816321133, 40.660670664940305], [-73.81140660968168, 40.660686863538274], [-73.81142149668759, 40.6607050624818], [-73.81143245672342, 40.660724812509216], [-73.81143921922407, 40.66074562606795], [-73.81149831122214, 40.660776177410646], [-73.8117054641712, 40.6608118760811], [-73.81199403997108, 40.66088996904534], [-73.81220466325644, 40.66089979956434], [-73.81281590117128, 40.661017226923335], [-73.8128476324838, 40.66100951629253], [-73.81287252009298, 40.661019906855415], [-73.81407345526837, 40.66126506362191], [-73.8141142532729, 40.661255641921336], [-73.8141267035501, 40.66125824985139], [-73.81413798832756, 40.66127293017234], [-73.81493715265891, 40.66141997173022], [-73.81508540456983, 40.661459882714944], [-73.81528124195414, 40.66149297010722], [-73.81575896428923, 40.66156963231751], [-73.81621480696883, 40.66163787278962], [-73.81656839875535, 40.661690806056995], [-73.81686613144069, 40.6617369898985], [-73.81689780305659, 40.661751701899114], [-73.81699404510309, 40.66176048025337], [-73.8170721153652, 40.66178820155075], [-73.8171604345205, 40.66179524128838], [-73.81718991097739, 40.66178407680208], [-73.81720804109922, 40.66178065536632], [-73.81724313336177, 40.6617867491833], [-73.81726686277027, 40.661806623415565], [-73.81735511777369, 40.661836948510086], [-73.81746603892432, 40.66186213505691], [-73.81759279884251, 40.661893385174274], [-73.81769128097959, 40.6619116524564], [-73.81771168625723, 40.66190478497017], [-73.81775019419428, 40.66190398355073], [-73.81775357123733, 40.661911751161355], [-73.81777053754436, 40.66191954030057], [-73.8179165742848, 40.66194219422514], [-73.81801507606332, 40.66195356260956], [-73.81807847301988, 40.66196314935153], [-73.8181679581757, 40.66195811638839], [-73.81830156202945, 40.66197126421071], [-73.81837398685735, 40.66199293999266], [-73.81845317149529, 40.66202756310681], [-73.81848323953413, 40.66204925589435], [-73.81849667296106, 40.66205174193633], [-73.81850715940104, 40.66210554531112], [-73.81867189288894, 40.66213298725428], [-73.81873303698315, 40.662048312586904], [-73.81874439468628, 40.662050340154344], [-73.81873799114157, 40.66207176247215], [-73.81880193629709, 40.662063443228014], [-73.81886428195173, 40.66204979225861], [-73.81892426395541, 40.66203097687981], [-73.81897271127576, 40.662011151766535], [-73.81899985670667, 40.6620015959443], [-73.81905442944014, 40.6619854551697], [-73.81919250867675, 40.66194219396646], [-73.81932891080704, 40.66189595044531], [-73.81946352477661, 40.661846762257326], [-73.81962034574538, 40.66179232152531], [-73.81977303707818, 40.66173145297589], [-73.81992114486225, 40.66166433756049], [-73.8200642288144, 40.66159117480091], [-73.82009861237917, 40.66157223907043], [-73.8201319856056, 40.661550036956825], [-73.82016247953618, 40.661525550772986], [-73.82018033317576, 40.6615088069148], [-73.82018690073131, 40.66150622664353], [-73.82019394339055, 40.66150452136184], [-73.82020126179641, 40.66150373934118], [-73.82020864878618, 40.66150390271829], [-73.82021589525574, 40.66150500686849], [-73.82022279607854, 40.661507020536476], [-73.82022915591229, 40.66150988672106], [-73.82025927848434, 40.661515050790314], [-73.82029007302556, 40.66151676241272], [-73.82032086102717, 40.66151498387529], [-73.82035096412434, 40.66150975436515], [-73.82037971904346, 40.661501189106325], [-73.82040649221668, 40.661489476820655], [-73.82061856253115, 40.66140411896701], [-73.82076957662187, 40.661343405483116], [-73.8209172840944, 40.66127814515653], [-73.82106144900045, 40.66120844223857], [-73.82120184105277, 40.66113440807726], [-73.82133823599376, 40.66105616093903], [-73.82136419507981, 40.66104170502374], [-73.82143730984893, 40.66100468496761], [-73.8215134434048, 40.660971380279626], [-73.82158370972842, 40.66094489951681], [-73.8217181259397, 40.66089648484588], [-73.82185398377906, 40.66085046299662], [-73.82193957306959, 40.660827392405324], [-73.82202184163974, 40.660798143487746], [-73.82210002252529, 40.66076298892437], [-73.8221733868732, 40.660722256451606], [-73.82224125073748, 40.660676325807174], [-73.82228434938322, 40.660645912990766], [-73.822332259163, 40.66062000896067], [-73.82238417239627, 40.660599050413666], [-73.82243921391427, 40.66058339067397], [-73.8224964558123, 40.66057329373722], [-73.82255315472136, 40.66056897739788], [-73.8226227774157, 40.66055999460139], [-73.82269131252144, 40.660547039760196], [-73.82275836985622, 40.66053018662863], [-73.82280568997277, 40.6605156358195], [-73.82285189813183, 40.66049914227766], [-73.82295042324705, 40.6604617450977], [-73.82304549561698, 40.66041948178828], [-73.82313670139534, 40.660372536323415], [-73.82322364356939, 40.66032111305753], [-73.823305943689, 40.66026543583709], [-73.82338324351308, 40.6602057470252], [-73.82345520656959, 40.660142306447035], [-73.82352151962043, 40.660075390257994], [-73.82403754350221, 40.65859867921765], [-73.82406197808747, 40.65854422248057], [-73.8240800643652, 40.65848834684702], [-73.82409166371447, 40.65843148062117], [-73.82409668723942, 40.658374059699526], [-73.82409509645046, 40.65831652422907], [-73.82387766032782, 40.65828496185774], [-73.82387951519932, 40.65825780833885], [-73.82393298572906, 40.65826467912035], [-73.8239347503239, 40.65827147191632], [-73.82408625283456, 40.65828935484831], [-73.82408489983058, 40.65816443101481], [-73.82410171821057, 40.65809888743305], [-73.82412449130834, 40.65803439533399], [-73.82415310845582, 40.657971268073304], [-73.82418743059175, 40.65790981237399], [-73.82421491336572, 40.65788030025944], [-73.8242370916942, 40.65784830428727], [-73.82425358457401, 40.657814374129785], [-73.82426410867653, 40.6577790926865], [-73.82410728599015, 40.6577503396416], [-73.82410368854457, 40.65776255450249], [-73.82404130456753, 40.657755670666944], [-73.82405929133697, 40.6576959540202], [-73.82411097425337, 40.657704180138694], [-73.82410557911007, 40.65772182318791], [-73.82426776040361, 40.65774651163842], [-73.8242874630355, 40.65768475507029], [-73.82430065317342, 40.65762200069774], [-73.82430724725508, 40.65755864616926], [-73.82430720351563, 40.65749509293534], [-73.82430741328439, 40.65748001664265], [-73.82431091078548, 40.65746517664335], [-73.82431759870339, 40.657450985840974], [-73.82433776276098, 40.657383901786005], [-73.82436660576136, 40.65731868224191], [-73.82440384045702, 40.65725597667436], [-73.82444909603728, 40.65719640951262], [-73.82489319190266, 40.655892632776755], [-73.82496862375538, 40.65565882975497], [-73.82498070378553, 40.65543197118367], [-73.82490473333498, 40.65514880880939], [-73.82459663375491, 40.65400040744441], [-73.82459044523068, 40.65396684623752], [-73.8245801511025, 40.65393387499496], [-73.82455852448105, 40.65388839487218], [-73.82452910764982, 40.65384552535447], [-73.82450787277001, 40.65382131583166], [-73.82448414511005, 40.653798488455635], [-73.82445750268344, 40.65375907902263], [-73.82443795699234, 40.65371732700223], [-73.82442585321732, 40.653673969771624], [-73.82442140510676, 40.65362977305578], [-73.82442469120276, 40.65358551740426], [-73.82443565345434, 40.653541984406125], [-73.82443162891542, 40.653496017424985], [-73.82442050473036, 40.65345073410311], [-73.82440243660656, 40.65340676822527], [-73.82437767743885, 40.653364735135646], [-73.8243465737663, 40.653325223126764], [-73.82431135183771, 40.653275594629434], [-73.82428324321734, 40.65322339987379], [-73.824262565271, 40.65316922820572], [-73.82424955146031, 40.65311369129191], [-73.82424434870848, 40.653057416212896], [-73.82424701574064, 40.65300103838198], [-73.82425752242335, 40.65294519437141], [-73.82424251845833, 40.65282333430408], [-73.8242195304614, 40.652702198315076], [-73.82418861602068, 40.65258208963839], [-73.82414985256523, 40.65246330893492], [-73.82410333716987, 40.65234615353951], [-73.82406059032242, 40.65229109526625], [-73.82402448893473, 40.65223334272411], [-73.82399532307905, 40.65217335997963], [-73.82397332709648, 40.65211162901894], [-73.82395867771312, 40.652048645874544], [-73.8239514926208, 40.651984916638774], [-73.82395182953364, 40.65192095339843], [-73.82394363846666, 40.65181204582774], [-73.82392805855035, 40.65170360702154], [-73.8239051315718, 40.65159592757305], [-73.82387491900566, 40.65148929603943], [-73.8238375018489, 40.651383998168065], [-73.82379298040433, 40.651280316131164], [-73.82374147401084, 40.6511785277698], [-73.82372063886339, 40.65114351707926], [-73.82370605009852, 40.65110671892028], [-73.82369796272235, 40.65106877653759], [-73.82369651809383, 40.65103035317576], [-73.82370174145375, 40.65099212048628], [-73.82370821868588, 40.650969025879284], [-73.82371869974587, 40.65094302509746], [-73.82373232493956, 40.650917884020465], [-73.82374042962869, 40.650880000340585], [-73.82374093211106, 40.650841618958275], [-73.82373382088048, 40.650803619721074], [-73.8237192589662, 40.650766873715064], [-73.82369758019378, 40.65073222329581], [-73.8236692815312, 40.65070046277918], [-73.82362275957273, 40.6506215600335], [-73.82358548153387, 40.650542936758306], [-73.82355596348255, 40.65046242400777], [-73.82353436803199, 40.65038046539848], [-73.82352081414116, 40.6502975125119], [-73.82351537645953, 40.650214022406225], [-73.82351808491744, 40.65013045509767], [-73.82352664158358, 40.65006056173423], [-73.823540898858, 40.64999121018287], [-73.82354044908438, 40.6499501814528], [-73.82353394983606, 40.64990945099478], [-73.8235214834271, 40.649869534596974], [-73.82350320773554, 40.649830937738386], [-73.82347935420376, 40.649794149186995], [-73.82346645408897, 40.64977439208409], [-73.8234573053961, 40.64975345583666], [-73.82345208765207, 40.64973175128903], [-73.82345090324336, 40.64970970436158], [-73.82345377540877, 40.649687747693505], [-73.82344270258008, 40.649575620008015], [-73.8234223678752, 40.649464247358345], [-73.82339285229136, 40.64935407310847], [-73.82335427337142, 40.64924553584911], [-73.82330678473657, 40.649139067652584], [-73.82318105938614, 40.64915748903266], [-73.82290717700891, 40.64809344600474], [-73.82293158913966, 40.64806811645884], [-73.8229611348693, 40.64804618729602], [-73.82299501065862, 40.64802825490866], [-73.8230322952084, 40.64801480699146], [-73.8230719745162, 40.64800620927739], [-73.82311296945245, 40.648002695592396], [-73.82315416510792, 40.64800436149545], [-73.82319444111556, 40.64801116168028], [-73.82338836405704, 40.64830562802025], [-73.82341209148474, 40.64834386783241], [-73.82343814617748, 40.64838121848777], [-73.82349305441758, 40.64844855730412], [-73.82353725987745, 40.64849460579652], [-73.82358519817106, 40.64853843415194], [-73.82363667760976, 40.648579867109135], [-73.82370866290353, 40.648639863655674], [-73.82377434375773, 40.64870391901734], [-73.82383332708838, 40.64877164985671], [-73.82386954563091, 40.64881873080938], [-73.8239036472566, 40.6488667201394], [-73.82392543625807, 40.64890252395542], [-73.82393903094528, 40.64894060436634], [-73.82394404601638, 40.648979882133446], [-73.82394033932346, 40.649019244081934], [-73.82392801590352, 40.649057574649156], [-73.82390742500239, 40.649093787501165], [-73.82390631956433, 40.649112447221775], [-73.82390937512567, 40.64913098037583], [-73.82391650358429, 40.649148852569226], [-73.82392749939734, 40.649165548465795], [-73.82394204550852, 40.649180586646885], [-73.8239597224891, 40.649193533493026], [-73.82398002063259, 40.64920401568732], [-73.8240443273407, 40.649257117223755], [-73.82411480773072, 40.64930546018296], [-73.82419085739201, 40.64934862998696], [-73.82427182414952, 40.649386256421], [-73.82432487379928, 40.64941938633568], [-73.8243939524766, 40.649462527416134], [-73.82445462918801, 40.649506883648385], [-73.82451072566741, 40.6495546094383], [-73.82456192030523, 40.6496054311751], [-73.82460791959153, 40.64965905749829], [-73.8246484597997, 40.64971518096831], [-73.82466363540958, 40.64972487509331], [-73.82468072865154, 40.649732496051804], [-73.82469925647074, 40.64973782847562], [-73.82471869527126, 40.64974072167033], [-73.82473849571241, 40.649741093874155], [-73.82475809823322, 40.64973893456873], [-73.82477694886632, 40.64973430477595], [-73.82479451489272, 40.6497273353339], [-73.82481029989705, 40.64971822319906], [-73.8248238577957, 40.64970722588016], [-73.82483480544411, 40.649694654161486], [-73.82484283346373, 40.64968086331909], [-73.82484771498439, 40.64966624308177], [-73.82481633061325, 40.64963560130075], [-73.82478828507706, 40.64960313550478], [-73.82476319575667, 40.64956818783311], [-73.82474197565908, 40.64953178337034], [-73.82473257069404, 40.649518802796834], [-73.82472624652519, 40.64950478115005], [-73.82472319274974, 40.64949013880001], [-73.8247234953338, 40.649475355330395], [-73.8247271324188, 40.64946083189745], [-73.82473399556079, 40.64944700152872], [-73.82474388012731, 40.6494342765872], [-73.82475320580947, 40.64942560338615], [-73.82476393924252, 40.649417932469426], [-73.82477589644336, 40.64941139532483], [-73.8247888724527, 40.64940610400621], [-73.82482722193953, 40.64939010268693], [-73.82486843170206, 40.6493789658187], [-73.82491144628239, 40.649372978636976], [-73.8249551639988, 40.64937229448418], [-73.82499846516194, 40.64937693088287], [-73.82505364853591, 40.64940028172764], [-73.82510406955905, 40.649429212992416], [-73.82514876208542, 40.649463170312714], [-73.82518686973117, 40.64950150301649], [-73.82521766228541, 40.64954347659191], [-73.82524054970115, 40.64958828676116], [-73.82525509340356, 40.649635074891165], [-73.8253618479136, 40.64991356569239], [-73.82548770814857, 40.649892156941455], [-73.82555494215006, 40.649658295884116], [-73.8255527020886, 40.64963073498585], [-73.82555634948751, 40.64960326142565], [-73.82556578811976, 40.64957659997198], [-73.82558076898194, 40.6495514539687], [-73.82560089686515, 40.6495284867804], [-73.82562564078114, 40.64950830429299], [-73.82565434796986, 40.649491438929864], [-73.82568626112003, 40.64947833560677], [-73.82572053834781, 40.649469339995136], [-73.82575627540461, 40.64946468940278], [-73.82579252953232, 40.64946450651432], [-73.82582834433249, 40.6494687961544], [-73.825859620933, 40.649475601604486], [-73.82588910281056, 40.64948606930044], [-73.82591605600747, 40.64949993864713], [-73.8259398095177, 40.649516864364486], [-73.82595977198939, 40.64953642508323], [-73.82597544644874, 40.64955813383485], [-73.82598644267202, 40.64958145017446], [-73.82599248690012, 40.649605793635175], [-73.82599342865467, 40.64963055817927], [-73.82602161775158, 40.649653163919965], [-73.8260445006524, 40.649679014042576], [-73.82606144501594, 40.64970739421813], [-73.82607198260253, 40.64973752020251], [-73.82607582221202, 40.649768559507606], [-73.8261036173385, 40.65001874952441], [-73.82610579237598, 40.65003318313088], [-73.82611120982533, 40.65004711324213], [-73.82611970664934, 40.65006012062907], [-73.82613102713692, 40.650071813832305], [-73.82614483059669, 40.65008184094253], [-73.82617108998623, 40.650082867264416], [-73.82619719575777, 40.65008047564895], [-73.82622238660916, 40.65007473584107], [-73.82624592792004, 40.6500658152262], [-73.82626713317553, 40.6500539739491], [-73.8262853839852, 40.65003955732691], [-73.82630014811815, 40.65002298577976], [-73.82627759205985, 40.64997688727781], [-73.8262491669272, 40.64993271153165], [-73.82621515084269, 40.649890890756296], [-73.82621444154577, 40.64986355138729], [-73.82620931351565, 40.649836487055964], [-73.82619984495574, 40.64981011047294], [-73.82618099389052, 40.64980229850085], [-73.8261640393419, 40.64979227753825], [-73.82614942337017, 40.64978030886488], [-73.82613752706081, 40.64976670454362], [-73.82612866058787, 40.64975181928378], [-73.82612305512723, 40.64973604119291], [-73.82612085682912, 40.649719781657176], [-73.82612212300786, 40.64970346461525], [-73.82612682064773, 40.649687515504866], [-73.82613482726364, 40.649672350170725], [-73.82614593409555, 40.64965836402087], [-73.82615987478752, 40.64964450204854], [-73.82617061838054, 40.64962907688924], [-73.82617786983299, 40.64961251215383], [-73.82618143000539, 40.64959526274829], [-73.82618120113091, 40.64957780238091], [-73.82617718949719, 40.64956061055321], [-73.82616950527576, 40.64954415939124], [-73.8261720453419, 40.6495269357049], [-73.82617842116277, 40.64951029791739], [-73.8261884442933, 40.64949473777115], [-73.82620181848995, 40.64948071515783], [-73.82621814846696, 40.649468644525705], [-73.826236951579, 40.64945888263074], [-73.82625767208633, 40.649451717992385], [-73.82628233078407, 40.6494515459032], [-73.82630664389826, 40.64944840785031], [-73.82633000036155, 40.64944238270315], [-73.82635181315084, 40.64943362189308], [-73.82637153404077, 40.64942234560773], [-73.82640208896747, 40.649362787132425], [-73.82642475828364, 40.649301218557234], [-73.82643931961192, 40.649238243906275], [-73.82644563012045, 40.64917448099631], [-73.82644362792341, 40.64911055537567], [-73.82643333268707, 40.649047094187246], [-73.82641484543666, 40.64898472001563], [-73.82638834756247, 40.64892404477985], [-73.82635641701073, 40.648861200527726], [-73.8263311331647, 40.6487966248865], [-73.82631265539997, 40.64873072495661], [-73.82630110018461, 40.64866391618588], [-73.82631136745069, 40.64863589290398], [-73.82631620636471, 40.64860703331523], [-73.82631551521101, 40.64857794409837], [-73.82630930852616, 40.64854923675898], [-73.82629771679228, 40.64852151477401], [-73.82627259971964, 40.64840502585524], [-73.82628836796471, 40.648382964435925], [-73.82630891763688, 40.64836331844286], [-73.82633362674252, 40.648346682513605], [-73.82636174739348, 40.648333560177385], [-73.82639242822242, 40.64832434880782], [-73.82671943599986, 40.64957881899986], [-73.82689623299987, 40.65034970899992], [-73.82701324399996, 40.65085989899991], [-73.82712509999998, 40.65132568599991], [-73.82724192599993, 40.65181216299987], [-73.82736078899997, 40.65230711899989], [-73.827478071, 40.65279547799985], [-73.8275964169999, 40.6532882659999], [-73.82770683599992, 40.653748035999875], [-73.82782507499988, 40.65424036199988], [-73.82807181099994, 40.65526770199991], [-73.82827925599996, 40.656033532999885], [-73.82859942399985, 40.657196972999884], [-73.82931419799993, 40.65889865149992], [-73.82991335799981, 40.65966566799994], [-73.83030563699988, 40.660424996999936], [-73.83107771099985, 40.66205622399986], [-73.83185737999993, 40.6636539299999], [-73.83247768199996, 40.664968212999916], [-73.832705339, 40.665405772999854], [-73.83248638399985, 40.66535491099996], [-73.83205765, 40.66525531699988], [-73.830031569, 40.66458188699991], [-73.82938864199997, 40.6643646499999], [-73.82910033499995, 40.664250737999886], [-73.82789758799997, 40.66390630699991], [-73.82713313599999, 40.66375536799993], [-73.82687353900002, 40.6637041069999], [-73.82629450900006, 40.663638333999856], [-73.82571227999983, 40.66359313999991], [-73.8250346749999, 40.663580133999844]]], [[[-73.74661040472857, 40.638724973885545], [-73.74660309901053, 40.63871038073897], [-73.74653979797041, 40.63871136266965], [-73.74654132522075, 40.63869678799164], [-73.7466105111792, 40.63869694045981], [-73.74659585423828, 40.6386800868929], [-73.74664302817305, 40.638662249807076], [-73.74666935864037, 40.63870604063595], [-73.74661040472857, 40.638724973885545]]], [[[-73.74693721618013, 40.63755610186007], [-73.74694320619099, 40.637529202088494], [-73.74704913170525, 40.63754513439954], [-73.74704316342154, 40.637566426459216], [-73.74693721618013, 40.63755610186007]]], [[[-73.74712154262465, 40.63716227570799], [-73.74706655474154, 40.63715424973436], [-73.74706485671244, 40.637160999268374], [-73.74702836326503, 40.63715567264489], [-73.74700628446362, 40.63715245006159], [-73.74699391228548, 40.63715064484292], [-73.74699985119135, 40.6371270278801], [-73.74701102390156, 40.63712870635767], [-73.74703789109672, 40.63713274210655], [-73.74706834853848, 40.637137318824294], [-73.74706683249946, 40.63714316627905], [-73.74712053523142, 40.63715124102521], [-73.74713631374684, 40.63705806030438], [-73.74715781614569, 40.637060524028364], [-73.7471492449571, 40.6371258313463], [-73.74715435857692, 40.6371263636065], [-73.74713019919187, 40.63721686134726], [-73.7471040262248, 40.63721125924123], [-73.74712154262465, 40.63716227570799]]]]}}, {\"id\": \"132\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 133, \"Shape_Leng\": 0.065134863162, \"Shape_Area\": 0.000157188987507, \"zone\": \"Kensington\", \"LocationID\": 133, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97819074399992, 40.64752599199992], [-73.97777380600002, 40.645341876999964], [-73.97684760999987, 40.645442020999944], [-73.97591918699987, 40.64554427299992], [-73.97482741800002, 40.645663420999924], [-73.97474803899986, 40.645263590999875], [-73.97467086799998, 40.64493508299988], [-73.974642245, 40.644813218999914], [-73.97442369399982, 40.644913064999876], [-73.9741399689999, 40.64502975399993], [-73.97310135399987, 40.645450975999914], [-73.972085748, 40.64586232499991], [-73.97174398199986, 40.64600110999986], [-73.97084113799984, 40.64637857099989], [-73.97076976099986, 40.646110906999866], [-73.97054554399976, 40.64531666799991], [-73.97031677099991, 40.64453794099992], [-73.9700889779999, 40.643745839999944], [-73.96988464699999, 40.643030296999896], [-73.9696861179999, 40.64235600999992], [-73.96948468100001, 40.641689896999935], [-73.9693990309999, 40.64138244599991], [-73.96891461099989, 40.63975715999986], [-73.968790992, 40.639255932999895], [-73.96872218399989, 40.63901853799988], [-73.96822343400004, 40.63729690599994], [-73.96803868499985, 40.636650984999925], [-73.96935965799996, 40.63650554999993], [-73.97029036199991, 40.63640634099988], [-73.96986869299992, 40.63419467499989], [-73.97087567699992, 40.63371951299991], [-73.97190427099986, 40.63323009399994], [-73.97216145999994, 40.63311162099996], [-73.97231136199997, 40.63304155499987], [-73.97241686999978, 40.632992044999874], [-73.97281084499981, 40.63280454799989], [-73.97320194999996, 40.632618080999855], [-73.97431883499996, 40.632090573999946], [-73.975171463, 40.631688580999906], [-73.97604935699995, 40.63127590599986], [-73.97643342199997, 40.63328912299986], [-73.97655064099989, 40.633903508999886], [-73.97663655999995, 40.6343538209999], [-73.9768886119999, 40.6356748619999], [-73.97802159000001, 40.63555270299984], [-73.97911188699993, 40.63544038599991], [-73.97928591699991, 40.63641419399991], [-73.97938644699988, 40.636932162999855], [-73.9794759209999, 40.63743136399991], [-73.97955441199994, 40.63781925899984], [-73.9796770499999, 40.6384647999999], [-73.97987360300003, 40.63951820599989], [-73.9799098319998, 40.639686357999906], [-73.98000693099998, 40.64020883699988], [-73.98007428599995, 40.640539699999906], [-73.98042057899988, 40.64238522699991], [-73.98123240499983, 40.642044912999935], [-73.98204321799996, 40.64170758199988], [-73.98244454199998, 40.6415386259999], [-73.98299419199994, 40.64132499699987], [-73.98331628499983, 40.6414786819999], [-73.98430293099989, 40.64208456299985], [-73.9845871949999, 40.64226200399988], [-73.985095196, 40.642573055999925], [-73.9858964449999, 40.643067594999906], [-73.98673080599988, 40.64357399299988], [-73.98749940600001, 40.64404550999989], [-73.98834986699991, 40.64456224399992], [-73.98528412799996, 40.64576793799989], [-73.98307340099996, 40.646625133999905], [-73.982309398, 40.646927403999875], [-73.98180533899996, 40.64712779899987], [-73.98132887699992, 40.64718167299983], [-73.98029115499993, 40.647297160999855], [-73.97911821899996, 40.647428060999914], [-73.97819074399992, 40.64752599199992]]]}}, {\"id\": \"133\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 134, \"Shape_Leng\": 0.0695878223284, \"Shape_Area\": 0.000202375182143, \"zone\": \"Kew Gardens\", \"LocationID\": 134, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82288529599995, 40.71192805799993], [-73.822526781, 40.711654884999845], [-73.82176010399989, 40.710992930999936], [-73.82098519099998, 40.710323844999934], [-73.82081877299998, 40.71017277799994], [-73.82065280899985, 40.70998970699988], [-73.82045236499991, 40.709774548999896], [-73.82023145099988, 40.70945750299994], [-73.81994398999993, 40.70901972299993], [-73.81955363300001, 40.7084252289999], [-73.81921543099992, 40.70785211599987], [-73.81916136299994, 40.70776033199991], [-73.81910729599991, 40.707668538999904], [-73.81905016999981, 40.70757022099986], [-73.81899305699982, 40.70747190299995], [-73.81892102499985, 40.70734990599987], [-73.81861275299995, 40.706853399999915], [-73.81828566399994, 40.706302849999865], [-73.81822457299995, 40.706177661999895], [-73.81809905199994, 40.70597371499988], [-73.81738330599997, 40.70460389799992], [-73.81735529099986, 40.70455013999986], [-73.81709738799995, 40.704029800999876], [-73.8169748309999, 40.70374317299995], [-73.81669961199981, 40.70315609299994], [-73.81658976900003, 40.70291543599995], [-73.81637824499992, 40.70245196399987], [-73.81647531299984, 40.70245042599987], [-73.81686030399995, 40.70244433199985], [-73.8170868249999, 40.702570330999876], [-73.82053608499999, 40.703227147999876], [-73.821478249, 40.703415603999936], [-73.82245890399994, 40.70365553099989], [-73.822708114, 40.70362481299992], [-73.82338938599995, 40.703443280999885], [-73.82387070899986, 40.703322241999906], [-73.82426993499999, 40.70321932799986], [-73.82517586799999, 40.70299178799996], [-73.82607186999988, 40.70277207899992], [-73.82706578399991, 40.702515056999886], [-73.82795308, 40.702289961999895], [-73.82884805700002, 40.702065798999875], [-73.82946917899983, 40.701910183999864], [-73.82973796799996, 40.7018403959999], [-73.830629147, 40.70161654599988], [-73.83152109599999, 40.70138962899994], [-73.83213204199994, 40.7027940809999], [-73.83302084599993, 40.702570325999915], [-73.83391709699988, 40.70234736699992], [-73.83460244999996, 40.70392746699988], [-73.83475458000002, 40.7038869019999], [-73.83548255899987, 40.70370111499994], [-73.83570634099988, 40.703614678999934], [-73.83577918999993, 40.70354987399992], [-73.83577924899996, 40.70354982099987], [-73.83577932099996, 40.70354977499989], [-73.8358710909998, 40.703493947999874], [-73.83597857799987, 40.70345153499989], [-73.8360961639999, 40.703425946999886], [-73.83621686499995, 40.703418726999864], [-73.83633362899995, 40.70342921199993], [-73.83644057000006, 40.70345494699991], [-73.83661882199989, 40.70342001799988], [-73.83696237799982, 40.70421245199992], [-73.83733095699989, 40.704854975999886], [-73.83784595199988, 40.70562953999993], [-73.83793637399988, 40.70574914799994], [-73.83793643299985, 40.705749219999866], [-73.83793647999991, 40.705749291999865], [-73.83801010900005, 40.70587971799994], [-73.83801013199992, 40.7058797729999], [-73.83801015599994, 40.70587982699987], [-73.83806401399981, 40.70601898699988], [-73.83806406099988, 40.70601912199987], [-73.83806408500001, 40.706019256999866], [-73.83809582399991, 40.7061644729999], [-73.83809584699989, 40.70616458099993], [-73.83809585899986, 40.70616468899991], [-73.83810401799988, 40.7063130239999], [-73.83808814499993, 40.70646092899994], [-73.83808812099979, 40.70646110899985], [-73.83808807400005, 40.706461289999936], [-73.83804879199987, 40.70660539699989], [-73.83804876799985, 40.706605495999945], [-73.83804872099981, 40.70660560399997], [-73.83798763399989, 40.70674305399993], [-73.83798760999981, 40.70674309799987], [-73.83798758599988, 40.706743142999834], [-73.83790724599992, 40.70687119099992], [-73.83790716199995, 40.7068713259999], [-73.83790705599984, 40.706871460999864], [-73.83781050699982, 40.70698817599987], [-73.83714702999997, 40.70795704599989], [-73.83743384099994, 40.70809296099992], [-73.83743389999984, 40.70809298699988], [-73.83743395899987, 40.70809301499986], [-73.83773266099993, 40.70821535599993], [-73.83804164799996, 40.70832329499994], [-73.83829235499992, 40.708396615999945], [-73.83835899099991, 40.70841610699994], [-73.83868260999992, 40.70849323899989], [-73.8386827869999, 40.70849328499984], [-73.83868296400001, 40.70849332099987], [-73.83901101199996, 40.70855449199996], [-73.83911666099988, 40.70857492899989], [-73.83938086799998, 40.70862603299986], [-73.83975789399979, 40.70868851299985], [-73.84069937599986, 40.708841168999896], [-73.8409080659998, 40.70887501099991], [-73.84144586899984, 40.70895751099992], [-73.84151213299998, 40.70896937999985], [-73.84159160299998, 40.70898160199991], [-73.84220736299994, 40.709076316999884], [-73.84280177899991, 40.70916947799993], [-73.84290926199998, 40.70918631999993], [-73.84296437499994, 40.7091949589999], [-73.84314625299982, 40.70922345599992], [-73.84304665799982, 40.70926414999996], [-73.84267962199995, 40.709414123999906], [-73.84263067999993, 40.709432976999885], [-73.84133936599986, 40.709946233999894], [-73.84069154199982, 40.710153213999924], [-73.84024206299998, 40.710261930999884], [-73.840191993, 40.710276828999916], [-73.83978025199983, 40.710385523999925], [-73.83972439899992, 40.71039876599992], [-73.83952312699982, 40.7104964509999], [-73.83805756099987, 40.71092087199988], [-73.8377089419999, 40.71103648399989], [-73.837368179, 40.711166928999944], [-73.83703708399995, 40.7113116139999], [-73.83703688199994, 40.711311704999915], [-73.83703669299996, 40.711311803999926], [-73.83671685300001, 40.71147007499992], [-73.83671661599998, 40.7114701909999], [-73.83671639099991, 40.711470315999854], [-73.83640902599988, 40.71164149999986], [-73.83640880199984, 40.71164162599989], [-73.83640857599988, 40.71164176899991], [-73.83611476599997, 40.71182514599994], [-73.83602786500002, 40.711749063999896], [-73.83600510799987, 40.71172967099988], [-73.83505175599991, 40.712253576999885], [-73.83426226999985, 40.71268337399986], [-73.83374150300001, 40.712969680999905], [-73.833657742, 40.71301572899994], [-73.83355895899987, 40.713068003999865], [-73.83330196199995, 40.71320007099996], [-73.83317906799991, 40.71326322499991], [-73.83243372799994, 40.71365996799988], [-73.83135959999987, 40.71423170499988], [-73.83129817899982, 40.714264132999844], [-73.83115264699987, 40.7143068559999], [-73.83103141799982, 40.71433672499987], [-73.8309675249999, 40.71435449099987], [-73.83082284299995, 40.71438342799993], [-73.83031312099993, 40.71449974499997], [-73.82995752599986, 40.71458088699995], [-73.82926493499993, 40.714792229999894], [-73.82850807699985, 40.71504432999991], [-73.82715524199998, 40.71538800399984], [-73.82700619999987, 40.715440297999855], [-73.82681044699991, 40.715550945999965], [-73.82646966099986, 40.715695567999916], [-73.82630698999996, 40.71579072399989], [-73.82627825999985, 40.71580687999991], [-73.82591945199992, 40.71598987599992], [-73.8258906989998, 40.715938525999924], [-73.8257365329998, 40.715704222999896], [-73.82570967299992, 40.71566664699994], [-73.82567283299986, 40.71561510999986], [-73.82559716299987, 40.71551433699989], [-73.82537094099992, 40.71526515499991], [-73.8251664649999, 40.71502786599996], [-73.82497234899986, 40.71480260599986], [-73.82471749499993, 40.71450684999993], [-73.82467827499991, 40.7144485109999], [-73.82428940599979, 40.71387006599987], [-73.82398559799991, 40.71321593899988], [-73.82385072099994, 40.71293320599995], [-73.82363860699985, 40.71266664399989], [-73.8234526099998, 40.71245950199993], [-73.82320247400004, 40.71220594799992], [-73.82288529599995, 40.71192805799993]]]}}, {\"id\": \"134\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 135, \"Shape_Leng\": 0.121793045659, \"Shape_Area\": 0.000467912240542, \"zone\": \"Kew Gardens Hills\", \"LocationID\": 135, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83126834299986, 40.74026227599991], [-73.8313011729999, 40.739561062999904], [-73.83143025500006, 40.73913573299994], [-73.83053551299994, 40.737520194999895], [-73.8294675199999, 40.73702448299991], [-73.82809152599998, 40.73708317499995], [-73.82794969099994, 40.73781959899991], [-73.8279436029999, 40.73848998199987], [-73.82788031199985, 40.73848979699994], [-73.82759017299992, 40.73849421699993], [-73.82718901699991, 40.7385498709999], [-73.82686746499992, 40.738628911999854], [-73.82634965799993, 40.73884326499992], [-73.825853684, 40.73912377999988], [-73.82559030399982, 40.73944759199988], [-73.8254265879999, 40.7394851379999], [-73.8249744, 40.739440426999906], [-73.82494800899998, 40.73956710599992], [-73.82481178499988, 40.739630959999886], [-73.8243501109999, 40.73975360599992], [-73.82355883499997, 40.73957874499993], [-73.82266847299987, 40.739384992999916], [-73.82177826400005, 40.739193646999894], [-73.82089716499988, 40.73900211099992], [-73.82006431399985, 40.73882813999993], [-73.81837989599988, 40.73859373999992], [-73.81708075599988, 40.7384147349999], [-73.81693283800001, 40.738389509999934], [-73.81677959599985, 40.73837979399986], [-73.81662531599994, 40.73838631099992], [-73.81647443699993, 40.73840887099993], [-73.81633109399993, 40.73844640099989], [-73.8160767679998, 40.73885793399994], [-73.81518583099992, 40.73857117099989], [-73.81501663899991, 40.738558544999876], [-73.81497208299996, 40.738560715999874], [-73.81483959999986, 40.737914413999924], [-73.81469391899994, 40.7371404799999], [-73.8146916309999, 40.73594999299991], [-73.81472294099993, 40.73533802999991], [-73.81474261099987, 40.73490936899987], [-73.81475217299992, 40.734701017999946], [-73.81475908299986, 40.73455042099991], [-73.81477621499994, 40.73424680499987], [-73.81480147399989, 40.733799159999926], [-73.81481416399997, 40.73359959799995], [-73.81481996299976, 40.733510727999935], [-73.81482699400004, 40.733402975999944], [-73.81483204799986, 40.7333255249999], [-73.81483811099991, 40.73323261899989], [-73.81484566899994, 40.7331167979999], [-73.81485170599991, 40.733024276999856], [-73.81485676300004, 40.73294679699994], [-73.81486255999994, 40.73285795599985], [-73.814868106, 40.73277295499989], [-73.81487391699997, 40.732683920999904], [-73.81487971599992, 40.73259505099986], [-73.81488551400004, 40.7325062089999], [-73.81489282099993, 40.732394229999976], [-73.81489836700001, 40.73230923099991], [-73.81491341999994, 40.73207858899988], [-73.81491646899985, 40.73203762099993], [-73.81492083199987, 40.731979832999855], [-73.81492410799987, 40.731930747999854], [-73.81492830299987, 40.73186790399991], [-73.81493851899984, 40.7317148709999], [-73.8149426019999, 40.73165369899988], [-73.81496058399995, 40.7313843389999], [-73.81496454700003, 40.73132495999994], [-73.81497202699998, 40.731212908999915], [-73.81497572599984, 40.73115749299992], [-73.81502002699983, 40.73049387199991], [-73.81502127700006, 40.730465669999866], [-73.81504493700004, 40.73039883499994], [-73.81514620799996, 40.729685964999945], [-73.81523144899997, 40.72879391199994], [-73.81517739599994, 40.728666952999916], [-73.81517732599985, 40.72866678199994], [-73.81517723199983, 40.728666627999864], [-73.81510037099989, 40.728543638999895], [-73.81510024099985, 40.72854342299991], [-73.81510007599998, 40.728543224999875], [-73.81500114199996, 40.72842749499995], [-73.81500104699994, 40.72842737799991], [-73.81500091699995, 40.728427258999936], [-73.81488194899997, 40.72832209599993], [-73.81488183099988, 40.72832198699994], [-73.81488170099985, 40.728321896999944], [-73.81474600600002, 40.72823021399993], [-73.81474579299991, 40.7282300699999], [-73.8147455559999, 40.72822995299986], [-73.81459716399995, 40.728153808999885], [-73.81459693999996, 40.728153690999875], [-73.81459667899993, 40.72815359199994], [-73.81443976499985, 40.7280940229999], [-73.81394369400002, 40.72785224499995], [-73.81280168299986, 40.72729564399989], [-73.81246807499996, 40.727133044999924], [-73.81151613699981, 40.72665799399994], [-73.81074426599986, 40.72626511299991], [-73.810693702, 40.72648898699987], [-73.81068815299999, 40.72662953899987], [-73.81068815299979, 40.72662962799992], [-73.81068816499996, 40.726629718999966], [-73.81070499699992, 40.72677121699989], [-73.81070500999995, 40.72677128899985], [-73.81070503300002, 40.7267713519999], [-73.81074434999987, 40.72691076199987], [-73.81074438399986, 40.726910887999985], [-73.81074444399991, 40.72691102299994], [-73.8108053549999, 40.7270451729999], [-73.81080542499983, 40.727045334999914], [-73.81080551900001, 40.72704547999992], [-73.81088635099985, 40.72717185599986], [-73.81101008099996, 40.72743995099989], [-73.8111179729999, 40.72781641599994], [-73.80985637299995, 40.7279736039999], [-73.80895751599999, 40.72807345899992], [-73.80806424099987, 40.7281700029999], [-73.80717042299993, 40.72826913299987], [-73.80704045699993, 40.727596503999905], [-73.80691166299997, 40.72691266599991], [-73.80678542899983, 40.72623219899988], [-73.80665182000001, 40.72555305099993], [-73.80652142399995, 40.7248730639999], [-73.80497988299993, 40.72503869699988], [-73.80487958299983, 40.725049592999895], [-73.80480892999992, 40.724354387999924], [-73.80473910599993, 40.72363153599986], [-73.80466720799983, 40.72290600699992], [-73.8045963239999, 40.72220220899997], [-73.80453391999993, 40.72151758799987], [-73.80452680199981, 40.721462545999884], [-73.80459931299991, 40.72144112099992], [-73.80475913799987, 40.72138373999996], [-73.80550258899997, 40.72111680899993], [-73.80578893899984, 40.7210187299999], [-73.80636387699998, 40.72082022699992], [-73.80722012300005, 40.72052512499991], [-73.80786105999998, 40.72030243799987], [-73.80808460099985, 40.72022476199991], [-73.80904032699983, 40.71991160499987], [-73.80902609799993, 40.719862791999894], [-73.80861283999984, 40.71844559999989], [-73.80823529499995, 40.71723813199987], [-73.80809856399999, 40.71680213799988], [-73.80784387599995, 40.715999733999894], [-73.80782231800004, 40.715911465999916], [-73.80780927699995, 40.715853895999906], [-73.81022434999987, 40.715912667999866], [-73.81114665699984, 40.71590720599987], [-73.81151841199983, 40.71590499699989], [-73.81291344199987, 40.715896708999885], [-73.81364203999982, 40.71589299199993], [-73.81524262499985, 40.71585683199992], [-73.81595932699985, 40.71584062699988], [-73.8162894489999, 40.71583316699988], [-73.81636080399981, 40.71583237099993], [-73.81643940199992, 40.7158318299999], [-73.81680638099989, 40.71582792999987], [-73.81720417899986, 40.715840727999925], [-73.81743591099992, 40.71585731399992], [-73.81767650099987, 40.71588471999992], [-73.817780212, 40.715901047999864], [-73.81789512299986, 40.715918370999894], [-73.81828452799981, 40.71600371399996], [-73.81852868899989, 40.71608211599988], [-73.81875295799986, 40.71617287499986], [-73.81931944099983, 40.71640861399995], [-73.81948747099985, 40.71647859799987], [-73.82018883899984, 40.71673522399984], [-73.82005442499995, 40.716810332999934], [-73.81930469499987, 40.71713608099988], [-73.81907861099985, 40.71720434499988], [-73.81887154599995, 40.71725932899994], [-73.81729568999994, 40.71757964499995], [-73.81725907799989, 40.71763787699993], [-73.81732944799997, 40.717660456999916], [-73.8191283259999, 40.71734991899995], [-73.81952343899992, 40.71733973299991], [-73.81962622499992, 40.7173404429999], [-73.82085213599989, 40.717348876999935], [-73.821704791, 40.717359889999905], [-73.82210623799986, 40.71740613099991], [-73.82251099399988, 40.717437075999904], [-73.82291752899988, 40.71745256099993], [-73.82332432099994, 40.71745250999989], [-73.8237298389999, 40.717437003999855], [-73.82487503799997, 40.71732243199995], [-73.82518614699985, 40.717288189999884], [-73.82531644999993, 40.71727569999994], [-73.8254192649999, 40.71739254399993], [-73.82545983199994, 40.71799986599993], [-73.82548692499986, 40.71869066199987], [-73.82547346999992, 40.718947467999875], [-73.82547345799988, 40.7189476569999], [-73.82547346900003, 40.71894785599994], [-73.82548231999995, 40.71920613499989], [-73.82548231999981, 40.71920620799988], [-73.82551374199993, 40.71946448599992], [-73.82551374199984, 40.719464530999865], [-73.82551375400004, 40.71946457499994], [-73.82556762200001, 40.71972086999989], [-73.82556765699992, 40.71972101499993], [-73.82556770399988, 40.719721167999936], [-73.82564366799977, 40.719973732999904], [-73.825643726, 40.71997393899991], [-73.82564380900001, 40.719974137999934], [-73.8257413029999, 40.72022165599988], [-73.82585945699992, 40.72046263199995], [-73.82672635699993, 40.72308159999988], [-73.82702267100002, 40.72394639099995], [-73.82706662699995, 40.7241260829999], [-73.82712924299986, 40.7243036399999], [-73.82712926599994, 40.72430371199989], [-73.8271293019999, 40.72430378399993], [-73.82721028599994, 40.7244775779999], [-73.8273090039999, 40.72464607699988], [-73.82742439499987, 40.72480760399992], [-73.82742443099994, 40.724807657999875], [-73.82742447699991, 40.72480770299991], [-73.82755527699997, 40.72496094999984], [-73.827704851, 40.72509162999989], [-73.8277049809999, 40.725091747999905], [-73.827705122, 40.72509185599993], [-73.82787086399985, 40.72521360999989], [-73.8280513119999, 40.725324759999864], [-73.82805145299996, 40.725324849999915], [-73.82805160699986, 40.72532493099987], [-73.82824491699995, 40.72542399599991], [-73.82844909199999, 40.72550982199991], [-73.82866154199978, 40.72558142299993], [-73.82887959600001, 40.725638191999906], [-73.828879714, 40.72563822699989], [-73.82887984299998, 40.725638253999904], [-73.82910097699988, 40.72567999799992], [-73.82933877699988, 40.72573592099995], [-73.82933887099998, 40.72573593899991], [-73.829572438, 40.72580840699989], [-73.82957252099986, 40.725808433999866], [-73.82957260399982, 40.72580846999996], [-73.8297990389998, 40.72589722599988], [-73.82979910999985, 40.72589725399995], [-73.82979916899991, 40.72589728099996], [-73.83001549299993, 40.72600159099995], [-73.830015693, 40.72600168999989], [-73.8300158949998, 40.72600180799986], [-73.83021940999994, 40.7261206249999], [-73.83021953999997, 40.72612069599989], [-73.83021964599999, 40.72612077799986], [-73.83040815499996, 40.72625279999993], [-73.83040820199987, 40.726252827999915], [-73.83040824999996, 40.7262528629999], [-73.83057943699991, 40.72639619699987], [-73.83057961499996, 40.72639635099986], [-73.83057976800002, 40.72639650399984], [-73.83073214599987, 40.72654932799991], [-73.83086471400001, 40.72670963399989], [-73.83140452799991, 40.72759762199997], [-73.83170465899991, 40.728040331999836], [-73.83198207999989, 40.728329997999865], [-73.83224087999997, 40.7286305209999], [-73.83248002199991, 40.7289408539999], [-73.83269855399999, 40.72925986199992], [-73.832895707, 40.7295863309999], [-73.83307087999992, 40.729919030999966], [-73.83386267199987, 40.730553188999885], [-73.83400368799985, 40.7315677099999], [-73.83448336599982, 40.73216391599991], [-73.83489939499987, 40.732967641999906], [-73.83584824699989, 40.73383374799992], [-73.83600708999997, 40.73397874199988], [-73.83606223299994, 40.734098694999865], [-73.8361277859999, 40.734241311999924], [-73.83635625099998, 40.73500859199989], [-73.83628669399984, 40.73513136699995], [-73.8361887679999, 40.73562665299992], [-73.83574733199998, 40.73699114199996], [-73.83563372699986, 40.737111629999916], [-73.83564585599999, 40.738491010999894], [-73.83627527100006, 40.73956837599992], [-73.83662419799995, 40.739365059999905], [-73.83669458799986, 40.73954125399997], [-73.83675088199985, 40.73972053399992], [-73.83679277400002, 40.73990191099992], [-73.83696272799982, 40.7406566619999], [-73.83710720699999, 40.74139622899986], [-73.83699037399995, 40.74209330099992], [-73.83690407799999, 40.74212583899989], [-73.83687304099996, 40.74213754699986], [-73.83609989199991, 40.7424290689999], [-73.83586131899989, 40.743227984999926], [-73.83382572699993, 40.742762420999874], [-73.83301515399995, 40.742600957999905], [-73.83207064899982, 40.74245093799993], [-73.83113122799989, 40.74233799999994], [-73.83117294899978, 40.741637645999916], [-73.83122032399993, 40.74095321999989], [-73.83126834299986, 40.74026227599991]]]}}, {\"id\": \"135\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 136, \"Shape_Leng\": 0.0906287154122, \"Shape_Area\": 0.000148524994129, \"zone\": \"Kingsbridge Heights\", \"LocationID\": 136, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89996501599988, 40.86801852399986], [-73.89942436199988, 40.86788083799993], [-73.89965220299995, 40.868554577999916], [-73.89960300399986, 40.86872820799991], [-73.8995340439999, 40.86889961299994], [-73.89944561599987, 40.869066841999924], [-73.89933851199986, 40.869228027999874], [-73.89921387699998, 40.86938138699987], [-73.8990732729999, 40.86952535499993], [-73.89891854299988, 40.86965857399992], [-73.89656060599998, 40.868565199999935], [-73.8974032329998, 40.86747418999989], [-73.89797842699994, 40.86672770799987], [-73.89887282999977, 40.86558402699991], [-73.90027587299991, 40.863787634999944], [-73.90107966999994, 40.86275638299992], [-73.901360904, 40.862408864999956], [-73.90162785099997, 40.86205443399997], [-73.90188007299994, 40.86169362499991], [-73.902117203, 40.86132694799995], [-73.90233887199986, 40.86095497099995], [-73.90254479399998, 40.86057825499988], [-73.90351390500003, 40.86091750999985], [-73.90408132700006, 40.859987847999875], [-73.90447464099996, 40.85934159499988], [-73.90486847199993, 40.8586932249999], [-73.90572262499991, 40.85899298899991], [-73.90679734899983, 40.85937358899988], [-73.90707003899985, 40.85936184899988], [-73.90725786199998, 40.859353437999836], [-73.90777320599983, 40.85960753799984], [-73.90956720099997, 40.85759162399991], [-73.90976165899988, 40.857182611999896], [-73.90980269899988, 40.857607548999916], [-73.91020818099983, 40.858288788999886], [-73.91030880899984, 40.85840932699993], [-73.91042952699988, 40.8585208569999], [-73.91056799299989, 40.858620592999955], [-73.91072104299988, 40.8587062319999], [-73.91088488399994, 40.85877618299992], [-73.91328437100006, 40.859618600999916], [-73.91307976699996, 40.85976164299991], [-73.91286092999991, 40.859894146999856], [-73.91262932399985, 40.86001494399989], [-73.91238663199995, 40.86012299899993], [-73.91213471699986, 40.86021749399987], [-73.91187557299992, 40.860297808999846], [-73.91161125399995, 40.860363549999896], [-73.91145048700004, 40.86040372399991], [-73.9112955739999, 40.860457380999875], [-73.91114937399996, 40.86052367099993], [-73.91101447299981, 40.86060126199992], [-73.91075688399998, 40.8607399549999], [-73.91050960899994, 40.86089126999988], [-73.91027422499997, 40.861054534999894], [-73.91005219099996, 40.86122888599986], [-73.9098448269999, 40.8614133699999], [-73.90965329599997, 40.86160687999992], [-73.90947862999988, 40.86180812899987], [-73.90947854799991, 40.86180822799987], [-73.90947846399986, 40.861808334999886], [-73.9093213549998, 40.862016161999854], [-73.90922437099982, 40.862135391999885], [-73.909224264, 40.862135517999874], [-73.90922418100003, 40.862135652999946], [-73.90914587699984, 40.86226418999998], [-73.90914584099987, 40.86226424499994], [-73.90914581699987, 40.86226429799993], [-73.90908830799995, 40.86239949799988], [-73.90908822499988, 40.8623996959999], [-73.90908816499991, 40.862399911999965], [-73.90905273599994, 40.86253873899995], [-73.90905268799997, 40.86253892899989], [-73.90905267599997, 40.86253910899997], [-73.9090395089999, 40.862678863999896], [-73.90922113399998, 40.862629947999906], [-73.909397036, 40.86256777199988], [-73.90956472199993, 40.86249301099984], [-73.90972193999986, 40.86240666799991], [-73.909866682, 40.86231006799993], [-73.91075595799987, 40.861651483999864], [-73.91181229199991, 40.861732385999886], [-73.91194992699995, 40.861750140999966], [-73.91208148799987, 40.8617680069999], [-73.91245929699984, 40.861820432999885], [-73.91266095199984, 40.861851634999894], [-73.91273784299996, 40.86186388599992], [-73.91278962899986, 40.86187247099988], [-73.91295585399997, 40.86191313599993], [-73.91329746899991, 40.86208962899986], [-73.91335994599989, 40.86211267299989], [-73.91343151700002, 40.862139065999855], [-73.91422823999919, 40.862490790359615], [-73.91341462193644, 40.863367493700686], [-73.91276271995461, 40.86302917415757], [-73.91227278976973, 40.86363214899671], [-73.91274397646826, 40.86389256817302], [-73.91276050612291, 40.86387691390985], [-73.91279762128936, 40.86389887543283], [-73.91275216216489, 40.86394505684288], [-73.91271607314991, 40.86392622943413], [-73.9127305402088, 40.863909790276885], [-73.9125260451679, 40.86379522383776], [-73.9126763402038, 40.86402111160245], [-73.91091292411947, 40.86593794649016], [-73.9105566861374, 40.866421949225746], [-73.91051287081301, 40.86645346270479], [-73.91047470999959, 40.86648896257217], [-73.91044282747865, 40.866527868551835], [-73.91041774440954, 40.86656954469076], [-73.91039987081115, 40.866613309754015], [-73.91037589901607, 40.866654644698166], [-73.9103586162944, 40.86669786058076], [-73.91034827112132, 40.86674233611599], [-73.9103450122364, 40.86678743190787], [-73.91034888650508, 40.86683249964201], [-73.91035983824382, 40.86687689140602], [-73.9101746817974, 40.8671912151159], [-73.90986176427579, 40.86794331561638], [-73.9096669484779, 40.86841703959003], [-73.9094966431223, 40.86883115714293], [-73.90927464107381, 40.86955413288569], [-73.90893714929904, 40.871019840308776], [-73.90864943177567, 40.87138578380336], [-73.90893235220783, 40.87215734798524], [-73.90872046399987, 40.87222866599996], [-73.90846367899988, 40.87262079199994], [-73.90774270799982, 40.87284595199988], [-73.90746489699994, 40.87354735399985], [-73.90669984700006, 40.87339676199991], [-73.90662887099995, 40.87338037899989], [-73.90636316599998, 40.873319047999885], [-73.90614668899998, 40.87326020499995], [-73.90608130599988, 40.873239628999904], [-73.90601809399988, 40.8732269889999], [-73.90591126399978, 40.87319581699988], [-73.905866168, 40.87318265999987], [-73.905714254, 40.87313832199987], [-73.904677283, 40.872619878999934], [-73.90459203199988, 40.87256455399988], [-73.90459194899984, 40.8725644999999], [-73.90459188899993, 40.872564445999956], [-73.90452088299993, 40.872495391999955], [-73.90452070499992, 40.87249521099988], [-73.90452057499998, 40.87249502299987], [-73.90446908599996, 40.872415627999935], [-73.90446901399983, 40.8724155209999], [-73.90446896699991, 40.872415394999905], [-73.90444010799983, 40.87233049899992], [-73.90444008499996, 40.872330418999866], [-73.90444007299996, 40.872330327999855], [-73.9044342879998, 40.87224530899993], [-73.90448860599989, 40.870578548999895], [-73.90442003099997, 40.87036127599987], [-73.90403592599995, 40.86994916299995], [-73.90374975899987, 40.86959562699992], [-73.90318667499999, 40.86914899099988], [-73.90235804799984, 40.8686961209999], [-73.90106211299997, 40.868282647999926], [-73.89996501599988, 40.86801852399986]]]}}, {\"id\": \"136\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 137, \"Shape_Leng\": 0.0461076242707, \"Shape_Area\": 0.000116455098947, \"zone\": \"Kips Bay\", \"LocationID\": 137, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97219641647804, 40.74040257875022], [-73.97221685422322, 40.740307625061504], [-73.97226753642127, 40.74031374327717], [-73.9722700133414, 40.74030857434567], [-73.97225332319164, 40.74030010555485], [-73.97224837803633, 40.74029117185863], [-73.97229979590853, 40.74003496274296], [-73.97231030439093, 40.74001851267486], [-73.97233141721735, 40.740010908199686], [-73.97243888981417, 40.74002174997318], [-73.97253745182257, 40.73966141394605], [-73.97265743803044, 40.73922275427695], [-73.97265528968896, 40.739178268480075], [-73.9726506037967, 40.73905826571552], [-73.97264431062423, 40.73901746233536], [-73.97264512721954, 40.73895866831098], [-73.97264674830716, 40.73884587151235], [-73.97263256455133, 40.7387894700528], [-73.97263100998646, 40.73874026500805], [-73.97262787457508, 40.73870906197223], [-73.97262630543392, 40.738665864084474], [-73.97262792189784, 40.738593869671085], [-73.97262084766338, 40.73852066599613], [-73.97261929656221, 40.73846306804588], [-73.97262753952535, 40.73840038548476], [-73.97263884800155, 40.73839530977717], [-73.97263833265164, 40.738379685976334], [-73.97262807328735, 40.73836796724543], [-73.97262452608486, 40.73824336395955], [-73.97263274644132, 40.73823242426211], [-73.97263327107359, 40.73822383329843], [-73.97262254718066, 40.73821706821171], [-73.97261946865977, 40.738044278747495], [-73.97261474017301, 40.73802986934807], [-73.9726060999748, 40.73793867228877], [-73.97261872947428, 40.73791467680058], [-73.9726124265646, 40.7378978720691], [-73.97260457307685, 40.73782227307573], [-73.97261404225611, 40.737798276835925], [-73.97260621196618, 40.73766627757035], [-73.97260583208758, 40.73749474970565], [-73.97257739454064, 40.7374943383613], [-73.97257959604842, 40.73735478687317], [-73.97256998441054, 40.737263787754955], [-73.97259412459493, 40.73726174867489], [-73.97259521395056, 40.73723114989953], [-73.97260003717395, 40.737221768395315], [-73.97259458103935, 40.7371886818028], [-73.97260563605106, 40.7371658837528], [-73.97259459485838, 40.737155083880765], [-73.97259775633782, 40.7371226850241], [-73.97259776670337, 40.73709748050961], [-73.97258360606305, 40.73701588054919], [-73.97258445692434, 40.73687548339179], [-73.97259038224665, 40.73676965534477], [-73.9725829245608, 40.73674468509609], [-73.972579797985, 40.73669068957926], [-73.97257525871015, 40.73664650893144], [-73.97255622610943, 40.73646148552752], [-73.9725633368432, 40.73644588163204], [-73.97248994174741, 40.73580328010761], [-73.97251016786011, 40.735787129859595], [-73.97286939178467, 40.73577086128555], [-73.97441385816511, 40.736414393858986], [-73.97445975070453, 40.73629401971835], [-73.974416773548, 40.73627772610679], [-73.97441154346602, 40.73628527904678], [-73.97439952343301, 40.73628408408965], [-73.97402692104305, 40.73612576724549], [-73.97404679254457, 40.7360999299346], [-73.97442932366866, 40.73626063349151], [-73.97442775474407, 40.7362661995439], [-73.97446647047282, 40.736278456829886], [-73.97450376013232, 40.73606458844401], [-73.97447471755295, 40.73606238071294], [-73.97444082924032, 40.736050479657045], [-73.97443372879933, 40.73605371686953], [-73.97414062018366, 40.73595565716734], [-73.97415127634777, 40.735933792668014], [-73.9744472233676, 40.73603347295399], [-73.97444828612812, 40.73603779285906], [-73.97447828183064, 40.736044185451], [-73.97450488285963, 40.736042280536694], [-73.97451459772603, 40.73589459841122], [-73.97450793938151, 40.73579694577236], [-73.97448737985142, 40.735876987533125], [-73.97444962990863, 40.73587039671474], [-73.97446125831894, 40.73583242506949], [-73.97424881734733, 40.73579823084531], [-73.97424533118316, 40.735781154932674], [-73.97425771965976, 40.73576732638401], [-73.974469391487, 40.73579916097889], [-73.97446862602692, 40.73578915679989], [-73.97447676009864, 40.73578915860296], [-73.97447066995052, 40.73572851519183], [-73.97446461207309, 40.73566817266457], [-73.97439147294988, 40.735668741593464], [-73.97439108143224, 40.7356731539856], [-73.97426298913933, 40.735672539756926], [-73.97426396904014, 40.73567694357618], [-73.97419155221249, 40.73568111586586], [-73.97418881323293, 40.735660898495176], [-73.97415673129369, 40.73565879219353], [-73.97414539712055, 40.7356725583133], [-73.97409217163273, 40.73564160471855], [-73.97408936427232, 40.73564553073938], [-73.97341851539187, 40.735367886022004], [-73.97343868734961, 40.735341388996], [-73.97305698712054, 40.73517745341553], [-73.97280661710455, 40.73552391852112], [-73.97293673973692, 40.7355992492996], [-73.9729303192644, 40.735607614396706], [-73.97276308380349, 40.73550508747355], [-73.97299695202474, 40.735186506891125], [-73.9729639526808, 40.735169066679866], [-73.97293185609784, 40.735205312369], [-73.97288512856927, 40.73518020390186], [-73.97286403116168, 40.735212267015186], [-73.97283745943452, 40.73519971670444], [-73.97294520758678, 40.73506517007111], [-73.97296719785788, 40.73507843076685], [-73.97294519031809, 40.73510770051308], [-73.97299560236563, 40.73512793848568], [-73.9729708363559, 40.735157216971274], [-73.97300291199267, 40.73517256003519], [-73.97303684713691, 40.73512794815009], [-73.97329895046123, 40.735242355077084], [-73.97332187458159, 40.73520191866485], [-73.97331638345146, 40.735196342781975], [-73.97329897672428, 40.73517681588362], [-73.97327881634085, 40.73514752741424], [-73.97327241144049, 40.7351189338514], [-73.97327700851481, 40.73507919579837], [-73.9732962736296, 40.735040856642584], [-73.97332929043594, 40.735010877391694], [-73.97336870942664, 40.734989274762654], [-73.97341821248345, 40.73497115816107], [-73.97345945915038, 40.73496837685118], [-73.9735227083429, 40.73497536084693], [-73.9735447289197, 40.73494432655072], [-73.97443700979296, 40.73531635606464], [-73.97467193099992, 40.735398302999904], [-73.97474900599997, 40.735425188999955], [-73.9748261859999, 40.7353816739999], [-73.9748930509999, 40.73537519599991], [-73.97495992999993, 40.73536871799993], [-73.97511055299996, 40.735390254999885], [-73.97522275599997, 40.735397907999875], [-73.97589093699993, 40.73567319899994], [-73.97636373299991, 40.73586797799992], [-73.97712441199997, 40.73619679799988], [-73.97792798399998, 40.73654414599988], [-73.97833820599989, 40.73672147099989], [-73.97849845599995, 40.73679095699989], [-73.97805226499996, 40.7375539539999], [-73.97891574499987, 40.737917658999876], [-73.97949062699986, 40.73815979199986], [-73.98041832999985, 40.73855053099987], [-73.98266497499989, 40.73949733299987], [-73.98221134499993, 40.7401145349999], [-73.98176150799992, 40.74073311299989], [-73.98131180499989, 40.741350923999896], [-73.98085965399994, 40.74196976799989], [-73.98040998099987, 40.74258715999986], [-73.97995899799997, 40.74320520099991], [-73.97950790899995, 40.743825200999915], [-73.9790584949999, 40.744443551999886], [-73.97860713299984, 40.745057830999876], [-73.97812378199983, 40.745727172999885], [-73.97735988999995, 40.74540421599994], [-73.97588117699996, 40.74477922399991], [-73.97522419999986, 40.74450468199987], [-73.97350835000003, 40.7437820339999], [-73.97223028499984, 40.743249026999884], [-73.97210897599994, 40.74320549099992], [-73.97206165299998, 40.74318802999993], [-73.97198880299989, 40.74316289599994], [-73.97191659199993, 40.74313739399983], [-73.97167921200406, 40.74305446402989], [-73.97188888913038, 40.7427248203555], [-73.97176261327057, 40.742672671534656], [-73.97174059715836, 40.742663579509], [-73.9717750124801, 40.74261591264714], [-73.97195955856255, 40.742337921543616], [-73.97204926316721, 40.7418877537763], [-73.97206949682803, 40.741788905934726], [-73.97210163980672, 40.741623202060914], [-73.97215263572176, 40.74138858760124], [-73.97219804813778, 40.741144668691746], [-73.97228618248482, 40.74073827407896], [-73.97228992099222, 40.74072103903176], [-73.9722128259937, 40.740711663608245], [-73.97219567275307, 40.74070957946137], [-73.9721826857009, 40.74070627990964], [-73.97216848387873, 40.740693120300485], [-73.97216477368652, 40.74067572212912], [-73.97216549760822, 40.74067216504987], [-73.97221525425884, 40.740426563519435], [-73.9722251561433, 40.740416696931206], [-73.97223504317171, 40.740413403401995], [-73.97223566211, 40.740405415225176], [-73.97219641647804, 40.74040257875022]]]}}, {\"id\": \"137\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 138, \"Shape_Leng\": 0.107466933508, \"Shape_Area\": 0.000536797294367, \"zone\": \"LaGuardia Airport\", \"LocationID\": 138, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.8712485624251, 40.78603815125499], [-73.86978081192638, 40.78533793332583], [-73.86923491782913, 40.78596991495815], [-73.86918113468879, 40.78594348978492], [-73.86969325713956, 40.78529616325476], [-73.86987007434347, 40.78508590706812], [-73.86943751657091, 40.78363839981371], [-73.86871414175995, 40.783718086052126], [-73.86871058864548, 40.78373350052022], [-73.86870358701877, 40.78374821705438], [-73.86869334724, 40.78376179349807], [-73.868680176961, 40.78377382194827], [-73.86866447188046, 40.78378394101112], [-73.86864670385509, 40.7837918466598], [-73.86862740672426, 40.78379730136971], [-73.8686071602691, 40.783800141254105], [-73.86858657279326, 40.783800280988885], [-73.86856626284758, 40.78379771637565], [-73.86854684064295, 40.78379252446806], [-73.86852888971897, 40.783784861256905], [-73.86851294940965, 40.78377495698293], [-73.86849949863941, 40.783763109219564], [-73.86848920723405, 40.78375008838399], [-73.86848193339068, 40.78373594038316], [-73.86847788300818, 40.78372106570472], [-73.86847717073857, 40.78370588540604], [-73.86847981674143, 40.78369082919537], [-73.8684857461148, 40.78367632326808], [-73.86849479101389, 40.78366277824263], [-73.8685066954039, 40.78365057753673], [-73.86852112230667, 40.78364006651474], [-73.86853766334073, 40.78363154271083], [-73.86855585028033, 40.783625247407215], [-73.86857516830997, 40.78362135880435], [-73.86859507059657, 40.78361998697619], [-73.86861499376909, 40.783621170754984], [-73.8686343738654, 40.783624876631684], [-73.86865135955154, 40.78363046770971], [-73.86866699619074, 40.783637996786204], [-73.86868090404359, 40.783647281015995], [-73.86869274535371, 40.783658094929415], [-73.86870223255126, 40.78367017590811], [-73.86870913523617, 40.783683230562595], [-73.86871328577288, 40.783696941857244], [-73.86942344795428, 40.78359131491769], [-73.87048029898958, 40.78230611231522], [-73.87146292358179, 40.78212640074081], [-73.87255732673727, 40.78082112154256], [-73.87238637036786, 40.780740527476986], [-73.87218014326744, 40.78052591567086], [-73.87118588705187, 40.78000159885229], [-73.87019485020025, 40.779546803981], [-73.87015751178103, 40.77949794563896], [-73.86825520270827, 40.77851264407573], [-73.86792643466, 40.77839424779842], [-73.8663290285311, 40.77764199906134], [-73.86615198692076, 40.777590680987565], [-73.86576064949364, 40.777358120694394], [-73.86513221186647, 40.776966470014244], [-73.86471066497087, 40.77673318625357], [-73.86215889693487, 40.775514553767344], [-73.85921146961684, 40.77408846731154], [-73.85911792602863, 40.774130425388854], [-73.85901031879031, 40.774134187405274], [-73.85892842217254, 40.77409513659952], [-73.85890298388608, 40.77400941135213], [-73.85893795943065, 40.773956124871965], [-73.85863356071195, 40.77380883459958], [-73.85840105714999, 40.773721336054216], [-73.8579615794531, 40.77348920588864], [-73.85773563893147, 40.77338481475658], [-73.85723387590264, 40.77318362981194], [-73.85690044014613, 40.77305877286938], [-73.85674398259154, 40.772979072572504], [-73.85656384625518, 40.7728996687605], [-73.8559380688681, 40.772636741565236], [-73.8555107354459, 40.77243896817119], [-73.85505106271819, 40.772195389989086], [-73.85504825390281, 40.772114350714084], [-73.85504803073704, 40.77203707882777], [-73.85505451774685, 40.771924657916436], [-73.85506798595154, 40.77187023014758], [-73.85508635910342, 40.77183138827741], [-73.85511549005498, 40.771813233787654], [-73.85518065336801, 40.7717835813866], [-73.8556587255464, 40.771150740240344], [-73.85642119168853, 40.770346713565665], [-73.85643866852624, 40.77033323611143], [-73.85645807349368, 40.77032086119881], [-73.85650122019709, 40.77030029368632], [-73.85652419256841, 40.77029246081006], [-73.85654757393833, 40.77028645581004], [-73.85657099964907, 40.7702823695339], [-73.85661890723391, 40.77027930513913], [-73.8566688368006, 40.77028247319014], [-73.85669390392543, 40.77028636785116], [-73.85671858798878, 40.77029173877042], [-73.85674255927898, 40.77029851350239], [-73.85676549671736, 40.77030660168677], [-73.8572320319643, 40.77052901036681], [-73.85728951464255, 40.770550973015304], [-73.85731372099039, 40.770558360560976], [-73.85733677055224, 40.77056407124349], [-73.85735910137694, 40.77056821382758], [-73.85738091328331, 40.77057080715644], [-73.8573967280712, 40.770569989604255], [-73.85740515395037, 40.77056793762192], [-73.85743064536503, 40.77055579456294], [-73.85744762320293, 40.770542920269385], [-73.85745955790304, 40.770528761177516], [-73.85746405294711, 40.7705201937719], [-73.85746766311485, 40.770510463144845], [-73.85747194415998, 40.77049130633454], [-73.85747816799682, 40.77044489329804], [-73.857484324398, 40.77042381024948], [-73.85748920023862, 40.770414505027546], [-73.85749528182235, 40.77040655222482], [-73.85750267630988, 40.77039981644549], [-73.85750688715996, 40.77039687772179], [-73.85752974542834, 40.770386307969765], [-73.85755023687742, 40.77038078501796], [-73.8575730336926, 40.77037761613721], [-73.8575984180025, 40.770376766494465], [-73.85762452189127, 40.770378022362266], [-73.85765412555506, 40.770381282303525], [-73.85776107882876, 40.77039997440541], [-73.8577762649952, 40.770405098333605], [-73.85783557421978, 40.770431619331895], [-73.85784854459581, 40.77043660666341], [-73.8578600035371, 40.77044006044041], [-73.85787912685437, 40.770442876015444], [-73.85789051085527, 40.77044210670594], [-73.85790102304883, 40.77043921025004], [-73.85790399937207, 40.770437863495], [-73.85810814036756, 40.77033328729675], [-73.85812062818592, 40.77033733686403], [-73.85813724838887, 40.77034723565637], [-73.85814676560885, 40.77035036435933], [-73.85815492427808, 40.77035094147851], [-73.85816356056772, 40.770350636994046], [-73.85817215042374, 40.77034947605987], [-73.85818022082987, 40.770347523596186], [-73.8581872858477, 40.77034491101544], [-73.85819573008206, 40.770339905685155], [-73.8582051105021, 40.77032345647716], [-73.85820236123325, 40.77030737932604], [-73.85818924879476, 40.770290460502615], [-73.85798709647926, 40.77017361371138], [-73.8577362357601, 40.77000314507246], [-73.85745009449602, 40.76980870552685], [-73.85692407357327, 40.76951252654717], [-73.85669054762312, 40.76938550821013], [-73.85621267610162, 40.76919579775782], [-73.85614888582384, 40.76907879647389], [-73.85614973578456, 40.76898684694476], [-73.85624565746174, 40.768885714198944], [-73.85631296048622, 40.76887844058347], [-73.85647771882165, 40.76890515740583], [-73.8566356595733, 40.76894032975357], [-73.85681833769972, 40.76903884617975], [-73.85704790262234, 40.76913955594884], [-73.85710288000817, 40.76909250012824], [-73.85716158430333, 40.76899140170871], [-73.85724041942667, 40.76888157464138], [-73.85731254450552, 40.768763608173636], [-73.85767800266241, 40.76827626539154], [-73.85797604163109, 40.76811483047016], [-73.85805261946525, 40.76809065637136], [-73.85819704236047, 40.767943314103064], [-73.85832813918591, 40.76782418555582], [-73.85843507175626, 40.76770544095867], [-73.85859539017551, 40.76748501377069], [-73.85877376553057, 40.767349995601], [-73.85892075977725, 40.76727356056681], [-73.85937510429778, 40.767075328942674], [-73.85963779907536, 40.76702796771879], [-73.85983028214702, 40.76703180469933], [-73.85998650669511, 40.766957936481205], [-73.86016778943647, 40.766942227274065], [-73.86033645988697, 40.766959433395115], [-73.86055456613312, 40.76700031925954], [-73.86069743030745, 40.76701133435791], [-73.86077468876901, 40.76699451566885], [-73.8608553594459, 40.76693211808921], [-73.86092884231329, 40.76694195062044], [-73.86098883676625, 40.76701427074401], [-73.86100762175549, 40.767059381327535], [-73.86100038221741, 40.76717391624675], [-73.86103345678849, 40.76725679418057], [-73.8611027715228, 40.76727194252982], [-73.86118998431375, 40.76720716686883], [-73.86121194395173, 40.767031089935585], [-73.86131386166113, 40.76694063174371], [-73.86143250938784, 40.76691478503673], [-73.86169177207913, 40.76691704203793], [-73.86181545316371, 40.76693736156739], [-73.86202142385251, 40.76691313355552], [-73.86205271633084, 40.76691261216039], [-73.86212976522565, 40.76691133536475], [-73.86222539270146, 40.766894601510174], [-73.86220059780494, 40.76683989374968], [-73.86226565137707, 40.766803167772686], [-73.86243485599873, 40.76688891752953], [-73.8626022140311, 40.7668939798399], [-73.86268864176631, 40.76687591050486], [-73.86275490196954, 40.76676452492579], [-73.8627202147973, 40.7666706422922], [-73.86224047985023, 40.76628308650575], [-73.86215785661702, 40.76619175792949], [-73.86216751063289, 40.766047994214446], [-73.86201493096908, 40.76589092640018], [-73.86176874486004, 40.765594004521994], [-73.86153844239246, 40.76534253221642], [-73.86123668863942, 40.764960616103124], [-73.86043337809801, 40.76386888243776], [-73.86057039699999, 40.76383121299987], [-73.86093691499987, 40.76372861499989], [-73.8610972439999, 40.76366447699992], [-73.86180888799989, 40.76454470699986], [-73.86286118599996, 40.765846235999874], [-73.86321805300003, 40.76625081599989], [-73.86340214900002, 40.76645551199989], [-73.86353063400003, 40.76657622399992], [-73.86404975699985, 40.76706394699995], [-73.86458486999996, 40.76756668999989], [-73.86509294099989, 40.76801531499993], [-73.8655058869999, 40.768345767999946], [-73.86606875999985, 40.768796181999946], [-73.86728193799998, 40.76965000299989], [-73.868098735, 40.77017200899994], [-73.8688938269999, 40.770649021999915], [-73.869437198, 40.77092448199991], [-73.86962970499988, 40.77100636599992], [-73.87016128299994, 40.77120898499992], [-73.87071447499993, 40.771390689999926], [-73.87143357299982, 40.77157646699987], [-73.87173533999989, 40.771622584999896], [-73.87223577699984, 40.77169565899993], [-73.87272561699997, 40.7717366429999], [-73.8732794149999, 40.77176740099993], [-73.87380524599989, 40.77176797799988], [-73.87426330499994, 40.7717533869999], [-73.87475943399997, 40.77171998699991], [-73.87517573299998, 40.771676710999955], [-73.87601656500001, 40.77155104399987], [-73.87608949699984, 40.77153690299987], [-73.87615314099988, 40.771523454999944], [-73.87665135299991, 40.771418945999926], [-73.87704537199986, 40.77131652899986], [-73.87743339799992, 40.77120025499991], [-73.87781416299993, 40.771070455999954], [-73.87818646999993, 40.77092753499992], [-73.87854915699987, 40.77077199599996], [-73.87897365099998, 40.77055642199982], [-73.87961158799999, 40.77023244799991], [-73.88020161599998, 40.769904189999856], [-73.88056341499993, 40.76970115299986], [-73.88068583599984, 40.76963245199993], [-73.88159926399999, 40.7691258519999], [-73.88277241799986, 40.76852944099993], [-73.8835659359999, 40.768180444999885], [-73.88426804199986, 40.767894764999916], [-73.88439729399991, 40.767842169999874], [-73.88506266099994, 40.76759894299993], [-73.88564823400002, 40.767407690999946], [-73.88602554199989, 40.767282957999875], [-73.88667620399993, 40.76710325899991], [-73.88671945999997, 40.767091312999874], [-73.88713429399999, 40.76698296599988], [-73.88762452899991, 40.76686519999991], [-73.88770661799987, 40.76683859799995], [-73.88772988199987, 40.766914433999936], [-73.88781814599999, 40.767202077999876], [-73.88817759199992, 40.7684282569999], [-73.88819008199995, 40.76850987299992], [-73.88822109399986, 40.768592289999944], [-73.88827131099981, 40.76867187099985], [-73.88833975599987, 40.76874480599987], [-73.88842375699991, 40.76880764399987], [-73.8885192579999, 40.76885784099994], [-73.88862144999995, 40.76889404899988], [-73.8887255129998, 40.76891623899995], [-73.88873647199979, 40.76898593099992], [-73.889040003, 40.77091623799991], [-73.88942374399998, 40.77304592899988], [-73.88945486901865, 40.77353295106357], [-73.88779843805143, 40.77371129742464], [-73.88767940155543, 40.77381879172323], [-73.88580762286652, 40.77402222447536], [-73.88581012437986, 40.77403601766623], [-73.88581792367333, 40.77407899874046], [-73.88582793733663, 40.77407794805797], [-73.88583361561382, 40.77410924043989], [-73.88579151867036, 40.774113655072405], [-73.88579034397598, 40.77410718725041], [-73.88568182351487, 40.774118568043136], [-73.88568292636904, 40.77412464546015], [-73.88566947657924, 40.774126056083816], [-73.88569498085096, 40.774266610445544], [-73.88570207086045, 40.77426586763382], [-73.88571221642924, 40.77432177112161], [-73.8856739890097, 40.77432577994886], [-73.88567349258264, 40.774323050631025], [-73.88565054964837, 40.774325457489574], [-73.88567007665036, 40.774433071216826], [-73.88562650739728, 40.774437640294636], [-73.88560236103582, 40.774304563939076], [-73.88564627381488, 40.774299959528406], [-73.88564767572932, 40.77430769122308], [-73.88566773133905, 40.77430558840732], [-73.88566130980055, 40.77427020514186], [-73.88566791315597, 40.77426951277721], [-73.88564276584745, 40.77413093288607], [-73.8856385336236, 40.77413137765435], [-73.8856323103537, 40.77409707478858], [-73.88567586836331, 40.77409250687013], [-73.88567708364381, 40.77409920206598], [-73.88578804109311, 40.77408756570365], [-73.8857868343147, 40.77408091775971], [-73.88579477727255, 40.774080084653825], [-73.88578718974873, 40.77403826926386], [-73.88578473697669, 40.77402471216453], [-73.88431556689014, 40.77418436398408], [-73.88408262463024, 40.774104480143286], [-73.88391627963797, 40.774132606591536], [-73.8839316828436, 40.77418464435257], [-73.88410836235168, 40.7741670833982], [-73.88455150532151, 40.77436062233214], [-73.88465982660422, 40.77448010121009], [-73.88479222517095, 40.774807063622696], [-73.88483742391051, 40.77495298074568], [-73.88485536080363, 40.77510221207482], [-73.88483519529828, 40.77522084211526], [-73.8847736190733, 40.77533228058652], [-73.8846752373344, 40.77539676647517], [-73.88460175355154, 40.77542051069444], [-73.88454882637299, 40.77542804960335], [-73.88445519420895, 40.77546480385051], [-73.88441988513166, 40.77545646620444], [-73.88428737712128, 40.775472056165945], [-73.8842883055274, 40.7754830970367], [-73.88433217018762, 40.77548297087071], [-73.88436257156576, 40.775493373732644], [-73.88440957187971, 40.775549360877285], [-73.88475744404441, 40.77725413868523], [-73.88482322075201, 40.77751239820956], [-73.88493197897338, 40.77790502210444], [-73.88499070411882, 40.77814521789072], [-73.88503248656444, 40.77829983172282], [-73.88505522658441, 40.77840531274414], [-73.88511399222635, 40.778475529034566], [-73.8852000839678, 40.77851357126351], [-73.88523914391557, 40.77855922117586], [-73.88525026986592, 40.77861290174238], [-73.88521875942243, 40.778661362805], [-73.88508345876635, 40.778671313161915], [-73.88505291740942, 40.77877543567209], [-73.88512416435394, 40.779061209282986], [-73.88508410716267, 40.77935257161611], [-73.88497842475354, 40.77974438290775], [-73.88493466381185, 40.77993691280691], [-73.88488113731053, 40.77998719674121], [-73.8847822502109, 40.780038480497915], [-73.88445032609359, 40.78015460132617], [-73.88385203219555, 40.780264319754494], [-73.88325271441283, 40.78026542290212], [-73.88324038254822, 40.78032572327379], [-73.88290491186557, 40.78034869723975], [-73.88287851751367, 40.780236320109324], [-73.88281116030053, 40.78021893669761], [-73.88274449717143, 40.780223424072155], [-73.88264030510811, 40.7802858579589], [-73.88231777091364, 40.78036054929843], [-73.88215899724348, 40.780367867563115], [-73.88211863000274, 40.780395863177205], [-73.88208293364791, 40.780450836167304], [-73.88202629453421, 40.780485043527804], [-73.88187200517194, 40.78049845486982], [-73.881684076837, 40.78050006263888], [-73.88155872181727, 40.78054662573665], [-73.88139075985553, 40.78060160840805], [-73.8812191473699, 40.780610617237734], [-73.88096087912395, 40.78058557723642], [-73.88087003883854, 40.78057306017537], [-73.88075025848272, 40.78057432624618], [-73.88038817410674, 40.780619416402864], [-73.88018929439087, 40.78063496792158], [-73.8800224049746, 40.78064471983977], [-73.87990587782392, 40.78063914517093], [-73.87973466843002, 40.780654818049825], [-73.87961343937161, 40.78069431945781], [-73.87958646834618, 40.780698110860044], [-73.8795907103769, 40.780779148716455], [-73.87960408126767, 40.780779162686564], [-73.87960692432854, 40.78085031306523], [-73.87947468643891, 40.78085695107503], [-73.87947341303463, 40.78070268891417], [-73.87944805023493, 40.7807004765119], [-73.8793580603621, 40.78065668012733], [-73.87930409174346, 40.7807594078809], [-73.87920483851617, 40.78076738933477], [-73.87913370828485, 40.78067822938221], [-73.87892488684261, 40.78067355215143], [-73.87870562518874, 40.78058590995002], [-73.87905115692989, 40.781730533363344], [-73.87930868295156, 40.781853658332345], [-73.878858374856, 40.78239817868118], [-73.8788883123546, 40.78241442661198], [-73.87888384757818, 40.7824189397462], [-73.87914510421407, 40.78254456779565], [-73.87916445729643, 40.78252313142975], [-73.87918078802262, 40.782529923616465], [-73.87916440606021, 40.78255136427313], [-73.87974777853563, 40.782836564207], [-73.8797016367226, 40.78288281807235], [-73.87967788425708, 40.782872629506535], [-73.87970469079072, 40.78283764835938], [-73.87945679555362, 40.78271542259296], [-73.87942106533049, 40.78275604079163], [-73.87940622567466, 40.782745861494845], [-73.87943006869374, 40.78270635948337], [-73.87914654749781, 40.782568286055174], [-73.87908251177446, 40.782650658660685], [-73.8790602451593, 40.78264047102351], [-73.87912427592009, 40.78256035616555], [-73.87886747171292, 40.78243699116004], [-73.87885705699975, 40.782444885571216], [-73.87883146781735, 40.78243071449443], [-73.87833923298054, 40.78302592074941], [-73.8751575489337, 40.781509449653285], [-73.87484527331479, 40.78168492407535], [-73.8712485624251, 40.78603815125499]]], [[[-73.8728719590193, 40.785975027902346], [-73.87282839783694, 40.78595446173833], [-73.8728226286446, 40.78596397466698], [-73.8727740594054, 40.78594192145881], [-73.87280920016116, 40.78590310909999], [-73.8728535000943, 40.78592565061053], [-73.87284047915952, 40.785943122879075], [-73.8728849699182, 40.785962633107935], [-73.8728719590193, 40.785975027902346]]], [[[-73.89008921683674, 40.77362643576237], [-73.89034195999996, 40.77323760499998], [-73.89183244902246, 40.77488019994431], [-73.8917254778065, 40.77505203213403], [-73.89171751335809, 40.77516363289303], [-73.89168935121086, 40.775254737843575], [-73.89152759083345, 40.77547813536736], [-73.89144361875265, 40.77562269470967], [-73.89135801404697, 40.77604940793021], [-73.8913676627334, 40.77634299857849], [-73.89145185712488, 40.776512769270866], [-73.89126414584786, 40.776855855374606], [-73.89110422665516, 40.777023206974675], [-73.89112044339463, 40.77712469973607], [-73.89106681416045, 40.77721292530485], [-73.89094860635713, 40.77728384517262], [-73.89073132114231, 40.77733204986776], [-73.89056250595364, 40.777403920964275], [-73.89049834642208, 40.777462635851975], [-73.89045915776707, 40.77755049725344], [-73.89046537395134, 40.777703219287076], [-73.89035193082609, 40.77780032027483], [-73.89017369508576, 40.77785081277782], [-73.89008729416794, 40.777784193606934], [-73.89002369095195, 40.77724712640633], [-73.88951644380981, 40.775936714445564], [-73.88947151143196, 40.775738056523366], [-73.8894710539975, 40.77553555096718], [-73.88955512480328, 40.77527870961293], [-73.88956149685701, 40.77509223860254], [-73.88958184665768, 40.77503504961379], [-73.88958935733879, 40.77495871069512], [-73.88961448773941, 40.77484568714223], [-73.88978900756075, 40.774236861169776], [-73.88984246558317, 40.77410037855676], [-73.88988671629585, 40.77401734907422], [-73.8900676015257, 40.77396248425602], [-73.89019171520681, 40.77364687622494], [-73.89008921683674, 40.77362643576237]]]]}}, {\"id\": \"138\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 139, \"Shape_Leng\": 0.0932997884483, \"Shape_Area\": 0.000446669880154, \"zone\": \"Laurelton\", \"LocationID\": 139, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.74125253099989, 40.68745783899989], [-73.7375999409999, 40.686362011999904], [-73.73744163699996, 40.68670076699988], [-73.737321908, 40.686664320999924], [-73.73657845699985, 40.68643801399987], [-73.73569987899988, 40.68617269699991], [-73.73483039799987, 40.68590609699992], [-73.73451246099985, 40.685808342999955], [-73.73396025399995, 40.68563856299993], [-73.73361575099986, 40.68553440399993], [-73.73355835299998, 40.6855170539999], [-73.73350033599996, 40.68550058399994], [-73.73308454599989, 40.6853825349999], [-73.73222005799998, 40.68512109599991], [-73.73132732799995, 40.68485054399987], [-73.73045844999994, 40.68458679399986], [-73.72959149799996, 40.68432533799995], [-73.72872058399992, 40.68406239099987], [-73.72785251200004, 40.68380044199992], [-73.72718272499995, 40.68359732999987], [-73.72720084199985, 40.68356123499992], [-73.72810238799995, 40.681765636999934], [-73.72830180799986, 40.68148497499995], [-73.7285165689999, 40.681210472999915], [-73.72874616799989, 40.680942857999895], [-73.7289899879998, 40.68068281299987], [-73.72924739999999, 40.68043098599987], [-73.72950555799994, 40.68005572599989], [-73.73047301299988, 40.6793479369999], [-73.73106142499994, 40.6783066209999], [-73.73187383000001, 40.67756892099986], [-73.73303223399991, 40.67649502599989], [-73.73332244199996, 40.676132565999865], [-73.73384204199998, 40.67546337699988], [-73.73403489799982, 40.6752376569999], [-73.73502751499981, 40.67402828999992], [-73.73505915299991, 40.67398974999989], [-73.73509933199999, 40.673924256999896], [-73.73561853899999, 40.67313052699989], [-73.73597469399992, 40.67256742099995], [-73.73624941800001, 40.67263060199991], [-73.73593438599976, 40.67243297099988], [-73.73718424499995, 40.670909812999895], [-73.73735527499991, 40.67061884099995], [-73.7375467379998, 40.67033434399988], [-73.73775794399985, 40.67005754699992], [-73.73798806, 40.6697896359999], [-73.73823608799982, 40.66953170599987], [-73.73850086499984, 40.66928477299994], [-73.73869946900007, 40.6690682359999], [-73.73900041699997, 40.66874928599985], [-73.73921986399984, 40.66851888399989], [-73.73993331699991, 40.66776382799986], [-73.74080404599995, 40.666843967999924], [-73.74117817299991, 40.666402953999885], [-73.74440564500003, 40.66640390899992], [-73.74536651700001, 40.66640403199993], [-73.74608613500001, 40.66640039999987], [-73.74865865799985, 40.66638352799988], [-73.75073016999994, 40.666401901999954], [-73.75175886599992, 40.6663953199999], [-73.75318579199988, 40.66640002899992], [-73.75336722899982, 40.666400624999916], [-73.7552623119999, 40.666394765999954], [-73.75551141799991, 40.66639876399988], [-73.75602173499989, 40.66639171599998], [-73.75756437999986, 40.666448683999924], [-73.75763129599993, 40.66645115699992], [-73.75731692799982, 40.66704031599992], [-73.75699533399998, 40.66774126799994], [-73.75671602499992, 40.66856252699993], [-73.75668454799994, 40.66875852199992], [-73.75643734399979, 40.669666378999906], [-73.75640621299988, 40.670372914999895], [-73.7564247869999, 40.67063356499994], [-73.75646127199984, 40.67109403099993], [-73.7564837459999, 40.671489865999874], [-73.75646388399996, 40.67168400299994], [-73.75636239300002, 40.67242923699999], [-73.75643906099987, 40.672441159999906], [-73.75814760699996, 40.672706914999935], [-73.75857198899986, 40.67263733099987], [-73.75859630299982, 40.67274790399996], [-73.75880484999992, 40.67300565399993], [-73.76031463999993, 40.67510997099984], [-73.75950918099984, 40.67544720999995], [-73.75801266799994, 40.676077675999835], [-73.75789961600002, 40.67612340999989], [-73.756918817, 40.676528870999924], [-73.75687349699994, 40.67641731799989], [-73.75680933199992, 40.67630971499991], [-73.75672742400002, 40.67620876499991], [-73.75662976499991, 40.676116912999916], [-73.75648885799987, 40.67602276199989], [-73.75576169199995, 40.67588590899988], [-73.75563646999986, 40.67621517899991], [-73.75562236499997, 40.67625847299991], [-73.75549392499991, 40.67662967299988], [-73.75523457799997, 40.67728840899988], [-73.75484909699989, 40.67792883799996], [-73.75417856899982, 40.67884448599991], [-73.75399395199992, 40.679094506999895], [-73.75372596099987, 40.679457427999935], [-73.75345604299982, 40.67982295599992], [-73.75328689999988, 40.68005114499995], [-73.75323817199994, 40.68011688599991], [-73.75321057900005, 40.68015428799992], [-73.75268290999999, 40.6808696169999], [-73.75247059899989, 40.681180513999934], [-73.75225828699998, 40.6814914109999], [-73.75180409199986, 40.682403322999946], [-73.75154215499991, 40.683046042999926], [-73.75125793499984, 40.683734125999926], [-73.75103020599988, 40.68422286699994], [-73.75025615899999, 40.685613109999906], [-73.75008991099996, 40.685923445999926], [-73.74944546799985, 40.686672021999854], [-73.74884601399998, 40.68744748399996], [-73.74856993199998, 40.687803737999914], [-73.747551754, 40.689347336999944], [-73.74746788899986, 40.68932218099992], [-73.74125253099989, 40.68745783899989]]]}}, {\"id\": \"139\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 140, \"Shape_Leng\": 0.0475842911325, \"Shape_Area\": 0.000114203907779, \"zone\": \"Lenox Hill East\", \"LocationID\": 140, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95854329689507, 40.758495622967885], [-73.95877790811335, 40.75827092092435], [-73.95893448599985, 40.75833515499994], [-73.95903681099995, 40.75838378299988], [-73.95972180399991, 40.758663063999926], [-73.96196573699997, 40.759619460999865], [-73.96177668399997, 40.75987971599993], [-73.96176766499995, 40.75989304099994], [-73.96172701399999, 40.759949806999906], [-73.96169733399985, 40.75998713799993], [-73.96167816899991, 40.76001396599985], [-73.96164363299995, 40.76005959199991], [-73.96162070799981, 40.76009443399988], [-73.96152208599995, 40.76023035999991], [-73.96106754499992, 40.76085809299985], [-73.96060649299993, 40.76148739799985], [-73.9601506909999, 40.76211487499993], [-73.95969458599983, 40.762740538999935], [-73.95923908799995, 40.76336615799994], [-73.95878081199999, 40.763990016999855], [-73.95832133999988, 40.76461662999985], [-73.95786874099996, 40.765242324999875], [-73.95741202499985, 40.76586875399986], [-73.95695607099981, 40.766493435999884], [-73.95649564199995, 40.76711891499989], [-73.95599908699987, 40.76780389299993], [-73.9555017079999, 40.76848474299992], [-73.95503700499988, 40.769120065999864], [-73.95457540899991, 40.76975485699994], [-73.9541124529999, 40.7703890789999], [-73.95364859999984, 40.771024107999914], [-73.95318747599984, 40.771656893999925], [-73.95268514999995, 40.772339404999926], [-73.95043560599999, 40.77139110299989], [-73.94807387899996, 40.77039439499987], [-73.94779385500001, 40.77024915199992], [-73.94771625099996, 40.77021606799992], [-73.94766538399982, 40.77019325299993], [-73.94761452999984, 40.77017042899995], [-73.94752257899997, 40.770129694999895], [-73.9474897547817, 40.77011515400953], [-73.94814040181448, 40.76941675635809], [-73.94866416477879, 40.76885762439947], [-73.94908536768632, 40.76827827079564], [-73.94937332131488, 40.76788219105435], [-73.94957912071412, 40.767628930179555], [-73.95006979303056, 40.76702508838324], [-73.95066345154245, 40.76638071134939], [-73.95122497666563, 40.76577119897351], [-73.95176480900415, 40.765163711843144], [-73.95230963181133, 40.764554288606035], [-73.95336559963573, 40.763368170911676], [-73.954418260786, 40.76218410495109], [-73.95557264605307, 40.761168326230354], [-73.95587222339078, 40.760904712115575], [-73.95633318967775, 40.76048658556384], [-73.95697841727626, 40.75989675444595], [-73.95731737587941, 40.759592212256514], [-73.95759869445276, 40.75934231825778], [-73.95763772742096, 40.75930673989833], [-73.9580049661682, 40.75897199962927], [-73.95825660841928, 40.758749261193095], [-73.9583612533255, 40.75866024195484], [-73.95838528673994, 40.75863979740061], [-73.95842620189605, 40.758603873006834], [-73.95845501771939, 40.75858090209874], [-73.95849119520824, 40.75854590391523], [-73.95854329689507, 40.758495622967885]]]}}, {\"id\": \"140\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 141, \"Shape_Leng\": 0.0415144638712, \"Shape_Area\": 7.66545579019e-05, \"zone\": \"Lenox Hill West\", \"LocationID\": 141, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96177668399997, 40.75987971599993], [-73.96196573699996, 40.75961945999995], [-73.96308073599987, 40.760083162999884], [-73.96355618899993, 40.76028081899991], [-73.96431353599988, 40.760599230999944], [-73.9665833839999, 40.76155093499988], [-73.9661274729999, 40.762179299999914], [-73.96566959999997, 40.76280455599994], [-73.96521328099986, 40.7634313589999], [-73.96475544899988, 40.76405706999991], [-73.9642979039999, 40.76468198699991], [-73.96384291099993, 40.76530968599996], [-73.96340154099988, 40.76591119099987], [-73.96292865800001, 40.76655862799987], [-73.9624699899999, 40.76718493299991], [-73.96201413099995, 40.76781230799988], [-73.96155824299983, 40.76843761199996], [-73.9611017069999, 40.76905931899988], [-73.96060180599979, 40.769749186999924], [-73.96010955199993, 40.77043221099989], [-73.959644408, 40.771064265999925], [-73.95918179999991, 40.771696823999946], [-73.9587170739999, 40.77233167499991], [-73.95825534899998, 40.772965336999896], [-73.95779380499984, 40.773599896999926], [-73.95729409999986, 40.7742835549999], [-73.95505481599999, 40.773336850999954], [-73.95268514999995, 40.772339404999926], [-73.95318747599984, 40.771656893999925], [-73.95364859999984, 40.771024107999914], [-73.9541124529999, 40.7703890789999], [-73.95457540899991, 40.76975485699994], [-73.95503700499988, 40.769120065999864], [-73.9555017079999, 40.76848474299992], [-73.95599908699987, 40.76780389299993], [-73.95649564199995, 40.76711891499989], [-73.95695607099981, 40.766493435999884], [-73.95741202499985, 40.76586875399986], [-73.95786874099996, 40.765242324999875], [-73.95832133999988, 40.76461662999985], [-73.95878081199999, 40.763990016999855], [-73.95923908799995, 40.76336615799994], [-73.95969458599983, 40.762740538999935], [-73.9601506909999, 40.76211487499993], [-73.96060649299993, 40.76148739799985], [-73.96106754499992, 40.76085809299985], [-73.96152208599995, 40.76023035999991], [-73.96162070799981, 40.76009443399988], [-73.96164363299995, 40.76005959199991], [-73.96167816899991, 40.76001396699989], [-73.96169733399982, 40.75998713699988], [-73.96172701399999, 40.759949806999906], [-73.96176766499995, 40.75989304099994], [-73.96177668399997, 40.75987971599993]]]}}, {\"id\": \"141\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 142, \"Shape_Leng\": 0.0381758942321, \"Shape_Area\": 7.56537920738e-05, \"zone\": \"Lincoln Square East\", \"LocationID\": 142, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98147898799989, 40.7677250589999], [-73.98155301499993, 40.76768021699989], [-73.98163412099993, 40.76764472399989], [-73.98173478100003, 40.76761932799989], [-73.98173489899985, 40.76761929999993], [-73.98173502899988, 40.76761929099996], [-73.98184353599999, 40.767609717999925], [-73.98184380799981, 40.7676096909999], [-73.98184408100003, 40.767609708999885], [-73.98195388499981, 40.76761757899992], [-73.981954182, 40.76761759699986], [-73.98195445399982, 40.76761766099985], [-73.98205860599991, 40.76764244899991], [-73.9820588189999, 40.767642502999934], [-73.98205899699985, 40.76764257499984], [-73.98215157199994, 40.767681797999906], [-73.98236545099999, 40.7673921529999], [-73.98521720299992, 40.76859462999991], [-73.98806289599996, 40.769790799999925], [-73.98761512099996, 40.770415442999884], [-73.98714552199996, 40.771054037999875], [-73.98669559399988, 40.771668182999896], [-73.98665479099989, 40.77172445399991], [-73.98649075499988, 40.77195067399991], [-73.98645451899998, 40.77200064599989], [-73.98623798899993, 40.77229925699988], [-73.98618447899992, 40.77237203099984], [-73.98613338199989, 40.77244152299989], [-73.98602063499989, 40.77259485899986], [-73.98596463199985, 40.772671022999894], [-73.98582184799984, 40.77286520799991], [-73.98577822599991, 40.77292453299991], [-73.98575709599992, 40.77295327099989], [-73.98562257199983, 40.77313621999988], [-73.98556790099997, 40.773210570999936], [-73.98551605299986, 40.77328108299985], [-73.98546970299988, 40.7733441169999], [-73.9853325459999, 40.77353064599993], [-73.98487946600002, 40.77415567099991], [-73.98441202599994, 40.77480807799986], [-73.98397150999992, 40.77542927299991], [-73.98349614799992, 40.77604586099993], [-73.98304666800001, 40.776682676999876], [-73.98258286200002, 40.77730283799988], [-73.98215547099988, 40.7779722949999], [-73.98211778999992, 40.77801318099993], [-73.98162126599993, 40.778608082999874], [-73.98113503699982, 40.779291386999915], [-73.980673654, 40.77992518199991], [-73.97783054399989, 40.77872973099989], [-73.97499744000007, 40.77753254599988], [-73.97546067199991, 40.77689850299986], [-73.97595517399989, 40.7762197549999], [-73.97645763799986, 40.77553235199982], [-73.97691168299995, 40.774909123999905], [-73.97737084999984, 40.77428100699992], [-73.97782639399986, 40.77365619199991], [-73.97828220099997, 40.773031878999866], [-73.97872246800002, 40.772418897999934], [-73.97919214499984, 40.77178126499996], [-73.97965233399994, 40.771155326999896], [-73.98011181599992, 40.770528368999884], [-73.980563707, 40.76990532499988], [-73.9810229009999, 40.76927283399994], [-73.98164804599995, 40.76843632199994], [-73.98154597299994, 40.76838337499988], [-73.98154591499996, 40.76838333899988], [-73.98145349899993, 40.76832060299992], [-73.98145334399989, 40.768320494999905], [-73.98145320300002, 40.76832036999991], [-73.98137263399991, 40.76824931599992], [-73.98132588899983, 40.76818837999989], [-73.98129467599998, 40.7681188019999], [-73.98128253299978, 40.76804427399991], [-73.98129085699999, 40.76796955199995], [-73.98129086899992, 40.767969452999914], [-73.9812909039998, 40.76796936299995], [-73.98131853299991, 40.76789900199987], [-73.98136213699995, 40.767836738999904], [-73.98141469499981, 40.767777872999865], [-73.98147898799989, 40.7677250589999]]]}}, {\"id\": \"142\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 143, \"Shape_Leng\": 0.0541798538849, \"Shape_Area\": 0.00015109426901, \"zone\": \"Lincoln Square West\", \"LocationID\": 143, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.981627874, 40.780328934999936], [-73.98151911299976, 40.78028175799991], [-73.98140948699984, 40.780235418999894], [-73.980673654, 40.77992518199991], [-73.98113503699982, 40.779291386999915], [-73.98162126499993, 40.778608082999924], [-73.98211778999992, 40.77801318099993], [-73.98215547099988, 40.7779722949999], [-73.98258286099983, 40.7773028379999], [-73.98304666800001, 40.776682676999876], [-73.98349614799992, 40.77604586099993], [-73.98397150999983, 40.77542927399991], [-73.98441202599994, 40.77480807799986], [-73.98487946600002, 40.77415567099991], [-73.9853325459999, 40.77353064599993], [-73.98546970299988, 40.7733441169999], [-73.98551605299986, 40.77328108299985], [-73.98556790099997, 40.773210570999936], [-73.98562257199983, 40.77313621999988], [-73.98575709599992, 40.77295327099989], [-73.98577822599991, 40.77292453299991], [-73.98582184799984, 40.77286520799991], [-73.98596463199985, 40.772671022999894], [-73.98602063499989, 40.77259485899986], [-73.98613338199989, 40.77244152299989], [-73.98618447899992, 40.77237203099984], [-73.98623798899993, 40.77229925699988], [-73.98645451899998, 40.77200064599989], [-73.98649075499988, 40.77195067399991], [-73.98665479099989, 40.77172445399991], [-73.98669559399988, 40.771668182999896], [-73.98714552199996, 40.771054037999875], [-73.98761512099996, 40.770415442999884], [-73.98806289599996, 40.769790799999925], [-73.99059644699987, 40.77085480999995], [-73.990913076, 40.770987758999894], [-73.99365606199987, 40.772145961999854], [-73.99378189799992, 40.77219909099988], [-73.99387445099994, 40.77223816799994], [-73.99350262199991, 40.772751632999885], [-73.9935908639999, 40.77279007999994], [-73.99372534699988, 40.772864197999915], [-73.9937873699999, 40.77290384599992], [-73.99383108201548, 40.77293178742534], [-73.9938875450122, 40.77295574408191], [-73.99389125221833, 40.772955194438325], [-73.99396258575715, 40.77294465345442], [-73.99401262490255, 40.772882846315966], [-73.99412205804117, 40.77292405851296], [-73.99413665279414, 40.772901870087004], [-73.99430134219553, 40.77297002883192], [-73.99428153506716, 40.77299380153467], [-73.99437655287542, 40.77303955105066], [-73.99431863484354, 40.773121440135085], [-73.99429402991179, 40.77315624399584], [-73.9950232764303, 40.77348119678813], [-73.99507108848482, 40.773414138832905], [-73.9950893919464, 40.7733884750195], [-73.99501396385818, 40.773358034713475], [-73.99505028484955, 40.77329715309492], [-73.99624065194922, 40.77378979119886], [-73.99619583723546, 40.77385235609209], [-73.99610846890408, 40.77381631600756], [-73.99610778171481, 40.773817377800505], [-73.99606316854084, 40.773886886804995], [-73.99612697786858, 40.773913354810404], [-73.99612850491377, 40.77391398495152], [-73.99612636120096, 40.7739167138271], [-73.9960988071848, 40.77395180514951], [-73.99611835151377, 40.77396032398064], [-73.99617945998676, 40.77398695417577], [-73.99609524511304, 40.77408618621879], [-73.9960512527533, 40.77406889580327], [-73.99557226516114, 40.773870731394304], [-73.99393587681126, 40.773179512586104], [-73.99386194292893, 40.773269531698894], [-73.99382239352724, 40.77338175862297], [-73.99376701931838, 40.773483981224885], [-73.99369846374411, 40.77356214105259], [-73.99362199595426, 40.77363228631499], [-73.99352708391201, 40.77371445195511], [-73.99344271598801, 40.773770566055774], [-73.99341107446543, 40.773802630980256], [-73.99335832646867, 40.77392688832792], [-73.99262266386546, 40.774974056036925], [-73.99257784276604, 40.774956016359354], [-73.9925277439515, 40.775002110439836], [-73.9924697458154, 40.77502415955177], [-73.99240383719177, 40.77501814039074], [-73.99226708903535, 40.77511603385849], [-73.99205908493721, 40.7754975981922], [-73.99212537239482, 40.77550907505336], [-73.99222686779687, 40.77548221102612], [-73.9923293466088, 40.77546890095854], [-73.99236175680107, 40.775501899766596], [-73.99238604296023, 40.77555718042467], [-73.99208768471263, 40.77598397082139], [-73.99201877991067, 40.776073340735394], [-73.99142953190211, 40.77687956054528], [-73.99092717414973, 40.77756687876296], [-73.9907304133948, 40.77756055024258], [-73.99039616003645, 40.777585065679084], [-73.99010126597656, 40.77799645751658], [-73.9896488729282, 40.77862756000903], [-73.9892009588643, 40.77925239865152], [-73.98920068089916, 40.77925278678187], [-73.98917577843798, 40.77928752401572], [-73.9890851947858, 40.77940759766202], [-73.98886861739992, 40.779692922911416], [-73.98887187449955, 40.779713738253], [-73.98921902288056, 40.77969789520934], [-73.98927785904421, 40.77972343927113], [-73.98940905418009, 40.77973770647196], [-73.98949861492709, 40.779725044389714], [-73.98959649338828, 40.77969814668337], [-73.98967981290244, 40.77967756865804], [-73.98975270293778, 40.779671244211556], [-73.98984224780634, 40.77968075267069], [-73.9899443022714, 40.77969659327116], [-73.99004010212052, 40.77970767769822], [-73.99013797752488, 40.77969976970478], [-73.99022336845653, 40.779682361211215], [-73.99033584033211, 40.77966179439508], [-73.99043059869707, 40.779664973055496], [-73.99050764615511, 40.779668139123864], [-73.99062219939667, 40.77967606491436], [-73.99074506950537, 40.779671328184044], [-73.99087211428213, 40.779646007643926], [-73.9909616722243, 40.77963968375175], [-73.99105747282958, 40.77965235262575], [-73.99115742949688, 40.77966977560645], [-73.99124281740438, 40.77967136708451], [-73.99125531828973, 40.77965078251653], [-73.99129488711995, 40.77963020920887], [-73.99132196764978, 40.77963179604143], [-73.99135945556924, 40.77958588333735], [-73.9915510592274, 40.779574821437386], [-73.99141982585954, 40.77975528028706], [-73.98888614411702, 40.77987889853275], [-73.98893965670626, 40.779956178440386], [-73.98892610353073, 40.78005929201365], [-73.98891168026452, 40.78009603714661], [-73.9889116606735, 40.780122366770236], [-73.98891926146848, 40.78022609434367], [-73.98838105020263, 40.78098107404577], [-73.98823241384693, 40.78123314421559], [-73.98821042083165, 40.78122548254207], [-73.98814418135414, 40.78136567132723], [-73.98813788174485, 40.781379281274035], [-73.98812746184332, 40.78140179672363], [-73.98799363199988, 40.781342816999874], [-73.98746219899976, 40.781108609999905], [-73.98746089399998, 40.781108033999885], [-73.98741432699994, 40.7810875109999], [-73.98736363999998, 40.78106280499984], [-73.98730772799986, 40.78104130299993], [-73.98707137499986, 40.78090638199994], [-73.98654378999994, 40.780657980999905], [-73.98567936099997, 40.780312632999944], [-73.98536952700006, 40.781078372999914], [-73.98507184299989, 40.781779680999946], [-73.98351422299999, 40.781125389999914], [-73.98232616399993, 40.78062377299991], [-73.981627874, 40.780328934999936]]]}}, {\"id\": \"143\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 144, \"Shape_Leng\": 0.0276201668505, \"Shape_Area\": 4.74789670369e-05, \"zone\": \"Little Italy/NoLiTa\", \"LocationID\": 144, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99580912300002, 40.71664465899988], [-73.99605872699998, 40.7162316399999], [-73.99698375299992, 40.716540542999866], [-73.99776285999997, 40.71681392799996], [-73.99858925599978, 40.7170995119999], [-73.99931241699989, 40.71755024199989], [-73.99995620900002, 40.71801709499987], [-74.00053195499999, 40.718430238999886], [-74.00127273399993, 40.71896142599997], [-74.00140055699994, 40.719052619999886], [-74.00188406299992, 40.71939757099991], [-74.001523638, 40.719820797999944], [-74.00142450999995, 40.71993719799993], [-74.000660725, 40.72083979399993], [-73.99985330399981, 40.72179178799992], [-73.99952163199998, 40.72218334699987], [-73.9988117219999, 40.72302140899991], [-73.9977850889999, 40.724234893999906], [-73.99684295899995, 40.725347837999905], [-73.9967717579999, 40.7254319439999], [-73.99584577599984, 40.72520835299988], [-73.99532907899994, 40.72508373999996], [-73.99490827399991, 40.72495122899995], [-73.99417930499997, 40.724692406999914], [-73.99345769099995, 40.724432895999925], [-73.99339891300001, 40.72441250499989], [-73.99267182399991, 40.72416025399983], [-73.99260322299989, 40.724136449999925], [-73.99263254999987, 40.72405686599991], [-73.99308859399994, 40.72281909799992], [-73.99325915400001, 40.72235330999995], [-73.99352298099996, 40.721631692999935], [-73.99379527399988, 40.72093644299991], [-73.99403499399988, 40.720323010999884], [-73.99437745899998, 40.71951908899993], [-73.99442117799985, 40.719391298999874], [-73.99480778999998, 40.71845742599988], [-73.99542875899999, 40.71727924299992], [-73.99580912300002, 40.71664465899988]]]}}, {\"id\": \"144\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 145, \"Shape_Leng\": 0.114313844981, \"Shape_Area\": 0.000346641733797, \"zone\": \"Long Island City/Hunters Point\", \"LocationID\": 145, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94345411699997, 40.75205360799992], [-73.94291412899997, 40.7518188289999], [-73.94270657799986, 40.7517297909999], [-73.94256342299988, 40.75166697299994], [-73.94206838500003, 40.75146200499995], [-73.94143064399995, 40.75118721999992], [-73.94126747099995, 40.75111691199991], [-73.94069186599994, 40.75085230099989], [-73.94055712800004, 40.75079503999992], [-73.94044179899981, 40.750737789999896], [-73.94003764200001, 40.750559900999875], [-73.93963348799986, 40.75038201099994], [-73.9388134609999, 40.75006471699992], [-73.9381182169999, 40.74979264499992], [-73.937966578, 40.74966908599984], [-73.93782704799992, 40.74941481699988], [-73.93773732499994, 40.7492513079999], [-73.93759818899993, 40.749117109999844], [-73.93752799699986, 40.749049403999926], [-73.93745906699988, 40.74898291299991], [-73.93739653799992, 40.748916439999896], [-73.93683480500002, 40.7483945519999], [-73.93663681899982, 40.748204093999895], [-73.93673442199996, 40.7481289489999], [-73.93812286699995, 40.74705995499992], [-73.93880511399983, 40.74653465899992], [-73.93952057299987, 40.74598377899985], [-73.93996463499987, 40.74574558399994], [-73.94032696100007, 40.74541314299995], [-73.94184736499994, 40.74490368999989], [-73.94272343099986, 40.74457686299997], [-73.94342742899987, 40.74423117199993], [-73.94383757199998, 40.74387181399988], [-73.94399533799991, 40.74365885099996], [-73.94401030399996, 40.7436394539999], [-73.94419561599983, 40.742882145999914], [-73.94297613999994, 40.743368929999875], [-73.94211068899993, 40.74371373399991], [-73.94174116799988, 40.743859151999885], [-73.94031559800004, 40.7444044979999], [-73.93925967199992, 40.74474786799993], [-73.9382576189999, 40.7450684369999], [-73.9376968159998, 40.74523857399986], [-73.93726110699994, 40.745176490999896], [-73.93765115799985, 40.74330513499994], [-73.93801851799995, 40.741455023999904], [-73.93819856599988, 40.74054323199988], [-73.93862452799989, 40.73840478099991], [-73.93864769800001, 40.7383258729999], [-73.93866464499993, 40.738268158999844], [-73.93867968899998, 40.73820830199989], [-73.9386966449998, 40.73814081699989], [-73.93982348499995, 40.73844550399986], [-73.93996324500002, 40.73864135599992], [-73.93993706299993, 40.738688177999904], [-73.9406430239998, 40.73898181299989], [-73.94140886345744, 40.73929957085316], [-73.94134983794343, 40.739361630801525], [-73.94101774041413, 40.73967977289772], [-73.94077801743279, 40.73992178145365], [-73.94050888805725, 40.7401139417531], [-73.94036159285052, 40.740335345658146], [-73.9405614496907, 40.74041832293418], [-73.94028230242282, 40.74079928241357], [-73.93992390124144, 40.74135443063875], [-73.93965611215269, 40.741769215868466], [-73.93910874138116, 40.74261703517674], [-73.93866991753185, 40.74253408202486], [-73.93858786976764, 40.74293088268746], [-73.93930746279257, 40.74306775402492], [-73.93974435898761, 40.7431595125988], [-73.93975601918531, 40.74314401748943], [-73.9400192297566, 40.74320827221074], [-73.9401485852122, 40.742832441459974], [-73.93965301521378, 40.74273108709567], [-73.94004815102228, 40.742071226340684], [-73.94008406290678, 40.742063244641486], [-73.94014181484297, 40.74196918618731], [-73.94079972883497, 40.740897668398425], [-73.9411881669617, 40.740290218294476], [-73.94159470030033, 40.73986412616895], [-73.94165001879604, 40.739864154209705], [-73.94183504992756, 40.73969262021148], [-73.94203367165663, 40.739656938561154], [-73.94208664916357, 40.73959258700146], [-73.94215937774742, 40.73955163403233], [-73.94237973008305, 40.73943047196953], [-73.9424132194964, 40.7394227371066], [-73.94244816527319, 40.7394138966175], [-73.9424816492286, 40.73941169998687], [-73.94251076458535, 40.73941171452452], [-73.94253405960497, 40.73940840276209], [-73.94257045548689, 40.739406207056014], [-73.94259666844879, 40.73939514780283], [-73.94261561041536, 40.73937522646037], [-73.94262581394155, 40.73935862247503], [-73.94264913635598, 40.739323200416734], [-73.94267972562717, 40.7393021776812], [-73.94270012786521, 40.73927782815616], [-73.94271034058505, 40.73925125852501], [-73.94272766121334, 40.7392315971958], [-73.94279045640067, 40.739193721204316], [-73.94271139005915, 40.73914369336378], [-73.94269588366701, 40.73913388083167], [-73.942706089921, 40.73911506297872], [-73.94279106937542, 40.73906903345401], [-73.94290141828863, 40.73900380150434], [-73.94329026866804, 40.73900862239309], [-73.9435316603867, 40.7389228054914], [-73.94356853368713, 40.738925509493264], [-73.94360554747992, 40.73892436668734], [-73.94364201133408, 40.738919398390706], [-73.94367724507745, 40.73891069727868], [-73.94375068797268, 40.73889263076304], [-73.94382272278396, 40.738871537881835], [-73.94389313413845, 40.73884748169998], [-73.94392187163798, 40.73883662926439], [-73.94394170057961, 40.73882061714279], [-73.9439578703344, 40.73880237987792], [-73.94396995657188, 40.73878239605988], [-73.94433882538458, 40.738646239764854], [-73.9445925287478, 40.73857388800322], [-73.94480159543222, 40.73850057832631], [-73.9452386207261, 40.73835682364524], [-73.9452942041489, 40.73831485856154], [-73.9453584794753, 40.73826948539712], [-73.94548236590111, 40.738207031293705], [-73.94601135253046, 40.73812081475249], [-73.94715087181176, 40.737900123344716], [-73.9503362620346, 40.739006985843396], [-73.95039319194342, 40.73902756033958], [-73.95072468223914, 40.73913050454873], [-73.95136030838314, 40.73931497403829], [-73.95138071451841, 40.73933647729512], [-73.9519220819223, 40.73948730592797], [-73.95219672758512, 40.73953625486484], [-73.95221537377105, 40.73950345675014], [-73.952297207924, 40.73951862852658], [-73.95243355272876, 40.73954393499918], [-73.95249805377391, 40.73955863944509], [-73.95256648437781, 40.739575777453766], [-73.95263400587451, 40.73959128886933], [-73.95269882536086, 40.73960617945535], [-73.9527541290097, 40.739619700259], [-73.95283134418221, 40.73963599520285], [-73.95291652689204, 40.73965211309403], [-73.95301574053705, 40.73966917328721], [-73.95331835122245, 40.739718598686494], [-73.95341454477163, 40.73973626995544], [-73.95350014162969, 40.7397539274847], [-73.95367148545417, 40.7397943388994], [-73.95375351918099, 40.73981300381796], [-73.95377724201458, 40.739817218253506], [-73.95379878808706, 40.73981986527195], [-73.95380883920839, 40.73982029297784], [-73.95382002744792, 40.73981982922857], [-73.95385881619836, 40.73981511667895], [-73.9538740654045, 40.739814069884254], [-73.95399631710347, 40.73981409130872], [-73.95404538116455, 40.73981205786132], [-73.95416096154288, 40.739803269755015], [-73.95421335151977, 40.73980073347309], [-73.95434254243432, 40.739798848660556], [-73.95445339268437, 40.739800279169174], [-73.95457431562856, 40.73980474928995], [-73.95467405648948, 40.739811533534606], [-73.95474210608039, 40.73981589973216], [-73.95478421779814, 40.73981591627342], [-73.95479667341917, 40.739814715140724], [-73.95481021948797, 40.739812243579415], [-73.95485696896272, 40.739799708492896], [-73.95487585469716, 40.73979578103484], [-73.95491398069144, 40.73978998749271], [-73.95493966279436, 40.7397872687069], [-73.95498631086818, 40.7397845773776], [-73.95502314341954, 40.73978318588669], [-73.95505997596828, 40.739781796059255], [-73.95520363590497, 40.739778808074156], [-73.95522880785674, 40.739776792301264], [-73.95529063771633, 40.739768090207626], [-73.95531915980796, 40.73976557112596], [-73.95536636374354, 40.7397648873269], [-73.95541178942047, 40.73976708310083], [-73.95541891596714, 40.73976795046336], [-73.95542672904534, 40.739769466418096], [-73.95545971983971, 40.73977779083517], [-73.95546765067904, 40.7397790905562], [-73.95547496705139, 40.73977967855461], [-73.95549441822823, 40.73977956093091], [-73.9555140840356, 40.73977796613924], [-73.95551509086066, 40.73977788527793], [-73.95553724485578, 40.73977463360252], [-73.95555130014233, 40.739771785048795], [-73.95556162429453, 40.73976969863884], [-73.95556803152874, 40.73976788244475], [-73.95557485207941, 40.739765218893865], [-73.95560214990203, 40.73975193871615], [-73.9556084627258, 40.73974967116736], [-73.95561450106388, 40.739748142471235], [-73.95562429351857, 40.739746723946034], [-73.95563528007237, 40.73974612491633], [-73.95567927370665, 40.7397476015125], [-73.95568919593795, 40.73974743294591], [-73.95569800427987, 40.73974671664131], [-73.95587541922043, 40.73972178613248], [-73.95598072513853, 40.739711308622894], [-73.95611897472277, 40.73970515704294], [-73.95614908424905, 40.73970175566426], [-73.95621492962265, 40.73969192882607], [-73.95624261164724, 40.739689013331585], [-73.9563659348309, 40.73968033400858], [-73.95654189613127, 40.73966890169955], [-73.95655856629828, 40.73966688125481], [-73.95657678916139, 40.739663286477715], [-73.9566348235059, 40.73964760341772], [-73.95665583006708, 40.73964273061248], [-73.95673650026704, 40.73962737934407], [-73.9567987103568, 40.73961736183668], [-73.95682353901007, 40.739614724063564], [-73.95685053222071, 40.73961355462707], [-73.95688022501504, 40.73961380062033], [-73.95691509079995, 40.73961548792784], [-73.95691900947337, 40.7396158857611], [-73.95692303345511, 40.73961658116053], [-73.95693155675914, 40.739618934408156], [-73.95695727726076, 40.73962950689662], [-73.9569658816245, 40.73963253597347], [-73.95697362315164, 40.739634375625094], [-73.95697731731507, 40.73963489030601], [-73.95698094007172, 40.73963513423747], [-73.95702052948604, 40.739634950835075], [-73.95706179227868, 40.73963183281364], [-73.95710461786645, 40.739625797720045], [-73.95714889104126, 40.739616862430125], [-73.95717457418114, 40.73960988496115], [-73.95722993362249, 40.7395921200965], [-73.95725341436346, 40.739585969037165], [-73.95728369206074, 40.73958014549551], [-73.957365367112, 40.7395671003273], [-73.95741470766006, 40.739556582011616], [-73.95746607442864, 40.73954341851144], [-73.95752016571689, 40.739527436022044], [-73.95757867555936, 40.739508141128056], [-73.9575845020466, 40.73950576741509], [-73.95758998468358, 40.739502761163195], [-73.95759520937837, 40.739499106488836], [-73.95760021934018, 40.73949474092096], [-73.95760956733187, 40.739484191373265], [-73.95763455804378, 40.73945032204577], [-73.95763940313131, 40.73944545299868], [-73.95764437858944, 40.739441303692615], [-73.95764957785607, 40.73943779341431], [-73.9576550372954, 40.739434885991216], [-73.95769167671632, 40.73942022163393], [-73.95778977239853, 40.73938959544457], [-73.9578129824211, 40.73938121104517], [-73.95783305440638, 40.73937267287573], [-73.95787174668567, 40.73935247028819], [-73.95796044362709, 40.739299912652086], [-73.95798514776511, 40.739287351325665], [-73.95800848806388, 40.73927730138706], [-73.95803720372618, 40.73926803625332], [-73.95810634348692, 40.73925060919697], [-73.9581231238344, 40.73924510435212], [-73.95813766520311, 40.73923911236206], [-73.95825512163869, 40.73918362971911], [-73.95836705562307, 40.73913333207909], [-73.95846682395378, 40.73909505247406], [-73.95858431411132, 40.73904116201234], [-73.95868260497008, 40.738998261143394], [-73.9587857625725, 40.73895460555049], [-73.95881278544756, 40.73894415168756], [-73.95887504953102, 40.738922192299086], [-73.9589024865604, 40.7389111977878], [-73.958924869193, 40.73890039093979], [-73.95898468852309, 40.73886827282493], [-73.95902844677197, 40.738848054061364], [-73.95913624628112, 40.73880122082438], [-73.95918469317864, 40.738779059353476], [-73.95920597529579, 40.73876691855057], [-73.95925086376484, 40.738735831492974], [-73.95926236387591, 40.738729243733246], [-73.95927346121162, 40.73872399754858], [-73.95939408153595, 40.738677357907704], [-73.95953181719855, 40.73861833273097], [-73.9596359543351, 40.738576270229785], [-73.9597242033727, 40.738535599249026], [-73.95978270346585, 40.73851171066125], [-73.95983699528468, 40.73849397164276], [-73.95985404654238, 40.7384900965041], [-73.95987349884916, 40.73848746390514], [-73.95993158356721, 40.738484639028044], [-73.95995298823041, 40.73848515960659], [-73.96000509976616, 40.73849124753222], [-73.96001767208647, 40.73849209891121], [-73.96002897845497, 40.73849216482071], [-73.96004172884936, 40.738490990697215], [-73.96005462382847, 40.73848817519884], [-73.96006757901837, 40.73848374962379], [-73.9600804629096, 40.738477748101616], [-73.9600932301298, 40.73847017095191], [-73.96010554777818, 40.73846119597235], [-73.96011713293024, 40.73845103347817], [-73.96012759268096, 40.73844002408245], [-73.96013085166251, 40.73843563903461], [-73.9601334583406, 40.738430769107055], [-73.96013545062728, 40.738425365897776], [-73.96013686315297, 40.73841934280767], [-73.96013848536926, 40.73837979256485], [-73.96014056496966, 40.73836686225712], [-73.96014265220762, 40.73836104512276], [-73.96014547202849, 40.738355895332084], [-73.96014908547855, 40.73835134907848], [-73.96015351682223, 40.73834734438553], [-73.96016047979936, 40.738342780024915], [-73.96016917242167, 40.73833850675278], [-73.96020045878585, 40.73832699024539], [-73.96021115211703, 40.738322365186036], [-73.96022036568877, 40.7383168661319], [-73.96022403678212, 40.7383138866028], [-73.96022715341631, 40.73831070048942], [-73.96027048974304, 40.73825993585103], [-73.9602854405676, 40.73824399232705], [-73.96072580373679, 40.73808017017914], [-73.96170910164201, 40.73821891866324], [-73.9619544165527, 40.73824088226492], [-73.96209998908978, 40.73826019177947], [-73.96213404613793, 40.73826588531026], [-73.9621641612824, 40.738272234909694], [-73.96216502515647, 40.73827250641872], [-73.96216578260618, 40.73827285746795], [-73.96216611373971, 40.738273064471755], [-73.9621664089579, 40.73827329843555], [-73.96216669315326, 40.73827355048873], [-73.96216693042085, 40.738273830000594], [-73.96216734434721, 40.73827446037087], [-73.96216763907424, 40.73827517128228], [-73.96216782846125, 40.73827599155365], [-73.96216791163884, 40.738276900579265], [-73.96216781615563, 40.73827865722968], [-73.96216739979442, 40.73828076340235], [-73.96216465998417, 40.738290353220926], [-73.96215854719352, 40.738316826581595], [-73.96213733911205, 40.738387860351764], [-73.96213345576498, 40.73840947182256], [-73.96213242556007, 40.738429038901685], [-73.96213266090855, 40.73843153327963], [-73.9621332643156, 40.738433982881446], [-73.96213424591258, 40.738436388213096], [-73.96213559336611, 40.7384387486004], [-73.96213942723989, 40.73844335063819], [-73.96214483395143, 40.73844786423642], [-73.96215095325073, 40.73845183729329], [-73.9621586111074, 40.73845597235087], [-73.96218880375358, 40.73846989550221], [-73.96226893180528, 40.73850949938995], [-73.96235370202787, 40.738546915671805], [-73.96239876314084, 40.738563751885486], [-73.96242479019499, 40.73857636858405], [-73.96244960919122, 40.738590171807274], [-73.9624729587812, 40.73860501036201], [-73.96249462716804, 40.738620739939805], [-73.96251438912807, 40.73863719897305], [-73.96253213723082, 40.73865429629341], [-73.96254775525007, 40.73867191560047], [-73.96256114655104, 40.73868995735329], [-73.962589594156, 40.73873601859644], [-73.96260615442901, 40.73876172434984], [-73.96262085588614, 40.73900493653315], [-73.96243311685276, 40.739661471070654], [-73.96220916495865, 40.740006175249036], [-73.96159204782984, 40.740003968231605], [-73.96152351376229, 40.74016007237984], [-73.96146496178848, 40.740300801408054], [-73.96145013844338, 40.74033783468036], [-73.96164163979053, 40.74035611292448], [-73.96162982563845, 40.74045892945751], [-73.96129339923257, 40.74043279936792], [-73.96126581236175, 40.74053747110824], [-73.96151773884108, 40.74060680575996], [-73.96172732138895, 40.74066968474287], [-73.96185009745176, 40.74072448163034], [-73.9619093021767, 40.74086783324394], [-73.96188806252042, 40.74098216980949], [-73.96180397489096, 40.741097673472154], [-73.96172630358241, 40.74111465495805], [-73.96164711213778, 40.74112691976451], [-73.96156689485383, 40.74113439133732], [-73.96148615243109, 40.74113702304026], [-73.96140538884809, 40.74113479844658], [-73.96139150357843, 40.741145045041016], [-73.96138001676614, 40.74115688552988], [-73.96137123973048, 40.74116999900953], [-73.96136541035027, 40.7411840300758], [-73.96136268661652, 40.74119859845559], [-73.9613631423506, 40.741213309312805], [-73.9613667652031, 40.741227763950015], [-73.96136759981154, 40.74122994626042], [-73.96137343155077, 40.741235563969695], [-73.96137796957133, 40.7412418345074], [-73.96138109036006, 40.74124858720552], [-73.96138270897617, 40.74125563827292], [-73.961382781365, 40.74126279579758], [-73.96138130555555, 40.74126986496982], [-73.96137832171493, 40.741276653384745], [-73.96137391105589, 40.7412829762789], [-73.96136819362471, 40.74128866155893], [-73.96136132503524, 40.74129355448597], [-73.96135349223279, 40.74129752188664], [-73.96134490840704, 40.74130045577837], [-73.9613358071877, 40.74130227630778], [-73.96132159596328, 40.74131197576757], [-73.96130559565955, 40.741319901099274], [-73.96128818552575, 40.741325864451376], [-73.96126977822759, 40.741329724476905], [-73.9612508100668, 40.74133138968266], [-73.96123173063957, 40.74133082059912], [-73.96121299217897, 40.74132803071502], [-73.96120668509533, 40.741326563405515], [-73.96119094440999, 40.74132588922623], [-73.96117524159504, 40.741326958520645], [-73.96115990869546, 40.74132974867761], [-73.9611452699347, 40.7413342006979], [-73.96113163485765, 40.74134022044097], [-73.96111929178639, 40.7413476806157], [-73.96110494965113, 40.741384571551684], [-73.96109729243683, 40.74142259809855], [-73.96109646058719, 40.7414610628525], [-73.96110246937008, 40.74149926037206], [-73.96111520859633, 40.74153649011531], [-73.9611165295705, 40.74154199815666], [-73.96111742661412, 40.7415458850702], [-73.96112791580276, 40.74160867474958], [-73.96113102375936, 40.7416719250793], [-73.96112672584678, 40.74173513509469], [-73.96111505608512, 40.741797804148945], [-73.96127192619346, 40.74183259794321], [-73.96129180598602, 40.74179635293707], [-73.96136329157102, 40.74181148128098], [-73.96136519326672, 40.74182328939675], [-73.96136466867709, 40.74183517907704], [-73.96136173044927, 40.741846863658154], [-73.9613564494236, 40.741858061421226], [-73.96134895292636, 40.74186850238442], [-73.9613394216989, 40.74187793481262], [-73.96110814501404, 40.74182802590792], [-73.96117277851152, 40.74188633414116], [-73.96155253168952, 40.74199326140724], [-73.96119029789011, 40.74264485525972], [-73.9611136948121, 40.742748549137914], [-73.96109548654543, 40.742781947731416], [-73.96097168385474, 40.74275405426372], [-73.96093449806607, 40.742829760022445], [-73.96064602180344, 40.74339270626316], [-73.96061022103159, 40.74343138134336], [-73.96056778448636, 40.74346595651384], [-73.96051951418062, 40.743495778327514], [-73.96046632238422, 40.74352028317143], [-73.96040921438393, 40.74353900791941], [-73.96034926948344, 40.74355159868528], [-73.96028762060436, 40.74355781751104], [-73.96022543287424, 40.743557546864594], [-73.9601638816042, 40.74355079186102], [-73.95978468552276, 40.74349771658391], [-73.95971981529016, 40.743693045465214], [-73.9598049477139, 40.74370675289005], [-73.95974393197373, 40.74387256884914], [-73.95975123850988, 40.74390227665779], [-73.96051032999415, 40.74404689195289], [-73.96052090981668, 40.744019740075224], [-73.96056856858455, 40.74402966658535], [-73.960672045794, 40.744051215855876], [-73.96064863700711, 40.74413031761407], [-73.96051732602352, 40.744105341081195], [-73.96011951918781, 40.74402967229443], [-73.96010602637564, 40.744058709806936], [-73.95992298789855, 40.744027042096036], [-73.95993311497531, 40.74399458841103], [-73.95963225023952, 40.74393463191414], [-73.95955401304717, 40.74401549720348], [-73.95936198209034, 40.7442139760372], [-73.9594036821734, 40.7442260747471], [-73.9593910660955, 40.74428332474386], [-73.95946013173084, 40.744292519797156], [-73.95946913213879, 40.74426775269981], [-73.95971280471103, 40.74431481791596], [-73.95972855228639, 40.744272115619374], [-73.96015525517686, 40.744368782605555], [-73.96012600962386, 40.74444393699146], [-73.96001147139248, 40.744422543044365], [-73.96002610056895, 40.74437386231268], [-73.95980151741632, 40.74432936858096], [-73.95979139383248, 40.74435584424729], [-73.95954322849366, 40.744308778289835], [-73.95952410819169, 40.744354895906056], [-73.95936852954134, 40.74432480534333], [-73.95929971633628, 40.74432422217917], [-73.95923368945668, 40.74431004302651], [-73.95921597110394, 40.74432602663238], [-73.95916115518868, 40.74436781692285], [-73.9591002268356, 40.7444043845031], [-73.95903404086978, 40.744435216324256], [-73.95897577051586, 40.744451507918825], [-73.95891561234897, 40.744463173779614], [-73.95885419935777, 40.74447009115741], [-73.95881841584692, 40.74450890726829], [-73.9587753700448, 40.74455559986341], [-73.95918723985115, 40.7446210117597], [-73.95922188572065, 40.74455380563087], [-73.95941134001788, 40.74459114362481], [-73.95941819442535, 40.74459275982058], [-73.95942479652716, 40.74459490145693], [-73.95943107519165, 40.74459754354726], [-73.9594369665561, 40.74460066211317], [-73.95944240940852, 40.74460422061215], [-73.9594473453993, 40.744608183508426], [-73.9594517260948, 40.74461250974094], [-73.95945549887863, 40.744617154059156], [-73.95945863272348, 40.7446220660267], [-73.9594610886709, 40.74462719738258], [-73.95946284296743, 40.74463249048935], [-73.95946387758197, 40.744637893575], [-73.95946417955433, 40.7446433481684], [-73.9594637511212, 40.74464879848418], [-73.95946329058697, 40.744654877347095], [-73.95946204747965, 40.74466089177205], [-73.95946003174586, 40.74466678513858], [-73.95945726676932, 40.74467249882063], [-73.95945377637196, 40.74467797871584], [-73.95944959362537, 40.74468317256721], [-73.95944476107415, 40.7446880262789], [-73.95943932500018, 40.74469249781804], [-73.95943333829659, 40.74469653979321], [-73.95942685803531, 40.74470011553626], [-73.95941994745536, 40.7447031888835], [-73.95941267265367, 40.744705732384], [-73.95940510501366, 40.744707719091174], [-73.95939731899777, 40.74470913043566], [-73.95938938708176, 40.744709954381165], [-73.95938138680653, 40.74471017989835], [-73.95937339680776, 40.74470980818752], [-73.95887919943338, 40.74462433145469], [-73.95888268557037, 40.74461262144167], [-73.95876372515237, 40.74459261916389], [-73.95864096676264, 40.744948314954996], [-73.95874827896084, 40.744969569298654], [-73.95875870330303, 40.74495372973999], [-73.95964910654475, 40.745144160772824], [-73.95965229755977, 40.745216588173435], [-73.95905544841918, 40.74510830469203], [-73.95907628880576, 40.74509246880723], [-73.95862255916191, 40.74500781617753], [-73.95849389303115, 40.74498786491623], [-73.95847854708339, 40.745038091625844], [-73.95847142298703, 40.745060241387236], [-73.95845252269615, 40.74511901232469], [-73.95862451222585, 40.745149431165274], [-73.95862450071114, 40.74516797195996], [-73.95873752408313, 40.745195089437466], [-73.95914044096016, 40.74529176370699], [-73.95916949892894, 40.74529287343873], [-73.95919801002645, 40.74529728278581], [-73.95922533022177, 40.745304892146244], [-73.95925084238363, 40.745315529633864], [-73.95927397022142, 40.74532895496046], [-73.95929419130337, 40.74534486486368], [-73.95931104885652, 40.74536289995734], [-73.95932982273783, 40.7453972477556], [-73.95934071931042, 40.745433510198616], [-73.95934342702446, 40.74547065053109], [-73.95933786845197, 40.7455076068965], [-73.95932420250084, 40.74554332269665], [-73.95930281987525, 40.7455767767997], [-73.95927433190408, 40.745607012735555], [-73.95924338775806, 40.745645143561696], [-73.95920557605818, 40.74567950684697], [-73.95916167785694, 40.74570939276158], [-73.95911259993665, 40.745734183962796], [-73.95905935607936, 40.74575336834631], [-73.95894743371304, 40.74577118529589], [-73.95878289860286, 40.74573332038775], [-73.9585686575644, 40.74568891742148], [-73.9584760944999, 40.745688884087585], [-73.95839861481154, 40.7457194194454], [-73.95838046238632, 40.74575468973873], [-73.9583696922635, 40.74579166941702], [-73.95836656305252, 40.74582947057589], [-73.95837114990087, 40.74586718558513], [-73.95838334268795, 40.74590390888174], [-73.95837927151231, 40.74595309009328], [-73.95899427160536, 40.74606464643515], [-73.95864684421029, 40.74654483695282], [-73.958670723744, 40.746643109693856], [-73.95854662645935, 40.746780687368926], [-73.95844912833226, 40.74687785428838], [-73.95840482607544, 40.74689804899328], [-73.95838708370567, 40.74693942457044], [-73.95834783270944, 40.74697694378918], [-73.95833641386992, 40.74702409778029], [-73.95831740194431, 40.747072210120145], [-73.9567945329049, 40.748839516955584], [-73.95485394268042, 40.74804079464104], [-73.95473347953265, 40.748018725711106], [-73.95467737259249, 40.7481331739454], [-73.95470926432743, 40.748150170963356], [-73.95465178220594, 40.748232644095495], [-73.95432006769612, 40.748103917081885], [-73.95439031474363, 40.748019022887604], [-73.95464547239432, 40.74812830865803], [-73.95470561241689, 40.74801362045242], [-73.95318382492853, 40.74773481196097], [-73.95305375571401, 40.74808748722927], [-73.95478387869785, 40.74842768777324], [-73.9563216250964, 40.749081554434866], [-73.95638045449142, 40.74903400751159], [-73.9564856879466, 40.74911169015581], [-73.95625582868091, 40.74933482748637], [-73.9561349924058, 40.74926462627477], [-73.95576039140944, 40.74959665953019], [-73.9561741642638, 40.74966509499414], [-73.95619904110015, 40.74967096066778], [-73.95620052208228, 40.74968568845317], [-73.95611932084378, 40.74975783785615], [-73.956086873372, 40.749766133278605], [-73.95606185080376, 40.74976141809474], [-73.95559517265838, 40.74968396904214], [-73.95546436936412, 40.74967421308257], [-73.95534928514437, 40.75001385690402], [-73.95521529965907, 40.74999196768931], [-73.95522809429409, 40.74994344604038], [-73.95513239152461, 40.74992642433233], [-73.95520275040869, 40.749676537409485], [-73.95506652560155, 40.74963273037535], [-73.95451584573382, 40.75058669897673], [-73.9546500041312, 40.75083104095946], [-73.95431654115386, 40.75125819070782], [-73.95394013937512, 40.75174554742943], [-73.95399353526572, 40.751786195519045], [-73.95380171765365, 40.75195818796781], [-73.953679096483, 40.75204656300798], [-73.95364139195567, 40.75203698846648], [-73.95297152022262, 40.75274650142646], [-73.95300921156827, 40.75277519542741], [-73.95284882052746, 40.75294002880028], [-73.95265700069434, 40.75310724080276], [-73.9521726236062, 40.75370276114808], [-73.9520814316973, 40.753757041221704], [-73.95177949644007, 40.75408671350756], [-73.95162538718246, 40.7542491588566], [-73.95136748557822, 40.754523883237226], [-73.9510179273249, 40.755086919725095], [-73.9508770195158, 40.75516691420223], [-73.95083917142055, 40.75521808361641], [-73.95080763809156, 40.75526368005708], [-73.9491129519999, 40.75452361999986], [-73.9482744449999, 40.75415377799993], [-73.94745112999996, 40.75379680499989], [-73.94655794699989, 40.75340986099995], [-73.94635771599988, 40.75332271699992], [-73.94597511599991, 40.75315619699993], [-73.9457177839999, 40.753044195999934], [-73.94552905199996, 40.752957406999876], [-73.94504698699998, 40.75273571599994], [-73.94481659399985, 40.75263169799989], [-73.94458401999987, 40.752534888999946], [-73.944052652, 40.75231325999992], [-73.94345411699997, 40.75205360799992]]]}}, {\"id\": \"145\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 146, \"Shape_Leng\": 0.0471405215717, \"Shape_Area\": 0.000104638923425, \"zone\": \"Long Island City/Queens Plaza\", \"LocationID\": 146, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93675697199996, 40.74914697199991], [-73.93739653799992, 40.748916439999896], [-73.93745906699988, 40.74898291299991], [-73.93752799699986, 40.749049403999926], [-73.93759818899993, 40.749117109999844], [-73.93773732499994, 40.7492513079999], [-73.93782704799992, 40.74941481699988], [-73.937966578, 40.74966908599984], [-73.9381182169999, 40.74979264499992], [-73.9388134609999, 40.75006471699992], [-73.93963348799986, 40.75038201099994], [-73.94003764200001, 40.750559900999875], [-73.94044179899981, 40.750737789999896], [-73.94055712800004, 40.75079503999992], [-73.94069186599994, 40.75085230099989], [-73.94126747099995, 40.75111691199991], [-73.94143064399995, 40.75118721999992], [-73.94206838500003, 40.75146200499995], [-73.94256342299988, 40.75166697299994], [-73.94270657799986, 40.7517297909999], [-73.94291412899997, 40.7518188289999], [-73.94345411699997, 40.75205360799992], [-73.944052652, 40.75231325999992], [-73.94401823399977, 40.75234992099993], [-73.94396284399986, 40.752404075999905], [-73.94392521099985, 40.752441104999924], [-73.94391620499988, 40.75245068799991], [-73.9438353679999, 40.75253012299991], [-73.94243020699994, 40.75399761299994], [-73.94123861599985, 40.755234414999954], [-73.94004034700002, 40.756486750999926], [-73.93910030499991, 40.7574941399999], [-73.93791705599988, 40.7587066049999], [-73.93679091699993, 40.75988288099991], [-73.93588175099988, 40.75945697999986], [-73.93579879199996, 40.759418044999904], [-73.93505987399978, 40.759071228999865], [-73.93503410199992, 40.75905925799991], [-73.93496053299995, 40.75902508599996], [-73.93425020699995, 40.75869513399993], [-73.93335306599997, 40.75827433399995], [-73.93215200299993, 40.757710384999925], [-73.93135128699987, 40.75733408499986], [-73.93055267499999, 40.75695894299989], [-73.92975052199988, 40.75658057799992], [-73.92895587499993, 40.75621258399985], [-73.92815372599988, 40.755835855999905], [-73.92735228500001, 40.75546007599991], [-73.92655412999981, 40.75508563499988], [-73.92575429299997, 40.75471089899986], [-73.92495400099993, 40.75433675199992], [-73.92415699999985, 40.753961141999845], [-73.92562743799989, 40.752137243999904], [-73.92596072499995, 40.75209646699995], [-73.92646866899993, 40.752068567999856], [-73.92790807399994, 40.75205079499992], [-73.92874215599983, 40.75204162499994], [-73.9291422419998, 40.75203153499992], [-73.93015432399986, 40.75201212499993], [-73.93126506799997, 40.75199252199999], [-73.93195863799987, 40.75197425899994], [-73.932276233, 40.75195388599997], [-73.93383269399993, 40.7516047019999], [-73.93474259199988, 40.75097904399989], [-73.93609498999984, 40.7497315299999], [-73.93616086999987, 40.74967406999993], [-73.93645660599992, 40.74941256799986], [-73.93675697199996, 40.74914697199991]]]}}, {\"id\": \"146\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 147, \"Shape_Leng\": 0.0587654949268, \"Shape_Area\": 0.000106418172757, \"zone\": \"Longwood\", \"LocationID\": 147, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89515277899991, 40.82783428999991], [-73.89483522899992, 40.82624684099993], [-73.89487088500002, 40.82393727199992], [-73.89389702299998, 40.82398399799995], [-73.89320004999998, 40.82400818299992], [-73.89390164399987, 40.823643120999904], [-73.89433874699982, 40.82341635299983], [-73.89488228899991, 40.82313435499987], [-73.89586019599982, 40.82263104599987], [-73.8958813589999, 40.82099257499987], [-73.89490478799985, 40.82098751999995], [-73.89387380299988, 40.820979334999926], [-73.89283653100001, 40.820974054999965], [-73.89290156199988, 40.82081122299989], [-73.89389462500002, 40.8198602899999], [-73.894930762, 40.81884671999994], [-73.89567610799998, 40.8181293679999], [-73.8970446669999, 40.8168018599999], [-73.8962135219999, 40.81631897099989], [-73.89580071599984, 40.816065648999874], [-73.89542858799986, 40.81584467199988], [-73.89685073199996, 40.81446016799987], [-73.89708341799987, 40.8142243469999], [-73.89733049099995, 40.81399633599995], [-73.89759123799989, 40.8137768729999], [-73.89775779499988, 40.81364891599991], [-73.89786486599995, 40.81356666999989], [-73.8981505179999, 40.81336635499989], [-73.89844723499995, 40.81317650299993], [-73.89912449199987, 40.81269524099988], [-73.90090498099991, 40.81142871299987], [-73.90259424199986, 40.81022335899995], [-73.90279087799998, 40.81008304699985], [-73.90307457199998, 40.80988061199988], [-73.90324273499989, 40.81071288499991], [-73.90341720399988, 40.811506882999886], [-73.90418036399996, 40.81212482099991], [-73.9044655149999, 40.81228195999994], [-73.90418337799991, 40.81308645499989], [-73.90380412299992, 40.81413184199994], [-73.9034699719999, 40.81507578799996], [-73.90306443799986, 40.8161067619999], [-73.90300983199995, 40.81631779199992], [-73.90257859099984, 40.81736979399992], [-73.90210743299994, 40.81850519199991], [-73.90166319399991, 40.81955042599993], [-73.90160121199982, 40.81969485899989], [-73.9012927759998, 40.820475442999914], [-73.90084561199987, 40.821541417999846], [-73.9006114109999, 40.822108807999875], [-73.90198758599993, 40.82244042299992], [-73.901846838, 40.822778632999885], [-73.90181687399985, 40.82285063499993], [-73.901176691, 40.8243889089999], [-73.90050540499992, 40.82600762199993], [-73.90164895599995, 40.82625697799991], [-73.90093510699997, 40.82792810199992], [-73.89979756799993, 40.827684353999906], [-73.89940994800001, 40.827734183999915], [-73.89932566999991, 40.82774501799988], [-73.898204693, 40.82788910899995], [-73.89700278099993, 40.8280504559999], [-73.89576668599983, 40.8282118029999], [-73.89548522799996, 40.82824854099993], [-73.89529156500001, 40.82814234999993], [-73.8951102089999, 40.82803689899988], [-73.89515277899991, 40.82783428999991]]]}}, {\"id\": \"147\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 148, \"Shape_Leng\": 0.0391305015633, \"Shape_Area\": 6.97489921327e-05, \"zone\": \"Lower East Side\", \"LocationID\": 148, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98447731699999, 40.72023423899987], [-73.985073423, 40.71908329399989], [-73.9859109239999, 40.7193389729999], [-73.98675136699994, 40.71959284599997], [-73.98736015299995, 40.71836979699993], [-73.98739393199992, 40.718303522999896], [-73.98743405799996, 40.71821307999992], [-73.98751208099989, 40.71807218599988], [-73.98787935799997, 40.7174089729999], [-73.98791645999997, 40.71733557399992], [-73.98795181500003, 40.717265627999915], [-73.98836626299988, 40.71644570199991], [-73.98915463799979, 40.716708646999905], [-73.98975418199988, 40.71555234799997], [-73.9902023599999, 40.71466442999988], [-73.99022155699988, 40.71457875599985], [-73.99022750999988, 40.714491668999926], [-73.99022012699989, 40.71440493999988], [-73.99075515399984, 40.71455407299985], [-73.99122551100002, 40.714486643999926], [-73.99213486499997, 40.7144198019999], [-73.99256242199989, 40.71438807699996], [-73.99282173499992, 40.71436522999989], [-73.99301588399996, 40.714355624999875], [-73.99395033499991, 40.71429083199993], [-73.99414039099992, 40.714268931999854], [-73.99431855699987, 40.71425088499997], [-73.99434786700003, 40.71424791599985], [-73.99450033499993, 40.71423116399996], [-73.99465684999993, 40.71421944699987], [-73.99472479499984, 40.71421571299987], [-73.99484048899991, 40.71420935199992], [-73.99647830999992, 40.71410726399991], [-73.99744448599988, 40.71406687299986], [-73.99750445299988, 40.71406913199995], [-73.99709223399998, 40.71461839799991], [-73.99680607099994, 40.7150369559999], [-73.99653128199989, 40.71549139099988], [-73.99622493199993, 40.71599579099993], [-73.99614929199988, 40.716084840999955], [-73.99605872699998, 40.7162316399999], [-73.99580912300002, 40.71664465899988], [-73.99542875899999, 40.71727924299992], [-73.99480778999998, 40.71845742599988], [-73.99442117799985, 40.719391298999874], [-73.99437745899998, 40.71951908899993], [-73.99403499399988, 40.720323010999884], [-73.99379527399988, 40.72093644299991], [-73.99352297999984, 40.72163169299988], [-73.99325915400001, 40.72235330999995], [-73.99308859399994, 40.72281909799992], [-73.99263254999998, 40.72405686499989], [-73.99260322199994, 40.72413644999987], [-73.99250385799984, 40.72410592499995], [-73.99135472799999, 40.72375461899987], [-73.99127307099994, 40.7237298129999], [-73.99102410399989, 40.723649431999895], [-73.99027361600001, 40.72342601799986], [-73.98939218099996, 40.723174057999884], [-73.9887568319999, 40.72297036599986], [-73.98863862799996, 40.7229337199999], [-73.98854361399988, 40.722903617999904], [-73.98786491500006, 40.72269708899991], [-73.98713584799998, 40.7224698109999], [-73.98628574000003, 40.722241919999874], [-73.98551686199984, 40.72198522499986], [-73.98467780299995, 40.72173060499991], [-73.98382387299982, 40.72147287199987], [-73.98386422799992, 40.72139637799989], [-73.98447731699999, 40.72023423899987]]]}}, {\"id\": \"148\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 149, \"Shape_Leng\": 0.0836805821837, \"Shape_Area\": 0.000270639618955, \"zone\": \"Madison\", \"LocationID\": 149, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94405560899996, 40.61199284199993], [-73.94380031999991, 40.610719118999945], [-73.94369953899981, 40.61021403899986], [-73.94364091199994, 40.609858816999896], [-73.94354698000001, 40.6093465549999], [-73.94231175899985, 40.60826819699985], [-73.94187229999983, 40.60855546699988], [-73.9415475769999, 40.608767735999905], [-73.94082897899995, 40.609229317999905], [-73.94011528799989, 40.60969107799986], [-73.93934703399997, 40.610187140999884], [-73.93753749399987, 40.60855738999986], [-73.93829679599983, 40.60806269399995], [-73.93901246899993, 40.60759947699994], [-73.93933968999978, 40.60738555999993], [-73.93939269099997, 40.60735090899988], [-73.93972613899992, 40.60713291499995], [-73.9384349359999, 40.605973862999896], [-73.93791557899979, 40.60549790099989], [-73.9360940729999, 40.603873161999886], [-73.93426266699994, 40.60223071099988], [-73.93503274599996, 40.60173378999984], [-73.93447158899997, 40.60123151199989], [-73.93525981999983, 40.60114234799988], [-73.93619686399984, 40.60105506999984], [-73.93715062999982, 40.6009488659999], [-73.93811308699989, 40.60084375699994], [-73.93904134699991, 40.60074143199989], [-73.93952543799986, 40.600688096999896], [-73.93996699299998, 40.60063944799986], [-73.94089263599994, 40.60053783599993], [-73.9411660979999, 40.600508015999885], [-73.94189057599992, 40.60042741899986], [-73.94288954099984, 40.600318649999934], [-73.94283742399996, 40.60004569399995], [-73.94278945999999, 40.599794549999956], [-73.94248473299992, 40.598200364999855], [-73.94341159099989, 40.598097814999896], [-73.943875965, 40.59804616299987], [-73.94433922599985, 40.597994625999945], [-73.94479667499994, 40.59794473299995], [-73.94526519199995, 40.5978936269999], [-73.94622941199994, 40.59778899199995], [-73.94719207000004, 40.59768208699993], [-73.948120131, 40.597579399999866], [-73.94855032699992, 40.59753296399993], [-73.94904681599999, 40.59747936199991], [-73.94997334599992, 40.59737744199985], [-73.9510058549999, 40.5972626189999], [-73.95220370799984, 40.59713035099986], [-73.95314947399989, 40.5970279189999], [-73.95411036199994, 40.596920738999856], [-73.95507631399984, 40.59681539099995], [-73.95545313399988, 40.59677338399993], [-73.95600127899989, 40.59671338599991], [-73.956927197, 40.596612075999865], [-73.95733246399982, 40.598730913999866], [-73.95775745999987, 40.600985888999936], [-73.95683001699994, 40.601087348999954], [-73.957283474, 40.60346964199992], [-73.95753057699993, 40.604823308999904], [-73.95772401299996, 40.60579983299993], [-73.95828056999996, 40.60874735399996], [-73.95859278499987, 40.61040303099988], [-73.95804731899992, 40.610462761999926], [-73.95766801399986, 40.61050371499993], [-73.95670471399993, 40.61061187799987], [-73.95574066999986, 40.6107148269999], [-73.95479552700002, 40.61082076599994], [-73.9535988889999, 40.61094986499993], [-73.95258786900004, 40.611112239999905], [-73.95256553099986, 40.61098718399993], [-73.95163234999984, 40.61112353899989], [-73.95070512899999, 40.611250451999915], [-73.94978380499997, 40.611371749999854], [-73.94993049299988, 40.6121446509999], [-73.94996296699989, 40.612315726999924], [-73.94999598099999, 40.612491460999884], [-73.95026514899993, 40.61392435199983], [-73.94930527399987, 40.614030734999886], [-73.94834517399988, 40.61413649599991], [-73.94772682600002, 40.61416971599985], [-73.94742608799993, 40.61422460099989], [-73.94657366799994, 40.61478913099987], [-73.94573691899997, 40.615363406999954], [-73.9448277099998, 40.61604620899986], [-73.94479547299986, 40.615888530999904], [-73.94457366500001, 40.61480377199996], [-73.94430888199999, 40.6134117379999], [-73.94417450600004, 40.612722537999936], [-73.94405560899996, 40.61199284199993]]]}}, {\"id\": \"149\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 150, \"Shape_Leng\": 0.089331491183, \"Shape_Area\": 0.000332568036088, \"zone\": \"Manhattan Beach\", \"LocationID\": 150, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9301907789999, 40.58508559299986], [-73.9300961629999, 40.584502075999886], [-73.93107641299986, 40.58437967599985], [-73.93106383699993, 40.584318467999886], [-73.93116473799996, 40.584312990999905], [-73.9320424499999, 40.5842339599999], [-73.932029438, 40.58414863699993], [-73.93186345464173, 40.58308286112594], [-73.9319425895196, 40.583109750227074], [-73.93197721282121, 40.583110543041684], [-73.9319281118675, 40.58286976893519], [-73.9323769574282, 40.5829185099781], [-73.93237196954628, 40.582961384699566], [-73.93201379961046, 40.5829259827501], [-73.93198873707402, 40.58294351160217], [-73.9320183941152, 40.58311112461856], [-73.93206437602129, 40.58307147129166], [-73.93209376133852, 40.58306764922298], [-73.93220291301584, 40.58304659240697], [-73.93225833516155, 40.58303062452617], [-73.93226590414368, 40.58301654813771], [-73.93228100685351, 40.58302423759583], [-73.93233559258715, 40.58300442924209], [-73.93234737144762, 40.58297755523919], [-73.93238094220585, 40.582985895017075], [-73.93254303390567, 40.582911747776755], [-73.93261354652961, 40.58291562909743], [-73.93266897618398, 40.582890700374826], [-73.93275209395242, 40.58287986868578], [-73.9327786668411, 40.58294102796514], [-73.93280322794682, 40.58295779393814], [-73.93283754945256, 40.58295861814763], [-73.93286703550758, 40.58294979451565], [-73.93289556463691, 40.582945099918334], [-73.93294538593976, 40.582941167101794], [-73.93297306018958, 40.58293898308529], [-73.93304245856554, 40.582948809280715], [-73.93310519300422, 40.58296769389286], [-73.93313721070093, 40.58298882945063], [-73.93329332900153, 40.58302794132647], [-73.93338721555227, 40.58303809112294], [-73.9334914563774, 40.58305871726035], [-73.93348952833713, 40.58296458804381], [-73.93371378016603, 40.582967184177306], [-73.93420971486945, 40.58297979889342], [-73.93420772865394, 40.58309425873808], [-73.93437027873017, 40.58311020468171], [-73.93437970143313, 40.58313903367363], [-73.93446928196295, 40.58315251128185], [-73.93455992902854, 40.58316541413149], [-73.93456279121907, 40.583124906302395], [-73.93456057101987, 40.58302017383224], [-73.93528194372138, 40.58309697687026], [-73.93527605262638, 40.58321782276441], [-73.93527611180885, 40.58324291418614], [-73.9354404068802, 40.58325396438772], [-73.93569071639773, 40.58327509318913], [-73.93568953636128, 40.58325484190385], [-73.93572423542369, 40.583253981456245], [-73.9357193387555, 40.58315269979287], [-73.93567676468071, 40.583153530701125], [-73.93567353145987, 40.58301942441257], [-73.93582439275757, 40.58302111555204], [-73.93582690671904, 40.58315194690377], [-73.93575981919531, 40.58315597669307], [-73.93576657211078, 40.58327750984968], [-73.9358603647903, 40.58328708780447], [-73.93612717900335, 40.58331433315848], [-73.9362018028412, 40.58326923582003], [-73.93625253184901, 40.58324718862198], [-73.9362543204545, 40.58318392412819], [-73.93627418270637, 40.58318123473924], [-73.93627045070224, 40.58305920974583], [-73.9362893368138, 40.58306024734306], [-73.93628752264294, 40.58300938406042], [-73.93628723189404, 40.583000903558265], [-73.93624507152867, 40.58300005559123], [-73.93624185128186, 40.58297615321488], [-73.93612802211166, 40.582979364521975], [-73.9361286594258, 40.5830054701533], [-73.93612881237642, 40.58301119807032], [-73.93609824969617, 40.583011638934096], [-73.93609807701682, 40.58300471585093], [-73.93609668691353, 40.582948745832866], [-73.93624023910758, 40.582946671994854], [-73.93624037244723, 40.58295200687669], [-73.93635149114577, 40.582950401904505], [-73.93656607029142, 40.58295764878638], [-73.93656436764802, 40.58298694422441], [-73.93635050713729, 40.582979721861655], [-73.93634906196031, 40.583004602702836], [-73.9363003120676, 40.58300295567551], [-73.93630169169033, 40.58300973334176], [-73.93630359862769, 40.583019096950025], [-73.93630860178921, 40.58301953831564], [-73.93631716129542, 40.58317905120109], [-73.93653912847077, 40.583163620917986], [-73.9365485548836, 40.5832214552796], [-73.93657592440762, 40.5832405055899], [-73.93662876045104, 40.583242629029996], [-73.93697427463773, 40.58323032641129], [-73.93697724711018, 40.583066047826165], [-73.93698651739236, 40.58306617302943], [-73.93698743671955, 40.58302662803935], [-73.93698767830983, 40.58301627317647], [-73.936931007221, 40.58301550478472], [-73.93693148518238, 40.58299498821779], [-73.93690397408103, 40.582994615772506], [-73.9368916774398, 40.582994450368595], [-73.93682584926127, 40.58299355778148], [-73.93674491407205, 40.58299246200374], [-73.93673190264388, 40.58299228630497], [-73.93666300283665, 40.5829913523961], [-73.93664922102909, 40.5829911658764], [-73.93663263829826, 40.582990942126486], [-73.93663307422491, 40.58297221190407], [-73.93705921677467, 40.58297798232017], [-73.93705623806854, 40.58302832312092], [-73.93705290451025, 40.583084699478114], [-73.9376326974329, 40.58309626828403], [-73.9376320432887, 40.583115319712995], [-73.9370535633145, 40.58310377574734], [-73.9370527874272, 40.583126392468074], [-73.93702675588715, 40.5831258725993], [-73.93703012691904, 40.58302768024278], [-73.93703051033472, 40.58301651679285], [-73.93700558408256, 40.5830160194714], [-73.93700520516775, 40.58302706598814], [-73.93700387324068, 40.58306585703719], [-73.93701423047526, 40.58306606358339], [-73.93701481021185, 40.583233272363955], [-73.93705106890114, 40.58323547162591], [-73.9370773777828, 40.5833297556945], [-73.93712459798441, 40.58334673016416], [-73.93729744072706, 40.583348841508204], [-73.93732049535753, 40.58334048130114], [-73.93732470759878, 40.58329281845608], [-73.93783180297176, 40.58329309395866], [-73.93788732683, 40.583343967667346], [-73.93811917695099, 40.58334047899514], [-73.9380942062777, 40.58311681071695], [-73.93823991479728, 40.58311688913321], [-73.93825745769908, 40.58327956714171], [-73.9383308913977, 40.583341198278134], [-73.93855276236503, 40.5833379849913], [-73.93847712885862, 40.5831778972711], [-73.9386684665775, 40.58316442817503], [-73.93866646107386, 40.58315914171804], [-73.93865999147236, 40.583142084269014], [-73.93866334157603, 40.58314134490724], [-73.93864663032168, 40.58310636722804], [-73.93862218897615, 40.583103004925135], [-73.93862755765123, 40.58306702179752], [-73.93878660441516, 40.583070937875405], [-73.9387826663899, 40.58309483804125], [-73.93873115333592, 40.583089453828734], [-73.93866851730543, 40.583084065331484], [-73.93866322813588, 40.58310503987518], [-73.93867832712033, 40.58313795155601], [-73.9386871999984, 40.58313942150383], [-73.93869745798771, 40.583164939366874], [-73.9392964038475, 40.58316726873495], [-73.93930153084881, 40.583333215054466], [-73.93982538850575, 40.58333144625053], [-73.93982883984603, 40.583170535692226], [-73.93985700196716, 40.58317497237716], [-73.9398585157681, 40.583331334095426], [-73.94027355766525, 40.58332993058913], [-73.94057745828908, 40.58332822557354], [-73.94057532545163, 40.58328213645074], [-73.94057495376235, 40.58327411762926], [-73.94057258161916, 40.58322286210951], [-73.94057214999937, 40.58321354507143], [-73.94056962801625, 40.58315906218662], [-73.94056916880331, 40.583149147053916], [-73.94056754325045, 40.58311477034367], [-73.9405660894776, 40.58308736830807], [-73.94054199309038, 40.583088130329386], [-73.94054229555766, 40.58309368157402], [-73.94047360621605, 40.58309385072781], [-73.9404735774075, 40.58308722039998], [-73.94010952396489, 40.5830929753204], [-73.94010903682874, 40.58307506968685], [-73.94012576769904, 40.58307479679], [-73.94021422135123, 40.58307335351443], [-73.94022518035786, 40.583073173091336], [-73.94026708883288, 40.58307249059309], [-73.94027941504933, 40.5830722889292], [-73.94036352490993, 40.58307091632378], [-73.94037688389578, 40.5830706975949], [-73.9404728145691, 40.58306913235311], [-73.94048659523155, 40.58306890679366], [-73.94056166659253, 40.583067681310204], [-73.94057550702213, 40.583067456273604], [-73.94065816747744, 40.5830661068271], [-73.94067657802263, 40.583065805567315], [-73.94067712191325, 40.5830857907708], [-73.94058568405266, 40.583085976891226], [-73.9405866718767, 40.583115240775406], [-73.94058727912824, 40.58314227246784], [-73.94058754647727, 40.58315420589594], [-73.9405884696085, 40.58319543000865], [-73.94058872809934, 40.58320694745642], [-73.94062855574232, 40.583205295765254], [-73.94063649330319, 40.58332834119565], [-73.9407001951476, 40.58332762473212], [-73.94069690323742, 40.58330243682431], [-73.94091645490734, 40.5833024065313], [-73.94090001047674, 40.58312295058611], [-73.94089710477411, 40.58308760707976], [-73.94073868464447, 40.583091242955916], [-73.94073840604624, 40.58307561281749], [-73.9407935538431, 40.58307460323665], [-73.94083081733403, 40.5830739208196], [-73.94084677123455, 40.583073629324375], [-73.94091637255394, 40.58307235421962], [-73.94097532797558, 40.58307127447183], [-73.94100408977756, 40.583070747956384], [-73.94102517918674, 40.58307036190615], [-73.94102584392368, 40.58308549769504], [-73.9409145036293, 40.58308833841558], [-73.94091736917896, 40.583123377834916], [-73.94092816033081, 40.58325534247295], [-73.94093422085363, 40.5833023064367], [-73.94115699675478, 40.58329922746803], [-73.94116258456305, 40.58327511614185], [-73.94113701228311, 40.58316528347316], [-73.94113089391008, 40.58316311083215], [-73.94110905972568, 40.5831411412236], [-73.94110624879366, 40.58312802407], [-73.94109844565489, 40.58309477669979], [-73.94114534994199, 40.583100179939045], [-73.94115074719811, 40.583103532454885], [-73.94120155728876, 40.58309046520493], [-73.94120561662946, 40.58309900307507], [-73.94115408610509, 40.583113222225236], [-73.94115833809383, 40.583129304549054], [-73.94119384397212, 40.58326360343945], [-73.9412092982398, 40.58329850488304], [-73.94124697402302, 40.58330137935952], [-73.94126351164643, 40.58333675629567], [-73.94131115116105, 40.58333662684795], [-73.94144050857183, 40.583340147476136], [-73.94148833784251, 40.58352873644648], [-73.94154438657303, 40.58351808329938], [-73.9415740808356, 40.583607555939324], [-73.94255133565284, 40.583514476170805], [-73.94202033352745, 40.583110423782486], [-73.9421493881021, 40.58310667955098], [-73.94270167762977, 40.58350879761404], [-73.94304709645994, 40.58349212946424], [-73.94256316190977, 40.58312734751645], [-73.94268774005667, 40.583122816572036], [-73.9431950615802, 40.583489662633205], [-73.94354467530928, 40.58347101436107], [-73.94309334781462, 40.583134545214385], [-73.9432281554656, 40.583130820211096], [-73.94351205889528, 40.583337045980464], [-73.94369153004567, 40.583467754265975], [-73.94406764167111, 40.58347200911032], [-73.94363589795726, 40.58314557390953], [-73.9437651164146, 40.583144718831555], [-73.9442340358713, 40.58347995164651], [-73.9446182866623, 40.5834966260877], [-73.94417458523772, 40.58315829121823], [-73.94432059818067, 40.58315942487763], [-73.94477544081295, 40.58350780440983], [-73.94517477781615, 40.58352137824417], [-73.94471270964229, 40.58316504453028], [-73.94484512940838, 40.58316423534613], [-73.94534992503475, 40.58352905211037], [-73.94573014930472, 40.58354487455877], [-73.94526670124631, 40.583182518567256], [-73.94537605237886, 40.5831819316505], [-73.945899118881, 40.58355266261021], [-73.94628634691863, 40.583568685707924], [-73.9457994125887, 40.583194640230225], [-73.945924129353, 40.58319410443476], [-73.94643429059336, 40.58356315259245], [-73.94678945068769, 40.583550467614124], [-73.94631603648743, 40.58320694021067], [-73.94645496477298, 40.583205168080084], [-73.946917962761, 40.58354603324974], [-73.94726832705229, 40.5835331825745], [-73.9468220294719, 40.58319119187112], [-73.94701390883573, 40.58318810097944], [-73.94746867861319, 40.58352819536238], [-73.94836747924887, 40.583488596760866], [-73.94958266755164, 40.58336143778913], [-73.95025076956786, 40.58328726469968], [-73.9511822698627, 40.583199265180156], [-73.95247447970824, 40.58307717692723], [-73.95349322048675, 40.58298819826098], [-73.95343216344405, 40.58267434332094], [-73.95341929823668, 40.5826082108862], [-73.95330135995673, 40.58200274963343], [-73.95286292151859, 40.58204114558685], [-73.95233099098505, 40.58208772664359], [-73.95193849433237, 40.582122095971016], [-73.95102369462639, 40.58220219549055], [-73.95008839464253, 40.58228480784728], [-73.9496967249941, 40.58231940106665], [-73.94937473522309, 40.582257483993565], [-73.94800448116914, 40.58199633761599], [-73.9469768187521, 40.581800471469556], [-73.94600616631155, 40.581615461082094], [-73.94502940359521, 40.581429276408024], [-73.94413672913842, 40.58125911318595], [-73.94306208267815, 40.58105626783241], [-73.94274062592493, 40.58099558955925], [-73.94216538347959, 40.581004699665165], [-73.94124120272384, 40.58101932764534], [-73.94035765693273, 40.581033307886806], [-73.93899065793353, 40.581054145630404], [-73.93805680414178, 40.5810683717811], [-73.93710561356725, 40.58108285258504], [-73.93616960757984, 40.58109709572432], [-73.93475193843821, 40.581118654156846], [-73.93381280663905, 40.581132925742], [-73.93381364630925, 40.58121398905245], [-73.93383615339322, 40.58121373631435], [-73.93383598074706, 40.58120413761315], [-73.93429954734847, 40.581199276901735], [-73.9351340770489, 40.58118378955771], [-73.93513789283365, 40.58130327675363], [-73.93510879115179, 40.58130381702098], [-73.93510615823921, 40.581221396222546], [-73.93509532596819, 40.581210618963134], [-73.93391404481234, 40.58122934798396], [-73.933836451114, 40.581230161411334], [-73.93383627277876, 40.58122031677366], [-73.93381304804846, 40.58122057898259], [-73.93381328089814, 40.581232615607576], [-73.93376391735075, 40.58123317233996], [-73.9337631473704, 40.58119356440797], [-73.93366295933798, 40.58119469455551], [-73.93366325722222, 40.58121290317207], [-73.93340042885238, 40.581215388869225], [-73.93339931868996, 40.58116480152509], [-73.93366243426809, 40.58116231381855], [-73.93366279643807, 40.581184599637325], [-73.93376313431382, 40.58118346622566], [-73.93375801356129, 40.581132041605834], [-73.93300348916418, 40.581144675417164], [-73.93300294726038, 40.58128140965124], [-73.93297161890672, 40.581283630074765], [-73.9329635037575, 40.58114517026216], [-73.93290958744812, 40.58114576736104], [-73.93291340814964, 40.58128373933714], [-73.93287295022381, 40.58128600047321], [-73.93286916364703, 40.58114848252055], [-73.93280636427473, 40.58114741868648], [-73.93277885607172, 40.58119843073621], [-73.93274944870564, 40.58123722558508], [-73.93269753266527, 40.58126928847301], [-73.93264074804816, 40.58127671947118], [-73.93258006568031, 40.58126474120341], [-73.93255268134347, 40.581240094945706], [-73.93252433664163, 40.58119604284196], [-73.93250341155822, 40.58114698863907], [-73.93245060616512, 40.58097120459655], [-73.93202124664519, 40.5795418689226], [-73.93199576875995, 40.5795210103253], [-73.93181497833324, 40.57895869723001], [-73.93182688427206, 40.57868047109542], [-73.93169026330237, 40.578178861649455], [-73.93164069394192, 40.578170809176015], [-73.93106028107997, 40.57626592009687], [-73.93106288335652, 40.57616310072335], [-73.93144225619588, 40.57570097093256], [-73.93150256744042, 40.57566050512672], [-73.93307310428824, 40.57560872400213], [-73.93331645009378, 40.575587631640914], [-73.93488263802315, 40.57557812179094], [-73.93490632576673, 40.57557530530583], [-73.93491838261332, 40.57555837288704], [-73.93492129931023, 40.57553735812245], [-73.93492001314723, 40.57527553866215], [-73.9349203490142, 40.57527515789016], [-73.93492231057287, 40.57527334496963], [-73.93492463227012, 40.575271799068425], [-73.93492725244316, 40.57527056124474], [-73.93493010150159, 40.57526966437435], [-73.9349331037764, 40.57526913227764], [-73.93493617952893, 40.57526897908671], [-73.93493924706893, 40.57526920887029], [-73.93494222492475, 40.57526981552538], [-73.93494503400603, 40.57527078293979], [-73.93494759970542, 40.575272085419236], [-73.93496611332279, 40.57548259278711], [-73.93504799627777, 40.575554349306], [-73.93530038281477, 40.57563414818021], [-73.93574425819364, 40.57575056398321], [-73.93623167905103, 40.575850406099825], [-73.93647107586584, 40.57586049635632], [-73.93656163438752, 40.575819201443274], [-73.93660358189786, 40.575772406452614], [-73.93662109585456, 40.575722715958214], [-73.93665097426798, 40.57562271008414], [-73.9366715385712, 40.575428459497424], [-73.93668273285745, 40.57542357989078], [-73.93669484323917, 40.57542020382507], [-73.93670752538155, 40.57541842729196], [-73.93672041869382, 40.57541830080356], [-73.93673315658057, 40.57541982795625], [-73.93674537686583, 40.57542296532855], [-73.93675673209019, 40.575427623715704], [-73.9367567450178, 40.57542766108188], [-73.93679335497849, 40.575537794561626], [-73.93679549487118, 40.575542401672834], [-73.93679859204515, 40.575546680769264], [-73.9368025634706, 40.575550517136115], [-73.93680730268137, 40.57555380792715], [-73.93681268262718, 40.57555646492191], [-73.93681855908139, 40.57555841689142], [-73.93682477450699, 40.575559611506414], [-73.93683116227929, 40.575560016741726], [-73.93683755115332, 40.575559621733554], [-73.93684376985449, 40.575558437071436], [-73.93684965167063, 40.575556494513954], [-73.9389240781322, 40.57548947143868], [-73.94250907828433, 40.575373554531794], [-73.94251918828813, 40.57538213343872], [-73.94252738350511, 40.575391827587254], [-73.94253345598342, 40.57540239099203], [-73.94253725163497, 40.57541355561044], [-73.94253867414574, 40.57542503814426], [-73.94253768741825, 40.57543654722796], [-73.94253447869173, 40.57544739394649], [-73.94252912968336, 40.57545773813323], [-73.94252176654372, 40.57546733583021], [-73.94251256292429, 40.57547596068517], [-73.94250173588297, 40.5754834092889], [-73.94248954076535, 40.57548950597304], [-73.9424762651813, 40.575494106952924], [-73.94248281376751, 40.5755095117629], [-73.94249255068962, 40.575523903400516], [-73.9425052183877, 40.57553690117669], [-73.94252048177572, 40.57554816127268], [-73.94253793710583, 40.57555738583487], [-73.94255712264767, 40.575564330853865], [-73.94257753090295, 40.57556881261888], [-73.94259862202954, 40.57557071257774], [-73.94274060818059, 40.575598788010446], [-73.94288748744692, 40.57563246688344], [-73.94311590561615, 40.57567677484502], [-73.94329220627851, 40.5757202853315], [-73.94347256946372, 40.57574893063853], [-73.94364149533129, 40.5757728599584], [-73.9438205742342, 40.57578965857652], [-73.94393460727696, 40.575801020361936], [-73.94404937431712, 40.57580662854106], [-73.94416437658232, 40.575806458741084], [-73.94427911427726, 40.57580051170004], [-73.94439308875675, 40.57578881326339], [-73.94450580469348, 40.57577141427217], [-73.94454293700004, 40.57575673864666], [-73.94457698614625, 40.57573821916512], [-73.94460727243747, 40.57571622551796], [-73.94463319129586, 40.57569119674712], [-73.94455272237937, 40.57529349621064], [-73.94456623438639, 40.5752690239822], [-73.94456942217181, 40.57524755241894], [-73.94460730835148, 40.57523383460922], [-73.94462715976421, 40.575245466207605], [-73.94464810794126, 40.57526111233924], [-73.94467689102792, 40.57533610054029], [-73.94470763999647, 40.575526453213364], [-73.94486926137374, 40.5755736624806], [-73.94499527706101, 40.575555595685046], [-73.94518784085109, 40.575559313062094], [-73.9454520345657, 40.575572780194], [-73.94554290851462, 40.575574035723626], [-73.94563360923209, 40.575569578814594], [-73.94572352013991, 40.575559439764916], [-73.94581203002984, 40.5755436874989], [-73.9458985372183, 40.57552242909964], [-73.94598245363807, 40.57549580908012], [-73.94603603744247, 40.57546565710266], [-73.94608366278499, 40.57543016399477], [-73.94612442150431, 40.57539000657711], [-73.94615753638364, 40.57534595061335], [-73.94618237597034, 40.57529883620686], [-73.94619846661749, 40.57524956178109], [-73.94620550151313, 40.57519906694626], [-73.94615898850593, 40.575189710744425], [-73.94595330078268, 40.575212017721306], [-73.94592148276949, 40.575198933599374], [-73.94590897955277, 40.575139259547086], [-73.94592390235844, 40.57511543230938], [-73.94665101734034, 40.57501942635745], [-73.94671197360067, 40.575020117228135], [-73.94677953289836, 40.575017328520985], [-73.94680134069225, 40.575015440344295], [-73.9469198029282, 40.57500518403111], [-73.94708500319491, 40.574980872126766], [-73.94776716764005, 40.57490728869874], [-73.94868549287553, 40.57480822367913], [-73.94896375973461, 40.574778204211434], [-73.94938813625923, 40.574689679862985], [-73.94954065902311, 40.57464998733389], [-73.94991862626001, 40.57455162473903], [-73.9500657581515, 40.57456212504081], [-73.9501855243287, 40.57455956843377], [-73.95030872902427, 40.57453352906245], [-73.95053081571785, 40.57452059915292], [-73.9505311591345, 40.57452057902976], [-73.95081177068961, 40.57449199818993], [-73.95104789991738, 40.57446078911594], [-73.95132509565437, 40.57442437791532], [-73.95147653230823, 40.574424441979914], [-73.95151329579528, 40.574424457502445], [-73.95176309165656, 40.57442195456104], [-73.95207791647857, 40.5743986028653], [-73.95224221101468, 40.57433344096678], [-73.9523584672133, 40.574274685845204], [-73.95242628499996, 40.57455910699991], [-73.95287451099996, 40.57688215899993], [-73.95300123199996, 40.57747651199985], [-73.95303970899995, 40.577679586999885], [-73.95309963899994, 40.577994155999896], [-73.95334153899987, 40.57926712799989], [-73.9537406319999, 40.58135589199988], [-73.9538380169999, 40.581745551999916], [-73.95392000699991, 40.58186979399993], [-73.95403970599988, 40.58251639099995], [-73.95407172899988, 40.58270239799989], [-73.95411176199985, 40.58291991799995], [-73.95412132099985, 40.58298298499989], [-73.95414287899986, 40.58310613999989], [-73.95436572599993, 40.58308107799995], [-73.95517441499999, 40.582915870999955], [-73.95571160099988, 40.58280458499993], [-73.95604146799998, 40.58273625399989], [-73.95645729099982, 40.582658850999906], [-73.95675618299992, 40.58283496999994], [-73.95682583599991, 40.5828863089999], [-73.95689047299997, 40.582933978999925], [-73.95695595999992, 40.5829822629999], [-73.95703134199992, 40.58303533199994], [-73.9571329139998, 40.58304572599995], [-73.95723610100002, 40.5830422709999], [-73.95733608399979, 40.58302513399989], [-73.95900739999992, 40.58283812799991], [-73.96004798699995, 40.58326987199995], [-73.96010509500002, 40.58340835699995], [-73.96014924099997, 40.58357589699987], [-73.96021792199998, 40.584514661999904], [-73.96022753900004, 40.584608958999915], [-73.96023124399987, 40.584676004999906], [-73.96023555999999, 40.584746675999895], [-73.9602394339998, 40.584791873999954], [-73.96024315399984, 40.58482708099993], [-73.96025825899996, 40.58502930799994], [-73.96021742199999, 40.58503625499989], [-73.96007463299998, 40.58505031599989], [-73.95917287700001, 40.58513910599985], [-73.95880175500001, 40.58518715099991], [-73.95777933699982, 40.58518352899989], [-73.95765292099998, 40.58519361399986], [-73.95761659699984, 40.58519750799994], [-73.95757916399988, 40.5852008269999], [-73.95690846199993, 40.58527652799987], [-73.95673945299994, 40.5853033079999], [-73.95657432099985, 40.585344345999864], [-73.95641613599993, 40.585399010999915], [-73.95626771099987, 40.58546620499987], [-73.95576287300001, 40.585500097999905], [-73.95485543099988, 40.585624776999914], [-73.95439984999999, 40.58567624799991], [-73.95398040799994, 40.5857118479999], [-73.95387860700005, 40.58572124599991], [-73.95388521699984, 40.58564255099988], [-73.9538683119999, 40.58555947099997], [-73.95382536099993, 40.58547826199986], [-73.95375778800003, 40.585405959999875], [-73.953671153, 40.585348497999895], [-73.95357366099992, 40.58530918699989], [-73.95347394899987, 40.58528824599992], [-73.9533602969999, 40.58526818099989], [-73.95324286999993, 40.58525898299989], [-73.95312439499999, 40.585261132999904], [-73.95300767399999, 40.585274583999926], [-73.95289540899982, 40.58529875299989], [-73.95287969899992, 40.58519841999987], [-73.95123865700005, 40.584924632999915], [-73.94986974599985, 40.58485639599989], [-73.94866230599997, 40.58489385599993], [-73.94763574499989, 40.584998861999956], [-73.9466911319999, 40.585073071999915], [-73.94555221999997, 40.585139491999946], [-73.9449308669999, 40.58517250599993], [-73.94386629099992, 40.5852983619999], [-73.94291014799995, 40.58540563799992], [-73.94191610399989, 40.585520172999914], [-73.94105998299987, 40.585661083999916], [-73.93920635200004, 40.58597168899993], [-73.93914504099992, 40.58597982499988], [-73.93908601799997, 40.58599196899989], [-73.93816896699991, 40.58614987199996], [-73.93724717299983, 40.58631062999991], [-73.93677580200001, 40.58639859399994], [-73.93633646899995, 40.58648058799982], [-73.935885376, 40.586532089999935], [-73.93541271099996, 40.58658605499993], [-73.93477406299994, 40.586639969999965], [-73.9344471859999, 40.586652074999925], [-73.93344399399984, 40.58653743799994], [-73.93244497799995, 40.586360289999874], [-73.93210850499989, 40.58628620399988], [-73.93145727300002, 40.58610127199996], [-73.93139947599984, 40.58607989499989], [-73.93134323699987, 40.58580563399986], [-73.9313279159999, 40.585737850999884], [-73.93131673999989, 40.58567129699987], [-73.93126944999997, 40.58543639299992], [-73.93119181099983, 40.58541372599994], [-73.9301907789999, 40.58508559299986]]]}}, {\"id\": \"150\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 151, \"Shape_Leng\": 0.0548899410999, \"Shape_Area\": 0.000128848901918, \"zone\": \"Manhattan Valley\", \"LocationID\": 151, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96371096499993, 40.7930129339999], [-73.96417598599989, 40.79236204499989], [-73.96525997299999, 40.79280137099989], [-73.96701397, 40.7935599909999], [-73.96987063299996, 40.794735204999895], [-73.96941543899999, 40.79536077899997], [-73.97073269599993, 40.79591916999989], [-73.97082734199992, 40.79595859999988], [-73.97093073199993, 40.796001047999866], [-73.97225661799988, 40.79656086499989], [-73.97365145399993, 40.79714831899984], [-73.97413148099996, 40.79734378899995], [-73.97507069999986, 40.797713349999874], [-73.97521014699983, 40.79777354799995], [-73.97539813999988, 40.79784774399989], [-73.97549419399992, 40.79788855199996], [-73.97563350899993, 40.79794620599991], [-73.97582955799997, 40.79802199699993], [-73.97635598099994, 40.79825402099994], [-73.97639951915279, 40.79827321034625], [-73.9762852788426, 40.79843523541014], [-73.97583055785226, 40.7990922809132], [-73.97578169321176, 40.79915625678024], [-73.97579140130186, 40.799209627886306], [-73.97576219486481, 40.79926017354925], [-73.97554385822002, 40.79952825063733], [-73.97526783234454, 40.79993284953168], [-73.97508668067901, 40.80018953363309], [-73.97496436808174, 40.800369634193885], [-73.97483924435981, 40.80055824326265], [-73.97466556722719, 40.80081351473424], [-73.9744872252097, 40.80105742889691], [-73.97414361823463, 40.80151689534118], [-73.97394098366696, 40.801809025864024], [-73.97389989052458, 40.801889863531166], [-73.9737747724601, 40.80204584594855], [-73.97372060455768, 40.802167815280235], [-73.97361322463878, 40.80229686038974], [-73.9735422772158, 40.80235641125034], [-73.97336671067806, 40.802630113346446], [-73.97320518045738, 40.80283005827629], [-73.97312859120979, 40.80297471550858], [-73.97307070537937, 40.80305554844743], [-73.97303522902078, 40.803073973741945], [-73.97292317001974, 40.80324982284389], [-73.97286807262148, 40.803320734176026], [-73.97287179090726, 40.8033561875739], [-73.97279907800002, 40.803329158999894], [-73.9726574479999, 40.80327651399991], [-73.97257779799997, 40.80324718399991], [-73.97250022199995, 40.80321661299996], [-73.97150381000002, 40.80283773599995], [-73.97032589799987, 40.80238456099995], [-73.97027400499987, 40.80235903599987], [-73.9702159479999, 40.802335850999924], [-73.97004742199996, 40.80226500999989], [-73.96858360799983, 40.801635469999916], [-73.96798415999996, 40.80139826599985], [-73.96787379699998, 40.80135169799993], [-73.96775900399992, 40.80130351599994], [-73.96571144299992, 40.80043806999986], [-73.96286980099991, 40.79923967699995], [-73.96147779999993, 40.79865415599994], [-73.96004456499999, 40.79804123499991], [-73.96052271699993, 40.79736846899988], [-73.96097971799988, 40.796738643999916], [-73.96144060699996, 40.796110826999964], [-73.96189985499994, 40.7954792699999], [-73.96235980199994, 40.79485206099987], [-73.96280590599986, 40.79423581299993], [-73.96371096499993, 40.7930129339999]]]}}, {\"id\": \"151\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 152, \"Shape_Leng\": 0.0579094768782, \"Shape_Area\": 0.000146637644064, \"zone\": \"Manhattanville\", \"LocationID\": 152, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95357262899982, 40.82197589199988], [-73.9529102909999, 40.82169694699989], [-73.95082880000002, 40.820827936999876], [-73.95037710599996, 40.82144561799986], [-73.94992194399984, 40.8220723139999], [-73.94854029799983, 40.821487622999925], [-73.94808289799983, 40.82210802199994], [-73.94723844699998, 40.8217539859999], [-73.94607828700005, 40.821263215999906], [-73.94640289799987, 40.82057204199995], [-73.94671831999985, 40.81989426799994], [-73.94703623199985, 40.81920837199988], [-73.94734953999985, 40.81853499799996], [-73.94766641299987, 40.817855508999905], [-73.94800732899995, 40.81711650899995], [-73.94835601499989, 40.81637430599994], [-73.94867681799998, 40.8156922859999], [-73.94883491299987, 40.815363980999955], [-73.94900927799986, 40.81503994299992], [-73.94919960099985, 40.81472085399986], [-73.949405505, 40.81440740899996], [-73.94962639600001, 40.81410043599992], [-73.94962653799999, 40.81410023899991], [-73.94962668099991, 40.81410004999989], [-73.94986218999995, 40.81379997999992], [-73.95035019299995, 40.8131593889999], [-73.95092724899993, 40.8125754289999], [-73.95098203399995, 40.812521629999885], [-73.95150537299989, 40.812007684999905], [-73.95210125199992, 40.811442853999935], [-73.95354382999984, 40.81205082099992], [-73.95408058399987, 40.81228011299989], [-73.95520337899985, 40.8135541289999], [-73.95570690100003, 40.81414395999996], [-73.95626957999987, 40.813375022999864], [-73.95707738499995, 40.8142980939999], [-73.95710690099989, 40.81433182099988], [-73.95716929999979, 40.81440312399995], [-73.95782821899992, 40.815156048999874], [-73.95819953200001, 40.81558041799985], [-73.95829447399994, 40.81569078999992], [-73.95839496699999, 40.81579688699991], [-73.9594932889999, 40.8170074739999], [-73.96068717899983, 40.81751051699993], [-73.9610201569998, 40.81764868599986], [-73.96138875999992, 40.81781807599991], [-73.96157544099992, 40.8179099709999], [-73.9616361359999, 40.81793985099995], [-73.96168805999997, 40.81796669399991], [-73.96181860199984, 40.818023153999874], [-73.9618998499999, 40.818047286999885], [-73.96203105797969, 40.81808633074085], [-73.9613183318058, 40.818894523557375], [-73.9609488984397, 40.819311288872676], [-73.96044728492026, 40.81987715679097], [-73.95999892229341, 40.82038294480238], [-73.95981462674504, 40.8202978752039], [-73.95939068797922, 40.82085517835582], [-73.95932566265215, 40.82094627625335], [-73.95928300647289, 40.820994974413715], [-73.95913061580634, 40.82116895377475], [-73.95920767709258, 40.82120790934943], [-73.95926255403748, 40.82123565535862], [-73.95930263598672, 40.821189897523], [-73.95942624349648, 40.82125238173802], [-73.95914685715196, 40.82157132855098], [-73.95903294155774, 40.82151374622844], [-73.95923467194001, 40.82127782527041], [-73.95917712794255, 40.82124942998514], [-73.9590896024009, 40.82120624705593], [-73.95881892233997, 40.82151852484672], [-73.95894172941705, 40.82156936400956], [-73.95874523660488, 40.82180551197357], [-73.95896360094618, 40.822373794142074], [-73.95900018289882, 40.822409728374346], [-73.95911304024106, 40.822454964645544], [-73.95922976213465, 40.82228691013226], [-73.95990999444398, 40.82257111982199], [-73.9596033246027, 40.82299468678336], [-73.95892614718308, 40.822711755441375], [-73.95904953331714, 40.822541335368236], [-73.95891721760434, 40.82248605137143], [-73.95886745433303, 40.822450239228], [-73.95883849863245, 40.82240504207782], [-73.958651260119, 40.821918436573355], [-73.95864585147663, 40.82190436963179], [-73.95852731524242, 40.82184375665217], [-73.9582001935402, 40.82228313691813], [-73.95816805363577, 40.822458100991234], [-73.95813063305538, 40.82253109146093], [-73.95807467943848, 40.82261135846493], [-73.95793904909117, 40.82277435354731], [-73.95822838183025, 40.82288786710083], [-73.95820639539099, 40.8229221319122], [-73.95814312769197, 40.82302072227142], [-73.95916458632307, 40.823451795483315], [-73.95918196037638, 40.82345912294556], [-73.95943846744429, 40.823562392292935], [-73.95946454189165, 40.823571135671656], [-73.95948269766942, 40.82357865266566], [-73.95949747410948, 40.82358681628588], [-73.95950361164493, 40.82359118579774], [-73.9595089796959, 40.82359578895142], [-73.95722203099996, 40.82322595899991], [-73.95715187499992, 40.82321376599985], [-73.95708264299999, 40.823201547999865], [-73.95651468499986, 40.82312012799985], [-73.95634010399982, 40.82309232799988], [-73.95576937299988, 40.822904320999896], [-73.95573074999992, 40.822891256999874], [-73.9556966079999, 40.82287647699992], [-73.953769274, 40.822058690999896], [-73.95367103499984, 40.82201753399993], [-73.95357262899982, 40.82197589199988]]]}}, {\"id\": \"152\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 153, \"Shape_Leng\": 0.024737239022, \"Shape_Area\": 3.24684964817e-05, \"zone\": \"Marble Hill\", \"LocationID\": 153, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90666530999995, 40.8757156089999], [-73.90692502800006, 40.87503467099994], [-73.90716476399984, 40.87437421899993], [-73.90746489800001, 40.87354735399992], [-73.90774270799982, 40.87284595199988], [-73.90846367899988, 40.87262079199994], [-73.90872046399987, 40.87222866599996], [-73.90893235220783, 40.87215734798524], [-73.908978513602, 40.8722243539186], [-73.90901544805163, 40.87228958702743], [-73.90905980364177, 40.87234132839487], [-73.90908937875491, 40.87237507530972], [-73.90909972105362, 40.87239419188019], [-73.90909969322314, 40.872414425326724], [-73.90913961647068, 40.872462794989914], [-73.90917513278698, 40.87248081419601], [-73.90919137619544, 40.87251342547653], [-73.90921055884137, 40.87256514682172], [-73.90924604694906, 40.87260452542323], [-73.90926526019165, 40.87263376123266], [-73.90927114016718, 40.87266299598997], [-73.90935540980549, 40.87277097703951], [-73.90943966199148, 40.87288233476205], [-73.90951136313969, 40.87297738355044], [-73.90954388318947, 40.87301787612437], [-73.90957936387078, 40.87306286600732], [-73.90958973101802, 40.87306400028053], [-73.90960006660967, 40.87308761824521], [-73.90963706316084, 40.873107881189355], [-73.90965332823252, 40.8731247512177], [-73.90967403485497, 40.87314725286421], [-73.9097095289807, 40.87318212947383], [-73.90971690053208, 40.87320461120855], [-73.90974797469848, 40.873218124915354], [-73.90978644251612, 40.87324738489861], [-73.90984559578646, 40.87331376126337], [-73.90990769944696, 40.873383498929115], [-73.90996168304555, 40.873431320340934], [-73.91001640976121, 40.87348194410383], [-73.91007556909462, 40.8735359559803], [-73.91009482143302, 40.87353708775016], [-73.91010961470225, 40.87355396586296], [-73.91010367477757, 40.873560705951654], [-73.91016876214526, 40.87361696400791], [-73.91017913401178, 40.87361472095862], [-73.91018355408711, 40.87362708835755], [-73.9101894638254, 40.87364283404717], [-73.9102737818055, 40.87370809418793], [-73.9103551397179, 40.87377785515433], [-73.91044539476847, 40.873838628058834], [-73.9105060659268, 40.873880268126626], [-73.91065439307907, 40.8739759430305], [-73.91080913168176, 40.874085022055525], [-73.91085171090067, 40.87410973724408], [-73.910921180048, 40.874146396019974], [-73.91098953820747, 40.87417963995649], [-73.9110690967962, 40.87422141212584], [-73.91111279481103, 40.87424272500016], [-73.91115761996681, 40.87426573964849], [-73.91119459821621, 40.87428194078671], [-73.91125288020052, 40.87431007188996], [-73.9112629469905, 40.87432370407537], [-73.91127192479053, 40.87431604794519], [-73.91127864317853, 40.874331369249326], [-73.91131058924775, 40.87434481582342], [-73.91148428297787, 40.87441792104997], [-73.91151006661373, 40.87442304598177], [-73.91155043066773, 40.87442818308494], [-73.91157173258563, 40.874431612885914], [-73.91159751286466, 40.87443928590142], [-73.91159974693575, 40.87444524899033], [-73.91162329185646, 40.87444781510173], [-73.91162776520517, 40.87445633733321], [-73.91163897503574, 40.87445719203627], [-73.91165017603142, 40.874473374319955], [-73.9116681095003, 40.87447594488272], [-73.91167146278335, 40.87448786152828], [-73.91168826975449, 40.87449043115864], [-73.91169275467155, 40.874489579272705], [-73.91170284178415, 40.87449639536275], [-73.91173423700737, 40.874501524916205], [-73.91173983012342, 40.8745057881888], [-73.91177009202042, 40.874516022935126], [-73.91179139183596, 40.87452114412123], [-73.91180933995194, 40.87452201416823], [-73.91183904256401, 40.87453267102166], [-73.91185921079003, 40.874541205994404], [-73.91188274672243, 40.874550578969], [-73.91189619479312, 40.874553993885804], [-73.91191525744192, 40.87455571920047], [-73.91192086201634, 40.874560828884455], [-73.91192983324362, 40.874558278608944], [-73.91193318113005, 40.874565944216506], [-73.9119444037699, 40.8745659524983], [-73.91194663053723, 40.874577021125766], [-73.91195447088614, 40.874578728965254], [-73.91195784463281, 40.87457447245355], [-73.91196570064146, 40.87457277735309], [-73.91196907175222, 40.87456256846286], [-73.91203072161403, 40.8745830378179], [-73.91208452800096, 40.87459584719716], [-73.91215290813037, 40.87461377507739], [-73.9121764462137, 40.874621456116486], [-73.91221454479957, 40.87463509998273], [-73.91221566296912, 40.874641908887504], [-73.91222814396944, 40.87465056297356], [-73.91222813320368, 40.87465854110613], [-73.91223791359347, 40.87465455896359], [-73.91225294388349, 40.87465513804567], [-73.91227699162117, 40.8746540217871], [-73.91227397278395, 40.87466770678098], [-73.91228449380333, 40.87466829175391], [-73.91229500780878, 40.874675710853595], [-73.91230627871727, 40.87467457487825], [-73.91231155258637, 40.87466944685153], [-73.91232355802433, 40.87467858616002], [-73.91233333490989, 40.87467688342118], [-73.91234234304949, 40.874683733076836], [-73.91233707714717, 40.87469171690411], [-73.91234308865762, 40.8746940081139], [-73.91234910104856, 40.87468659318885], [-73.9123558652191, 40.8746940179661], [-73.9123498419076, 40.87470028136261], [-73.9123588548898, 40.87470257521208], [-73.9123648748739, 40.87469858203207], [-73.91236862605753, 40.87470600416496], [-73.91237764718186, 40.8747020128729], [-73.91238741516419, 40.87470716280456], [-73.91239492490995, 40.87470716815496], [-73.91240469493584, 40.87471174086121], [-73.91242271779869, 40.87471859877234], [-73.91243099100983, 40.874715183168675], [-73.91244000191651, 40.87471861183008], [-73.91245203282763, 40.87471804391961], [-73.91245729299712, 40.874723757170756], [-73.91247757264037, 40.87472833869735], [-73.91248809220902, 40.8747306241713], [-73.91249033807661, 40.8747369028109], [-73.91250236313708, 40.87474033369901], [-73.91250687871081, 40.87473406062238], [-73.91251965496579, 40.87473521388557], [-73.9125256581892, 40.87474377377167], [-73.912529410846, 40.87474948519839], [-73.912551200576, 40.874751780277656], [-73.91255570224403, 40.87475692486938], [-73.91259176808397, 40.87476436313473], [-73.91261880851071, 40.874779790908924], [-73.91264284973255, 40.87478379887097], [-73.91267517023708, 40.87478895588346], [-73.9126962150604, 40.874789548318994], [-73.91270634490408, 40.87480010903186], [-73.91272287889383, 40.874802400107136], [-73.91272859671744, 40.87480240476915], [-73.91274316499636, 40.87480241502598], [-73.91275367183313, 40.874814976325524], [-73.9127687078019, 40.87481099814735], [-73.91278749082916, 40.874816712059406], [-73.91280025722797, 40.87482528618445], [-73.91280626515456, 40.874829846761436], [-73.91282129694946, 40.87482985793962], [-73.91284834382581, 40.87483957727122], [-73.91285660205791, 40.87484757013019], [-73.91286937917496, 40.874847012921634], [-73.91287914404397, 40.874855007183385], [-73.91288441177969, 40.87484588008938], [-73.9128949292361, 40.87484988736233], [-73.91290469377688, 40.874857881036384], [-73.9129212312108, 40.874857893776465], [-73.91293626418764, 40.87485619390873], [-73.91294378259612, 40.87484934731965], [-73.91350066356716, 40.87501790507695], [-73.91348952397077, 40.87503607785827], [-73.91349978845808, 40.875034131379806], [-73.9135091950303, 40.87503478681044], [-73.91350901692014, 40.87504437616451], [-73.91353440086448, 40.87504719658453], [-73.91354822577777, 40.87505316849612], [-73.91356898724574, 40.875052129242526], [-73.91358236087551, 40.875058794422635], [-73.91358789193708, 40.87506580397602], [-73.9136017368192, 40.87506476096734], [-73.91362249791955, 40.87506547903326], [-73.9136317314757, 40.875061983121626], [-73.91364510961307, 40.87506479319905], [-73.91365295999721, 40.87505814421496], [-73.91377097505355, 40.87509610769932], [-73.91386393524303, 40.87512409223816], [-73.91387403109368, 40.87512410000559], [-73.91388267200851, 40.875129572112556], [-73.91407508569827, 40.875182799046975], [-73.91406883175179, 40.87520834074647], [-73.91535186479148, 40.87559473482516], [-73.91538349409625, 40.87559320888148], [-73.91540287121052, 40.87559941029489], [-73.91542631477185, 40.875607945317405], [-73.91545588514181, 40.875621131955185], [-73.91546939921095, 40.87563214618999], [-73.91548768177182, 40.875636787460444], [-73.91551075196118, 40.87564341479566], [-73.91552990793934, 40.875645742470525], [-73.91555253991932, 40.87565402177963], [-73.91557648264494, 40.875658336101445], [-73.91558562076571, 40.875662969433066], [-73.9156178241692, 40.875677866462915], [-73.91567267909491, 40.875685508350976], [-73.91570881079427, 40.87569313613778], [-73.91572098193815, 40.87570967097484], [-73.9157431844291, 40.87571332287017], [-73.91578609320084, 40.87571718248032], [-73.91574356, 40.875760999999954], [-73.91571027699979, 40.875795454999924], [-73.91487449199987, 40.87664214599991], [-73.91205829999997, 40.878119319999904], [-73.91148913199984, 40.87901885899989], [-73.91033193599998, 40.879038046999895], [-73.90949832099987, 40.87877673199989], [-73.90906235999998, 40.878486782999886], [-73.90835569799991, 40.87791831399994], [-73.90765281699993, 40.87742802299989], [-73.90703748899998, 40.87689516099993], [-73.9068187309999, 40.87661958799993], [-73.90666530999995, 40.8757156089999]]]}}, {\"id\": \"153\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 154, \"Shape_Leng\": 0.22733157076, \"Shape_Area\": 0.00178968692433, \"zone\": \"Marine Park/Floyd Bennett Field\", \"LocationID\": 154, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.93531145199985, 40.60886003999987], [-73.93423157599979, 40.607897714999915], [-73.93351310700004, 40.60834345599989], [-73.93201257899985, 40.60699707199991], [-73.93169716599986, 40.6067140589999], [-73.92986880199989, 40.60507134099995], [-73.9295009169999, 40.60530869199985], [-73.92915810699985, 40.60552985699993], [-73.92844217799984, 40.60600119699996], [-73.92769136199986, 40.606480507999926], [-73.92696222699988, 40.60696137499984], [-73.92624885099994, 40.60741352199989], [-73.92444144399987, 40.60579356899993], [-73.92373054999989, 40.60625280699984], [-73.9230107439999, 40.60671540299993], [-73.92230145599989, 40.607178630999925], [-73.92158429099997, 40.607639084999875], [-73.92065861699989, 40.60675454299997], [-73.91984929899996, 40.60722139799987], [-73.91970928699986, 40.607302162999886], [-73.9185276079998, 40.60611336699987], [-73.91847519000007, 40.60614556799994], [-73.91794980550603, 40.60620925217101], [-73.91701787907759, 40.6053817785037], [-73.91712470643999, 40.6053104238462], [-73.91560086104275, 40.60393625315706], [-73.91549935833606, 40.60391435136834], [-73.91541600735383, 40.60395000593016], [-73.91476853163945, 40.60336022167265], [-73.91489358102517, 40.60328888376004], [-73.91442813462285, 40.60286391956325], [-73.91399608925019, 40.60278621349814], [-73.91396961253687, 40.602775505488346], [-73.91395185624506, 40.60276832278435], [-73.91389434799432, 40.60270357776866], [-73.91386005212335, 40.60275901226622], [-73.91384885636978, 40.60275647820452], [-73.9138010981717, 40.60287282172692], [-73.91382118712602, 40.60287785981948], [-73.91382020162834, 40.60288818005007], [-73.91380709007022, 40.60288445810617], [-73.91368718510047, 40.60285042101265], [-73.91369332211234, 40.60283939097751], [-73.91378662750739, 40.60286814468284], [-73.91383531832668, 40.602755727409026], [-73.91381966764283, 40.60275117194802], [-73.91382428947968, 40.60274325664473], [-73.91385524392098, 40.602690251506765], [-73.91382869588112, 40.60266843672914], [-73.91383692737334, 40.602664180288436], [-73.9138375968917, 40.60266383283232], [-73.91375408736835, 40.60259677510992], [-73.91373455897015, 40.60260791471353], [-73.91364075667397, 40.6025241817043], [-73.91332700399015, 40.60276377776985], [-73.91348903819966, 40.60289869030679], [-73.91347072649002, 40.60291169068994], [-73.91329651030489, 40.602765614677885], [-73.91363834620358, 40.60250187020004], [-73.91363102852749, 40.60249415232777], [-73.91366089338922, 40.602468910027454], [-73.91365030202735, 40.60245677496811], [-73.91357808200848, 40.60242539323002], [-73.91349826413605, 40.602366976067756], [-73.91338351755155, 40.602446833564045], [-73.91337620538674, 40.60244217995588], [-73.91333835460931, 40.60247561654664], [-73.91335175598681, 40.60248678208413], [-73.91327117511105, 40.602552721097645], [-73.91338693272549, 40.60263461155053], [-73.91319891628773, 40.60278506201215], [-73.91318430081162, 40.60276924744309], [-73.91335399077845, 40.602643882243676], [-73.91335643976032, 40.60263644847511], [-73.91326383824178, 40.60256758846607], [-73.91324920347246, 40.602566647465736], [-73.91301723241872, 40.60275424871759], [-73.91300017523358, 40.60274029168344], [-73.91329806292846, 40.602509050780895], [-73.91328222596215, 40.60249602416064], [-73.91331885909023, 40.60246072776627], [-73.91332373554695, 40.60246259135304], [-73.91336279315738, 40.60243938115535], [-73.91335670636211, 40.60243100982436], [-73.91347511544238, 40.6023492958734], [-73.91340321353769, 40.602310559901], [-73.91336876508088, 40.60229082781672], [-73.91331444593793, 40.60225844819831], [-73.91329854635799, 40.60225035124641], [-73.91324485254827, 40.60224677368356], [-73.91317591595909, 40.6022396477051], [-73.9131165873372, 40.6022202559288], [-73.91290000359777, 40.60248795355999], [-73.91290604871081, 40.602491169005475], [-73.91273866862339, 40.60270867701646], [-73.91270696033584, 40.6026947094838], [-73.91285319795077, 40.60250740557457], [-73.91272398622357, 40.6024532629461], [-73.91274890958807, 40.60242270427682], [-73.91288693485376, 40.6024847036405], [-73.91292596573744, 40.60243458637435], [-73.91292175545611, 40.60243208532434], [-73.91309691306819, 40.602214453925555], [-73.91305994409429, 40.602206210166656], [-73.9129903429, 40.60220009427759], [-73.91292868939256, 40.602200047584176], [-73.91281401274672, 40.60219086548635], [-73.91277093173233, 40.60218274884301], [-73.912674147097, 40.60217913779714], [-73.91256809408638, 40.60216541442624], [-73.91243685475213, 40.60214762929161], [-73.9122678192514, 40.60213688891289], [-73.91212130476976, 40.602139808917805], [-73.91195753841404, 40.60215484195359], [-73.91176263325637, 40.602155703683316], [-73.91158427689317, 40.60213548840341], [-73.9115783655672, 40.60221787119118], [-73.91181195390864, 40.60223276224483], [-73.91216040266461, 40.60225973592647], [-73.91230939977284, 40.60227508447791], [-73.9123215242302, 40.60228332812274], [-73.91254170182432, 40.602364439955366], [-73.91252801774947, 40.60239217890725], [-73.9125051581593, 40.60238604116291], [-73.91251410692396, 40.60237254361556], [-73.91230602075088, 40.602297891165485], [-73.91225514662504, 40.602289891851136], [-73.91225218426689, 40.602324447174134], [-73.91202422222575, 40.60230280855631], [-73.91182941479134, 40.60228942881859], [-73.91165189230664, 40.60227825784424], [-73.91164855920921, 40.60229392864238], [-73.9115770834805, 40.602288741416], [-73.91157412225583, 40.602321512034315], [-73.91155964713951, 40.60232040358004], [-73.91154411182994, 40.60243932310391], [-73.91162277849257, 40.602450445743415], [-73.91162568786596, 40.60245085641668], [-73.9120862627353, 40.60251597258292], [-73.91208793773214, 40.60251620941022], [-73.91219875740704, 40.60253187685288], [-73.91220966467009, 40.602533418412236], [-73.91223208880625, 40.60253658877339], [-73.91225690633148, 40.602540098026296], [-73.91227574643298, 40.60254276171483], [-73.91229559896387, 40.602545567735085], [-73.91231860788426, 40.602548821452615], [-73.91234251100597, 40.60255219981918], [-73.91236146275128, 40.602554879494036], [-73.91238406029152, 40.60255807408209], [-73.91240284062849, 40.602560729495345], [-73.91243560024542, 40.602565360171845], [-73.91245058778549, 40.60256043098706], [-73.91250311483626, 40.60251641751612], [-73.91261098676527, 40.602532006593805], [-73.91257290407538, 40.60268330003465], [-73.91255889942957, 40.602679693701695], [-73.91259129985662, 40.60254942169341], [-73.91251889258052, 40.60254030930361], [-73.91248882820365, 40.602674150771854], [-73.91247528777578, 40.602673398809095], [-73.9124900672872, 40.6026068487114], [-73.91247938988259, 40.60259292443454], [-73.91245933636156, 40.60258967031997], [-73.91240792926915, 40.60258262474647], [-73.9124011270441, 40.602581692958154], [-73.91229536833681, 40.60256719658126], [-73.91229292482903, 40.602566861500954], [-73.91225841311291, 40.60256213110643], [-73.91225241960679, 40.60256131083097], [-73.91212846663146, 40.602544321310965], [-73.91211791202088, 40.60254287500265], [-73.9120873814935, 40.60253868939025], [-73.91206013131695, 40.60253495542559], [-73.91203349801042, 40.60253130418354], [-73.91199845033863, 40.602526499755314], [-73.91197311405725, 40.60252302740842], [-73.91194080371508, 40.602518599486935], [-73.91192036060177, 40.602515797308534], [-73.91188459718971, 40.602510895130145], [-73.91186977947989, 40.602508863534], [-73.91182630076685, 40.602502904175346], [-73.91181118946098, 40.6025008334797], [-73.91178116749288, 40.60249671904106], [-73.91175977525427, 40.60249378593528], [-73.9117202557417, 40.60248836918768], [-73.91170103252993, 40.602485734263695], [-73.91166041994477, 40.60248016788978], [-73.911637820139, 40.60247707032464], [-73.91160007717856, 40.602471896981314], [-73.91156976933146, 40.602467742228065], [-73.9115436136135, 40.60246415724636], [-73.91151505923007, 40.60246024305987], [-73.91149232921046, 40.60245871812983], [-73.91146180823614, 40.6024566690258], [-73.91143632064774, 40.602454959018345], [-73.91140473154945, 40.60245283821128], [-73.91136460864192, 40.60245014590576], [-73.91133770304369, 40.60244833996134], [-73.91130574252344, 40.60244619521881], [-73.91128075538136, 40.60244451806375], [-73.9112492528857, 40.60244240331183], [-73.91122379607837, 40.602440694453655], [-73.9111897377734, 40.602438408674125], [-73.91117505476424, 40.602437423479735], [-73.91113463385533, 40.602434710592746], [-73.91111283637792, 40.60243324661912], [-73.91111012276473, 40.60243306442648], [-73.91104924152094, 40.602428978696274], [-73.9110172449766, 40.60242683049499], [-73.9109726966291, 40.602423840445034], [-73.91095957598459, 40.60243305225949], [-73.91095224971822, 40.602498921738224], [-73.91094007982434, 40.60249781432192], [-73.9109565554933, 40.602375711045866], [-73.91094634346705, 40.60236464218658], [-73.91089355171218, 40.60236306418761], [-73.91088639353035, 40.60249233806018], [-73.91086536585769, 40.60249048328842], [-73.91087683378183, 40.60234235483657], [-73.91098752750092, 40.60234836998357], [-73.91098165500402, 40.602403206448635], [-73.91103082003085, 40.60240682595815], [-73.9110514931486, 40.60240834888494], [-73.911509537572, 40.60244206876105], [-73.91151957194396, 40.6023968685693], [-73.91154234369162, 40.60222454078566], [-73.91151135057045, 40.602220762597476], [-73.91108453833075, 40.60216871624981], [-73.91106495246135, 40.60216632737419], [-73.91105289498671, 40.60216485735604], [-73.91105083185793, 40.60216460647585], [-73.91094453904712, 40.60215164384616], [-73.91092886882552, 40.60220894310243], [-73.91091626763057, 40.60220676550788], [-73.91093999639928, 40.60212847422249], [-73.91096008663212, 40.60213172980368], [-73.91095863542682, 40.60213922142222], [-73.91112022777182, 40.60216031730542], [-73.91132243660034, 40.60218545009838], [-73.91155222594735, 40.602213158584746], [-73.911555291408, 40.60215546472977], [-73.91148569138097, 40.60214745066214], [-73.91148979015881, 40.602124745572645], [-73.91091553413821, 40.602050454175654], [-73.91075830846516, 40.602027500983425], [-73.91074229839457, 40.60215033418664], [-73.91073679022985, 40.6024184967246], [-73.91074939384244, 40.60241889132751], [-73.91075123972264, 40.6024372998625], [-73.91070227869017, 40.602442040098055], [-73.91068388222125, 40.6024438218831], [-73.91066114943791, 40.60244602313811], [-73.91062491178121, 40.60244953083417], [-73.9105976573654, 40.60245216932951], [-73.9105785304056, 40.60245402089169], [-73.91054326019862, 40.60245743650342], [-73.91052511652381, 40.602459192827894], [-73.91050174667028, 40.602461455541146], [-73.91046885872737, 40.60246463911455], [-73.91044228268643, 40.60246721226255], [-73.91041217225653, 40.60247012792705], [-73.91038614226254, 40.60247264821339], [-73.91035235455522, 40.6024759183345], [-73.91032083788154, 40.602478969581966], [-73.91029630204754, 40.60248134610121], [-73.91027032458491, 40.60248386037127], [-73.91024244561032, 40.60248655977797], [-73.91021480448809, 40.602489235239496], [-73.91018938472047, 40.602491696652756], [-73.91015123956895, 40.60249538933629], [-73.91013268083921, 40.602497185816304], [-73.91009326067807, 40.60250100380062], [-73.91008979311793, 40.602501338156685], [-73.91005288052807, 40.602504912658624], [-73.91004612016683, 40.602505566594765], [-73.9099910825552, 40.602510895159526], [-73.90996588677584, 40.602513334752366], [-73.90993167607782, 40.602516646133346], [-73.90991192081292, 40.60251855857538], [-73.90987755469114, 40.60252188523109], [-73.90986311771721, 40.602523283667296], [-73.90981528119745, 40.60252791501045], [-73.90979379588099, 40.602529994608894], [-73.90975051902264, 40.602534184378825], [-73.9097384814668, 40.6025353491375], [-73.90971635138919, 40.60253749187627], [-73.90968750292872, 40.60254028487322], [-73.90966007411464, 40.60254294059984], [-73.90963956416478, 40.60254492594714], [-73.90960749565794, 40.602548030341985], [-73.90958653110046, 40.60255005988507], [-73.90955752545297, 40.60255286746831], [-73.90952963345006, 40.60255556836719], [-73.9094999608392, 40.602558440747195], [-73.90946895123075, 40.60256144273933], [-73.9094369077679, 40.60256454492836], [-73.90942194736621, 40.60256599282036], [-73.90939318505413, 40.60256877693231], [-73.90936888678965, 40.60257112931664], [-73.90933598554832, 40.60257431423067], [-73.90930709023294, 40.60257711145099], [-73.90928958025673, 40.60257880709976], [-73.9092614174048, 40.6025815335193], [-73.90923282104954, 40.60258430180697], [-73.90920512642842, 40.60258698200922], [-73.90918404797516, 40.602589022780954], [-73.90915072464523, 40.602592249023225], [-73.90913303330453, 40.60259396092274], [-73.90909403356345, 40.602597736009244], [-73.90907342707447, 40.602599731230946], [-73.90904427233983, 40.60260255330935], [-73.90905328065217, 40.60267738366233], [-73.90903689530528, 40.60267734322018], [-73.90901848567071, 40.60252482766916], [-73.90903202754455, 40.602523822826754], [-73.90903931176975, 40.60256941964953], [-73.90904999552458, 40.60257799186162], [-73.90906874090433, 40.602576203912996], [-73.9095731102396, 40.60252809362135], [-73.90959719234452, 40.60252579597342], [-73.9096261497805, 40.602523034412805], [-73.90964070192067, 40.60252164598109], [-73.90965952112998, 40.6025198514605], [-73.90968807012342, 40.602517127424825], [-73.90972417524168, 40.6025136830786], [-73.90973128995621, 40.60251300464531], [-73.90975811637514, 40.60251044559564], [-73.9097846079935, 40.60250791928021], [-73.90980438696396, 40.6025060315054], [-73.90983331888152, 40.60250327322307], [-73.90985487053688, 40.60250121611811], [-73.90988173079909, 40.602498654385705], [-73.90990680276435, 40.602496262962624], [-73.90992660678593, 40.6024943741813], [-73.90994939430503, 40.602492198913986], [-73.90997533502276, 40.602489724727064], [-73.90999151650696, 40.60248818172513], [-73.91002236530495, 40.60248523877154], [-73.91004411786929, 40.60248316369524], [-73.9100671150931, 40.60248097047561], [-73.91009512657575, 40.6024782975171], [-73.91011106096981, 40.60247677725651], [-73.91013283507168, 40.60247470000241], [-73.91015996416098, 40.60247211261566], [-73.91018019243549, 40.60247018291016], [-73.91020916982755, 40.602467419040266], [-73.91022711673475, 40.60246570600216], [-73.91025100644434, 40.60246342783533], [-73.91027633909889, 40.60246101073568], [-73.9103029227871, 40.602458474488394], [-73.91042934860617, 40.6024464141566], [-73.91043547430496, 40.60244582989799], [-73.91071154457025, 40.60241949317829], [-73.9107226252365, 40.602158828886026], [-73.91071565416222, 40.60214811818797], [-73.91070633053948, 40.602144515265316], [-73.9106587683434, 40.60214675368312], [-73.91062829606578, 40.60214818749471], [-73.91062070288947, 40.602148545128], [-73.91058959271591, 40.60215000959379], [-73.91058133746694, 40.60215039786989], [-73.91055007891823, 40.60215186958073], [-73.91054088529246, 40.602152302021395], [-73.91051037212463, 40.60215373794783], [-73.91049990864467, 40.60215423104707], [-73.9104711442972, 40.60215558489541], [-73.91045997763976, 40.602156110279026], [-73.91042744923308, 40.602157641780295], [-73.91041650590408, 40.602158156444325], [-73.91038608351266, 40.60215958855539], [-73.91037483200662, 40.60216011771767], [-73.91033802012439, 40.60216185007125], [-73.91032666200859, 40.6021623846742], [-73.91029576547827, 40.60216383951039], [-73.91028579107838, 40.60216430817758], [-73.91025051699418, 40.60216596882822], [-73.91024293172588, 40.6021663267779], [-73.91021056882964, 40.60216784896541], [-73.9101967479393, 40.602168499739435], [-73.91016998540684, 40.60216975891488], [-73.91015378417322, 40.60217052140797], [-73.91012895752176, 40.60217168994618], [-73.91010418626243, 40.60217285533925], [-73.91008138165765, 40.602173929287105], [-73.91005595359917, 40.6021751253163], [-73.91004020253801, 40.60217586670196], [-73.91001923951535, 40.60217685379102], [-73.90999627049041, 40.60217793429433], [-73.90997268049513, 40.60217904446189], [-73.90995100154383, 40.602180064986115], [-73.90993251206359, 40.60218093488413], [-73.90990426776705, 40.60218226487402], [-73.90988771570824, 40.60218304347249], [-73.909862033319, 40.60218425249423], [-73.90982047676913, 40.60218620762759], [-73.90981964785841, 40.60218624651408], [-73.90980032704539, 40.60218715577802], [-73.9097800893954, 40.602188108881656], [-73.90975360754564, 40.60218935561577], [-73.90973216268853, 40.602190364220455], [-73.90970812690573, 40.602191495427235], [-73.90969237078123, 40.60219223676134], [-73.90965206539157, 40.60219413300904], [-73.90964573480426, 40.60219443159515], [-73.90963361898599, 40.6021950012175], [-73.90963206821161, 40.602195074716235], [-73.90951989175076, 40.602200353589325], [-73.90951755361037, 40.6022004631553], [-73.90942704334952, 40.60220472265799], [-73.90942469395955, 40.60221004622426], [-73.90943138265835, 40.60227062921277], [-73.90941924579835, 40.602271305847204], [-73.90940934304653, 40.60219076575531], [-73.91025181119574, 40.602137494065], [-73.9102805229684, 40.6021356791902], [-73.91061414321331, 40.60211458185991], [-73.91072006391992, 40.602107882275334], [-73.91073123013267, 40.60202137602758], [-73.9106762807201, 40.6019992338873], [-73.91059343421288, 40.601983505990546], [-73.91046418020109, 40.60196875141985], [-73.91032498284824, 40.60195399011824], [-73.91029382758019, 40.60195194506716], [-73.91026730340195, 40.60195647093537], [-73.91018507191166, 40.60197661876324], [-73.91008627743624, 40.60198917304694], [-73.9100086835059, 40.6020108406834], [-73.90992843512186, 40.602035536455276], [-73.90987671309695, 40.60204509667052], [-73.9098276518062, 40.60204758503004], [-73.9097964992642, 40.60204301300348], [-73.90975806473323, 40.6020313608281], [-73.90971898460656, 40.602007077101696], [-73.90969049558633, 40.60199341197879], [-73.90964543475901, 40.601979732146084], [-73.90960169091879, 40.601971613793054], [-73.90951771292929, 40.601956558870725], [-73.9094139369032, 40.60192680753293], [-73.90932461695023, 40.60187137556538], [-73.909277170665, 40.60183726971905], [-73.90919062963887, 40.601792485818294], [-73.90915998079214, 40.60173284130614], [-73.90911535948183, 40.60167744399389], [-73.90904839322077, 40.601617770725504], [-73.90899811500016, 40.60161347266756], [-73.90894225138031, 40.60160704069869], [-73.90888637011734, 40.601613384130545], [-73.90886397208234, 40.601649564740526], [-73.90884156530149, 40.60169213306004], [-73.90877728320666, 40.60171337401026], [-73.9086850788645, 40.6017239472669], [-73.9085928408292, 40.60176007298139], [-73.90848381882816, 40.60181108882298], [-73.90845303178325, 40.601853650392755], [-73.90848647031369, 40.60191542677815], [-73.9084165741453, 40.60195582722336], [-73.908324335646, 40.601991951548804], [-73.90827682294537, 40.60200681954892], [-73.90822367322265, 40.602057879720974], [-73.90812586795658, 40.60207909420144], [-73.90805043964112, 40.602079033878404], [-73.90796940375016, 40.60209387425563], [-73.90798331091122, 40.602138600943555], [-73.90793857675921, 40.60216411700606], [-73.90788270462727, 40.60216407223665], [-73.90783802758442, 40.60214913101066], [-73.90780451831077, 40.602138457653936], [-73.90773743343888, 40.60216608449968], [-73.90766193140539, 40.602219255928695], [-73.90770657902236, 40.60225548965716], [-73.90767859330253, 40.602291666720006], [-73.90757801658788, 40.60229584363696], [-73.90746904037098, 40.60231279065254], [-73.90741315696371, 40.60232126232739], [-73.90734337430463, 40.602278621005574], [-73.90730710229315, 40.60224665164589], [-73.90722328631641, 40.602250842271935], [-73.90714497577616, 40.602314657574404], [-73.90708065697879, 40.60236145110831], [-73.90700516441797, 40.60240823389003], [-73.90694091369421, 40.60240605241943], [-73.90683199105865, 40.602384671713516], [-73.90676215947408, 40.60237822736157], [-73.90669509431964, 40.60239094887762], [-73.90665322757249, 40.602363233026466], [-73.90657223869778, 40.602344004725296], [-73.90651073122, 40.602378023109374], [-73.906438010992, 40.602439713683964], [-73.90628991431532, 40.60246301494524], [-73.906197694686, 40.6024842321717], [-73.90610553378727, 40.602462864456804], [-73.90602170167863, 40.60247983013725], [-73.90594063485345, 40.60251596168406], [-73.90585959129852, 40.60253505949457], [-73.9058121025746, 40.60253289061944], [-73.9057757380013, 40.602566930385514], [-73.90571429217962, 40.602556232842254], [-73.90567795028957, 40.602573238395365], [-73.90567510895289, 40.602607304464634], [-73.90562200191036, 40.60262642453002], [-73.90556336027852, 40.60260934215845], [-73.90551306816398, 40.602613559079295], [-73.90545995334338, 40.60263906728644], [-73.90538730361908, 40.602649653479254], [-73.9053566064639, 40.60262620628312], [-73.90531188284139, 40.60264320364504], [-73.90527559951937, 40.60261975182469], [-73.90516387803422, 40.60260262448319], [-73.90497116186197, 40.60257052619083], [-73.90489568749855, 40.60260240286534], [-73.90484544064694, 40.60257468042593], [-73.90457452701459, 40.602525481994604], [-73.90434259119111, 40.60257000473499], [-73.90413019682194, 40.60262305982415], [-73.90399039590088, 40.602705987007184], [-73.9039344950867, 40.60272510278642], [-73.90385350349119, 40.602708000858364], [-73.90376136276835, 40.60267385533676], [-73.90368597722761, 40.6026439820409], [-73.9036692796328, 40.602599252494066], [-73.90358548780357, 40.60258640628678], [-73.90341789367874, 40.602569231366644], [-73.90335926212688, 40.60254576015766], [-73.90332579345151, 40.60250740417733], [-73.90332866108642, 40.60245630398559], [-73.90330627451772, 40.60248183586858], [-73.90327267391594, 40.60253504033162], [-73.90323626727096, 40.602596759271336], [-73.90319707837811, 40.60264995950904], [-73.90313284608291, 40.602634999478695], [-73.90306024236759, 40.60261364530094], [-73.90302393445054, 40.602607227452005], [-73.90289817410734, 40.602639059703], [-73.9028365403142, 40.60275824936524], [-73.9027507945804, 40.60274725745621], [-73.90268897007445, 40.60277717174257], [-73.90264403730299, 40.60277713368875], [-73.902576614479, 40.60279419971392], [-73.90255408560772, 40.60283698901412], [-73.9024866508553, 40.602862616555804], [-73.90242477475519, 40.6029267774762], [-73.90234610004825, 40.60295667676806], [-73.90225623536327, 40.602956600375705], [-73.90208761376871, 40.60304207364253], [-73.90201449163602, 40.6031147858365], [-73.90195271046693, 40.60311473314686], [-73.90186285142322, 40.60311037507594], [-73.9017504831144, 40.60313596548357], [-73.90171674023382, 40.6031659015738], [-73.9016493483106, 40.603161562546916], [-73.90143013761238, 40.603272677456], [-73.90131218457975, 40.60327685643546], [-73.90113803879981, 40.60329811130186], [-73.90103135038235, 40.60328089632067], [-73.90098085235894, 40.603246606507476], [-73.90087406740976, 40.60329360372773], [-73.90077857954726, 40.60329780153354], [-73.90062134878224, 40.60327626115008], [-73.90050901645664, 40.60327616396207], [-73.90041343986005, 40.603340294117864], [-73.9002785777926, 40.603382985579266], [-73.90014380076332, 40.60337002677496], [-73.90002589744076, 40.603339957658044], [-73.8998349343069, 40.60333979138547], [-73.8995314914598, 40.60343798627703], [-73.89919990850507, 40.603574683382845], [-73.89900742307576, 40.6036161748893], [-73.89892158010787, 40.603636282503146], [-73.8988450766934, 40.60365191820799], [-73.89880755323304, 40.60365709041244], [-73.89876962314602, 40.603660078461445], [-73.89870283930945, 40.60365761989448], [-73.89863711790828, 40.60364824841523], [-73.89857368491457, 40.60363213883993], [-73.8985137236098, 40.6036095916779], [-73.8984333798716, 40.60359194240895], [-73.8983561492238, 40.60356751987693], [-73.89828301111473, 40.603536633812695], [-73.89821489308682, 40.603499675917796], [-73.89815265901343, 40.603457114897516], [-73.89807627731452, 40.60340231401518], [-73.89800185480577, 40.603345968944865], [-73.89793178098228, 40.60332534051492], [-73.89788865968686, 40.60340217851053], [-73.89787509197035, 40.603417120613685], [-73.89785879654409, 40.60342830060089], [-73.89783200629752, 40.60342865657719], [-73.89780602200346, 40.60342259052971], [-73.89776039488824, 40.60340618553031], [-73.89769932584397, 40.60338646962651], [-73.89762630991217, 40.603363476229845], [-73.89758896908246, 40.60336376461838], [-73.89755308037533, 40.603359253237464], [-73.89752215614736, 40.60335529560035], [-73.89748056561501, 40.60335662320115], [-73.89746504398501, 40.6033586920774], [-73.8974642755875, 40.60336511312106], [-73.8974846020403, 40.603374384561285], [-73.89749017011742, 40.603381936550555], [-73.8974808989678, 40.603390461883], [-73.89746746061941, 40.60339467364118], [-73.89743288129814, 40.60340144329913], [-73.89739961901024, 40.60341680367943], [-73.8973833796039, 40.60341722559446], [-73.89735739381571, 40.60341115994674], [-73.8973370660497, 40.60340188915287], [-73.89732290509438, 40.6034071705926], [-73.8973060199438, 40.60340599938736], [-73.8972926689453, 40.60340214316442], [-73.89723313102549, 40.603372247966966], [-73.89715888700431, 40.60333256693186], [-73.89712805517956, 40.60331787718355], [-73.89710904509039, 40.60331722556555], [-73.89708289097219, 40.60332458013396], [-73.89706804280142, 40.603330931605875], [-73.8970419760686, 40.60333021753445], [-73.8969821840918, 40.60332178246927], [-73.89688024236807, 40.60330504166095], [-73.8968036118789, 40.60329005708448], [-73.89673399613444, 40.603277797256105], [-73.89666088711941, 40.603265530894205], [-73.89658493981707, 40.603252139821095], [-73.89654755812677, 40.60325514582819], [-73.89651302448962, 40.6032565353855], [-73.89649469553737, 40.60325747630931], [-73.89648830856083, 40.60326441212933], [-73.8964853737435, 40.60327406955487], [-73.89647618572916, 40.60327724416621], [-73.89644297536412, 40.603284534602686], [-73.89640069803063, 40.6032869586776], [-73.8963675258509, 40.60329158876303], [-73.89634068580082, 40.603299985078124], [-73.89632014324167, 40.60330954050688], [-73.89629815197993, 40.6033238966817], [-73.89628190774995, 40.60332700895573], [-73.89624673323121, 40.603324088668614], [-73.89619757325677, 40.60331036736221], [-73.89617091227475, 40.6032999921527], [-73.89614004022837, 40.6032879936268], [-73.8961028735264, 40.60327217358965], [-73.89607199797904, 40.603262863885924], [-73.89604384457012, 40.603259978867975], [-73.89600302238496, 40.603254884899656], [-73.89596997504184, 40.60325147341411], [-73.89593693744821, 40.60324265525808], [-73.89590669508553, 40.603240317810844], [-73.89585609066721, 40.60322873557059], [-73.89578792525599, 40.603211645611054], [-73.89571626303791, 40.6031972434598], [-73.89566221716372, 40.603177562497564], [-73.89561514064073, 40.60316598335385], [-73.89554633671868, 40.60314461079189], [-73.89551469074578, 40.603141722972616], [-73.89546191575252, 40.60313601199152], [-73.89540348623548, 40.603130816688456], [-73.89535002955812, 40.60312348603335], [-73.89530920285445, 40.603121080155425], [-73.89526973559381, 40.603121916019255], [-73.89521755076294, 40.60312855521406], [-73.89518646376794, 40.60313532687785], [-73.8951532864784, 40.60314264494937], [-73.89515269577515, 40.60315731650589], [-73.89514890576154, 40.60317170791783], [-73.89514202241102, 40.60318541677339], [-73.89513292735533, 40.6031973104271], [-73.89512150442954, 40.60320797401802], [-73.8951080345795, 40.60321714527521], [-73.89509815412535, 40.60322228325695], [-73.89508387334074, 40.60322790697738], [-73.89506870434732, 40.603231960455474], [-73.89505294263799, 40.60323436472936], [-73.89501492149341, 40.60323303232298], [-73.89496911593581, 40.60323542500599], [-73.89494020094996, 40.603236326354434], [-73.89490978913655, 40.60324740791423], [-73.89488150427202, 40.60325796972546], [-73.89486245266193, 40.60325998074217], [-73.89481732811159, 40.603263992989824], [-73.89478274837147, 40.60327076067778], [-73.89475157385648, 40.60328557388049], [-73.8947295822229, 40.60329995672739], [-73.89469984783439, 40.60331532073108], [-73.89467867139669, 40.60331787759096], [-73.89465268435441, 40.60331181133016], [-73.89461326656757, 40.60330726714865], [-73.89453455145652, 40.60329011131931], [-73.89450571278715, 40.60328832210944], [-73.89443392639163, 40.60328195910609], [-73.89437553712958, 40.60327407346864], [-73.89431791280668, 40.60326242864694], [-73.89420904879614, 40.603232202718694], [-73.89414509368002, 40.60321676247209], [-73.8940881482603, 40.60320673816763], [-73.89400870714775, 40.603193339993965], [-73.89393058604774, 40.60318584436743], [-73.89386368619678, 40.60318544118078], [-73.89381784436425, 40.60318783286892], [-73.8937973886478, 40.60318929291597], [-73.89378250052948, 40.603198332603206], [-73.89377037361662, 40.60321382454771], [-73.89376531210335, 40.60322400028398], [-73.89375752898599, 40.603227696992754], [-73.8937406432839, 40.60322652526301], [-73.89372095675178, 40.60322156310541], [-73.89370547536312, 40.603220943840185], [-73.89368284421465, 40.603230989737106], [-73.89366433856775, 40.60324804032466], [-73.89363933020215, 40.60327743263171], [-73.8935930455252, 40.60332277395287], [-73.89355599214248, 40.603362252074405], [-73.89351762125344, 40.60339316637787], [-73.8934743652867, 40.60341811930486], [-73.89345382364209, 40.60342764775284], [-73.8934263057628, 40.603431760880326], [-73.89338190366958, 40.60343470272277], [-73.89335643009105, 40.6034436770822], [-73.89334081897587, 40.60345378655275], [-73.89330402166809, 40.60347180383682], [-73.89326447374671, 40.60347798898237], [-73.8932130138628, 40.603480868752726], [-73.89316163463332, 40.60347839685457], [-73.89312081407311, 40.60347330186404], [-73.89306395211347, 40.60345792473964], [-73.89293110384769, 40.60342915255508], [-73.89282355444473, 40.603407517101985], [-73.89274764309623, 40.60339414999597], [-73.89268436914168, 40.60338033008215], [-73.89263941349019, 40.60337089389566], [-73.89260628922699, 40.603370142572416], [-73.89256260761059, 40.603372015432825], [-73.89252807514794, 40.603373404464], [-73.89249498473285, 40.60337265381054], [-73.8924519450272, 40.6033788349078], [-73.89239904115597, 40.6033838528129], [-73.89235957546367, 40.603384686506374], [-73.89229339638528, 40.603383185094465], [-73.89223628261263, 40.60338658014452], [-73.8921960955575, 40.60338848361697], [-73.89215789933397, 40.603403260882686], [-73.89211472969849, 40.60342017281502], [-73.89208920339586, 40.60343718690596], [-73.89205313344806, 40.603451444614585], [-73.89202702408144, 40.60345344685694], [-73.89199316756299, 40.60345911824154], [-73.89191759402632, 40.60347420942617], [-73.89186106878775, 40.6034899530861], [-73.89182992930482, 40.603504793507206], [-73.89180164974196, 40.603512664716725], [-73.89177200539345, 40.60351729746736], [-73.89174104754184, 40.60351336521503], [-73.89171433958727, 40.60350834014505], [-73.89169922125943, 40.60350582653147], [-73.8916841035943, 40.60350331174379], [-73.89164186812114, 40.60350304409135], [-73.89158964434058, 40.60350970974736], [-73.8914387062907, 40.60352644567225], [-73.89138371189244, 40.60353201113372], [-73.8913632105614, 40.60353884772208], [-73.89134760000071, 40.6035489574253], [-73.89135661721187, 40.6035592034919], [-73.89135516744993, 40.603564031821016], [-73.89132743458912, 40.60358694504053], [-73.89128919675657, 40.60360440964911], [-73.8912679733093, 40.60361234570178], [-73.89123976857209, 40.603617527497455], [-73.89122153254316, 40.60360773608249], [-73.89121034637888, 40.60360069931966], [-73.89118930023031, 40.60359249659941], [-73.89117096536285, 40.603596153992335], [-73.89116301115692, 40.603613269620354], [-73.89115550834605, 40.603656183497584], [-73.89115796922074, 40.603686813349874], [-73.89114851886245, 40.60371413762713], [-73.89113273209944, 40.603740328932645], [-73.89111205598965, 40.603763304739005], [-73.89108508500112, 40.60378245708838], [-73.89104110517215, 40.6038084789253], [-73.8910106050327, 40.6038275994314], [-73.8909692995164, 40.60386814420291], [-73.89094875605967, 40.60387767052611], [-73.89092024791846, 40.603909721461804], [-73.89088766987723, 40.60392400959349], [-73.89085789102629, 40.603942061237156], [-73.89083019806574, 40.60396228335258], [-73.8908017889319, 40.603980914426046], [-73.89076073860079, 40.60399996821626], [-73.89071833666605, 40.60401043064679], [-73.89068367020164, 40.60402523875584], [-73.89064619902992, 40.60403628382534], [-73.89060446917648, 40.60405374555329], [-73.89055147645125, 40.604066831232615], [-73.8905125583027, 40.60408267710517], [-73.89045726371826, 40.604112391171796], [-73.89041056052695, 40.60413196213995], [-73.89034344230693, 40.60415032666846], [-73.89030103578878, 40.60416348046474], [-73.89027849507592, 40.604162796369955], [-73.89022137677411, 40.6041688780859], [-73.89017331472813, 40.60418251712604], [-73.890142308947, 40.604183938463436], [-73.8901105870756, 40.6041837362525], [-73.89005980212846, 40.60419092410768], [-73.89001812356094, 40.604200318135675], [-73.88997427282537, 40.604215608604726], [-73.88992616445805, 40.60423462786481], [-73.88988162700245, 40.60425098815236], [-73.8898498133187, 40.604261518539545], [-73.88981931636272, 40.604277977865934], [-73.889787415091, 40.60429654771056], [-73.88975202422739, 40.60431242418273], [-73.88970753820436, 40.60432074489342], [-73.88966942825904, 40.604327480348], [-73.88962213646899, 40.60433469766528], [-73.88957343459487, 40.60434405563658], [-73.88953659378703, 40.60436473471655], [-73.88949061795029, 40.60438057144903], [-73.8894431084006, 40.604406562307766], [-73.88941698840019, 40.604413915164706], [-73.88937241465213, 40.60443027521992], [-73.88934064576748, 40.60443542564425], [-73.88930393936333, 40.60444271182473], [-73.889272206251, 40.604447889236795], [-73.88925094029365, 40.60445848674722], [-73.88916194261427, 40.60450731646343], [-73.88911991617734, 40.60452771482308], [-73.88910208281249, 40.604532661339505], [-73.88908312889893, 40.604539445739746], [-73.88906807134859, 40.60454313393745], [-73.8890537022189, 40.60454405002204], [-73.88903616246353, 40.604545318708205], [-73.88902417118193, 40.60454558003524], [-73.88900775526638, 40.60454501351478], [-73.88898582146595, 40.60454701859619], [-73.88897627762157, 40.60454813316309], [-73.88896957425946, 40.60455100701696], [-73.8889619907474, 40.60456183882948], [-73.88895962644617, 40.60457377484336], [-73.8889624822399, 40.60458681371929], [-73.88896802301711, 40.604606826413736], [-73.88897128950605, 40.60463051451192], [-73.88897061353639, 40.6046445645324], [-73.88896224867293, 40.604667909685894], [-73.88894909569777, 40.604691029576436], [-73.88892758849589, 40.60471496429544], [-73.88890320391097, 40.604737110535325], [-73.88887864752958, 40.60475470115008], [-73.88885369531336, 40.60477292229178], [-73.88883148283712, 40.604788321033254], [-73.88880141294047, 40.60480845910823], [-73.88876767301007, 40.60482655965154], [-73.8887311613169, 40.604844384764014], [-73.88870705524045, 40.604853137607904], [-73.8886955641107, 40.6048558153236], [-73.88853217218396, 40.604903764487716], [-73.88844750219316, 40.60492935333916], [-73.88840497731677, 40.60494730804252], [-73.88837063642058, 40.60495991996444], [-73.88834931982103, 40.60497866654847], [-73.88834454377714, 40.60498974982812], [-73.88834371671093, 40.60498944596436], [-73.88832014479517, 40.60500061635721], [-73.88829471413327, 40.60500511509905], [-73.88826766445584, 40.60500903529082], [-73.8882394945908, 40.605014821332176], [-73.8882184920877, 40.60502014944432], [-73.88818728742056, 40.60503087044411], [-73.88816018183915, 40.605044534053846], [-73.88813757394237, 40.60506056252109], [-73.88810556920826, 40.60509354081569], [-73.88808576758608, 40.60510981961291], [-73.88805490559625, 40.60512968113553], [-73.88802478686374, 40.605137029278495], [-73.88798465970856, 40.60514458444247], [-73.88794618738609, 40.60515271767376], [-73.88781168218313, 40.605214695727426], [-73.88776291623786, 40.60523796633564], [-73.88770414238029, 40.605259937003005], [-73.88764437104099, 40.60527704731357], [-73.88756220282258, 40.60529371954204], [-73.8874557241072, 40.60531305457465], [-73.8873463911858, 40.60531932245006], [-73.88725806021515, 40.60532028993035], [-73.8871577037901, 40.605321490799454], [-73.8870953366259, 40.60532191153127], [-73.88704036373788, 40.605316661597534], [-73.88699136586393, 40.605311308283596], [-73.8869180480688, 40.60529577088031], [-73.88686309321677, 40.60528080653206], [-73.88682674518223, 40.60527088233573], [-73.88677527065971, 40.605263137357106], [-73.88670784167516, 40.605255732461735], [-73.88661834051047, 40.605247257189376], [-73.88650617257574, 40.60523074135186], [-73.8864281664883, 40.60521959011375], [-73.88635267579495, 40.60521083044373], [-73.88626714040382, 40.605200740487135], [-73.88619211092349, 40.60519442290595], [-73.8861267655826, 40.60519001096642], [-73.88605682072867, 40.60518021355977], [-73.88598136294254, 40.6051729616086], [-73.88586517865721, 40.605166266412866], [-73.88575595461785, 40.60515455733594], [-73.88566338591882, 40.605149507593005], [-73.88558856481752, 40.605147745111296], [-73.8854949801629, 40.6051475805454], [-73.88524221829852, 40.60515782180433], [-73.88512439308187, 40.605161825651166], [-73.88504138218454, 40.60516694191991], [-73.88495200984666, 40.60516910542513], [-73.88491057829488, 40.60515440512451], [-73.88486646196644, 40.605145312204954], [-73.88482095644369, 40.60514209370178], [-73.88477539810481, 40.60514484413417], [-73.88473112487893, 40.60515348272891], [-73.88468943695659, 40.605167755792976], [-73.88465155860544, 40.60518724416357], [-73.88461860221912, 40.60521137551736], [-73.88459153564902, 40.60523944117771], [-73.88457115377963, 40.6052706169266], [-73.88455805518677, 40.60530398720932], [-73.88456018029669, 40.605315084782355], [-73.88455974878485, 40.60532629510395], [-73.88455677371944, 40.605337278638245], [-73.88455134520697, 40.60534770271806], [-73.88454362766481, 40.60535725162074], [-73.88453385483898, 40.60536563613088], [-73.884522322727, 40.60537260229965], [-73.88450938061123, 40.60537793913673], [-73.8839804295303, 40.60556651972071], [-73.88297332801775, 40.604652817522734], [-73.88269358924953, 40.60422497014918], [-73.88296945455222, 40.60378535567241], [-73.88289983055381, 40.60350161304467], [-73.88332673656372, 40.60305392914338], [-73.88329452020042, 40.60291610455577], [-73.88325380693556, 40.602779592190394], [-73.88320468752659, 40.60264469618466], [-73.88314727145664, 40.602511717072225], [-73.88308168669094, 40.60238095111473], [-73.88300807939133, 40.602252689641375], [-73.88292661358965, 40.6021272184002], [-73.88283747082252, 40.60200481692117], [-73.88274084972663, 40.601885757893825], [-73.88269460170932, 40.601860424173175], [-73.88265422903095, 40.60182980626406], [-73.88262075958285, 40.6017946837073], [-73.88259504549387, 40.601755950731935], [-73.8825777414367, 40.60171459348781], [-73.882569287959, 40.601671664937165], [-73.88256990027074, 40.60162825804578], [-73.88257956276458, 40.601585477956135], [-73.88257696119135, 40.60150002245466], [-73.88256638638296, 40.601414924813234], [-73.88254789200822, 40.60133061666853], [-73.88252157190462, 40.60124752565083], [-73.88248755960187, 40.601166073215715], [-73.88244602764492, 40.60108667250561], [-73.88239718671782, 40.601009726255015], [-73.88233964771186, 40.60093638831372], [-73.88228939242069, 40.60086000382321], [-73.8822466968278, 40.600780992297096], [-73.88221179539481, 40.60069978767551], [-73.88218487977502, 40.60061683594176], [-73.88216609776086, 40.60053259267283], [-73.88217863808364, 40.600473586712326], [-73.88218296677205, 40.60041390270305], [-73.88217903583141, 40.600354202706136], [-73.8821668888879, 40.600295148958395], [-73.88214666070611, 40.60023739652581], [-73.88211857569163, 40.600181586037884], [-73.88208294540055, 40.60012833658115], [-73.88204016508439, 40.600078238832246], [-73.88199070930334, 40.60003184850537], [-73.88193512666273, 40.59998968018904], [-73.88190810309368, 40.59995704012172], [-73.88188677253875, 40.59992203573894], [-73.88187148466092, 40.59988524086714], [-73.88186249006357, 40.59984725868326], [-73.88185993618363, 40.59980871182692], [-73.88185916615947, 40.5997665517899], [-73.88185134273189, 40.599724811480996], [-73.88183659388517, 40.59968417366449], [-73.88181516088457, 40.59964530306987], [-73.88178739432897, 40.59960883551799], [-73.88175374841572, 40.59957536752163], [-73.88171477351027, 40.59954544652754], [-73.88168557128205, 40.599524965934926], [-73.88166095629371, 40.599501240742846], [-73.88164154069716, 40.59947486097972], [-73.8816278073381, 40.59944648269101], [-73.88162009774692, 40.59941681162293], [-73.88161860364681, 40.59938658567113], [-73.88162336218662, 40.599356556530054], [-73.88163425501722, 40.599327470998325], [-73.88164104467455, 40.59928761224311], [-73.88163908296839, 40.59924744673533], [-73.88162842405987, 40.59920808310126], [-73.88160936216501, 40.59917060783245], [-73.88158242343145, 40.599136055296974], [-73.88154835141519, 40.599105379190554], [-73.88151984246439, 40.59906582301651], [-73.88149989695842, 40.59902332868791], [-73.88148901661386, 40.59897896515862], [-73.8814874751096, 40.59893384840017], [-73.88149531120334, 40.5988891133295], [-73.88151232775914, 40.59884588525993], [-73.88153809670827, 40.59880525159409], [-73.8815506426699, 40.59864930989324], [-73.88155375804612, 40.59849309329212], [-73.88154743629828, 40.598336932969204], [-73.88153169089445, 40.598181159981095], [-73.88150655528008, 40.59802610456117], [-73.88147208280688, 40.59787209541922], [-73.8814283466183, 40.59771945904484], [-73.88137543949483, 40.597568519014914], [-73.8813134736566, 40.59741959530841], [-73.88124258052439, 40.59727300362764], [-73.88116291044133, 40.59712905472902], [-73.88107463235342, 40.596988053764875], [-73.88097793345088, 40.5968502996364], [-73.88087301877087, 40.59671608436004], [-73.88082159887102, 40.59663895524159], [-73.88077787665533, 40.596559115172724], [-73.88074209612957, 40.596477009777004], [-73.88071445697005, 40.59639309731992], [-73.88069511341199, 40.59630784615116], [-73.88068417338692, 40.596221732090925], [-73.88068169792272, 40.59613523577336], [-73.88068770080325, 40.59604883996433], [-73.88070214849132, 40.59596302686683], [-73.8807249603179, 40.59587827542993], [-73.88075600893184, 40.5957950586752], [-73.88076742388394, 40.59563444273594], [-73.8807718709499, 40.59547362699728], [-73.88076934532417, 40.595312787053864], [-73.8807598498148, 40.59515209852563], [-73.88074304729449, 40.59498895277266], [-73.88071906028287, 40.59482632973357], [-73.88068791594144, 40.594664413198295], [-73.87990713258998, 40.59204799432243], [-73.87941617176985, 40.592132005798014], [-73.8794080701356, 40.59210340689228], [-73.8798990285191, 40.59202063885202], [-73.87971492228009, 40.59139889182903], [-73.87935729205896, 40.590843735297476], [-73.87897881256478, 40.59089541936104], [-73.87896890438742, 40.59089677311754], [-73.8789241193116, 40.590712780281514], [-73.87929326377282, 40.59066143187066], [-73.87924495401445, 40.59044504782277], [-73.8792343976634, 40.590266409958495], [-73.87901743697853, 40.59004820262238], [-73.87900019672759, 40.59003086209085], [-73.87898885895304, 40.589984666635424], [-73.87896831259086, 40.58990093504831], [-73.87896840364142, 40.589730418130316], [-73.87907498390977, 40.58960053485112], [-73.87913901161363, 40.589373200231236], [-73.87905394992099, 40.589097100566306], [-73.87903458169106, 40.58897017721139], [-73.87901194248275, 40.58884356883525], [-73.8789686363146, 40.58863835066147], [-73.87891677450195, 40.58843428870589], [-73.87885640979461, 40.58823159021744], [-73.8787876035763, 40.58803046105838], [-73.87871042580291, 40.58783110549548], [-73.8786249549299, 40.58763372599223], [-73.87853127783266, 40.58743852300377], [-73.87842948971884, 40.58724569477255], [-73.87831969403057, 40.58705543712823], [-73.87820200233887, 40.58686794328778], [-73.87807653423106, 40.58668340365991], [-73.8780288676496, 40.58664783223147], [-73.87798683582405, 40.58660834694272], [-73.87795098768318, 40.58656546347434], [-73.87792179139247, 40.58651974188609], [-73.87789962824264, 40.586471779302926], [-73.87788478766805, 40.58642220211564], [-73.87671672482064, 40.584914430200875], [-73.87970231664, 40.58013510069815], [-73.881742900221, 40.57959170076465], [-73.88186724526732, 40.57945462578763], [-73.88174676983894, 40.5791588072124], [-73.88206958203642, 40.579073978696584], [-73.88214694143117, 40.57918997182285], [-73.88232347768535, 40.57917709878924], [-73.88250066233414, 40.57917140864374], [-73.88262601549968, 40.579162942917755], [-73.88275043256014, 40.57914853491062], [-73.88287342943622, 40.57912824068064], [-73.88299452757539, 40.579102139188315], [-73.88311325581375, 40.579070331988504], [-73.88322915221077, 40.57903294283601], [-73.88334176584539, 40.57899011720318], [-73.88348620025185, 40.57891877181205], [-73.8836344631034, 40.578852148301955], [-73.88378628902927, 40.57879036591524], [-73.88394140628381, 40.57873353522914], [-73.88403827055893, 40.57869235816558], [-73.884138362902, 40.57865594491177], [-73.88424128082936, 40.57862444188815], [-73.88434661049753, 40.57859797577038], [-73.88445392836726, 40.57857665298039], [-73.88456280290566, 40.57856055925844], [-73.88467279632157, 40.578549759318044], [-73.88477619541953, 40.57853957203976], [-73.88488017951568, 40.57853380579347], [-73.88498442372081, 40.57853247859521], [-73.88510319335664, 40.57852048655188], [-73.88522271298856, 40.5785141691984], [-73.88528096576366, 40.57850752790356], [-73.88533792081009, 40.57849608132347], [-73.88539289841202, 40.57847996606461], [-73.88544524245411, 40.5784593744507], [-73.88549432825205, 40.57843455222713], [-73.88553401637762, 40.57840968228591], [-73.88557041998473, 40.57838204518409], [-73.88558494208567, 40.57837687354125], [-73.88560035671478, 40.57837352506468], [-73.88561627300489, 40.57837208466162], [-73.88563228736759, 40.578372588856105], [-73.88564799372838, 40.57837502486346], [-73.88565798789254, 40.57838167379117], [-73.88566928215334, 40.57838697642667], [-73.88568156784665, 40.57839078785232], [-73.88569450921206, 40.57839300390419], [-73.88570775257024, 40.57839356401887], [-73.88571671041561, 40.57839299322059], [-73.8857255282976, 40.57839166197947], [-73.88573148504992, 40.57838900602619], [-73.88583872882465, 40.57834509319989], [-73.88594966973379, 40.578306891516924], [-73.88606378539488, 40.5782745808538], [-73.8861805384788, 40.578248313348745], [-73.88629937924031, 40.578228212684515], [-73.88641974810582, 40.57821437350694], [-73.88654107830757, 40.57820686097903], [-73.88666279855236, 40.57820571047401], [-73.88668284232692, 40.578206290129856], [-73.88675923021812, 40.5782047176485], [-73.88683500337885, 40.57819717763759], [-73.88690936203807, 40.57818374968068], [-73.8869815213548, 40.578164575507394], [-73.8870507197036, 40.57813985749736], [-73.88705637693425, 40.578137963105526], [-73.88715103597593, 40.57811032685045], [-73.88724859434285, 40.57808935630708], [-73.88734824650479, 40.578075224627106], [-73.88744916964511, 40.57806804849395], [-73.88755053045395, 40.57806788716012], [-73.88765149200802, 40.57807474195774], [-73.88769395836898, 40.57806323884464], [-73.88774071588243, 40.57805666628503], [-73.88780498557446, 40.578059277360616], [-73.88795268621801, 40.578065274489646], [-73.88805727369287, 40.57809295941428], [-73.88816417438308, 40.57811491495664], [-73.88827284915907, 40.57813103038835], [-73.88838274994282, 40.578141224434766], [-73.888493322472, 40.57814544568404], [-73.88860400909583, 40.57814367284707], [-73.88871425158845, 40.57813591486514], [-73.8888234939638, 40.57812221086377], [-73.88893118528091, 40.57810262995653], [-73.88903678242181, 40.57807727089549], [-73.88908067850879, 40.57802862334688], [-73.8891306130966, 40.57798348819286], [-73.88918610590937, 40.57794229953753], [-73.88924662321557, 40.57790545352723], [-73.8893115829617, 40.57787330454105], [-73.88938036037099, 40.57784616178202], [-73.88945229395125, 40.577824286304], [-73.88952669185717, 40.57780788850105], [-73.88964060608498, 40.57779536875448], [-73.88975533585844, 40.57778841649257], [-73.88987041362529, 40.57778706004757], [-73.88998537041505, 40.57779130494721], [-73.89009973775032, 40.577801133892564], [-73.89021304955556, 40.57781650682823], [-73.89032484405563, 40.57783736110555], [-73.89104014553803, 40.57787823343675], [-73.89111239391109, 40.57787862393499], [-73.89118413759032, 40.577872112300845], [-73.89125424685513, 40.5778588010704], [-73.89132161772321, 40.5778388998507], [-73.89138518933413, 40.57781272201844], [-73.89144396065697, 40.577780679785086], [-73.89149700625241, 40.577743277705956], [-73.8915434908459, 40.57770110473469], [-73.89158268247964, 40.57765482494874], [-73.89174347713703, 40.577588929101594], [-73.89190796865317, 40.57752857091994], [-73.89207582919903, 40.57747387069375], [-73.89224672423306, 40.57742493743623], [-73.89242031316941, 40.57738186866758], [-73.8925962500554, 40.577344750220306], [-73.89277418426028, 40.577313656068284], [-73.89295376117498, 40.57728864817913], [-73.89314154718112, 40.57725693789829], [-73.89332751457036, 40.577219512066144], [-73.89351136423606, 40.57717643087818], [-73.89369280047919, 40.57712776362629], [-73.89387153148483, 40.57707358858636], [-73.89404726978998, 40.57701399289281], [-73.89418492279262, 40.57696594323416], [-73.89432083569274, 40.576915095306695], [-73.89443269880123, 40.576874546111796], [-73.89454772077411, 40.57683950970764], [-73.89466543488585, 40.576810128260085], [-73.89478536348979, 40.57678652098937], [-73.89490701995419, 40.57676878368561], [-73.89502991063868, 40.576756988320746], [-73.89515353689528, 40.5767511827561], [-73.89527739709217, 40.57675139054873], [-73.8954009886479, 40.57675761085538], [-73.89552381007165, 40.576769818436354], [-73.89557605010988, 40.57698156050732], [-73.89563599317809, 40.57719211290272], [-73.89570359297613, 40.57740131273656], [-73.89577879727906, 40.57760899816773], [-73.8958615479775, 40.57781500852495], [-73.89595178112192, 40.57801918443166], [-73.89604942697282, 40.57822136792882], [-73.89615441005304, 40.57842140259736], [-73.89626664920769, 40.57861913367957], [-73.89638605766544, 40.578814408198255], [-73.89651254310586, 40.57900707507545], [-73.89660090211501, 40.57907166583643], [-73.89668301620476, 40.57914088196421], [-73.89675846995434, 40.57921437330631], [-73.8968268816326, 40.579291768081596], [-73.89688790512886, 40.579372674760734], [-73.89694123170634, 40.579456684046434], [-73.89698659156097, 40.57954337094459], [-73.8970282801111, 40.57965647058665], [-73.89706196811126, 40.579771109626805], [-73.89708755890891, 40.57988695930394], [-73.89710497907294, 40.580003687383694], [-73.89711417860228, 40.58012095911049], [-73.8971151310708, 40.58023843816832], [-73.89710783370369, 40.58035578764454], [-73.89709230738518, 40.58047267099644], [-73.89706859660006, 40.58058875301628], [-73.89703676930591, 40.58070370079286], [-73.89699691673887, 40.58081718466606], [-73.8969491531528, 40.58092887917255], [-73.8968936154914, 40.58103846397899], [-73.89683046299629, 40.58114562480122], [-73.89675987675102, 40.58125005430533], [-73.89663622640884, 40.58157138211056], [-73.89651771599009, 40.58189383393112], [-73.89648974288556, 40.58201651924973], [-73.89646977893447, 40.58214011078842], [-73.89645787324548, 40.582264304802216], [-73.89645405512407, 40.58238879606405], [-73.8964583339991, 40.58251327861426], [-73.8964706993996, 40.58263744651309], [-73.89649112098013, 40.58276099459256], [-73.89651954859457, 40.5828836192059], [-73.89655591241934, 40.583005018974646], [-73.89660012312444, 40.58312489552905], [-73.89690145326705, 40.583631140141904], [-73.8969306950178, 40.58366582869708], [-73.89695376340694, 40.583703124871526], [-73.89697026562979, 40.58374239360962], [-73.89697992068518, 40.58378296626746], [-73.89698256415994, 40.58382415199714], [-73.89697815102886, 40.583865249510886], [-73.89697560657291, 40.58387686054261], [-73.89698071894091, 40.58390011302919], [-73.89699309708226, 40.583928121658865], [-73.89703138264092, 40.583987579866005], [-73.89720897798324, 40.58417595175524], [-73.89731639815692, 40.58432286681173], [-73.89743775799184, 40.584466062322235], [-73.89754679748128, 40.584567167830166], [-73.8976690506408, 40.58466790069001], [-73.89782641488564, 40.584833196672875], [-73.8979560997442, 40.5849481533903], [-73.89807119409498, 40.58509370264428], [-73.89812203191424, 40.585162477710405], [-73.89813542477576, 40.5851856550598], [-73.8981433096778, 40.58520992718846], [-73.89814405045641, 40.58521956213088], [-73.89814492456449, 40.58523658071856], [-73.89814920164099, 40.58527475426075], [-73.89814195208848, 40.585306903778694], [-73.89812462627432, 40.585340466687704], [-73.89807651852621, 40.5854024856551], [-73.89800527113263, 40.585494117516134], [-73.89796085650754, 40.58556684501418], [-73.8979120612451, 40.58571759730073], [-73.89789705159707, 40.58582125572848], [-73.89790736589434, 40.585912999511976], [-73.89792276360309, 40.5859618833834], [-73.89794445722232, 40.586011643328135], [-73.89802850979339, 40.586149390157516], [-73.89808148539954, 40.58621873957948], [-73.89814988350828, 40.586308274201855], [-73.89820561988033, 40.58640007077294], [-73.89825781710847, 40.58647301292908], [-73.89827886597834, 40.586518437747046], [-73.89827579278028, 40.586557129139486], [-73.89825559212163, 40.5865953605759], [-73.89823853677991, 40.586622708394664], [-73.89822306945675, 40.58665126098159], [-73.89822124402095, 40.58668197153612], [-73.89824518891376, 40.58674359215488], [-73.89826305550277, 40.586785395352344], [-73.89830506227048, 40.5868411850034], [-73.898384918464, 40.5869397155655], [-73.89846957082578, 40.58705274680053], [-73.8985455781012, 40.58713411598312], [-73.89859712057444, 40.58719835474862], [-73.89864072346847, 40.58725776984761], [-73.8986961284725, 40.5873440055765], [-73.89876737942862, 40.587425381473125], [-73.89880458585415, 40.58746425427504], [-73.89886748676379, 40.587505021657066], [-73.89891599908806, 40.58753743606102], [-73.89905144380202, 40.58745417790854], [-73.89909217892532, 40.587483891029194], [-73.89918075194707, 40.58742416277747], [-73.8991872325975, 40.58743040910319], [-73.89927787511728, 40.58737574348943], [-73.89925698447745, 40.587355611253955], [-73.8993501676844, 40.587299416256286], [-73.89938087234081, 40.587275057226364], [-73.89965842539361, 40.587097199550456], [-73.89973341305188, 40.58716521210989], [-73.8996324668466, 40.58723076754464], [-73.89961007389714, 40.58724531081463], [-73.89944274924315, 40.58735182480616], [-73.89932907018108, 40.5874245283683], [-73.89931117446991, 40.587408188785545], [-73.89923739812788, 40.587455151470785], [-73.89923001526552, 40.587448410814794], [-73.89921710048566, 40.587456631717345], [-73.89922478113836, 40.58746364369793], [-73.89914367094656, 40.58751527580471], [-73.89915650729063, 40.58752699536932], [-73.89899371735159, 40.58762794668685], [-73.8990115999968, 40.58765322476687], [-73.89903425420563, 40.587676171745805], [-73.89906116730546, 40.58769626832067], [-73.89909173024091, 40.58771305969453], [-73.89916108529071, 40.58773080587935], [-73.89922716301633, 40.58775474114151], [-73.89928901709786, 40.587784522697845], [-73.89934576169931, 40.58781972403837], [-73.89939658415568, 40.58785984103415], [-73.89944075661062, 40.587904299156776], [-73.89948132548207, 40.587937272259744], [-73.89952784020072, 40.58796531318173], [-73.89957927948808, 40.587987806251505], [-73.89963451393776, 40.58800425760683], [-73.8996923308126, 40.58801430603837], [-73.89975146067313, 40.5880177309204], [-73.89981060525021, 40.58801445705558], [-73.89984402458235, 40.58797915724739], [-73.89990604460492, 40.587794089248185], [-73.89994811282996, 40.587805210472645], [-73.89994383459322, 40.587814097702314], [-73.89993476711301, 40.58781183519476], [-73.8998797733794, 40.587966924074884], [-73.89987827846566, 40.587971138818126], [-73.89994345059304, 40.587983416525645], [-73.8999446503535, 40.587979950883664], [-73.89999572400494, 40.58783251164106], [-73.90000385255242, 40.58783414860932], [-73.9000093251489, 40.58781834931443], [-73.90006476759744, 40.58782951128399], [-73.90030360847027, 40.58773256542617], [-73.90035530331365, 40.587806585406234], [-73.90055860047781, 40.58772772381792], [-73.90115709211169, 40.58749551116587], [-73.90110684815338, 40.5874202450345], [-73.90117826603625, 40.58739872567171], [-73.90121791883185, 40.5873833400231], [-73.90123018530907, 40.587401715480965], [-73.90116046868427, 40.58742876598385], [-73.90120752373674, 40.58749925509589], [-73.90095285629886, 40.58759806116519], [-73.90091898871088, 40.58761120045025], [-73.90034256835513, 40.58783483590821], [-73.90029373007684, 40.587761675314546], [-73.900031156696, 40.58786974774757], [-73.89996820251265, 40.58801714868562], [-73.89996767315942, 40.588017130131504], [-73.89996122599122, 40.588016904887446], [-73.89995774139376, 40.5880265860603], [-73.89995410717111, 40.58803668977955], [-73.8999523407064, 40.5880415986969], [-73.90007855320306, 40.58806069212094], [-73.90012745158445, 40.58808439598863], [-73.90016550698388, 40.58808313020387], [-73.90023561971621, 40.5880910040035], [-73.90027496146784, 40.588096263565845], [-73.90031112653362, 40.58810032001037], [-73.9003520358427, 40.58809832487129], [-73.90044462743533, 40.58809117422714], [-73.90051186093923, 40.58809010599277], [-73.90057465441193, 40.58808855960117], [-73.90062128251243, 40.58809114873182], [-73.90065268067511, 40.58809037660059], [-73.90071546045651, 40.588083995388345], [-73.9008115206208, 40.58806910272747], [-73.90088030290644, 40.588053524382154], [-73.9009424181541, 40.58803481230604], [-73.9009927926347, 40.58801466876068], [-73.90101909683604, 40.58800713333243], [-73.90107897284466, 40.58798093035283], [-73.90113980175646, 40.58795569262862], [-73.90116957776658, 40.58794162365689], [-73.90121296750546, 40.58791882972239], [-73.9012601653681, 40.58789627291517], [-73.90130739238987, 40.587884595307685], [-73.90133052222224, 40.587875856565546], [-73.9013450777913, 40.58786326163603], [-73.90135392693685, 40.58785091594769], [-73.90140231439533, 40.5877986179319], [-73.90149093994187, 40.587729573813405], [-73.90155711564758, 40.58768764446793], [-73.90161062709599, 40.58765419815086], [-73.90164797681828, 40.587624884329685], [-73.90167929071572, 40.58759267946963], [-73.90172868375056, 40.58756068825203], [-73.90177365047886, 40.58753426509109], [-73.90180023790927, 40.587514156473816], [-73.90181383140684, 40.58749696901538], [-73.90182711495812, 40.587482684408165], [-73.90184072984661, 40.58747395820692], [-73.90185214591399, 40.587473457999295], [-73.90187308305745, 40.587475360420406], [-73.90191970554051, 40.587475531402355], [-73.90194760410006, 40.58747162028164], [-73.9019732551163, 40.587456590820544], [-73.90198305508368, 40.587444002809384], [-73.9019906415914, 40.587434803705435], [-73.90201280116307, 40.58741905423124], [-73.90203783294848, 40.58740982778253], [-73.90206190935329, 40.58739963654965], [-73.90207582339582, 40.58738438245556], [-73.90207865778105, 40.587376399580194], [-73.9020846574694, 40.587365994245914], [-73.90209953736353, 40.58735654117641], [-73.9021448686587, 40.587347767910146], [-73.90217786310491, 40.58735182810351], [-73.90221211386367, 40.587351534428755], [-73.90224476490202, 40.58734519927617], [-73.90227615524799, 40.58734200740233], [-73.90230280005393, 40.587343659451946], [-73.90233643624191, 40.58735037867021], [-73.90239958293348, 40.58736285530437], [-73.90246145296427, 40.58737243289574], [-73.90257722282765, 40.587375883544674], [-73.90260767161384, 40.58737656322494], [-73.90263812040308, 40.58737724122183], [-73.9027037766589, 40.58737980056432], [-73.90276847512406, 40.587378976471875], [-73.90281793711038, 40.587373581986355], [-73.90285597557542, 40.58736627066676], [-73.90289526590239, 40.58735218684215], [-73.90294629774272, 40.58734074682029], [-73.90298179375928, 40.587331261627426], [-73.90301852886407, 40.587310170861926], [-73.90304607979559, 40.587294653578944], [-73.90307999027851, 40.58728517254634], [-73.90316147474502, 40.587275618526185], [-73.90325153084287, 40.58726991931488], [-73.9032851377987, 40.58726575736088], [-73.90329940237604, 40.587263317740984], [-73.90330193885434, 40.58726307176453], [-73.90332316116782, 40.587252401914284], [-73.90339384113169, 40.5872351262527], [-73.903429952794, 40.58721863016428], [-73.90345147414308, 40.58720143007638], [-73.90346445668251, 40.58719367264418], [-73.90350374038445, 40.587177170988454], [-73.90353637445696, 40.58716478965208], [-73.90354523920348, 40.587158490113076], [-73.90355282546453, 40.58714929040312], [-73.90355753648133, 40.5871311495783], [-73.90355589541075, 40.587110600044134], [-73.90355615784537, 40.58708956359199], [-73.90356373306015, 40.587075528637186], [-73.90358685482148, 40.587064370643716], [-73.9036905070502, 40.58704269248063], [-73.90372220483816, 40.58703660019962], [-73.9037910282247, 40.58703673641881], [-73.90389474443407, 40.58703923748239], [-73.90401746259435, 40.58703203728439], [-73.90409070029818, 40.5870229779695], [-73.90413063108551, 40.58701155345195], [-73.90416896287779, 40.5869952945298], [-73.90418425170782, 40.58698537201804], [-73.90419725298479, 40.586973707146164], [-73.90420762950359, 40.58696060245887], [-73.90421511213714, 40.586946397843775], [-73.9042195068151, 40.5869314617167], [-73.90422415294174, 40.58692614778621], [-73.90422991153025, 40.58692150985193], [-73.9042366164965, 40.586917681677], [-73.90424407446119, 40.58691477367025], [-73.90425207032871, 40.5869128697018], [-73.90426037348962, 40.58691202468422], [-73.90426874447125, 40.58691226298856], [-73.90427694184564, 40.586913577742024], [-73.90428472919143, 40.58691593102567], [-73.90429188191315, 40.58691925496826], [-73.90429819371803, 40.58692345370346], [-73.90436592432967, 40.58696016202152], [-73.90486485460703, 40.58707466945402], [-73.90508188414515, 40.58720732955103], [-73.9051553687622, 40.58723286693648], [-73.9052355542867, 40.58724567170061], [-73.9058534506541, 40.58748821093106], [-73.90586686933293, 40.5874922000416], [-73.90587919858659, 40.58749787595081], [-73.90589007690757, 40.58750507223483], [-73.90589918533036, 40.58751357789051], [-73.90590625678568, 40.587523143522574], [-73.9059110839291, 40.58753348865561], [-73.90591352522198, 40.587544309958545], [-73.9058923383927, 40.58755238362624], [-73.90587306877613, 40.58756288429728], [-73.90585619071548, 40.58757555348632], [-73.90584211968375, 40.58759007932756], [-73.90583120205694, 40.58760610425149], [-73.9058237065873, 40.58762323378685], [-73.9058198177865, 40.587641046270406], [-73.90583401259678, 40.58772110307146], [-73.90585651561567, 40.58780004538562], [-73.90588718425934, 40.58787737291435], [-73.90600054222729, 40.587924248892826], [-73.9061160973374, 40.58796789581832], [-73.90623369139416, 40.5880082539362], [-73.90630341444427, 40.58802236332109], [-73.90637485425499, 40.58802999406097], [-73.90644697891925, 40.58803103593458], [-73.90651874663754, 40.588025473892536], [-73.90658912076653, 40.58801338827529], [-73.90665708479355, 40.587994953652625], [-73.90672165702053, 40.58797043630181], [-73.9067932505373, 40.58792889657512], [-73.90685972549076, 40.58788267665216], [-73.90692055722614, 40.58783214133196], [-73.90697526563262, 40.587777689472716], [-73.90702341893197, 40.58771975084334], [-73.90706463708528, 40.58765878273196], [-73.90709859479351, 40.587595266335505], [-73.90712502406382, 40.58752970296232], [-73.90710529235673, 40.5874945864288], [-73.90707804625505, 40.58746252397108], [-73.90704407257009, 40.58743444147446], [-73.90700435238543, 40.58741114989218], [-73.90696003272562, 40.587393321826745], [-73.90691239343307, 40.58738147210734], [-73.90686281021054, 40.587375942923465], [-73.90681271489419, 40.587376893944146], [-73.90678349384332, 40.58738720498031], [-73.90675260499656, 40.587394136896236], [-73.9067180522822, 40.587397660293924], [-73.90668320402682, 40.58739695785735], [-73.90664894381285, 40.587392047396925], [-73.90661476130273, 40.587390112812926], [-73.90658136569024, 40.58738422645951], [-73.90654953832612, 40.58737452605873], [-73.90652002386878, 40.587361238569216], [-73.9064935128608, 40.58734467487575], [-73.90647062557228, 40.5873252225155], [-73.90645189748875, 40.58730333661102], [-73.90643776678166, 40.587279529221284], [-73.90634513023984, 40.58718288944877], [-73.90633517612736, 40.587131926250784], [-73.90638772801907, 40.58710187945027], [-73.90640863377209, 40.58709169243014], [-73.90641463144355, 40.58708128569555], [-73.90641557145699, 40.58707741519096], [-73.9064092070858, 40.58706896317273], [-73.90641708967891, 40.5870520248279], [-73.90643924223771, 40.587033856374696], [-73.90647377807485, 40.587020989415], [-73.90665451225826, 40.58700499090302], [-73.90683620925472, 40.586993586042155], [-73.90708063256595, 40.58695645187199], [-73.9073476580153, 40.58691770836381], [-73.90746679470584, 40.58690073278497], [-73.90761945884292, 40.58688140981628], [-73.90780535419943, 40.58685414075541], [-73.90795644961685, 40.586828728060624], [-73.90810885035972, 40.58679650530153], [-73.90830027751733, 40.58674515750672], [-73.90846615145995, 40.586697394105244], [-73.90864547838638, 40.586638956636016], [-73.90889649531321, 40.58653738414822], [-73.90900182081849, 40.58649922817851], [-73.90910459453215, 40.58646202467639], [-73.90931521062699, 40.58638768338272], [-73.9095133439717, 40.586321335257146], [-73.90964850230635, 40.58628011260677], [-73.9098759422859, 40.58622381688529], [-73.91002610774325, 40.58618044593877], [-73.91021202252327, 40.58613376555101], [-73.91037404988951, 40.58609477476435], [-73.910444885584, 40.5860777299313], [-73.91058545398192, 40.586039195892155], [-73.91069745365972, 40.58600245868274], [-73.91078707408076, 40.585975354188705], [-73.91088554171192, 40.5859476450044], [-73.91097564635562, 40.585948278232635], [-73.91106530706425, 40.58595511825114], [-73.91115378975734, 40.58596810905844], [-73.9112403699983, 40.58598714429473], [-73.91132433892447, 40.586012068112495], [-73.91140500905156, 40.58604267645197], [-73.91148171990072, 40.586078718712315], [-73.91155384340597, 40.58611989980268], [-73.9116207890572, 40.58616588255845], [-73.91168200873301, 40.586216290501035], [-73.9117278225857, 40.58626100614427], [-73.911783401795, 40.58631186553526], [-73.91183849093947, 40.58634949623099], [-73.91191811783656, 40.58642796543401], [-73.91197278431736, 40.586697357471316], [-73.91196717937, 40.58673559260821], [-73.91195662718775, 40.58679420692518], [-73.91193704810675, 40.58685672345449], [-73.91191493562087, 40.586909665238714], [-73.91189636497208, 40.586962125574345], [-73.91188936124115, 40.587019030956576], [-73.91188736476137, 40.58703423891995], [-73.9118824889411, 40.58704526766378], [-73.91187505429691, 40.58705285573592], [-73.9118731269851, 40.58705211428022], [-73.91186318039331, 40.58704521934867], [-73.9118532688722, 40.587029736549376], [-73.91184694361347, 40.58701510905806], [-73.91184339080382, 40.587006893871965], [-73.91183668831881, 40.58699387286532], [-73.91182673638394, 40.58698820491201], [-73.91180968955037, 40.586985954306115], [-73.91178779388788, 40.586988597698166], [-73.91175461101325, 40.58699612042423], [-73.91171623638176, 40.58701393601307], [-73.9116762087755, 40.58704156134442], [-73.9116606646332, 40.58706384990722], [-73.91163434544393, 40.587121933733705], [-73.91163069835724, 40.58714695094646], [-73.91162872325218, 40.58715725114704], [-73.91161866478295, 40.58717611945393], [-73.9115924344312, 40.58721334843776], [-73.91158315677313, 40.5872363191326], [-73.91158104928806, 40.58724153538241], [-73.9115767744986, 40.58726262488539], [-73.91157988212737, 40.58728839548044], [-73.91159461349682, 40.58730511703834], [-73.91160194900532, 40.58732083777991], [-73.91161178372174, 40.58735349532297], [-73.91164837700495, 40.58745050340242], [-73.91166871545042, 40.58749837298168], [-73.91167832660508, 40.58752099700941], [-73.91170855730851, 40.58760081383638], [-73.91173099495536, 40.58767108319772], [-73.91174031291423, 40.587700263761086], [-73.91177111790168, 40.58779627473992], [-73.9118037815627, 40.58790897675168], [-73.91182775574836, 40.58799251995675], [-73.911845001611, 40.58805261359304], [-73.91187205689816, 40.588122608702555], [-73.91188215862513, 40.58816827022503], [-73.9118919257537, 40.58821687665697], [-73.9119018954122, 40.58829320735019], [-73.91191138386733, 40.58840683122709], [-73.91191080135235, 40.58846743387369], [-73.91190444550038, 40.58852336036854], [-73.91188405404365, 40.58862488698604], [-73.91186659394712, 40.58871832434471], [-73.91186181002288, 40.58878284093906], [-73.91186370033844, 40.58879217092881], [-73.91186785886741, 40.588798069980626], [-73.91187556560362, 40.58880225893319], [-73.91187843365023, 40.58880889144386], [-73.9118728889092, 40.5888258078285], [-73.91186737695497, 40.588835362661015], [-73.91185931058577, 40.58884024933746], [-73.91185216172038, 40.588855933652034], [-73.91185049421847, 40.588869424185546], [-73.91185621380923, 40.58888636897875], [-73.91185779782703, 40.58889250696331], [-73.91185520460895, 40.58889642566458], [-73.91184811016666, 40.588899843015334], [-73.91184394026183, 40.58889639842734], [-73.9118407333424, 40.588893936132585], [-73.9118134253641, 40.58895717009084], [-73.91179582600458, 40.589008160572924], [-73.91175419016416, 40.589110125360335], [-73.91172949088498, 40.58916575998968], [-73.91170191749868, 40.58921599008427], [-73.91169119529175, 40.589239516876425], [-73.91168855492587, 40.589254476227346], [-73.91169217786889, 40.58926689813333], [-73.91169299451158, 40.58926969951739], [-73.91170456851212, 40.589272918777525], [-73.91171803818274, 40.58928423937156], [-73.91172344502144, 40.58929922082089], [-73.91172131485254, 40.58930792401163], [-73.9117191842418, 40.589316628877455], [-73.91171142607774, 40.58932470637737], [-73.91169916292542, 40.58933228134486], [-73.91169295508318, 40.58935385787823], [-73.91169740685693, 40.589440970887765], [-73.91170091348322, 40.589523420827035], [-73.91170276783197, 40.589541336504844], [-73.91170915177597, 40.5895536199743], [-73.91171909424732, 40.58956174122943], [-73.91172902556457, 40.589572317450646], [-73.91173189429051, 40.58957894895986], [-73.91173250930532, 40.5895853299567], [-73.91172411356864, 40.589591933546636], [-73.9117192808355, 40.58959314793991], [-73.91170314503307, 40.589602922441266], [-73.91169762851035, 40.589613704749304], [-73.91169302433782, 40.58963651100751], [-73.91167493064509, 40.58972724793806], [-73.91167636550992, 40.58976773486286], [-73.9116801316113, 40.58979007277551], [-73.9116871419636, 40.5898062834486], [-73.91169737043431, 40.58982250262556], [-73.91170952314022, 40.58984068973997], [-73.91171911495387, 40.589855435674934], [-73.91172134034741, 40.58986182058934], [-73.91171584839755, 40.58986646882846], [-73.91171231150811, 40.58986572345317], [-73.9116955715619, 40.58986666249181], [-73.91168717578466, 40.58987326708404], [-73.911688730822, 40.589885538633965], [-73.9116960606178, 40.58990248684886], [-73.91170756485637, 40.589921653846886], [-73.91171620703926, 40.58993296134928], [-73.9117219311665, 40.58994867866315], [-73.91171803975364, 40.58995578361016], [-73.91170351762429, 40.589964334543616], [-73.91168189842355, 40.589977530301326], [-73.91167767127622, 40.589987579652835], [-73.91168470756043, 40.58999765692761], [-73.9116988350344, 40.59000554417884], [-73.91169946208502, 40.59000947138194], [-73.91169362007794, 40.590020743630305], [-73.91167938675306, 40.59003739097523], [-73.91166124503799, 40.59006482604859], [-73.91164502688518, 40.59009422855554], [-73.91161981427781, 40.59011894638711], [-73.91161168877787, 40.59013732866452], [-73.91159973505589, 40.590148094456126], [-73.91159136405783, 40.590148563966714], [-73.91158038765167, 40.59015663244755], [-73.91157259435383, 40.59017207081651], [-73.9115701795048, 40.590209850016485], [-73.91157817747674, 40.59022091059742], [-73.91158392479421, 40.59023172133506], [-73.91159414412351, 40.59025039381368], [-73.91160082544711, 40.590268323089724], [-73.91160104928282, 40.590291140763306], [-73.91159440572522, 40.590338969310366], [-73.9115798597866, 40.59042799780452], [-73.91158031302237, 40.59047240851565], [-73.91158480689317, 40.59051074202458], [-73.91159162634092, 40.590535739806406], [-73.91159189283387, 40.5905487443116], [-73.91157925684546, 40.59056785052824], [-73.9115714665765, 40.59058328839631], [-73.91157038299714, 40.59058590118818], [-73.911563334329, 40.59060289647682], [-73.91156200782149, 40.590612216790255], [-73.91157093448166, 40.59063162339421], [-73.91157219974197, 40.59063702283452], [-73.91157085091264, 40.59065125140628], [-73.91156242154459, 40.5906652148584], [-73.91155657497599, 40.59067771341064], [-73.91155361662597, 40.59069193689023], [-73.91155706528343, 40.59071353676128], [-73.91156021933259, 40.590728266695926], [-73.91155792085337, 40.59073905700149], [-73.91154559397185, 40.590761353378795], [-73.91153745141267, 40.59078341474937], [-73.91153412800526, 40.59080794301396], [-73.91153153170995, 40.590887432696064], [-73.91152817634813, 40.59091932083589], [-73.91149952765117, 40.59105662393773], [-73.91148579005616, 40.59110875301849], [-73.91149028340267, 40.59113603327592], [-73.91149942867085, 40.591159000554384], [-73.91150421772528, 40.59118701984338], [-73.91150127545596, 40.591215615677534], [-73.91149998055923, 40.59124302097649], [-73.91150304364807, 40.5912653725903], [-73.91151015669493, 40.591291236895614], [-73.91155194294743, 40.59137218859723], [-73.91156823019618, 40.59139309941536], [-73.91159055475737, 40.59141584681003], [-73.9116014447898, 40.5914258682505], [-73.91160429919019, 40.591445032454594], [-73.9116006977331, 40.59145597923551], [-73.91159418044754, 40.591467601764364], [-73.91159070977163, 40.59147487774218], [-73.91159103013511, 40.59149300902121], [-73.91159882997333, 40.591508600613395], [-73.91164618788592, 40.59159554506943], [-73.91165439035092, 40.59160848212956], [-73.91166900773801, 40.59163153049634], [-73.91168364528845, 40.59165363082819], [-73.9116970064868, 40.59168452128027], [-73.9117037400627, 40.5917120924049], [-73.91170979527712, 40.59174062814445], [-73.91170706136631, 40.59177241306373], [-73.91170750309422, 40.59179617894986], [-73.91171221410116, 40.59181734119196], [-73.91172068205198, 40.59184127189481], [-73.91172408255711, 40.591863141778646], [-73.91172153092963, 40.591880723029156], [-73.91171505699691, 40.59189112196328], [-73.91169903912827, 40.59189887413861], [-73.91168026066258, 40.59192101898199], [-73.91168012436576, 40.59192171613444], [-73.911677119813, 40.59193712032686], [-73.91167975585125, 40.59196240202967], [-73.9116843620899, 40.59200462497692], [-73.91168840929198, 40.59202255707856], [-73.9116899148627, 40.59202923143222], [-73.911701055142, 40.59204121716916], [-73.91171148620498, 40.592046085713186], [-73.91172195119445, 40.592059038243576], [-73.911721688759, 40.592066380350715], [-73.91172181055562, 40.592077648970985], [-73.91172811225438, 40.59212141365091], [-73.91173517224131, 40.59216510972304], [-73.91174298986893, 40.5922087294806], [-73.9117658107778, 40.59226806328435], [-73.9117685787143, 40.59227804881847], [-73.91177208348462, 40.59228789871696], [-73.91177631875895, 40.59229758013915], [-73.91178126787422, 40.59230706392187], [-73.91178691570899, 40.592316318222956], [-73.91179324384265, 40.592325312873065], [-73.91179735715085, 40.59232983781491], [-73.91180212323454, 40.59233397626503], [-73.91180747755769, 40.592337671045954], [-73.91181335161605, 40.59234087486194], [-73.91181966657005, 40.592343545435114], [-73.91182634159016, 40.592345648694746], [-73.91183421361004, 40.592347467402426], [-73.91184229063613, 40.59234864730989], [-73.91185048565515, 40.59234917729345], [-73.91185870813305, 40.592349050749974], [-73.9118627564853, 40.59234934585878], [-73.911866645357, 40.59235025467266], [-73.91187021744544, 40.59235173954454], [-73.91187332558673, 40.59235374072081], [-73.91187584440306, 40.592356176183856], [-73.91187766985718, 40.59235894600619], [-73.91187754125806, 40.592363636226565], [-73.9118766600256, 40.592368278124056], [-73.9118750381175, 40.592372802517914], [-73.91187270133815, 40.59237714023845], [-73.91186968361524, 40.59238122765036], [-73.91186625423705, 40.592389503315246], [-73.91186366808383, 40.592397957040276], [-73.91186193995095, 40.59240653539507], [-73.91186108023207, 40.592415189636725], [-73.91186109470812, 40.592423869343115], [-73.91186198256916, 40.592432522077075], [-73.91186508779114, 40.59244189956843], [-73.91186904133893, 40.59245108890479], [-73.91187382854677, 40.59246004618197], [-73.91187942178375, 40.592468726313456], [-73.91188579473291, 40.59247708789912], [-73.9118920142423, 40.59248481711112], [-73.91189741400538, 40.592492894825476], [-73.91190196199851, 40.59250127377429], [-73.91190562686765, 40.592509898649034], [-73.91190838846245, 40.59251871750037], [-73.91192559639886, 40.59257584948509], [-73.91192892700222, 40.59258881294534], [-73.91193145554234, 40.59260188016145], [-73.91193317875779, 40.59261502415898], [-73.91193145844181, 40.59262700939735], [-73.91192893169222, 40.59263891109059], [-73.91192560206756, 40.59265069690819], [-73.91192148125323, 40.59266233770897], [-73.91191657786295, 40.59267380066374], [-73.91191090622381, 40.59268505378502], [-73.91190512494188, 40.592699454698426], [-73.91190010295739, 40.592714018026946], [-73.91189584754873, 40.59272872417544], [-73.91189236578171, 40.59274354835491], [-73.91188966274626, 40.59275846426741], [-73.9118877426437, 40.5927734526502], [-73.91188660880523, 40.59278848787402], [-73.91188620080706, 40.59287933989359], [-73.9118887680066, 40.59288975900381], [-73.91189206631022, 40.59290005939369], [-73.91189608499211, 40.59291020888914], [-73.91190081178132, 40.59292018000604], [-73.91190623375056, 40.59292994358427], [-73.91191233423666, 40.59293947046083], [-73.91194884120097, 40.592970926943565], [-73.91195445566146, 40.59297721795141], [-73.91195933586212, 40.59298385534954], [-73.91196344868533, 40.59299078801645], [-73.91196675991245, 40.59299796600213], [-73.91196924544082, 40.59300533500876], [-73.9119707313167, 40.59301065000996], [-73.9119730246225, 40.59301579609012], [-73.91197609314611, 40.59302070235942], [-73.9119798958833, 40.59302529892666], [-73.91198437819286, 40.59302952443413], [-73.91198948257448, 40.593033318862425], [-73.91199681783436, 40.59304093801839], [-73.91200328706624, 40.59304900012829], [-73.91200884617297, 40.593057447361026], [-73.91201345501351, 40.593066221050755], [-73.91201735581953, 40.5930734505828], [-73.91202048561775, 40.59308089412947], [-73.91202282754696, 40.593088505607476], [-73.91202436365306, 40.5930962339064], [-73.91202508652567, 40.59310403194514], [-73.91202498832092, 40.593111847951036], [-73.91202495141847, 40.59312718037484], [-73.91202384383816, 40.59314248785471], [-73.91202166563248, 40.59315772968092], [-73.91201842675004, 40.59317286096334], [-73.91201166013033, 40.593178996402514], [-73.91200554044276, 40.593185514471], [-73.9120001029016, 40.593192376161646], [-73.91199538031132, 40.59319953710441], [-73.91199140041407, 40.59320695811859], [-73.91198819074486, 40.593214591144715], [-73.91198576718821, 40.593222389956715], [-73.91198414452755, 40.59323030966812], [-73.91198455038396, 40.59323818871593], [-73.91198574768993, 40.593246019953924], [-73.91198772727473, 40.59325375847721], [-73.911990481066, 40.593261359884295], [-73.91199399066633, 40.59326877624766], [-73.91199823503203, 40.5932759665064], [-73.91202925752236, 40.593315543432084], [-73.91203370135146, 40.59332497354185], [-73.91203724472012, 40.593334622593055], [-73.91203986813295, 40.59334444165227], [-73.91204155737292, 40.59335437910986], [-73.91204230481588, 40.593364382858205], [-73.91204210635345, 40.59337440179784], [-73.91203694234129, 40.59339082761376], [-73.91203260974942, 40.59340739244511], [-73.91202911454454, 40.59342407217239], [-73.91203036412719, 40.593430430029336], [-73.91203088861565, 40.59343684563139], [-73.91203068499284, 40.59344327324081], [-73.9120297533253, 40.593449662431134], [-73.91202809960842, 40.59345596596393], [-73.91202361395224, 40.59346359801385], [-73.91201981347797, 40.59347144083771], [-73.91201671339157, 40.593479462616266], [-73.91201432846694, 40.59348762600208], [-73.91201267282268, 40.593495890966494], [-73.91201193232598, 40.59351297403493], [-73.91201202628058, 40.59353006712406], [-73.91201295164508, 40.59354714376174], [-73.9120147064696, 40.593564183172916], [-73.9120126440893, 40.5935832442531], [-73.91201151347225, 40.59360234993945], [-73.91201131619253, 40.59362147443348], [-73.91201205074663, 40.59364059143211], [-73.91201371717094, 40.59365967413067], [-73.91201741606962, 40.5936715536191], [-73.91202187051888, 40.593683281736844], [-73.91202706693066, 40.59369482982578], [-73.91203299391199, 40.59370617140749], [-73.91203963699338, 40.59371728033616], [-73.91206899130812, 40.593765967557175], [-73.91207199728977, 40.593785689063566], [-73.91207382123704, 40.59380549477043], [-73.91207445836996, 40.59382534228926], [-73.91207504734324, 40.59384170784155], [-73.91207482931073, 40.59385807981272], [-73.91207380540216, 40.59387443424664], [-73.91207197762718, 40.593890746853425], [-73.91206934645322, 40.593906995686694], [-73.91205513595628, 40.593951807951605], [-73.91205274438181, 40.59396317566611], [-73.91204959056493, 40.593974434070525], [-73.9120456848716, 40.59398555519549], [-73.91204103569373, 40.59399650771939], [-73.91203565427318, 40.59400726601856], [-73.91203056523912, 40.59404097538782], [-73.91202433770972, 40.594074574656005], [-73.91201697918692, 40.59410804020712], [-73.9120152168351, 40.59411688373552], [-73.91201457177262, 40.59412274676776], [-73.91201423721664, 40.59412579755542], [-73.91201404829762, 40.594134739958214], [-73.9120146479311, 40.59414367308048], [-73.91201603617307, 40.5941525547049], [-73.91201820450559, 40.59416134461792], [-73.91202104425462, 40.59419230872268], [-73.91202116543411, 40.59419658081293], [-73.91202054052064, 40.59420082871042], [-73.9120191775284, 40.594204974520096], [-73.91201710402474, 40.59420894522009], [-73.9120143561458, 40.5942126699733], [-73.91200977299259, 40.59421653119777], [-73.9120058383051, 40.5942207869476], [-73.9120026088656, 40.594225375448026], [-73.91200013311465, 40.594230228049106], [-73.91199844762731, 40.594235274416576], [-73.91199467115827, 40.59424687879309], [-73.91199173403426, 40.594258621521305], [-73.91198964223281, 40.5942704697703], [-73.91198840283423, 40.594282387861696], [-73.91198802138084, 40.59429433944581], [-73.91198849462178, 40.59430629017682], [-73.91198982634339, 40.59431820253105], [-73.91199201065984, 40.594330040317345], [-73.911995037946, 40.59434177002283], [-73.91199890341385, 40.594353356127385], [-73.91201894144422, 40.594393994481024], [-73.91202439676188, 40.5944140807314], [-73.912029141862, 40.59443427013824], [-73.9120331732451, 40.59445454963199], [-73.91203648719129, 40.59447490597495], [-73.9120390802086, 40.59449532056859], [-73.91204234015626, 40.594506954288846], [-73.912046385103, 40.59451844436708], [-73.91205120454023, 40.59452975996983], [-73.91205678620321, 40.594540869256996], [-73.91206311298801, 40.59455174340022], [-73.91207017020878, 40.5945623530707], [-73.91207793921957, 40.594572671784384], [-73.91208876235244, 40.594580369325], [-73.91210029925591, 40.59458744051589], [-73.91211248448944, 40.594593845769886], [-73.91212525458131, 40.59459955270503], [-73.91213033657822, 40.59460241782418], [-73.91213482999255, 40.5946058076975], [-73.91213864261131, 40.59460965222715], [-73.91214169190117, 40.59461387445433], [-73.91214392062031, 40.59461838420445], [-73.91214527987934, 40.594623089801686], [-73.91217622268415, 40.59466848558399], [-73.91218511716173, 40.59468283121475], [-73.91219324868815, 40.59469743643484], [-73.91220060718297, 40.59471227878772], [-73.91220717597442, 40.5947273346393], [-73.91221294959934, 40.59474257935856], [-73.91221791556377, 40.594757987974425], [-73.91222206642739, 40.594773535519415], [-73.91222953960647, 40.594796945071586], [-73.91223617877866, 40.5948204994016], [-73.91224198154815, 40.59484418158753], [-73.9122469415631, 40.59486797453655], [-73.91225105686692, 40.59489186132671], [-73.91225432088841, 40.594915824362694], [-73.91225708819063, 40.59492743525123], [-73.9122605681491, 40.59493893376909], [-73.91226475530168, 40.59495029595531], [-73.91226963979554, 40.59496149348982], [-73.91227521023197, 40.59497250441751], [-73.9122814541188, 40.594983302259244], [-73.91228835808013, 40.59499386405295], [-73.91229590763923, 40.595004168343955], [-73.91230408481061, 40.59501418814588], [-73.91231287313701, 40.595023903845416], [-73.9123268081393, 40.59503596620071], [-73.91233993149896, 40.59504854643835], [-73.9123522089721, 40.595061613371996], [-73.91236361225184, 40.59507513297205], [-73.91237274394311, 40.59508547247283], [-73.91238112616925, 40.595096174772486], [-73.91238873303661, 40.59510720919352], [-73.91239553887561, 40.5951185422108], [-73.91240152834456, 40.59513014080939], [-73.91240732222434, 40.595145921795016], [-73.91241193979788, 40.59516192637302], [-73.91241536508893, 40.59517810293238], [-73.91241665059879, 40.59518259168705], [-73.91241721205836, 40.595187165497244], [-73.91241704471923, 40.59519175852045], [-73.91241614779253, 40.59519630156652], [-73.91241453674903, 40.595200728808216], [-73.91241223475028, 40.59520497492655], [-73.91240927462518, 40.59520897729063], [-73.91240524323416, 40.595213879810615], [-73.91240188485837, 40.595219068145894], [-73.9123992347286, 40.595224491562156], [-73.91239732039394, 40.595230091109734], [-73.91239616280818, 40.59523580984511], [-73.9123957728195, 40.59524158729829], [-73.91239615270095, 40.59524736651102], [-73.91239821377802, 40.59525505298608], [-73.91240121133356, 40.59526255957853], [-73.91240511732963, 40.59526981590438], [-73.91240990085376, 40.59527676397495], [-73.91241551452005, 40.59528333875267], [-73.91242496499889, 40.59529764139762], [-73.91243521413207, 40.595311619140794], [-73.91244624568724, 40.595325248348175], [-73.91245633809108, 40.59534109911516], [-73.91246724474364, 40.595356633033454], [-73.91247895249303, 40.59537182395806], [-73.91248969835243, 40.59538378133634], [-73.91249966064815, 40.59539612527704], [-73.91250882029765, 40.59540882695103], [-73.91251715052985, 40.59542185450746], [-73.91252463292044, 40.59543517911728], [-73.9125312514718, 40.59544876592269], [-73.91253698820049, 40.595462584922174], [-73.91254065272486, 40.595476022387885], [-73.91254508158896, 40.59548932440134], [-73.91255026383445, 40.59550246817019], [-73.91255619048279, 40.59551542922814], [-73.91256284970204, 40.595528180593945], [-73.91257022789713, 40.59554069863545], [-73.91257834444755, 40.59555312174647], [-73.91258726075219, 40.59556521995584], [-73.91259695465172, 40.59557696459904], [-73.91260740442424, 40.595588328352015], [-73.9126185828551, 40.59559928254628], [-73.91268220126986, 40.59565996410998], [-73.91270381891817, 40.59567274660089], [-73.91272477246828, 40.59568615715323], [-73.91274502546243, 40.59570017647381], [-73.91276455309242, 40.595714784273206], [-73.91277985941181, 40.59572641522043], [-73.91279450379669, 40.59573852982305], [-73.91280846012137, 40.595751106617506], [-73.91282170555613, 40.595764124813265], [-73.91282654397065, 40.59577167953136], [-73.91283219292015, 40.59577889980546], [-73.91283861291132, 40.59578573534673], [-73.9128457585103, 40.59579214005007], [-73.91286098688525, 40.59580289474907], [-73.91287570551383, 40.59581405414548], [-73.91288989639123, 40.5958256051585], [-73.91289538757611, 40.59583061693884], [-73.91290021343968, 40.595836009678685], [-73.91290432766792, 40.5958417374402], [-73.91290769231603, 40.59584774172696], [-73.91291115854254, 40.59585101888135], [-73.91291520859951, 40.59585388050289], [-73.91291975751173, 40.595856266886514], [-73.91292471215944, 40.59585812720044], [-73.91292996621935, 40.59585942400496], [-73.91293819362873, 40.5958630929312], [-73.91294589167049, 40.59586737896887], [-73.91295297974408, 40.59587223766164], [-73.91295938648818, 40.595877618529144], [-73.91296504296281, 40.595883467575135], [-73.91298785020784, 40.59590313770714], [-73.91300999866758, 40.59592324190676], [-73.91303147033749, 40.59594376675855], [-73.9130354003512, 40.595946502470404], [-73.91303874499019, 40.59594965823763], [-73.91304143226364, 40.59595316046053], [-73.91304339787871, 40.595956930519236], [-73.91304459975052, 40.595960882271754], [-73.91304500854287, 40.595964927910714], [-73.91304461800443, 40.59596897344744], [-73.91304385180312, 40.59597711780641], [-73.91304397224778, 40.59599466424475], [-73.9130461086363, 40.59600281850639], [-73.91304898958688, 40.5960108388054], [-73.91305260328602, 40.596018683417874], [-73.91305692802226, 40.596026316643616], [-73.9130619464842, 40.59603369960276], [-73.91306763102718, 40.59604079592028], [-73.91307395466518, 40.5960475698921], [-73.91308088732873, 40.5960539908373], [-73.9130883928005, 40.59606002354743], [-73.9130973506202, 40.5960648320462], [-73.9131058559836, 40.59607009755708], [-73.91311386562552, 40.596075795755745], [-73.91312134243871, 40.596081899809555], [-73.91312825261531, 40.596088380542874], [-73.91313721910822, 40.596098015563776], [-73.91314515659491, 40.596108163618496], [-73.9131520161545, 40.59611875598327], [-73.91315586956061, 40.59612987932162], [-73.91316073754624, 40.596140770393326], [-73.91316659227054, 40.59615137473005], [-73.91317047431652, 40.59615619530302], [-73.91317505024338, 40.59616064649523], [-73.91318026188453, 40.596164670968], [-73.91318604095544, 40.59616821774061], [-73.91319231257388, 40.596171239010545], [-73.91319550464623, 40.596174947504515], [-73.91319800271661, 40.59617895015827], [-73.91319976094164, 40.59618317741242], [-73.91320074910033, 40.59618754564708], [-73.9132009442157, 40.59619197811666], [-73.91320034508173, 40.59619638803996], [-73.91319873952403, 40.5962065915384], [-73.9131982336504, 40.596216861538814], [-73.91319883018978, 40.59622712751362], [-73.91320294366942, 40.59623668196742], [-73.91320776245195, 40.59624604211734], [-73.91321326987233, 40.59625517880027], [-73.9132194503669, 40.596264061179326], [-73.91322695723878, 40.596271412001684], [-73.91323516183053, 40.59627831587961], [-73.91324402023648, 40.5962847344157], [-73.91325348063961, 40.59629062853625], [-73.91325944330885, 40.596296998814594], [-73.91326472422526, 40.59630370782493], [-73.91326928805553, 40.59631071650626], [-73.91327310716622, 40.59631797943744], [-73.91327615678135, 40.59632545086422], [-73.91328697424066, 40.59636932037021], [-73.91328916240832, 40.59637689819784], [-73.9132904963672, 40.59638459013845], [-73.91329096739976, 40.596392339728126], [-73.91329057205931, 40.596400092852356], [-73.91329069144659, 40.59641782708908], [-73.91329178469968, 40.59643554229221], [-73.91329384966288, 40.59645320662939], [-73.91329346814814, 40.59646029835845], [-73.91329410422776, 40.59646737711784], [-73.91329574855533, 40.59647436282157], [-73.91329838432034, 40.59648116917924], [-73.91330197888098, 40.59648771541724], [-73.91330730701902, 40.59649412475108], [-73.91331323182914, 40.5965002182397], [-73.91331972741393, 40.596505968053215], [-73.91332675579794, 40.59651133831195], [-73.91333427965073, 40.596516304360705], [-73.91334225834795, 40.596520839364004], [-73.91335065170345, 40.59652491732447], [-73.91337377983909, 40.596543817823395], [-73.91338608350591, 40.59656044080802], [-73.9133976309511, 40.59657737683623], [-73.91340840923466, 40.596594604622254], [-73.9134184062952, 40.596612103718705], [-73.91342761073062, 40.596629853008274], [-73.91343023115952, 40.59663629815819], [-73.91343210282828, 40.596642890337314], [-73.91343321041076, 40.59664958463628], [-73.91343354606566, 40.59665632609949], [-73.9134331081, 40.59666306429952], [-73.9134318974596, 40.59666974747053], [-73.9134299249803, 40.59667632418901], [-73.91342789757822, 40.59668644327255], [-73.91342688038962, 40.59669665039993], [-73.91342688228153, 40.596706887445116], [-73.91342790179394, 40.59671709443138], [-73.91342993174986, 40.59672721288587], [-73.91343570243254, 40.59674066832563], [-73.9134423569443, 40.59675388402633], [-73.91344987598562, 40.596766828645336], [-73.91345823828297, 40.5967794676552], [-73.91346273839629, 40.59678684132681], [-73.91346795271964, 40.596793935091924], [-73.91347385295029, 40.596800711067736], [-73.91348040397632, 40.596807128853015], [-73.91348757199944, 40.59681315139831], [-73.91349531310574, 40.596818746169596], [-73.9135160129812, 40.59684056920232], [-73.91353599720709, 40.59686277650728], [-73.91355525415447, 40.59688535266384], [-73.91355914647922, 40.59689009750761], [-73.91356372390055, 40.59689447178929], [-73.91356892495125, 40.59689842051282], [-73.91357468332745, 40.59690188985151], [-73.91358092215793, 40.59690483920523], [-73.91358756193775, 40.59690722612937], [-73.91359489178382, 40.5969145734435], [-73.9136015376347, 40.59692229081758], [-73.91360746723453, 40.596930338020556], [-73.91361265469588, 40.59693867901431], [-73.91361707127864, 40.596947274072846], [-73.91362069747747, 40.59695608046155], [-73.91362754399033, 40.596972976578456], [-73.91363513536471, 40.59698968662716], [-73.91364346371596, 40.59700619016292], [-73.91364815411255, 40.59701175850704], [-73.9136521076035, 40.59701765180668], [-73.91365528558478, 40.597023810225004], [-73.91365765692197, 40.59703017610849], [-73.91365920125445, 40.5970366879587], [-73.91365990349821, 40.597043283108306], [-73.91365975450121, 40.59704990073744], [-73.91365462264105, 40.597058207003464], [-73.91365035020185, 40.597066792851045], [-73.91364696318834, 40.597075603853234], [-73.91364447990745, 40.59708459009979], [-73.91364291845582, 40.597093693974514], [-73.91364393033042, 40.59709692151604], [-73.91364562396872, 40.59709997718234], [-73.91364795661879, 40.59710277985689], [-73.91365086288786, 40.59710525058437], [-73.91365426308154, 40.59710732279575], [-73.91365806430431, 40.597108940633944], [-73.91366216133986, 40.597110058619364], [-73.91366644565318, 40.597110648190466], [-73.91366677328989, 40.5971106923293], [-73.91371188344324, 40.59711490874536], [-73.91373985785499, 40.597120020793724], [-73.91376762576394, 40.59712574583744], [-73.91379516585833, 40.59713207900235], [-73.91382245374955, 40.597139015244394], [-73.91384946966531, 40.59714654918794], [-73.91385644798187, 40.59714771271643], [-73.91386322491958, 40.597149433295954], [-73.91386972380057, 40.597151691100436], [-73.91387587476702, 40.59715446078091], [-73.91388160467154, 40.597157711624554], [-73.91388684938416, 40.597161407899726], [-73.91389155335746, 40.59716550500207], [-73.91389566060403, 40.597169958996986], [-73.9138991250328, 40.59717472109913], [-73.91390157926351, 40.5971781835899], [-73.91390477485574, 40.59718127555866], [-73.91390862028274, 40.59718390245039], [-73.91391299827909, 40.59718598795134], [-73.91391778211195, 40.59718746931054], [-73.9139228280989, 40.59718830420283], [-73.91392798726093, 40.59718846671731], [-73.91394197787766, 40.59719035519822], [-73.91395581589491, 40.59719281115238], [-73.91396946373938, 40.597195827515314], [-73.91398287746675, 40.597199395375256], [-73.91399602346324, 40.597203504822886], [-73.9140159093215, 40.59721714268064], [-73.91403514020566, 40.59723131814898], [-73.91405368888863, 40.59724601110429], [-73.91407153341738, 40.59726120209714], [-73.91409231747883, 40.59727609607741], [-73.91411253415568, 40.59729143827323], [-73.91413216544208, 40.597307215771714], [-73.91415119772826, 40.597323415998154], [-73.91416961235151, 40.59734002369387], [-73.91418739482286, 40.597357026283746], [-73.91419763070647, 40.597369362404834], [-73.91420703005407, 40.59738207835948], [-73.914215567629, 40.59739514447633], [-73.91422322106098, 40.5974085233799], [-73.91422997039156, 40.597422182219574], [-73.91423579808338, 40.59743608563393], [-73.91424010990987, 40.59744162619628], [-73.9142437083946, 40.59744745387397], [-73.91424655909593, 40.59745351821497], [-73.91424863681233, 40.59745976207313], [-73.9142499220557, 40.5974661286413], [-73.91425040281214, 40.597472560113324], [-73.91425007476137, 40.597478997515346], [-73.91424555953537, 40.597487736325284], [-73.9142403740324, 40.597496256679435], [-73.9142345349943, 40.59750452843461], [-73.91422806465675, 40.59751252195489], [-73.91422728518754, 40.597515524167655], [-73.91422729136438, 40.59751858576936], [-73.91422808180374, 40.59752158697586], [-73.91422962654339, 40.597524412001945], [-73.91423186594204, 40.597526952579294], [-73.91423471572932, 40.597529110472614], [-73.91423806326388, 40.59753080216825], [-73.91424177939865, 40.59753196156355], [-73.91424572089046, 40.59753254532877], [-73.9142649680164, 40.59753520732194], [-73.91428433609785, 40.59753729846243], [-73.91430379568328, 40.59753881872816], [-73.91431230076894, 40.597539554157485], [-73.91432066037476, 40.59754096244276], [-73.91432877979503, 40.597543027095554], [-73.91433657378231, 40.59754572660846], [-73.91434395226241, 40.59754903243475], [-73.9143508416565, 40.597552907160384], [-73.9143589072927, 40.59755937996822], [-73.9143662694607, 40.59756632434871], [-73.91437287920782, 40.59757369402756], [-73.9143786948325, 40.59758144424332], [-73.91438367903727, 40.59758952370446], [-73.9143878000206, 40.597597880285896], [-73.91439103037744, 40.59760646069284], [-73.9143930041245, 40.597610156685874], [-73.91439420905029, 40.59761404040906], [-73.91439461648184, 40.59761801902976], [-73.9143942168692, 40.59762199872459], [-73.91439301780787, 40.59762588333749], [-73.91439105128462, 40.597629580750926], [-73.91438836334468, 40.597633004219254], [-73.91438501497187, 40.597636073206026], [-73.91437892135427, 40.597640119523504], [-73.91437342274712, 40.597644634360826], [-73.91436858185395, 40.597649567003245], [-73.91436445039524, 40.597654861367154], [-73.91436107635855, 40.597660459355886], [-73.91435849718793, 40.59766629783877], [-73.91435674220051, 40.597672310328505], [-73.91435582906274, 40.597678431167004], [-73.91435680387828, 40.597686081614334], [-73.91435872213594, 40.597693629064025], [-73.91436156897954, 40.59770100381298], [-73.91436531702114, 40.59770814033665], [-73.91436993469345, 40.59771497495033], [-73.91437825854496, 40.59772234490082], [-73.91438579166037, 40.59773019624309], [-73.91439248443643, 40.597738475666425], [-73.9143982969349, 40.59774713438988], [-73.91440319054587, 40.597756116262644], [-73.91440668436618, 40.597762691037566], [-73.91440943330197, 40.597769467800504], [-73.91441142005344, 40.59777639711732], [-73.9144126281986, 40.59778343072762], [-73.91441304901245, 40.59779051652331], [-73.91440975037757, 40.59779335653535], [-73.91440718683135, 40.59779660720647], [-73.91440544444663, 40.59780016154966], [-73.91440457964612, 40.59780389597014], [-73.91440462445449, 40.597807689368366], [-73.91440557464115, 40.59781141257617], [-73.91440740025686, 40.59781494042653], [-73.91441564958077, 40.59782302566855], [-73.91442482452722, 40.59783050933165], [-73.9144348458325, 40.597837330543335], [-73.91444164298508, 40.597840836797886], [-73.91444792848162, 40.597844857655964], [-73.91445363534353, 40.59784935018052], [-73.91445870780481, 40.59785426825937], [-73.91446308483542, 40.59785955740039], [-73.91446411807439, 40.597870338174175], [-73.91446404312785, 40.59788114710573], [-73.91446286403466, 40.59789191902902], [-73.91446495001337, 40.597900661085205], [-73.91446774120679, 40.597909287903505], [-73.9144712319452, 40.59791776530377], [-73.91447540490731, 40.597926060437175], [-73.91448024628512, 40.59793414364075], [-73.91448573502264, 40.597941981392914], [-73.91449185269425, 40.597949545702434], [-73.91449857362512, 40.59795680522224], [-73.91450293005995, 40.59796239522872], [-73.91450800213941, 40.59796762122386], [-73.91451374091785, 40.597972431069415], [-73.91452008601821, 40.597976775299365], [-73.91452697332055, 40.59798060947064], [-73.91453433232283, 40.59798389550154], [-73.91454207391614, 40.597983617464045], [-73.91454978714879, 40.59798420452611], [-73.91455730478492, 40.59798564182066], [-73.9145602378312, 40.59799146078435], [-73.91456210042733, 40.59799753091625], [-73.91456285733523, 40.598003736929805], [-73.91456249507513, 40.598009964057766], [-73.91456305954144, 40.598016787441125], [-73.91456436959103, 40.59802355224103], [-73.9145664136381, 40.59803020986512], [-73.91456917723676, 40.59803671322722], [-73.9145726426407, 40.59804301892369], [-73.91458404111373, 40.598058811667], [-73.91459614131674, 40.598074297514565], [-73.91460892986605, 40.59808945886551], [-73.91462239206152, 40.59810427594023], [-73.91463651165726, 40.59811873448663], [-73.91465127197414, 40.59813281589602], [-73.91466665632866, 40.59814650407323], [-73.91468264716022, 40.59815978208439], [-73.91468682814674, 40.59816354940297], [-73.91469159870707, 40.59816688627378], [-73.91469688616588, 40.598169734845285], [-73.91470260090514, 40.59817205300098], [-73.91470865484172, 40.59817380097122], [-73.91471495064236, 40.598174954056546], [-73.91472138415484, 40.598175491908876], [-73.91472785736643, 40.59817540557637], [-73.91473426565655, 40.59817469732672], [-73.91474393078973, 40.59817582593815], [-73.91475345299705, 40.59817752270195], [-73.91476277163233, 40.59817977651638], [-73.91477183528372, 40.59818257360564], [-73.91478058572818, 40.59818589851413], [-73.91478897244201, 40.59818973043085], [-73.91479694380537, 40.5981940461986], [-73.91480445347518, 40.59819882115611], [-73.91480840413203, 40.598204142468305], [-73.91481312012188, 40.59820908656918], [-73.91481854174168, 40.598213592601326], [-73.91482459719681, 40.59821760120608], [-73.91483120919096, 40.598221059554305], [-73.91483829359821, 40.598223926539], [-73.91484575771953, 40.59822616305701], [-73.91484986099398, 40.59822590491933], [-73.91485395050435, 40.5982262718228], [-73.91485786186767, 40.59822725074609], [-73.9148614415051, 40.59822880203846], [-73.91486454641684, 40.598230862938124], [-73.9148670538468, 40.59823335210232], [-73.91486886237944, 40.59823617128399], [-73.91486876431405, 40.5982475807905], [-73.91486766802393, 40.59825895990486], [-73.91486557862625, 40.59827025920975], [-73.91486389752937, 40.59827660614996], [-73.91486296908423, 40.59828304093063], [-73.9148627986268, 40.5982895158096], [-73.91486339040608, 40.59829597366265], [-73.91486473898978, 40.59830236573483], [-73.91486683411426, 40.59830864025189], [-73.91486965518267, 40.59831474777747], [-73.91487318314006, 40.598320636865864], [-73.91487739057096, 40.5983262619288], [-73.91488571429966, 40.598336073580455], [-73.91489473543446, 40.59834552070194], [-73.91490442500042, 40.59835457479133], [-73.91491475600024, 40.59836320768375], [-73.91492344431688, 40.59837512544512], [-73.9149310281033, 40.598387475451446], [-73.91493746765627, 40.598400196357694], [-73.91494867331002, 40.59842098883165], [-73.9149588192007, 40.59844209782064], [-73.91496788713279, 40.598463485952465], [-73.91499162482152, 40.598514968496566], [-73.91499813627118, 40.59852561544044], [-73.91500554115406, 40.59853591609163], [-73.91501380787247, 40.59854583038715], [-73.9150229010989, 40.59855531273264], [-73.91503046214223, 40.598562421898194], [-73.91503744295963, 40.59856987038346], [-73.91504381238073, 40.59857762750775], [-73.9150495502191, 40.59858566628413], [-73.91505463343391, 40.598593957712914], [-73.91505904404208, 40.598602470955754], [-73.91506276560158, 40.59861117249449], [-73.91507760103967, 40.59863097569759], [-73.91509155225391, 40.598651149825145], [-73.91510460476944, 40.59867167024017], [-73.91511061052513, 40.59867955005853], [-73.91511584905167, 40.59868774108177], [-73.91512029051009, 40.59869620224337], [-73.91512391364171, 40.59870488561415], [-73.91512669959891, 40.5987137484606], [-73.91512862910307, 40.59872274134727], [-73.91512606406265, 40.598757598266225], [-73.9151251089278, 40.59876170219458], [-73.9151250150574, 40.598765868744856], [-73.91512578543986, 40.59876999472125], [-73.91512739866899, 40.598773975904685], [-73.91512981376925, 40.598777714427904], [-73.91513297459494, 40.59878111593029], [-73.91513679883606, 40.59878409407355], [-73.91514356433069, 40.598788057617604], [-73.91515075811886, 40.598791558761704], [-73.91515832155062, 40.59879457149558], [-73.91516620168738, 40.5987970718231], [-73.91517458370498, 40.5988001801492], [-73.91518251480994, 40.598803910177836], [-73.915189920761, 40.59880822550064], [-73.915196724909, 40.59881308166579], [-73.91520285851922, 40.598818432552214], [-73.91520826276223, 40.598824220318996], [-73.91521502464106, 40.598832768796264], [-73.91522241638398, 40.59884100814373], [-73.91523041274708, 40.59884891354834], [-73.91526129154352, 40.598863427767505], [-73.915265766048, 40.59886509664248], [-73.91526987290355, 40.59886724220175], [-73.9152735282043, 40.59886982283655], [-73.91527664872287, 40.59887278186068], [-73.91527917322075, 40.598876054395], [-73.91528104485967, 40.59887957238083], [-73.91528222417162, 40.598883261741335], [-73.91528527806723, 40.59889317217403], [-73.91528935512915, 40.59890286540518], [-73.91529442927784, 40.59891228211032], [-73.9153005489014, 40.59891619742323], [-73.91530715795862, 40.59891961905266], [-73.91531418704089, 40.59892251226914], [-73.91532156629518, 40.59892484586045], [-73.91533297981634, 40.59892781652164], [-73.91534470118378, 40.59892998092272], [-73.91535663173353, 40.59893132257308], [-73.9153623700253, 40.5989326420714], [-73.91536783815265, 40.59893451807008], [-73.91537293989077, 40.59893691749507], [-73.91537759243302, 40.59893979857063], [-73.91538171429588, 40.598943115669016], [-73.91539958417555, 40.5989656087354], [-73.91542612796813, 40.59901882299184], [-73.91544471569526, 40.599042194935045], [-73.9155344739184, 40.59911079798628], [-73.91556585249903, 40.59912773010479], [-73.91560394477511, 40.599161847937815], [-73.91563449423268, 40.59918009587316], [-73.91567311869402, 40.599190883785916], [-73.91573129686093, 40.59920270108504], [-73.91578387793759, 40.59922618082099], [-73.91580033757546, 40.59925117048404], [-73.91580410672464, 40.59925779433561], [-73.91581805281605, 40.599282309554646], [-73.91585358956556, 40.59931384531452], [-73.91588501406622, 40.5993243001967], [-73.91591433020129, 40.599321795309486], [-73.91594923488863, 40.59931248795741], [-73.91603690126853, 40.599384329099], [-73.91606364579803, 40.59939379075025], [-73.91611372324502, 40.59940332488126], [-73.91616551967118, 40.599421614484044], [-73.91620786703439, 40.599450878044635], [-73.91624224253353, 40.59946169014241], [-73.91627624656071, 40.59945395429647], [-73.91628276233509, 40.59945507780331], [-73.91629727311577, 40.599457283675974], [-73.91632181903475, 40.59946649605735], [-73.91633776063104, 40.599475730147404], [-73.9163513221006, 40.59948674668899], [-73.91635614296348, 40.59949095032296], [-73.91637158318203, 40.5994968897488], [-73.91638281598057, 40.59949879184002], [-73.91639933954947, 40.59950349803017], [-73.91642042780603, 40.59951391559324], [-73.91643680104468, 40.59952301277696], [-73.91645444113797, 40.59952755508971], [-73.91647730002269, 40.59953306130268], [-73.91648935966295, 40.59953655651727], [-73.91650681425914, 40.59954496727198], [-73.91652506718975, 40.59954841152992], [-73.91654703433908, 40.599547109866506], [-73.91656158487507, 40.59954566678058], [-73.91657083730628, 40.59954745651137], [-73.91658101917103, 40.59955421667035], [-73.91661611022751, 40.599598268145286], [-73.91662775280886, 40.599618067153784], [-73.91663530587657, 40.59962581198032], [-73.91665164333688, 40.59963438859402], [-73.91666986159849, 40.5996372838107], [-73.91668199820639, 40.599635837451466], [-73.91669770973537, 40.59962695672514], [-73.91670898542586, 40.599623395520304], [-73.91672219926376, 40.59962543739327], [-73.91673777424802, 40.59963897884077], [-73.9167494102652, 40.59966531096908], [-73.91676213565182, 40.599682585580254], [-73.91677260011099, 40.599693654108165], [-73.91678498028277, 40.5996995912621], [-73.9168344055606, 40.59971261055429], [-73.9168626221947, 40.59972619028238], [-73.91687661605516, 40.599737618693815], [-73.9168770324865, 40.599749504020124], [-73.91687230406697, 40.5997583937209], [-73.91686466092482, 40.5997662932257], [-73.91685914082419, 40.59977358982049], [-73.91685865887294, 40.59978520058676], [-73.91686357759053, 40.59979752826426], [-73.91687133689634, 40.599812136226134], [-73.91688619095041, 40.599827517272594], [-73.91690389584289, 40.59983730155216], [-73.91691904905846, 40.59984313314279], [-73.91693467357098, 40.59984520303614], [-73.91695138598205, 40.59984290723196], [-73.91696976278126, 40.59983413829388], [-73.916988318111, 40.59982566969421], [-73.91700881093413, 40.599823351985556], [-73.91702558933866, 40.59982577953689], [-73.91707040741538, 40.59983728512503], [-73.91709128513465, 40.599844491045154], [-73.91711367578142, 40.59985090312656], [-73.91713887012357, 40.59986135149608], [-73.91715858719805, 40.599874183370744], [-73.91716562558177, 40.599890629761006], [-73.91716475112281, 40.59989817732702], [-73.91716038381828, 40.599906381267665], [-73.91715709354423, 40.59991755007885], [-73.91716075835816, 40.59992372851747], [-73.91716709068046, 40.59992820828963], [-73.9171737867899, 40.59992911777356], [-73.91719133448245, 40.59992056716322], [-73.9172176133197, 40.5998998074473], [-73.91723155883308, 40.59989216006649], [-73.91724668706972, 40.59988909562718], [-73.91726346642493, 40.59988971215368], [-73.91727721874068, 40.59989323648137], [-73.91728693511226, 40.59989777195707], [-73.91730057566032, 40.59990387514625], [-73.91731295816078, 40.599908001253986], [-73.917318748244, 40.59991409828968], [-73.91732032431776, 40.59992087849323], [-73.9173239521627, 40.59992834754511], [-73.9173338821583, 40.5999360950405], [-73.9173483850213, 40.599942501401955], [-73.91743211525555, 40.599967457435376], [-73.91750461425308, 40.599988670784796], [-73.91754276861181, 40.60000343889447], [-73.91756644521524, 40.60001654795263], [-73.91758695033738, 40.600034568530916], [-73.91760853457674, 40.60005193115436], [-73.91762670792177, 40.60006267694057], [-73.91766806318164, 40.60007903698742], [-73.91769264353164, 40.600090610759196], [-73.91770566532304, 40.600101983590754], [-73.91770929031334, 40.60011181194219], [-73.91770801389576, 40.600124190096444], [-73.91768435883627, 40.60018055243014], [-73.91767925353922, 40.60020264386546], [-73.91768406403119, 40.60021519106045], [-73.9176961462692, 40.60022939027532], [-73.91771262127872, 40.600243758488254], [-73.9177454576067, 40.60027737662655], [-73.91778123630412, 40.6003190402727], [-73.91779521853181, 40.60034068097581], [-73.91782637491528, 40.60039282645142], [-73.91783658859188, 40.60040430699156], [-73.9178452953706, 40.60041007672315], [-73.91787279962206, 40.60041712372899], [-73.91789299943636, 40.60041834521039], [-73.91790488417723, 40.600417338138676], [-73.91791951170606, 40.60041150215275], [-73.91792643852354, 40.600401791075996], [-73.91793394476751, 40.600388099918824], [-73.91794587383234, 40.60038160378019], [-73.91795951878667, 40.60038353491406], [-73.9179747044877, 40.60039172469689], [-73.91799042415569, 40.60040556913691], [-73.91800776563622, 40.60041837236503], [-73.91803500633199, 40.60043422916977], [-73.91806182036798, 40.60044547509257], [-73.91807514078245, 40.600449134715234], [-73.91809152834422, 40.600446210446876], [-73.91811422430293, 40.600438869575434], [-73.91812603535004, 40.60043863138738], [-73.91813182934679, 40.60044288941647], [-73.91813383910213, 40.60044774969767], [-73.91813309247705, 40.600469431967795], [-73.91813278460573, 40.60048551790256], [-73.91813356095673, 40.60049910524135], [-73.91814175737943, 40.60050992391695], [-73.91814855814971, 40.600514266091864], [-73.91818895561349, 40.600519070779136], [-73.91820814640923, 40.60052207684534], [-73.91822251015459, 40.600526313493845], [-73.91824003666655, 40.60053579605467], [-73.91824881183966, 40.60054444847924], [-73.91826447883913, 40.60057212656762], [-73.91830543680375, 40.60061969561864], [-73.91832875071364, 40.60063708731235], [-73.91837703552672, 40.60067066364188], [-73.91840751312273, 40.60069044669305], [-73.91843788919837, 40.60070446732973], [-73.91850912125022, 40.60073204759907], [-73.91852934477951, 40.600744002915896], [-73.91854477885144, 40.60075537618888], [-73.91855603487761, 40.60076801084183], [-73.91857336923096, 40.600786824423906], [-73.91859190395141, 40.60079639057196], [-73.91860918373875, 40.600800299875196], [-73.9186209992879, 40.600798222168876], [-73.91863602304811, 40.600791727647554], [-73.91866855912362, 40.600776160243235], [-73.91867828675998, 40.600772351817], [-73.91869312203318, 40.60077334989828], [-73.91875723059746, 40.60079546352166], [-73.91878422409624, 40.600807559612996], [-73.91880409704811, 40.60081235024749], [-73.91882519627006, 40.600814396779036], [-73.91883736467854, 40.60081712269675], [-73.91884779871687, 40.60082399206091], [-73.91887901850234, 40.60085629295704], [-73.91889402037988, 40.600867802581654], [-73.91891144376474, 40.600873853968906], [-73.91893729843603, 40.600873570823495], [-73.91896161093118, 40.60086938901804], [-73.91898059854394, 40.600860810168115], [-73.91898232576955, 40.600860557076516], [-73.91899367352788, 40.60085889849522], [-73.9190048336398, 40.600861540757606], [-73.91905184816414, 40.60087458436685], [-73.91906947725295, 40.60087921723785], [-73.91907478072157, 40.60088061263736], [-73.9190859350907, 40.60088745501234], [-73.91909805291067, 40.60090220259423], [-73.91910082286103, 40.60090519577777], [-73.9191079478483, 40.60090887939911], [-73.91916353375625, 40.600921353030095], [-73.91917533854992, 40.6009276188831], [-73.91917814168286, 40.60093171057719], [-73.91917705749938, 40.60093601916811], [-73.91916343676967, 40.60094245929477], [-73.91914135970333, 40.60094452983742], [-73.91912104778446, 40.60094588794384], [-73.91911171349467, 40.600951398948276], [-73.91911145577778, 40.600955487987], [-73.9191160236535, 40.60096065193879], [-73.91913369661202, 40.600968102839204], [-73.91918590709885, 40.60097266787889], [-73.91920714797703, 40.60097685546285], [-73.91922071571277, 40.60098369901857], [-73.91923257897216, 40.601000698167034], [-73.91924192974514, 40.60101190356203], [-73.91925362988023, 40.60101476581045], [-73.9192689379233, 40.60101200429177], [-73.91928471935168, 40.60100419327109], [-73.91930014678056, 40.60099157759767], [-73.91930740174625, 40.60097829829693], [-73.91930761917936, 40.600946788721586], [-73.91931068795509, 40.60094023026551], [-73.91931886749276, 40.60093666780384], [-73.91933788168978, 40.600935171042664], [-73.91935182054645, 40.60093301232717], [-73.91935921049398, 40.600927884753524], [-73.91935979294124, 40.60092206670571], [-73.91935533192367, 40.600918495371204], [-73.91932309982607, 40.600920311214324], [-73.91931363050709, 40.6009195361101], [-73.91930265004916, 40.60091771673973], [-73.9192980448084, 40.6009138164419], [-73.91929798058312, 40.60090676224505], [-73.91930356890065, 40.60090234708414], [-73.91932071242007, 40.60089994368452], [-73.91940336080107, 40.60089739408333], [-73.9194177594273, 40.60090217959982], [-73.9194285176335, 40.600909132567935], [-73.9194438593342, 40.60093801865419], [-73.91944888545889, 40.60095196512916], [-73.919449917555, 40.60096039240342], [-73.91945196503714, 40.600965773703344], [-73.91946214741152, 40.600972532433424], [-73.9194835657342, 40.600979409436704], [-73.91951117923657, 40.6009862359931], [-73.91954271573707, 40.60099279003866], [-73.9195731752888, 40.60099821970376], [-73.91962467121434, 40.60099809104886], [-73.91967426193835, 40.60099524371564], [-73.91969353230722, 40.60099146834274], [-73.9197120904502, 40.60098118831556], [-73.91972869395262, 40.60097914131516], [-73.91974280611898, 40.600982006203495], [-73.91976249301169, 40.60099192980434], [-73.91977670330691, 40.60100239739924], [-73.91978251184943, 40.601023398557224], [-73.9197807746835, 40.60103064365225], [-73.91977266662413, 40.60103527670831], [-73.91975984495103, 40.60103727140605], [-73.91975375422574, 40.60104075310076], [-73.91975014484001, 40.60104764005091], [-73.9197558270934, 40.60105447853934], [-73.91976935752436, 40.601063160972004], [-73.91978570053212, 40.601067564660354], [-73.91980392083788, 40.601068647217595], [-73.91982352098546, 40.60106133283121], [-73.91985292611216, 40.60104288090303], [-73.91987865386812, 40.601030272167236], [-73.91989763488907, 40.60102822679992], [-73.91992204048402, 40.601035297959406], [-73.91993193709642, 40.60104249621279], [-73.91993779023873, 40.601055674786], [-73.91994313488395, 40.601072065544265], [-73.91995763043886, 40.601086815696604], [-73.91997796402242, 40.601098028680205], [-73.92001158049183, 40.60111232506652], [-73.92003732206925, 40.60111827116776], [-73.92004992437437, 40.60111959738493], [-73.92006098575499, 40.60111463700143], [-73.92007226783052, 40.60110506720478], [-73.92009753006873, 40.60109023431383], [-73.92010700786292, 40.601084505373585], [-73.92012260482421, 40.60108053454887], [-73.9201439928501, 40.6010826904354], [-73.92016017744025, 40.60108652184814], [-73.92016573798129, 40.60108783880482], [-73.92017300752794, 40.60109127485216], [-73.92017433407824, 40.601096162097456], [-73.92017439212422, 40.601099454253884], [-73.92017443595286, 40.601101953149644], [-73.92016993010388, 40.60110568290524], [-73.92015118519022, 40.601121644628705], [-73.92012940107826, 40.601150175962374], [-73.92010883532043, 40.60118252227571], [-73.92009786339072, 40.601202908478385], [-73.92009810034679, 40.6012144648065], [-73.92010831237522, 40.60122830512586], [-73.92012777059982, 40.601247038194444], [-73.92014597657983, 40.60126014408959], [-73.92018070185937, 40.6012826750884], [-73.92022921531151, 40.60130689093989], [-73.9202524656747, 40.601317747889254], [-73.9202745033028, 40.60131932823683], [-73.92028646897721, 40.60131102038816], [-73.92029336077799, 40.601298947486654], [-73.92030281623185, 40.601281194610365], [-73.92031373899856, 40.601272309065145], [-73.92033578660204, 40.60126551780687], [-73.92035956000656, 40.60126114217105], [-73.92037954507622, 40.60126151386084], [-73.92039721693821, 40.601268963393125], [-73.92042017776066, 40.601281551019575], [-73.9204353244499, 40.6012933915802], [-73.92044273110251, 40.601303168182405], [-73.92044747436779, 40.60131047322529], [-73.92045575359825, 40.60131454120359], [-73.92046928978824, 40.6013166641199], [-73.92048430764656, 40.60131617929522], [-73.92051076120704, 40.601328632113784], [-73.9205300788218, 40.601345772986924], [-73.92055062419102, 40.60136016990734], [-73.920577982519, 40.60136924611485], [-73.92059529862183, 40.60137370581253], [-73.92062779408764, 40.601392337401286], [-73.92065543050411, 40.60140934679178], [-73.92066643721799, 40.60142005967257], [-73.9206684076847, 40.60142856833214], [-73.92066785275189, 40.60144091954055], [-73.92066993292822, 40.60144866233248], [-73.92067554164554, 40.601456268678085], [-73.92068392280545, 40.60146374090319], [-73.92069756657507, 40.60146803219695], [-73.92072395869437, 40.6014710428115], [-73.9207393708553, 40.60147226115963], [-73.92075503628809, 40.601471200948374], [-73.92076807095478, 40.601472390178586], [-73.92077599006171, 40.60147527764228], [-73.92080851865347, 40.60149627035892], [-73.92081935415786, 40.6015001490891], [-73.92083397239482, 40.60150213432865], [-73.92085298676878, 40.601502449312605], [-73.92087747193757, 40.601504552245764], [-73.92089593937085, 40.601509395327206], [-73.92092552893064, 40.6015205325913], [-73.92094881399765, 40.60153193891907], [-73.92095589791772, 40.601540563000256], [-73.92095761605037, 40.60154948427314], [-73.92095576692455, 40.6015593099279], [-73.92095521008692, 40.60157349960146], [-73.92096222064357, 40.60158338680168], [-73.92097770063809, 40.60158748666638], [-73.92099710909586, 40.60158950315624], [-73.92102209672717, 40.60159311600929], [-73.92104070841883, 40.601597741713746], [-73.9210588114722, 40.60160741471995], [-73.92107492332471, 40.6016247729296], [-73.92109423755731, 40.60164608635416], [-73.9211245708161, 40.60166556669541], [-73.92114084135589, 40.601671232138145], [-73.92114912406117, 40.60167112859218], [-73.92115698009401, 40.60166641233958], [-73.92116559830849, 40.60165672970426], [-73.92117651951577, 40.60164781660151], [-73.92118462754671, 40.60164504988942], [-73.92120515319725, 40.60164506396386], [-73.92122870387188, 40.601645849234515], [-73.92124868755369, 40.6016480582321], [-73.92126610981683, 40.6016541092608], [-73.92128115139077, 40.60166432912758], [-73.92128996036959, 40.601675342330516], [-73.92129495169256, 40.60168692888356], [-73.92129954787023, 40.60169810270904], [-73.92131055774985, 40.60170700336676], [-73.92132078400631, 40.60170830101291], [-73.92136278007645, 40.60170138550414], [-73.92138561263522, 40.60170035741795], [-73.92139987012983, 40.6017030022362], [-73.92141210729977, 40.60170915892549], [-73.92142034348977, 40.60171814060289], [-73.92142504461891, 40.60173148299273], [-73.9214224428487, 40.601738426638434], [-73.92141105247315, 40.601748766289525], [-73.92138907143205, 40.60176077281501], [-73.92137497382096, 40.601774651842405], [-73.92136346453296, 40.60179300667328], [-73.92135790599158, 40.601804477277994], [-73.92135890295563, 40.601812903337425], [-73.92136829024517, 40.60182410908578], [-73.92141645365716, 40.601870418779754], [-73.92145617208807, 40.60190346643382], [-73.92147002357932, 40.6019145922976], [-73.92147830140904, 40.60191865969836], [-73.92148780695602, 40.601919983139275], [-73.92151689981944, 40.60192365414108], [-73.92152612096316, 40.601922535646985], [-73.92153322059893, 40.60191732480882], [-73.92153694106672, 40.60190837928073], [-73.92153598005682, 40.60189814108446], [-73.92153811694328, 40.60188839854404], [-73.92154854419883, 40.601870180163765], [-73.92155965231296, 40.6018561343936], [-73.9215697796087, 40.60184982830339], [-73.92158706910077, 40.60184772605043], [-73.92160810002534, 40.60184689001953], [-73.92162693823371, 40.601842702911206], [-73.92164091966228, 40.601835082328684], [-73.92165116024191, 40.601824383526086], [-73.9216535155744, 40.60181184205503], [-73.92165032453889, 40.60180006458902], [-73.92165094648091, 40.60179114441956], [-73.92166349282464, 40.60177885683339], [-73.92168255247921, 40.6017700584587], [-73.92170315115612, 40.60177062148704], [-73.92171928075352, 40.601774145896606], [-73.92172730617746, 40.601778653876686], [-73.92173932549888, 40.60178524897408], [-73.92175468270877, 40.601802113033195], [-73.92176479249822, 40.60181016165112], [-73.9217794065531, 40.601815798401006], [-73.9217946389653, 40.60181668712286], [-73.92180753372094, 40.601813923623894], [-73.92182079481722, 40.6018063301717], [-73.92184685489362, 40.60178652986499], [-73.92188701180113, 40.601752850463434], [-73.92190795213722, 40.60173604024183], [-73.92191657304511, 40.6017305471217], [-73.92192301744058, 40.601726443684605], [-73.92193443677061, 40.60172266260216], [-73.92194967363287, 40.601719352187736], [-73.9219649789354, 40.60171897776647], [-73.92197269359112, 40.60172046463248], [-73.92197455638636, 40.60172082323691], [-73.92198506448837, 40.601726402893526], [-73.92198887708614, 40.60173109864389], [-73.92199106492465, 40.601738072358046], [-73.92199421934278, 40.601751139764374], [-73.92199853119115, 40.60175857998109], [-73.92200425704225, 40.601759956266065], [-73.92201383653271, 40.60175817975392], [-73.92202446625261, 40.6017533560952], [-73.92203559501866, 40.60175182656513], [-73.92204131784406, 40.60175504131139], [-73.92204581235028, 40.601760974222806], [-73.92205027568491, 40.60176273228579], [-73.92205567929052, 40.601761117618885], [-73.92206036685496, 40.60175587713474], [-73.92206921126198, 40.60173741119817], [-73.92208135790413, 40.60172759301031], [-73.92209187580309, 40.60172535055902], [-73.92210098576165, 40.601726810384946], [-73.92214223054567, 40.60174756152526], [-73.92215543963513, 40.60175380172069], [-73.92217127987671, 40.60175724344635], [-73.92218456872658, 40.601756154626194], [-73.92219667576958, 40.60175182549274], [-73.92223055154116, 40.60172912141327], [-73.92225538189477, 40.601713877142856], [-73.92226763245095, 40.601708038996286], [-73.92227371755679, 40.6017087572887], [-73.92228552206859, 40.601714502475765], [-73.92229192411692, 40.60172183486584], [-73.92230106397129, 40.601728016840845], [-73.92231078303837, 40.60173019123886], [-73.92232385875587, 40.601728279857774], [-73.92233524073562, 40.601725789187405], [-73.9223477078364, 40.601718963091564], [-73.92237055773482, 40.60170358075426], [-73.92241545175197, 40.60168376506143], [-73.92243703508662, 40.601672416126014], [-73.92244561623141, 40.60166402284182], [-73.92245081383555, 40.60165373335662], [-73.92244898816395, 40.60164374219245], [-73.92244053684487, 40.60163517273299], [-73.92242394500137, 40.60162522537915], [-73.92240501735296, 40.60161398580108], [-73.92239609427259, 40.6016068158936], [-73.92239159974326, 40.60160088517372], [-73.92239106545904, 40.601597069000825], [-73.92239460312709, 40.60158908292863], [-73.92240228150452, 40.601582254095455], [-73.92241237195188, 40.60157539891303], [-73.92244162733235, 40.60156320377257], [-73.92245607084489, 40.601560688854136], [-73.92247457913338, 40.601561881081146], [-73.9224939509947, 40.6015651609661], [-73.92251886505412, 40.60156954139786], [-73.9225372978724, 40.60157383687443], [-73.92254931939387, 40.60157914074125], [-73.92255331166082, 40.60158359116378], [-73.92255532265295, 40.601588450194875], [-73.92255848474096, 40.60159366750894], [-73.92256431597028, 40.601596664346815], [-73.92258882756775, 40.60160713853344], [-73.92259804503615, 40.60160837876164], [-73.92260931861337, 40.60160665711751], [-73.92262311519376, 40.6016028797453], [-73.92263492944167, 40.60160080112303], [-73.92264400244554, 40.60160171239053], [-73.92265062818646, 40.60160210116198], [-73.92266222777391, 40.60159862419391], [-73.92267444374241, 40.601592236485004], [-73.92269254017482, 40.60157555973792], [-73.9227048796416, 40.60155457026569], [-73.92272227920652, 40.60151936683121], [-73.92272837333098, 40.60151171233242], [-73.92274484167972, 40.60150151283558], [-73.92278407843642, 40.60148504294147], [-73.92283614734508, 40.60145691625867], [-73.92287038507452, 40.60143355300594], [-73.92291904736955, 40.60139279849783], [-73.92294803045895, 40.60136613907888], [-73.92296591366738, 40.601348035969636], [-73.92297287919708, 40.60133469978733], [-73.92297908904168, 40.60132081527757], [-73.92298643636853, 40.601288788957206], [-73.92299641354248, 40.60125580261341], [-73.92300226467997, 40.60124076559623], [-73.92301384197246, 40.60122529214809], [-73.92303904049196, 40.60120285709951], [-73.92305075467893, 40.60119369674138], [-73.92306336458508, 40.60118849067048], [-73.92307369907836, 40.601187755936685], [-73.92307895564846, 40.60118817174443], [-73.92308518768189, 40.60118685779584], [-73.92308987388277, 40.60118345908437], [-73.92309867078826, 40.601174625628886], [-73.92310638907756, 40.60116414564429], [-73.9231134887283, 40.601158935378386], [-73.92312353670413, 40.601157541556205], [-73.92313272021084, 40.60115826204332], [-73.92313916397367, 40.601160132125614], [-73.92314957058, 40.60115994810159], [-73.92315994773388, 40.601155562257624], [-73.92317061869976, 40.601144727405725], [-73.92317419374658, 40.6011354815147], [-73.92317781181823, 40.60112074270534], [-73.92317706933007, 40.60111006490415], [-73.92317293760227, 40.601101115246124], [-73.92317345039767, 40.60109422561627], [-73.92317990463356, 40.60108775274996], [-73.92318988107249, 40.601085865500586], [-73.92319949464685, 40.60108644772329], [-73.9232125308212, 40.601087636174626], [-73.92322315123742, 40.60108882351326], [-73.92322859023683, 40.60108827765089], [-73.9232335960521, 40.60108732256026], [-73.92324120433761, 40.60107939346007], [-73.92325253786898, 40.60105601596548], [-73.92325991357451, 40.60103052110628], [-73.92326212649822, 40.601017676948665], [-73.92326199310563, 40.6010077132288], [-73.92326142664844, 40.60099914909372], [-73.92326053419595, 40.600992341954644], [-73.92326511697703, 40.60098367035337], [-73.92327200343347, 40.60097708854135], [-73.92328317206356, 40.600971907785954], [-73.92330154702422, 40.600963136662806], [-73.92331675430847, 40.60095513254967], [-73.92332825119229, 40.600946411782346], [-73.92334959532695, 40.600923480049914], [-73.92337084434382, 40.60089110563525], [-73.92337276154241, 40.60088234999545], [-73.92337086771961, 40.600870738122445], [-73.92336759441474, 40.600866262575494], [-73.92335996492247, 40.60086397875585], [-73.92334872870467, 40.60086391681534], [-73.92333785168488, 40.600865529567336], [-73.9233248849953, 40.600867221673404], [-73.9233166766265, 40.60086603528337], [-73.92331228746465, 40.60086172367157], [-73.92330984628042, 40.600855162437234], [-73.92330748891582, 40.600838965985226], [-73.92331229486442, 40.60082332380894], [-73.9233176713275, 40.60081520340354], [-73.92332671714138, 40.6008077436682], [-73.92334123743036, 40.60080212588176], [-73.9233564413047, 40.60079648174456], [-73.92336764802194, 40.600789490518466], [-73.92338073597212, 40.6007755830977], [-73.92338507405101, 40.600760818459406], [-73.92338760309362, 40.600722228791625], [-73.92338805690385, 40.600704086018034], [-73.92339040365536, 40.60069755494926], [-73.923397576587, 40.60069105457211], [-73.92340842111713, 40.60068708180908], [-73.92342059661388, 40.600684345124556], [-73.92342816371738, 40.600680068288], [-73.92343688775689, 40.60067200438672], [-73.92344796102553, 40.60065504893015], [-73.92345016908628, 40.6006464037169], [-73.92345061633692, 40.60063243358808], [-73.9234451349292, 40.60060789143381], [-73.9234454346592, 40.60059779029338], [-73.9234481416064, 40.600592440209], [-73.923456435849, 40.60058215274005], [-73.92346760593718, 40.60057697196762], [-73.92348532507853, 40.60057476098872], [-73.92351009864078, 40.60057700025719], [-73.9235278484509, 40.60057948295748], [-73.92353778463892, 40.60058253575205], [-73.92354527007507, 40.60058737210636], [-73.92355232434709, 40.60058946169988], [-73.92355873559974, 40.600588973596174], [-73.92357051667331, 40.60058401218073], [-73.92358039242698, 40.6005763058717], [-73.92359095650694, 40.60056442854256], [-73.92361544051083, 40.600536008078514], [-73.92362272640692, 40.60052563817303], [-73.92362727560467, 40.60051589719593], [-73.92362495167734, 40.600500771786145], [-73.92360604612871, 40.60046916656549], [-73.92359340322476, 40.60044094222395], [-73.92358553540072, 40.60042474211327], [-73.92358353124993, 40.600413870816496], [-73.92358480111764, 40.60040569271802], [-73.92359010595837, 40.60039518311809], [-73.92360027447677, 40.60038341525659], [-73.92360961936981, 40.60036824282096], [-73.92361600655249, 40.60035649924917], [-73.92362170569025, 40.600347719648724], [-73.92363165286676, 40.60034056237583], [-73.92364332452888, 40.600336864050234], [-73.92367599533473, 40.6003289264404], [-73.9236940481655, 40.60031949631534], [-73.92370521869141, 40.600312531334076], [-73.92371520547469, 40.60030224496983], [-73.9237213054254, 40.60029044809471], [-73.92372755454412, 40.60027293874013], [-73.92372968924906, 40.60023264821258], [-73.92373103394813, 40.60022186039228], [-73.92373389127943, 40.600210278649705], [-73.92373840639789, 40.600198205638904], [-73.92374789127815, 40.600186464785224], [-73.92375575253068, 40.60017628623612], [-73.92376899292714, 40.60015612014826], [-73.92378100763126, 40.60013452656747], [-73.92380187302743, 40.600090267965136], [-73.92382682188565, 40.60003406991408], [-73.92385067646083, 40.59998819327638], [-73.92385501257137, 40.59997578960699], [-73.92386215202568, 40.59996692919447], [-73.92386986398152, 40.599960619625364], [-73.92387703838902, 40.59995464191007], [-73.92388449965362, 40.59994825092206], [-73.9238934428192, 40.59993735990808], [-73.92389971842576, 40.59992874652924], [-73.92389948837919, 40.599908819113615], [-73.92390086636624, 40.59990042094948], [-73.92390469376221, 40.599892544752954], [-73.92391525222916, 40.599884290041075], [-73.92392747048704, 40.59987609104144], [-73.92392988662432, 40.59987244150935], [-73.92393411506927, 40.599858447075874], [-73.9239277423936, 40.5998270691614], [-73.92392760768657, 40.59981894558113], [-73.92393478234803, 40.599810605030385], [-73.92394552203531, 40.59980320114894], [-73.92396069286788, 40.599794674904764], [-73.923971574733, 40.59978941264071], [-73.92398177222428, 40.599782365234404], [-73.92398834060724, 40.59976966064603], [-73.9239973367064, 40.59974312494408], [-73.92400307854606, 40.59973014522799], [-73.92401237890165, 40.59972227381946], [-73.92402682516602, 40.5997155860555], [-73.92404544701309, 40.59971235952233], [-73.92406547246631, 40.59970910643898], [-73.92408013809667, 40.59970143122844], [-73.92408709514405, 40.59969462877763], [-73.92409477905095, 40.59968123872038], [-73.92410988673973, 40.59966615280867], [-73.92412523564633, 40.59965792854792], [-73.92415617704242, 40.59964993370466], [-73.9241638512213, 40.599648962073736], [-73.92418280970283, 40.59964656082188], [-73.92418462982292, 40.599646329827124], [-73.92421174312555, 40.59964802199761], [-73.92424541012143, 40.59965087192058], [-73.92427578653687, 40.599655177672034], [-73.92429073928069, 40.59965729604619], [-73.92430715377998, 40.59966276898078], [-73.92431291117896, 40.59966702667425], [-73.924315205492, 40.599675621093205], [-73.92431019279303, 40.599682561542345], [-73.92430039289493, 40.59968713835158], [-73.92429552785268, 40.59969023678251], [-73.92429127228051, 40.59969586078259], [-73.92429123088938, 40.59970135035228], [-73.92429647697783, 40.599710138872], [-73.9243065894021, 40.599715853588044], [-73.92432195634315, 40.5997248660577], [-73.92434437540875, 40.599737808879766], [-73.92435708039476, 40.59974437785398], [-73.92437141172346, 40.59974444273834], [-73.92438186307997, 40.59973695687192], [-73.92439249970637, 40.59972559868911], [-73.92440486987621, 40.599709330191985], [-73.92441456365636, 40.599703160771334], [-73.92442659380441, 40.599700094522106], [-73.92444650700794, 40.59970123357916], [-73.92446511640095, 40.5997082170372], [-73.92448153095812, 40.59971366247213], [-73.92452211365516, 40.59971459496307], [-73.92455239376244, 40.599719172310515], [-73.92457042932716, 40.599723602732396], [-73.92457841785493, 40.59972939945981], [-73.92458240393556, 40.599738570278916], [-73.92458307728964, 40.599748177515366], [-73.9245857956167, 40.599763192261264], [-73.92459068384736, 40.59977263808477], [-73.92459877781147, 40.59978005438489], [-73.92461752940672, 40.59978863044405], [-73.92463430497592, 40.599793416866525], [-73.92465076177538, 40.599793401858], [-73.9246638050116, 40.599786766871006], [-73.92467381667127, 40.59978540103436], [-73.92468717251317, 40.599789032941636], [-73.92470272386295, 40.59979420336635], [-73.92471629808001, 40.5997955846586], [-73.92473567944916, 40.599788707882375], [-73.92474940221008, 40.599786218505685], [-73.92476743780465, 40.59979065006929], [-73.92478060994458, 40.59979815193755], [-73.92479626607707, 40.5998049424256], [-73.92482542770848, 40.599811492965735], [-73.92485419122107, 40.59981870430839], [-73.92486491934976, 40.599821482423216], [-73.92487924455054, 40.599827008859535], [-73.92489500836548, 40.599835390764405], [-73.92491642219117, 40.59984643890011], [-73.92495896714752, 40.599863705108035], [-73.92501594498844, 40.599888116121775], [-73.9250452756294, 40.59990334087569], [-73.92507625606675, 40.59992460579027], [-73.92510107941986, 40.59994773262404], [-73.92513064453816, 40.59997871379738], [-73.92514689619352, 40.60000057186655], [-73.9251724338564, 40.60002787347486], [-73.92519060360202, 40.60004094939815], [-73.9252079878428, 40.60004831675258], [-73.92521529692843, 40.60004994151645], [-73.92524212099853, 40.60005336249182], [-73.92527811813908, 40.60006534229355], [-73.92530612196603, 40.60007689957384], [-73.92532417702287, 40.600096261599504], [-73.9253808749357, 40.60014586947818], [-73.9254249818069, 40.600183612782885], [-73.92547665986197, 40.60021271262071], [-73.92555357867363, 40.60024490462461], [-73.92562198775776, 40.600287273661635], [-73.92568778087471, 40.600317481027375], [-73.92573201532659, 40.600338397496294], [-73.92579962957868, 40.60040672305363], [-73.92579971739785, 40.60040681089572], [-73.92580642995411, 40.600416210098814], [-73.92582320504269, 40.600446970910184], [-73.92582991943321, 40.600455517054456], [-73.92583164324368, 40.60045771179413], [-73.92583327596893, 40.60045979071229], [-73.92583775483408, 40.60046321036055], [-73.92584251867842, 40.60046611906081], [-73.9258433548601, 40.600466630061845], [-73.92585007362811, 40.60047005115492], [-73.92586100945621, 40.600474548160044], [-73.92586463613146, 40.60047603983401], [-73.92586498616973, 40.60047612181405], [-73.92591841275298, 40.60048888664136], [-73.92592836870797, 40.60049243496076], [-73.92593521516956, 40.600494876755434], [-73.9260337234564, 40.60053180231572], [-73.92605619362678, 40.6005402257355], [-73.92608531584787, 40.60055220186512], [-73.92612390674402, 40.60057033742145], [-73.92615812012767, 40.600586415406696], [-73.92617044246994, 40.60059069364249], [-73.92617274590835, 40.600591280301636], [-73.92618388603849, 40.60059412071109], [-73.9262499858011, 40.60060868256752], [-73.92626038449839, 40.6006123897923], [-73.92626678761923, 40.60061467145995], [-73.92627350927923, 40.600618094205224], [-73.92627826204523, 40.60062051319246], [-73.92628023072332, 40.60062151410195], [-73.92629142748436, 40.600629210035734], [-73.92630150229996, 40.60063861253884], [-73.92633619879054, 40.60067707133951], [-73.92634118598816, 40.60068196622084], [-73.92635187240187, 40.60069245654335], [-73.92637114016009, 40.600707168402415], [-73.92638994148001, 40.60072152312234], [-73.92643249022989, 40.60075315340743], [-73.92647728272885, 40.60078393205362], [-73.92658366943729, 40.600849770346294], [-73.92659428951238, 40.6008572852854], [-73.92662958157993, 40.600882258110396], [-73.92666092795997, 40.60090961067159], [-73.92672473632462, 40.60097286080263], [-73.9267362962585, 40.60098299430626], [-73.9267549643135, 40.60099935805846], [-73.92677813208405, 40.601018653922324], [-73.92680422726917, 40.60104038969803], [-73.92683610747942, 40.601063107186036], [-73.92683782009782, 40.601064327553665], [-73.92685087595353, 40.601072482975866], [-73.92688709503545, 40.60109510939491], [-73.92689898565789, 40.60110116208164], [-73.92693413537623, 40.6011190552739], [-73.92702874570602, 40.60116065575206], [-73.92704502576358, 40.60116781237987], [-73.92710895443535, 40.601196718891124], [-73.92713015240506, 40.601206303443234], [-73.92713043379761, 40.60120645406285], [-73.92715815311817, 40.60122084228769], [-73.92717864611902, 40.60123172909081], [-73.92718391352784, 40.60123452664747], [-73.92720295053422, 40.60124649610229], [-73.92721137751529, 40.601253308652886], [-73.92722198514105, 40.601261882980175], [-73.9272802035588, 40.60131231528864], [-73.92734386492113, 40.60136510948414], [-73.92734514011707, 40.60136617091746], [-73.92734631436632, 40.601367288241796], [-73.92736305373747, 40.60138326368278], [-73.92737648337224, 40.6013995015584], [-73.9273876687054, 40.60141915426929], [-73.9274077913679, 40.60146529111266], [-73.92741547780352, 40.60147967868183], [-73.92743015841607, 40.60150715955345], [-73.92744917922722, 40.60153535901532], [-73.92745701242593, 40.60154476050356], [-73.92746597086004, 40.60155245323387], [-73.92748948522387, 40.601568696957756], [-73.92751524676548, 40.601581524834224], [-73.92757797229089, 40.60160975233355], [-73.92759648550174, 40.60162008111463], [-73.92760709237857, 40.6016259995657], [-73.92761976678575, 40.60163463999164], [-73.92770115744216, 40.60169012126253], [-73.92781313709692, 40.60176877472591], [-73.92785120812964, 40.60179698541436], [-73.92793854543464, 40.601866227241345], [-73.92796961892401, 40.601887691291275], [-73.92798557836042, 40.60189871566504], [-73.92799229777866, 40.60190299186896], [-73.92800013798627, 40.601906412365466], [-73.92801806164896, 40.60191325710751], [-73.92808304299008, 40.60192867244247], [-73.92810544832876, 40.60193551880251], [-73.9281155299934, 40.60193979659986], [-73.92812176618881, 40.601943368703864], [-73.92812448977602, 40.60194492725578], [-73.92813120697132, 40.60195005600606], [-73.9281379252257, 40.60195603949035], [-73.9281502908348, 40.60196825825464], [-73.92815695657418, 40.60197484279971], [-73.9281594595408, 40.601978189754064], [-73.92816590857194, 40.60198680746447], [-73.92816966445174, 40.60199213472644], [-73.92817374140687, 40.60199791619285], [-73.92817437712357, 40.60200133116942], [-73.92817485418183, 40.60200389617147], [-73.92817597007449, 40.60200902141836], [-73.92817653573007, 40.6020107515104], [-73.92817708692468, 40.6020124393761], [-73.92817627850464, 40.60201489602115], [-73.92817596272405, 40.60201585425813], [-73.92817491178074, 40.60202063890424], [-73.92817258416396, 40.60203122797281], [-73.92817258048822, 40.60203464472772], [-73.92817257865096, 40.60203635251887], [-73.9281725773517, 40.60203756023576], [-73.928172576813, 40.602038060980135], [-73.92817286010552, 40.602038494220054], [-73.92817481375147, 40.60204147896779], [-73.92817534501715, 40.60204309930675], [-73.92817593148192, 40.60204489642348], [-73.92817885678181, 40.60204712974002], [-73.92817929074548, 40.60204746104889], [-73.92817982602546, 40.60204800602062], [-73.9281826490905, 40.602050879904674], [-73.92819048928519, 40.60205515512195], [-73.9282117688975, 40.602066272751415], [-73.92821735995149, 40.60206968914761], [-73.92822072840617, 40.60207311186188], [-73.92823304157285, 40.602085077615364], [-73.92824423625846, 40.602100459761466], [-73.9282542979957, 40.60211840439525], [-73.9282561205666, 40.60212207860975], [-73.92828225525899, 40.60217479557378], [-73.92829344377773, 40.6021910326123], [-73.92830575510735, 40.60220556021253], [-73.92831359349337, 40.60221154438527], [-73.92831931739906, 40.60221591326261], [-73.92832255237369, 40.602218382314504], [-73.92833263229963, 40.6022235136526], [-73.92833967200662, 40.60222595776936], [-73.92834495508235, 40.60222779282726], [-73.9283707227971, 40.60223464190169], [-73.92837440833208, 40.60223529354171], [-73.92838528891583, 40.60223721451037], [-73.92839873568074, 40.60223807594712], [-73.92852648878832, 40.60223986443279], [-73.92864863242363, 40.602247627404495], [-73.92865203667742, 40.60224719746011], [-73.92865535739779, 40.60224677684564], [-73.92866320311964, 40.602245928657375], [-73.92868072657457, 40.6022417690622], [-73.92868077406598, 40.60224175485166], [-73.92868113858397, 40.60224166829777], [-73.92868351783146, 40.60224070246098], [-73.92869795603175, 40.602234845386015], [-73.92872798506362, 40.60222187933012], [-73.92873943918403, 40.602216933768375], [-73.92874531826534, 40.60221494632949], [-73.9287596173283, 40.60221011393654], [-73.92876522242689, 40.60220926267705], [-73.92876747029989, 40.602209264070304], [-73.92876970498486, 40.60220926545536], [-73.9287741871033, 40.60220926823322], [-73.92877575426226, 40.602209269204465], [-73.92877866988114, 40.602209271011326], [-73.9287811637933, 40.60221022278224], [-73.9287831508356, 40.60221098325558], [-73.92878478744825, 40.60221139873692], [-73.92878651195417, 40.60221183839689], [-73.92878753889543, 40.60221262072412], [-73.92878987124975, 40.6022144030043], [-73.92879235977134, 40.60221566939122], [-73.928793231457, 40.60221611287849], [-73.92879430416593, 40.60221726095029], [-73.92879882926825, 40.602222095127104], [-73.92880442554089, 40.602228078379746], [-73.92880762827701, 40.602232971042305], [-73.92881337630807, 40.60224175028251], [-73.9288149185336, 40.602245759205296], [-73.92881896369747, 40.602256274835334], [-73.9288335008079, 40.602295568419656], [-73.92883420122563, 40.602297292728345], [-73.92884690357079, 40.60232804169031], [-73.92887540522597, 40.60240875913059], [-73.92887707111007, 40.60241347677732], [-73.92888713241265, 40.602437398962394], [-73.92888764503715, 40.60243838033041], [-73.92889831522541, 40.602458760487266], [-73.92890726601246, 40.60247328661358], [-73.92891552365388, 40.602484006215754], [-73.92891845677121, 40.60248781563248], [-73.92893189064395, 40.6025006360672], [-73.92893711921303, 40.60250436179799], [-73.92894868747497, 40.60251260401329], [-73.92896227304934, 40.6025216784189], [-73.92896660726448, 40.60252457298606], [-73.92898564680316, 40.60253398129473], [-73.92898846106694, 40.60253505572274], [-73.92900581177689, 40.60254168082792], [-73.92902597637304, 40.60254767256533], [-73.92905667218389, 40.60255513701545], [-73.92907527403578, 40.60255966206693], [-73.92911224821778, 40.602565662486256], [-73.92912414379957, 40.60256623741147], [-73.92913017763841, 40.602566528277464], [-73.92918172865058, 40.60256656005458], [-73.92919101377166, 40.602568083087085], [-73.92919741628921, 40.602569132245996], [-73.92920525844762, 40.602570844869156], [-73.92920595124994, 40.602571056717665], [-73.92921086019335, 40.60257255728442], [-73.92930719474792, 40.60261276192554], [-73.92940240951566, 40.60265382120168], [-73.92943377206495, 40.60266836038134], [-73.92946177257461, 40.60268289815907], [-73.92950671202776, 40.60270923350431], [-73.92952653371674, 40.60272782849604], [-73.9295311997649, 40.60273352734808], [-73.92954217933408, 40.60274694076711], [-73.92956585098817, 40.602775394689466], [-73.92956616907622, 40.602775775845686], [-73.92958404325599, 40.60279760847477], [-73.9295988094564, 40.60283113206136], [-73.92962398500417, 40.60286232833962], [-73.92963395191336, 40.602903972470614], [-73.92963408761969, 40.6029076412744], [-73.92963545750591, 40.602944678785796], [-73.92964821803915, 40.6029686497986], [-73.9296478101645, 40.603012592107646], [-73.92964063354694, 40.603057137398515], [-73.92962907767883, 40.60308743232412], [-73.92962552383216, 40.603110679172005], [-73.92962544552631, 40.603137841628545], [-73.92962541070168, 40.60314981876793], [-73.92961589749179, 40.60318963997944], [-73.92959887783277, 40.60321653791767], [-73.92959504624501, 40.60322259275404], [-73.92956844608068, 40.6032457425525], [-73.92951172074437, 40.6032849583449], [-73.92947942819046, 40.603309065913386], [-73.92944931704055, 40.60331434436946], [-73.92942835807844, 40.60331496275152], [-73.92941520249596, 40.60332412287207], [-73.92940808221259, 40.60334909508271], [-73.9293999160438, 40.60337371081844], [-73.92939237228437, 40.603390696486464], [-73.9293791374652, 40.603408667916646], [-73.92936630105368, 40.603423811865845], [-73.92934132980231, 40.60343771098355], [-73.92932709555383, 40.603446513593866], [-73.92931073811064, 40.603454601222296], [-73.92930010623508, 40.60346170260863], [-73.92928201173615, 40.60347700799476], [-73.92924206782794, 40.60351612474901], [-73.92918120003131, 40.603555831367814], [-73.92907960400987, 40.603629521410284], [-73.92903906112069, 40.60365666994538], [-73.92901905320171, 40.60367521127385], [-73.9290116971971, 40.60368541846881], [-73.92900966703503, 40.60369746691503], [-73.9290135071661, 40.60371053404616], [-73.92901612396753, 40.60372124027128], [-73.92902457501721, 40.60373296582849], [-73.9290372022207, 40.6037441724455], [-73.92903956400407, 40.60375968358305], [-73.92904052647938, 40.603768823226396], [-73.92905279131001, 40.603782856671955], [-73.92909011185046, 40.60380651186568], [-73.9291125387482, 40.603814238573996], [-73.92912996631458, 40.60381732465399], [-73.92914897671425, 40.60381997026785], [-73.92915768588944, 40.603826893081695], [-73.92916490474228, 40.60383863478946], [-73.92916685323532, 40.603841802961], [-73.92916872325594, 40.603846216663115], [-73.92917220501894, 40.60385443175683], [-73.92916767703367, 40.60387932597249], [-73.92916956895058, 40.603895629565365], [-73.92918035528923, 40.60391207752092], [-73.92919568365807, 40.60392520767894], [-73.92921374589248, 40.60394025973739], [-73.92924015235188, 40.60396503462514], [-73.92925238099349, 40.60397906752425], [-73.92927458471804, 40.60399357480484], [-73.92930104928406, 40.603998805096374], [-73.92935726345164, 40.604001529677674], [-73.92940872349553, 40.604003619894215], [-73.92942158380257, 40.60399923601647], [-73.92944103951854, 40.60399030056702], [-73.9294646491881, 40.60397003046513], [-73.9294913330262, 40.60393809786157], [-73.92950334816175, 40.60392664406198], [-73.92953282447988, 40.60389854388255], [-73.92957181899767, 40.60387183231382], [-73.92961228066397, 40.603853468099025], [-73.92964239204595, 40.603848187925294], [-73.92966717688859, 40.603841066689476], [-73.92969815565213, 40.60383219302955], [-73.92973379494495, 40.603811161103735], [-73.92977444433969, 40.60378601670778], [-73.92981112592351, 40.60376537138195], [-73.92984643610663, 40.60374914429055], [-73.92987492996905, 40.60374232598875], [-73.92989750871342, 40.6037432460956], [-73.9299235720069, 40.60375127624858], [-73.92997300169458, 40.60376538720001], [-73.93004747973626, 40.60379795900567], [-73.93006118620376, 40.60380952233383], [-73.93007711500097, 40.60383461951268], [-73.93008075602481, 40.60386528106569], [-73.93008511429105, 40.60389907216383], [-73.93008748758332, 40.60393615584436], [-73.93009881658116, 40.60398414190381], [-73.93010981569321, 40.60400459837055], [-73.93012524946275, 40.60401970413938], [-73.9301425251583, 40.60402959534135], [-73.93016762708739, 40.60402845833087], [-73.93019328021879, 40.60401771318433], [-73.93021404882279, 40.60399354414058], [-73.93023442879164, 40.603961388576344], [-73.93025730770022, 40.60391638671609], [-73.93026947522583, 40.60388727234369], [-73.93028462271236, 40.60386606480682], [-73.93031739463554, 40.60383037434031], [-73.93034486413778, 40.60380360174428], [-73.93037621988707, 40.603781112870344], [-73.93039048333974, 40.603773328557814], [-73.9303971176974, 40.60376970724544], [-73.9304436695569, 40.603747585397194], [-73.93048920100134, 40.60373586462894], [-73.93052669571796, 40.60373116683175], [-73.93056523214435, 40.6037268258186], [-73.93061949814351, 40.60373281542373], [-73.93066836360131, 40.603734985264325], [-73.93071390997461, 40.6037448391521], [-73.93074706415364, 40.603758226283595], [-73.93078896040734, 40.60377856241736], [-73.93082574338456, 40.603801038156], [-73.93085406295367, 40.60382257330861], [-73.93087590379082, 40.60383990611027], [-73.93090288636984, 40.60386586120507], [-73.93091985385577, 40.603880117411514], [-73.93092861464277, 40.603887478210694], [-73.93094389424502, 40.60391153184247], [-73.93095561984985, 40.60392496048305], [-73.93098128579337, 40.60393568050735], [-73.93098650258456, 40.60393570943957], [-73.9310033614306, 40.60393580466654], [-73.93104826056619, 40.60391000197197], [-73.93106343895494, 40.6038926914137], [-73.93107488876831, 40.60388862112461], [-73.93108217243231, 40.60388603331433], [-73.93109906602827, 40.603882475240546], [-73.9311228428378, 40.603875655476635], [-73.93117476161734, 40.603850406434695], [-73.93120849798017, 40.603822621823745], [-73.93123128465788, 40.60379669826616], [-73.9312344359677, 40.603778994500985], [-73.93124811330779, 40.603751802319714], [-73.93127496724915, 40.603727307361844], [-73.93130497314321, 40.60368508283118], [-73.93135649064573, 40.603631535786974], [-73.93142546481485, 40.6035858755184], [-73.93144846344933, 40.603563793151665], [-73.9314815991787, 40.60352453486918], [-73.93151404507826, 40.603490683838736], [-73.93154406622969, 40.60346912765781], [-73.93156257164631, 40.603437984626545], [-73.93160741952673, 40.603391514450706], [-73.93166059968239, 40.603330995863324], [-73.93169449676314, 40.603286386755855], [-73.93172784434582, 40.60325097064041], [-73.93177214165783, 40.60321369574673], [-73.93182093031916, 40.60318547962285], [-73.93186823221386, 40.60316802383074], [-73.93191463221174, 40.60315210406218], [-73.93193669261566, 40.60313155862344], [-73.93196227064436, 40.60312163761542], [-73.93197205593805, 40.6030968573047], [-73.93198888391238, 40.6030519624194], [-73.93200606453784, 40.60301472566441], [-73.93203695430982, 40.6029502966486], [-73.93207363026065, 40.602900993731865], [-73.93209319849413, 40.602851434260984], [-73.93211937076838, 40.602753487894745], [-73.93212319646146, 40.602709739248745], [-73.93211608839734, 40.60268851664704], [-73.93209824153307, 40.60267297162927], [-73.93205807277273, 40.60265348619675], [-73.93200644100638, 40.602645057091074], [-73.93198823448277, 40.602646013631585], [-73.93197337876066, 40.602646794236705], [-73.93192029481557, 40.602649123289446], [-73.93187888608789, 40.60264421360292], [-73.93183431132815, 40.602636335920295], [-73.93178896067144, 40.602613198367706], [-73.93176200342052, 40.602596384217804], [-73.9317550722581, 40.60257897750979], [-73.93174384565035, 40.60253568600244], [-73.93172110351152, 40.60248278019162], [-73.93170559532443, 40.6024342704514], [-73.93168375514567, 40.602379800613754], [-73.93163945294982, 40.6023175777339], [-73.93157427847974, 40.60224065663869], [-73.93152590725897, 40.60217703141789], [-73.93150390891184, 40.60213941404541], [-73.93148823885095, 40.60210772913052], [-73.93144561470642, 40.602059231162755], [-73.93139614519956, 40.60201399540885], [-73.93137235644889, 40.602000174290765], [-73.93134168560594, 40.60198958739512], [-73.93128455811797, 40.60193232479023], [-73.93124308428159, 40.601886106165814], [-73.93119934599105, 40.601835355009364], [-73.93114405923458, 40.60177499180074], [-73.93108919528449, 40.601722258264104], [-73.93102539821933, 40.60167209995919], [-73.93096575788032, 40.601643985424694], [-73.93091035991495, 40.60162114287192], [-73.93087733573208, 40.60162288022755], [-73.9308535603595, 40.60162969994252], [-73.93082481679065, 40.60163665386861], [-73.93078208265746, 40.60162567642215], [-73.93073018821799, 40.60159276186143], [-73.93068245865955, 40.60155759802955], [-73.9306822289305, 40.60155742717873], [-73.93066320829776, 40.60152794698997], [-73.93065201488477, 40.60151512810837], [-73.93064082322839, 40.60150400763745], [-73.93063996976649, 40.60150336816609], [-73.93063298119036, 40.601498033002564], [-73.93063046460767, 40.60149668873736], [-73.93062178134748, 40.60149204695867], [-73.93057921705011, 40.60147237561322], [-73.93057105199372, 40.601467388866006], [-73.93056241907169, 40.601462115863264], [-73.93052582299812, 40.6014370870029], [-73.93050866654544, 40.601425353609415], [-73.93040899205376, 40.60135376604912], [-73.93033845548122, 40.601303105382705], [-73.9303188583356, 40.60128857358182], [-73.93032404489969, 40.60125460482645], [-73.9303275332228, 40.601223892645955], [-73.93033638091511, 40.6012006501541], [-73.93032538113468, 40.60118184218983], [-73.93030680986901, 40.601171674640895], [-73.93028452329249, 40.60116773491007], [-73.93026651635373, 40.60116904210491], [-73.93025369099225, 40.601174002902304], [-73.93023945031183, 40.6011897488696], [-73.93022055592759, 40.60121326033738], [-73.9301855361498, 40.601255607196066], [-73.9301580737881, 40.601261996322904], [-73.93015219266587, 40.60126311368638], [-73.93014462384336, 40.60126455067462], [-73.93013683387696, 40.601264545939365], [-73.93013341782661, 40.60126454386272], [-73.93007178366099, 40.60126280243787], [-73.9300269612016, 40.601259354545704], [-73.93001555567409, 40.60125803015145], [-73.92982751092616, 40.601236171206125], [-73.92976745752982, 40.60123027870774], [-73.9297311454627, 40.601226716284856], [-73.92970313339589, 40.601222427160174], [-73.92969849898986, 40.60122141479462], [-73.92967960538135, 40.601217288216624], [-73.92967475251649, 40.6012160517298], [-73.92966952368374, 40.60121471952117], [-73.92966621007596, 40.60121303332258], [-73.92965944160925, 40.60120958829843], [-73.9296536647654, 40.60120664780871], [-73.92964936173404, 40.60120445757877], [-73.92964040417411, 40.60119761807765], [-73.92963530032544, 40.601192609013744], [-73.92962473188327, 40.60118223432973], [-73.92960072089012, 40.601156354292584], [-73.92958667176653, 40.60114121143541], [-73.92956636704699, 40.60111943159284], [-73.9295396584037, 40.60109078649569], [-73.92953297631976, 40.60105924184886], [-73.92949818814161, 40.601026585494914], [-73.92944300324639, 40.601007529025466], [-73.9293789210336, 40.60099104885453], [-73.92935962488771, 40.60098626107385], [-73.92934268039237, 40.600969151121994], [-73.92933017967741, 40.600940432285654], [-73.92932027997658, 40.600903234720626], [-73.92933595940595, 40.60085608875241], [-73.9293528014468, 40.600831861990756], [-73.92939430707477, 40.600778611543895], [-73.92941042444443, 40.60075976352278], [-73.9293783012559, 40.60075993761867], [-73.92932650927015, 40.60076835930303], [-73.92927494280951, 40.60080123701228], [-73.92927955903113, 40.60079344522294], [-73.92929361057489, 40.60075324267776], [-73.92930300852304, 40.60072083340616], [-73.92931151719048, 40.60071062689882], [-73.92932802274737, 40.60069943772429], [-73.92934730529184, 40.600683557762316], [-73.92935953042145, 40.60066712446622], [-73.92936653782064, 40.60064700885382], [-73.92936224319976, 40.600621122788205], [-73.92935568467544, 40.60059070634699], [-73.929330732723, 40.60055393761681], [-73.9292974468055, 40.600531190805064], [-73.92925599005152, 40.60050561122917], [-73.92922698259743, 40.60048810851607], [-73.92919242166285, 40.60047993559241], [-73.92914823594411, 40.60047971570148], [-73.92910537836164, 40.60048556368234], [-73.92907270660847, 40.600494930183245], [-73.92904353846012, 40.60049425395592], [-73.92901401862485, 40.600485947052206], [-73.92898762768309, 40.60048060583295], [-73.92894839890866, 40.60045955583641], [-73.92891648851871, 40.600463543569106], [-73.928887533228, 40.60046670954803], [-73.92886892721346, 40.60045654174437], [-73.92880778459828, 40.60041854344655], [-73.92874943482549, 40.60039652075547], [-73.9286978030231, 40.60038809133292], [-73.92865250413834, 40.600385591828676], [-73.92862706586206, 40.60039935547633], [-73.9286258273593, 40.600413929578814], [-73.9286316082062, 40.60042905753612], [-73.92863995628161, 40.60043570458212], [-73.9286416693992, 40.60044942963665], [-73.92864212258888, 40.600498396018146], [-73.92865155628814, 40.60056545794396], [-73.92864686122837, 40.60057909709786], [-73.92864116613579, 40.60058345786463], [-73.92862225814346, 40.60058561445284], [-73.92861094896605, 40.60058722576032], [-73.92861018260628, 40.60059724486953], [-73.9286223818028, 40.600604607983165], [-73.92862464646834, 40.60060776597455], [-73.92862453839156, 40.60060832528744], [-73.92862261946878, 40.600618111988155], [-73.92860303341341, 40.60061450581846], [-73.9285923060007, 40.60061161597626], [-73.92857870563411, 40.60060136998411], [-73.92857518798455, 40.60059877997471], [-73.92856862903965, 40.60059395223051], [-73.9285546210608, 40.600593065167494], [-73.92850907632634, 40.60058603814537], [-73.9284747682273, 40.600575778754894], [-73.92843482019475, 40.600554097388084], [-73.92840113840293, 40.6005306358932], [-73.92838858614411, 40.60051682115461], [-73.92838295485663, 40.60049672570302], [-73.92838329688392, 40.60047866526482], [-73.92838857780214, 40.60045758785562], [-73.92840663787993, 40.60043937528304], [-73.9284193577783, 40.60043191626298], [-73.92843056920009, 40.60041998394299], [-73.9284354807066, 40.600408266157125], [-73.92843329354068, 40.60039802725411], [-73.92842495526673, 40.600384105914415], [-73.9284064979437, 40.600369820931], [-73.9283768102623, 40.60035009530454], [-73.92833177428656, 40.600336947682656], [-73.92830671347868, 40.60033380243893], [-73.92827775792114, 40.60033724234181], [-73.92825300571437, 40.60034922265162], [-73.92823188564527, 40.60036562280153], [-73.9281960446756, 40.60037537188879], [-73.92816870920258, 40.60037777024236], [-73.928146997769, 40.600374242745275], [-73.92811939022465, 40.600363136556744], [-73.92808461583712, 40.600351916648], [-73.92799773750467, 40.60033772647209], [-73.92792338512085, 40.60032864886874], [-73.92786714717562, 40.600318704903096], [-73.92784799166246, 40.60031605898452], [-73.92782946477035, 40.600300181834655], [-73.92782473100979, 40.60028198033372], [-73.9278263353957, 40.600263563684756], [-73.92782833586782, 40.6002445160346], [-73.9278263345313, 40.60023040714474], [-73.92781734374648, 40.600219422305074], [-73.92780219076201, 40.60021180964185], [-73.9277785060595, 40.600202846188445], [-73.92776246340723, 40.60018625808715], [-73.92775178900308, 40.60016758627985], [-73.9277274696526, 40.600145750311476], [-73.92771292360688, 40.60014233763988], [-73.92769451952992, 40.60014545450232], [-73.92767834403922, 40.60015170304727], [-73.9276600797816, 40.60015803202941], [-73.92762853105334, 40.600160564808725], [-73.92760041413544, 40.60015560599309], [-73.92757953964676, 40.60014368034258], [-73.92756399058695, 40.60013672679311], [-73.92755662150886, 40.60012469991844], [-73.92754632704413, 40.60012115152656], [-73.9275341173215, 40.600122462450464], [-73.9275020849737, 40.600106658652564], [-73.9274690122, 40.600087369749325], [-73.9274210074881, 40.60005657136589], [-73.92736149014415, 40.60001549899463], [-73.92731870493384, 40.599988105212454], [-73.92728762410275, 40.59995844226587], [-73.92726658379864, 40.59993460412226], [-73.92726648709433, 40.59992431106728], [-73.92726483978696, 40.59991662345015], [-73.92725756679881, 40.59991491876052], [-73.92724506651244, 40.59991847854939], [-73.92722641495031, 40.59991679295599], [-73.9272113065423, 40.59991256852067], [-73.92720240400442, 40.59991008043419], [-73.92719082214731, 40.59989818757071], [-73.92719043353631, 40.59989014508548], [-73.92719261227055, 40.59988011096436], [-73.9271928227896, 40.59987913993097], [-73.92720914776773, 40.599867401952174], [-73.92722777404134, 40.59986041454598], [-73.92723610197662, 40.59984985291512], [-73.92723463181115, 40.59984540542423], [-73.92722768616635, 40.599841421035165], [-73.92720540235808, 40.59983531405302], [-73.92717227591551, 40.59983177914042], [-73.92716032319036, 40.5998292463976], [-73.92715594268309, 40.59981744152431], [-73.92710413921029, 40.59977036228994], [-73.92708901013796, 40.599758127919365], [-73.92705304954458, 40.59972904749481], [-73.92701978356502, 40.59968914504339], [-73.92700202272425, 40.59966327730648], [-73.92698531092736, 40.59963381488914], [-73.92696222590705, 40.59960294921509], [-73.92695110992662, 40.599592017905714], [-73.92693981836622, 40.59957784697149], [-73.92692599660758, 40.59957149770611], [-73.92690317700756, 40.59956283818453], [-73.92689425681218, 40.59955344534665], [-73.92687471342725, 40.59954275510411], [-73.92684710643931, 40.59953164793891], [-73.92682702785184, 40.59951837790045], [-73.9268059068983, 40.599501621079256], [-73.9267652631132, 40.59949154862736], [-73.92674017770368, 40.59947973171431], [-73.92672334147932, 40.59946445990761], [-73.92669369939772, 40.599437651743486], [-73.92665925151499, 40.59942417918013], [-73.92663891620732, 40.59941477953946], [-73.92659961215756, 40.59939726891725], [-73.92653503480115, 40.59937340449817], [-73.92653118070083, 40.59937230187013], [-73.9264951855822, 40.5993620150088], [-73.92648406971782, 40.59935108415667], [-73.9264754552105, 40.5993388109844], [-73.926467243388, 40.59932711018144], [-73.92646120556235, 40.59931634725728], [-73.92645110447422, 40.59930022883919], [-73.92643732732675, 40.599286797413555], [-73.92641473713509, 40.59926556621463], [-73.92638616901631, 40.599243863943386], [-73.92636956306761, 40.59921602050503], [-73.92636463944082, 40.599210378484756], [-73.92635478144055, 40.599199076175985], [-73.92635044516359, 40.59918021664433], [-73.92634920303678, 40.59916955177652], [-73.92634891129076, 40.59916704066172], [-73.92634446148519, 40.59915367115678], [-73.92633410188601, 40.59914142215219], [-73.92631564601325, 40.599127136831584], [-73.92629448276806, 40.59911746168336], [-73.92628368473243, 40.59911295375025], [-73.92626891935092, 40.59911338286961], [-73.92625127323285, 40.59911518421302], [-73.92623619163915, 40.59910918988767], [-73.92622687393289, 40.59910045647248], [-73.92622216715536, 40.599090928410064], [-73.92621944449444, 40.59907813547178], [-73.92619933031128, 40.599064893612514], [-73.92616352553107, 40.599041484373416], [-73.92612301467624, 40.59900854822989], [-73.92609843902208, 40.5989905820088], [-73.92604902566563, 40.59896565487881], [-73.92602488490398, 40.59895353368099], [-73.92602433232163, 40.5989531704571], [-73.92601425313464, 40.59894718418077], [-73.92595057866393, 40.598903884613996], [-73.92594258905342, 40.598898451351666], [-73.92593811050776, 40.598895031707116], [-73.92593475338458, 40.59889075805116], [-73.92593375346489, 40.598889666624025], [-73.92593156242029, 40.59888727725046], [-73.92592691844457, 40.59888221118934], [-73.92590679030292, 40.598844615882506], [-73.92590119587815, 40.59883607046336], [-73.92589568516014, 40.59882933776401], [-73.92589560021958, 40.59882923233347], [-73.92589261456003, 40.598826526821675], [-73.92585977761183, 40.598796751950665], [-73.92583753317133, 40.59877976873093], [-73.92582058901954, 40.59876683090566], [-73.92580598355872, 40.59875830328665], [-73.92580155223895, 40.59875571477978], [-73.92575563312391, 40.598731767479336], [-73.92575251558674, 40.5987296690046], [-73.92573659804448, 40.598718943048226], [-73.92565822746133, 40.59865397681586], [-73.92562669063508, 40.59862942443688], [-73.92560112639357, 40.59860952351154], [-73.9255641743498, 40.59858472888034], [-73.92555233718649, 40.59857815507231], [-73.9255272166921, 40.59856420405059], [-73.92548129626529, 40.59854281966875], [-73.92541297158328, 40.59851373482396], [-73.92536255938539, 40.598490455906635], [-73.92531665072475, 40.598469255413406], [-73.92528528476437, 40.5984512923106], [-73.92527992898941, 40.598447201785625], [-73.92527969577682, 40.5984470232153], [-73.92525618334719, 40.59842992430041], [-73.92523363547856, 40.59841506470128], [-73.92514980439398, 40.59835981327223], [-73.92511070317778, 40.598331863859535], [-73.92509717876669, 40.598322196147876], [-73.92504791458926, 40.598284581694294], [-73.92500649201055, 40.59824868012971], [-73.92494604709893, 40.59818970271445], [-73.92492254260961, 40.59816491665724], [-73.92491081971119, 40.59815125469593], [-73.92490127818121, 40.59814013205909], [-73.92486099248995, 40.59809056481995], [-73.92483837071387, 40.598059146840164], [-73.9248363788443, 40.598056381472716], [-73.92483261008998, 40.59805062370843], [-73.92482295530831, 40.59803587230784], [-73.92479387712824, 40.597984604080075], [-73.92478465432619, 40.59796997156951], [-73.92477933494938, 40.597961531650796], [-73.92477918550237, 40.59796133118798], [-73.9247659085614, 40.59794358617715], [-73.9247622735302, 40.59793931615762], [-73.9247513614209, 40.59792649302673], [-73.92474009937212, 40.59791605209875], [-73.92473568852431, 40.59791196166436], [-73.92473193763267, 40.597908894930534], [-73.924720014367, 40.59789913926471], [-73.92470915116704, 40.59789207246627], [-73.92468978154238, 40.597879473805804], [-73.92460691614966, 40.59782816909613], [-73.9245206957021, 40.597770030950876], [-73.92447607835382, 40.597736604516996], [-73.92446135436103, 40.59772557459884], [-73.9244487071711, 40.59771545706825], [-73.92441433342832, 40.59768796067522], [-73.92430462445286, 40.5975922224009], [-73.92418594655079, 40.59749989419293], [-73.92415907942458, 40.59747766828646], [-73.92413669063639, 40.597457153123095], [-73.92410219522203, 40.597422268683566], [-73.92409527585879, 40.597415271442955], [-73.92408559602251, 40.59740499617655], [-73.92402924289875, 40.59734518668049], [-73.92394641925141, 40.597254592392986], [-73.92394638258862, 40.597254556685016], [-73.92392851138528, 40.59723920420347], [-73.92390611558727, 40.59722381521734], [-73.9238803590299, 40.597209276054535], [-73.92383668205115, 40.59718789300036], [-73.92382324111028, 40.597181905977806], [-73.92380644037645, 40.59717591555349], [-73.9237336273626, 40.5971536580692], [-73.92371794598832, 40.597147669379936], [-73.92370450736209, 40.59714082710826], [-73.92369666779267, 40.59713599153241], [-73.92367539301226, 40.59712287041878], [-73.92364852239879, 40.597103206414346], [-73.92363815280564, 40.597094256980604], [-73.92362277341711, 40.59708098112426], [-73.92361261437688, 40.59706972997629], [-73.9236003920106, 40.597056195539864], [-73.92356905544636, 40.59701603033871], [-73.92353550130476, 40.59696475713423], [-73.92352432308421, 40.596941686781676], [-73.92351196722836, 40.59691003379076], [-73.92350197762798, 40.5968844434145], [-73.92349415008601, 40.59686991710314], [-73.92348519926999, 40.59685709900924], [-73.9234807233837, 40.59685197147678], [-73.92347892351393, 40.59685032229311], [-73.92347512574173, 40.59684684152181], [-73.92346168981793, 40.59683829143005], [-73.92346159030858, 40.59683824680095], [-73.92344488896383, 40.59683059248918], [-73.92341050780455, 40.59681549245017], [-73.92340008997358, 40.59681091698383], [-73.9233877705673, 40.59680407592339], [-73.92337769380057, 40.59679723518436], [-73.92336749391434, 40.596789092827436], [-73.92335306472157, 40.596777573107154], [-73.92329597364638, 40.596725431071505], [-73.92326350666963, 40.59669978562925], [-73.92313249869694, 40.596614280937516], [-73.92307478221241, 40.596573974482205], [-73.92300149949263, 40.59652279748471], [-73.92297792333675, 40.59650640184731], [-73.92293879754466, 40.5964791933248], [-73.92292364204584, 40.596466217233726], [-73.92290185533977, 40.59644756381632], [-73.92288730417671, 40.59643388768615], [-73.92287275532037, 40.59641764902809], [-73.92282911735096, 40.596362099027296], [-73.92278324901831, 40.59629715263754], [-73.92276646638513, 40.59627578623685], [-73.92276271417477, 40.596272307147785], [-73.9227507936471, 40.596261255773726], [-73.9227317627059, 40.596246721877826], [-73.92271048717738, 40.5962338959377], [-73.9226853149523, 40.59621958189009], [-73.92265785437878, 40.59620396358915], [-73.92264423377209, 40.596195295671365], [-73.92263098168353, 40.59618686354615], [-73.92260747216093, 40.596167200386155], [-73.92259679370879, 40.596156471991236], [-73.92258620658549, 40.59614583211233], [-73.92257973624291, 40.596138694524846], [-73.9225660626608, 40.59612361086123], [-73.92255213594396, 40.596103752776436], [-73.92254928091384, 40.59609968307412], [-73.92254139384103, 40.59608633690025], [-73.92253362477693, 40.59607319284546], [-73.92252021192988, 40.59604328854053], [-73.92251016113933, 40.59601423972752], [-73.9224889511689, 40.595945038851276], [-73.92248775867337, 40.5959418057372], [-73.92247666275819, 40.59591171853983], [-73.9224715207821, 40.59590174322611], [-73.92246212866127, 40.59588352126436], [-73.92245408604424, 40.59587035122003], [-73.92244647095809, 40.59585788525396], [-73.92235359972804, 40.59574165606875], [-73.92235294893801, 40.59574073773622], [-73.92233898026504, 40.595663161908696], [-73.92233231530048, 40.595649954924916], [-73.92232477376976, 40.59563702343636], [-73.92231637101301, 40.595624405482546], [-73.9223071320517, 40.59561212989518], [-73.92229707948252, 40.59560023153538], [-73.9222862427194, 40.59558874175086], [-73.9222691205748, 40.59557234798532], [-73.92225291508969, 40.59555541807352], [-73.922237653476, 40.595537986545466], [-73.92223294047409, 40.5955320539925], [-73.92222896451868, 40.59552581569452], [-73.92222576159047, 40.595519324782984], [-73.92222335755983, 40.595512635722265], [-73.92222177609516, 40.59550580615867], [-73.92222189417565, 40.5954953765359], [-73.9222208713644, 40.59548497696643], [-73.92221871637015, 40.59547467865593], [-73.92221639107898, 40.59546963813651], [-73.92221332882592, 40.59546483266424], [-73.92220956821986, 40.595460323748576], [-73.9222051577627, 40.59545617005755], [-73.9222001549719, 40.59545242657963], [-73.92219462792109, 40.59544914227926], [-73.92218864667122, 40.595446357745985], [-73.9221807631201, 40.59544369539331], [-73.9221732644179, 40.59544044929365], [-73.92216622766996, 40.595436650659764], [-73.92215972118292, 40.5954323368971], [-73.9221538077597, 40.595427553113566], [-73.92214854800336, 40.59542234709614], [-73.92214399438339, 40.595416767129116], [-73.92212734219116, 40.59539698722357], [-73.92211008461364, 40.59537750996584], [-73.92209223262553, 40.595358347425034], [-73.92207379764585, 40.59533950832016], [-73.92205623831359, 40.59532439012891], [-73.92203933617749, 40.59530884417597], [-73.92202310814187, 40.595292885885776], [-73.92200756890878, 40.59527653503714], [-73.92199273933893, 40.59525980638696], [-73.92197863347785, 40.595242717032924], [-73.92196095458273, 40.59521034322984], [-73.92194049917715, 40.59518032048667], [-73.92193661822674, 40.59517538595588], [-73.92191664456314, 40.59516361585085], [-73.92190952281642, 40.595159996232226], [-73.92190432655661, 40.59515671917879], [-73.92189984052709, 40.59515182159452], [-73.92189748415403, 40.595147642985516], [-73.92189474842519, 40.59514254622555], [-73.92189371865047, 40.595140026890405], [-73.92189304385532, 40.59513822197385], [-73.92189248507893, 40.595135684865674], [-73.92189199328271, 40.59513375492911], [-73.92189104834448, 40.59513142060401], [-73.92188936906379, 40.59512933841374], [-73.9218873709001, 40.59512724779788], [-73.92188482570957, 40.59512553927943], [-73.92188195708184, 40.59512452176692], [-73.92187863799909, 40.59512456005261], [-73.92187439945658, 40.595125471210935], [-73.92187211183328, 40.595125836879895], [-73.921869209959, 40.59512618571319], [-73.92186672506277, 40.59512647887521], [-73.92186156866788, 40.59512651323071], [-73.92185614886566, 40.595127023357726], [-73.92185068798891, 40.59512694995196], [-73.92184529548462, 40.595126294093], [-73.92184007726809, 40.59512506975808], [-73.9218351352871, 40.595123300302916], [-73.92183056795825, 40.59512102231607], [-73.92182564491118, 40.5951160337983], [-73.92182139587858, 40.59511069995835], [-73.92181785926209, 40.59510507074623], [-73.92181507323924, 40.59509920046745], [-73.92181306389156, 40.59509315012058], [-73.92179366710775, 40.595052734051755], [-73.92179034544655, 40.59504672067288], [-73.92178615002189, 40.595041032208606], [-73.92178113524761, 40.59503574559187], [-73.92177536324344, 40.595030926201154], [-73.92176925703451, 40.59502848466075], [-73.92176284122208, 40.59502655621905], [-73.92175619006348, 40.595025163375475], [-73.92174938265764, 40.59502432260152], [-73.92174249613383, 40.59502404400164], [-73.921735611145, 40.595024330813644], [-73.92172880461726, 40.595025178734375], [-73.92172216029513, 40.59502657843905], [-73.92171575006248, 40.59502851523408], [-73.92170964954919, 40.59503096487945], [-73.92170488103618, 40.59503364662241], [-73.92169971505945, 40.59503586085531], [-73.92169423033926, 40.59503757261787], [-73.92168851107718, 40.59503875834593], [-73.92168264322856, 40.595039397826966], [-73.92167671823155, 40.59503948107162], [-73.92167082443592, 40.5950390081399], [-73.92166304515766, 40.59503725600919], [-73.92165551001412, 40.595034971636316], [-73.9216482785463, 40.595032170809766], [-73.92164141577734, 40.59502887903827], [-73.92163497617989, 40.59502512299623], [-73.92163061144817, 40.59501764318771], [-73.92162741235373, 40.595009828780334], [-73.9216254262431, 40.59500178568509], [-73.92161997607357, 40.594997467753664], [-73.92161400124391, 40.59499357482092], [-73.921607558418, 40.594990139426194], [-73.92160070953007, 40.59498719729538], [-73.92159352026091, 40.59498477527775], [-73.92158605585645, 40.59498289653709], [-73.92157838684278, 40.594981578879306], [-73.92155692042513, 40.59497126184707], [-73.92153504859822, 40.594961448461916], [-73.9215127928866, 40.594952149795574], [-73.92149017174252, 40.594943373063884], [-73.92146720976831, 40.59493512833541], [-73.92144392629557, 40.59492742265944], [-73.92142034439162, 40.59492026291996], [-73.92139648558721, 40.594913655162806], [-73.92137237470664, 40.594907607278735], [-73.92134803130321, 40.5949021246417], [-73.92130891011638, 40.594893629086094], [-73.92126955278182, 40.59488579476093], [-73.9212299773131, 40.59487862737437], [-73.92119020436793, 40.59487212727535], [-73.92115025195898, 40.59486630067481], [-73.92111013964636, 40.5948611464131], [-73.92106988895912, 40.59485667103859], [-73.92105821375999, 40.594855499360364], [-73.92104646227406, 40.594854907112335], [-73.92103468548993, 40.59485489298929], [-73.92102293043708, 40.59485545853127], [-73.92101124898306, 40.59485660276881], [-73.9209996883861, 40.59485831970343], [-73.9209882965644, 40.594860602666905], [-73.92097712429542, 40.59486344298249], [-73.92096226118214, 40.59486699326067], [-73.92094715815956, 40.59486991446692], [-73.92093186359128, 40.594872195745], [-73.92091642451642, 40.594873830928684], [-73.92090089039222, 40.59487481385336], [-73.92088531154974, 40.594875142375706], [-73.92086973568264, 40.594874814350824], [-73.92085872405313, 40.59487450923571], [-73.9208477744972, 40.59487356570814], [-73.9208369518422, 40.594871989341264], [-73.92082631563939, 40.59486978670982], [-73.92081593070311, 40.594866973773854], [-73.92080585767157, 40.59486356716052], [-73.92079615542374, 40.59485958433361], [-73.92076979996794, 40.59484676659261], [-73.92075810533983, 40.59484067554049], [-73.92074398898465, 40.59483332349921], [-73.92071874971512, 40.594819264789145], [-73.92068762918156, 40.59480772182258], [-73.92065610723425, 40.594796832772275], [-73.92062420430545, 40.594786602845474], [-73.92059194829402, 40.594777041777725], [-73.92055936050805, 40.59476815695497], [-73.92052646577419, 40.59475995459244], [-73.92050846362989, 40.594756678000174], [-73.92049030170058, 40.594753964695656], [-73.92047200877086, 40.59475181922186], [-73.92045361560604, 40.59475024361038], [-73.92043515582486, 40.59474924274285], [-73.92041665557934, 40.59474881764265], [-73.92040747865146, 40.59474868882287], [-73.9203983436945, 40.59474801170814], [-73.9203893065268, 40.59474679002272], [-73.92038042340398, 40.59474502950153], [-73.9203717496947, 40.594742741910224], [-73.92036333944645, 40.59473994186103], [-73.92035216727967, 40.59473550564875], [-73.92034142062136, 40.59473049510492], [-73.92033115284846, 40.59472493405576], [-73.92032141052387, 40.59471884699308], [-73.92031224305792, 40.59471226628424], [-73.92030509875002, 40.59470593458601], [-73.92029855552282, 40.59469923490743], [-73.92029264564238, 40.59469220194931], [-73.92028739654019, 40.594684868566645], [-73.92028283542275, 40.59467727280757], [-73.92027898708089, 40.594669451378344], [-73.92027084089638, 40.59465588959137], [-73.92026182183206, 40.594642649021814], [-73.92025195380327, 40.59462976352719], [-73.92024126006544, 40.59461726729927], [-73.92022976717702, 40.59460518933874], [-73.92021832170109, 40.594592611837655], [-73.92020621097927, 40.5945804000925], [-73.92019345696568, 40.59456857371926], [-73.92018008139189, 40.59455715434407], [-73.9201661064344, 40.59454615940523], [-73.9201532223689, 40.59453999710266], [-73.92013990335815, 40.5945344005778], [-73.92012618982102, 40.5945293856064], [-73.92011212547445, 40.59452496712884], [-73.92009775711064, 40.5945211610928], [-73.92008747949009, 40.5945161595627], [-73.9200776279219, 40.594510681204646], [-73.92006823842108, 40.59450474916264], [-73.92005935007998, 40.594498386080765], [-73.92005099561682, 40.59449161543553], [-73.92004320576882, 40.59448446304839], [-73.92003601258827, 40.59447695758905], [-73.92002944043563, 40.59446912772211], [-73.92001963315487, 40.594459161112475], [-73.92000925647663, 40.594449536032876], [-73.91999833213883, 40.59444026908375], [-73.9199868814336, 40.59443138222617], [-73.91997492719942, 40.59442288971577], [-73.91996249051269, 40.59441480982779], [-73.91994960036463, 40.59440715866513], [-73.91993627761585, 40.5943999503146], [-73.91992254950124, 40.5943931990351], [-73.91990844633234, 40.594386918919916], [-73.91990113765013, 40.59438393297682], [-73.91989341612258, 40.59438159123086], [-73.91988613097047, 40.59437987668323], [-73.91988592795829, 40.594379828795795], [-73.919870986523, 40.59436645724651], [-73.91986434478065, 40.59436023560279], [-73.91981166463727, 40.59431089414366], [-73.91980271208584, 40.59430149285307], [-73.91978033156241, 40.594275000398554], [-73.91978030061485, 40.59427496687101], [-73.91977651555337, 40.59427124943402], [-73.91977249912516, 40.594267305670755], [-73.9197720703081, 40.59426697483598], [-73.9197637997452, 40.59426066208601], [-73.91976354218846, 40.59426046589697], [-73.91976147753289, 40.59425928421134], [-73.91974562641109, 40.59425020377594], [-73.91970419135835, 40.594229673935175], [-73.91969653244487, 40.59422499468329], [-73.91968739555371, 40.59421941309034], [-73.91958214734348, 40.59415185968702], [-73.91949369648138, 40.5940928614478], [-73.91942232132244, 40.5940451644121], [-73.91938621479679, 40.594021035307115], [-73.91938524979608, 40.594019931282176], [-73.91938397688439, 40.59401847288416], [-73.9193817387566, 40.59401590711049], [-73.91937950018671, 40.59401334351429], [-73.91937767575332, 40.59401125347896], [-73.91937726205724, 40.59401077941583], [-73.91937391060262, 40.594003089982266], [-73.91937138796564, 40.59399407647595], [-73.91936721603241, 40.59397916930396], [-73.91936557868348, 40.593976667110084], [-73.91936497849724, 40.59397575047111], [-73.91936274228013, 40.593972332141696], [-73.91936148012405, 40.59397088581316], [-73.91936050349338, 40.593969768042456], [-73.91936038119364, 40.59396967464281], [-73.91935714497409, 40.593967203659915], [-73.9193123626819, 40.59393556698295], [-73.91926645872196, 40.59390393050561], [-73.91916792844246, 40.59384150766922], [-73.91914889160033, 40.5938312451687], [-73.91913418741463, 40.59382435137779], [-73.91909961781475, 40.59380814756753], [-73.91909107643377, 40.59380379818905], [-73.9190861794543, 40.59380130525551], [-73.91908101874826, 40.59379736334261], [-73.9190761054956, 40.59379360990664], [-73.91905683743724, 40.59377794608303], [-73.91904140279738, 40.59376539915938], [-73.91900670477759, 40.59373462369858], [-73.91896976736547, 40.59370128613031], [-73.91893059566054, 40.593662821414355], [-73.9188835901415, 40.59361580833309], [-73.9188688709467, 40.59359871916609], [-73.91885561687559, 40.59358333011416], [-73.91883493800617, 40.59355663931501], [-73.91882317352201, 40.59354145335177], [-73.91881163274063, 40.59352412115971], [-73.91879073786008, 40.59349274323214], [-73.91878459182179, 40.593482959699905], [-73.91877731813608, 40.59347138029359], [-73.91875127627858, 40.59342050785744], [-73.9187471353387, 40.593412421472536], [-73.91873259542515, 40.59339020249929], [-73.91871581289689, 40.593369691909764], [-73.9187111966399, 40.59336518713153], [-73.91869566731967, 40.59335003149475], [-73.91867439973275, 40.59333207857866], [-73.91865648749493, 40.593319252588145], [-73.91864938775757, 40.59331446945863], [-73.91863745400322, 40.59330642697359], [-73.91859266312099, 40.593282479268254], [-73.91856448495857, 40.59327011322934], [-73.9185400270365, 40.59325937904349], [-73.91846946904094, 40.59323199575791], [-73.91845419250951, 40.59322676319679], [-73.91843698766141, 40.593220868864314], [-73.91838825446025, 40.59320736892711], [-73.91835969863051, 40.59319946002227], [-73.91833601898574, 40.59319068727656], [-73.91832273846754, 40.593185767385805], [-73.91831043438131, 40.59318030907836], [-73.91828802138625, 40.59317036736588], [-73.91825890692178, 40.5931558250503], [-73.91825864842129, 40.5931556888327], [-73.91825802733138, 40.593155350149104], [-73.91823203190397, 40.59314128599627], [-73.91821261029659, 40.59312857244701], [-73.91820851885728, 40.59312589392409], [-73.91820047103653, 40.59312025971295], [-73.91818165129382, 40.59310708336827], [-73.91817732323076, 40.59310363504601], [-73.9181559042382, 40.593086565307736], [-73.91813127757429, 40.593066047872085], [-73.91810889365574, 40.59304553152686], [-73.91809745143634, 40.59303370920417], [-73.91808986807042, 40.5930258723057], [-73.91804511790264, 40.592970317495016], [-73.91804476691296, 40.59296997632252], [-73.9180372829567, 40.59296262649914], [-73.91802720748579, 40.59295493222924], [-73.91800056987351, 40.59293955932895], [-73.91797682082668, 40.59292585406084], [-73.91796562587395, 40.59291815898637], [-73.91795667035201, 40.592910465509625], [-73.91794892365819, 40.59290232877514], [-73.91793876571165, 40.59289166129555], [-73.9179208625227, 40.59287114811292], [-73.91790088378806, 40.59284586958068], [-73.91788506186869, 40.59282585142069], [-73.91786089090307, 40.592788229842974], [-73.91785486631765, 40.59277885210294], [-73.91778329999832, 40.5926609259451], [-73.91777407793404, 40.592647538443856], [-73.91776092863202, 40.59262845163156], [-73.9177487889554, 40.59261185385344], [-73.91773967608387, 40.5925993947076], [-73.91767143080135, 40.592513929328334], [-73.91762241276545, 40.59245520431751], [-73.91762220143639, 40.59245495655773], [-73.91762199009659, 40.59245471767684], [-73.91757632591728, 40.59240281969285], [-73.91753156213116, 40.59235666239407], [-73.91752270311702, 40.592349045680784], [-73.91752260805403, 40.592348968213955], [-73.91751141325035, 40.59234127309487], [-73.9174712855077, 40.59231627152164], [-73.91746886335424, 40.592314766708846], [-73.91745319419307, 40.59230364892901], [-73.91743976237933, 40.59229082596563], [-73.91743720400157, 40.59228769836504], [-73.91743416901262, 40.59228398909525], [-73.91743081571296, 40.59227715433682], [-73.91739059704649, 40.59218145813764], [-73.9173783961284, 40.59214931104008], [-73.91737049255926, 40.59212848548096], [-73.91736754058746, 40.59212216482172], [-73.9173593183346, 40.59210456095687], [-73.9173567809973, 40.592099712340655], [-73.91735484702652, 40.59209601642528], [-73.91734813434157, 40.59208746977875], [-73.9173470150399, 40.592086614239044], [-73.91734615416519, 40.59208630452812], [-73.91731149774752, 40.59207384791265], [-73.91729184574098, 40.59206727365566], [-73.91728393191366, 40.59206114593453], [-73.91727656176853, 40.59205219372457], [-73.91727322348392, 40.5920414211951], [-73.91727190030073, 40.59203714824434], [-73.91726735796777, 40.592011975700956], [-73.91726698858045, 40.59199048309584], [-73.91726390596146, 40.59197911803125], [-73.91725387046417, 40.59197216637892], [-73.91723104894635, 40.59196638643098], [-73.91716471428259, 40.59194575262577], [-73.91710528993583, 40.59192882817625], [-73.91707540693278, 40.59191724826304], [-73.91707030734244, 40.59191527158978], [-73.91704490749379, 40.59189930597031], [-73.91703487550618, 40.59188831952458], [-73.91700972126809, 40.591848035682645], [-73.91699331669324, 40.591836604363515], [-73.91696966389631, 40.591806065103945], [-73.91694809719017, 40.59177635142395], [-73.91693014832455, 40.59176124204538], [-73.9169153646122, 40.591749400602176], [-73.9169084937787, 40.59174368757418], [-73.91690749444442, 40.591737208659325], [-73.91691067278238, 40.59173163943408], [-73.91691557738284, 40.591723051237935], [-73.91693431558186, 40.59171098876717], [-73.91695572071207, 40.59169689541954], [-73.9169750017462, 40.59168197762907], [-73.91697824907432, 40.59167712112166], [-73.91697883177392, 40.591672262689], [-73.91696929521748, 40.59166654656599], [-73.91694002403251, 40.591666389880295], [-73.91689804588674, 40.591661308016995], [-73.91689171821308, 40.5916547971035], [-73.9168860982938, 40.59162718202434], [-73.9168714828245, 40.59159708787204], [-73.91685466142087, 40.59157226158969], [-73.91683837899292, 40.591550704192514], [-73.91683475739015, 40.59153810252669], [-73.9168285112746, 40.59152346867303], [-73.91682064118416, 40.59151124857899], [-73.91679898924541, 40.591491689934934], [-73.91677733382835, 40.591476166731226], [-73.91676254866707, 40.591464325937054], [-73.91675251814394, 40.59145331199244], [-73.91674198048536, 40.59144312058357], [-73.91672819934682, 40.59143575455125], [-73.91671383362063, 40.59143527807313], [-73.91670116708866, 40.5914332245311], [-73.916695798276, 40.591432353842194], [-73.91667518648008, 40.59141724249276], [-73.91663666876241, 40.59138092937334], [-73.91661805304851, 40.59135899314295], [-73.91660452811988, 40.591343055809965], [-73.9165856229251, 40.59131537394508], [-73.91658262287172, 40.591295910049716], [-73.91658296231469, 40.59125534323072], [-73.9165895000649, 40.591237533072494], [-73.9165870007566, 40.5912213117363], [-73.91657705029935, 40.591202200135236], [-73.91656176922584, 40.59118508781114], [-73.91655181759432, 40.59116798103086], [-73.91655843588346, 40.59114410603504], [-73.91656556230647, 40.59111737687511], [-73.91656464778491, 40.59110074272386], [-73.91655940040121, 40.59109258743761], [-73.91655148738137, 40.591086487643445], [-73.91653349833246, 40.59107543992184], [-73.916515885389, 40.591079790931566], [-73.91650784540845, 40.591087883291635], [-73.91649447616238, 40.591097946949525], [-73.91647682208814, 40.5911063606777], [-73.91646245643696, 40.59110588249288], [-73.91644709062327, 40.591098928066856], [-73.91643813849961, 40.591088326665556], [-73.91642922308051, 40.591077724117596], [-73.91641498243874, 40.591063057621426], [-73.91639753524665, 40.59105121484327], [-73.91637259668555, 40.59104054685506], [-73.91634498527034, 40.59103594348855], [-73.91633765358978, 40.59103612260055], [-73.91631783688462, 40.59103660992124], [-73.916297637387, 40.591036896594304], [-73.91627585918232, 40.591031527830346], [-73.91626238685089, 40.59102213352129], [-73.91625733271275, 40.591009221030916], [-73.91625681924144, 40.59100282421826], [-73.91625612032115, 40.59099412591964], [-73.9162446648196, 40.5909848495112], [-73.91622395296595, 40.59097506242519], [-73.91621251436143, 40.590962133718165], [-73.91620682762219, 40.59094775890301], [-73.91619091467014, 40.59093554881502], [-73.91617051377338, 40.59092770964246], [-73.91615011770604, 40.59091865336889], [-73.91613262466446, 40.590902789477134], [-73.9161122698496, 40.59088399675563], [-73.91609926411591, 40.590863763618216], [-73.91609259372333, 40.59085571482529], [-73.91607925140498, 40.59083961723719], [-73.91605602036711, 40.59082154830961], [-73.9160273196699, 40.59081295804486], [-73.91599381110436, 40.590809223614706], [-73.91595519207301, 40.590805964235926], [-73.91593573367871, 40.5908015352684], [-73.9159143732943, 40.590793937428145], [-73.91590163861409, 40.59078538679159], [-73.91590103695928, 40.590776622722714], [-73.91589154083256, 40.590757127751594], [-73.91587977537915, 40.590745901707976], [-73.91584504731276, 40.59072829038698], [-73.91581825525529, 40.590721653519765], [-73.91578440969224, 40.59072205575175], [-73.9157639779409, 40.59072151875405], [-73.91572984627538, 40.590713890072635], [-73.9157129766521, 40.59070740821815], [-73.91569924434596, 40.59070213085369], [-73.91568618670998, 40.59069406528883], [-73.91568430582039, 40.59068602936783], [-73.91568656639039, 40.59068019279565], [-73.91568148602673, 40.590673364926744], [-73.91567001088868, 40.59066895550933], [-73.9156604731002, 40.59065919679231], [-73.9156589448785, 40.5906433713494], [-73.91565291595063, 40.59063434965595], [-73.91563604009976, 40.59062359963988], [-73.91559140326441, 40.59060888391547], [-73.91554898746243, 40.59059685295645], [-73.9155301632934, 40.59059388457218], [-73.91552281863378, 40.59059411086591], [-73.9155071807511, 40.590592368454764], [-73.9155019977137, 40.59058872289026], [-73.91549954144826, 40.590586994023646], [-73.91550277477633, 40.590577266630824], [-73.91549995436303, 40.59056460168198], [-73.91548911690852, 40.59056043697325], [-73.91548049738573, 40.59056017431362], [-73.91546805766974, 40.590557221169355], [-73.91545150377355, 40.590545983852465], [-73.91544164940774, 40.59053549412119], [-73.91543403496343, 40.59052403503735], [-73.91542160053244, 40.590519865796246], [-73.91540945612026, 40.59052251387243], [-73.91539254361412, 40.590520280818936], [-73.91538043592861, 40.590514409045554], [-73.91536421759947, 40.590499034992895], [-73.91534736250536, 40.590483416211086], [-73.91533015912064, 40.59047436767196], [-73.91531739818042, 40.59047190161762], [-73.91529822663234, 40.59047550511644], [-73.91527428392196, 40.5904742293195], [-73.91525101186438, 40.590465896502415], [-73.91520164257466, 40.590437781822295], [-73.9151783846261, 40.59042579786577], [-73.9151567040808, 40.59041868598575], [-73.91514265685389, 40.590418163978185], [-73.91513242627919, 40.5904215457093], [-73.91512345801782, 40.590427853267194], [-73.91511643660108, 40.59042759226316], [-73.91509954328245, 40.59042049211745], [-73.91507343618841, 40.59040314492715], [-73.91504419876621, 40.59037118752542], [-73.91502991287625, 40.59035167955904], [-73.91501972942815, 40.590344108412275], [-73.91499456432393, 40.59033017263103], [-73.91497578584391, 40.59031625245941], [-73.91496944173487, 40.590306500403514], [-73.91496407656578, 40.59029163992844], [-73.91496060892453, 40.59028116543617], [-73.91495457131008, 40.590274577900985], [-73.91493577149045, 40.59026552527297], [-73.91491473902336, 40.590256224553016], [-73.9149007396338, 40.59024474914581], [-73.91488771456264, 40.5902293834213], [-73.91488456355718, 40.590219639254286], [-73.91488937785802, 40.59021356585976], [-73.91490185955986, 40.59020678229064], [-73.91491911877063, 40.590202443998365], [-73.9149335231024, 40.59019396058867], [-73.91493443939666, 40.5901919680025], [-73.91493900169378, 40.59018204727478], [-73.91493875864109, 40.5901642772542], [-73.91493149706875, 40.590145030794716], [-73.91490766991242, 40.59011697995945], [-73.91485837901661, 40.59007061286331], [-73.91481733329107, 40.59003716401749], [-73.91476036036907, 40.589995158004406], [-73.91470241341574, 40.58995704412016], [-73.91468712274471, 40.58994873085014], [-73.91466350346147, 40.589946968157875], [-73.91464529511366, 40.589949112579674], [-73.91461045994836, 40.589957059253685], [-73.91458713986239, 40.58995967916908], [-73.91456415888068, 40.58995816118874], [-73.91454758323465, 40.58995179078809], [-73.91454059121104, 40.58994422917544], [-73.91454447795088, 40.58993109541749], [-73.91456114732735, 40.58991555907641], [-73.91457011491376, 40.58990925156168], [-73.91457974374597, 40.589897105301254], [-73.91458331076498, 40.58988445831207], [-73.91458333215583, 40.589879589589174], [-73.91457227110722, 40.589852787722336], [-73.91454321966893, 40.58977701838073], [-73.91453482435507, 40.58972442231839], [-73.91453781571583, 40.58969692543289], [-73.91454975235602, 40.58966799042616], [-73.9145651300669, 40.589655615073355], [-73.91457952837368, 40.589648348804126], [-73.9145849811801, 40.58964252065709], [-73.9145856549886, 40.58963424671419], [-73.91457929644203, 40.589628145760756], [-73.91456756604065, 40.58960840023556], [-73.91456705345101, 40.589578947015895], [-73.91457292166801, 40.589550727703354], [-73.91457938937448, 40.58953127079049], [-73.91458236504873, 40.58950742619976], [-73.91458120974193, 40.58947894467517], [-73.91457013332167, 40.58945579393127], [-73.91457116498613, 40.58943851454465], [-73.91458143799926, 40.589425396589455], [-73.9145891302395, 40.58941860061435], [-73.91459902334721, 40.58941935514877], [-73.91461210133193, 40.58942255211448], [-73.91462008284306, 40.589422572285436], [-73.91462521460305, 40.589417229230484], [-73.91462335306716, 40.58940432456643], [-73.91461767341197, 40.589388733747064], [-73.91461455401743, 40.58937168732432], [-73.91461687594222, 40.589351246276095], [-73.91462236929549, 40.58933568352719], [-73.91461921395867, 40.58932715577965], [-73.91461314912446, 40.58932665387823], [-73.91460418691328, 40.58933174379697], [-73.91459071646592, 40.58934607044759], [-73.91458106544532, 40.589363085432126], [-73.91456983819307, 40.58937522665172], [-73.91454905473766, 40.58938247778482], [-73.91453148417652, 40.589384868093454], [-73.9145082017845, 40.58937896864891], [-73.91449040241481, 40.58935993947872], [-73.91448014568856, 40.589294439749025], [-73.91447191089588, 40.58927908524732], [-73.91445507154582, 40.589259813533374], [-73.91443404829512, 40.589248078696556], [-73.91440312160414, 40.58923802115436], [-73.91437694166262, 40.58923771349842], [-73.91435841159172, 40.58924034514562], [-73.91433571458899, 40.58924686137036], [-73.91429566541713, 40.58927962035802], [-73.91427801098399, 40.589301482673804], [-73.91427056247792, 40.589326047479624], [-73.91426381938237, 40.5893350360962], [-73.91425965751901, 40.58933770474941], [-73.91424721366302, 40.58933597007819], [-73.91423671673226, 40.58932645042781], [-73.91422754452432, 40.589306468549864], [-73.9142206451326, 40.58927700017323], [-73.91421687783043, 40.58926214303232], [-73.91421185535758, 40.589241928201965], [-73.91420586362733, 40.5892243877569], [-73.9142056620668, 40.58919688363613], [-73.91421550107218, 40.589136057374354], [-73.91422882674009, 40.58908083807475], [-73.91425076198561, 40.58902831722852], [-73.91426299448393, 40.589004981303084], [-73.91427808727641, 40.58898457290839], [-73.91430342280617, 40.58895834933212], [-73.9143124426024, 40.588939872540664], [-73.91431473855607, 40.588925517318664], [-73.91431860453066, 40.58891725229077], [-73.91434427671781, 40.58888688930503], [-73.91440356140954, 40.58883592346303], [-73.91444263086916, 40.58880827117578], [-73.9144822973293, 40.588790601726366], [-73.91451559869842, 40.58876804773154], [-73.91453034938282, 40.58875299362349], [-73.91454699795425, 40.58874232600697], [-73.91457898189549, 40.58872901662642], [-73.914714923843, 40.588670937662435], [-73.91481632371293, 40.58862640331883], [-73.91493950490353, 40.58856585747423], [-73.91503100554014, 40.58852178603493], [-73.91519736488564, 40.58844309137683], [-73.91531479153443, 40.58838399235684], [-73.91548337245831, 40.58830798006198], [-73.91560044596098, 40.58825666809438], [-73.91574214296386, 40.58819592473776], [-73.91589662195774, 40.58813277847436], [-73.91605017778171, 40.58806135353307], [-73.9161180019858, 40.588028905209704], [-73.91629459488155, 40.587945609540284], [-73.91634791504127, 40.58792177476038], [-73.9163574633772, 40.58791638860476], [-73.91636700973292, 40.587911003116986], [-73.91638561041509, 40.5878930382358], [-73.91639678848273, 40.58788022701922], [-73.9164143062014, 40.58786385387987], [-73.91642231217018, 40.587853840607245], [-73.91643920245625, 40.58785031266023], [-73.91645091822839, 40.587838737526255], [-73.9164547143442, 40.58782509920444], [-73.91645584254458, 40.5878154377661], [-73.91646960332542, 40.58781069981946], [-73.9164774856515, 40.587812736675964], [-73.91648378328203, 40.5878151841674], [-73.9164911196411, 40.58782205150422], [-73.91650162698664, 40.58782612139334], [-73.91652639955635, 40.58782465776247], [-73.91654700076931, 40.587819539458216], [-73.91656706158336, 40.58781321303844], [-73.9165930400693, 40.58779808104179], [-73.91666397617632, 40.58776308083631], [-73.91671105957896, 40.5877420349171], [-73.91674383919741, 40.58772855479769], [-73.91678987631491, 40.587709099449924], [-73.91683061848151, 40.587691618270895], [-73.91688299692045, 40.5876685992666], [-73.91693796566116, 40.587647585260235], [-73.91698977173137, 40.58762320662742], [-73.91701314900615, 40.5876122064447], [-73.91708037510332, 40.58757879416149], [-73.91713793359288, 40.5875463383291], [-73.9172548455425, 40.58747791255727], [-73.91736664654539, 40.58740469057485], [-73.91747299849544, 40.587326893829655], [-73.917573579769, 40.58724475760418], [-73.91766808619886, 40.587158530303896], [-73.9177562319928, 40.58706847270638], [-73.91783775059895, 40.58697485717178], [-73.91791239551053, 40.58687796681936], [-73.91797994101194, 40.586778094671175], [-73.9180401828607, 40.58667554276565], [-73.91809293895267, 40.58657062112203], [-73.91819087799986, 40.58660685499996], [-73.91972510799992, 40.587987277999886], [-73.92095992099988, 40.58909565099986], [-73.922307925, 40.590308207999875], [-73.92332177899988, 40.591211915999935], [-73.92405498699993, 40.591871702999875], [-73.92440413399991, 40.59218425199987], [-73.9254714779999, 40.59314878799995], [-73.92653305100005, 40.59410431499987], [-73.92759936700004, 40.5950607789999], [-73.92866850999998, 40.59602045099991], [-73.92956858299992, 40.5968202649999], [-73.929124371, 40.5971073039999], [-73.92879010199995, 40.59732329499992], [-73.92813139699999, 40.59777792799987], [-73.92992224899984, 40.599437382999895], [-73.93173390199983, 40.601062068999894], [-73.9335511719998, 40.60268728099991], [-73.93426266699994, 40.60223071099988], [-73.9360940729999, 40.603873161999886], [-73.93791557899979, 40.60549790099989], [-73.9384349359999, 40.605973862999896], [-73.93972613899992, 40.60713291499995], [-73.93939269099997, 40.60735090899988], [-73.93933968999978, 40.60738555999993], [-73.93901246899993, 40.60759947699994], [-73.93829679599983, 40.60806269399995], [-73.93753749399987, 40.60855738999986], [-73.93709375999993, 40.60884198299992], [-73.93676116199995, 40.60905528699991], [-73.936046277, 40.609514868999966], [-73.93572378299984, 40.60922748599993], [-73.93531145199985, 40.60886003999987]]], [[[-73.91990064335972, 40.59960052259278], [-73.91996319600395, 40.599568698599455], [-73.91998356109615, 40.599552134089706], [-73.91998771390962, 40.59954305170421], [-73.91998332969219, 40.59953596652177], [-73.91997188266981, 40.599531347812], [-73.91991463613738, 40.59952439215991], [-73.91986505165498, 40.59952240822561], [-73.91981967414765, 40.59952610895855], [-73.91977249931006, 40.59952733920583], [-73.919690043808, 40.59951951328441], [-73.91966742919745, 40.59952084240323], [-73.9196387882035, 40.59953092271564], [-73.91955859161355, 40.59956221383823], [-73.91951280094558, 40.599581339915694], [-73.9194856681038, 40.59959433072599], [-73.9194634744964, 40.59960356513311], [-73.9194367504255, 40.599607828281286], [-73.91939817858747, 40.59961191843318], [-73.91935694674213, 40.599611367989176], [-73.91933109688695, 40.59960734355337], [-73.91928832602004, 40.5995990236748], [-73.91926585786376, 40.59959771741539], [-73.91923769619382, 40.59959830190617], [-73.91920264897199, 40.5996054143509], [-73.91917567513987, 40.599608276885995], [-73.91915856902736, 40.59960928091884], [-73.91914776993256, 40.599605979027366], [-73.91914244716914, 40.59959990871454], [-73.91914116057045, 40.59959170139073], [-73.91913595045415, 40.59958299675999], [-73.91912742272757, 40.5995769241914], [-73.91911572215265, 40.59957463722451], [-73.91909908366019, 40.59957663033501], [-73.91908229418473, 40.59958260215572], [-73.91907123177553, 40.599589648518695], [-73.91905614206664, 40.59959079066001], [-73.91903590917147, 40.599586769611356], [-73.91901705219169, 40.599577286233526], [-73.91899643132949, 40.59956533122335], [-73.91897393464515, 40.59955730110181], [-73.91896144073314, 40.59955638595922], [-73.91894948394368, 40.59955695449778], [-73.91894166672142, 40.59955892582915], [-73.91893450034259, 40.59955977165504], [-73.91892780618915, 40.599556719073874], [-73.91892306220868, 40.599548975374844], [-73.91892389949841, 40.599542004739845], [-73.91893179296305, 40.59953498363031], [-73.91893936296279, 40.59952971850089], [-73.91894466693066, 40.59952047328899], [-73.9189433474669, 40.59950984832563], [-73.9189367403538, 40.599495572353945], [-73.91892365348792, 40.59947780318228], [-73.91890901034738, 40.599466512271036], [-73.9188916996778, 40.599458622496165], [-73.91887261711781, 40.5994563034897], [-73.91885687930014, 40.59945670399817], [-73.91884394589192, 40.59946152456533], [-73.91883561589877, 40.599471099175894], [-73.91883409341436, 40.59947996324421], [-73.91883099294384, 40.5994835019519], [-73.91881748280643, 40.59948868062133], [-73.91880275327955, 40.599488615768344], [-73.91878975891339, 40.59948481707567], [-73.91877290619072, 40.59948351569318], [-73.91875262975405, 40.59948695983317], [-73.91872292022468, 40.59948784515845], [-73.91869397302803, 40.59948304926218], [-73.91865887598058, 40.59947242837068], [-73.91862464667487, 40.599458378257275], [-73.91860193800093, 40.599447081595414], [-73.91859431220341, 40.59944131268618], [-73.91858809185996, 40.59943310018574], [-73.9185809834588, 40.59941596763278], [-73.9185740177184, 40.599398643024266], [-73.9185676996966, 40.59938258204469], [-73.91856022204298, 40.59937231172802], [-73.91854795461344, 40.5993623388988], [-73.91852481921545, 40.599347390787244], [-73.9185162581545, 40.59934024763766], [-73.91850101350444, 40.59932017235565], [-73.91845879275182, 40.59927394689382], [-73.9184567146926, 40.5992665356019], [-73.9184602497121, 40.59926154255631], [-73.91846550873942, 40.59926033856943], [-73.91847353794809, 40.59926113969438], [-73.91850866798785, 40.59927414695239], [-73.91851396151061, 40.59927349248356], [-73.91851518774826, 40.5992725607179], [-73.91851393301229, 40.599266768343], [-73.9185095875017, 40.59925836568167], [-73.9185056337515, 40.599251144546464], [-73.91850154385789, 40.59923985989337], [-73.91850202337783, 40.599230363839396], [-73.91849962223338, 40.59922042597848], [-73.91849250402807, 40.59921136195501], [-73.91844534064039, 40.59917314942299], [-73.91843548283042, 40.5991663908317], [-73.91842418174826, 40.599161550932635], [-73.91839995371869, 40.59915590793322], [-73.91838252841951, 40.59915254631499], [-73.91837341721651, 40.599152458089534], [-73.91836574328038, 40.59915662411631], [-73.91836476827352, 40.59915898340791], [-73.91836188575677, 40.599160958205026], [-73.91835846285828, 40.599161971335704], [-73.91834996643696, 40.59916023606274], [-73.9183307496015, 40.59914954459783], [-73.91826766956041, 40.59911274635402], [-73.9182424763976, 40.59910142155158], [-73.91822768169669, 40.5990964972257], [-73.91820399820863, 40.59908863028691], [-73.91818826639198, 40.59908420050544], [-73.91817444689813, 40.59907694385496], [-73.91816725209881, 40.59907106482224], [-73.91815912501023, 40.59906191830538], [-73.91814945079742, 40.59905172932431], [-73.91813441167821, 40.599041095248786], [-73.9181046173184, 40.59902367242492], [-73.91807651254089, 40.599008171185275], [-73.91804585326516, 40.59899099433261], [-73.91799442804891, 40.59896397740136], [-73.91796123355886, 40.59894806021192], [-73.91792064845438, 40.59893147951226], [-73.91788652720797, 40.59891712787], [-73.91786446493568, 40.598907505308006], [-73.91783715009365, 40.598894394211015], [-73.91779176637456, 40.59887322626767], [-73.91777298385558, 40.59886248174546], [-73.91775902524327, 40.598851189501225], [-73.91773905845905, 40.59883657286049], [-73.91773056760844, 40.59883036300761], [-73.9177162426041, 40.59882425988116], [-73.91770871940791, 40.59882131753943], [-73.91767804865373, 40.59881413168811], [-73.91766195936232, 40.598808795245105], [-73.91764666435176, 40.59880134582036], [-73.91763291686205, 40.598793541335525], [-73.9176158979434, 40.59878298853225], [-73.91760114715821, 40.598772849642174], [-73.91758866362653, 40.59876312416288], [-73.91757625561311, 40.59874955595091], [-73.91756133029189, 40.59873499834698], [-73.91754687082093, 40.59872159233633], [-73.91752841691161, 40.59870662040945], [-73.91751290831591, 40.59869667312017], [-73.91745827841918, 40.59866946031888], [-73.91743193445951, 40.59865766676976], [-73.91741142408016, 40.5986448053876], [-73.9173912030149, 40.598631533930664], [-73.91737141259789, 40.59861944201796], [-73.91735212322997, 40.59861042579631], [-73.91732851226746, 40.59860118683176], [-73.91730300654208, 40.59859247455115], [-73.91725216844844, 40.598575110073114], [-73.91722082113657, 40.598561363502576], [-73.91718598486707, 40.59854432061712], [-73.91716367077981, 40.59853412194051], [-73.91713135620083, 40.59851614704808], [-73.91711134867724, 40.598504440921204], [-73.91708785377304, 40.59848940882983], [-73.91706910902833, 40.5984767423239], [-73.91705557819927, 40.598468553613884], [-73.91703301312747, 40.5984587366974], [-73.91700990567614, 40.5984506514697], [-73.91698503186986, 40.598442344157654], [-73.9169657041723, 40.59843524806346], [-73.91694338962024, 40.5984260094537], [-73.91692359695185, 40.59841584103256], [-73.9168949456315, 40.59840654120745], [-73.91683159151566, 40.598387529390436], [-73.91676075195072, 40.5983655471843], [-73.9167168437384, 40.598347865831904], [-73.9166990656994, 40.598337505503245], [-73.91668204701033, 40.59832698003602], [-73.91666733468917, 40.59831492083318], [-73.91665388086415, 40.59830286086588], [-73.91663492212592, 40.59828865588008], [-73.91661660534268, 40.598279063269345], [-73.91659706414211, 40.598269470269734], [-73.91657399383551, 40.598260423698285], [-73.91654685169436, 40.59825269061], [-73.91651917038025, 40.5982456981055], [-73.91648413507367, 40.598243092531916], [-73.91645838918683, 40.59824172889399], [-73.91643318584649, 40.59823957021607], [-73.91638820908456, 40.598241294715784], [-73.91637413205817, 40.59823867637769], [-73.91636304525383, 40.59823531906491], [-73.91635253692556, 40.598230206633104], [-73.91634329033828, 40.59822320120079], [-73.91633635086238, 40.59821564795147], [-73.91632987687207, 40.598208286191465], [-73.91631995192462, 40.59819856226887], [-73.91630524090209, 40.59818554173761], [-73.91628872592389, 40.59817424862463], [-73.91626393108346, 40.59816113731273], [-73.9162427571722, 40.598105826845305], [-73.91624338937461, 40.59809961314378], [-73.91624301761748, 40.59809338648106], [-73.91624164529821, 40.59808724201592], [-73.91623929361789, 40.59808127425041], [-73.91623616293185, 40.59807327579766], [-73.9162337438657, 40.598065136301386], [-73.91623204956238, 40.59805689078467], [-73.91623108832657, 40.598048576612705], [-73.91623083342898, 40.598029089425175], [-73.91622985454572, 40.598009615447076], [-73.91622814769968, 40.59799017176382], [-73.91622571836957, 40.597970771949015], [-73.91622256719477, 40.59795143191843], [-73.91622033377298, 40.59793039364274], [-73.91621716882148, 40.59790942437992], [-73.91621307912544, 40.59788854775656], [-73.91620806619363, 40.59786778789787], [-73.9162021368083, 40.59784717010559], [-73.91619501706217, 40.597825339370765], [-73.91618718702189, 40.59780365085234], [-73.91617865392307, 40.597782118292876], [-73.91616941972902, 40.597760754258296], [-73.91615949277804, 40.59773956981153], [-73.9161493611691, 40.597716671892755], [-73.91613854295507, 40.59769395624642], [-73.91612704757107, 40.5976714361141], [-73.91611487786153, 40.597649121884345], [-73.9161020401838, 40.59762702679646], [-73.9160885433178, 40.597605160740876], [-73.91607538176632, 40.59758528417079], [-73.91606157945606, 40.59756566194317], [-73.91604714626143, 40.597546306629624], [-73.91603208942446, 40.59752722744943], [-73.9160164174995, 40.597508438313476], [-73.91600014145929, 40.59748995229667], [-73.91594641294235, 40.59743498904763], [-73.91593875715338, 40.597427744843735], [-73.91593052924073, 40.597420875151236], [-73.91592176104105, 40.5974144052903], [-73.91591248351033, 40.59740836158555], [-73.91590273640362, 40.5974027645043], [-73.91589255287641, 40.59739763903289], [-73.91588197356779, 40.59739300128343], [-73.915874244011, 40.597390587894814], [-73.91586622618826, 40.597388802864415], [-73.91585800601077, 40.59738766434827], [-73.91584967642903, 40.59738718615175], [-73.9158413264497, 40.597387372193175], [-73.91583867158363, 40.59738868552114], [-73.91583568395936, 40.59738948211257], [-73.91583253506784, 40.59738971367722], [-73.91582940272734, 40.59738936878566], [-73.91582646691447, 40.59738846700332], [-73.91582389767622, 40.597387060221045], [-73.91582183952798, 40.59738522795367], [-73.91582041124065, 40.59738307499418], [-73.915820555297, 40.59738125305151], [-73.9158214952229, 40.59737957509693], [-73.91582312088312, 40.59737823454589], [-73.91582524438459, 40.59737738974341], [-73.9158276154909, 40.59737713884606], [-73.91582996053887, 40.59737750946309], [-73.91583687009222, 40.59737695296551], [-73.91584364635065, 40.59737578371556], [-73.91585019918199, 40.59737401873499], [-73.91585643470812, 40.59737168241448], [-73.91585826736222, 40.5973689784939], [-73.9158593627108, 40.59736604954425], [-73.91585967137382, 40.59736301397252], [-73.91585918616788, 40.59735999105443], [-73.91585703476252, 40.59735158654673], [-73.91585409966335, 40.597343318169045], [-73.91585039795517, 40.597335232004326], [-73.91584594694979, 40.5973273687748], [-73.91584076989571, 40.597319767364304], [-73.9158348937766, 40.5973124668265], [-73.91582834821361, 40.59730550672027], [-73.91582116481455, 40.597298919401595], [-73.9157495649093, 40.597223687004295], [-73.91574424372803, 40.597217280469884], [-73.91573821833664, 40.59721124684144], [-73.91573153351578, 40.59720562937444], [-73.91572424173867, 40.59720047149736], [-73.91571639548738, 40.597195809434886], [-73.91570805163852, 40.59719168008486], [-73.91570226933402, 40.59718929994338], [-73.91569695610016, 40.597186352556726], [-73.91569220857743, 40.597182888924515], [-73.91568811152045, 40.59717897243522], [-73.91568473912828, 40.59717467266789], [-73.91568223654275, 40.59716852853891], [-73.9156789426856, 40.59716260513549], [-73.91567489088993, 40.59715696094965], [-73.9156701213025, 40.5971516539759], [-73.91566468287105, 40.59714673467651], [-73.91565862762218, 40.59714225167274], [-73.91565201637599, 40.597138252085216], [-73.91564282109223, 40.597132567433924], [-73.91563310734465, 40.59712740559522], [-73.91562292982492, 40.59712279341374], [-73.91561234036882, 40.597118756391886], [-73.91560139499292, 40.597115316181856], [-73.91559014950262, 40.59711248790232], [-73.91558087865403, 40.59710851701494], [-73.91557202032934, 40.5971040299373], [-73.91556362790365, 40.597099051837766], [-73.91555574309857, 40.597093611897435], [-73.91554841247242, 40.59708773812803], [-73.915541676642, 40.59708146406529], [-73.9155355744642, 40.59707482491902], [-73.91553013578316, 40.59706785723263], [-73.91552539527324, 40.5970606009037], [-73.9155213755222, 40.597053095318245], [-73.91548351001532, 40.59698422735675], [-73.91547435092545, 40.59696352154214], [-73.9154643275876, 40.596943048961904], [-73.91545344898562, 40.596922832071265], [-73.91544151292464, 40.59689314983561], [-73.91544059707802, 40.59689125742116], [-73.91543090572634, 40.59687747636852], [-73.91542043769203, 40.596864028461596], [-73.91540921250193, 40.59685094068697], [-73.91538570765951, 40.596822833080616], [-73.91536307918675, 40.596794310642146], [-73.91534133629257, 40.59676539029938], [-73.9153297919423, 40.59675114549202], [-73.91531746930818, 40.5967372850923], [-73.91530438923654, 40.59672383474726], [-73.91529057697453, 40.596710816421535], [-73.91527605403034, 40.59669825425498], [-73.9152608491663, 40.596686170884965], [-73.91525094460707, 40.596676961365844], [-73.91524180941101, 40.59666730226208], [-73.91523347892155, 40.59665722593314], [-73.91522598582947, 40.596646776463615], [-73.91521935953651, 40.59663599223946], [-73.91520347223593, 40.596618060246556], [-73.91518685632093, 40.596600516381514], [-73.91516952473685, 40.596583378244326], [-73.91515149921888, 40.59656666394374], [-73.91511318266967, 40.59653016603197], [-73.91507420366649, 40.596494077729304], [-73.91503457187356, 40.59645840390068], [-73.91502350117507, 40.59645010201071], [-73.91501168661961, 40.59644241825517], [-73.91499918969747, 40.59643539003826], [-73.91498606904307, 40.59642905409221], [-73.91498054457234, 40.59642568434199], [-73.91497565747576, 40.59642178533562], [-73.91497149030398, 40.59641742515101], [-73.91496812229593, 40.596412682920636], [-73.91493964092737, 40.59638334639756], [-73.91493012378385, 40.596376492096105], [-73.9149200834365, 40.596370086552646], [-73.91490955501757, 40.596364154922256], [-73.91489857718574, 40.59635871482398], [-73.91488718947119, 40.59635378990855], [-73.91487543339265, 40.596349394949385], [-73.91486335090366, 40.59634554890827], [-73.91481284944607, 40.59633162280349], [-73.9148069635742, 40.59632988317191], [-73.9148014723568, 40.59632750805907], [-73.91479649616662, 40.59632454932079], [-73.91479213624417, 40.59632106784538], [-73.91478848941418, 40.5963171400978], [-73.91476597964072, 40.596263627285836], [-73.91476050464186, 40.596253717721325], [-73.91475405756363, 40.59624415338347], [-73.91474667965433, 40.59623498908468], [-73.91473840952902, 40.59622627695512], [-73.91473210938248, 40.59622127019139], [-73.91472520510443, 40.596216749819206], [-73.9147177586143, 40.59621276111716], [-73.91470984041722, 40.59620933848096], [-73.91470152607288, 40.596206516142686], [-73.91469289246983, 40.59620432046137], [-73.91468402067956, 40.596202772438275], [-73.91467499376066, 40.59620188520234], [-73.91466363901262, 40.596200605391275], [-73.91465244289965, 40.59619867350541], [-73.91464147134002, 40.59619610115331], [-73.91463078827157, 40.59619290262249], [-73.91461719909148, 40.59618694238924], [-73.91460416462144, 40.5961803045756], [-73.91459173933087, 40.59617301719954], [-73.91457997899913, 40.59616511498144], [-73.91457185074239, 40.59615822884687], [-73.91456444944365, 40.596150882044014], [-73.91455782032091, 40.596143118164406], [-73.91455200243439, 40.59613498431266], [-73.91454540714601, 40.596128090436956], [-73.91453812803861, 40.59612161118993], [-73.91453020968119, 40.59611558212074], [-73.91452169796004, 40.596110040622364], [-73.91451264755382, 40.59610502241922], [-73.9145031109535, 40.5961005555275], [-73.91443985397817, 40.5960654299941], [-73.91443288710968, 40.5960609013487], [-73.91442641042542, 40.59605596848465], [-73.91442046212607, 40.59605066275853], [-73.91441508107104, 40.59604501653242], [-73.91441030018171, 40.5960390650121], [-73.9144061486421, 40.59603284440575], [-73.91440265387341, 40.5960263942708], [-73.91439983516479, 40.59601975466103], [-73.91439649157746, 40.59600835414823], [-73.91439239968507, 40.59599709631534], [-73.91438757505767, 40.595986008648765], [-73.91438202425076, 40.595975120303336], [-73.91437576239598, 40.5959644575924], [-73.9143688074807, 40.59595404750138], [-73.91436433605422, 40.59594723477415], [-73.91436067574172, 40.59594014606041], [-73.91435785789893, 40.59593283733824], [-73.91435590267103, 40.59592536675547], [-73.91435482800439, 40.5959177924583], [-73.91435464041491, 40.5959101749295], [-73.9143560521526, 40.59590495040827], [-73.91435655303478, 40.59589963088913], [-73.91435613855116, 40.595894306164574], [-73.91435481363932, 40.59588906821179], [-73.91435260279731, 40.59588400784973], [-73.91434954174001, 40.59587921037758], [-73.91434568376329, 40.5958747614424], [-73.91433382129797, 40.5958612207571], [-73.91432278175411, 40.595847282131885], [-73.9143125910306, 40.59583297255868], [-73.91430326644782, 40.595818324216005], [-73.91428110220451, 40.59577771098373], [-73.91423688589165, 40.595720092648975], [-73.91420936445135, 40.59568460816652], [-73.91418267734579, 40.59564875540079], [-73.91415683400918, 40.59561254591909], [-73.91410718137445, 40.59553984248572], [-73.91405688721092, 40.5954673960429], [-73.91400595151501, 40.595395210778236], [-73.91397259619512, 40.59533855454572], [-73.91393854415261, 40.59528213986122], [-73.91390379823967, 40.5952259717521], [-73.9138976159783, 40.59521113474642], [-73.91389057169397, 40.595196521081114], [-73.91388267633273, 40.59518216427002], [-73.91387395030226, 40.595168088955226], [-73.91386844511618, 40.59516164935874], [-73.91386363944557, 40.595154891478046], [-73.91385956751664, 40.59514786057181], [-73.91385625476713, 40.59514060021671], [-73.9138537235489, 40.59513316052115], [-73.91383346024931, 40.59506968556602], [-73.91382898589073, 40.59506029437178], [-73.91382370947856, 40.5950511491789], [-73.91381765162052, 40.59504228987472], [-73.91381083754081, 40.59503375500995], [-73.91380330081489, 40.59502558364391], [-73.9137950717308, 40.59501780746226], [-73.91377770759524, 40.59499804943177], [-73.91376127018658, 40.5949778359118], [-73.91374578320783, 40.59495719221758], [-73.9137351992186, 40.59494315285629], [-73.91372371955862, 40.59492952779164], [-73.91371136923529, 40.59491635339595], [-73.91369817809823, 40.59490366085192], [-73.91368464019415, 40.59489383413722], [-73.91367059780576, 40.59488443038825], [-73.9136560746552, 40.59487545984242], [-73.91364109137724, 40.59486694094281], [-73.91362567389052, 40.59485888476592], [-73.91360984591306, 40.59485130489927], [-73.91360323117576, 40.59484925456106], [-73.91359637356418, 40.5948477325947], [-73.91358934624068, 40.59484675631079], [-73.91358222589302, 40.59484633581883], [-73.9135750867971, 40.594846477038075], [-73.91356800390113, 40.594847175501506], [-73.91356105347381, 40.59484842573824], [-73.91355430761881, 40.59485021356228], [-73.91354783910492, 40.594852520935625], [-73.9135426389073, 40.59485414792102], [-73.91353715522014, 40.59485508145676], [-73.91353154060256, 40.59485529351242], [-73.91352596208925, 40.5948547793555], [-73.91352057636503, 40.594853549993246], [-73.91351554205342, 40.594851646254675], [-73.91350860144662, 40.594847519475174], [-73.9135022555663, 40.59484286994874], [-73.91349656939875, 40.59483774932377], [-73.91349160331386, 40.59483220991465], [-73.91348741306079, 40.59482630872336], [-73.91348404008961, 40.59482011229072], [-73.91347100128986, 40.5948048973463], [-73.91345712503106, 40.59479011985895], [-73.91344243589239, 40.59477580732157], [-73.9134350503717, 40.59476870974023], [-73.91342688750656, 40.59476212588778], [-73.91341800788979, 40.59475610590107], [-73.91340847783526, 40.59475069422487], [-73.91340112506057, 40.59474394043619], [-73.91339441539593, 40.59473680818051], [-73.91338838307945, 40.59472933333482], [-73.91338305751242, 40.59472155277809], [-73.91337671207945, 40.59470895096393], [-73.91337114602034, 40.594696136807386], [-73.91336637028736, 40.59468313879649], [-73.91336239693057, 40.59466998491807], [-73.91335923316663, 40.594656702652166], [-73.91332902450166, 40.59457167282163], [-73.91330050724518, 40.594461283001635], [-73.91329739741934, 40.59444811486208], [-73.91329339161908, 40.59443508777599], [-73.91328850694117, 40.59442223677016], [-73.9132827505952, 40.59440959585814], [-73.91327614121455, 40.59439720140774], [-73.91325680990705, 40.59433707512253], [-73.91324982633047, 40.594325138748296], [-73.9132420076644, 40.59431350865692], [-73.91323337694475, 40.594302215523214], [-73.91322395917989, 40.59429129504987], [-73.91321568749602, 40.59427870885973], [-73.91320854177944, 40.59426572882043], [-73.91320254876632, 40.59425241258229], [-73.91319569584229, 40.59424243606125], [-73.9131879710342, 40.59423283716218], [-73.91317941032952, 40.594223658632295], [-73.91317005235025, 40.594214944560704], [-73.91315993990132, 40.594206734348994], [-73.91311219156655, 40.59418111669704], [-73.9131053617516, 40.59417721061265], [-73.9130990532032, 40.59417282294015], [-73.91309332564262, 40.59416799711505], [-73.91308822934407, 40.59416277438753], [-73.91308381127278, 40.59415720505207], [-73.91308011158317, 40.59415133822508], [-73.91307588369632, 40.59414520043865], [-73.91307252694513, 40.59413875539214], [-73.91307008145094, 40.59413207699639], [-73.91306857370918, 40.594125239654595], [-73.91306801812564, 40.594118319603524], [-73.91306842449033, 40.59411139257373], [-73.9130634000125, 40.594072110687016], [-73.91306674591392, 40.594062746821784], [-73.91307078498586, 40.594053541628526], [-73.91307549850795, 40.59404452575074], [-73.91308087435495, 40.59403572782687], [-73.91308748220385, 40.59402425405111], [-73.91309338179742, 40.594012558600554], [-73.91309856211481, 40.59400066592642], [-73.91310301037561, 40.59398860265618], [-73.91310671753543, 40.593976394917675], [-73.91310967476895, 40.59396406917397], [-73.91311187786731, 40.59395165172404], [-73.91311413594114, 40.59390441485457], [-73.91311358259483, 40.593898981962894], [-73.9131121247657, 40.5938936465596], [-73.9131097854148, 40.59388849661484], [-73.91310660421107, 40.59388361676075], [-73.91310263291687, 40.59387908644552], [-73.91309793495263, 40.59387497775433], [-73.9130943311372, 40.59386908034234], [-73.91309148232251, 40.5938629462797], [-73.91308941502501, 40.593856633886666], [-73.91308814719179, 40.593850199634495], [-73.91308768841496, 40.59384370350564], [-73.91308804543189, 40.59383720296805], [-73.91308954844835, 40.59382774271889], [-73.9130903025535, 40.59381823097446], [-73.91309030418766, 40.59380870157294], [-73.91308955462387, 40.59379918969643], [-73.91308805667423, 40.59378973002564], [-73.91308581425143, 40.593780355901856], [-73.91308753818599, 40.59376106402651], [-73.91309004685282, 40.59374182266848], [-73.91309333561674, 40.593722647236795], [-73.91309740334691, 40.59370356135215], [-73.91310224584822, 40.59368457975377], [-73.91310785650141, 40.5936657217026], [-73.91311755291649, 40.59360174774314], [-73.9131188115122, 40.59359636229193], [-73.91311912036745, 40.59359089654621], [-73.91311847430313, 40.5935854464964], [-73.91311688681535, 40.59358011166508], [-73.91311438459752, 40.59357498354342], [-73.91310672252689, 40.59356148929389], [-73.91310267588344, 40.59355222085704], [-73.91310288411862, 40.5935300106469], [-73.91310256497235, 40.593525651124985], [-73.91309389861794, 40.59348982546132], [-73.91309391243468, 40.593485288275076], [-73.91309471978188, 40.5934807945765], [-73.91309630912718, 40.59347642443576], [-73.91309865070238, 40.59347225405626], [-73.91310170265045, 40.59346836097237], [-73.9131054101615, 40.59346481198675], [-73.91310970633701, 40.59346167523326], [-73.91312251746469, 40.593448409393204], [-73.91313472793975, 40.593434819766266], [-73.91314632499565, 40.59342092175523], [-73.91315729476963, 40.59340672892019], [-73.91316762383072, 40.5933922606841], [-73.91317730072866, 40.593377534461766], [-73.91318631401867, 40.593362563479566], [-73.91319188128803, 40.59335489266442], [-73.91319665185189, 40.593346916008585], [-73.91320059444199, 40.5933386807321], [-73.91320368767602, 40.5933302367425], [-73.91320591324575, 40.59332163629507], [-73.91320725394404, 40.59331292963591], [-73.91320770530692, 40.59330416970074], [-73.91320633592203, 40.593302314284315], [-73.9132042803228, 40.59330087114179], [-73.91320174007198, 40.593299982657896], [-73.91319896559207, 40.59329973630441], [-73.91319622563225, 40.59330015372812], [-73.91319399257524, 40.59329574819792], [-73.91319261072314, 40.59329114244283], [-73.91319211005799, 40.593286433652914], [-73.91319249858384, 40.59328171950357], [-73.91319377112114, 40.59327709598525], [-73.91319821480876, 40.59323806886176], [-73.91319833653526, 40.59323301626912], [-73.91319764464198, 40.59322799221203], [-73.9131961495811, 40.59322307074632], [-73.91319387213517, 40.593218325600546], [-73.91319084913009, 40.59321383051525], [-73.91318712553303, 40.593209650693474], [-73.91317870050439, 40.59320321656298], [-73.91317118195181, 40.593196160577854], [-73.91316464867866, 40.593188554165245], [-73.91316509019187, 40.593181964230084], [-73.91316637333283, 40.59317543976474], [-73.91316848659537, 40.59316904040104], [-73.91317141121499, 40.59316283062339], [-73.91317511613073, 40.59315686736827], [-73.91317956676434, 40.5931512075696], [-73.91318686755433, 40.593141672792555], [-73.91319341143057, 40.59313182332591], [-73.91319917702849, 40.593121695004704], [-73.91320414430363, 40.59311132333017], [-73.9132082951851, 40.59310074698814], [-73.91321161534506, 40.59308999913862], [-73.91321409220434, 40.59307912064933], [-73.91321571824042, 40.593068150381505], [-73.9132194198598, 40.59305524654973], [-73.91322230519827, 40.593042220810254], [-73.91322436872045, 40.593029105491986], [-73.91322560159801, 40.59301592973842], [-73.91322600686817, 40.593002724377286], [-73.91322557899919, 40.59298951872207], [-73.91322432146595, 40.592976344941064], [-73.913222236431, 40.592963230343045], [-73.91321932890631, 40.592950208437685], [-73.91321837160567, 40.59293379989187], [-73.91321661540135, 40.592917429609386], [-73.9132140622409, 40.592901120878395], [-73.91321071517604, 40.592884893804666], [-73.91320657966746, 40.59286877435919], [-73.91320166030528, 40.59285278214622], [-73.91319596453002, 40.59283694196596], [-73.91318854871699, 40.59282019750264], [-73.91318189134228, 40.592803269664756], [-73.91317599853208, 40.59278617939812], [-73.9131708761973, 40.59276894362762], [-73.91316653199776, 40.592751586651225], [-73.91316297118759, 40.59273412371785], [-73.9131562860421, 40.592669615850184], [-73.91315719168685, 40.592659825282425], [-73.91315900750725, 40.5926501096181], [-73.91316172772541, 40.59264051743607], [-73.91316533689628, 40.592631096135634], [-73.91316981451872, 40.592621893446974], [-73.9131801117352, 40.59260190425546], [-73.91318957392643, 40.59258167453008], [-73.91319819007435, 40.592561227381836], [-73.91320595268233, 40.59254058240629], [-73.91321285557164, 40.59251975953468], [-73.91321888794455, 40.59249878171076], [-73.91322404779922, 40.592477668198995], [-73.91323154928529, 40.592432178255464], [-73.91324027512778, 40.59238681773063], [-73.91325022046983, 40.59234160136324], [-73.91325425697755, 40.5923286878981], [-73.91325758511213, 40.5923156569626], [-73.91326019890842, 40.592302533346675], [-73.91326209482493, 40.59228933664858], [-73.91326326931149, 40.592276093335386], [-73.91326371992723, 40.59226282200077], [-73.91326344707652, 40.59224954961761], [-73.91326310633481, 40.592239141097345], [-73.91326180340411, 40.59222877624604], [-73.91325954327222, 40.59221850733644], [-73.91325634037796, 40.59220838480609], [-73.91325220674099, 40.59219846093276], [-73.91324371783442, 40.59217802381779], [-73.91323411215068, 40.59215787635474], [-73.91322340744581, 40.59213805357044], [-73.91321774738982, 40.592130790417], [-73.91321127587692, 40.592123931736296], [-73.91320404118419, 40.592117532514415], [-73.91319610149007, 40.59211163836298], [-73.91318751519033, 40.59210629656935], [-73.91317943909615, 40.592102883088366], [-73.91317103402652, 40.59209996775797], [-73.91316235094138, 40.592097569882654], [-73.91315344761742, 40.59209570609116], [-73.91314437568768, 40.59209438496666], [-73.91313519733325, 40.59209361610526], [-73.91312596704604, 40.5920934065844], [-73.91311674306567, 40.592093754772776], [-73.91308249063543, 40.592096374325415], [-73.91304815192677, 40.5920982482968], [-73.91301375793527, 40.59209937067968], [-73.91297933679013, 40.592099743170124], [-73.91294510479689, 40.592094434035815], [-73.91293872980943, 40.59209312482252], [-73.91293559105239, 40.59209234075243], [-73.91293275402934, 40.59209105348519], [-73.91293035161615, 40.592089325777565], [-73.91292850303765, 40.5920872399767], [-73.91292729629032, 40.59208489499163], [-73.9129267888009, 40.592082407298896], [-73.91292700612708, 40.59207989536096], [-73.91292793536515, 40.592077480124445], [-73.91292953548812, 40.59207527665066], [-73.91295122631982, 40.59205432996962], [-73.91298199276169, 40.5920338842014], [-73.91300897860152, 40.59201915467987], [-73.91304661071742, 40.5920037364541], [-73.91309803285056, 40.59198525012644], [-73.9131688710089, 40.591964663893584], [-73.91324094882303, 40.59194694632045], [-73.91329641035666, 40.59193539029229], [-73.91335188024114, 40.59192144845817], [-73.9134236499028, 40.59190301250723], [-73.91346218665768, 40.59189570961358], [-73.91347065385504, 40.591892390364734], [-73.91347506019093, 40.591886912289866], [-73.91347854059687, 40.5918780909722], [-73.91348546354506, 40.5918699946328], [-73.9134901741942, 40.59186642662111], [-73.91351840022506, 40.591854564005644], [-73.91363440815343, 40.591813090574156], [-73.91367483520123, 40.591802928968804], [-73.91369584515164, 40.591794866820265], [-73.91371184740139, 40.59178607734835], [-73.91372473055898, 40.59177489204053], [-73.91374261964579, 40.59176443622637], [-73.91376301345376, 40.59175374791542], [-73.91382323701731, 40.5917266927106], [-73.9139295519438, 40.59168209130958], [-73.91400292654167, 40.59165411305678], [-73.914047785409, 40.59163250769115], [-73.91408669435624, 40.591611603325816], [-73.91411744004438, 40.59159592931138], [-73.91422122921253, 40.591556333539884], [-73.91428863303828, 40.591533828689535], [-73.91434753262654, 40.59152299710505], [-73.91437041211175, 40.59151684991431], [-73.91439422677783, 40.59151165799815], [-73.91441866087001, 40.591507901594845], [-73.9144415190658, 40.59150652644781], [-73.91448160066902, 40.59150399978676], [-73.91453231311067, 40.591504364701976], [-73.9145654842131, 40.59150754914055], [-73.91459077606592, 40.59150874600867], [-73.91462401406288, 40.59151031935027], [-73.91466971094331, 40.59151234102924], [-73.91476051002702, 40.59150898680034], [-73.91479274617357, 40.59151121422223], [-73.91482496106988, 40.591518215368254], [-73.91484340298724, 40.5915247040461], [-73.91485433907953, 40.591529505164885], [-73.91486620116964, 40.59153764783905], [-73.91487868248505, 40.59154722435184], [-73.91489431446409, 40.59155203715426], [-73.91490745180603, 40.591554694014796], [-73.91492622875452, 40.59155593373022], [-73.91496940541205, 40.59156176873745], [-73.91501037646702, 40.59157093810925], [-73.91505290195363, 40.591582497751595], [-73.9151707793104, 40.59161643916384], [-73.91519173782501, 40.59162030998717], [-73.91520206369185, 40.59162129095316], [-73.91520629438757, 40.59162022574053], [-73.91521052507932, 40.591619163375654], [-73.91521524065092, 40.59161440131527], [-73.91522318452532, 40.59158697675797], [-73.91522916065571, 40.59158030964545], [-73.91524478304683, 40.59158750817847], [-73.91525180122734, 40.591630242720946], [-73.91524864533646, 40.59163620159027], [-73.91524390832531, 40.5916457357098], [-73.91524795391862, 40.59165147255335], [-73.9152573297239, 40.591655075150484], [-73.91529735058633, 40.59166686890398], [-73.91532923434016, 40.59167792487982], [-73.91536859466888, 40.59169783006022], [-73.91537950656559, 40.591708596663615], [-73.91539010272311, 40.59171983831438], [-73.91540228983338, 40.59172511843073], [-73.91542229154537, 40.59173280599863], [-73.9154447907877, 40.59174264564021], [-73.91545320606065, 40.59175125731214], [-73.91545911639327, 40.591760102686926], [-73.91547285817636, 40.591767772405554], [-73.91548850030455, 40.59177019717104], [-73.91549880725888, 40.591771557383005], [-73.9155050826615, 40.59177238607439], [-73.91552070534733, 40.59177958440247], [-73.91552850159098, 40.59178956148058], [-73.9155300466648, 40.59179153996536], [-73.91554218725159, 40.59180755933398], [-73.91555622818305, 40.591818333078535], [-73.91559773374159, 40.591849220635204], [-73.91563923277573, 40.59188130215355], [-73.91566887157266, 40.59190523989587], [-73.91569351970946, 40.59192487014006], [-73.91571036195032, 40.59193970716734], [-73.91572345856227, 40.59195191140187], [-73.91573749889469, 40.5919626846212], [-73.91576028801653, 40.59197801451728], [-73.91578588626045, 40.59199502142596], [-73.91581693482594, 40.59201295624624], [-73.9158414740981, 40.59202713681674], [-73.9158848745586, 40.5920539726193], [-73.91591233782945, 40.59207408685876], [-73.91594282730497, 40.592092782656636], [-73.91595741179528, 40.59210097227442], [-73.91596485962249, 40.592105862876934], [-73.91596584109006, 40.592107785491294], [-73.91596920508806, 40.59211437554015], [-73.9159654848041, 40.592123539857454], [-73.9159592077002, 40.59213363611337], [-73.91595375997414, 40.5921416475703], [-73.91595820693891, 40.592156472813755], [-73.91596453027877, 40.59216674229601], [-73.9159793868295, 40.592178446707926], [-73.91599809766262, 40.59218732654275], [-73.91601764425313, 40.59219219867833], [-73.91603722801986, 40.592195121801495], [-73.91604992895557, 40.59220259752877], [-73.91605675687171, 40.592213060200834], [-73.91605951520725, 40.59222384926242], [-73.91605928472141, 40.5922353498157], [-73.91605692678102, 40.59224992297811], [-73.91605485567764, 40.592264111535265], [-73.916060851914, 40.59227762146071], [-73.91607271953715, 40.59228910337583], [-73.91609232720587, 40.592302236750065], [-73.91609987302387, 40.59231346957348], [-73.9161023777214, 40.59232656415503], [-73.91609389321475, 40.59234546850413], [-73.91608090794963, 40.59236258655105], [-73.9160693358746, 40.59237490172501], [-73.91605729853244, 40.59238321076388], [-73.916038492975, 40.59239123927052], [-73.9160209498559, 40.5923983080836], [-73.91601348867175, 40.592404532208946], [-73.91600796522113, 40.592415426122685], [-73.91601046924518, 40.592428520705674], [-73.91601536006375, 40.5924343432443], [-73.91602654750764, 40.592442174035114], [-73.91604396293694, 40.59245102510567], [-73.91606440192857, 40.59246108593364], [-73.91608387128535, 40.59246977431967], [-73.91615216892161, 40.59250366644124], [-73.91616756691072, 40.592512708177104], [-73.91617734507624, 40.592526275637255], [-73.91618089572287, 40.59253687261481], [-73.91618772457261, 40.59254733527939], [-73.91619967037674, 40.592554013837045], [-73.91621295176718, 40.59255783883498], [-73.91623119728854, 40.592564603736875], [-73.9162453771656, 40.59257169570553], [-73.91625850431825, 40.59258320599391], [-73.91626562385585, 40.592590402546286], [-73.91626723431409, 40.592598281284076], [-73.9162635552358, 40.5926045653089], [-73.91625655485981, 40.59261485320969], [-73.91625229497743, 40.592624784594584], [-73.9162484985648, 40.59263683204502], [-73.91624945251799, 40.59265124294339], [-73.9162518590549, 40.592656075573146], [-73.91626390260747, 40.592665792172184], [-73.91626411548197, 40.59266603055409], [-73.91626880100847, 40.592670331263164], [-73.91627402851688, 40.592674251907894], [-73.91627974421226, 40.59267775458736], [-73.91628589099996, 40.592680803576094], [-73.91630517416277, 40.592693815385715], [-73.9163239447397, 40.59270725552634], [-73.91634218846329, 40.59272111025055], [-73.91635988820347, 40.59273537033198], [-73.91636444870188, 40.59274106797324], [-73.91636827963086, 40.592747071997174], [-73.91637134677796, 40.59275332525121], [-73.91637362076389, 40.59275977226213], [-73.91637508232326, 40.59276635371025], [-73.91637571922077, 40.59277301195688], [-73.91637390921291, 40.592788109220535], [-73.9163729318465, 40.59280325048028], [-73.91637278979103, 40.592818410440955], [-73.91637348374238, 40.59283356062327], [-73.91637341049615, 40.5928420348735], [-73.91637406096247, 40.5928504937351], [-73.91637543518614, 40.59285890202709], [-73.91637752617831, 40.592867224730554], [-73.91638032365744, 40.592875424311316], [-73.9163838190933, 40.592883468597634], [-73.91638799341142, 40.592891321221764], [-73.91639485647836, 40.5929003121534], [-73.9164010264833, 40.59290958805044], [-73.9164064832413, 40.592919121925846], [-73.91641121031341, 40.59292887875456], [-73.91641519169556, 40.59293882719668], [-73.91643201092783, 40.59299310351871], [-73.91643342713132, 40.59306930191187], [-73.91643113115212, 40.59307567423039], [-73.91642961583784, 40.59308218264789], [-73.91642889466654, 40.593088769711514], [-73.91642897496094, 40.59309537930433], [-73.9164298561325, 40.593101955303645], [-73.91643152880289, 40.593108440240016], [-73.91643398161207, 40.59311478032825], [-73.91643718935549, 40.59312092060032], [-73.91644112902213, 40.593126809440484], [-73.91644576551357, 40.59313239572667], [-73.91645105691396, 40.5931376316823], [-73.9164604226294, 40.59314372097866], [-73.91646921811346, 40.59315028480526], [-73.9164774023079, 40.59315729264223], [-73.91648493921161, 40.593164711795275], [-73.91649178886854, 40.59317250872964], [-73.91649792297807, 40.593180643887884], [-73.91650331125844, 40.59318907988913], [-73.91651428195776, 40.593207295114446], [-73.91652592370005, 40.59322526774106], [-73.91653822727343, 40.59324298285205], [-73.91655118236952, 40.5932604245244], [-73.91657651607466, 40.593292712416904], [-73.91660115824598, 40.593325308893306], [-73.91662510339982, 40.59335820507118], [-73.91664834560989, 40.59339139407792], [-73.91665387103079, 40.5934019681806], [-73.91666019740298, 40.593412277329776], [-73.91666730147057, 40.59342229001333], [-73.91667515910903, 40.59343196617362], [-73.91668374507812, 40.59344127949051], [-73.91669303041512, 40.59345019258349], [-73.91670267014563, 40.59345883084959], [-73.9167115691328, 40.59346792040453], [-73.91671968698407, 40.59347742503284], [-73.91672699122175, 40.593487306012044], [-73.91673345332673, 40.59349752244427], [-73.91673904632025, 40.59350803142278], [-73.91675388927177, 40.59352940601761], [-73.91676946488826, 40.59355047322224], [-73.91678576241264, 40.59357122364692], [-73.91680276978079, 40.59359163969211], [-73.91682047250607, 40.593611706604435], [-73.91683861744997, 40.593629454390594], [-73.91685613451003, 40.59364756408544], [-73.91687301557144, 40.59366602194602], [-73.91688924636479, 40.59368481523006], [-73.9169048159175, 40.59370393086264], [-73.91691971237843, 40.5937233556006], [-73.9169380100018, 40.593749855045], [-73.91695706909132, 40.59377604326464], [-73.91697687977835, 40.59380190366676], [-73.91699743042804, 40.593827426525614], [-73.9170187162244, 40.59385259692734], [-73.91702744789585, 40.59386535723883], [-73.917036957087, 40.593877789753485], [-73.91704722361061, 40.59388986882459], [-73.91705822442664, 40.593901566290455], [-73.91706993363321, 40.593912857002785], [-73.91708232664955, 40.59392371397133], [-73.91709537713228, 40.59393411405782], [-73.91710905280128, 40.59394403596259], [-73.91713242391876, 40.59396632852089], [-73.91715648940665, 40.59398818733887], [-73.91718123499682, 40.59400959933874], [-73.91720664817603, 40.59403055328651], [-73.91723271379725, 40.59405103560095], [-73.91726380465204, 40.5940777841082], [-73.91729430957737, 40.59410492303116], [-73.91732421979414, 40.594132442479726], [-73.91735352563852, 40.59416033758895], [-73.9173812292882, 40.594190964779415], [-73.91740967232887, 40.594221193439594], [-73.91743885037874, 40.59425101301163], [-73.91746875136242, 40.59428041376948], [-73.91749936364604, 40.594309385149906], [-73.91753067933053, 40.59433791675988], [-73.91755486430114, 40.594363431068714], [-73.91757838205497, 40.59438930675703], [-73.91760122161998, 40.59441552991266], [-73.91762337882922, 40.59444209349656], [-73.91766779440684, 40.594489151472224], [-73.91771131849671, 40.594536689098746], [-73.91775394649545, 40.594584696489314], [-73.91779072246149, 40.59462625767345], [-73.9178268959165, 40.59466812583093], [-73.917862462914, 40.5947102932533], [-73.91787543558024, 40.59472218152493], [-73.91788919657135, 40.5947335394579], [-73.91790371163492, 40.594744340725505], [-73.91791893971238, 40.594754553467645], [-73.91792445130464, 40.59476036262944], [-73.91793069331456, 40.59476572551208], [-73.91793760669219, 40.594770585448195], [-73.91794512270774, 40.5947748946429], [-73.9179531651492, 40.59477861249942], [-73.9179616525297, 40.59478169875255], [-73.91797050181093, 40.594784124360366], [-73.91803383971062, 40.594814362354946], [-73.91804703994809, 40.594825668772536], [-73.9180608211166, 40.59483656448606], [-73.91807516103935, 40.59484703289376], [-73.91809003643904, 40.59485705873409], [-73.91810542448017, 40.59486662490218], [-73.91812129858931, 40.5948757154639], [-73.91813784076567, 40.59488377116829], [-73.91815480335815, 40.59489130179787], [-73.91817216066987, 40.594898293094225], [-73.91818987864859, 40.59490473464592], [-73.918207930059, 40.59491061252868], [-73.91822628524082, 40.59491591918241], [-73.91834221191117, 40.594959918416315], [-73.91835270288986, 40.59496364569098], [-73.91836279205062, 40.59496797092646], [-73.91837241458613, 40.59497287296792], [-73.91838151998475, 40.59497832229455], [-73.91839005092659, 40.5949842851923], [-73.91839795822357, 40.59499072778556], [-73.91840519752516, 40.59499761419172], [-73.9184117231708, 40.59500490132322], [-73.91844064659385, 40.59505652405937], [-73.91847777885674, 40.595109414274674], [-73.91848190838297, 40.595116828029745], [-73.91848673204085, 40.59512399433376], [-73.9184922270216, 40.595130874471074], [-73.9184983634789, 40.59513743290424], [-73.91850510739273, 40.595143633255056], [-73.91851242429686, 40.59514944366872], [-73.9185202753283, 40.59515483396208], [-73.91852862030223, 40.59515977612917], [-73.91853741375868, 40.595164242662904], [-73.91854543930093, 40.59516764651593], [-73.91855384950556, 40.595170464120976], [-73.9185625679228, 40.595172669456815], [-73.91857151831273, 40.595174244208536], [-73.91858062355114, 40.59517517458422], [-73.91858980233157, 40.59517545164696], [-73.91859897399996, 40.59517507282669], [-73.91860806009106, 40.5951740422563], [-73.91861698059509, 40.59517236926053], [-73.91862616546092, 40.59516999718411], [-73.91863557006518, 40.595168195532175], [-73.9186451352711, 40.595166978670484], [-73.91865479689959, 40.59516635090902], [-73.9186644953832, 40.59516631940976], [-73.91867416347108, 40.59516688395733], [-73.91868374161186, 40.59516803864602], [-73.91869316783504, 40.59516977890888], [-73.91870237798345, 40.59517209046051], [-73.91871131514995, 40.59517496186866], [-73.91871991914397, 40.595178370474564], [-73.91873017339927, 40.595182525584526], [-73.91874002174238, 40.59518721348489], [-73.91874941782382, 40.595192415547004], [-73.91875831332682, 40.59519810409446], [-73.91876666762612, 40.595204251791635], [-73.91877443878163, 40.595210828285914], [-73.9187815888106, 40.595217802390344], [-73.91883082616154, 40.59526512952389], [-73.91883095718875, 40.59526871156018], [-73.91883027377388, 40.595272256834754], [-73.9188287940674, 40.59527565881198], [-73.9188265634708, 40.59527881265043], [-73.91882365132508, 40.59528162625802], [-73.91882014322945, 40.59528401174227], [-73.91881614564056, 40.595285898983654], [-73.91880918204893, 40.595290247312256], [-73.91880278708891, 40.595295077018946], [-73.91879701861106, 40.59530034726739], [-73.9187919274468, 40.59530600615954], [-73.91878755958476, 40.59531200816002], [-73.91879054354642, 40.595316466715296], [-73.9187941658632, 40.59532064025169], [-73.91879838286098, 40.59532447831192], [-73.91880313812132, 40.59532792858707], [-73.91880837367252, 40.59533095049392], [-73.9188140216552, 40.5953335022696], [-73.91882001089783, 40.59533555404361], [-73.91884599683691, 40.59534882647734], [-73.91887141082583, 40.59536272757385], [-73.91889622518737, 40.59537724592191], [-73.91890052068642, 40.59537924824792], [-73.91890515304664, 40.59538075291447], [-73.91891002230878, 40.5953817280204], [-73.91891502696288, 40.595382152050355], [-73.91892005888636, 40.595382018896856], [-73.91892501148821, 40.595381328484834], [-73.91892978123445, 40.595380096823995], [-73.91893452077733, 40.595373873774165], [-73.91894005455104, 40.59536804330292], [-73.91894633061598, 40.59536266518148], [-73.91895328033219, 40.59535779699134], [-73.91896083661622, 40.595353481404985], [-73.91896892270665, 40.595349767956954], [-73.91897330083931, 40.59534959865648], [-73.91897765653323, 40.59534999089809], [-73.91898186320948, 40.595350934373336], [-73.9189858022233, 40.595352401691365], [-73.9189893639569, 40.59535435139519], [-73.9189924469416, 40.595356727793465], [-73.91899496422666, 40.59535946448354], [-73.91899684447867, 40.59536248451919], [-73.91899803461551, 40.595365702255336], [-73.91899850156396, 40.59536902485768], [-73.91899822960946, 40.595372361849364], [-73.9189972289798, 40.595375615902874], [-73.9189936002051, 40.595389071642394], [-73.91899108397412, 40.59540267475423], [-73.91898968914384, 40.595416371467714], [-73.91899144585678, 40.5954221306876], [-73.91899405904522, 40.5954277012031], [-73.91899749210195, 40.59543300441697], [-73.9190017033528, 40.59543797211569], [-73.91900663155917, 40.59544253908685], [-73.9190122172361, 40.59544664430801], [-73.91901838396703, 40.59545023394794], [-73.91903518537742, 40.59545894239933], [-73.91905259799357, 40.59546692168885], [-73.91907056667698, 40.59547415100351], [-73.91908337195937, 40.595478832747695], [-73.91909582870996, 40.595484033083174], [-73.91910789892468, 40.595489738078484], [-73.91911954724104, 40.59549592978321], [-73.91913074203036, 40.59550259326509], [-73.91914145013469, 40.59550970487937], [-73.9191516399281, 40.595517246510674], [-73.91920661061333, 40.59555895121167], [-73.91922797092022, 40.59557345114395], [-73.91924989028144, 40.59558746010177], [-73.91927234651445, 40.59560096600699], [-73.91934995256923, 40.59565303331372], [-73.91935841663299, 40.595660103795794], [-73.91936748210831, 40.59566672622332], [-73.91937710903207, 40.595672870915486], [-73.91938725435921, 40.59567851204243], [-73.91939787065067, 40.59568362209594], [-73.91940891375984, 40.59568817792587], [-73.9194203294221, 40.595692162071025], [-73.91943207194953, 40.59569555355806], [-73.9194409479348, 40.59569784957085], [-73.91944950735554, 40.595700763879194], [-73.91945767332409, 40.595704268619286], [-73.91946538126052, 40.59570833526618], [-73.91947256021903, 40.59571292909149], [-73.91951047302153, 40.59574646904678], [-73.91952660766293, 40.59575964111161], [-73.91954342840789, 40.59577230419794], [-73.91956091022702, 40.59578443751435], [-73.91957915728098, 40.59579441006847], [-73.91959794087923, 40.595803782402626], [-73.91961723004908, 40.59581254142757], [-73.91963698811024, 40.595820667851584], [-73.91966624713781, 40.595840850960045], [-73.91969471112844, 40.5958616825138], [-73.9197223605453, 40.595883143569075], [-73.91973094952398, 40.59589130003547], [-73.91974017130073, 40.59589904348096], [-73.91974999206418, 40.59590634489922], [-73.9197603773393, 40.595913179639155], [-73.91977128891902, 40.59591951902648], [-73.91978407275373, 40.59592405459171], [-73.91979651486983, 40.59592911093314], [-73.91980857945877, 40.59593467596356], [-73.91982023116063, 40.59594072972267], [-73.91983143262976, 40.595947258112034], [-73.91984215554052, 40.59595424017083], [-73.91985236431644, 40.595961653257774], [-73.91986050188099, 40.59596854237572], [-73.91986921247154, 40.5959750113942], [-73.91987845722032, 40.59598103331365], [-73.91988819857727, 40.59598658130333], [-73.91989839371136, 40.59599163388922], [-73.9199090000144, 40.59599616742015], [-73.91991463502488, 40.59599825574648], [-73.91992055131291, 40.59599982610054], [-73.91992666802464, 40.59600085798744], [-73.91993290056156, 40.596001338280715], [-73.91993916387872, 40.59600125921493], [-73.91994537248216, 40.5960006212329], [-73.9199514406448, 40.59599943533209], [-73.91995728615548, 40.595997714019774], [-73.9199628254628, 40.59599548638811], [-73.91996798579088, 40.59599277751557], [-73.91997217930066, 40.595990999097296], [-73.91997667682188, 40.59598972146795], [-73.91998137524033, 40.595988976553876], [-73.91998616838778, 40.595988777851126], [-73.91999094482841, 40.59598913248555], [-73.91999559600244, 40.595990031670304], [-73.92000001753891, 40.595991456569415], [-73.92000410728487, 40.59599337042288], [-73.92000777078299, 40.59599573396309], [-73.92001092612406, 40.59599849084323], [-73.92001350063795, 40.59600157718424], [-73.92001543462604, 40.5960049255976], [-73.92001668708448, 40.596008456813756], [-73.92001821814745, 40.59601204970692], [-73.92002053170106, 40.59601539067809], [-73.9200235621528, 40.59601837866141], [-73.92002721751561, 40.59602093049816], [-73.92003139283277, 40.596022965031324], [-73.9200359617981, 40.59602442739062], [-73.92004079325876, 40.59602527174944], [-73.92004591909364, 40.59602493824694], [-73.9200510403239, 40.59602532997528], [-73.92005597806748, 40.59602643290499], [-73.92006056555643, 40.59602821090136], [-73.92006464528366, 40.596030602874265], [-73.9200680762517, 40.59603352512852], [-73.92006959498332, 40.59603787507715], [-73.92007023995146, 40.596042347887156], [-73.92006999325366, 40.5960468436986], [-73.92006886665922, 40.596051261499575], [-73.92006688182877, 40.59605549860948], [-73.9200640850284, 40.59605946074168], [-73.92006156134629, 40.59606255945037], [-73.92005970503378, 40.596065921644204], [-73.9200585623432, 40.59606946442845], [-73.92005815755803, 40.59607309768921], [-73.92005850682558, 40.59607673532772], [-73.92005959706685, 40.59608028737167], [-73.92006140223293, 40.59608366635251], [-73.9200638780322, 40.59608678612941], [-73.92006696565075, 40.5960895742889], [-73.92010096592858, 40.59611273607112], [-73.920104099939, 40.59611587825121], [-73.92010780825348, 40.5961186333353], [-73.92011200852703, 40.59612094078793], [-73.92011661027355, 40.596122748947174], [-73.92012151177822, 40.596124022058774], [-73.92012660538668, 40.59612472938959], [-73.92013178166758, 40.596124855451166], [-73.92013692435978, 40.59612439898984], [-73.92014235789823, 40.59612516200853], [-73.92014755503682, 40.596126592299235], [-73.9201523762624, 40.596128651568435], [-73.9201566897743, 40.59613128460758], [-73.92016038049596, 40.59613441846129], [-73.92016334764742, 40.596137971138084], [-73.92016551508158, 40.59614184558559], [-73.92016682094476, 40.59614593839513], [-73.92017033748222, 40.596153034027274], [-73.92017468291108, 40.59615985615716], [-73.9201798214724, 40.596166350648], [-73.9201857101547, 40.596172463022725], [-73.92019230374304, 40.59617814382866], [-73.920199545372, 40.596183345280544], [-73.92023160987183, 40.596197914443636], [-73.92023608923323, 40.596201727335966], [-73.92024108818286, 40.59620514538713], [-73.92024654479414, 40.596208126169365], [-73.92025239471131, 40.59621063680195], [-73.92025857071978, 40.59621264591043], [-73.92026499570562, 40.59621412931699], [-73.92027159562298, 40.596215070887304], [-73.92027640871201, 40.59621603818988], [-73.92028136826723, 40.596216374846115], [-73.92028632990525, 40.59621607137432], [-73.92029115471638, 40.59621513605456], [-73.92029570289029, 40.596213594924386], [-73.92029984712111, 40.59621149312845], [-73.92030346909885, 40.59620888922004], [-73.92031436616476, 40.59620390122158], [-73.92032581833102, 40.59619969864911], [-73.92033772885611, 40.5961963142711], [-73.92034331494288, 40.596195960300804], [-73.92034891109198, 40.596196198553145], [-73.92035441005862, 40.59619702359291], [-73.92035970724424, 40.59619842228043], [-73.92036469872751, 40.596200366231024], [-73.92036929048967, 40.59620281668045], [-73.92037339269461, 40.596205729003806], [-73.92037692695145, 40.596209044511404], [-73.92037982410287, 40.59621270250997], [-73.92038203104995, 40.59621662908213], [-73.92040460677839, 40.59624807321162], [-73.92042393591537, 40.59626406352911], [-73.92044395180272, 40.59627955642175], [-73.92046462962641, 40.59629453478409], [-73.92048594962638, 40.59630898117897], [-73.92049712078564, 40.596317445617856], [-73.92050881398065, 40.59632549159359], [-73.92052100044532, 40.596333098144974], [-73.92053365382506, 40.59634024950575], [-73.92054674359254, 40.59634692722666], [-73.9205602394341, 40.59635311738153], [-73.92057411125808, 40.59635880537442], [-73.92058357869517, 40.59636118563962], [-73.92059328162206, 40.596362922084374], [-73.92060314884166, 40.59636400544523], [-73.92061310036657, 40.59636442511241], [-73.92062306411049, 40.596364180031074], [-73.92063295831434, 40.59636327064729], [-73.9206427133015, 40.59636170294411], [-73.92065224971363, 40.59635949177685], [-73.92066630069343, 40.59635609292733], [-73.9206800923512, 40.59635212362763], [-73.92069358511307, 40.59634759524239], [-73.92070673786347, 40.59634252148065], [-73.92071951388583, 40.59633691387691], [-73.92073187491275, 40.596330793848416], [-73.92074378994201, 40.59632417276596], [-73.92074781350654, 40.59632133089407], [-73.92075236920975, 40.59631899817517], [-73.92075734401656, 40.59631723283175], [-73.92076261853461, 40.596316079344064], [-73.92076806416244, 40.59631556375809], [-73.92077354813901, 40.59631569871438], [-73.92077893925905, 40.59631648194469], [-73.92078410260292, 40.59631789308497], [-73.92078891821032, 40.59631989955179], [-73.92079326702036, 40.59632245133962], [-73.92079398510812, 40.59632554140927], [-73.92079547525519, 40.596328465654445], [-73.92079768374862, 40.596331112631205], [-73.9208005263122, 40.59633338176388], [-73.92080389184022, 40.59633518686576], [-73.92080765250182, 40.5963364603343], [-73.92081828920803, 40.596336025721676], [-73.9208294770879, 40.59633469068289], [-73.92083196259102, 40.59633707868207], [-73.92083361357264, 40.59633985594688], [-73.92083433001174, 40.5963428553818], [-73.92083406860239, 40.59634589920838], [-73.92083284385042, 40.59634880181437], [-73.92083321038464, 40.59637498689091], [-73.92083481154756, 40.596392349386576], [-73.92083729585357, 40.59640965268268], [-73.92084065586505, 40.59642686762391], [-73.92084489116979, 40.59644397176089], [-73.92084999213027, 40.59646093811481], [-73.92085595328383, 40.59647774037959], [-73.92086367015504, 40.596560830826114], [-73.92086732334573, 40.59657008933353], [-73.92087175325507, 40.59657914801057], [-73.92087694168622, 40.596587969485654], [-73.92088286582369, 40.59659651872934], [-73.920889501097, 40.596604758198104], [-73.92089682095298, 40.59661265403258], [-73.92090479224294, 40.59662017421186], [-73.92091148132826, 40.59662677678802], [-73.92091875480446, 40.5966330118716], [-73.92092657556758, 40.59663884727151], [-73.92093491024322, 40.59664425532217], [-73.92094371358773, 40.59664921019301], [-73.92095294738532, 40.59665369007873], [-73.92096256639095, 40.59665766948375], [-73.92097252446897, 40.59666113313091], [-73.9209827719717, 40.59666406172024], [-73.92099193374358, 40.5966660783707], [-73.92100080948812, 40.5966687321049], [-73.92100932472403, 40.59667200260107], [-73.92101740410085, 40.59667586132717], [-73.9210249801875, 40.59668027355864], [-73.92103198797186, 40.596685203064474], [-73.92103836464742, 40.59669060758424], [-73.92104405708474, 40.59669643816309], [-73.92107997175623, 40.59674441389509], [-73.92111376902572, 40.59677963097669], [-73.92114826240214, 40.59681445265428], [-73.92118344574286, 40.59684887021132], [-73.92118802880371, 40.596853728352336], [-73.92119323341322, 40.59685820646053], [-73.92119900511614, 40.5968622617784], [-73.92120528506241, 40.596865851378105], [-73.92121201088028, 40.59686893618235], [-73.92121911029548, 40.59687148766174], [-73.9212265105916, 40.59687347946415], [-73.92123413992638, 40.59687488942642], [-73.92124191611856, 40.59687570341971], [-73.92124976005617, 40.596875913516016], [-73.92125759174067, 40.596875518152565], [-73.9212628804995, 40.59687387379104], [-73.9212684105677, 40.59687279617859], [-73.92127408543934, 40.596872304849825], [-73.92127979345429, 40.596872409780126], [-73.92128542867985, 40.59687310972428], [-73.92129088541984, 40.59687438919708], [-73.9212960626006, 40.59687622685317], [-73.9213008598254, 40.59687858627001], [-73.92130518989238, 40.59688142466787], [-73.92131129376092, 40.596888406106025], [-73.92131675353417, 40.59689569234113], [-73.92132153738551, 40.596903248170335], [-73.92132562535527, 40.59691103906907], [-73.9213289977091, 40.596919025989536], [-73.92133163778813, 40.59692717105889], [-73.92133506368913, 40.59693634596581], [-73.921339203422, 40.59694534847099], [-73.92134404273862, 40.59695414723672], [-73.92134956431755, 40.59696270706993], [-73.9213557508287, 40.59697100065125], [-73.92136257461662, 40.59697899646632], [-73.92137001659407, 40.59698666669202], [-73.92137804712212, 40.59699398383316], [-73.92140073316898, 40.59701504370633], [-73.92142420334383, 40.597035598507844], [-73.92144843480666, 40.597055633479], [-73.92147341219564, 40.597075129845614], [-73.92148452797672, 40.597083243343604], [-73.9214962795578, 40.59709082101375], [-73.9215086173052, 40.5970978319964], [-73.92152149882783, 40.597104253143634], [-73.92153487426533, 40.597110058789106], [-73.92154100821479, 40.59711354942833], [-73.92154754687742, 40.59711658466404], [-73.92155443094379, 40.59711913815345], [-73.92156160087892, 40.59712118740702], [-73.9215689887934, 40.59712271311244], [-73.92157653514104, 40.597123702496916], [-73.92158416850413, 40.59712414579481], [-73.92159182338932, 40.59712404178887], [-73.92159738021682, 40.59712470815831], [-73.92160273958795, 40.59712601502461], [-73.92160777230781, 40.59712793164191], [-73.92161235974629, 40.59713041386902], [-73.92161639559961, 40.59713340182552], [-73.92161978522482, 40.5971368252523], [-73.92162244475769, 40.59714060501833], [-73.92162431386555, 40.597144649946564], [-73.92162977256656, 40.59718903652344], [-73.92163298634603, 40.597198472622125], [-73.92163717072572, 40.59720768422334], [-73.92164229873923, 40.597216614348525], [-73.9216483381444, 40.597225207523415], [-73.9216552494449, 40.597233409274], [-73.92165833417285, 40.59724063624328], [-73.92166215299599, 40.5972476556416], [-73.92166668222728, 40.59725442573801], [-73.92167189378407, 40.597260904295716], [-73.92167775452769, 40.597267050582495], [-73.9216842315336, 40.59727282788704], [-73.92169128308619, 40.59727819982688], [-73.92169886504772, 40.5972831337037], [-73.9217069323978, 40.59728759949889], [-73.9217231009897, 40.59729143336318], [-73.9217395650497, 40.5972944495486], [-73.92175625404047, 40.59729663644746], [-73.92176534005367, 40.5972990657764], [-73.92177411768235, 40.59730208443022], [-73.9217825208026, 40.59730566673183], [-73.9217904898824, 40.59730978818166], [-73.92179796210176, 40.59731441657138], [-73.92180488497279, 40.59731951785692], [-73.92181120601097, 40.597325055146285], [-73.92181554237384, 40.5973272522272], [-73.92181942480326, 40.59732989646964], [-73.92182277731894, 40.59733293287253], [-73.9218255309803, 40.597336300910875], [-73.92182763092089, 40.59733993320073], [-73.9218290323914, 40.59734375549621], [-73.92182971196769, 40.597347689210075], [-73.92182965039831, 40.597351657601], [-73.92182885437087, 40.597355579924404], [-73.92182733386989, 40.597359376108], [-73.92182038141432, 40.59736714842113], [-73.92181284405521, 40.597374597674474], [-73.92180474666027, 40.59738169674528], [-73.92179611673029, 40.597388421695356], [-73.92178980016666, 40.59739200704583], [-73.92178403246993, 40.59739609451963], [-73.92177888270717, 40.597400636417944], [-73.92177440962753, 40.59740557565352], [-73.92177066802131, 40.59741085681143], [-73.92176701421403, 40.597416047060555], [-73.92176407307934, 40.59742149394702], [-73.9217618734752, 40.59742714069807], [-73.92176044206576, 40.59743292735643], [-73.9217597899089, 40.597438795294416], [-73.92175992630537, 40.597444685212885], [-73.92176084869334, 40.597450532779106], [-73.92176254791133, 40.597456278848824], [-73.92176626013794, 40.59747813874907], [-73.92177087842761, 40.597499898915856], [-73.92177639687284, 40.59752153706385], [-73.92178280956709, 40.597543030572325], [-73.9217901116981, 40.59756436033947], [-73.92179757049145, 40.597587795050195], [-73.92180597171647, 40.597611044276626], [-73.92181530858868, 40.597634084392325], [-73.92182557190223, 40.597656894617266], [-73.9218367544302, 40.59767945350236], [-73.92184230236823, 40.59770071137544], [-73.92184676836521, 40.597722115603034], [-73.9218501460818, 40.597743636025825], [-73.92185138891014, 40.597754754949], [-73.92185342501575, 40.59776580689731], [-73.92185625114323, 40.59777675668741], [-73.92185985963846, 40.59778757198074], [-73.9218642366908, 40.59779822261277], [-73.92186937091307, 40.597808673394326], [-73.9218845536419, 40.597836596097125], [-73.92190045183087, 40.59786428708856], [-73.9219170588992, 40.597891735977036], [-73.92192886505762, 40.597905807742734], [-73.92193988286631, 40.597920246865684], [-73.92195009521463, 40.59793502518995], [-73.92195948103144, 40.597950118074415], [-73.92197229278264, 40.59796695843641], [-73.92198589025554, 40.59798343378146], [-73.92200025874622, 40.59799952583874], [-73.92200887509807, 40.598011814092544], [-73.92201817026452, 40.59802381348323], [-73.92202812405263, 40.59803550104563], [-73.9220344665146, 40.59804309308286], [-73.9220399434606, 40.59805106716933], [-73.92204451209867, 40.5980593666513], [-73.92204814326828, 40.59806793069599], [-73.92208576446559, 40.59813720860585], [-73.92208881386274, 40.598141450838085], [-73.92209251176975, 40.59814538056534], [-73.92209680990216, 40.59814893962219], [-73.92210164172575, 40.598152075861684], [-73.92210693564516, 40.59815474299734], [-73.92211261324087, 40.59815690311411], [-73.92211859455087, 40.598158522819766], [-73.92212478773406, 40.5981595765873], [-73.92213110335145, 40.598160052964346], [-73.92213744624773, 40.598159942001935], [-73.92214437212587, 40.598159357661395], [-73.92215114207612, 40.59815810510834], [-73.92215764684087, 40.59815620571232], [-73.92216378352414, 40.59815369073182], [-73.92216944943885, 40.59815060063942], [-73.92217455243843, 40.59814698479378], [-73.92217877811844, 40.5981467103945], [-73.92218299295898, 40.59814706187694], [-73.92218704005073, 40.59814802472743], [-73.92219076954105, 40.59814956433329], [-73.92219403907933, 40.598151620790404], [-73.9221967302905, 40.598154117625235], [-73.92219873932149, 40.59815696480448], [-73.92220420296172, 40.59816412250407], [-73.92220902320445, 40.598171544966654], [-73.9222131820685, 40.59817919649644], [-73.92221665717726, 40.598187041897056], [-73.92221943647932, 40.59819504916259], [-73.92222268629754, 40.59820403729503], [-73.92222514648115, 40.598213169805774], [-73.92222680740622, 40.59822240664892], [-73.9222276601082, 40.59823170710867], [-73.92222770111806, 40.598241030138105], [-73.92222847126045, 40.59825686702995], [-73.92223027928547, 40.598272654700615], [-73.92223312018483, 40.598288354111936], [-73.92224024769264, 40.598303369721556], [-73.92224651074955, 40.59831860755951], [-73.92225189927653, 40.59833404031221], [-73.92225970057747, 40.59834799022031], [-73.92226641356453, 40.598362266073835], [-73.92227201543713, 40.598376818436115], [-73.92227376607877, 40.598386970383636], [-73.92227461683909, 40.5983971910797], [-73.92227456096101, 40.59840743176858], [-73.9222736000364, 40.59841764621338], [-73.92227335717662, 40.59841896517427], [-73.92227173873475, 40.598427788011605], [-73.92226949878852, 40.59843845609801], [-73.9222663622689, 40.5984489929053], [-73.92226234307493, 40.598459352874386], [-73.92225745971561, 40.5984694954761], [-73.92225173004306, 40.59847937783468], [-73.92222389025856, 40.59852557210151], [-73.92219508751424, 40.59857142210698], [-73.92216532643818, 40.59861691612657], [-73.92215443198057, 40.59863245529204], [-73.92214281543683, 40.598647688728896], [-73.92213049353394, 40.598662596177306], [-73.92211748277803, 40.59867715838249], [-73.92210379615744, 40.598691357259696], [-73.92208945303467, 40.598705174226176], [-73.92207447255038, 40.5987185927092], [-73.92205887142968, 40.59873159462676], [-73.92204267189015, 40.598744163073356], [-73.9220134910915, 40.59876688670699], [-73.92198501540479, 40.59879012680193], [-73.9219572644096, 40.59881386745661], [-73.92193024669301, 40.59883809611275], [-73.92191222987921, 40.59885500884677], [-73.92189352936738, 40.5988714868774], [-73.92187416539961, 40.59888751212504], [-73.92185415447607, 40.59890307036068], [-73.92178152944105, 40.59895906919443], [-73.92174186243156, 40.59898908068103], [-73.92163283677688, 40.599057697722095], [-73.9215182161155, 40.599119652557384], [-73.92140953753236, 40.599180401445935], [-73.92129088298472, 40.59924265618808], [-73.92116996805429, 40.5992977152176], [-73.92107369910642, 40.59933923223918], [-73.920948314347, 40.599398434313336], [-73.9208916323198, 40.59943111393279], [-73.92086878083218, 40.59944764924901], [-73.92086156604135, 40.59945840334221], [-73.92086133639023, 40.59946987642224], [-73.9208663980296, 40.599484427307], [-73.92087757811551, 40.599499943686666], [-73.92088123526705, 40.599513230601026], [-73.92088050854765, 40.59951943270231], [-73.92087055640596, 40.59953007735755], [-73.92084893682663, 40.59954139732843], [-73.92083168628147, 40.59954264861636], [-73.92079917621065, 40.59953667005008], [-73.92077725291253, 40.599530725577786], [-73.9207545719535, 40.5995266761719], [-73.920738838775, 40.59952279405688], [-73.92072869332861, 40.59951592506269], [-73.92071257127054, 40.599506554753866], [-73.92069036019608, 40.59950050083734], [-73.92066385993064, 40.59949760054105], [-73.92063321092299, 40.59950191570488], [-73.92055176948652, 40.599519372265526], [-73.92051636737814, 40.59952343750909], [-73.92046271021108, 40.599524937466384], [-73.9204158269711, 40.599523833995015], [-73.92036966536112, 40.59952083708674], [-73.92030873371878, 40.59952150902786], [-73.92028841969866, 40.59952629833728], [-73.92027043637923, 40.599538033178455], [-73.92025813429498, 40.59955647011201], [-73.92025584763479, 40.59957153694035], [-73.92025331946004, 40.59957902960548], [-73.92024296948648, 40.59959032997471], [-73.92023641368432, 40.599591369466836], [-73.92022539730596, 40.5995890556131], [-73.92021492291106, 40.59958583697428], [-73.92020343507266, 40.59958684456041], [-73.9201920865931, 40.59959004804745], [-73.92018340395254, 40.599594159215464], [-73.92017911629651, 40.599596736354876], [-73.92017364161147, 40.59959706191467], [-73.92016241311848, 40.59959145528841], [-73.9201479477897, 40.5995829072796], [-73.92013207395463, 40.59957683135624], [-73.9201178545925, 40.59957396577037], [-73.9200925395554, 40.599573345401176], [-73.92001964207236, 40.599582655051165], [-73.91999918067488, 40.59959007860375], [-73.91996786796024, 40.59960729371514], [-73.91994336348232, 40.59962061582612], [-73.91992622063209, 40.59962348689316], [-73.91991448104537, 40.599623093898245], [-73.91990148636712, 40.59961874818084], [-73.91989630775923, 40.59961300837114], [-73.91989599129685, 40.59960620110357], [-73.91990064335972, 40.59960052259278]]], [[[-73.91513331918175, 40.58626938540072], [-73.91509711064722, 40.5862665836932], [-73.91505538740107, 40.58626722096251], [-73.91498196714757, 40.58626925759485], [-73.9149089072476, 40.586262172972], [-73.91488111429749, 40.5862576662828], [-73.91486137370094, 40.58626057605091], [-73.9148490627048, 40.58626572331663], [-73.91483964457827, 40.586275068332206], [-73.91482249731176, 40.58627650659437], [-73.91479664555939, 40.5862715106301], [-73.9147763034473, 40.58626431121139], [-73.91475822751109, 40.586256129234776], [-73.91473689701499, 40.58625287070833], [-73.91472719418067, 40.586252847089966], [-73.91470909811237, 40.58624959581705], [-73.91469360490532, 40.58624240835947], [-73.91468133993774, 40.58623645999916], [-73.91465937411907, 40.58623048801491], [-73.91457444535509, 40.58620093754455], [-73.9145518229785, 40.58619718280631], [-73.91453176360426, 40.58619935163311], [-73.91451879447324, 40.58620671730121], [-73.91451002779564, 40.58621507803655], [-73.91450223072464, 40.586223195234766], [-73.91448795703413, 40.58623376141627], [-73.91447306751336, 40.58623668343948], [-73.91444848600902, 40.586237115793516], [-73.9144106863387, 40.586228145088704], [-73.91430221625379, 40.58618275690621], [-73.91427510080042, 40.58617110092186], [-73.9142566771619, 40.58616834287503], [-73.91424534967837, 40.58617078118889], [-73.91422561352081, 40.58617245733182], [-73.91421300745357, 40.586170699918306], [-73.91419138861141, 40.58615930551065], [-73.91416592220669, 40.58614025732272], [-73.91410914905579, 40.58610609149561], [-73.91408557111488, 40.586098882499265], [-73.91405550969763, 40.58609535597158], [-73.91403188386349, 40.58609924139808], [-73.91401569268363, 40.5861041332044], [-73.91399595873644, 40.58610581031467], [-73.91397561478601, 40.58609860857474], [-73.9139358660144, 40.58607463454717], [-73.91389131223131, 40.58605621407723], [-73.91384327421697, 40.586043893231235], [-73.91379317603624, 40.58603803725476], [-73.9137425028251, 40.58603881974524], [-73.913692756766, 40.586046217506315], [-73.91364541255665, 40.586060011234764], [-73.91360187369521, 40.586079792022346], [-73.91359786062695, 40.58608525007121], [-73.91359508016339, 40.586091137118316], [-73.91359360808936, 40.58609729270503], [-73.91359348452897, 40.58610354905289], [-73.91359471284989, 40.58610973563731], [-73.91359725957363, 40.58611568383495], [-73.91360105528585, 40.58612123152021], [-73.91360599652994, 40.5861262274837], [-73.91361194862614, 40.586130535553814], [-73.91360925219124, 40.5861357838103], [-73.91360551856437, 40.58614064877595], [-73.9136008384908, 40.58614501220781], [-73.91359532572005, 40.586148768052475], [-73.91358911423974, 40.58615182502403], [-73.9135823550201, 40.58615410882269], [-73.91357521234436, 40.58615556394073], [-73.91356785981523, 40.58615615501118], [-73.91356047613637, 40.586155867668325], [-73.91349859152439, 40.58615686418724], [-73.91343694901619, 40.58615257755136], [-73.913376322177, 40.58614306155435], [-73.91331747182582, 40.5861284356148], [-73.91326113648611, 40.58610888327645], [-73.9131550869676, 40.58609426395945], [-73.91311536491438, 40.5860687287984], [-73.91307177986289, 40.58604716501875], [-73.91302501082505, 40.58602990856117], [-73.91297578641507, 40.58601722826301], [-73.91291458410106, 40.58598697147525], [-73.91286589107678, 40.58594317173063], [-73.91282142952436, 40.58589683033704], [-73.91278142692737, 40.585848184406025], [-73.91275274022804, 40.58579265559755], [-73.91271619323912, 40.585743423938645], [-73.91268773524789, 40.585717024550554], [-73.91266190734372, 40.58568909761717], [-73.91256608953849, 40.58560928558327], [-73.9125395226262, 40.58532147103919], [-73.91254564022545, 40.585278707249124], [-73.91254367736798, 40.585235715910294], [-73.91253367438031, 40.585193379922245], [-73.9125158367049, 40.585152568724595], [-73.91249053068118, 40.58511412044127], [-73.91245827602079, 40.585078824668194], [-73.91245182213171, 40.58503349325462], [-73.91245221064955, 40.58498789654783], [-73.9124594364618, 40.58494263288399], [-73.9124734047356, 40.58489829622763], [-73.9125006491395, 40.58474179788662], [-73.91250278714065, 40.5847316239427], [-73.91250598875575, 40.58470730696595], [-73.91249831364493, 40.584686019518024], [-73.91247773546301, 40.58466517741908], [-73.91244299981304, 40.584650746770585], [-73.9124137811953, 40.58464469411026], [-73.91239412142289, 40.58463736260747], [-73.9123819934839, 40.58462315339449], [-73.91237790672498, 40.58459386763484], [-73.91236972324565, 40.58456165217832], [-73.91236790288794, 40.58452617424718], [-73.91237486072286, 40.584505246076574], [-73.91239232328932, 40.58449966926963], [-73.9124193285938, 40.58450239244318], [-73.91248473796232, 40.58450618455642], [-73.91250885205581, 40.58448432792841], [-73.91252051530321, 40.58446125242105], [-73.91257873642861, 40.58437882582973], [-73.91260847719191, 40.58431994827597], [-73.91261720032453, 40.58427768337179], [-73.91261758823649, 40.58424974980263], [-73.91261808094076, 40.58422485647048], [-73.91263170246216, 40.58420023497082], [-73.91263982584111, 40.58417985163811], [-73.9126341968073, 40.58417261862883], [-73.91261674746674, 40.58416867210214], [-73.9126020924363, 40.584159939036105], [-73.9125942554235, 40.58414518964306], [-73.91258466852842, 40.584120114076704], [-73.9125855857062, 40.58408940042326], [-73.91259318315109, 40.58406758280877], [-73.91261266604457, 40.58402458326315], [-73.91263830734084, 40.583966932248096], [-73.91263581397888, 40.58395029852837], [-73.91261666905709, 40.58392647501517], [-73.91258990362563, 40.58391609463292], [-73.91253889560856, 40.58391613103657], [-73.9125210309118, 40.58391378927905], [-73.91250423830841, 40.58390055205037], [-73.9125046003567, 40.583886356213235], [-73.91251327125534, 40.58387575300659], [-73.91253467541472, 40.58387545803292], [-73.91256813712123, 40.58388633321259], [-73.91260664321682, 40.58389463042295], [-73.91265005310576, 40.583894272084706], [-73.9126727844487, 40.58388795641479], [-73.91269014904348, 40.583875153606506], [-73.91269902546921, 40.58385270453265], [-73.91270605956096, 40.58383062848673], [-73.91270280580808, 40.583753879755044], [-73.91271079739748, 40.58369994166922], [-73.91270485854531, 40.583675242138966], [-73.9126928262588, 40.58365030745778], [-73.91267910298275, 40.58362345009425], [-73.91266228121616, 40.583605996819294], [-73.91265111155093, 40.583604150362085], [-73.91263067653858, 40.58360846127351], [-73.91262686025286, 40.58361073325424], [-73.91261090987317, 40.583620229594345], [-73.9125955974672, 40.583642927405656], [-73.91258263821426, 40.583654042174096], [-73.91256416724003, 40.58365563053855], [-73.91254323349644, 40.58364476824473], [-73.91252584404202, 40.58362708449973], [-73.91248355319789, 40.58356693082128], [-73.9124518134371, 40.583540028377854], [-73.9124242085302, 40.583514934039734], [-73.91240864017935, 40.583488475465195], [-73.9124007270715, 40.58345276119278], [-73.9124066060869, 40.58342480550645], [-73.91242103069457, 40.58339278402727], [-73.91243878847268, 40.583369974014715], [-73.91246022035992, 40.58335175442639], [-73.91249511439021, 40.5833375354724], [-73.91257072971156, 40.58331285848093], [-73.9126052251269, 40.583290724427194], [-73.91262124306265, 40.583271353280885], [-73.91264957500252, 40.583245914027785], [-73.91267748287285, 40.58322632474245], [-73.91270404052639, 40.58320432501546], [-73.91275025844384, 40.58314540498738], [-73.91281681585251, 40.58309287336548], [-73.91286173083516, 40.58307045982732], [-73.91296715647701, 40.58303417059139], [-73.91304660950463, 40.58300754849561], [-73.91306274695064, 40.58300796478858], [-73.9130834248184, 40.58301392236903], [-73.91310063491991, 40.583025555232624], [-73.91310764522147, 40.58303932844308], [-73.91310803072021, 40.58305564660949], [-73.91309366069213, 40.58307811786334], [-73.91303116125995, 40.58311149572026], [-73.91301314865379, 40.58312940288127], [-73.91300400920699, 40.583151161727585], [-73.91301010544078, 40.583173509583744], [-73.91302765232814, 40.583184710654315], [-73.91305439371872, 40.58318671645332], [-73.91307702388137, 40.58317317373953], [-73.91308233206294, 40.58314917329266], [-73.91309259176921, 40.583133266757365], [-73.91311537518962, 40.58311743000316], [-73.91315375727191, 40.58311012491582], [-73.91320747763812, 40.58310644953777], [-73.91334530004634, 40.583087744026905], [-73.9134426509044, 40.583066501794434], [-73.91351194436805, 40.583040871371054], [-73.91356029877663, 40.58303070923508], [-73.91359114656434, 40.58303030925951], [-73.91362890961973, 40.583032296241164], [-73.9136704080064, 40.58302812126697], [-73.91370574416129, 40.583020700121565], [-73.91379817883399, 40.582991336421415], [-73.91384198140268, 40.58298096820854], [-73.91388783020813, 40.582984682701294], [-73.913933638425, 40.58298954623003], [-73.91398276542384, 40.582989880552624], [-73.91401448220044, 40.58298626974121], [-73.91404880615279, 40.58297597979547], [-73.91408049248979, 40.58296818137606], [-73.91412574717617, 40.58296742195705], [-73.91415019662927, 40.58296724421898], [-73.91418696590387, 40.582956841468935], [-73.91421440940195, 40.58294421934995], [-73.91424209153641, 40.58292388518252], [-73.91426461163289, 40.58290733047502], [-73.91428757031191, 40.58290173067609], [-73.91430480864925, 40.58289543755721], [-73.91431347825309, 40.58288483538588], [-73.9143203815262, 40.58287345673761], [-73.9143386907524, 40.582856266618926], [-73.91434813864615, 40.58285200266575], [-73.91436272478424, 40.58285769530131], [-73.91438024403833, 40.58286468177312], [-73.91440852626428, 40.582866745532776], [-73.91442531787224, 40.58285787143888], [-73.91443075792112, 40.58284528564937], [-73.91441656474707, 40.58282958359571], [-73.91441354718485, 40.58281154193981], [-73.9144260182388, 40.582798993218404], [-73.91444562078021, 40.58279792088553], [-73.91448137951106, 40.58280678954632], [-73.91450871967749, 40.58280905411826], [-73.91454056069132, 40.58279893188524], [-73.91457136618546, 40.58277756781225], [-73.91459109911118, 40.58276577165373], [-73.9146096161663, 40.58275884651334], [-73.91463166275301, 40.582757634905384], [-73.91472075055118, 40.58277283584654], [-73.91475316846913, 40.58277671090445], [-73.91478662998793, 40.58278758426655], [-73.91481206768587, 40.582803984666235], [-73.91482092346082, 40.58281738588486], [-73.91482435715882, 40.582833823240975], [-73.91481966904605, 40.58284853136757], [-73.91481396714835, 40.5828604269273], [-73.91482516810426, 40.58286646059711], [-73.9148566460506, 40.582870507316706], [-73.91488143663862, 40.582869871315225], [-73.91489741970577, 40.58287261036763], [-73.91490695367519, 40.582885095863055], [-73.91490200810955, 40.582894927353856], [-73.91488463973496, 40.58291191856394], [-73.91485961850306, 40.582938163958055], [-73.91485126450586, 40.58296204629953], [-73.91485751324602, 40.58298209951916], [-73.91487850168284, 40.58300552279338], [-73.91487974591004, 40.583027004010866], [-73.91487128569776, 40.5830478456147], [-73.91486183617289, 40.58307425013897], [-73.91487113272144, 40.583094422049186], [-73.91489759501914, 40.58310526003698], [-73.91492120779233, 40.583112480957304], [-73.91495087200077, 40.58312114358797], [-73.91496680213771, 40.583133403794704], [-73.91498253486131, 40.58314919202401], [-73.91498685680594, 40.5831528403285], [-73.91499355016039, 40.583153361271734], [-73.91500434950726, 40.58315145121622], [-73.91502932438901, 40.58313056686786], [-73.91504738947053, 40.58312522296865], [-73.91506390193432, 40.58312520820453], [-73.91507218234854, 40.58312458692769], [-73.91508577200474, 40.58311791773898], [-73.9151053823759, 40.583108503838766], [-73.91514044992658, 40.58310451791601], [-73.91516678218403, 40.58310394229694], [-73.9152135907857, 40.58309515122808], [-73.91525733442549, 40.58308729219161], [-73.91528333907931, 40.58307595337238], [-73.91535364351934, 40.58305308548065], [-73.91539584617905, 40.58304482787612], [-73.91542990602406, 40.58304271142218], [-73.91545449816903, 40.58303970913197], [-73.91547650540166, 40.58303267233262], [-73.9155053193231, 40.58302253721877], [-73.9155269382117, 40.58300825263115], [-73.91554499756505, 40.58299017825141], [-73.91556719011433, 40.58297732193168], [-73.91560713091408, 40.58296458737209], [-73.91565304415474, 40.58295545303635], [-73.9156987077508, 40.58294563313593], [-73.91573006889206, 40.582941840323116], [-73.9157501285275, 40.582936393405475], [-73.91575888276965, 40.58293167834109], [-73.9157616045728, 40.58291447087011], [-73.91577896476002, 40.582908499946264], [-73.91579646420027, 40.58290675605841], [-73.91590083079711, 40.582908369582256], [-73.9159559503259, 40.58290714791531], [-73.91601352716066, 40.582898049776674], [-73.91604528931649, 40.582891428938204], [-73.91607986037056, 40.58288302829906], [-73.91612226132845, 40.582861238545796], [-73.91614199473273, 40.582857437633876], [-73.91615620879065, 40.58286260825071], [-73.91616526389136, 40.58287515862866], [-73.91616340652098, 40.58289332785657], [-73.91616117295024, 40.58292200995522], [-73.9161653343486, 40.582934911924504], [-73.91619291399608, 40.58296092642285], [-73.91625458407773, 40.5830191062994], [-73.91628745922038, 40.58304193956392], [-73.91632545582766, 40.583058572764635], [-73.9163582813012, 40.58306600817026], [-73.91640110933905, 40.58307572857577], [-73.9164738146702, 40.583111534088786], [-73.91652537447123, 40.58314033821524], [-73.91656345737127, 40.583152013699674], [-73.91660432992357, 40.58315938686152], [-73.91674435706148, 40.5831492431489], [-73.91680904215018, 40.583136139106514], [-73.91684198716314, 40.583112409956115], [-73.91686865501184, 40.58309836381437], [-73.91688414354172, 40.58309884711799], [-73.91690459603636, 40.58310521664009], [-73.91691664570561, 40.583119129463526], [-73.91692804679539, 40.58314216882283], [-73.91694732728378, 40.58318825923233], [-73.91697421629549, 40.583211780204266], [-73.91700811433819, 40.58322991161618], [-73.91704549903487, 40.58324266348338], [-73.91722154944054, 40.583252789287265], [-73.9172715063787, 40.58324885900864], [-73.91729747607816, 40.58323588813918], [-73.91730410990102, 40.58322701133874], [-73.91731074592352, 40.583218131188886], [-73.91729861410717, 40.58319080174952], [-73.91708382946543, 40.583058977652826], [-73.91687676635951, 40.582923370897454], [-73.91684561145445, 40.58289288546194], [-73.91681802428016, 40.582870439325305], [-73.91680237542735, 40.5828431210236], [-73.91680996859863, 40.582817869441286], [-73.91683095894794, 40.58279632912118], [-73.91683298733994, 40.58278236834692], [-73.91681104385314, 40.5827620485571], [-73.91676942718226, 40.58274770268724], [-73.91671303114217, 40.582734479091386], [-73.91663555189481, 40.582726696272644], [-73.91659049396183, 40.582725777507214], [-73.9165602576571, 40.582731786446736], [-73.91648384235516, 40.58278464746007], [-73.9164297950498, 40.58281113226658], [-73.9164107855549, 40.58281066007735], [-73.91639106239693, 40.582808581595586], [-73.9163755408857, 40.582802731561685], [-73.9163509359136, 40.58280818348835], [-73.91633310071417, 40.582819260116366], [-73.916326405681, 40.58281886992297], [-73.91631125527731, 40.582813918108805], [-73.91629330489032, 40.582801883953884], [-73.9162906180456, 40.582791313398594], [-73.91629316588654, 40.582769906326476], [-73.91628476885714, 40.582747556968165], [-73.9162755368557, 40.582730807511425], [-73.9162610483292, 40.58271602989832], [-73.91624727128278, 40.582706879112145], [-73.91620991224312, 40.5826982068487], [-73.9161613368347, 40.58267785930009], [-73.91614359890356, 40.58266887196805], [-73.91612707682087, 40.58266806280264], [-73.91611252487061, 40.58267337710359], [-73.91609776307327, 40.58267449279288], [-73.91608276193867, 40.58266602729392], [-73.91606608507493, 40.58264325958138], [-73.91606105132732, 40.58263826034202], [-73.91604831118616, 40.58263545077821], [-73.91604312676826, 40.582635119302275], [-73.91603398469314, 40.58263376652196], [-73.91602653797953, 40.58262777742867], [-73.91602327833968, 40.58261580824715], [-73.91603150828996, 40.58259846708605], [-73.91604510737572, 40.582576793681795], [-73.91605924441413, 40.582557699618015], [-73.91605982993855, 40.582550233591384], [-73.91605533428852, 40.58254666209619], [-73.9160421323442, 40.58253891208453], [-73.91601234094581, 40.58252384934534], [-73.91600076262644, 40.58251382295231], [-73.91599673914028, 40.582506271801925], [-73.91599507564251, 40.58248313074189], [-73.91598930325056, 40.5824647912558], [-73.91598506756877, 40.58245422123571], [-73.91597323763864, 40.58244229956228], [-73.91596151080888, 40.582435758340175], [-73.9159447356366, 40.582434264437964], [-73.91592914771844, 40.58243326544656], [-73.91590736628613, 40.58243445672382], [-73.91589979876528, 40.58244084712577], [-73.9158960153524, 40.58244402775174], [-73.91588532189255, 40.58244380029226], [-73.9158787393108, 40.58243877310615], [-73.915875254895, 40.582433801295096], [-73.91586237611384, 40.582425612907045], [-73.9158442416109, 40.58241824357294], [-73.91582308234135, 40.582410816567645], [-73.9158042230052, 40.582406713521394], [-73.9157805779702, 40.58240062961325], [-73.9157652891067, 40.5823914518614], [-73.91574397996054, 40.58238753698734], [-73.91573004424852, 40.582391919557736], [-73.915718940778, 40.58240335669329], [-73.91570859960468, 40.58241040377399], [-73.91566729548903, 40.58241844319196], [-73.9156141628191, 40.58241456205834], [-73.91557828001777, 40.58240706924327], [-73.91555618378293, 40.58239986163409], [-73.91554158161824, 40.58238859660441], [-73.91553029051371, 40.582378104308006], [-73.91552366963937, 40.58237540841372], [-73.91551693880298, 40.5823750466268], [-73.91550413660777, 40.58239112289203], [-73.91549292208342, 40.582406072682495], [-73.91548489271247, 40.58240755008249], [-73.91547157698861, 40.58240331284745], [-73.91546395768319, 40.58239312446376], [-73.9154627460607, 40.582383762694164], [-73.91545552568118, 40.582371927285905], [-73.91543278029661, 40.58236565348722], [-73.91541290900807, 40.58236407379874], [-73.91537680116978, 40.58236358114242], [-73.91533636981164, 40.5823648683054], [-73.91529755299925, 40.582371564999264], [-73.91526029387259, 40.58236942346994], [-73.91518549237695, 40.58236324792052], [-73.9151453494571, 40.58236406916207], [-73.91510347182249, 40.582370707574896], [-73.91507188576279, 40.58238152573492], [-73.91503244299057, 40.58239799219667], [-73.91499231015695, 40.582430576817295], [-73.91495621705263, 40.58246266266324], [-73.91492361437683, 40.582481465155716], [-73.91489145347467, 40.582490855991736], [-73.91485429620485, 40.58249409310831], [-73.91481200341404, 40.582486926341616], [-73.91477047211993, 40.58247536683082], [-73.91469476968891, 40.58244163202668], [-73.9146614460973, 40.58243164463571], [-73.9146294466244, 40.582427503658316], [-73.91456038746365, 40.58240650895869], [-73.91442069856869, 40.58238222378727], [-73.91438667636007, 40.5823831580494], [-73.91435652574022, 40.58239633807557], [-73.9143386047637, 40.582418612504824], [-73.91432797974475, 40.58245154222201], [-73.91433402510343, 40.58248061477258], [-73.9143418953653, 40.58249151692641], [-73.91436373972239, 40.58249801040327], [-73.91439432847737, 40.58250750330599], [-73.9144126777686, 40.582516739642635], [-73.91443205047094, 40.582541155245934], [-73.91444439356765, 40.58257338957388], [-73.91445578619143, 40.58258926242984], [-73.91446429302799, 40.58260927781602], [-73.91445859867211, 40.58261405068004], [-73.91444177524455, 40.582622600250765], [-73.91440306841196, 40.582626964211045], [-73.91439308844033, 40.58263236462761], [-73.91438026345521, 40.58263957274417], [-73.91437177016184, 40.58263728885106], [-73.91436649738984, 40.58262155658218], [-73.91436533351332, 40.58260215110844], [-73.91435321731488, 40.582589542854066], [-73.91434396357897, 40.58259167631232], [-73.91432932462531, 40.58260818966933], [-73.9143108166107, 40.582611332458086], [-73.91429776347408, 40.58260009640314], [-73.9142833750802, 40.582590697986724], [-73.91425741969694, 40.582589004832556], [-73.9142369244519, 40.582597251922735], [-73.91421410726333, 40.58261987951505], [-73.9141761412379, 40.58266417862779], [-73.91416846209651, 40.58270067941076], [-73.91416685228447, 40.58272071574064], [-73.91418200427732, 40.582751138352975], [-73.91418770187882, 40.582772936352654], [-73.91418220880657, 40.58278964750494], [-73.91417322815995, 40.58280136166247], [-73.91415391948266, 40.5828112571092], [-73.91413967884692, 40.5827983453569], [-73.9141325725104, 40.582781871226885], [-73.9141239526859, 40.58276536849044], [-73.91410586572161, 40.58274913275424], [-73.9140810716333, 40.58274022091247], [-73.91405490474298, 40.58273550833787], [-73.91402181932013, 40.58273625220923], [-73.91393182914146, 40.58272627546224], [-73.91388053298886, 40.5827219556406], [-73.91385050619411, 40.582722756034606], [-73.91381669671354, 40.58272676495641], [-73.91379074062979, 40.58272507169629], [-73.91376111494455, 40.58272307378791], [-73.91372600624901, 40.582728893874226], [-73.91370947032047, 40.5827381314296], [-73.91370286431712, 40.58275201466303], [-73.91369614547608, 40.58276938477012], [-73.91367567620946, 40.58278531669264], [-73.91364667393216, 40.582801297650484], [-73.91361432670016, 40.582815353015185], [-73.913527587149, 40.582880614215384], [-73.91343525105174, 40.582930582894676], [-73.91335911446285, 40.58295402101425], [-73.91333506920232, 40.58295073655695], [-73.91331008978153, 40.58294649077557], [-73.91329310293676, 40.5829419489652], [-73.91327543023212, 40.58293949220915], [-73.91326014818266, 40.582938603488394], [-73.91324486679412, 40.582937713593495], [-73.91321912488108, 40.582939066554836], [-73.9131830787931, 40.58294510422574], [-73.91314502174606, 40.58294856191882], [-73.91311276013543, 40.58295139155832], [-73.9130704563263, 40.58295311677563], [-73.91303517141988, 40.58295591612842], [-73.91300639684994, 40.58296374611722], [-73.91297786818073, 40.58297459282094], [-73.91296905167876, 40.582972802624006], [-73.91294541913625, 40.5829566727388], [-73.912928919258, 40.58293813281428], [-73.91293062903803, 40.582924630227694], [-73.91294501682768, 40.58290624991009], [-73.91295907838796, 40.582889490861454], [-73.9129720797649, 40.582858703428556], [-73.91298036028758, 40.58283134342263], [-73.91299537214289, 40.582804319049316], [-73.91300493732533, 40.582785140144566], [-73.91300682242084, 40.58277468387819], [-73.913048114127, 40.58274891341535], [-73.91305439577195, 40.582735496041046], [-73.91304170419419, 40.58272146053163], [-73.91301932318424, 40.582713539213536], [-73.91299384246965, 40.58270674016928], [-73.9129818894498, 40.58270719718403], [-73.91296334252092, 40.58271264694238], [-73.91294625687493, 40.58272934875089], [-73.91293793874237, 40.582756735028134], [-73.91294388704215, 40.58277924745802], [-73.91295300420718, 40.582799511027105], [-73.91295787683184, 40.58281804214046], [-73.91295444392058, 40.582828497233265], [-73.91294622469934, 40.58283579227632], [-73.91293207294984, 40.582838306222314], [-73.91290960457906, 40.582841582823235], [-73.91289270581302, 40.58285243999625], [-73.91288847348098, 40.58286846600925], [-73.91288948472226, 40.582892538586236], [-73.91288587847073, 40.58292536426114], [-73.91288830211467, 40.5829440856767], [-73.9128947357864, 40.58297925047148], [-73.9128904277056, 40.58299758347567], [-73.91287356643893, 40.583007286731004], [-73.91285379588716, 40.5830122672988], [-73.91282996485275, 40.58301084934427], [-73.91281481249887, 40.58300705117809], [-73.91280395957409, 40.58299257727315], [-73.91279155667922, 40.58297925496104], [-73.91276986017412, 40.58296924753424], [-73.91275312226192, 40.5829666002565], [-73.91272345895841, 40.58296575431993], [-73.91269329275185, 40.58296468735702], [-73.91266221096731, 40.582972651540324], [-73.91264228805896, 40.58298457689103], [-73.91258267952516, 40.5830322357435], [-73.91253079033142, 40.58306771501689], [-73.9125121275724, 40.5830789539308], [-73.91250111083939, 40.58307919315001], [-73.9124923404176, 40.58306856292292], [-73.91250405700487, 40.5830562477066], [-73.91250705583342, 40.5830486200271], [-73.91250735235212, 40.583040495575254], [-73.9124970981475, 40.58303634258394], [-73.91247445044137, 40.58303890545499], [-73.91244999095223, 40.583049564646466], [-73.91240270924914, 40.58305861339707], [-73.9123793498798, 40.58305442343077], [-73.91234714435846, 40.58304191054728], [-73.91227025577561, 40.58300712903057], [-73.91225008134586, 40.582990672782415], [-73.91223166714833, 40.582976111293725], [-73.91222052272708, 40.58296333954539], [-73.91222499686906, 40.58295689257552], [-73.91224058548934, 40.58295673980216], [-73.91232749133063, 40.582957328172405], [-73.91236962524636, 40.582947945660976], [-73.91241554492984, 40.58293422868159], [-73.91245483289673, 40.58292662866108], [-73.91248403760483, 40.58292025547194], [-73.91249831647532, 40.58290426359514], [-73.91249951422785, 40.58289589339357], [-73.91249009464558, 40.5828861696764], [-73.91248404877419, 40.58288490240152], [-73.91244325020692, 40.58289129372831], [-73.91240202673642, 40.582892772416734], [-73.91237553234416, 40.58289085829274], [-73.91235104822734, 40.5828938583312], [-73.91232914954863, 40.58290084201079], [-73.91230632607945, 40.58289923256003], [-73.9122884759112, 40.5828949090706], [-73.91226320479217, 40.58289348989912], [-73.91223403674643, 40.58289871079228], [-73.91213692649706, 40.58291049296738], [-73.91204284826749, 40.5829146485911], [-73.911996442554, 40.58291579330097], [-73.9119816384673, 40.58292039321009], [-73.91197619502306, 40.58292703091419], [-73.91197405738336, 40.582936746992246], [-73.91197064789375, 40.58295598430477], [-73.91196332932957, 40.582964213106855], [-73.91195554655069, 40.582968709163175], [-73.91193607084406, 40.5829678433548], [-73.91191058860355, 40.58296337725262], [-73.91188540427932, 40.58295078619268], [-73.91186572974851, 40.58293691038208], [-73.91184835219073, 40.58292863466334], [-73.91182877719504, 40.582921264092924], [-73.91180279364123, 40.58291306468987], [-73.9117865670517, 40.582906464135164], [-73.91175246333535, 40.58288670280388], [-73.91168534279711, 40.5828435035064], [-73.91163746024111, 40.58281695196503], [-73.91155605037828, 40.582770886898416], [-73.91152073412317, 40.58274297140646], [-73.91150233164481, 40.58271957145567], [-73.91148034599208, 40.582684722958305], [-73.91144146900237, 40.58259998891141], [-73.91138691369105, 40.58250118787197], [-73.91132007841033, 40.58237915824153], [-73.91128635983833, 40.58231358629239], [-73.91128207390518, 40.58229642025531], [-73.91127375383395, 40.58226309908052], [-73.91127823803255, 40.58222132721823], [-73.91129057746076, 40.582173907635244], [-73.91134231551266, 40.58208888446338], [-73.91139720194924, 40.58199277743049], [-73.91145206083493, 40.581887830989714], [-73.91147523338645, 40.5818436295997], [-73.91148327601971, 40.58183216186746], [-73.9114996013066, 40.58181878146464], [-73.91151740121359, 40.581806552787675], [-73.91154161171625, 40.5817940271557], [-73.91157100347282, 40.58178298764022], [-73.91160531805478, 40.581778101410514], [-73.91175016920778, 40.58176215505988], [-73.911838086706, 40.58176021890369], [-73.9119719708825, 40.58176103573951], [-73.91213518740672, 40.58176966899819], [-73.91231554202783, 40.581775845385856], [-73.91248781450348, 40.581769306880936], [-73.91262397247574, 40.58176575987497], [-73.91280910196407, 40.58175840776044], [-73.91284574967929, 40.58175695343282], [-73.91298249383611, 40.581745995443704], [-73.91317650008669, 40.58171940785814], [-73.91334451057884, 40.58169463881006], [-73.91352729918786, 40.581657612453704], [-73.91362394588555, 40.58164094124915], [-73.9137112998617, 40.58162873719517], [-73.91375780594964, 40.58162946149775], [-73.91375946773572, 40.581629488042125], [-73.91381856564966, 40.58164114285245], [-73.91395583736505, 40.58166809050037], [-73.91407954472982, 40.58168893211234], [-73.91415185936313, 40.58169713882522], [-73.91427450114413, 40.58170634366887], [-73.9143867033746, 40.581716033642344], [-73.91444081609372, 40.58171269722758], [-73.91449886342396, 40.581701542768116], [-73.91463184406278, 40.58166550687895], [-73.91492725370496, 40.58154746122126], [-73.9150142040297, 40.581539295516976], [-73.91505520170746, 40.581543607722544], [-73.9152602172066, 40.58157867360728], [-73.91543478914738, 40.58160309317328], [-73.91553381168441, 40.581615077866104], [-73.91565428472096, 40.581632842913194], [-73.91579767708303, 40.5816587498145], [-73.91602025693952, 40.581697394419], [-73.91614001524881, 40.581709560603244], [-73.91620858371672, 40.581720945509836], [-73.91631145428028, 40.581736253610735], [-73.91642536736876, 40.58175900863262], [-73.91648646817089, 40.581761427290445], [-73.91656440839974, 40.58178202990354], [-73.91664609625747, 40.58180961848238], [-73.91675250020047, 40.581849358573585], [-73.91684419368605, 40.581879232997046], [-73.91694046369045, 40.58190496765121], [-73.91705904915005, 40.58193466918452], [-73.91717068751475, 40.58196326877144], [-73.91730082303066, 40.581998743071146], [-73.91740031696504, 40.582037351118785], [-73.91747275685563, 40.58206147565619], [-73.91754282640314, 40.58208038107559], [-73.91765594991877, 40.582103656331086], [-73.91774511391259, 40.582110718912936], [-73.9178718350551, 40.582112868168814], [-73.91795316745137, 40.58210826125213], [-73.91801544298364, 40.58211256011891], [-73.9180700430864, 40.582122754872714], [-73.9181401186957, 40.58213869717279], [-73.9182225636432, 40.58216510659864], [-73.91846993349417, 40.582247295964045], [-73.91856541463021, 40.582271298953046], [-73.91864198877482, 40.58229310137833], [-73.91866398779692, 40.58229936463159], [-73.91876881548053, 40.58233558897786], [-73.9188552089828, 40.58237075616669], [-73.91903244654955, 40.58242819203943], [-73.91914800330909, 40.58246261143317], [-73.91927583490043, 40.58249868514514], [-73.91940452561472, 40.582539453493446], [-73.91955473892287, 40.58258534206253], [-73.91967029776173, 40.58261976093144], [-73.91979661930037, 40.58265525720839], [-73.91988900617487, 40.58267810299279], [-73.92001983914027, 40.58270363785086], [-73.92016136884439, 40.582727342195156], [-73.92029134800215, 40.582745246187834], [-73.92038597491594, 40.58276164340871], [-73.9204837313241, 40.582782104972424], [-73.92058072742617, 40.58280374734509], [-73.92069085443077, 40.58283467497548], [-73.92081803189946, 40.58287777320651], [-73.92097048597475, 40.58291721174889], [-73.92110154983231, 40.58296324929815], [-73.9213218375613, 40.58302801245795], [-73.92144124962209, 40.583062404204284], [-73.92154835139512, 40.58309508615944], [-73.9216538372547, 40.5831242816297], [-73.92181939703667, 40.583164219846466], [-73.92195641792519, 40.58319499987735], [-73.92207260289808, 40.583219425126714], [-73.92221271877774, 40.583251332916454], [-73.9223789353746, 40.583284245254845], [-73.9225235966035, 40.58330910090225], [-73.92264745221506, 40.58333056697821], [-73.92281136591295, 40.5833640814189], [-73.92294226666579, 40.58339546060754], [-73.9230930457788, 40.58342262484577], [-73.92321384460404, 40.58344293381905], [-73.92330531313662, 40.58345230033658], [-73.92336456215884, 40.58346131526237], [-73.92343223286434, 40.58347201022718], [-73.92352834449206, 40.58348310839798], [-73.9236306089874, 40.58349651662117], [-73.92374136944463, 40.58351163115181], [-73.92381898015073, 40.58351876477829], [-73.92391812130148, 40.58352514298099], [-73.92400495251304, 40.58352986714733], [-73.92408408214602, 40.58353172994967], [-73.92414240638934, 40.58353017761237], [-73.92428618071578, 40.583544490504956], [-73.92437765038743, 40.58355388196902], [-73.92443604094977, 40.58355820344426], [-73.92452046724112, 40.58355474551057], [-73.92462570164496, 40.583554649411376], [-73.92473925816661, 40.583547505006955], [-73.92485051379164, 40.58353802647261], [-73.92496324822743, 40.58352618713361], [-73.92509444897401, 40.58351540393564], [-73.92521875540417, 40.58350642424622], [-73.92557372740384, 40.583513901056286], [-73.92559188192438, 40.58349861603501], [-73.92559744025867, 40.58346657292388], [-73.92560365807826, 40.583455383134215], [-73.92562049996914, 40.58345382127171], [-73.92563446397422, 40.583461138616], [-73.92563805833993, 40.5834773939806], [-73.92564570811074, 40.58348876245475], [-73.9259909277576, 40.58352317241188], [-73.92626785352488, 40.583550691936296], [-73.92630382039857, 40.58355293393953], [-73.92712531836796, 40.583579273883316], [-73.92823025252058, 40.58352576122202], [-73.92943745840644, 40.583418929257036], [-73.92944661932395, 40.583372643646015], [-73.92947294984545, 40.583374315505374], [-73.92946601177597, 40.583415628198516], [-73.92963248074892, 40.58339880862553], [-73.92981333853066, 40.58337400435509], [-73.92999308017578, 40.58334486355774], [-73.93017152683852, 40.58331141522934], [-73.93034850096225, 40.583273692651765], [-73.93043425282119, 40.58325401844997], [-73.93051721854357, 40.583228321926526], [-73.9305966722979, 40.583196827890276], [-73.93067191898056, 40.583159811869955], [-73.93074230029701, 40.58311759770328], [-73.93080720052055, 40.5830705547039], [-73.93084016643408, 40.58302926053867], [-73.9308656330301, 40.58298498948704], [-73.93088314750095, 40.582938528733884], [-73.93088649890265, 40.58292841324016], [-73.93088752472832, 40.58291800937567], [-73.93088619456917, 40.58290762556057], [-73.93088254785809, 40.582897569620606], [-73.93087669270234, 40.582888139661826], [-73.93086880267725, 40.582879615232706], [-73.93085911168123, 40.58287224903773], [-73.93084790700138, 40.5828662594457], [-73.93083552079771, 40.58286182401629], [-73.93076371083933, 40.582858754345594], [-73.93072893058803, 40.58285815808527], [-73.93071285975189, 40.58284281760607], [-73.9307501649442, 40.58279278313103], [-73.93074445590659, 40.58275391486452], [-73.93080777939004, 40.58275296878157], [-73.93082583214515, 40.58275756895427], [-73.93090885941557, 40.58278974134694], [-73.93094864986666, 40.58272735668951], [-73.93170213621669, 40.58281040746728], [-73.93171529784688, 40.5828884275619], [-73.93174431011353, 40.58287545908992], [-73.93182555909537, 40.58291076876258], [-73.93184655018902, 40.58294105529734], [-73.93183622347397, 40.58298913146625], [-73.93182358895909, 40.58300978135734], [-73.93180776058966, 40.58300977196394], [-73.93178761860817, 40.583009760007464], [-73.93177777573123, 40.583041453920025], [-73.93186345464173, 40.58308286112594], [-73.932029438, 40.58414863699993], [-73.9320424499999, 40.5842339599999], [-73.93116473799996, 40.584312990999905], [-73.93106383699993, 40.584318467999886], [-73.93107641299986, 40.58437967599985], [-73.9300961629999, 40.584502075999886], [-73.9301907789999, 40.58508559299986], [-73.93119181099983, 40.58541372599994], [-73.93126944999997, 40.58543639299992], [-73.93131673999989, 40.58567129699987], [-73.9313279159999, 40.585737850999884], [-73.93134323699987, 40.58580563399986], [-73.93139947599984, 40.58607989499989], [-73.93131301499997, 40.58605420599988], [-73.92942869199987, 40.585488201999915], [-73.92858912, 40.58520725999994], [-73.92840477199982, 40.58526206699991], [-73.92762689299991, 40.58503505599989], [-73.92712081899994, 40.58487935699989], [-73.92678292699999, 40.58486143799989], [-73.92583335908428, 40.584953077110605], [-73.92571918722876, 40.58490337694544], [-73.92562656290309, 40.58486435424147], [-73.92552355379907, 40.58482507449003], [-73.92546985614952, 40.58478284309026], [-73.92542431690644, 40.584734620218555], [-73.92537889694988, 40.58470963220566], [-73.92536636945633, 40.58467613841216], [-73.92534111881649, 40.584656605871054], [-73.92532658357095, 40.58464687987799], [-73.9253052757855, 40.58464178904685], [-73.92529083990867, 40.58464073583231], [-73.92528564357285, 40.584651766138535], [-73.92528656418784, 40.58466505214801], [-73.92528057806628, 40.58467435291047], [-73.92525943415265, 40.58468375411181], [-73.92519787876832, 40.584676604775304], [-73.92517439669376, 40.58468369893753], [-73.92514493484022, 40.58469430202819], [-73.92511237793357, 40.584703776872516], [-73.9250699258873, 40.58470808620263], [-73.92502668581022, 40.58471069338554], [-73.92496894248234, 40.58470642798466], [-73.92491416077436, 40.58469461802292], [-73.92488747621792, 40.584689483876296], [-73.92482669484389, 40.58467779007523], [-73.92476270182271, 40.58465968695739], [-73.92465843183506, 40.58463370782377], [-73.92461433600955, 40.584628794034025], [-73.92457168715535, 40.58461575537051], [-73.92455343880236, 40.58461181885605], [-73.92454209517814, 40.58461477476193], [-73.92453696487618, 40.58463159715458], [-73.92453331122891, 40.58464608802476], [-73.92449855479425, 40.58465902004985], [-73.92447425376244, 40.58465861915651], [-73.92443091389757, 40.584652524955104], [-73.9243921770264, 40.58465101706278], [-73.92435810855726, 40.5846599151546], [-73.92432019498605, 40.58466304721778], [-73.92428379150421, 40.58466675592001], [-73.92423972762631, 40.584664723322064], [-73.92420858699315, 40.58466316601205], [-73.92419040422422, 40.584665020237125], [-73.92414943223648, 40.58466702478021], [-73.92411977008088, 40.58466316292721], [-73.92409849487534, 40.58466097989905], [-73.92407731884595, 40.58466747023642], [-73.9240456500917, 40.584687349179895], [-73.92402980071381, 40.58469439377297], [-73.92400490615093, 40.5847096101003], [-73.92397692219477, 40.58471788127378], [-73.92393894021765, 40.58471810400934], [-73.92390415048486, 40.58472818091492], [-73.92387090509489, 40.58474169108284], [-73.92383976587256, 40.5847401598742], [-73.92380030285037, 40.58474271518985], [-73.92375627067007, 40.58474356317658], [-73.92374259286065, 40.584741330996394], [-73.92373128195398, 40.58474719795259], [-73.92372605222299, 40.58475534520569], [-73.92372841881188, 40.584763416853924], [-73.92372776000664, 40.5847732710952], [-73.92369747506775, 40.584779234114414], [-73.92368152483749, 40.584780514140945], [-73.92366251887553, 40.58477772918944], [-73.9236321999026, 40.5847808109355], [-73.92360336279948, 40.58478155982771], [-73.9235745230812, 40.58478233678799], [-73.92354117972411, 40.58478717278925], [-73.92350796271555, 40.584806474244104], [-73.92348001315959, 40.58481765402428], [-73.92344966307486, 40.58481785398265], [-73.92336761870938, 40.58481316077543], [-73.92330757405333, 40.58480658717107], [-73.92327791316033, 40.5848027249586], [-73.92327255614914, 40.58479638092647], [-73.92325805473139, 40.5847895362346], [-73.92323447490277, 40.58478795582878], [-73.9232025479314, 40.58478178624694], [-73.92316062204868, 40.584767568191914], [-73.9231127445224, 40.58476267883551], [-73.9230877558018, 40.58476628396649], [-73.92306503049772, 40.58477222552998], [-73.92303385793461, 40.58476778496191], [-73.9229448471968, 40.58474752403041], [-73.92287786521419, 40.58473114700482], [-73.92282834680518, 40.584714095224555], [-73.92278033793175, 40.58469762217819], [-73.92276198979245, 40.584687893241565], [-73.92274216376683, 40.5846776150609], [-73.92271703950689, 40.58467254710608], [-73.92269579781666, 40.5846732740365], [-73.92266396482276, 40.58467866100204], [-73.92261003898298, 40.584674369055755], [-73.92258724882197, 40.58467451822009], [-73.92253407944777, 40.58466907546317], [-73.92245200364532, 40.58466147167662], [-73.9223965951657, 40.58466242301596], [-73.92234875056444, 40.584660387027085], [-73.92232306860294, 40.58467096404474], [-73.92229117085057, 40.58467058609264], [-73.92227673549401, 40.58466953294011], [-73.92225174191464, 40.58467604905821], [-73.92221382879028, 40.58467917977125], [-73.92216374696794, 40.58468065586813], [-73.92211281118044, 40.58467461004742], [-73.92205510173692, 40.584673226016584], [-73.9219678630824, 40.584676679788], [-73.92192310809092, 40.58468159097784], [-73.92187674408291, 40.584674366949294], [-73.92182955621489, 40.584665387446805], [-73.92177263622186, 40.58466575988431], [-73.92170654401191, 40.58465978702819], [-73.92165711794213, 40.58465431860372], [-73.92160322497085, 40.58465290952334], [-73.92153193395391, 40.5846579938254], [-73.92148629379541, 40.58464961783133], [-73.92143088467141, 40.58465053919734], [-73.92139293822753, 40.58465076177563], [-73.92135650311464, 40.584651587750194], [-73.9213147401029, 40.58465183215767], [-73.92126153570996, 40.584646389845624], [-73.92123411391309, 40.58463613316941], [-73.92120843103847, 40.58464671006229], [-73.92118797609473, 40.58465204824697], [-73.92114999573468, 40.5846493871823], [-73.92110968386204, 40.58464153728771], [-73.92108374176723, 40.58462897524493], [-73.92107282496437, 40.5846047309178], [-73.92104987508381, 40.58458753278874], [-73.92101255523748, 40.58457501880612], [-73.92097161604326, 40.58457990304708], [-73.92092557569407, 40.584604492442864], [-73.92086664424144, 40.584631488024314], [-73.92081748274697, 40.58464627634087], [-73.9207736146413, 40.58466159023174], [-73.92073119220957, 40.58467168976596], [-73.92071070781651, 40.58467126255376], [-73.9206673680099, 40.584665166807305], [-73.92062166083436, 40.584650972220665], [-73.92059058710986, 40.58465520518997], [-73.92055119089721, 40.584663576865765], [-73.92051853961806, 40.58466146803614], [-73.92047746876065, 40.58465477015312], [-73.92044708686242, 40.58465205946204], [-73.9204266297896, 40.58465739624062], [-73.92038424385059, 40.58466749706092], [-73.92035159113443, 40.58466538770472], [-73.92032272007835, 40.58466325417151], [-73.92028625217345, 40.584661170222994], [-73.92024896609914, 40.58465156511864], [-73.92021920679475, 40.584639028142014], [-73.92019563076693, 40.5846345379349], [-73.92018513914945, 40.58464792511611], [-73.92017316611688, 40.584663616800896], [-73.92014899576304, 40.584674771837605], [-73.92012015689424, 40.5846755198759], [-73.92009138359737, 40.5846820598474], [-73.92005107000836, 40.58467420901524], [-73.92000618720687, 40.58466465286822], [-73.91998043707675, 40.5846723486847], [-73.9199457111396, 40.58468818977426], [-73.91991558489718, 40.584711526425174], [-73.91988779254024, 40.58474005399562], [-73.91988338562933, 40.58475281382352], [-73.9198682575901, 40.58475870424489], [-73.91984853078816, 40.584757098097676], [-73.91983330502325, 40.584754287910506], [-73.91982199311455, 40.58476015392272], [-73.91981686088313, 40.584776975958334], [-73.91980409869021, 40.584790939066565], [-73.91977009476832, 40.584805626996406], [-73.91972168778938, 40.58482211613802], [-73.91967772176935, 40.58482875678562], [-73.9196610169716, 40.584828305063716], [-73.91964661390365, 40.58483013376188], [-73.91964214014378, 40.58483710290164], [-73.91964529779912, 40.58484690373697], [-73.91964457281003, 40.58485096511039], [-73.91962490654521, 40.58485803380253], [-73.91958942401223, 40.584875053196185], [-73.91955459723633, 40.58488510328241], [-73.91951750146096, 40.58489575386746], [-73.91947054421905, 40.5849041485363], [-73.91942049622301, 40.58490850321154], [-73.91937580400347, 40.58491920492258], [-73.9193287168366, 40.58491601510199], [-73.91929297386154, 40.584909896860836], [-73.91927705899954, 40.58491114777053], [-73.91920586215205, 40.584927786223666], [-73.91916952508687, 40.58493728597216], [-73.91912638043381, 40.58494853608943], [-73.9190947752347, 40.584974205955085], [-73.91906471319794, 40.58500333392852], [-73.91905651889367, 40.585015940773815], [-73.91904558254164, 40.585027273269596], [-73.91903223468157, 40.585036989876855], [-73.9189907896268, 40.58504159988288], [-73.91895996358046, 40.58504819221252], [-73.91891800789304, 40.585059691014564], [-73.91888191484048, 40.585074432801896], [-73.91887078202592, 40.58508136983834], [-73.91885704690988, 40.58509642845811], [-73.91885072980678, 40.585110231197056], [-73.91885357481188, 40.58513861378098], [-73.9188477987526, 40.58515162016086], [-73.91883139681732, 40.58516870923593], [-73.91880491087741, 40.585189084310024], [-73.91878746251156, 40.58520779193054], [-73.91877854743201, 40.58522568102734], [-73.91875783174495, 40.58523711184285], [-73.91873721872057, 40.58524552776272], [-73.9187126627855, 40.58525555293776], [-73.91867386954468, 40.58527032022578], [-73.91864411675294, 40.58528341897724], [-73.91860367512984, 40.585292503038126], [-73.91856856184334, 40.58530156299117], [-73.91853027186833, 40.585315478905926], [-73.91849417580627, 40.58533431108198], [-73.91845484179895, 40.58534987385609], [-73.9184219442395, 40.585371918004185], [-73.91839850282939, 40.585375249219055], [-73.91836770486005, 40.58538793784241], [-73.9182735591321, 40.58541630618432], [-73.91824643996013, 40.58542331388145], [-73.91823895680054, 40.58541844979083], [-73.91823036218165, 40.58541117092257], [-73.91823404896664, 40.585399369839514], [-73.91823982570483, 40.5853863645642], [-73.91824820621798, 40.585363175901406], [-73.91826511878484, 40.58534120305663], [-73.91827146785576, 40.58533140808124], [-73.91826712194714, 40.58532168793035], [-73.91825592986801, 40.585318468763305], [-73.91823994698544, 40.58531650814224], [-73.9182245319517, 40.58532182211893], [-73.9181937597851, 40.58534266121693], [-73.91816241278411, 40.58536226392253], [-73.91815930380118, 40.58537324092017], [-73.91814559186984, 40.58539845590683], [-73.91814993016737, 40.58541226592164], [-73.91816118681702, 40.58542358221694], [-73.91815112990798, 40.58543296197253], [-73.91812882886319, 40.58544483180023], [-73.9180990127636, 40.58544980595572], [-73.9180447140579, 40.585455667958115], [-73.91802016905545, 40.58544845985359], [-73.91798290071343, 40.58545263278522], [-73.9179361149062, 40.58546742024709], [-73.91792602311764, 40.58547680106673], [-73.9178910616563, 40.58550820433879], [-73.91787462576838, 40.58552328917214], [-73.91785340457652, 40.58553554292887], [-73.91783158505767, 40.5855384641074], [-73.91781340858658, 40.585533647905095], [-73.9177926701432, 40.58553494996174], [-73.91778308723293, 40.58553944504271], [-73.91776931508163, 40.585554531821394], [-73.91776246144475, 40.58556509491475], [-73.91774656917059, 40.58557735206884], [-73.91773324586171, 40.58557942908905], [-73.91771672592182, 40.58557623213945], [-73.9177055554971, 40.58558316860573], [-73.91769711376763, 40.58559620098828], [-73.917684391637, 40.58560760983215], [-73.91767005590565, 40.5856133637893], [-73.91761790796343, 40.58562611720423], [-73.91756842886592, 40.585634837923806], [-73.91752738910465, 40.58563252983091], [-73.9175140048618, 40.58562445016409], [-73.91750062805787, 40.58561230912371], [-73.91748137764621, 40.58560304522633], [-73.91745201487552, 40.58559094791693], [-73.9174183969112, 40.585583347497554], [-73.91736083787715, 40.58557584007939], [-73.91734059880369, 40.58558038228433], [-73.91733000611228, 40.5855864951505], [-73.91732099320654, 40.585596288141474], [-73.91731785281506, 40.58560320298929], [-73.917332790029, 40.585606809205586], [-73.91735086460804, 40.58560553188344], [-73.91735676211395, 40.585610779089116], [-73.91735949096542, 40.58561687400961], [-73.91735050897289, 40.5856307008998], [-73.91733679717491, 40.585655943826815], [-73.91731725148759, 40.585682005105895], [-73.91730608489028, 40.585686909904645], [-73.91728692782108, 40.58568983298939], [-73.91727624210911, 40.585683759690035], [-73.91726764686683, 40.58567647995025], [-73.917253758239, 40.585669223039254], [-73.91723939050807, 40.58567088809596], [-73.9172325088335, 40.58567538378926], [-73.91722134212345, 40.58568028891875], [-73.91719685570743, 40.58568323522974], [-73.9171515959741, 40.58568745687068], [-73.91713348553323, 40.58568873491521], [-73.91711540304983, 40.58569610515653], [-73.91709732310542, 40.58570144483368], [-73.91707762657224, 40.585703132108954], [-73.91706646222588, 40.58570600583345], [-73.9170643642343, 40.58571333286391], [-73.91706334718288, 40.5857210449409], [-73.91707140079066, 40.585729148198695], [-73.91707247680206, 40.58573362310762], [-73.91706848518903, 40.585736469753876], [-73.91703907035588, 40.58575849447719], [-73.91703265900931, 40.585760165309765], [-73.91702634637925, 40.5857699305656], [-73.9170274534233, 40.58577844085546], [-73.91703876934456, 40.585797910754], [-73.91703777686546, 40.585813746427874], [-73.91702778060761, 40.585833281686234], [-73.91702143445264, 40.58584101796852], [-73.9170225449732, 40.58584549281176], [-73.91702577438326, 40.58585482686872], [-73.91701358970802, 40.585867444762776], [-73.91699186041458, 40.58588255345676], [-73.91697482395934, 40.585888304160385], [-73.91695779321046, 40.58588999361683], [-73.91692798089822, 40.58589090637801], [-73.91690133788347, 40.58589099583869], [-73.91689071090967, 40.585897109678164], [-73.91687849705549, 40.58590366029057], [-73.91685137450696, 40.585914729607154], [-73.916826413246, 40.585922534830516], [-73.91679336548403, 40.585920205362974], [-73.91677519324223, 40.58591335700052], [-73.91676079252959, 40.58591298973111], [-73.91674588603664, 40.585913446673274], [-73.91672834696432, 40.58591798975894], [-73.91669002681094, 40.5859278701741], [-73.91664168755146, 40.58594713216046], [-73.91658474943861, 40.5859611162353], [-73.91653688591346, 40.58597348753456], [-73.91647093984702, 40.58599320205486], [-73.91640867322334, 40.58600723829094], [-73.91636452334217, 40.58601590663648], [-73.91632781802159, 40.58602943924281], [-73.91628477437516, 40.58604664566343], [-73.91625445075606, 40.58605447315751], [-73.91623259946527, 40.586047879018004], [-73.91621741107193, 40.5860451303454], [-73.9162060865486, 40.58604633416271], [-73.91618892396225, 40.58605147032728], [-73.91613643951098, 40.58607328457132], [-73.9161118133772, 40.586084811718], [-73.91607976201513, 40.58609286959774], [-73.9160548296259, 40.58609995764872], [-73.91603052799135, 40.58611099264595], [-73.91600427151542, 40.58612498243353], [-73.91597154288236, 40.5861401882691], [-73.91594435422995, 40.58614579251291], [-73.9159168763074, 40.58614325812616], [-73.91588942769874, 40.58613332766475], [-73.91586132497898, 40.58612561564516], [-73.9157995907863, 40.58611732663337], [-73.91578373549699, 40.58611926053254], [-73.91576429525115, 40.586127841379856], [-73.91575583739197, 40.58613940894997], [-73.91575157363133, 40.58615320555848], [-73.91572385197135, 40.58619441321772], [-73.91570763002198, 40.58620670132931], [-73.9156936900972, 40.58621431017305], [-73.9156609884719, 40.58622335169441], [-73.91560372717565, 40.586227895097174], [-73.91549894226416, 40.58622837528565], [-73.91546951520041, 40.58622756256615], [-73.91544945077013, 40.58623096456613], [-73.91543485999519, 40.58623955881032], [-73.91541313034281, 40.58625405237225], [-73.91538726288206, 40.58625275474167], [-73.91535428832987, 40.58624996152778], [-73.91527796350833, 40.58625026596539], [-73.91524915677222, 40.58625586550119], [-73.91523328597421, 40.5862614960175], [-73.91522159402, 40.58627305563373], [-73.91521120965783, 40.58628141355368], [-73.91519436770284, 40.58628728928113], [-73.9151659108836, 40.58628623294222], [-73.91515204156394, 40.586277815404024], [-73.91513331918175, 40.58626938540072]]]]}}, {\"id\": \"154\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 155, \"Shape_Leng\": 0.169808473032, \"Shape_Area\": 0.000868261018498, \"zone\": \"Marine Park/Mill Basin\", \"LocationID\": 155, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90464209220775, 40.62513319188153], [-73.90396106711259, 40.62483361555412], [-73.90385540661482, 40.62494019869969], [-73.90387959925984, 40.6249494946642], [-73.9038667886285, 40.62496884844075], [-73.9042953851381, 40.625183232289196], [-73.90500868576218, 40.62554514495986], [-73.90546206137252, 40.62577481968345], [-73.90544562242668, 40.62579278066394], [-73.9054129826676, 40.625775618132344], [-73.90532958044659, 40.62586495608443], [-73.90531870381365, 40.62585846560609], [-73.90539726701537, 40.625767267628675], [-73.90531020209858, 40.62572688098664], [-73.90529638710873, 40.62571990107963], [-73.90521440773834, 40.625678482723266], [-73.90520180604905, 40.6256721159555], [-73.90512160694797, 40.62563159594963], [-73.90510413545952, 40.625622769682664], [-73.9050236675441, 40.62558211318544], [-73.90500969727343, 40.62557505521469], [-73.90492895768172, 40.62553426239311], [-73.90491413580976, 40.62552677382754], [-73.90483360089087, 40.62548608430568], [-73.9048175268759, 40.62547796344271], [-73.90473668710963, 40.62543711930621], [-73.90471864600124, 40.62542800368862], [-73.90463581485054, 40.625386154332936], [-73.90462553017346, 40.625380958774535], [-73.90454359250501, 40.62533955907061], [-73.90452861581954, 40.62533199242571], [-73.904448123116, 40.62529132411126], [-73.90443316820429, 40.6252837683614], [-73.90435248995925, 40.625243005339264], [-73.90433438062468, 40.62523385509351], [-73.90425570849342, 40.6251941060489], [-73.9042399340885, 40.625186137303], [-73.90416026610588, 40.62514588444185], [-73.90414446601989, 40.625137900918794], [-73.90406470521154, 40.62509760234598], [-73.904047766152, 40.62508904306776], [-73.90396618557334, 40.6250478245145], [-73.90395484264204, 40.625042093260014], [-73.90386956154684, 40.62499900561046], [-73.90385806241612, 40.62499319514218], [-73.9037806175106, 40.624954064811114], [-73.90377017455432, 40.624948788797305], [-73.90369982053774, 40.624913241924276], [-73.90368742940352, 40.624906980541795], [-73.90361911089087, 40.62487246194098], [-73.90360313362555, 40.62486438923693], [-73.9035373680356, 40.62483116052178], [-73.90352248737133, 40.62482364324692], [-73.90345432810342, 40.6247892030849], [-73.90344089448031, 40.624782416602585], [-73.90342223545693, 40.6247729884905], [-73.90335985536355, 40.62474146930103], [-73.90329059682475, 40.62470647577493], [-73.90327787444245, 40.62470004721078], [-73.90320897829281, 40.62466523688162], [-73.9031407632169, 40.6247453816744], [-73.90312443792637, 40.624737953261196], [-73.90320848278458, 40.62464076938306], [-73.90315226124025, 40.6246133871958], [-73.90307005655593, 40.6247054984719], [-73.90305433084016, 40.62469945601756], [-73.90312436585704, 40.62461885378026], [-73.9030375683653, 40.624575406893996], [-73.90296568765461, 40.624539426225304], [-73.9029303690017, 40.62452174741623], [-73.90288263908505, 40.62449785653049], [-73.90286791182726, 40.62449048471511], [-73.90279838836062, 40.6244556839956], [-73.9027845026159, 40.624448732875614], [-73.90271162732654, 40.62441225448336], [-73.9026992455198, 40.624406056662714], [-73.90262747173355, 40.624370129146264], [-73.90255865671568, 40.6244488881276], [-73.90254656112616, 40.62444423916955], [-73.90263059517203, 40.624349364751005], [-73.90313231786656, 40.62460225619683], [-73.90317372279071, 40.624555741015534], [-73.90320334369764, 40.62457058890886], [-73.9032704594402, 40.62448418394102], [-73.90302283562494, 40.62436795022765], [-73.90288536052951, 40.624310565122265], [-73.90269333271974, 40.62421333487328], [-73.90283262223241, 40.62405099444945], [-73.9027862895168, 40.624039182861765], [-73.90275548257644, 40.62406806005402], [-73.9027189882889, 40.62405239371349], [-73.90241543050296, 40.62446290725004], [-73.9022870793228, 40.624409853930224], [-73.90229694049818, 40.62439651880779], [-73.9024038097296, 40.62444239705086], [-73.90271369573058, 40.624023326835896], [-73.90261908735016, 40.62398271330393], [-73.90262364456693, 40.62397655126536], [-73.90271937115764, 40.62401764604501], [-73.90274531019497, 40.62398256701517], [-73.90263558802576, 40.62393546354271], [-73.90264355554693, 40.62392468930722], [-73.90270209085729, 40.62394981859843], [-73.90271236759573, 40.62393592106214], [-73.90283449085966, 40.62398834946012], [-73.90283456578874, 40.62398838135389], [-73.90283958140392, 40.6239905336476], [-73.90284448473464, 40.62399263877051], [-73.90284816261747, 40.6239942178223], [-73.90287492374995, 40.62395802673141], [-73.90287435266721, 40.62395778031665], [-73.90287130277893, 40.623956472355346], [-73.9028681966426, 40.62395513804433], [-73.90286800877114, 40.62395505763931], [-73.90273926215669, 40.62389978674032], [-73.90275168375602, 40.62388298807522], [-73.90288181620222, 40.62393885251317], [-73.90291619574111, 40.62395453125644], [-73.90288823724384, 40.623936305300425], [-73.9028968143141, 40.62392696594419], [-73.90291133815845, 40.623911151761725], [-73.90293442885333, 40.62389407175921], [-73.9029594628466, 40.62382474734768], [-73.90295453165464, 40.623787220244544], [-73.9029283597793, 40.62378054859442], [-73.9029246363893, 40.62376914628601], [-73.90295209682361, 40.62374684608528], [-73.90294467424503, 40.62370694265951], [-73.90293596678084, 40.62369363569648], [-73.90294328774249, 40.623636338303484], [-73.90284160059183, 40.623452737299786], [-73.90280760743192, 40.62345833803819], [-73.90278104436183, 40.6234357983618], [-73.90276039965381, 40.6234065089225], [-73.90273527475455, 40.62341099067617], [-73.90271602565339, 40.6234391205746], [-73.90267467098022, 40.62342670167291], [-73.90263183552673, 40.62341653325734], [-73.90258451617723, 40.623441261639854], [-73.90258004442525, 40.62346827807484], [-73.90254016047618, 40.62346036287619], [-73.90251801478948, 40.62344796020075], [-73.90245738131404, 40.62348393635161], [-73.90236135937829, 40.62346921851604], [-73.90234516923223, 40.62342529629276], [-73.90234523676943, 40.62337913612289], [-73.90233347621846, 40.62333859537674], [-73.90232022224446, 40.62330818667751], [-73.90229070515272, 40.62328451791349], [-73.90227302978802, 40.62324622405616], [-73.90222431373307, 40.62321578452483], [-73.90219926673512, 40.623166225777446], [-73.90221554084374, 40.623152729257036], [-73.90219931622605, 40.62313245016115], [-73.9021638700949, 40.62312228718644], [-73.9021033754338, 40.62306369206217], [-73.90207683368587, 40.62302538945052], [-73.90206356590346, 40.62300511285368], [-73.9020901880153, 40.622987121990164], [-73.90205918813636, 40.62296795608351], [-73.90202817351549, 40.622957797009555], [-73.90199713928754, 40.62296114777196], [-73.90197796683574, 40.62293636295585], [-73.90196913303357, 40.62291496416703], [-73.90193219544054, 40.622913806702414], [-73.90189823888299, 40.62289463890007], [-73.90184060318614, 40.622901344634165], [-73.90181844313125, 40.622899073285616], [-73.90178890563845, 40.6228889154084], [-73.90176672305017, 40.622902406820494], [-73.90174161518455, 40.62289450514146], [-73.90169728160188, 40.62289896974676], [-73.90167662514698, 40.6228786867867], [-73.90164559635443, 40.62287866024487], [-73.90163675271114, 40.62286401685523], [-73.9016116367975, 40.622861744110466], [-73.90161018895355, 40.622841476905876], [-73.90158064619612, 40.62283582264878], [-73.90156294269181, 40.62281666748739], [-73.90151570542957, 40.62278622959029], [-73.90149057503623, 40.622794088281545], [-73.90146543654427, 40.622807577772804], [-73.90145658297767, 40.622799688789115], [-73.90142702629066, 40.62280304116689], [-73.90139898058246, 40.62278387711191], [-73.9013517044865, 40.62277933404919], [-73.90100332746997, 40.62255611565839], [-73.9009413108596, 40.62252791487642], [-73.90093851338591, 40.622422082640874], [-73.90095693119575, 40.622406398877395], [-73.90095657102668, 40.62237189650028], [-73.90091875871514, 40.622332657807604], [-73.90092000809851, 40.62232324979318], [-73.90093689110948, 40.622319500136506], [-73.90094433639536, 40.62229504164157], [-73.90094270330684, 40.62228625853887], [-73.90092994243166, 40.622285932923496], [-73.90089885737538, 40.622261051548925], [-73.90084422502528, 40.62223623596836], [-73.90084869383492, 40.622211469674454], [-73.9008162734123, 40.622154022864166], [-73.90085618850341, 40.62213942146962], [-73.90086511381486, 40.62209889907488], [-73.90088142638618, 40.62205950834013], [-73.9008858937174, 40.62203586850723], [-73.90092435763049, 40.62200437775499], [-73.90089196289043, 40.621928917947564], [-73.9008240556225, 40.621888328119624], [-73.90082851794222, 40.62186806717416], [-73.90074590109352, 40.62178355612913], [-73.90075776586589, 40.621753168927974], [-73.90071952540437, 40.62163492049855], [-73.9006205611866, 40.621613444735786], [-73.90057471065899, 40.62164492849703], [-73.90052594431344, 40.62164938948678], [-73.90050676271676, 40.62163135867501], [-73.90051418685034, 40.621607722586994], [-73.9005024221681, 40.62156943356443], [-73.90048169037065, 40.621600939528456], [-73.90046101789774, 40.62159191427346], [-73.90042112502233, 40.62159075443377], [-73.90041231877402, 40.62155134200269], [-73.90035318038561, 40.62157605923211], [-73.90032359224014, 40.62160080255892], [-73.9002836790366, 40.621614277809286], [-73.9002747811528, 40.62163566135083], [-73.90016300923918, 40.62167709474681], [-73.90011689338732, 40.62168736190277], [-73.90009355346207, 40.621695447003816], [-73.90006989359243, 40.621702970767856], [-73.900045936871, 40.621709928858955], [-73.90001980348657, 40.6217199833311], [-73.89999332307809, 40.62172949956018], [-73.89996651632606, 40.62173846935516], [-73.899939401271, 40.62174688485742], [-73.89991199748695, 40.621754742398316], [-73.89988432499028, 40.621762036466656], [-73.89985640556002, 40.621768758872115], [-73.89982825789053, 40.62177490628037], [-73.89979990441465, 40.621780474355056], [-73.89977136294782, 40.621785459258405], [-73.89974265811907, 40.621789858331205], [-73.89971380554923, 40.621793665388324], [-73.89968483030536, 40.62179687894402], [-73.89965575349534, 40.6217994985138], [-73.89962659403483, 40.62180152009361], [-73.89959737567133, 40.62180294219624], [-73.89956811841279, 40.621803765341745], [-73.89955445196465, 40.62179690348092], [-73.89954039322575, 40.62179051890058], [-73.89952597097869, 40.62178462419038], [-73.89951121357092, 40.62177922925895], [-73.89949615483924, 40.62177434854341], [-73.89948081895608, 40.621769990106245], [-73.89946524438575, 40.621766161184446], [-73.89944945771525, 40.62176287285836], [-73.89943349257463, 40.62176013068557], [-73.89941738171501, 40.621757939888234], [-73.89940115832995, 40.62175630401324], [-73.89938485407163, 40.62175522811426], [-73.89936850279855, 40.621754712053374], [-73.89935213792377, 40.621754759712836], [-73.89933579374747, 40.621755369615066], [-73.89931950127078, 40.6217565411171], [-73.89930329567727, 40.62175827039643], [-73.89928720972922, 40.6217605553042], [-73.8991827974815, 40.62178180294394], [-73.89907849548186, 40.62180337877469], [-73.89897430878847, 40.621825280790944], [-73.89896106322365, 40.621821172231556], [-73.89894753866744, 40.62181763603917], [-73.8989337746791, 40.621814682635105], [-73.89891981455663, 40.6218123219417], [-73.8989057042379, 40.621810562208], [-73.89889148527308, 40.62180940615118], [-73.89887720250371, 40.62180886000912], [-73.8988629051774, 40.62180892483031], [-73.89884863352947, 40.62180960014752], [-73.89883443439226, 40.621810884326706], [-73.89882035196477, 40.62181277305115], [-73.89880643066871, 40.621815259658625], [-73.89879271426739, 40.62181833698409], [-73.8987792458673, 40.62182199601911], [-73.89876429026145, 40.621823370659314], [-73.8987492634061, 40.62182417429647], [-73.89873420070202, 40.6218244052862], [-73.89871914172656, 40.62182406265846], [-73.89870412473198, 40.62182314912738], [-73.89868918489748, 40.621821664221294], [-73.89867436046924, 40.621819614507615], [-73.8986596896992, 40.62181700337062], [-73.89864520775548, 40.62181383737513], [-73.898630952885, 40.62181012258581], [-73.89861695826836, 40.62180587176444], [-73.89861434721522, 40.62180165074944], [-73.89861109970155, 40.62179769202599], [-73.89860726224767, 40.62179405510761], [-73.89860288776403, 40.62179079046723], [-73.89859803554845, 40.62178794087707], [-73.89859277633204, 40.62178554895142], [-73.89858718283976, 40.6217836472553], [-73.89858133351967, 40.621782263667605], [-73.8985753103544, 40.62178141568348], [-73.89856919775052, 40.621781116612425], [-73.89856308232926, 40.62178136921108], [-73.89855704742023, 40.6217821725479], [-73.89855118164952, 40.62178351229351], [-73.89854556221432, 40.621785371595784], [-73.89850515009056, 40.62179683661469], [-73.8984643680204, 40.62180751574969], [-73.89842324306126, 40.621817400145325], [-73.89838180270591, 40.621826484464655], [-73.89834211373747, 40.62183010167243], [-73.89830237647831, 40.62183351996709], [-73.89826260368, 40.62183673952729], [-73.8982228118341, 40.621839759865054], [-73.89818301655689, 40.62184257697343], [-73.89811767568024, 40.621851562126416], [-73.89810966198586, 40.621850642661855], [-73.89804258216508, 40.62184295208139], [-73.89800838382129, 40.62183749146526], [-73.89796188892215, 40.621830068883995], [-73.8978722356107, 40.621813760416295], [-73.89772094578028, 40.6217854380091], [-73.89763690014603, 40.62176657182496], [-73.8976223342305, 40.62176228737035], [-73.89758984528348, 40.62174944682932], [-73.89758135050047, 40.62174712652557], [-73.89757415829762, 40.62174516137009], [-73.89757314558577, 40.62174460494207], [-73.89750149948757, 40.621705209826835], [-73.89748094470643, 40.6216956671465], [-73.89744594264127, 40.62168764798257], [-73.89739897897786, 40.621682061709464], [-73.89705223967074, 40.621683965621365], [-73.89699450437195, 40.621673510026135], [-73.89692451062226, 40.621655116447066], [-73.89684302276225, 40.621620064831596], [-73.89680032629262, 40.621571098359645], [-73.89675081965714, 40.62151432225982], [-73.89666488195714, 40.6214773501577], [-73.89661882635251, 40.62146194501101], [-73.89657304085416, 40.621462369893536], [-73.89651464215949, 40.62144447618244], [-73.89647475412445, 40.621428246088904], [-73.89645921169515, 40.6214159073261], [-73.89644123346895, 40.621394756190135], [-73.89636674052934, 40.621231895372894], [-73.89630006047612, 40.62107129230283], [-73.89630054609012, 40.62101472309059], [-73.89630687672837, 40.620997464612906], [-73.89631329635806, 40.62096919789126], [-73.89631753490583, 40.62090700527], [-73.89631373620297, 40.62082432867684], [-73.8963058886522, 40.620775573536164], [-73.89630104744656, 40.620761077751766], [-73.89628847802915, 40.62073649997461], [-73.8962790496671, 40.62073050834114], [-73.89627652049418, 40.62073582929], [-73.89627175922178, 40.62076313622585], [-73.89626695075945, 40.62079675560786], [-73.89625097822169, 40.62082998177603], [-73.89623478359779, 40.62084341604472], [-73.89622243312262, 40.62084022100945], [-73.89621383518804, 40.62083293947778], [-73.89620462845744, 40.6208002132736], [-73.89612551516502, 40.62064423757569], [-73.89610054469308, 40.620625880311806], [-73.89608370284704, 40.620615433734294], [-73.89607842200743, 40.6206062889185], [-73.89607523203932, 40.62059558177542], [-73.89608043035841, 40.62056607994352], [-73.89608070054213, 40.62053149635783], [-73.89607184210558, 40.620506015579394], [-73.89605303119886, 40.620487691348096], [-73.89603159920458, 40.62048725967842], [-73.89602082120946, 40.620492546859325], [-73.89600274852945, 40.620485559801054], [-73.89599175101233, 40.62046946540124], [-73.89596971501871, 40.62044043297343], [-73.89594222035427, 40.6204242686892], [-73.89590647248617, 40.62040963350072], [-73.89587644639052, 40.62040038434475], [-73.89585305268275, 40.62038737966316], [-73.89581582469421, 40.620351664785574], [-73.89572932995614, 40.6202409152581], [-73.8957180308928, 40.620211316554965], [-73.89571728892822, 40.62020157189784], [-73.89572611095683, 40.620180581658545], [-73.89574243647453, 40.6201529838475], [-73.8957467992111, 40.620126308991814], [-73.89573667625618, 40.62010425853446], [-73.89574632647593, 40.62008354412711], [-73.89576993937993, 40.620071406625165], [-73.89577962782305, 40.62004915399918], [-73.89577740344706, 40.620019920532144], [-73.89576287108353, 40.61998433476828], [-73.89574043818135, 40.61995593385401], [-73.89572773952914, 40.61994549215356], [-73.89570835936627, 40.61994508975928], [-73.8956988067658, 40.61995004916275], [-73.89567985869817, 40.619950581464444], [-73.89566550350563, 40.619939533809564], [-73.89568213618082, 40.61992231269127], [-73.89570116683535, 40.61991549427644], [-73.8957091110666, 40.61990202461988], [-73.89570509771258, 40.61988942230158], [-73.89569532006122, 40.61987618467692], [-73.89567436959614, 40.619867271388216], [-73.89565708473634, 40.61986374295433], [-73.89564146603549, 40.619854534287015], [-73.89561388501926, 40.61980128715059], [-73.89561842775316, 40.61975259916217], [-73.89564527067563, 40.61970151431144], [-73.89569398800683, 40.61959898559129], [-73.89571166246276, 40.61956010794417], [-73.89571934602989, 40.61952995015875], [-73.89571674120862, 40.619516004195766], [-73.89571326932348, 40.61950090467971], [-73.89569612286115, 40.619477668776796], [-73.89566886189472, 40.6194505529362], [-73.8956408733815, 40.61942727803452], [-73.89562440512043, 40.61940846131174], [-73.89561243615935, 40.61936928268317], [-73.89561044677646, 40.619350533384385], [-73.89562799133904, 40.61932694465632], [-73.89563743758063, 40.61932091465213], [-73.8956786407143, 40.61932443799253], [-73.89570104654028, 40.61932344226026], [-73.89572065812946, 40.619313085152875], [-73.89574031871707, 40.61929443842012], [-73.8957637468164, 40.61926207163293], [-73.89580103532275, 40.61921157268145], [-73.89582666510604, 40.619175886113155], [-73.89585293851255, 40.61914461987047], [-73.89586628685244, 40.61910831772145], [-73.89587519415221, 40.619078628270934], [-73.89590091378314, 40.619031934110886], [-73.89589974097906, 40.61899830982436], [-73.89589203704358, 40.61897184299282], [-73.89590229663737, 40.61895370923519], [-73.89592275902997, 40.6189284756239], [-73.89594745359078, 40.61891592744519], [-73.89597444319298, 40.618886306956284], [-73.89598707714167, 40.61884561254246], [-73.89599673768316, 40.618817542595544], [-73.8960086507563, 40.618777946516545], [-73.89601458972035, 40.618758135058414], [-73.89601480773337, 40.61873334867912], [-73.89601372545526, 40.61868871732069], [-73.8960199184326, 40.618644147492454], [-73.89603565085702, 40.61857976879041], [-73.89604900757561, 40.618537950232856], [-73.89606087614197, 40.618503870667716], [-73.89606105380388, 40.618481831223356], [-73.89606127673136, 40.61845430055603], [-73.89607322875578, 40.61841193123012], [-73.89608357745956, 40.6183827913869], [-73.89611869448018, 40.618338329449024], [-73.89616025744242, 40.61829502726245], [-73.89619952947552, 40.6182660222577], [-73.8962576889639, 40.61822775854356], [-73.89630432968094, 40.618185008204], [-73.89631961692115, 40.6181757176027], [-73.89632614336394, 40.618171358187105], [-73.89633851449186, 40.618160939661244], [-73.89634089502, 40.61813560791294], [-73.89635415386694, 40.61811034092256], [-73.89637957358076, 40.61809394935237], [-73.89640197406328, 40.61809569827314], [-73.89642073010339, 40.618102961974365], [-73.89643882054831, 40.618097544136745], [-73.89645347974444, 40.618075571010415], [-73.89645145560387, 40.618056849669884], [-73.89642638670509, 40.61803193178124], [-73.8964078514044, 40.61799988337679], [-73.89641441392176, 40.617995497016445], [-73.89642471559748, 40.61797187416253], [-73.89640262071784, 40.61793430548624], [-73.89639712615588, 40.617901774118366], [-73.89640301407015, 40.617890224450385], [-73.89643071664314, 40.61786502474569], [-73.89645032769116, 40.61785466717717], [-73.89647498570088, 40.61784211868376], [-73.89648375473702, 40.61783169522282], [-73.89646516413855, 40.61781065342933], [-73.89644505819918, 40.61779183454973], [-73.89643947672181, 40.61776756446039], [-73.89645130037977, 40.61773897490343], [-73.89647607844687, 40.617718164157246], [-73.89649856930203, 40.617708906488446], [-73.89653978363197, 40.617704167673814], [-73.89654715229243, 40.617690422858736], [-73.89652920407661, 40.61767380073216], [-73.8965316687724, 40.61764020719644], [-73.89654641678631, 40.61760722748475], [-73.8965560811242, 40.617576440171455], [-73.8965794728729, 40.617544045203], [-73.89660510673086, 40.61750561366508], [-73.89661900303167, 40.61748750941414], [-73.89662829741033, 40.61741648415351], [-73.89664335225238, 40.61734771314744], [-73.89664253106987, 40.61727277837982], [-73.89663729815841, 40.6172099447852], [-73.89663050089753, 40.61715756911899], [-73.89662424028307, 40.61712890782723], [-73.8966012956822, 40.61710621516143], [-73.89657411018267, 40.61707632732681], [-73.89654702264502, 40.61702991652715], [-73.8965215956256, 40.61695926895687], [-73.89648639538667, 40.61684892563744], [-73.89647077897715, 40.61681523238749], [-73.8964538113097, 40.616769981769615], [-73.89645340049675, 40.616732515555384], [-73.89645289839982, 40.616708827511644], [-73.89645900601768, 40.616672492460786], [-73.89646731947158, 40.6166301193066], [-73.89647055156632, 40.61658991141067], [-73.89647877731848, 40.61655857330992], [-73.89647976597487, 40.61652440147233], [-73.89647499164604, 40.61649077343723], [-73.89647385439008, 40.61645717615205], [-73.89646687893172, 40.6164268404237], [-73.89647146180651, 40.61639821518922], [-73.89648031491386, 40.6163795311165], [-73.89650208809937, 40.616368626492594], [-73.89651297380907, 40.616363174680885], [-73.89651162235695, 40.61635158984058], [-73.89650091074138, 40.61633777396605], [-73.89647860282666, 40.61632221941884], [-73.89645129330266, 40.616303364396884], [-73.89644304793725, 40.61625595380669], [-73.89644124174151, 40.61621242027649], [-73.8964538748985, 40.61617172647014], [-73.89646281830267, 40.6161420088535], [-73.89646002707948, 40.61612987389046], [-73.8964493155107, 40.61611605784324], [-73.8964104897835, 40.61608997528783], [-73.89638390254667, 40.61606999611067], [-73.8963594695086, 40.616055014918814], [-73.89633131361525, 40.61604826592165], [-73.8963002241372, 40.61605142076262], [-73.8962900198324, 40.616058521654324], [-73.89627832083232, 40.61607607668025], [-73.89627315092359, 40.616089274471946], [-73.89625578586792, 40.61609085133173], [-73.8962363544032, 40.61607919588967], [-73.89622640450689, 40.616061510421964], [-73.8961955415309, 40.61603436396911], [-73.89617052459134, 40.6160011837037], [-73.89616498434603, 40.615974169341214], [-73.89617244709056, 40.61594664688142], [-73.89620658955951, 40.615925350102344], [-73.89622718456361, 40.615907334871025], [-73.89624255855993, 40.615888574948116], [-73.89624565770123, 40.61586395634527], [-73.89623756851961, 40.6158542335352], [-73.8962143578366, 40.61584029594833], [-73.89617619316832, 40.61583035331735], [-73.8961674495314, 40.615824444713695], [-73.89615450745923, 40.615785812732575], [-73.89615177996474, 40.61575635850464], [-73.89616825783165, 40.61572173409067], [-73.89618196016559, 40.61568880892708], [-73.89617336977584, 40.61565449123097], [-73.89616531514629, 40.615644767944175], [-73.89614423131907, 40.6156302564668], [-73.89611041756297, 40.615624626422786], [-73.89609539639183, 40.61562590231658], [-73.89607657418811, 40.61561627851904], [-73.89607175222872, 40.61559011685286], [-73.89606843901986, 40.61556722169051], [-73.89608178433468, 40.61555449903938], [-73.89610524671173, 40.615545488759594], [-73.89609054371857, 40.61552653748974], [-73.89606075325278, 40.61550614305179], [-73.89603259976464, 40.61549717070589], [-73.8960050700419, 40.61548166461637], [-73.895987253043, 40.61545071409955], [-73.89598860943872, 40.6154119318619], [-73.89598948311652, 40.61538242610541], [-73.89599489372915, 40.615331570035174], [-73.89599134458564, 40.61529772345346], [-73.89598371160828, 40.61527323302143], [-73.89595293093242, 40.61524026762303], [-73.8959266731785, 40.61521710575263], [-73.89590336269954, 40.61519770522586], [-73.8958885243663, 40.61517329001603], [-73.8958410010153, 40.61509082135501], [-73.89580772810929, 40.61503929881658], [-73.89579481816479, 40.61500341262093], [-73.8957928770522, 40.61497829658824], [-73.8957918237687, 40.61496028978367], [-73.89578507514733, 40.61494562588269], [-73.89575285496711, 40.61491211130448], [-73.8957344515297, 40.6148877471201], [-73.89573706509346, 40.614875151477094], [-73.89575106482042, 40.614858612476155], [-73.89577659016369, 40.61484356672358], [-73.89578215002605, 40.614834762084286], [-73.89578754392873, 40.61481777668518], [-73.89580527744951, 40.61480936621943], [-73.89582832172617, 40.61481512262845], [-73.89584423213397, 40.61482092802116], [-73.89585156740996, 40.61482903234952], [-73.89586898182053, 40.6148408502098], [-73.89589284502115, 40.614850973321055], [-73.89591310967744, 40.61486109300155], [-73.89594911474681, 40.61486889391844], [-73.89597637791361, 40.614870757056906], [-73.89598917340122, 40.61486459284621], [-73.89600183471937, 40.61485296689965], [-73.89601717605761, 40.61483148848228], [-73.89602881667831, 40.61480460005198], [-73.89602128163696, 40.614785572327946], [-73.89600719861568, 40.614762776648455], [-73.89597285228832, 40.61472986347974], [-73.89593641173241, 40.61470024101983], [-73.89591934865109, 40.614670938652736], [-73.8959092860416, 40.61463338076165], [-73.89589071515321, 40.61460081121421], [-73.895866416726, 40.61456886722174], [-73.89586224922375, 40.61453886285349], [-73.89586112894177, 40.614518137494784], [-73.89584505008592, 40.614504097565536], [-73.89581611284078, 40.61449078712887], [-73.89574845601021, 40.61447683713038], [-73.89571673713985, 40.61446791474621], [-73.89568501893534, 40.61445899386163], [-73.89561883888243, 40.61444556547204], [-73.89558999916949, 40.61443768970803], [-73.8955825661107, 40.61442412315789], [-73.89557732518234, 40.61441272860193], [-73.89557757652038, 40.61438978120849], [-73.89557717490436, 40.614370650202936], [-73.89557193332423, 40.614359253970754], [-73.89555890646692, 40.614351749400186], [-73.89553639672285, 40.61435854486383], [-73.89553252926262, 40.614359712209314], [-73.895521927757, 40.614368019886996], [-73.89552942926865, 40.61438433079629], [-73.89552681480382, 40.61439692760706], [-73.89551772066596, 40.6144085017948], [-73.89550221294368, 40.61442177308391], [-73.89547940515911, 40.614426966855596], [-73.89545646109381, 40.61442667157818], [-73.8954181943245, 40.614414010107076], [-73.89535161149779, 40.614381451728946], [-73.8952740916741, 40.6143408404477], [-73.8951798123067, 40.614284596107396], [-73.89508186256663, 40.614225631353406], [-73.89495144332771, 40.61415612272576], [-73.89487398981485, 40.614118256029336], [-73.89478995879114, 40.614073877606], [-73.89473568611905, 40.614047204541144], [-73.89465257429408, 40.614012653776086], [-73.8946011447671, 40.61398702511607], [-73.8945274945732, 40.61396003112316], [-73.89445096385644, 40.6139319639275], [-73.894399016324, 40.6139156395982], [-73.89432769529049, 40.61389625125662], [-73.89427581408144, 40.61388264324812], [-73.89420959882699, 40.6138692161911], [-73.89415922473533, 40.613858876835586], [-73.8940930419325, 40.613848165051664], [-73.89401835332855, 40.6138397519273], [-73.89395164950118, 40.613838345483984], [-73.89391529273703, 40.613848028452225], [-73.89386882050955, 40.61385405161237], [-73.8938207402506, 40.613851316693626], [-73.89376473888517, 40.61384703970085], [-73.89371319266695, 40.6138498449018], [-73.8936774541801, 40.613855714400685], [-73.89364109666259, 40.61386542525889], [-73.89359452455123, 40.613865957451836], [-73.8935701409316, 40.613865139449835], [-73.89353934479138, 40.61386604473844], [-73.89348292372811, 40.61387653138529], [-73.89343222994763, 40.61388639307022], [-73.89337269463596, 40.61388491168508], [-73.89330589222904, 40.61387804148649], [-73.89326360875235, 40.613877426936696], [-73.89323713137844, 40.613879927010046], [-73.89320355276945, 40.613885220727596], [-73.89315059976579, 40.61389022201502], [-73.89306942949449, 40.61388075921319], [-73.8929960778081, 40.61387015153529], [-73.89291775644425, 40.613859043965135], [-73.89283216522163, 40.61384526819792], [-73.89277688473038, 40.61383991921006], [-73.89270552819843, 40.61382052815285], [-73.89267109788662, 40.61381873930027], [-73.89261149496224, 40.61381451215945], [-73.89255539456104, 40.613804770023], [-73.89249926249595, 40.61379231218374], [-73.8924350055877, 40.61377012902539], [-73.89230358303553, 40.613721942893896], [-73.89218519034605, 40.613678520124765], [-73.89207536848573, 40.61363554286668], [-73.89196849544119, 40.61359635711157], [-73.89189118166033, 40.61356392216907], [-73.8918680042854, 40.61355273020211], [-73.89179484323651, 40.613513685222244], [-73.891733954042, 40.61348052565789], [-73.89165791434924, 40.61344043434066], [-73.89157666468313, 40.61339166558281], [-73.89149171553888, 40.61333748560506], [-73.89142496896234, 40.61329674401335], [-73.891375801745, 40.61327600186071], [-73.89134028451792, 40.6132562074032], [-73.8913155690617, 40.61323899991267], [-73.89127831370406, 40.61320229475102], [-73.89123107591735, 40.61317005365965], [-73.89121431130582, 40.61315713671799], [-73.891209806409, 40.6131473231262], [-73.89120383206885, 40.613134317552834], [-73.89118725926157, 40.61309302069822], [-73.8911734991628, 40.6130500235712], [-73.89114036104236, 40.61300670788002], [-73.89108702975123, 40.61295595988698], [-73.89103674500853, 40.61291449435869], [-73.89098063257914, 40.61286816443583], [-73.89095414564672, 40.61283404731705], [-73.89094355366382, 40.612769177725546], [-73.89094910253573, 40.612723784626944], [-73.8909586166174, 40.61269744300786], [-73.89098259882945, 40.61267913104102], [-73.89102790276476, 40.612649666939646], [-73.89104355929425, 40.612623770900264], [-73.89104723360161, 40.612600855905775], [-73.8910457011399, 40.61256934475072], [-73.89103710302403, 40.61254068092289], [-73.89104410644784, 40.612509560997914], [-73.89106691390678, 40.612481475929734], [-73.89109229075407, 40.61246914839408], [-73.89112774540011, 40.61246042607494], [-73.89116522429462, 40.61244843934729], [-73.89120818997512, 40.61242767391717], [-73.89123374315199, 40.6123951991375], [-73.89125604245064, 40.61234809263646], [-73.89127258158636, 40.61229889410793], [-73.89130942516735, 40.61223417980853], [-73.89134386172833, 40.612187440430176], [-73.89137297960706, 40.61215491521179], [-73.89138307694229, 40.61214771530805], [-73.89141430474191, 40.61212544769949], [-73.89145771186591, 40.612120986688616], [-73.89149618167237, 40.61211877175369], [-73.89154010439341, 40.61210775226804], [-73.8915846815305, 40.6120929460533], [-73.89161708480994, 40.612077769328295], [-73.89164657746798, 40.612058831168355], [-73.8916642358476, 40.61205311061254], [-73.89168017762498, 40.61206158015833], [-73.8916763660208, 40.612079032927184], [-73.89167276013539, 40.612104639026576], [-73.89168448105909, 40.61211697418019], [-73.89171982690954, 40.612131115774474], [-73.89174190772161, 40.612129681248], [-73.89175874328289, 40.61211962374798], [-73.89177898056371, 40.612101390512024], [-73.89179636682488, 40.612084800226675], [-73.89181155973806, 40.61206609499764], [-73.89182922786112, 40.61203210473392], [-73.8918596094373, 40.612020220329356], [-73.89200214578041, 40.61198703197176], [-73.89207363854527, 40.61196582695135], [-73.89209633768667, 40.61196057880317], [-73.89211560559829, 40.611960843324525], [-73.89212863177866, 40.611968239058285], [-73.89211576721836, 40.61199455015632], [-73.892109144477, 40.61201367458646], [-73.89209765347529, 40.61203776369333], [-73.8920861604436, 40.612061879769286], [-73.89209637445802, 40.612070975129214], [-73.89212013413923, 40.61207821508561], [-73.89215284463695, 40.61207390912371], [-73.89217749989469, 40.61206265168467], [-73.8922013661422, 40.61204705667544], [-73.8922285622075, 40.612023256194334], [-73.89226337354145, 40.61199010366619], [-73.89232310695562, 40.6119565650354], [-73.89240262899736, 40.61191401173188], [-73.89247429597349, 40.61187268722245], [-73.89256424515882, 40.61181911110156], [-73.89263748842207, 40.611783744291934], [-73.89266643069745, 40.61177133731429], [-73.89270638042095, 40.61174406335355], [-73.89274879586955, 40.611729831137026], [-73.89279796128262, 40.61172743359008], [-73.89286362222313, 40.61172697329185], [-73.89293363286878, 40.61173079888306], [-73.89297737859249, 40.61173992438857], [-73.89301197002011, 40.611752445882814], [-73.89303367142404, 40.61176297769133], [-73.89307714661389, 40.61176123445901], [-73.89311499848357, 40.61176283471967], [-73.89315045293061, 40.61175411127376], [-73.89320774838846, 40.61173585727901], [-73.89326463906255, 40.61170127080633], [-73.89330629990764, 40.61168538969998], [-73.89335882918584, 40.611674816601614], [-73.8934180488139, 40.61164780670522], [-73.89348377908387, 40.61162450939241], [-73.8935342917296, 40.611591646780575], [-73.89359262137847, 40.61155758232602], [-73.89364470697576, 40.611530677282694], [-73.89369638020445, 40.61151573726294], [-73.89376203873022, 40.611515277622146], [-73.89383650564831, 40.61152610597933], [-73.89388460631375, 40.611536800218445], [-73.893932400722, 40.61153659672418], [-73.89396490703528, 40.61152413792637], [-73.89400217954798, 40.61150399862141], [-73.89403393868307, 40.6114643655653], [-73.8940422790149, 40.611428333563204], [-73.89404032982233, 40.61140878883044], [-73.89404098799922, 40.61137942058624], [-73.8940308819601, 40.61134748877293], [-73.89398851362914, 40.611310642562756], [-73.893968869809, 40.61129679123333], [-73.89396616744754, 40.611275625798044], [-73.89395838926941, 40.61125127229274], [-73.89396648650073, 40.611232669460605], [-73.89396385060894, 40.61121422258077], [-73.89394013317667, 40.61118142665126], [-73.89391997408066, 40.61117410746641], [-73.89389343874454, 40.61116853910193], [-73.893880751707, 40.611174703181234], [-73.89386562665591, 40.611196154046304], [-73.89384782444569, 40.61122470985778], [-73.8938323248957, 40.611232545619906], [-73.89379152722755, 40.611227183018094], [-73.8937168229822, 40.61120822975432], [-73.89364561695476, 40.611186480602996], [-73.89359381403101, 40.611170402472155], [-73.89358085539476, 40.61116572495799], [-73.8935747216994, 40.61115004643157], [-73.89358175848243, 40.61111895435713], [-73.89357185579416, 40.61109517582816], [-73.89357265103773, 40.61107124084786], [-73.89358009404614, 40.61105642577179], [-73.89359497528555, 40.61105240477843], [-73.8936083475598, 40.6110451170771], [-73.89360608756934, 40.6110402561395], [-73.89359851326351, 40.61102408173476], [-73.89357760194986, 40.61101514178717], [-73.89354938853043, 40.61100089763647], [-73.89351887151976, 40.61100734872457], [-73.89347916625445, 40.611017190957504], [-73.89345807815877, 40.61102839820613], [-73.89344216311285, 40.61104822945014], [-73.89338856819977, 40.61107186618585], [-73.89336405167376, 40.61108855982998], [-73.89331326937933, 40.61111055400427], [-73.89325185819875, 40.61113541861309], [-73.89319644725846, 40.61114497392651], [-73.8931596216847, 40.61115586511136], [-73.89313335787313, 40.61116116560753], [-73.89311590293912, 40.61117501006328], [-73.8930885375207, 40.611193375284905], [-73.89304180314119, 40.611206066529085], [-73.8929933904562, 40.611210056909705], [-73.89293616670399, 40.611205502116555], [-73.89290770999314, 40.61120866016716], [-73.89289532812593, 40.61122569320594], [-73.89288082130972, 40.61124332820533], [-73.89286007197273, 40.611268122937865], [-73.8928193004153, 40.6112910308815], [-73.89277578448072, 40.61131832785792], [-73.89274228034354, 40.6113465966149], [-73.89269269747018, 40.61136093254012], [-73.89266036251936, 40.611378825781514], [-73.89262435670531, 40.61139408018489], [-73.89257532615905, 40.61140188444926], [-73.89248131228022, 40.6114082199462], [-73.89243289959188, 40.611412236895426], [-73.8923714192018, 40.61143438481997], [-73.89226399024325, 40.61147353473117], [-73.89217291854987, 40.61150918713816], [-73.89210656779056, 40.611536297253885], [-73.89204347623411, 40.611552487161966], [-73.89202221714038, 40.61155825817056], [-73.8919986952916, 40.61155916987878], [-73.89197535223015, 40.61153993436919], [-73.89194175364253, 40.6115371854665], [-73.89191302416836, 40.61152947356082], [-73.89189389086438, 40.61153467124988], [-73.89187232044357, 40.611555126028655], [-73.89186052081706, 40.611568345368944], [-73.89186089427974, 40.61158193249546], [-73.89184576980698, 40.611603382592506], [-73.89181625045815, 40.6115940499134], [-73.89179465309756, 40.6115862074842], [-73.89176776994219, 40.61159532121979], [-73.89175552504005, 40.61161781755489], [-73.89175102759246, 40.6116363681692], [-73.89175568450037, 40.61165152338132], [-73.89176751025295, 40.61166654897497], [-73.89176733298619, 40.61168666821168], [-73.89173382823894, 40.61171493550106], [-73.89168982834119, 40.61175148105728], [-73.89162453754801, 40.611791082358025], [-73.8915780373995, 40.61181464435661], [-73.89153473422412, 40.61182182232983], [-73.89149026555634, 40.611813819880346], [-73.89145981786032, 40.611822987306134], [-73.8914422940607, 40.611834114623306], [-73.89142291623855, 40.611856712912115], [-73.89137304754392, 40.611914033108384], [-73.89134842585895, 40.611927979750995], [-73.89132284639948, 40.61193218199241], [-73.89127724585015, 40.61193452850363], [-73.89125615692345, 40.61194570569444], [-73.89122728330689, 40.611960829033386], [-73.89119093957225, 40.61196249756812], [-73.89115352994774, 40.61197720169141], [-73.891114883189, 40.61199953457441], [-73.89108638304268, 40.612028246179136], [-73.89101629376063, 40.6120499734344], [-73.89099709323197, 40.61205245241628], [-73.89098820988829, 40.61206671696542], [-73.89099866192876, 40.612083964780616], [-73.8910319492454, 40.61210139775337], [-73.89107089230458, 40.61211820650221], [-73.8911380178029, 40.61214654095964], [-73.8911505614258, 40.612163212598944], [-73.89115950685309, 40.612177193160996], [-73.89116138821068, 40.61219402005372], [-73.891152639307, 40.612213746860185], [-73.8911383350983, 40.61223950749253], [-73.89112317309483, 40.61226095628509], [-73.89109920436923, 40.61227111683316], [-73.89107794632615, 40.612276861868516], [-73.89102257650492, 40.61226086095502], [-73.89097777129396, 40.61223924491228], [-73.89088095934763, 40.61219342532005], [-73.89082997815156, 40.61218171152772], [-73.890751328315, 40.61217474733707], [-73.89069540915486, 40.61216527904197], [-73.8906436077181, 40.61214920125983], [-73.89058827569782, 40.61213320017273], [-73.89054936434638, 40.61211910978468], [-73.8904906752201, 40.61208576016451], [-73.89044497991411, 40.61205708774631], [-73.89036856635639, 40.612026712902626], [-73.89033390698104, 40.61201147423185], [-73.89032551194437, 40.61199096150182], [-73.89030196651542, 40.61196357450043], [-73.89025541408637, 40.61193059250822], [-73.89018932416231, 40.61188647426966], [-73.89013698090788, 40.611848657001254], [-73.89006245439326, 40.61180955364933], [-73.88999149500485, 40.611770399383474], [-73.88992780018391, 40.61173657844982], [-73.88987373662218, 40.61171566560969], [-73.88981830085875, 40.61169692042769], [-73.88979163130142, 40.611685915674585], [-73.88976671387411, 40.61166075002261], [-73.8897357979812, 40.611625339904236], [-73.88971513183803, 40.611598996904085], [-73.88970208064967, 40.61156330202818], [-73.88969643663843, 40.61151282043638], [-73.88969631846447, 40.611453559592235], [-73.88970771942019, 40.61139848244817], [-73.88972080425104, 40.61132649904574], [-73.88973659161245, 40.611275680507404], [-73.88974857431269, 40.61121678927907], [-73.88975684996826, 40.61117804120628], [-73.88975069109794, 40.61113409057785], [-73.88974339996096, 40.61110048780423], [-73.88974700785134, 40.61107488176401], [-73.88975400969885, 40.61104378938594], [-73.88976640213248, 40.610998457178844], [-73.88978451776185, 40.6109552440053], [-73.8898030753878, 40.61092830904874], [-73.8898508863007, 40.61087428184732], [-73.88988195927155, 40.610835747764455], [-73.88993676636512, 40.610776182686614], [-73.88998862619619, 40.610712855576274], [-73.89003276807986, 40.61065618921911], [-73.8900687474416, 40.61061266422905], [-73.89010750322278, 40.610567495071145], [-73.89012431460466, 40.61052916574382], [-73.89015491427742, 40.610471609210286], [-73.8902171644626, 40.61039725778826], [-73.89030211389475, 40.610317656862506], [-73.89042145603655, 40.61021687144161], [-73.89051720497692, 40.610139806989906], [-73.89058822830458, 40.61007402697117], [-73.89066106589074, 40.610022356259655], [-73.89071377146458, 40.60999163782227], [-73.89078567758789, 40.60995843874219], [-73.89086026035157, 40.60991816154118], [-73.89093412025686, 40.609878954116766], [-73.89099306795256, 40.609841076421816], [-73.8910330863643, 40.60981654840235], [-73.89107790656506, 40.609784339019306], [-73.89116264069276, 40.609724858835946], [-73.8912635280208, 40.60965369715776], [-73.89136506644253, 40.609581470018504], [-73.89147510697336, 40.60950691495885], [-73.89156049550925, 40.609443617613245], [-73.89169657038079, 40.60932733847845], [-73.89181714695249, 40.60921892281937], [-73.89190421803634, 40.60913874690266], [-73.89194461446813, 40.60910225214836], [-73.89197293667654, 40.609093660068], [-73.89198356658119, 40.60909078805483], [-73.89199014557242, 40.60909718915862], [-73.89199251089269, 40.60910476805299], [-73.89198153250831, 40.60912235080599], [-73.89196496903469, 40.60914325108069], [-73.89195059673268, 40.60916632073706], [-73.89194730240311, 40.60917724254853], [-73.8919469214283, 40.60918920946491], [-73.89191293485874, 40.60922670000972], [-73.89189568455569, 40.60924872341373], [-73.89187706053777, 40.60927294300222], [-73.89186495088494, 40.60930084564539], [-73.89186552721634, 40.60932258442983], [-73.89188030062537, 40.60934140011946], [-73.89191115290005, 40.60934853873729], [-73.89194567584633, 40.60935834216501], [-73.89197251628632, 40.60937478239907], [-73.89198053798371, 40.60938173368455], [-73.89198204467463, 40.60938497394069], [-73.89197710758087, 40.60938722005443], [-73.89195434226023, 40.609389751297826], [-73.89191491605392, 40.609382193512914], [-73.8919042874189, 40.609385065520975], [-73.89184147817522, 40.609383853666735], [-73.8918129158035, 40.60938429335456], [-73.89179320143182, 40.609393304615686], [-73.89176826872492, 40.609421964301866], [-73.89173846460528, 40.60945558797274], [-73.89174617296977, 40.609477224245765], [-73.89175255064889, 40.60947547303946], [-73.89176235782821, 40.609468263445976], [-73.89177213049341, 40.60946105381919], [-73.89178632544525, 40.609458103426824], [-73.89179300947643, 40.60946724927001], [-73.89179602504251, 40.609473729956825], [-73.89178570212476, 40.60948747270277], [-73.8917687630466, 40.60949481178404], [-73.89176138601042, 40.60951231765992], [-73.89176940682302, 40.60951926895924], [-73.89178661669833, 40.609522797410925], [-73.89182975218749, 40.60950746822367], [-73.89185947818015, 40.609499398508056], [-73.8919186964821, 40.609472391053586], [-73.89195381174605, 40.60945010835341], [-73.89198865066955, 40.60941967442162], [-73.89200226922794, 40.60939498404644], [-73.8920126272566, 40.60938124231913], [-73.89203673967039, 40.609348244480756], [-73.89207449626187, 40.6093188562996], [-73.89208619107376, 40.609302892039366], [-73.89209579987548, 40.609261977217095], [-73.89210797938067, 40.6092367622846], [-73.89214806385068, 40.609214952433085], [-73.89218982624992, 40.60920181662087], [-73.89225205821351, 40.609181288998805], [-73.89226735458966, 40.6091652731466], [-73.89225854392797, 40.60915670163759], [-73.89223996351477, 40.609155366770366], [-73.89223736489394, 40.609136918708735], [-73.89223415091612, 40.60909673199515], [-73.89226636666562, 40.60901955154592], [-73.89228712956259, 40.608940959978945], [-73.89231046617176, 40.608852489616744], [-73.89232388063873, 40.60881967384889], [-73.89233475269599, 40.608799373641006], [-73.89235474828892, 40.60877298626251], [-73.89236651131546, 40.60875974053201], [-73.89237210877512, 40.608728125155125], [-73.89236659371454, 40.608708632501155], [-73.89235117112669, 40.60869094155797], [-73.89232233910616, 40.60868048485801], [-73.89226903237174, 40.608661193349526], [-73.89222992108611, 40.60863895120194], [-73.89222484752898, 40.60863576255378], [-73.89222320409746, 40.60862708685753], [-73.89224086147313, 40.60862139536124], [-73.89227092888656, 40.60862419464573], [-73.89230884579733, 40.60862851205484], [-73.89233887804373, 40.60863131246133], [-73.89237419500388, 40.60861718296012], [-73.89240577649015, 40.608597642111185], [-73.89242165582378, 40.60857783895004], [-73.89242327207693, 40.608558242738695], [-73.89241874979803, 40.60854852181697], [-73.89240003217414, 40.608541725230076], [-73.89237230295815, 40.608518230783446], [-73.892350466836, 40.60850226407465], [-73.89232081489433, 40.60848749507675], [-73.89230655271741, 40.608487701847714], [-73.89228759092025, 40.60849833390584], [-73.8922660530439, 40.60852150779184], [-73.89225154669337, 40.6085391152419], [-73.89222497128918, 40.60855910100926], [-73.89220227177583, 40.6085643491832], [-73.89217649206044, 40.60856040046503], [-73.89217280422021, 40.608550970309274], [-73.89217035937482, 40.60854472135724], [-73.89218566106376, 40.608503150962925], [-73.89220095663808, 40.60848713679302], [-73.89220613849824, 40.60846748830895], [-73.89221975673654, 40.608442826050535], [-73.89224455411595, 40.608408730822276], [-73.89224765128853, 40.608364103297085], [-73.89226476757497, 40.60833664506823], [-73.89227063393139, 40.60831589873411], [-73.89226491754677, 40.608288254420984], [-73.89227551823753, 40.60825708232818], [-73.89228652890434, 40.60824224336817], [-73.89232674661802, 40.60822586654712], [-73.89234067063052, 40.608212046375264], [-73.89236254719486, 40.60820246007422], [-73.89242809957986, 40.608199283106224], [-73.89245861387248, 40.60819283297451], [-73.89246694577112, 40.608185101426876], [-73.89248324041495, 40.60815330506725], [-73.89249634968839, 40.6081096190317], [-73.8925039094105, 40.608043695516365], [-73.89251225672591, 40.60798213871313], [-73.89251265000011, 40.60791631872831], [-73.89251606825799, 40.60782873601951], [-73.89253161366011, 40.60776976371556], [-73.89253453424524, 40.60774525542628], [-73.89255332887906, 40.60772647170547], [-73.89257263734851, 40.60770118330553], [-73.89257644693231, 40.60768372882252], [-73.89257923318526, 40.60765375878902], [-73.89258925681183, 40.60760087535537], [-73.8925925566787, 40.60756440064485], [-73.89259877163929, 40.60752894321462], [-73.89262744796882, 40.60748011247971], [-73.89264562768527, 40.60743958942035], [-73.89265629812327, 40.60741091621993], [-73.89266288822178, 40.607388881068196], [-73.89266865111792, 40.607365226309525], [-73.89267753876939, 40.60734807962479], [-73.89269805366088, 40.60731211452456], [-73.89271349483414, 40.607272960625906], [-73.89272944999256, 40.607227192316465], [-73.89274578072478, 40.60719514834253], [-73.89277298224872, 40.60716563967521], [-73.89282052082659, 40.60714347802739], [-73.89285972311285, 40.60713187733299], [-73.89291755142094, 40.60711612049411], [-73.89295881188569, 40.607101172943395], [-73.89301735849152, 40.60708706537739], [-73.89305100460894, 40.6070815516433], [-73.89307868475188, 40.60706848482656], [-73.89311202433865, 40.607052047051546], [-73.89313133653025, 40.60702376715936], [-73.89314509341307, 40.60700162773867], [-73.89316193846592, 40.606962996099575], [-73.89319508034636, 40.606889549790836], [-73.89321442938613, 40.6068612689135], [-73.8932400878229, 40.60682851986018], [-73.89325682785129, 40.606810144728605], [-73.89330488145153, 40.60678139694991], [-73.89333077816713, 40.606756854512696], [-73.89334621420008, 40.606743446234645], [-73.89335633266543, 40.60674438993363], [-73.89337437125589, 40.606746822104746], [-73.89339858556959, 40.60673929637263], [-73.89343192495704, 40.606722831029515], [-73.89346029068597, 40.60670866677011], [-73.89350216842729, 40.60668987580754], [-73.89353228048533, 40.60668718621479], [-73.89355865577508, 40.606679086652896], [-73.89356637422146, 40.60667236901307], [-73.89356829553775, 40.6066635869038], [-73.89361034669517, 40.60665028666278], [-73.8936431345325, 40.60664040883159], [-73.89366875603983, 40.606630661065246], [-73.89370261066202, 40.6066076350085], [-73.8937209246845, 40.60659527288074], [-73.89375096918045, 40.60658983853831], [-73.89376811931783, 40.60658518987687], [-73.89379102987849, 40.606582601783764], [-73.89380526239005, 40.60657687890428], [-73.89382398895224, 40.6065782131519], [-73.89384899207721, 40.60657233603772], [-73.8938776645142, 40.60656909452417], [-73.89390098811438, 40.60655445943423], [-73.89392804932636, 40.606545262434686], [-73.89395425232833, 40.60653167185509], [-73.89399630332582, 40.60651837147253], [-73.89403704781694, 40.606510009969746], [-73.89406640667247, 40.606505672228515], [-73.89411984237347, 40.60648834752128], [-73.89417838753694, 40.60647423936249], [-73.89422747013796, 40.606455316892294], [-73.8942667062879, 40.60644371457632], [-73.89433115077942, 40.606434414492256], [-73.89436554952162, 40.60643057602173], [-73.89438918052521, 40.60642689165829], [-73.89443956109264, 40.60642877707575], [-73.89447793734078, 40.60643855681627], [-73.89453092576935, 40.606430563215945], [-73.89454769743276, 40.6064379346426], [-73.89457965967956, 40.60644946467214], [-73.89461789972682, 40.606453781617866], [-73.89466683917881, 40.60645514392866], [-73.89471331217747, 40.60644615618609], [-73.89476314750108, 40.60642888216109], [-73.89479799307118, 40.60641571159137], [-73.8948279702857, 40.60640753093549], [-73.89486168368407, 40.60640476320584], [-73.89489159174688, 40.60639386514719], [-73.89492901063724, 40.60637076123968], [-73.89496005370184, 40.606349435678624], [-73.89499201826885, 40.60633519190579], [-73.89502195866986, 40.60632703981315], [-73.89506033940964, 40.60631104466821], [-73.89509196395277, 40.60628584884134], [-73.89512849348962, 40.606255663102246], [-73.89516502304126, 40.60622544702877], [-73.89519469177483, 40.60620631595524], [-73.89524390832825, 40.60619291191852], [-73.89527724698775, 40.60617644603921], [-73.89531041351154, 40.60615451787293], [-73.89534756041574, 40.60612048934003], [-73.89541355308576, 40.60606567975876], [-73.89545306479322, 40.606039255313895], [-73.89549686545347, 40.60601168248963], [-73.89558134136269, 40.60597300441302], [-73.89562963265263, 40.60595246229328], [-73.89569445337851, 40.605931110576776], [-73.89575210848106, 40.605909890703586], [-73.8958174463283, 40.60588192617328], [-73.89587853199558, 40.60585516480121], [-73.89591344424802, 40.605844711276376], [-73.89596629793114, 40.60583122729712], [-73.89600848153306, 40.60582341589371], [-73.89604504219373, 40.60581897443151], [-73.8960933645966, 40.60582417886363], [-73.89611282804842, 40.60582577475023], [-73.89613229172254, 40.605827369461075], [-73.89616247045461, 40.605827424204975], [-73.89619251418554, 40.60582198972617], [-73.89622585106429, 40.60580555104469], [-73.89625089867457, 40.60579250789101], [-73.89627865673783, 40.60577749200914], [-73.89630914096797, 40.60576637564334], [-73.89634239220534, 40.605759048426656], [-73.8963741968421, 40.60575655244215], [-73.89641374038979, 40.605755710905825], [-73.89646612349583, 40.605743652448226], [-73.89651361336718, 40.60572835289767], [-73.89655332325336, 40.60571405970929], [-73.89661922910476, 40.60568971838731], [-73.8966696507696, 40.60566473051763], [-73.89669873597367, 40.60565037394948], [-73.89672573151199, 40.605636537517185], [-73.89676038286802, 40.605632451451555], [-73.89682399044987, 40.60562745823586], [-73.89688900043505, 40.60562301557119], [-73.8969462734801, 40.605616917445865], [-73.8970027013133, 40.60561996043331], [-73.89702463446004, 40.60561901997688], [-73.89705299760507, 40.605605732010865], [-73.89709908395112, 40.60558988146551], [-73.89714512964417, 40.605576720211026], [-73.8971819549379, 40.60556401725866], [-73.89722931501039, 40.60556216569965], [-73.89726958519435, 40.605557562483746], [-73.89730704372322, 40.605554577251134], [-73.89733808216624, 40.60555853002059], [-73.89736261836337, 40.605574828062345], [-73.89737384283042, 40.6055818927383], [-73.89740632780372, 40.60558101540747], [-73.89746649122829, 40.60556792207759], [-73.89753375190506, 40.60555219947498], [-73.89760739642848, 40.605532200642834], [-73.8976789082389, 40.60551541072862], [-73.89774123587092, 40.60549910714499], [-73.89777027644381, 40.605490157604734], [-73.89779935721683, 40.60547849110483], [-73.89782852308653, 40.60545875445862], [-73.89787099896003, 40.60544825161065], [-73.89790846177605, 40.60544254886297], [-73.8979473239477, 40.60544011416416], [-73.89799106940531, 40.605446299643795], [-73.89802202818252, 40.60545563252431], [-73.89805944668541, 40.60545533635665], [-73.89810829528953, 40.60544593756187], [-73.89814371654832, 40.60543271103574], [-73.89819758590764, 40.605413134062594], [-73.89823457150493, 40.60538967185719], [-73.89824445577001, 40.60537963354985], [-73.898258390021, 40.60535746804029], [-73.89826235401648, 40.60535402614452], [-73.8982860129763, 40.60533347550902], [-73.89831730436643, 40.60531217656739], [-73.8983387811778, 40.605303659788], [-73.8985021104329, 40.605296690182925], [-73.89855419496172, 40.60529446731979], [-73.89864718918473, 40.605297980361165], [-73.89867496480568, 40.60532029209345], [-73.89871019993686, 40.60535284240422], [-73.89891955307318, 40.60544424567243], [-73.8989845177309, 40.60546141388184], [-73.89901959721236, 40.60545837925951], [-73.89902693503325, 40.605463794050095], [-73.89903664073196, 40.60547321503482], [-73.89905646541636, 40.60548607154766], [-73.89907731163696, 40.6054917523401], [-73.89912474245436, 40.60551589388616], [-73.89916181724695, 40.60552808535133], [-73.89919567313135, 40.60552545250933], [-73.89922946325662, 40.60552007496962], [-73.89927388742446, 40.605506307498224], [-73.89930511062286, 40.60548226359843], [-73.89933650290519, 40.60546648053332], [-73.89938188499723, 40.60546259405863], [-73.89944780783917, 40.60547442804639], [-73.89951668033191, 40.605490051236664], [-73.89953705115354, 40.60549748003192], [-73.89954599733558, 40.60551167790592], [-73.89955484411396, 40.60552035979494], [-73.89959332756771, 40.605530329575124], [-73.8996612397111, 40.60553607063497], [-73.89971831657213, 40.60553919460991], [-73.89975502217835, 40.60553486232848], [-73.89978363261964, 40.605523523273895], [-73.89981289975292, 40.6055082877214], [-73.89983224542372, 40.605503117043106], [-73.89983742177343, 40.60550907822095], [-73.89984767580246, 40.60551556483904], [-73.89986221809437, 40.60552087444708], [-73.89986880363854, 40.605524640544004], [-73.89986625885673, 40.605540064076465], [-73.89987585891427, 40.605550392905975], [-73.8998911542277, 40.60555735115053], [-73.8999067538063, 40.60557806045104], [-73.8999143951397, 40.60559719725947], [-73.89990948636967, 40.60560496131165], [-73.89990091168008, 40.60560726005029], [-73.89989449398105, 40.60561175596911], [-73.8998968912271, 40.60562216077978], [-73.8999065606021, 40.60563523446464], [-73.89993286393346, 40.60565029830364], [-73.89996859770162, 40.60566745692013], [-73.89998399429273, 40.605679904835725], [-73.89999705604232, 40.60569457399698], [-73.90000916147456, 40.6057081701382], [-73.90003750449233, 40.60575126017139], [-73.90007152836574, 40.60579916171551], [-73.900112441877, 40.60583775294134], [-73.9001507242467, 40.60587065291168], [-73.90019598545987, 40.605954983341476], [-73.90025988076917, 40.60603271386394], [-73.90033080688517, 40.60614533621357], [-73.90037522509617, 40.60619151476267], [-73.9004762407651, 40.606249685469535], [-73.90051858955579, 40.606277762992335], [-73.90059787827084, 40.60640999415862], [-73.90064279017145, 40.60647842793533], [-73.90070783005758, 40.60662674374107], [-73.9007575634707, 40.60670561015739], [-73.90083447110563, 40.60684327927139], [-73.90087037418694, 40.60691946077631], [-73.9008767251003, 40.60698012134726], [-73.90086969928235, 40.60702091884978], [-73.90084067879594, 40.607067087125195], [-73.90080499748406, 40.60711397755872], [-73.90071969724089, 40.60720030571525], [-73.9006068015383, 40.60737825993728], [-73.90056595304027, 40.60747450979], [-73.9005613486067, 40.60754930538066], [-73.90056214377256, 40.607603995614454], [-73.90056573940005, 40.60763447914039], [-73.90053369436052, 40.60778913221905], [-73.90052614806866, 40.60788177414695], [-73.90052116645809, 40.60801477170054], [-73.90050222623539, 40.608120598522966], [-73.90049322225323, 40.60816619282188], [-73.90049794019995, 40.6082097013678], [-73.9005111987812, 40.608280766797975], [-73.9005104144828, 40.608339612393024], [-73.90048274556077, 40.60842805989011], [-73.90045970284832, 40.608499275154514], [-73.90044035823007, 40.60856789909399], [-73.90044931995531, 40.60861453322088], [-73.90047105749785, 40.60865400632223], [-73.90048641763464, 40.60870691911335], [-73.9004757485306, 40.60877161396943], [-73.9004517588019, 40.608837753786], [-73.90044787076532, 40.608888370508105], [-73.90046543832625, 40.608927897627424], [-73.9004850051341, 40.60898234561103], [-73.90048848205193, 40.60906403212218], [-73.90047946417721, 40.60916559820864], [-73.90047685714813, 40.60921746982693], [-73.90046890043159, 40.60923506203564], [-73.90044173391954, 40.60925194973193], [-73.90042781786602, 40.609264214041495], [-73.90041127533874, 40.609308630937235], [-73.90039892292792, 40.60937271167001], [-73.90036972500583, 40.60941093292075], [-73.90033566913091, 40.60943712990715], [-73.90029844130726, 40.609451920092006], [-73.90028079199116, 40.60946518631982], [-73.9002510678944, 40.60951740547905], [-73.90023451276159, 40.60961779581529], [-73.90022454304199, 40.60988379201693], [-73.9002164372098, 40.609988843136406], [-73.9001978588538, 40.610016750920806], [-73.90016673111657, 40.61004322816152], [-73.9001561993653, 40.61007643818693], [-73.9001638970745, 40.61010369029465], [-73.9001830218845, 40.61011934507378], [-73.90020146175749, 40.61012292442926], [-73.90023385289848, 40.610115829212276], [-73.90025430873604, 40.61011620162476], [-73.90027897234978, 40.61011811091024], [-73.90034074595498, 40.610156433489266], [-73.90041033828338, 40.610208648792124], [-73.90043779117453, 40.610242324307904], [-73.90045104607532, 40.610275546249454], [-73.90045278253653, 40.61029746708983], [-73.90043054227256, 40.6103295547728], [-73.9004090206505, 40.61037530892323], [-73.90041773606399, 40.61041081605207], [-73.90044811559746, 40.610444771360704], [-73.90053422078529, 40.61048882412864], [-73.90066942279951, 40.6105452829159], [-73.9007083743688, 40.610570541757326], [-73.90076147221444, 40.61061279103337], [-73.90084903639666, 40.610703892346436], [-73.90090494093566, 40.6107597811774], [-73.9009186838028, 40.61077741356415], [-73.9009318497063, 40.61078773816927], [-73.90097600589962, 40.61080307245985], [-73.9010298537826, 40.610822733140516], [-73.90107410161538, 40.610860962329035], [-73.90109862677392, 40.61089435740115], [-73.90109605210759, 40.610909974757824], [-73.9010863069046, 40.61092218342888], [-73.90108427487063, 40.6109435170087], [-73.90108769605926, 40.61096605261141], [-73.90111116924587, 40.610989602767624], [-73.90115641441548, 40.611016370443465], [-73.90120565297312, 40.61103513615773], [-73.90122318207342, 40.61103522862133], [-73.90124817814846, 40.611033317818915], [-73.90129606828728, 40.61104764834428], [-73.90134997400646, 40.61108861510195], [-73.90139131154872, 40.61114628207606], [-73.90142494070084, 40.61123266980432], [-73.90144034047158, 40.61132501631332], [-73.90143162425723, 40.61138131401363], [-73.90143121409612, 40.611383967824764], [-73.90142267880147, 40.61139425399251], [-73.90140134363516, 40.61139689705971], [-73.90139687678773, 40.61139744875438], [-73.90137555983165, 40.6113770508916], [-73.90135576888217, 40.61136903724439], [-73.90133145872336, 40.61138302582838], [-73.90130949868475, 40.61142783171896], [-73.90131215941024, 40.61156677221145], [-73.90131357332729, 40.611668203552924], [-73.90133303458649, 40.611717882905985], [-73.90139558106458, 40.611753332952986], [-73.9014638578319, 40.61176485702418], [-73.90152882294919, 40.61179645975701], [-73.90155357611931, 40.61182126537447], [-73.90156274501301, 40.61183959202547], [-73.9015869204988, 40.61185709103187], [-73.9016146158425, 40.61186404809772], [-73.9016957601147, 40.611837919499436], [-73.90170411628955, 40.61183522909303], [-73.90170562488021, 40.611834700656146], [-73.90188714552353, 40.61177117644259], [-73.90195417433628, 40.61174518940402], [-73.90198714602955, 40.61174540197324], [-73.90202288769994, 40.61175766156236], [-73.90217513917953, 40.61186764623178], [-73.90221791366727, 40.61189857146497], [-73.90228989425327, 40.61196157467679], [-73.9023059306886, 40.61195208254238], [-73.9023162562616, 40.611945970979455], [-73.9023323476477, 40.61196142077917], [-73.90244757158227, 40.61188564757185], [-73.90248711043368, 40.6118618584917], [-73.90245961268204, 40.61183936733821], [-73.90249392985292, 40.611814999032276], [-73.90121944263846, 40.61057524923629], [-73.90120761505968, 40.61058154218464], [-73.90119253465456, 40.61056381754662], [-73.90120767284094, 40.61055467777574], [-73.90123565151285, 40.61053778456722], [-73.90125039672986, 40.61052888100419], [-73.90127679496273, 40.610512942181124], [-73.90129324731704, 40.61050300860235], [-73.9013139570879, 40.61049050408986], [-73.90132962659396, 40.61048104310328], [-73.90134497358643, 40.610498821059025], [-73.90130184418443, 40.61052505778401], [-73.90131057398872, 40.6105333206096], [-73.90125388461304, 40.61056475632969], [-73.90124836865569, 40.610580274499064], [-73.90130898852283, 40.61063895269083], [-73.90253555121055, 40.61182621119939], [-73.90255730736897, 40.611827233143984], [-73.9026400798301, 40.61177163991647], [-73.90265077946786, 40.61178062116415], [-73.90250227199049, 40.61187818918433], [-73.90249848536895, 40.61187265516861], [-73.90246715531369, 40.611897561265714], [-73.9024724301828, 40.61190700784576], [-73.90236032096202, 40.61198041218261], [-73.90240693535868, 40.612016016677245], [-73.90250700965525, 40.61194329096753], [-73.90251499233746, 40.61194329773796], [-73.90256308414796, 40.61192431374954], [-73.90257168768603, 40.61192096694346], [-73.90271854260655, 40.6118260252959], [-73.90272806388138, 40.611836881864335], [-73.90263899474436, 40.61189478976389], [-73.90263827747394, 40.61190652255531], [-73.90266710890624, 40.61193180733682], [-73.90268153110615, 40.611944456096296], [-73.90271107896187, 40.611970368869414], [-73.90272726591957, 40.6119845649073], [-73.90275819543582, 40.612011691077484], [-73.90277356656735, 40.612025170567065], [-73.90280506848971, 40.61205279778554], [-73.90281973905196, 40.61206566519541], [-73.90284371811899, 40.61208669411005], [-73.90292666983426, 40.61215944197638], [-73.90302781199232, 40.61208425621744], [-73.90304392812484, 40.61209521498717], [-73.90291750370345, 40.61218654344852], [-73.90290113117334, 40.61217503059358], [-73.90290953065015, 40.61216840906741], [-73.90280546663435, 40.612078745857225], [-73.9027066445074, 40.611994227282295], [-73.90262918245355, 40.61192580917084], [-73.90259744740868, 40.611948363690104], [-73.90258114049809, 40.61193640119693], [-73.90252788313913, 40.611958379247454], [-73.90242274717579, 40.61203356000482], [-73.90255975123588, 40.612163974449174], [-73.90271124133432, 40.61224706649885], [-73.90417612379218, 40.613550753418046], [-73.90418042590366, 40.61355508896952], [-73.90418982390513, 40.61356456018194], [-73.90587105277362, 40.615063313977224], [-73.90590695505037, 40.615038572703504], [-73.90587336970381, 40.61500950305286], [-73.9057345590125, 40.61488297394373], [-73.90575026311356, 40.61487529925099], [-73.9057790314495, 40.61490217237043], [-73.90578708389997, 40.614909694823254], [-73.90580868591384, 40.61492987413683], [-73.90581675725312, 40.61493741352346], [-73.90584343332286, 40.614962332864536], [-73.90588571114608, 40.615001826090804], [-73.90599790513686, 40.61492589692781], [-73.90600865413123, 40.61493485327302], [-73.90603328924851, 40.614953389219465], [-73.90598587691342, 40.61498998083147], [-73.90595950755578, 40.61497014075064], [-73.90592831880679, 40.61499075664415], [-73.90596638222885, 40.61502410025575], [-73.90592374609002, 40.61505396225649], [-73.90591926792638, 40.615050541335215], [-73.90587937620926, 40.61507746141139], [-73.90595891316937, 40.61515017846609], [-73.90596246364093, 40.615153424061575], [-73.9069651511667, 40.61606387150079], [-73.90696283645082, 40.61611401757884], [-73.9071474543953, 40.616000150327935], [-73.90717291149826, 40.61600987533847], [-73.90698679725256, 40.61613782775689], [-73.90673848666401, 40.616295919990094], [-73.90681023172979, 40.61635959428093], [-73.9070631177642, 40.61620388316397], [-73.90725685565037, 40.61608758137682], [-73.90727466091477, 40.61610506465549], [-73.90708346968198, 40.61622136802846], [-73.90722592437267, 40.616351525932345], [-73.90741202194343, 40.616235218724015], [-73.90744255412062, 40.616258535700595], [-73.90725494717977, 40.6163780969955], [-73.90702160678437, 40.61653360035669], [-73.9071059539902, 40.616609860214204], [-73.90735919243342, 40.616443881554964], [-73.90752735132158, 40.61634443254885], [-73.90754332867888, 40.616362736714564], [-73.90737516888878, 40.61646218574317], [-73.90724169382695, 40.616551501887514], [-73.9072949558038, 40.61660844991536], [-73.9073429922425, 40.61658613341431], [-73.90747503705722, 40.61671274456844], [-73.90758977173546, 40.61662937645129], [-73.90767882862006, 40.616695440008044], [-73.90755135602237, 40.616780739630535], [-73.90765229512199, 40.61691575661923], [-73.90768940958893, 40.61694708904508], [-73.90771656608526, 40.61693202218735], [-73.90778588935453, 40.616893557888496], [-73.9079680653862, 40.61704658677693], [-73.90796856793588, 40.617053617698886], [-73.90797062107767, 40.617082327102054], [-73.90776459701048, 40.6172171745568], [-73.9078114318726, 40.617264864560234], [-73.90775403862018, 40.617316440382986], [-73.9078138860334, 40.61737406758582], [-73.90815289956946, 40.61716189299594], [-73.90839229831543, 40.61738842851889], [-73.90852252572826, 40.61732090816455], [-73.90860351593429, 40.61726549705418], [-73.90879088017641, 40.617436396721416], [-73.90886910990886, 40.61739079305857], [-73.90881396020816, 40.61733103799171], [-73.90863333912958, 40.617167854909376], [-73.90866392881254, 40.617150410808485], [-73.9088369111078, 40.61731164662762], [-73.90907145153933, 40.61716043833179], [-73.90835403214685, 40.61653099896152], [-73.9083744262503, 40.61651742845357], [-73.9087942081916, 40.61687295732468], [-73.90913839448129, 40.616632552115426], [-73.90913853186629, 40.61653162172904], [-73.90859831133378, 40.616085198376894], [-73.90847891987522, 40.61598019021542], [-73.90810560476338, 40.61566901642846], [-73.90799087259747, 40.61575238674425], [-73.90797306390525, 40.615736844061175], [-73.90815663851605, 40.615601124051494], [-73.90817445255608, 40.615612783891855], [-73.90812345822495, 40.61565156228688], [-73.90850653891894, 40.61596935984318], [-73.90862861698722, 40.61606482263631], [-73.90917675105388, 40.61652194698016], [-73.90916894153112, 40.616646161849005], [-73.908814562086, 40.61689044188239], [-73.90895706128724, 40.616991484819316], [-73.90911449272927, 40.61686646159229], [-73.9094302789907, 40.61667050578894], [-73.90945370835946, 40.61668640833571], [-73.90964406040352, 40.61657537149951], [-73.90963625834844, 40.616565437433664], [-73.90959740910328, 40.616392669799325], [-73.90902960565275, 40.615796291369776], [-73.90902020844351, 40.615786150105656], [-73.90900782730678, 40.6157727869073], [-73.90865024676272, 40.615484555687225], [-73.9085747448177, 40.61544019631195], [-73.90840539767107, 40.615436709578084], [-73.90834295344649, 40.61537312326995], [-73.90785596197065, 40.61520595414655], [-73.9074058282047, 40.61475340341135], [-73.90730153384293, 40.61480891300767], [-73.9068723978097, 40.61426255834329], [-73.90674467587257, 40.61430216486308], [-73.90544378702947, 40.61301450900802], [-73.90535513021162, 40.61306605793205], [-73.90521241748837, 40.61298965918143], [-73.90505053373573, 40.61290299700339], [-73.90509214724543, 40.61286577023568], [-73.90509488171134, 40.61286332355426], [-73.90508447750591, 40.612857870940516], [-73.90496210660214, 40.61279372164133], [-73.90490995852686, 40.61282147677114], [-73.90424103859327, 40.61236227321875], [-73.90437410766295, 40.612225386296956], [-73.90431684436872, 40.612187614791175], [-73.90428823036942, 40.612155823778274], [-73.90427007250842, 40.61210021448232], [-73.90428600143811, 40.612035358829765], [-73.90429103725161, 40.61201485612886], [-73.90420814113486, 40.611955152099085], [-73.90410629882949, 40.61204241028689], [-73.90404332529339, 40.61199868643175], [-73.90406770985663, 40.61197796216748], [-73.90411637142162, 40.61201075637194], [-73.90416083877471, 40.61197258159144], [-73.90409929902891, 40.611927767007664], [-73.90414089145172, 40.611896139383646], [-73.90384925159573, 40.61168886793209], [-73.90350980782237, 40.611406490372545], [-73.90339509282545, 40.611479543900096], [-73.90340510537716, 40.611490470015156], [-73.90331047012505, 40.611547163863655], [-73.90328328895731, 40.61151984615855], [-73.90337362137012, 40.61146642441069], [-73.90337648208998, 40.61146970301354], [-73.90349407447779, 40.61138900874268], [-73.90345549862127, 40.611347473371715], [-73.90342187266508, 40.61131126705753], [-73.9033385604731, 40.61127148761568], [-73.90313578707003, 40.610967538157105], [-73.90286496065242, 40.61088193328191], [-73.90274504537493, 40.61093146873142], [-73.90262906864334, 40.61081018839824], [-73.90268277928772, 40.61075272266691], [-73.90239138175355, 40.61048244972481], [-73.9023236133439, 40.61050423258507], [-73.90225859966276, 40.61042277259991], [-73.9022144276591, 40.610343316032065], [-73.90222497639522, 40.61025596362203], [-73.90227449074713, 40.610246077790606], [-73.90220516294403, 40.609553084558726], [-73.90207490818688, 40.60954900355571], [-73.90214599092712, 40.60904276556271], [-73.90210336163577, 40.609037916486464], [-73.90196164219753, 40.60861970003265], [-73.90196196584279, 40.60839931125429], [-73.90199063921322, 40.608378928150536], [-73.90201440809034, 40.60835068139818], [-73.90200434649165, 40.60831419285539], [-73.90200130936587, 40.608307085021536], [-73.9019968182827, 40.60829973286323], [-73.90197885878756, 40.608290541141805], [-73.90197307401526, 40.608287573456096], [-73.90197086408783, 40.608279854343095], [-73.90194746000888, 40.60805972911199], [-73.90191623151603, 40.608059674992155], [-73.90187803412165, 40.60780778415157], [-73.90191354153009, 40.60780748124328], [-73.90191177585922, 40.60778705575983], [-73.9018877939877, 40.607510421771124], [-73.90187641024093, 40.607345418990526], [-73.90190752124978, 40.607345166108004], [-73.90189745276427, 40.60723317753213], [-73.90186893018571, 40.60723309002504], [-73.90171996150777, 40.60634753342995], [-73.90170495677201, 40.60625836846017], [-73.90169872618107, 40.60622136984965], [-73.90167739606466, 40.60609454913335], [-73.90168632809845, 40.606095133744354], [-73.90171150437247, 40.60609676640491], [-73.90170081037174, 40.60587222129953], [-73.90165858100026, 40.6058569219289], [-73.90164808286106, 40.60579724375498], [-73.90164211790079, 40.6057633607389], [-73.90177669809985, 40.605674686555815], [-73.90181267270403, 40.6056678646822], [-73.90185037412324, 40.60571038411893], [-73.90187376201806, 40.605703551476786], [-73.90187582791519, 40.60570585827417], [-73.90189710069492, 40.60572961179576], [-73.90291909122419, 40.60547586392895], [-73.90381289962677, 40.60525393333968], [-73.90470757613124, 40.605031778563855], [-73.90520530979795, 40.60490818290941], [-73.90524300992108, 40.60491009032442], [-73.90528793023383, 40.60493479702361], [-73.90539181422342, 40.60522680882991], [-73.90533534473163, 40.6053242701596], [-73.90542440003202, 40.605540297589286], [-73.90560690377077, 40.60589785864346], [-73.90570001056875, 40.60600553334022], [-73.90575205936467, 40.60611650427578], [-73.90588481979447, 40.606297611034876], [-73.90597461177855, 40.60638402870432], [-73.90623422405241, 40.60635398000769], [-73.90695508400546, 40.606358922935556], [-73.90710057938108, 40.606371463395725], [-73.90724679877958, 40.60637734405589], [-73.90739321785888, 40.60637654382807], [-73.9075393115617, 40.60636906558195], [-73.90768455599682, 40.60635493613421], [-73.90782843032022, 40.60633420615307], [-73.90797041860135, 40.606306949975966], [-73.90811001167562, 40.60627326534302], [-73.90818770904363, 40.60635433731642], [-73.90821683308761, 40.60637297909434], [-73.90828607999654, 40.606402870303384], [-73.90837376514945, 40.606396627742306], [-73.90841450766717, 40.60652757551175], [-73.90844815360893, 40.60661674673967], [-73.90848093177033, 40.606635399569484], [-73.90856971576302, 40.606653284607326], [-73.90861472394567, 40.60666083776884], [-73.90863959535422, 40.606658722047854], [-73.90870529107144, 40.60664693764751], [-73.90874599099338, 40.606608616355295], [-73.90882919865997, 40.606564966000455], [-73.90881822284258, 40.60653819384666], [-73.90881136337046, 40.60652146330246], [-73.90878731055211, 40.60646279780028], [-73.90880856612377, 40.60645735304997], [-73.90877243515098, 40.606384339645544], [-73.90874058936066, 40.60631382849622], [-73.90870825751658, 40.606243242903496], [-73.90868276737514, 40.60618759326795], [-73.90865739211544, 40.606132193503576], [-73.90861508773968, 40.60604515043975], [-73.90859415171684, 40.606001271315144], [-73.90857234429065, 40.60600738587101], [-73.90856864929573, 40.605999987358274], [-73.90855248523931, 40.605967620435536], [-73.90853062658859, 40.60592385046389], [-73.9085146951497, 40.60589194928403], [-73.90851058586274, 40.605883721505506], [-73.90853378311802, 40.605876801749574], [-73.9084923668496, 40.60577213812816], [-73.90846664297462, 40.60577949344964], [-73.90843408138487, 40.605728732655336], [-73.90845649277887, 40.605722290919644], [-73.90844212792871, 40.60569268997905], [-73.90842412709902, 40.60569779365287], [-73.90841705422126, 40.60568758616309], [-73.90839936710282, 40.60566205855359], [-73.9083936515068, 40.60565380921938], [-73.90841899874358, 40.60564634053193], [-73.90840420569957, 40.605613883543526], [-73.90839062290351, 40.60561845715274], [-73.90838757402335, 40.605612781683384], [-73.9083830686712, 40.60560439493423], [-73.90836406109749, 40.60556900868457], [-73.90835983309901, 40.60556113982059], [-73.90838206875905, 40.605553618736536], [-73.90836540085215, 40.60552223243477], [-73.90836357914405, 40.60551081202883], [-73.90835197687807, 40.605489421369015], [-73.90833426588041, 40.605484383716146], [-73.90830727294615, 40.60549220187808], [-73.90828019855472, 40.60544106881038], [-73.90826584843533, 40.60541396571814], [-73.90826141716497, 40.6054055954409], [-73.90827578546285, 40.605401225184075], [-73.90825771968117, 40.605366625714375], [-73.90825600624365, 40.60536712777165], [-73.90825589550575, 40.605357017467234], [-73.908241104175, 40.60532277861826], [-73.90804594062307, 40.605380648200445], [-73.90803306097814, 40.60535860211721], [-73.90840361574526, 40.605243297059864], [-73.90841800626492, 40.605267722381726], [-73.90829586897208, 40.60530550148997], [-73.90831206762074, 40.6053376011024], [-73.9083302893999, 40.60533160184376], [-73.9083539845863, 40.60538151410142], [-73.90837359285274, 40.60542281578357], [-73.90837834785442, 40.60543283312589], [-73.90838993959262, 40.605457249686026], [-73.90839498062358, 40.605467869020394], [-73.90840933442644, 40.60549810171105], [-73.90841392944104, 40.60550778185575], [-73.90838831344149, 40.60551588275037], [-73.90840357563164, 40.605547623760366], [-73.90842905394325, 40.60553959412006], [-73.9084329685158, 40.60554905459373], [-73.90844522073026, 40.60557866641763], [-73.9084491664133, 40.605588204983995], [-73.90846767150063, 40.60563292872581], [-73.908490614308, 40.60562652359302], [-73.90851327699045, 40.605679690252465], [-73.90849314617617, 40.60568434142088], [-73.90859589280906, 40.60590752909505], [-73.9086011754084, 40.60591900217281], [-73.90862096434266, 40.605961988291895], [-73.90867297919127, 40.6060513250056], [-73.9086732940981, 40.6060518670451], [-73.90867801158534, 40.606059967978744], [-73.90866141097756, 40.606064510574505], [-73.90868421931644, 40.60610910367544], [-73.90870624207278, 40.606103228668616], [-73.90871078443426, 40.60611128875227], [-73.90872998669624, 40.606145361501085], [-73.90873438126897, 40.606153159618394], [-73.9087169598315, 40.60615837135179], [-73.90873265005878, 40.60619398492398], [-73.9087358277961, 40.60620095263826], [-73.90875297445298, 40.606238540354525], [-73.90877458860328, 40.60628592168103], [-73.90878902084852, 40.60631755760773], [-73.90879343156898, 40.60632722552804], [-73.90876710091456, 40.60633468212509], [-73.90877708012741, 40.60635914661732], [-73.90881635624245, 40.6064261458749], [-73.90882146047558, 40.60643485205685], [-73.90882788621846, 40.6064458127211], [-73.9088430576895, 40.60646413200209], [-73.90886670981311, 40.60647128783417], [-73.90892100027693, 40.606458594970306], [-73.90892527516411, 40.60646650338679], [-73.90887097975077, 40.606482818228415], [-73.90887338346825, 40.6064900126619], [-73.90885497326904, 40.606494719034394], [-73.90888574422165, 40.60653651882835], [-73.90906279646923, 40.60646614621082], [-73.90908737385216, 40.606466165653984], [-73.90913322764858, 40.60648368458371], [-73.90918728471067, 40.60649371840981], [-73.9092344962553, 40.606483646661], [-73.90927254334011, 40.60645257592507], [-73.90933804983145, 40.606477603024665], [-73.90984528326653, 40.60616830550708], [-73.90980883736027, 40.6060808600487], [-73.90985478785177, 40.60602719798733], [-73.9098515484547, 40.605999722177685], [-73.90977661382514, 40.605822677231416], [-73.90960943418337, 40.605657796050735], [-73.90959070936646, 40.60566519716739], [-73.90915233587795, 40.605293847167914], [-73.90898535789192, 40.605150509642996], [-73.908945318632, 40.60504942652807], [-73.90885691315668, 40.605060073797134], [-73.90872476424393, 40.604739969994476], [-73.90874285904927, 40.604729267021185], [-73.9086135794197, 40.60451654785956], [-73.90857840990381, 40.6045254997872], [-73.90854436867018, 40.604445855981616], [-73.90845796643069, 40.60446109885738], [-73.9083096617489, 40.604128950077566], [-73.9083253683517, 40.60407337391859], [-73.90889944956521, 40.60391367252357], [-73.90893375673267, 40.603976528676505], [-73.90912526593854, 40.603927107041976], [-73.90913452972944, 40.60394410684935], [-73.91092270021187, 40.60350547610065], [-73.91247254015315, 40.60375190241971], [-73.91306509093998, 40.60384611337231], [-73.91329003999543, 40.60387843583746], [-73.91352500916965, 40.60392913916153], [-73.91365119759595, 40.60398251383695], [-73.9136253383353, 40.6040164870757], [-73.91377776447158, 40.60420645695329], [-73.91384142172905, 40.60415768468768], [-73.91497779074928, 40.60520409454706], [-73.91491345354056, 40.60524538625554], [-73.91500401101551, 40.60532198810563], [-73.91504389099256, 40.60535572210597], [-73.91509616292997, 40.60532360703458], [-73.91567006907964, 40.60583883549568], [-73.91591887691341, 40.60606219940042], [-73.9159512486574, 40.6060997417252], [-73.91626594416164, 40.606440633658885], [-73.9163482853367, 40.606460597248386], [-73.91673556486197, 40.60681456155016], [-73.91669736890354, 40.60683596955618], [-73.91675554034506, 40.606906442611034], [-73.91669121427903, 40.60693854852309], [-73.91671583604739, 40.60698858703718], [-73.91660442147958, 40.60708939344468], [-73.91670644426729, 40.60716337556576], [-73.91661465264755, 40.60720525980096], [-73.91666955358512, 40.60731949967672], [-73.91672115759047, 40.60762484678011], [-73.91664171593676, 40.60783414643551], [-73.91632230017511, 40.60867566266899], [-73.91668775150282, 40.60875341754165], [-73.9167839443274, 40.60877388439219], [-73.91657073983414, 40.609423539545915], [-73.91662568373555, 40.60943683665814], [-73.91683981118769, 40.6087600147205], [-73.91648700927323, 40.608699208722385], [-73.91650924535926, 40.60862975409001], [-73.91688964587311, 40.60871001368691], [-73.9168553679984, 40.608760932307376], [-73.9168739501378, 40.60876294889292], [-73.91682803243748, 40.60890605736671], [-73.91699867098345, 40.608936730311996], [-73.91705165775807, 40.60881413108985], [-73.91708838503304, 40.608820854898525], [-73.91702822741907, 40.608974877413424], [-73.91681964529576, 40.60893138688516], [-73.91665676533457, 40.60943674997084], [-73.91681036958981, 40.60946691562431], [-73.91679953090683, 40.609494192231004], [-73.91670379289528, 40.60947271234202], [-73.91668075282955, 40.609551335120976], [-73.91664099809648, 40.60954543188988], [-73.9166637973001, 40.6094572855876], [-73.91656604065834, 40.60943782267781], [-73.91638585145205, 40.61008889290487], [-73.91627678158125, 40.61010958945703], [-73.9162350895359, 40.61018764764725], [-73.91618801442762, 40.61026390742237], [-73.91613568922112, 40.6103381533554], [-73.91607693139578, 40.610411746853806], [-73.91606077132579, 40.610430509362935], [-73.9161647503772, 40.61051557507611], [-73.9156289810894, 40.61088347754488], [-73.91554577672987, 40.61092847019066], [-73.91492035932123, 40.611266657338064], [-73.91499950855344, 40.61133572880869], [-73.91489001105649, 40.61139315949259], [-73.91481827328002, 40.611342077848214], [-73.91464933054556, 40.61145502607295], [-73.91455938763573, 40.611570851641694], [-73.91449542205973, 40.61165322366488], [-73.91458057649909, 40.61189538467202], [-73.91474395639995, 40.61270828753798], [-73.91486259053775, 40.613298542099834], [-73.91482568233826, 40.61332521117222], [-73.91476336459147, 40.613370238318836], [-73.91479289781104, 40.613532687277505], [-73.91485570360723, 40.613500333207455], [-73.91486977791232, 40.613493082590374], [-73.91493418790425, 40.61395815505491], [-73.91487851298005, 40.613993630319015], [-73.91415762351875, 40.61445296931084], [-73.91386053060674, 40.61464226907088], [-73.91407985868283, 40.61488823339901], [-73.91418158175665, 40.61480520748114], [-73.91416424991556, 40.61479197449439], [-73.91418409377795, 40.614778767589314], [-73.91422619595673, 40.61480335289653], [-73.9141120615841, 40.61490147801327], [-73.9141657924689, 40.61491992819076], [-73.91422238665953, 40.614932456704416], [-73.91428072663338, 40.61493881616294], [-73.91433966039311, 40.6149388809906], [-73.91439690440212, 40.61491893691919], [-73.914450878778, 40.6148942602153], [-73.91450091243084, 40.61486515769846], [-73.91454638326994, 40.61483199121626], [-73.91457541233427, 40.61480636505005], [-73.9146017069913, 40.61477907860197], [-73.91460363331858, 40.61477281863788], [-73.91461912939545, 40.61472243073407], [-73.91466875774388, 40.61467525055095], [-73.91475621277912, 40.614676830570964], [-73.91477780531795, 40.61467722100888], [-73.91508764077801, 40.61446832874461], [-73.91526818192825, 40.61435094742979], [-73.91580224199757, 40.6140037143864], [-73.91596393246677, 40.61389020969622], [-73.91642099003494, 40.61356935469271], [-73.91640428251647, 40.61355234521991], [-73.91614556120368, 40.61373023354987], [-73.91592907529404, 40.6138590549774], [-73.9159145211068, 40.61384537631227], [-73.9159560237015, 40.613819782939224], [-73.91572538303375, 40.613618030803636], [-73.91576807356562, 40.61354033333579], [-73.91575004260955, 40.61361719582282], [-73.91597173175394, 40.613807835366686], [-73.91624768959588, 40.61362610079118], [-73.91588490852118, 40.61332858719697], [-73.91576488361797, 40.61340281130991], [-73.91574585230263, 40.61338486070303], [-73.9159657250026, 40.613236396468864], [-73.91598251878865, 40.61325178355086], [-73.91588941550151, 40.613309799605595], [-73.91626452766656, 40.613605612704106], [-73.91638679683788, 40.61352968074032], [-73.91633753617162, 40.61348352081669], [-73.91639586745879, 40.61344683403471], [-73.91599307131547, 40.612881939344255], [-73.91588093410674, 40.61292456583794], [-73.91587086536354, 40.612910037390556], [-73.9160031884947, 40.61285973859123], [-73.91641827930147, 40.61345026708692], [-73.91645866206082, 40.61342467104202], [-73.91650568236642, 40.61346912199881], [-73.91646978847766, 40.61349045050587], [-73.9164991210398, 40.613520542098016], [-73.91662804371141, 40.6134435892654], [-73.91573181927129, 40.61183084451381], [-73.91674652389301, 40.6111784554681], [-73.91669154817431, 40.611127656686364], [-73.91667108215461, 40.61114322406103], [-73.9165893648604, 40.61109075400423], [-73.91662098248936, 40.611075194734006], [-73.91665069880038, 40.61109363057433], [-73.91671582048875, 40.611039850701495], [-73.9166731127402, 40.61100582325054], [-73.91669171257122, 40.61099592085718], [-73.91678827905612, 40.61106540009289], [-73.91678083391083, 40.61107389428802], [-73.9168292081243, 40.611118077722935], [-73.91805286362053, 40.61027534308027], [-73.91811036679019, 40.61020839405809], [-73.91810261499006, 40.61012428359595], [-73.91774340916777, 40.609806569439826], [-73.91773657839127, 40.60979484804834], [-73.91773185485035, 40.60978253819983], [-73.91772932254797, 40.60976985881372], [-73.91772902651729, 40.60975703538167], [-73.9178663862696, 40.60932723380082], [-73.91781719473106, 40.60931749853926], [-73.91770422310164, 40.609667630156096], [-73.91767870757266, 40.6096628649489], [-73.91796270083073, 40.60878593712031], [-73.91798455297189, 40.608791773345025], [-73.9178197277143, 40.60930430006589], [-73.91787400308746, 40.60931132766885], [-73.9180158470274, 40.60886652988326], [-73.91802177099734, 40.608856219431885], [-73.91802964454176, 40.608846696743456], [-73.91803929189071, 40.60883817440283], [-73.91805049767602, 40.60883084266266], [-73.91806301173895, 40.60882486519667], [-73.9185566612059, 40.60849830904342], [-73.91809925398383, 40.608093358258444], [-73.91800500501185, 40.60814977851462], [-73.91818728590098, 40.60802458091069], [-73.91812303859335, 40.60808069425832], [-73.91857357570947, 40.60848794628732], [-73.91887978314314, 40.60830074753582], [-73.91888852404472, 40.60829445902877], [-73.91889855118033, 40.6082894069123], [-73.91890956709287, 40.60828574105838], [-73.91892124499323, 40.60828357021529], [-73.9189314197851, 40.608282950052484], [-73.91894160444555, 40.60828346763999], [-73.91900737147333, 40.60826725497113], [-73.91824893434108, 40.60758887292728], [-73.91814759534256, 40.60765393376545], [-73.91813814889971, 40.607645386907706], [-73.91836246737634, 40.60750136941865], [-73.91841039790253, 40.60754473166755], [-73.91830794532584, 40.6076105077084], [-73.91902310765325, 40.60825686819942], [-73.91935398484114, 40.60804292228766], [-73.91837826940144, 40.60717402193569], [-73.91827215643065, 40.60724314341611], [-73.91825269092949, 40.6072293506405], [-73.91850300413901, 40.60707123602699], [-73.91851674502753, 40.60708466836772], [-73.91840159359653, 40.607154852898034], [-73.91937100429925, 40.60803264124188], [-73.9194228518384, 40.607998657974846], [-73.91947494229278, 40.60798083621683], [-73.91950358918922, 40.60797490397521], [-73.91950459282141, 40.60792207773474], [-73.91952209795996, 40.607738797434784], [-73.91924135011546, 40.607385413860605], [-73.91905150176778, 40.60721662408656], [-73.9188771952235, 40.607113323711154], [-73.91868471829673, 40.60696635699282], [-73.91858079216594, 40.60679365825723], [-73.91858342114791, 40.606771834596465], [-73.91794980550603, 40.60620925217101], [-73.91847519099996, 40.60614556799992], [-73.91852760899995, 40.60611336699985], [-73.91970928699986, 40.607302162999886], [-73.91984929899996, 40.60722139799987], [-73.92065861699989, 40.60675454299997], [-73.92158429099997, 40.607639084999875], [-73.92230145599989, 40.607178630999925], [-73.9230107439999, 40.60671540299993], [-73.92373054999989, 40.60625280699984], [-73.92444144399987, 40.60579356899993], [-73.92624885099994, 40.60741352199989], [-73.92696222699988, 40.60696137499984], [-73.92769136199998, 40.60648050699993], [-73.92844217799984, 40.60600119699996], [-73.92915810699985, 40.60552985699993], [-73.9295009169999, 40.60530869199985], [-73.92986880199989, 40.60507134099995], [-73.93169716599986, 40.6067140589999], [-73.93201257899985, 40.60699707199991], [-73.93351310700004, 40.60834345599989], [-73.93423157599979, 40.607897714999915], [-73.93531145199985, 40.60886003999987], [-73.93572378299984, 40.60922748599993], [-73.936046277, 40.609514868999966], [-73.93676116199995, 40.60905528699991], [-73.93709375999993, 40.60884198299992], [-73.93753749399987, 40.60855738999986], [-73.93934703399997, 40.610187140999884], [-73.94011528799989, 40.60969107799986], [-73.94082897899995, 40.609229317999905], [-73.9415475769999, 40.608767735999905], [-73.94187229999983, 40.60855546699988], [-73.94231175899985, 40.60826819699985], [-73.94354698000001, 40.6093465549999], [-73.94364091199994, 40.609858816999896], [-73.94369953899981, 40.61021403899986], [-73.94380031999991, 40.610719118999945], [-73.94405560899996, 40.61199284199993], [-73.94417450600004, 40.612722537999936], [-73.94430888199999, 40.6134117379999], [-73.94457366500001, 40.61480377199996], [-73.94479547299986, 40.615888530999904], [-73.9448277099998, 40.61604620899986], [-73.94467599499988, 40.61616511299997], [-73.94395779499983, 40.616816308999894], [-73.943469619, 40.61738500099992], [-73.94325340699987, 40.617412063999936], [-73.94314510099991, 40.617425616999896], [-73.94221251499997, 40.617526054999864], [-73.94138899899984, 40.617617418999885], [-73.941282159, 40.617629269999924], [-73.94078914099987, 40.61769678999986], [-73.94045274699987, 40.617929049999894], [-73.94037010899984, 40.617992385999884], [-73.9397167159999, 40.618406095999866], [-73.93949341999998, 40.618550790999905], [-73.93900236399998, 40.61886727099992], [-73.93864732399993, 40.61909724599994], [-73.93828802399999, 40.619329018999906], [-73.93781351899986, 40.6196520959999], [-73.93757616399986, 40.619795528999866], [-73.93696272700001, 40.62019914099994], [-73.93686699599992, 40.620253280999904], [-73.93501858699992, 40.61860011099992], [-73.93320457599995, 40.6169704419999], [-73.93248727399993, 40.61743170199987], [-73.93173179199982, 40.61792692099988], [-73.93060244599985, 40.61692226299993], [-73.92989884499987, 40.616294271999934], [-73.92946094800007, 40.61589681999992], [-73.92842985399992, 40.61497277999991], [-73.92808935399992, 40.614667884999946], [-73.92776130099995, 40.61437342499987], [-73.92769203599991, 40.614312640999835], [-73.92739580799999, 40.614052690999884], [-73.92629604599998, 40.61311679899991], [-73.92629769899993, 40.61319215499985], [-73.9263159899999, 40.61343967599986], [-73.92640044599996, 40.61447853499988], [-73.9253954579999, 40.614540562999885], [-73.92446651399993, 40.61459941799995], [-73.92349786399998, 40.61465815799991], [-73.92252930300005, 40.61471903299993], [-73.92159370099995, 40.614778183999874], [-73.92062400299997, 40.6148386819999], [-73.9205269409999, 40.61394269499987], [-73.92010700000002, 40.614212128999874], [-73.91960797099988, 40.61453446499991], [-73.9193708869999, 40.61469022299991], [-73.91893348499988, 40.61496777999992], [-73.91862838199995, 40.615168606999916], [-73.91788753399999, 40.61564657999988], [-73.9173907319999, 40.61596630599988], [-73.91719587499983, 40.61609171599987], [-73.91690302699992, 40.61627422899993], [-73.91697651699994, 40.61693221299989], [-73.9170081, 40.6172019239999], [-73.91711554499987, 40.61792519099986], [-73.91716779499997, 40.618387961999886], [-73.91717137699983, 40.61871125199989], [-73.91726440899994, 40.619548037999934], [-73.91741279799999, 40.62095853099987], [-73.91746991599989, 40.62151242899988], [-73.91751556199989, 40.62194031599993], [-73.91770927100005, 40.6237176859999], [-73.91775240199999, 40.624164716999864], [-73.91799464599991, 40.62645772299993], [-73.91816260199982, 40.62800449499984], [-73.91824254499994, 40.62877007499986], [-73.91835491799992, 40.629795297999856], [-73.91860649599992, 40.632090414999894], [-73.91716071492723, 40.63173012599842], [-73.91722315056269, 40.63171150745382], [-73.91728187053637, 40.63168685088904], [-73.91733587397775, 40.63165657654042], [-73.917384240479, 40.63162120039297], [-73.91742614571034, 40.6315813253835], [-73.91746087546879, 40.63153763112484], [-73.91748783785057, 40.63149086232209], [-73.9175065733389, 40.631441816079864], [-73.91751676263425, 40.631391328316354], [-73.91751823209556, 40.631340259516655], [-73.91731998946489, 40.63100715271805], [-73.91535460301414, 40.62999169507067], [-73.9147993677344, 40.62974079537697], [-73.9091803329726, 40.62720140975426], [-73.90826357306823, 40.62678363831664], [-73.90464209220775, 40.62513319188153]]]}}, {\"id\": \"155\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 156, \"Shape_Leng\": 0.144476890476, \"Shape_Area\": 0.0010521217443, \"zone\": \"Mariners Harbor\", \"LocationID\": 156, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-74.17989568340026, 40.64526879647802], [-74.17988406361408, 40.64511024036652], [-74.17984376950355, 40.64511030338695], [-74.17984839409118, 40.64516485991633], [-74.17977452733285, 40.6451666798479], [-74.1797298775203, 40.64438070211185], [-74.17933117249432, 40.64440051177255], [-74.17931081177709, 40.64443438951364], [-74.17931782514128, 40.64444160879773], [-74.17932325144032, 40.64444958205001], [-74.17932695411096, 40.64445810860941], [-74.17932883996812, 40.64446697388958], [-74.17932886154996, 40.64447595477976], [-74.17932701831296, 40.64448482525929], [-74.17932335664443, 40.64449336208633], [-74.17931796869652, 40.64450135041595], [-74.1793109900664, 40.64450858920713], [-74.17930259638361, 40.644514896281905], [-74.17929299889084, 40.64452011291114], [-74.17928832154499, 40.64453014013177], [-74.17928575184081, 40.644540599924156], [-74.17928535251289, 40.64455123693612], [-74.17928713331075, 40.644561791489025], [-74.17929105076134, 40.64457200591713], [-74.17929700922963, 40.64458163085817], [-74.1793048632539, 40.64459043134046], [-74.1792961300211, 40.64460784416022], [-74.17929148664743, 40.64462614574843], [-74.17929107338853, 40.644644783309396], [-74.17929490273038, 40.644663193898964], [-74.17930285901164, 40.64468082142819], [-74.17931470191704, 40.64469713345993], [-74.17932222939821, 40.644733502255704], [-74.17928957026453, 40.64482410160242], [-74.17928032241431, 40.6448297387919], [-74.17927240605663, 40.644836449806654], [-74.17926603337476, 40.64484405477081], [-74.17926137517728, 40.64485234984769], [-74.17925855631869, 40.64486111270339], [-74.17925765235418, 40.64487010846589], [-74.17925868751354, 40.6448790960206], [-74.17926163405197, 40.64488783447277], [-74.17926641299384, 40.64489608960429], [-74.1792721959103, 40.64491212506001], [-74.17928120941691, 40.64492727107846], [-74.17929323036354, 40.64494115268136], [-74.17930796114265, 40.644953426194064], [-74.17931836177972, 40.644963712154436], [-74.17931963374144, 40.644965286783716], [-74.17932091270441, 40.644967545644946], [-74.17932168785909, 40.64496993371643], [-74.17932193820279, 40.64497238629502], [-74.17932165695264, 40.644974836929514], [-74.1793208517291, 40.64497721922148], [-74.17931954434904, 40.64497946862411], [-74.17931777023503, 40.64498152419132], [-74.17931557745557, 40.64498333022861], [-74.17931302542274, 40.64498483780261], [-74.17920271291945, 40.645046927855404], [-74.17919288123369, 40.64504342540781], [-74.17917151967094, 40.64503075364965], [-74.17915399385407, 40.64501472114984], [-74.17914433076402, 40.644988406555434], [-74.17912788280461, 40.64495022357106], [-74.1791061325883, 40.6448768145412], [-74.17907466583888, 40.64464650705652], [-74.17908144140392, 40.64450569700578], [-74.17906767743065, 40.644451590080905], [-74.17905647832927, 40.64440286989942], [-74.17905513344847, 40.64435993740098], [-74.17905352462097, 40.64431978260426], [-74.17904565802472, 40.64426220585288], [-74.1790494161959, 40.64417323689502], [-74.1790596462759, 40.64407011625795], [-74.17904410570114, 40.64398533088518], [-74.17903966591282, 40.64390863393696], [-74.1790261907259, 40.643857669392986], [-74.17901676603547, 40.643822662185926], [-74.17900895017787, 40.643793883704575], [-74.17900604586508, 40.64375655796839], [-74.17900196853455, 40.64371771675018], [-74.17900148715438, 40.64368420739952], [-74.17900814920617, 40.64364952980866], [-74.1790187639507, 40.64355625188807], [-74.17903563701373, 40.64348039474873], [-74.17903555928557, 40.64339478702068], [-74.17904538824455, 40.64337769584999], [-74.1790659065113, 40.64335293163664], [-74.17907959489555, 40.643338395464895], [-74.17910124230973, 40.64332028004216], [-74.17911394582485, 40.643300667436826], [-74.17913343601879, 40.64329292305074], [-74.17916203645535, 40.64329416360915], [-74.17917908126803, 40.643293652392295], [-74.17920770481655, 40.64328384455109], [-74.1792363054284, 40.64326812113158], [-74.17925404421624, 40.64325258048979], [-74.17926967251843, 40.64324228075333], [-74.17928587918527, 40.643238262943115], [-74.17930807826693, 40.64323747679809], [-74.1793295090677, 40.643233970677656], [-74.17933890723586, 40.64322913069166], [-74.17935247481581, 40.643218940035844], [-74.17937498303432, 40.64317631728316], [-74.17938596172658, 40.64315482381221], [-74.17939533407673, 40.64314406775224], [-74.17940482802032, 40.64312896647635], [-74.17940832646521, 40.6431106318674], [-74.17940578317518, 40.64309419674307], [-74.17939583435012, 40.64308170641112], [-74.17938080286824, 40.643070276422115], [-74.17937080384323, 40.64306291673371], [-74.17937253045648, 40.643047833104006], [-74.1793807514003, 40.64304147814551], [-74.17939283485812, 40.643037679177965], [-74.17941014309534, 40.64303439139715], [-74.17942351984416, 40.64302776269126], [-74.17943548051005, 40.64301134497633], [-74.17943368167664, 40.6429917154257], [-74.17942116737598, 40.64297383785648], [-74.17940773684909, 40.64295727145489], [-74.17941686680491, 40.642903247956625], [-74.17941092276742, 40.64288021030177], [-74.17940728473074, 40.6428614284672], [-74.17940530835232, 40.64282942066175], [-74.17939451000967, 40.642816127516774], [-74.17938838983962, 40.64281207532977], [-74.17938528361432, 40.642799103901645], [-74.17939512553815, 40.642749680637856], [-74.1794015785891, 40.64274167442416], [-74.17940401630874, 40.64273169277698], [-74.17940844705623, 40.64268073317076], [-74.17941578547331, 40.642656260180104], [-74.17942844383572, 40.64261143168502], [-74.1794392413294, 40.642593341501545], [-74.17945445077008, 40.64256589515555], [-74.17946977164037, 40.64255011835088], [-74.17947260813814, 40.64254449266458], [-74.17948402624658, 40.642522720064], [-74.17949462433023, 40.64248676106769], [-74.17952382009699, 40.642397627894425], [-74.17952914022524, 40.642377275065385], [-74.17953248940147, 40.64237198832542], [-74.17954253318479, 40.642356126938104], [-74.17954188859916, 40.64233864698984], [-74.179546962925, 40.642305168498616], [-74.17955292697471, 40.642286606047705], [-74.17955611609806, 40.642243011595355], [-74.17957035016218, 40.642210142509974], [-74.17958207750978, 40.64218652995007], [-74.17959150128301, 40.64217435030055], [-74.1796007467559, 40.642149775476206], [-74.17960437994127, 40.642121483074625], [-74.17960229355927, 40.6420778035459], [-74.17961213567315, 40.64204407384993], [-74.17960967337291, 40.642032892135944], [-74.17960596648476, 40.64196631043893], [-74.17960181816804, 40.64193149924906], [-74.1795975821489, 40.64192187162108], [-74.1795871372141, 40.64191768346404], [-74.17956888859027, 40.64191719429718], [-74.1795575356884, 40.641924001629974], [-74.17955383456189, 40.64193587581603], [-74.1795490462878, 40.641962040059425], [-74.17954304267344, 40.64204811398006], [-74.17953398205535, 40.642147852464], [-74.17952274275386, 40.64218202175159], [-74.17950717898177, 40.64222233302617], [-74.17948560678192, 40.642263985362575], [-74.179483390033, 40.64228161971553], [-74.17947213042252, 40.64234169936051], [-74.17946208703786, 40.64237325114848], [-74.17940491070927, 40.64250729651524], [-74.1793935826967, 40.64253526652582], [-74.17937786209193, 40.64256237609079], [-74.17936728876705, 40.64260380987648], [-74.1793401762252, 40.6426895473017], [-74.17932835858704, 40.64270696107765], [-74.17932201695179, 40.6427266385057], [-74.17931618712414, 40.64274665438112], [-74.17931361676912, 40.64277087517095], [-74.17931410709717, 40.64281281422096], [-74.17930397810764, 40.64288524146344], [-74.17929353576812, 40.64291243442381], [-74.1792748897107, 40.642938971062776], [-74.17924404807638, 40.642968345044125], [-74.17922112974624, 40.64297824334816], [-74.17918728067563, 40.64298753999647], [-74.17915856161366, 40.64299064348189], [-74.17913986136752, 40.64299006113411], [-74.17912286465878, 40.642985441200274], [-74.17908174926843, 40.64296632352994], [-74.17906331341108, 40.642962962964084], [-74.17905058329731, 40.64295969520147], [-74.17903926810845, 40.64294925064916], [-74.17901881653239, 40.64292390473272], [-74.17898445948865, 40.642859846472284], [-74.1789462643082, 40.64276521883891], [-74.17893820887467, 40.64274513194574], [-74.17893187791923, 40.64272692673195], [-74.17892204821959, 40.64271008897003], [-74.17890900554816, 40.64269776490691], [-74.1788900868082, 40.6426880074771], [-74.17888889246925, 40.642687392326735], [-74.17883755862744, 40.64257373791307], [-74.17882094373519, 40.64254503123109], [-74.1788118087668, 40.64253013085433], [-74.17880190796937, 40.642512509828094], [-74.1787907843686, 40.64249850345951], [-74.17878114482909, 40.6424950715423], [-74.17877071758866, 40.64249996468183], [-74.17873543165993, 40.642527485532895], [-74.17867612428056, 40.64256417401502], [-74.17864062006429, 40.6425723740862], [-74.17859909822172, 40.642571425701284], [-74.17851356154608, 40.64254796376642], [-74.1784831382736, 40.64252117573299], [-74.1784702390646, 40.64249345634389], [-74.17842482969903, 40.642399211979544], [-74.17839512367179, 40.64232938166007], [-74.17836990148625, 40.6422802242935], [-74.17834923580007, 40.642252522967404], [-74.17833396352167, 40.64223281930481], [-74.17832032191038, 40.64220829599009], [-74.17830667986333, 40.64218377049652], [-74.17829126398283, 40.642162496971416], [-74.17827261248601, 40.64213981770079], [-74.17825662146018, 40.64212922786822], [-74.17824494768865, 40.64213182054494], [-74.178236006626, 40.64214728758991], [-74.17823909996137, 40.64218105189991], [-74.17828674141411, 40.642333567082176], [-74.17834761707203, 40.64247788162495], [-74.17836001223176, 40.642517070157126], [-74.17836830862727, 40.642545428951436], [-74.17836130963984, 40.642565132642886], [-74.17834127011962, 40.6425894759958], [-74.17831712917216, 40.642602989778524], [-74.17828874493674, 40.642604104799396], [-74.17824602581983, 40.6425925534066], [-74.17820217655361, 40.642580696333994], [-74.17817324087041, 40.64256448217049], [-74.17815221653208, 40.642549819122095], [-74.17813298867733, 40.64253782067964], [-74.17810992650661, 40.64252918461098], [-74.17806133296799, 40.64253019082056], [-74.17800763457008, 40.64254330299947], [-74.17798349357308, 40.64255681721493], [-74.177959521865, 40.642577817352304], [-74.17795599825318, 40.64259023534429], [-74.17794003358942, 40.642619489792104], [-74.1779307085354, 40.64262511408804], [-74.1778907931597, 40.64263039369054], [-74.17786677190573, 40.64262259720307], [-74.17784088061585, 40.64261134848987], [-74.17782359552945, 40.64260358714371], [-74.1778123760877, 40.642599844142076], [-74.17779804018457, 40.64260731346272], [-74.17778978404121, 40.64263565684429], [-74.17778543224081, 40.642650591890394], [-74.17777330613944, 40.6427273808469], [-74.1777693527224, 40.642802944427494], [-74.17775957363406, 40.64284883544459], [-74.17774310515362, 40.64288955731427], [-74.17772806566205, 40.642907282314525], [-74.17772066748056, 40.642916000216225], [-74.17768636299833, 40.64294859651953], [-74.17764021549506, 40.64297629966751], [-74.17753128407477, 40.64301404792928], [-74.17748837295586, 40.64302619231757], [-74.17745864523722, 40.64303526866406], [-74.17744138462227, 40.64303342601289], [-74.17742254808788, 40.643021702433245], [-74.17739591374877, 40.6430091163797], [-74.17738574360622, 40.64301209096009], [-74.17735372544297, 40.64303868795123], [-74.17732826853079, 40.6430640495109], [-74.17726935092438, 40.64309783322198], [-74.17725898931039, 40.64309935879524], [-74.1772411227328, 40.643109508083995], [-74.17721118570941, 40.643122778603065], [-74.17717352557739, 40.64311396330433], [-74.17714042132805, 40.64309601937171], [-74.17713480593916, 40.64308986715251], [-74.17713784267173, 40.643083780512065], [-74.17714344429942, 40.64307530531671], [-74.17714838240748, 40.64306907418532], [-74.17715454548046, 40.643050311693315], [-74.17715510495633, 40.643040027248084], [-74.17715528395615, 40.64302684648237], [-74.17715118704483, 40.64301765060813], [-74.17714537910915, 40.64301004822236], [-74.17713187740289, 40.6430096222208], [-74.1771187583656, 40.643012090692075], [-74.17709899306084, 40.643022387292966], [-74.17709330160174, 40.64303745504741], [-74.17708741934894, 40.643051072799175], [-74.17707592215211, 40.64305853816041], [-74.1770577629456, 40.643059200719556], [-74.17703597759045, 40.64304697216207], [-74.17698088246381, 40.64300072270873], [-74.17692854506882, 40.642953530825196], [-74.17688733532462, 40.6429106053441], [-74.17686764286218, 40.64289968009173], [-74.17684737674092, 40.64288440901792], [-74.17682386895896, 40.64285914598332], [-74.17681290478336, 40.64282706735281], [-74.17681193300832, 40.64280519562604], [-74.17681514794688, 40.642785929402294], [-74.17681380742438, 40.64277578994426], [-74.17680514054855, 40.64276109340946], [-74.17679314456522, 40.64274299295006], [-74.17678179644722, 40.64270801816725], [-74.17677282288471, 40.64266920248744], [-74.17676738531827, 40.64264986831356], [-74.17677183831798, 40.642632700495454], [-74.17677419851748, 40.642614231012814], [-74.1767718111685, 40.64260344164405], [-74.17676961503881, 40.64259410060224], [-74.17676112814671, 40.642566221118834], [-74.17674814546467, 40.642511620842946], [-74.1767361217451, 40.64246426035349], [-74.17672611574272, 40.642439423848366], [-74.17671593066747, 40.6424277697836], [-74.1767048915117, 40.64241691128393], [-74.17669557683358, 40.64241909073793], [-74.17670101435597, 40.64243842324039], [-74.17669931832964, 40.6424546486591], [-74.17669428881098, 40.64246747015741], [-74.1766862119555, 40.64247174801642], [-74.17668345519618, 40.64247269041297], [-74.1764825728471, 40.64239153953694], [-74.17639077729885, 40.64234371455394], [-74.17636880109187, 40.64233003804384], [-74.1763578661754, 40.64232722056289], [-74.17634950665703, 40.64233663978608], [-74.1763588355939, 40.64234909179549], [-74.17638356878426, 40.64236182591777], [-74.17659218717394, 40.64247969257936], [-74.17665211551696, 40.64251167173469], [-74.17667571342811, 40.64253034458886], [-74.17668723791225, 40.642552138621205], [-74.17671674393047, 40.64278982023799], [-74.17672442495068, 40.642971386120585], [-74.17671685309467, 40.64311022814015], [-74.17670588284568, 40.64317251994693], [-74.17670848832692, 40.64321401716326], [-74.17671109337599, 40.64325551605539], [-74.17670779048979, 40.64328137265503], [-74.17668390509117, 40.643362211136676], [-74.17667625101974, 40.64337403249797], [-74.17666795538246, 40.643386844771605], [-74.17665028095568, 40.64339844229373], [-74.17661645169186, 40.643418594349434], [-74.17658489192628, 40.64342686897757], [-74.17653877163028, 40.64341943332853], [-74.17651461386725, 40.64341104555984], [-74.17649654093123, 40.64340511597792], [-74.1764732522114, 40.643410560047975], [-74.17643380731708, 40.64342456030131], [-74.17637126443755, 40.64344545241627], [-74.17633666713805, 40.6434598102465], [-74.1762882750846, 40.643464255263815], [-74.17610392257231, 40.643471814162666], [-74.17598183191912, 40.64346362228083], [-74.17589757805769, 40.64345106540613], [-74.17585002747882, 40.6434400825729], [-74.17577964702862, 40.643416220500136], [-74.1757312282689, 40.64339140358598], [-74.17570735046618, 40.64337787259119], [-74.17568964875981, 40.643360211117106], [-74.17568450673052, 40.6433503645987], [-74.17567564816922, 40.64333421847934], [-74.17566973981538, 40.64331857781007], [-74.17566478766791, 40.64331017911892], [-74.17564120364021, 40.64330613531523], [-74.17562684579914, 40.64330650587082], [-74.17560250954419, 40.643311298672444], [-74.17558265458499, 40.643328183047956], [-74.17553221024086, 40.643375214669454], [-74.17552797662422, 40.64338269844047], [-74.17551664116927, 40.643402743967044], [-74.17548417914058, 40.64346229451269], [-74.17548579327112, 40.64347100404518], [-74.1754908667434, 40.643498359102445], [-74.17550408022687, 40.643598297623036], [-74.17550345643659, 40.64364443296586], [-74.17550405793433, 40.643678037636654], [-74.17550644511681, 40.6436888270304], [-74.17551985806496, 40.64369584645083], [-74.1755395485597, 40.64370677093072], [-74.17557190131993, 40.64373355126404], [-74.17558456226031, 40.643749405111045], [-74.17558649028975, 40.64377851932653], [-74.17556511464507, 40.64379844733605], [-74.17555532735325, 40.643804319080985], [-74.17553233126134, 40.64381925184159], [-74.17551922607672, 40.643836349888076], [-74.1755021389856, 40.64386692331762], [-74.1754840199676, 40.64391147255843], [-74.17547691201511, 40.64393762257317], [-74.17547770557813, 40.64397267624063], [-74.17547659531657, 40.64400787487742], [-74.17548032439193, 40.64402880424374], [-74.17549159458244, 40.644084999290904], [-74.17549255121834, 40.6440922412695], [-74.1754775450055, 40.644109486118765], [-74.1754716625528, 40.64412310546115], [-74.17547291355909, 40.64413983464345], [-74.17547711304717, 40.64415706995424], [-74.17549423978275, 40.6441703863076], [-74.17551972507742, 40.644174285007885], [-74.17554692124178, 40.64417658755786], [-74.17556911636981, 40.644220972066464], [-74.17562352359606, 40.64434920675859], [-74.17570768504721, 40.64457172467191], [-74.17570879961221, 40.6446455248858], [-74.17569977789418, 40.64465719029011], [-74.17569009214982, 40.64467109974981], [-74.17569001562464, 40.64469232089322], [-74.17568470742573, 40.64471028489506], [-74.17566940804059, 40.64471804256369], [-74.17565170598887, 40.64470038108639], [-74.17562477603211, 40.644678305245755], [-74.17560174068048, 40.64464934673441], [-74.17559031828651, 40.64463559348451], [-74.17557528468836, 40.64462357832872], [-74.17555892305174, 40.64461605513463], [-74.17552706974192, 40.644614840590755], [-74.17549931299419, 40.64462282196991], [-74.17546691157193, 40.64464652110229], [-74.17544240953579, 40.64467912706691], [-74.17540528962303, 40.64473976558174], [-74.1753750953006, 40.644787437021854], [-74.17535053232227, 40.64485589168496], [-74.17534628381469, 40.64488913702664], [-74.17534660403777, 40.644927883075006], [-74.17532749146807, 40.64503030222789], [-74.17529691545047, 40.645075076818856], [-74.17525879608037, 40.645084585210796], [-74.17520888416514, 40.64509207223039], [-74.17516638744445, 40.64509752552763], [-74.17511371862008, 40.64510595536824], [-74.17504185709666, 40.645129024407616], [-74.17495335456692, 40.645149711499684], [-74.17489620525124, 40.64514604796289], [-74.1748554056802, 40.64513527793489], [-74.17478835445861, 40.64511481705891], [-74.17475209647739, 40.64508028888075], [-74.17472003854463, 40.64506299652923], [-74.17468018172738, 40.64504483883493], [-74.17464451191779, 40.64502928553519], [-74.17460875182422, 40.645020323777494], [-74.17458621477577, 40.64501693253468], [-74.17455132269359, 40.64502180312211], [-74.1745492383894, 40.64502291968124], [-74.17452262296979, 40.64503717253804], [-74.17450182079058, 40.645061444499355], [-74.17448253856479, 40.645082675515674], [-74.17446515842171, 40.64510375990347], [-74.17445955562798, 40.645112236642674], [-74.1744475844544, 40.64512339646471], [-74.17443209299536, 40.64512970547127], [-74.17439463489723, 40.645136967354205], [-74.17436572943608, 40.64513625864542], [-74.17415373990465, 40.64508009422506], [-74.1741110252966, 40.64505484096361], [-74.17405908286237, 40.64502517320027], [-74.1740548408656, 40.64502187545243], [-74.1740333926389, 40.64500519709438], [-74.17401245134988, 40.644977542658836], [-74.17399636979123, 40.6449648766939], [-74.17395013353845, 40.6449347715418], [-74.17388619569034, 40.6448870034496], [-74.17378734092976, 40.64481484800369], [-74.17364112552038, 40.64473315555264], [-74.173598692665, 40.64470275904731], [-74.17354483786617, 40.644658606895206], [-74.17348147469656, 40.64461518558201], [-74.1734312431562, 40.64458392292855], [-74.17338319488704, 40.644547374447015], [-74.17332562596047, 40.644496923017186], [-74.17323931641718, 40.644417952054134], [-74.17315481958882, 40.644345425303094], [-74.17309117778854, 40.64430714397382], [-74.17304465948314, 40.64428218150936], [-74.17300042799177, 40.64425997026013], [-74.17297421830348, 40.64424146476255], [-74.17296865141027, 40.64423753436872], [-74.1729412389165, 40.64420452285587], [-74.17292333373825, 40.64417078169328], [-74.17292292555673, 40.644138626012946], [-74.17291679919698, 40.64409227735584], [-74.17291021359736, 40.64406425250436], [-74.17288641310363, 40.64402950028471], [-74.1728769823726, 40.64400900811271], [-74.1728888372495, 40.64397517998534], [-74.17291705460552, 40.64394887542739], [-74.17295430469129, 40.64392553644085], [-74.17298278854645, 40.643879459059164], [-74.17300185492398, 40.643827520941635], [-74.17301624948163, 40.643762040966145], [-74.17302818117629, 40.64370699218261], [-74.17301975911775, 40.6436432620273], [-74.17300154585647, 40.6435854050796], [-74.17297317769635, 40.64354515039891], [-74.17294804849875, 40.64351489062813], [-74.17292112211442, 40.6434928141482], [-74.17289614187574, 40.64347368106336], [-74.17289334095112, 40.64347153554611], [-74.1728717473641, 40.64346075570622], [-74.17284673524703, 40.64345316387827], [-74.17282297214408, 40.64346230075672], [-74.17278382065997, 40.64348578696891], [-74.17275397064707, 40.64349246595072], [-74.17271527709515, 40.6434976266597], [-74.1726771463489, 40.643492503934105], [-74.17263662669752, 40.643476591745426], [-74.17251090723764, 40.64344862653893], [-74.17250700754171, 40.64344775956428], [-74.17244784485074, 40.64342157075082], [-74.17243394765843, 40.6434036159276], [-74.17242498838898, 40.643379431347945], [-74.17241479273052, 40.643353147141276], [-74.1724089734546, 40.64333091425274], [-74.17240430879276, 40.64332015618673], [-74.17240344811884, 40.64331817025154], [-74.17238840157013, 40.643291526091694], [-74.17237260369193, 40.64327371734715], [-74.17236165386568, 40.64325626755184], [-74.17237019114741, 40.64323366736316], [-74.17239812675159, 40.64321250598125], [-74.17242340763579, 40.64320032509587], [-74.17245953399646, 40.643197555283656], [-74.17247912367155, 40.643200443081305], [-74.17250802827286, 40.64320115176178], [-74.17252685277766, 40.64319824588118], [-74.17254529348257, 40.643192442661814], [-74.1725422440035, 40.64318389774486], [-74.17252141705573, 40.64317891050155], [-74.1725008695514, 40.64316878221698], [-74.17249030372325, 40.64315422926671], [-74.17252033132723, 40.64313436959799], [-74.17254923656448, 40.64313507994243], [-74.17255588536146, 40.64312725441678], [-74.17255348547414, 40.643101834698925], [-74.17257085513344, 40.643066120329216], [-74.17261030040636, 40.64305212140045], [-74.1726368189117, 40.64304204176554], [-74.17268776272148, 40.64302057839417], [-74.17271969180938, 40.643000573585695], [-74.17273403548249, 40.642985574304745], [-74.17273173837609, 40.642968192798314], [-74.17272896940098, 40.6429545053384], [-74.17272145305189, 40.64294849899658], [-74.17269113227853, 40.642958871002655], [-74.17264446337221, 40.64297634848098], [-74.17259008697096, 40.64298637125607], [-74.17257192685548, 40.642987031432995], [-74.17256259721644, 40.642974579119304], [-74.17255990601879, 40.64293967218476], [-74.17255246456675, 40.64291244402013], [-74.17253903933235, 40.64289079516259], [-74.17250592523112, 40.64285822080247], [-74.17249012522633, 40.64284041274658], [-74.17247983025173, 40.64283226518402], [-74.17246652969651, 40.64282173903219], [-74.17244027862134, 40.64281204816389], [-74.17239397993379, 40.64281779211244], [-74.17236984813191, 40.64283866349687], [-74.17234875391364, 40.64285344836139], [-74.17232689255582, 40.64286244005617], [-74.17231215386455, 40.64285991285702], [-74.17230254351885, 40.64285260189635], [-74.17228105389907, 40.64284986132471], [-74.17222686800724, 40.64286133226365], [-74.17214841028964, 40.642841744342306], [-74.17209783689812, 40.6428514745503], [-74.17205258429262, 40.64285787015777], [-74.1720323441418, 40.642871858762526], [-74.17200715298469, 40.64287744750859], [-74.1719853686188, 40.64286521966928], [-74.17196330565986, 40.64285813269325], [-74.17195513744909, 40.64286899993016], [-74.17192969117691, 40.642908991228566], [-74.17188085297778, 40.642946387285384], [-74.1718725950498, 40.64296384439196], [-74.17185540509317, 40.6429863790718], [-74.17184049894703, 40.64301166216294], [-74.17179385518948, 40.64305839881658], [-74.17175033852062, 40.64309246075225], [-74.17165832740466, 40.643122924347885], [-74.17157200738855, 40.64313832290443], [-74.17151077414584, 40.64313460447501], [-74.17150811099722, 40.643134442090826], [-74.17147481628083, 40.64311504986822], [-74.17144598761874, 40.64309311861381], [-74.17142961518836, 40.64307096506935], [-74.17142369357941, 40.64304069392105], [-74.17143327838322, 40.64301874442955], [-74.17144886157108, 40.64300584644929], [-74.17146730099273, 40.64300004172946], [-74.17148298554169, 40.64299518145323], [-74.17149115230349, 40.64298431358229], [-74.1714726101952, 40.64298207959421], [-74.1714537863652, 40.6429849853013], [-74.17143219470348, 40.64297420351295], [-74.17140640381228, 40.64294618860576], [-74.17134786953622, 40.64287386440186], [-74.17133587281808, 40.642855765052], [-74.17134793296881, 40.64283801517441], [-74.17136827527347, 40.642832065402914], [-74.17139545714402, 40.6428197403485], [-74.1714405829384, 40.64277604458667], [-74.17150330552997, 40.64274197439646], [-74.17164594223647, 40.64265130043636], [-74.17176948707788, 40.64258330486633], [-74.17181957513452, 40.64256263808035], [-74.17184238051536, 40.64254625872996], [-74.17185834485531, 40.64253625652773], [-74.17186042582567, 40.64252292851884], [-74.1718325674352, 40.64252287080461], [-74.17178988074738, 40.64252687552601], [-74.17176078401346, 40.64252471715889], [-74.17175012787148, 40.642516756697276], [-74.17175449245057, 40.64250617945189], [-74.17179591691541, 40.642470814583454], [-74.17182336830281, 40.64243871661524], [-74.17184624853066, 40.64240111779713], [-74.17184927245123, 40.642380402708156], [-74.17184012461776, 40.642354769071545], [-74.17182251389802, 40.642330514946245], [-74.17180272014537, 40.64231155061577], [-74.17178294030919, 40.642307213696114], [-74.17175555524433, 40.642303460745524], [-74.17173434531898, 40.64229557870037], [-74.17172311588905, 40.64228327188951], [-74.17172813024924, 40.64225582034926], [-74.17175833773516, 40.64222277844376], [-74.17177029647218, 40.642196990308676], [-74.1717645795822, 40.642182797270586], [-74.17174621554558, 40.64216737922054], [-74.17171872782124, 40.64215558855386], [-74.1716960887233, 40.642144155854176], [-74.17168216664471, 40.642096943089214], [-74.1716712815499, 40.64204364399354], [-74.17167999810484, 40.64200786142031], [-74.1716942410576, 40.64198482288188], [-74.1717122849496, 40.64196149319705], [-74.171721869505, 40.64193954418145], [-74.17171130316294, 40.64192499166151], [-74.17169759758744, 40.64190848524939], [-74.17166487828337, 40.64189343774843], [-74.17162797232027, 40.641875784546365], [-74.17160447942199, 40.64186515003683], [-74.17159136106805, 40.641867619049684], [-74.1715798619212, 40.64187508403212], [-74.17156189474113, 40.64187719238308], [-74.17155066453365, 40.64186488555612], [-74.17153942223504, 40.64183795013771], [-74.17152666189581, 40.64181405545097], [-74.17150857950509, 40.64179349598847], [-74.17149496395666, 40.64177040017785], [-74.17148857094216, 40.64174382256751], [-74.17148332476262, 40.64172593597715], [-74.17147878989451, 40.641702509993536], [-74.17148353018582, 40.64168751580476], [-74.17148750672796, 40.64166672743326], [-74.17148506880689, 40.64165191683831], [-74.17147872144523, 40.641622045376614], [-74.171468008905, 40.641548249028986], [-74.17146506842424, 40.641500560496276], [-74.17145387097246, 40.64147032803623], [-74.17145228873805, 40.6414547212406], [-74.1714630212239, 40.64144146158959], [-74.17147622922754, 40.641432402212075], [-74.17148316419237, 40.64142674967047], [-74.17149114120241, 40.641414433961884], [-74.17149378298565, 40.641390821537875], [-74.1714963733101, 40.64136318967349], [-74.17149521934651, 40.64134718603417], [-74.17149054869456, 40.64133364361031], [-74.17148663794484, 40.64131858107016], [-74.17148462205134, 40.64129605866502], [-74.17148493532513, 40.641272990998544], [-74.17149328066832, 40.64124894186521], [-74.17150704445345, 40.64122228259365], [-74.17151573437815, 40.64121174061923], [-74.17152229426043, 40.641210505529756], [-74.17154059420163, 40.64120727340807], [-74.17156055313183, 40.64119842793582], [-74.17157689873703, 40.641191323114086], [-74.17158967454701, 40.64117534693891], [-74.17159279987557, 40.6411626700665], [-74.17160210287882, 40.6411458636061], [-74.17160617556183, 40.641125799144945], [-74.17159955189405, 40.64110838453246], [-74.17159008305946, 40.64109850324171], [-74.17158113805186, 40.64108894818463], [-74.17157328411733, 40.64107674772806], [-74.17157427204839, 40.64106606462145], [-74.17158024474797, 40.641045856267034], [-74.1715872192374, 40.64102959339448], [-74.1715971917376, 40.6410178555225], [-74.17160070795778, 40.64101539095349], [-74.17162486188907, 40.64101388919833], [-74.17165119164028, 40.64101225184257], [-74.1717056688524, 40.64101026839081], [-74.17175073432414, 40.641009738880086], [-74.17178249117184, 40.64101023055248], [-74.17183706459231, 40.64100897145658], [-74.1719130762053, 40.641006435883796], [-74.17197078141105, 40.640999816986145], [-74.17200429275435, 40.640995417484795], [-74.1720231585248, 40.64098921566286], [-74.17205432959555, 40.64097073115046], [-74.17208977321141, 40.640948261930674], [-74.17225779312604, 40.64086076172909], [-74.17229575775352, 40.640839194999465], [-74.17233215308795, 40.64081665291115], [-74.17236527396382, 40.64080204284475], [-74.17239216946237, 40.64078754395921], [-74.1724160191034, 40.64077181511472], [-74.17243121713608, 40.64075601915977], [-74.17244071098945, 40.64074066046268], [-74.17244758967797, 40.64072367362157], [-74.17246079147431, 40.6407072986612], [-74.1724794616066, 40.64069233295827], [-74.17251006213091, 40.64067681962939], [-74.17254523063315, 40.64066680711362], [-74.17261757416877, 40.64065467584057], [-74.17265255879144, 40.64065053025726], [-74.17270403672576, 40.64064402136012], [-74.17275233141466, 40.640638855609325], [-74.1727962478836, 40.640629634012406], [-74.17284329736425, 40.64061505330324], [-74.17290384335851, 40.64059358477594], [-74.17293896175397, 40.64057955269739], [-74.17296661535327, 40.64056353204564], [-74.1729879887585, 40.64054360451508], [-74.17299923819058, 40.640523356316514], [-74.17300388381133, 40.640507638145564], [-74.17299926424006, 40.6404981157247], [-74.17299051114733, 40.640491122002416], [-74.17298798800311, 40.640489105066244], [-74.17297990582061, 40.64048606719563], [-74.1729675456654, 40.64048701573994], [-74.17295918809823, 40.64049643471536], [-74.17294123198201, 40.64051317422253], [-74.17291914361653, 40.64053132684717], [-74.17288949955986, 40.64055408346093], [-74.17285348835655, 40.64057952423441], [-74.17282511932686, 40.640593768956556], [-74.17279941322334, 40.64060634912527], [-74.17275611439214, 40.64061661819036], [-74.17271366501747, 40.64061877673409], [-74.17262782121746, 40.64063047986057], [-74.17255704830454, 40.64064358932917], [-74.17250125093443, 40.64065737773337], [-74.17246114186268, 40.64067362066843], [-74.17243738848725, 40.64069007394007], [-74.17237264271998, 40.64074880691058], [-74.17230798868354, 40.64080094946075], [-74.17221543257125, 40.6408563282019], [-74.1721097577176, 40.64091417577344], [-74.17206200514698, 40.64094161338942], [-74.17202446166853, 40.6409554653829], [-74.1719924762055, 40.640964135763646], [-74.17196571583546, 40.64096874797935], [-74.17193676659375, 40.64097133351591], [-74.17189978466081, 40.64097490104444], [-74.1718640859913, 40.64097727266596], [-74.17181716981398, 40.64098196774045], [-74.17177628856427, 40.6409851024651], [-74.17173545900432, 40.64099225762033], [-74.17169348843088, 40.64099803436283], [-74.17165773761847, 40.64099638765365], [-74.17162317131236, 40.640994648532406], [-74.17160107265416, 40.64099634322898], [-74.171584960751, 40.64100160153897], [-74.17156994685418, 40.641011532092755], [-74.17155907481232, 40.64102736051345], [-74.1715539568967, 40.641046773840586], [-74.17154741606059, 40.64106995349325], [-74.17154914531588, 40.64109030297516], [-74.17156634010112, 40.64112958542867], [-74.171565452232, 40.641148307922435], [-74.1715588562989, 40.641160153836005], [-74.17154669424497, 40.64116986385235], [-74.17152934464757, 40.6411730220176], [-74.17151170840289, 40.64117400960279], [-74.17149730887861, 40.64117767281818], [-74.17149326948423, 40.64117981215319], [-74.17147992035214, 40.64119144364453], [-74.1714558850582, 40.641213037583995], [-74.17144203866056, 40.641253602025266], [-74.17143859751879, 40.641343845504196], [-74.17143965724362, 40.64135912690085], [-74.1714310172024, 40.6413736888144], [-74.17140864510931, 40.641396983696104], [-74.17139744134563, 40.64141393688061], [-74.17138651859484, 40.64142273652408], [-74.17138614681488, 40.64147972331705], [-74.17140176158097, 40.64149946089585], [-74.1713905404507, 40.64176250593573], [-74.17137382350516, 40.64208501526809], [-74.17132740135541, 40.642085084439465], [-74.17115412449108, 40.642399175687814], [-74.17117734635514, 40.642403561368766], [-74.17105017239898, 40.642593819129914], [-74.17099208752254, 40.64257180445464], [-74.17127524445598, 40.64211168276777], [-74.17121130105224, 40.64206757586895], [-74.17119260951203, 40.64173962651395], [-74.17028635441105, 40.64177530825585], [-74.17007878651035, 40.64208616057553], [-74.16915295569137, 40.641872859874304], [-74.16906431628871, 40.6421300167986], [-74.16840232530689, 40.64199124154421], [-74.16824343577274, 40.641941350361655], [-74.1682342144124, 40.64192427424841], [-74.16820402897147, 40.64186837353088], [-74.16812547492654, 40.64182533149997], [-74.16801680461005, 40.64183766681536], [-74.16774094051358, 40.64172777449842], [-74.16757607092507, 40.641844793608016], [-74.1667738499355, 40.64158163822526], [-74.16671693757165, 40.641478809199725], [-74.16653375792842, 40.641422637883714], [-74.16647971882989, 40.64143487167775], [-74.1662159991704, 40.642424100227906], [-74.1660998989604, 40.642406587001155], [-74.16631887292456, 40.6414366750049], [-74.16611608669034, 40.641370843361386], [-74.16604534908741, 40.64130051310063], [-74.16591014147886, 40.641257551347], [-74.1658011075589, 40.64122451144941], [-74.16563409763799, 40.641214371375646], [-74.16544310082392, 40.64228054670857], [-74.16494139041178, 40.64220899988932], [-74.16519041758224, 40.64064230577938], [-74.16481109266343, 40.6404337150713], [-74.16473426536795, 40.640391467024884], [-74.16457132366716, 40.64141866159588], [-74.16441459649108, 40.64139678472601], [-74.16459656597705, 40.640315746047506], [-74.16431603722535, 40.640104784619226], [-74.16416886813627, 40.63999411023031], [-74.16400179001414, 40.641065860006854], [-74.16386249420347, 40.64105279734373], [-74.16404731516639, 40.63994665879948], [-74.16399656455822, 40.639929798274785], [-74.16394491366823, 40.63991460682116], [-74.16384976588608, 40.63989150971772], [-74.16375256354029, 40.639874069881785], [-74.16365388535758, 40.63986239114689], [-74.16355431885064, 40.63985654304622], [-74.16345445681993, 40.639856560398236], [-74.16322855844138, 40.64094851032994], [-74.16285868894607, 40.64092993814722], [-74.16284583292605, 40.641019590245065], [-74.1627935602801, 40.64102962400666], [-74.16274123160056, 40.641016418743845], [-74.16263650082281, 40.64096013084526], [-74.16250131292811, 40.64092380472348], [-74.16242722719876, 40.64092390939185], [-74.16206517092041, 40.64078499076366], [-74.16206502578969, 40.64072523582194], [-74.16167534694229, 40.640610784766764], [-74.1622846525209, 40.63876565277233], [-74.16202564070505, 40.63869931460562], [-74.16181856612248, 40.6386462787776], [-74.1614491782676, 40.63942833666835], [-74.16130221801548, 40.63939373445705], [-74.16166010706984, 40.63866499331326], [-74.161240158151, 40.63859501161622], [-74.16115789550317, 40.63869406237822], [-74.16090490770398, 40.639386140816065], [-74.16073654872477, 40.63935101447539], [-74.16106428065294, 40.63868123971489], [-74.1610816175891, 40.63856776831252], [-74.16089198961086, 40.63854118915292], [-74.16081126840741, 40.638632958888756], [-74.16076221947098, 40.638623690257305], [-74.16075060309191, 40.63851840043888], [-74.1604218697275, 40.638465412322525], [-74.16034819993155, 40.63844372645346], [-74.15994899634474, 40.639940136777724], [-74.15986410465037, 40.639927823692716], [-74.16020691922398, 40.63860957529811], [-74.1602347053031, 40.63841031692488], [-74.16015283988946, 40.63838621753958], [-74.16012967800789, 40.63858162399912], [-74.16010280780186, 40.63862087570508], [-74.16009307313972, 40.63865076683728], [-74.1600857788726, 40.638675053080085], [-74.16005754066094, 40.63872235274986], [-74.16002615418796, 40.63873367347836], [-74.15999281101924, 40.63874106672675], [-74.15995835194632, 40.63874434606485], [-74.1599236458997, 40.63874342879966], [-74.1598895680379, 40.638738338061366], [-74.15985697767896, 40.638729202219416], [-74.15978175306086, 40.638649332401585], [-74.15973010466857, 40.63857844417414], [-74.15973732907446, 40.63852428097451], [-74.15972000040932, 40.63845334506176], [-74.1596709437014, 40.63844034192757], [-74.15964146528947, 40.638414239827924], [-74.1596314936956, 40.63834516180006], [-74.15963118099671, 40.63821444591709], [-74.15957712397919, 40.63799258321185], [-74.15950258014311, 40.63795644765375], [-74.15945704784016, 40.63793297921156], [-74.15944770353745, 40.63792816298503], [-74.1594197309645, 40.63790693181592], [-74.15931954037283, 40.63781040574547], [-74.1592271162294, 40.637796119756985], [-74.15916415812242, 40.637848201478775], [-74.15916107601004, 40.63785687116967], [-74.15914460807416, 40.63788998122662], [-74.15912711237603, 40.63792545603159], [-74.15911062904141, 40.6379514764553], [-74.15909621101908, 40.637976705628795], [-74.15908695137247, 40.63799641345514], [-74.15907664464935, 40.6380113955703], [-74.15905622378884, 40.638039310319826], [-74.15901521285012, 40.6382686180716], [-74.15898941372843, 40.63857894244465], [-74.15902532863447, 40.63858386637523], [-74.15900957887435, 40.63882257790974], [-74.15889532350845, 40.63881527713119], [-74.15893299160653, 40.63826401781902], [-74.15864580008073, 40.63824900845806], [-74.15863472442109, 40.63812847909774], [-74.15815277195976, 40.63810996561787], [-74.15814883915031, 40.6379675114477], [-74.15789262953875, 40.637965051231134], [-74.15784122272909, 40.638873914966155], [-74.15773352572349, 40.63887903516537], [-74.15777365937035, 40.63795716024865], [-74.15718782782525, 40.63795080522895], [-74.15710126690709, 40.63927771860018], [-74.15697723294244, 40.63927540119181], [-74.15707077305574, 40.63794713564092], [-74.15636848598677, 40.63790584773889], [-74.15636552863708, 40.63795074226029], [-74.15610276490888, 40.6379395908861], [-74.15611488264744, 40.63753074716438], [-74.15569862053141, 40.6375179389041], [-74.15568515769907, 40.63778748240713], [-74.1556182932615, 40.63847022002475], [-74.15549268934303, 40.638466779526425], [-74.15555271196432, 40.63778433017423], [-74.1555891193463, 40.63778562248824], [-74.15558643758918, 40.63773077586534], [-74.15482932461856, 40.63772385541876], [-74.15475933665317, 40.63919632730392], [-74.15469263864514, 40.63919396811424], [-74.15477829011976, 40.63772154555092], [-74.15464669833945, 40.63744436854495], [-74.15459670163321, 40.63734334406899], [-74.15454919896747, 40.637296472567364], [-74.15445425716959, 40.637229807259075], [-74.1543830825974, 40.637193798384054], [-74.15425270073524, 40.63717231088821], [-74.15415791876254, 40.63717424412366], [-74.15402997235756, 40.637181635569675], [-74.15388072157435, 40.63719808283956], [-74.15376464086046, 40.63721268010741], [-74.15362965258818, 40.637248963820376], [-74.1535112231501, 40.637272589413755], [-74.15341406597796, 40.63727271907927], [-74.1532759228053, 40.63726700478363], [-74.15313666098517, 40.6372062971501], [-74.1530299679276, 40.63718116680883], [-74.15296592921918, 40.63715597795897], [-74.15292086749692, 40.63713979145949], [-74.15288058323338, 40.63713984502593], [-74.15282279548838, 40.63713494070998], [-74.15279703511403, 40.637132754852544], [-74.15270060752232, 40.63719243556223], [-74.15269603885832, 40.637266454488504], [-74.15271277799731, 40.63733322496383], [-74.15272473935788, 40.63738195005758], [-74.1527247451835, 40.63738449548157], [-74.15272506988086, 40.637526366764654], [-74.1527251690349, 40.637569690402586], [-74.15269700449673, 40.63764184172464], [-74.15242876126817, 40.63766334125416], [-74.15221806884477, 40.63769258171682], [-74.15229786175496, 40.63827428826357], [-74.15213675899915, 40.6382907479556], [-74.15205180954048, 40.63770506596079], [-74.15204639233272, 40.63766721531547], [-74.15119545007182, 40.63773078853614], [-74.15126280621466, 40.63848705662542], [-74.15139517613655, 40.639396513057434], [-74.15124827966567, 40.63940934288591], [-74.15112965090489, 40.63850068007395], [-74.1505563884209, 40.63855933066728], [-74.15044035571407, 40.63843017599427], [-74.15043886424382, 40.638427351903644], [-74.15041179728392, 40.638376057015414], [-74.15039275370141, 40.63833817252051], [-74.15030855893832, 40.638281016934656], [-74.15011380443705, 40.63826728554247], [-74.14996782090535, 40.6381500752493], [-74.14979386126164, 40.63789278086416], [-74.1496788983088, 40.63751767426054], [-74.14950908093832, 40.637435128284935], [-74.14949405062916, 40.63742469208777], [-74.14947691985856, 40.637416342754534], [-74.14945818220086, 40.63741032084707], [-74.1494383775278, 40.63740679986927], [-74.1494180764536, 40.63740588126793], [-74.14939786389473, 40.63740759150982], [-74.14937832221757, 40.6374118813193], [-74.14936001445896, 40.6374186270979], [-74.1493123559734, 40.63752595814033], [-74.14945194107334, 40.637885156579], [-74.14941881708341, 40.63829164054704], [-74.14931701723211, 40.63839012084782], [-74.14930052539914, 40.63843346602415], [-74.14929120382706, 40.63850388089491], [-74.14929120773513, 40.638505627711645], [-74.14929130075687, 40.63854720620637], [-74.14928187482029, 40.638570686367906], [-74.14924406471185, 40.63861767029022], [-74.14919418155682, 40.63866833137192], [-74.1488817554376, 40.63870520797325], [-74.14855815092196, 40.63743783752723], [-74.14852816684945, 40.6374452135166], [-74.14851602490765, 40.63754707359788], [-74.14877792005885, 40.6388762511277], [-74.14761237249002, 40.63903703784254], [-74.14749939740649, 40.63890959781476], [-74.14602173101719, 40.639094549700694], [-74.14605733618387, 40.63922925768986], [-74.1451927948153, 40.63937849765469], [-74.14484204603131, 40.63935381470644], [-74.14467133234159, 40.639304267726864], [-74.14459085274906, 40.639201285704395], [-74.14433923109627, 40.639096025578574], [-74.1442614355038, 40.639039226440964], [-74.14422822731413, 40.639024826622084], [-74.14416181663793, 40.63899783257256], [-74.14409302447481, 40.63896542493142], [-74.14405027073543, 40.638920348614874], [-74.14402416136578, 40.6389005233065], [-74.1439530270115, 40.63888075604148], [-74.14388678558862, 40.63888460968336], [-74.14386163595883, 40.63888796251683], [-74.14383745787333, 40.63889421166692], [-74.14381482250369, 40.63890320950684], [-74.14379426457702, 40.638914743475986], [-74.1437762697454, 40.63892854110129], [-74.14376126311204, 40.63894427643435], [-74.14374959918894, 40.63896157775063], [-74.14384192519628, 40.639088355055904], [-74.14381734530275, 40.639207478923986], [-74.14374287110046, 40.63922942280874], [-74.1436836356135, 40.63923310741767], [-74.1436457188332, 40.639233154789], [-74.14361490860774, 40.639231387648564], [-74.14357308593789, 40.6392267852571], [-74.14348833765422, 40.63932555253008], [-74.14342476830103, 40.63946633799697], [-74.14345479580568, 40.639477458118186], [-74.14336103010835, 40.63970984518118], [-74.14328000083096, 40.63971866302346], [-74.14317676696162, 40.639692146338845], [-74.14312802471521, 40.63968037306379], [-74.14359998299996, 40.638514125999905], [-74.14365267799991, 40.63848492799989], [-74.14478228999994, 40.637953779999904], [-74.14556478899993, 40.63545901799992], [-74.14566932799993, 40.635051931999946], [-74.14583905999997, 40.634574971999925], [-74.1464212309999, 40.632719835999836], [-74.14659947199989, 40.63212929899992], [-74.14666005299989, 40.63195271099987], [-74.146830166, 40.6313826539999], [-74.14691085899993, 40.631156230999856], [-74.1473730989999, 40.6298270839999], [-74.14752618999992, 40.62938928799988], [-74.14635596699999, 40.629154846999924], [-74.14628562599995, 40.62913738599987], [-74.14622202799995, 40.62910695699992], [-74.14617085499992, 40.62906625599992], [-74.14612099499995, 40.628994393999896], [-74.14608756399993, 40.62891590299991], [-74.14607246899999, 40.628834198999854], [-74.14607595499996, 40.62875294999986], [-74.14593544599994, 40.62782411499994], [-74.14565757199988, 40.62782953799991], [-74.14554071999994, 40.62783252199995], [-74.14554348099993, 40.62752118999992], [-74.14559821999993, 40.62698877599997], [-74.14560791099998, 40.626894508999904], [-74.14563382799992, 40.62672514999989], [-74.1457144359999, 40.62643373899992], [-74.1457599769999, 40.626301354999946], [-74.145917453, 40.625843585999846], [-74.14622812499992, 40.6249253619999], [-74.14650354399993, 40.62420010099991], [-74.14678442599998, 40.62360755099992], [-74.14710957699992, 40.623032259999924], [-74.14728835499992, 40.62276964599989], [-74.14745462999993, 40.622530399999896], [-74.14753387799992, 40.62241637399988], [-74.14772830399991, 40.62213661899995], [-74.14818936799996, 40.621601078999944], [-74.14825244699995, 40.62152904599993], [-74.14831982299995, 40.62144951599987], [-74.14859494699986, 40.621167623999945], [-74.14911237299995, 40.62063746199988], [-74.1492283809999, 40.62052057799983], [-74.14926443299994, 40.620484252999915], [-74.149823327, 40.6199211269999], [-74.15019521699998, 40.61953838199983], [-74.15037556699993, 40.619352764999896], [-74.15071865699997, 40.61899965099989], [-74.15093894399993, 40.618774638999916], [-74.15135466199992, 40.618349997999864], [-74.15165757899996, 40.6180405579999], [-74.15183858099991, 40.61782347299985], [-74.15205801199987, 40.617533619999904], [-74.15219115799992, 40.61735094499989], [-74.15231170399994, 40.61716275599988], [-74.15241908699996, 40.61696983799988], [-74.15251285699996, 40.616773003999924], [-74.1525926639999, 40.61657308099992], [-74.15281391799995, 40.61582562099988], [-74.15290527299994, 40.615516992999865], [-74.152956191, 40.615329732999896], [-74.15321287899991, 40.61438556299991], [-74.15368213999994, 40.612659589999915], [-74.15371281699993, 40.61254675799992], [-74.15375060999999, 40.612399390999876], [-74.15379116699992, 40.612255611999906], [-74.15386832299995, 40.612320627999964], [-74.15474080199988, 40.61299231699992], [-74.1547926529999, 40.61303223099987], [-74.15651187899992, 40.61435448199988], [-74.157373837, 40.615031910999924], [-74.15792673899992, 40.615457424999896], [-74.15861741699992, 40.61593931499988], [-74.15912798199999, 40.61629637499986], [-74.15986704899998, 40.61681322499988], [-74.16052177399989, 40.61722978899994], [-74.16163010299996, 40.61793493499987], [-74.1623545779999, 40.618335637999955], [-74.16337391799988, 40.61885458999991], [-74.1639409499999, 40.61914326199996], [-74.16484190399989, 40.61957240499988], [-74.16570682499993, 40.61995403599987], [-74.16603193299996, 40.62009453499998], [-74.16742666699999, 40.620677283999896], [-74.1676971039999, 40.620790273999916], [-74.16808086299993, 40.62093625599991], [-74.16847311299996, 40.62104581499989], [-74.16878471999995, 40.621124028999866], [-74.16888894299989, 40.62115114299992], [-74.16923624499995, 40.6212164329999], [-74.16953656499994, 40.621270499999916], [-74.16985740699987, 40.621312386999925], [-74.17095546499992, 40.621442180999935], [-74.17181177999997, 40.62154338399994], [-74.17207112899989, 40.62157862099991], [-74.17236260999994, 40.62163579899988], [-74.17270752199991, 40.62171153799988], [-74.1732673079999, 40.621838983999915], [-74.17361243999996, 40.621948740999855], [-74.17395016799996, 40.622073168999854], [-74.17427887999997, 40.62221179199985], [-74.17459706299995, 40.622364010999874], [-74.17490328599996, 40.622529106999885], [-74.17519625699995, 40.62270624399988], [-74.17610052899997, 40.623406241999895], [-74.17775298599997, 40.62493844999993], [-74.1778150209999, 40.62499096299995], [-74.1780882019999, 40.62526042699992], [-74.17828177399998, 40.62545136099991], [-74.17888390599995, 40.62593287099991], [-74.17918377399995, 40.626137884999906], [-74.17924623499991, 40.626180580999886], [-74.1795447069999, 40.62636235399988], [-74.17993706099998, 40.626601296999894], [-74.18033424799995, 40.626834629999905], [-74.18190235699991, 40.62775580699991], [-74.18368103499995, 40.62874067099991], [-74.18649341199989, 40.63021905099987], [-74.1921598649999, 40.633230970999946], [-74.19248359599999, 40.633405071999945], [-74.1955597527974, 40.635023986316575], [-74.19552753757247, 40.635054599201126], [-74.19550852184707, 40.63507698291241], [-74.19548692692484, 40.63509729984497], [-74.19547347096992, 40.6351099614397], [-74.195460045448, 40.63514705490001], [-74.19542894127432, 40.63517593464232], [-74.19539218181946, 40.63519020256258], [-74.19536142041393, 40.63519932032491], [-74.195329349119, 40.63521215576101], [-74.19530704356409, 40.635233895517125], [-74.19530406581976, 40.635257195642076], [-74.19530747310547, 40.63528711471202], [-74.19528935031731, 40.635300943009256], [-74.1952551116096, 40.63533379058962], [-74.1952126270202, 40.63537717331163], [-74.19518577309628, 40.63541578104539], [-74.1951782949683, 40.635442944149275], [-74.19518601247457, 40.63546657139275], [-74.19520661370926, 40.63549123956764], [-74.19521832967706, 40.63551143455646], [-74.19521533742005, 40.63552091792131], [-74.19519958010655, 40.635524072102406], [-74.19519126481347, 40.63552688696693], [-74.19518208481396, 40.63553909111777], [-74.19517890816915, 40.635553052173], [-74.19518280919583, 40.635575632975176], [-74.19519896069322, 40.635591155928466], [-74.19521804797338, 40.63560329500408], [-74.19522420248127, 40.63562740034538], [-74.19521834364427, 40.635647986363836], [-74.19520941676511, 40.63566342892389], [-74.195205241607, 40.63567824858528], [-74.19520450148279, 40.63568925481752], [-74.19520096900095, 40.635707944646235], [-74.19517661084649, 40.63573314917541], [-74.19515479613113, 40.63574519082253], [-74.19449480774462, 40.63644952194788], [-74.19451352160544, 40.636456336361896], [-74.19453800288491, 40.636474699338216], [-74.19454716543115, 40.63648953862543], [-74.19454462292255, 40.63651620314385], [-74.19452606843777, 40.63653434575063], [-74.19449323469706, 40.63654971561236], [-74.19445852262734, 40.63655548454359], [-74.19443606542332, 40.636564072022715], [-74.19442256854931, 40.63658210621315], [-74.19441386949622, 40.6366335841286], [-74.19444962836066, 40.63680402638214], [-74.19447252206251, 40.636874485003666], [-74.194496454051, 40.636918696617556], [-74.19450295976435, 40.63697061655945], [-74.19451851091635, 40.6370342919478], [-74.19454212579782, 40.63707002762785], [-74.19456129013619, 40.637081565084394], [-74.19459412279457, 40.637093190311745], [-74.19462135058987, 40.63709028300463], [-74.19464418330966, 40.63707821728935], [-74.19465612062449, 40.63705906092511], [-74.19466484842447, 40.63704884313686], [-74.19467950093836, 40.637034642034195], [-74.19469106179373, 40.63703246234289], [-74.19470211317162, 40.637034750838815], [-74.19473403883907, 40.637040775395356], [-74.19477123463915, 40.63708007013124], [-74.19478689682471, 40.637131511664265], [-74.19477657384647, 40.63717686213754], [-74.19477150701454, 40.637196294021436], [-74.19477077585893, 40.637199101536524], [-74.19475539880962, 40.63723051596478], [-74.19473713646656, 40.63726735349718], [-74.19462310145185, 40.637391183563665], [-74.19454065381193, 40.63741367009579], [-74.19450031360971, 40.63742228013341], [-74.19442405923806, 40.63741899183633], [-74.1943903812557, 40.63740538243714], [-74.19438004804958, 40.63739734997404], [-74.19436689245398, 40.637389691179415], [-74.19435954275859, 40.63737008161258], [-74.1943594357064, 40.63736919137743], [-74.1943580610104, 40.637357710600014], [-74.19435651895958, 40.63733022238395], [-74.194337169516, 40.63731630227532], [-74.19431119833348, 40.63731570385956], [-74.19428109920928, 40.63732721369496], [-74.1942575100169, 40.63733586721419], [-74.19423516733062, 40.6373371297609], [-74.19420411105152, 40.63732307611526], [-74.19417412801486, 40.63731055069036], [-74.194132757168, 40.63730641221756], [-74.19408281695587, 40.63730439838809], [-74.19397308627491, 40.637309114196356], [-74.19389836339376, 40.63732469610308], [-74.19382692423089, 40.637346408674595], [-74.19377428207832, 40.63736732324974], [-74.19372224854173, 40.637390939351555], [-74.1936265512071, 40.6374446558369], [-74.19358317694844, 40.637469116884674], [-74.19371964621938, 40.637566814230105], [-74.19323428828837, 40.63796129947735], [-74.18702347003286, 40.64298164152387], [-74.18652496897599, 40.64338866217145], [-74.1862320980967, 40.64318739685526], [-74.18610078573344, 40.64327121851158], [-74.18606318400101, 40.643447852318566], [-74.18598170439189, 40.643513462230736], [-74.18590896667844, 40.64363143950175], [-74.18563857496096, 40.64376937987739], [-74.1855741946739, 40.643802222588285], [-74.18550116554881, 40.64381543579884], [-74.18535926743486, 40.643792747842085], [-74.18534542963745, 40.64378629130628], [-74.18530332013262, 40.643766646729574], [-74.18520452940678, 40.64378972313489], [-74.18513580682765, 40.64380620369449], [-74.18511431614886, 40.64380623828999], [-74.18508860902325, 40.643835744455565], [-74.18510588327057, 40.64386518143945], [-74.1851743550746, 40.64390294493063], [-74.18524793597288, 40.643943526193354], [-74.1851963785221, 40.64395015675915], [-74.18516634593075, 40.64396984836739], [-74.18513627010243, 40.64397317080408], [-74.1849818130333, 40.64407163676705], [-74.1849268329308, 40.644071725191544], [-74.18489155397833, 40.644071781916836], [-74.18486580195882, 40.6440849187032], [-74.184846586784, 40.64412644260538], [-74.18483594580836, 40.644128266338726], [-74.18482549519861, 40.64411711231906], [-74.18478881821554, 40.64407218744858], [-74.18477310833003, 40.6440452445851], [-74.18477004263795, 40.644025785686964], [-74.1847708510447, 40.64400884019649], [-74.18477183957653, 40.64399856541102], [-74.18476664308692, 40.643978449537975], [-74.18475773562074, 40.64395738052475], [-74.1847539079795, 40.643945431488284], [-74.18474860069836, 40.643939074257645], [-74.18472119483637, 40.64392855576273], [-74.18470012186833, 40.643924333039635], [-74.18468425564609, 40.64392058058057], [-74.18465961650686, 40.643911858226716], [-74.184640374521, 40.643890629725746], [-74.18462815701118, 40.64384775913575], [-74.1846258908309, 40.643831000853474], [-74.18461771832088, 40.64381695847983], [-74.1846048914571, 40.64380280664229], [-74.18459493028341, 40.643796339437415], [-74.18458607648822, 40.64379058848246], [-74.1845779388453, 40.64378676044492], [-74.18455325306144, 40.643787471016395], [-74.18453535594595, 40.643788948860255], [-74.18451247582924, 40.64378719335035], [-74.18449226409504, 40.64378134685901], [-74.18447219805573, 40.64377195455214], [-74.18445533338135, 40.64375372432168], [-74.18444714306519, 40.64373457669358], [-74.18444479428362, 40.64371703902874], [-74.18444338933051, 40.643698657180494], [-74.18444476870414, 40.64368717857925], [-74.18444458664025, 40.64368051055708], [-74.18444233837069, 40.643668860361124], [-74.18443576130949, 40.64366022207072], [-74.18442444966344, 40.643650691576], [-74.18440200669028, 40.64363830195875], [-74.18429967762147, 40.64360678782117], [-74.18424412017556, 40.64359836906536], [-74.18420047676955, 40.64359040664668], [-74.1841781328029, 40.64358390378721], [-74.18415521885615, 40.643571933698], [-74.18413275658622, 40.6435544364428], [-74.18411319064435, 40.64353008362579], [-74.1840953581969, 40.64348283707021], [-74.18406923513074, 40.64341055572148], [-74.18406377506524, 40.64339453765575], [-74.18406031079249, 40.643384379231534], [-74.18405559628631, 40.643344194787886], [-74.18406483232063, 40.6433239904458], [-74.18407613737229, 40.6433087672738], [-74.18408854781073, 40.643294263207466], [-74.18410655661627, 40.64327902712314], [-74.18411386017405, 40.643269940062865], [-74.18411737168672, 40.643259118784826], [-74.18412096541132, 40.64324907923165], [-74.18412999241754, 40.64323674404399], [-74.18414084291625, 40.643227048534335], [-74.18415359802326, 40.64322077493147], [-74.18416475732954, 40.6432090970599], [-74.18417552561037, 40.64319862166465], [-74.18418691194765, 40.64318417953793], [-74.18419239221019, 40.643172453799494], [-74.18419743534481, 40.6431713609731], [-74.18420698929333, 40.64317392578003], [-74.18421664086912, 40.64318237688753], [-74.18422196499976, 40.6431938415652], [-74.18422232688232, 40.64320717794932], [-74.18422669045312, 40.64321437988862], [-74.18423281336433, 40.64322854468718], [-74.18423961563511, 40.64323441924199], [-74.18424970411742, 40.6432322357593], [-74.18426010141027, 40.64322807008332], [-74.18427310093755, 40.64322413678721], [-74.18428327075355, 40.64322273553051], [-74.18429202526654, 40.64322259736244], [-74.18429794940448, 40.64322498905803], [-74.18431436846345, 40.64322910020108], [-74.18432170877976, 40.64323022696153], [-74.18432855692002, 40.64322666923377], [-74.18433208756102, 40.643220954362135], [-74.18432725184827, 40.64321417485346], [-74.18431524134853, 40.64320783052454], [-74.18430519961646, 40.643200581734234], [-74.18430518133162, 40.64319547481716], [-74.18431076052859, 40.64318963537222], [-74.18432296259024, 40.64318300362269], [-74.18433282380548, 40.643183582878514], [-74.18434756615507, 40.64318151053774], [-74.18436157408281, 40.64317241053612], [-74.18436264465592, 40.643162915635614], [-74.18435922446363, 40.64315487286987], [-74.18435234019536, 40.643148216764445], [-74.18434174390217, 40.643140608177035], [-74.18433358916838, 40.64313167437568], [-74.18433277341413, 40.64312386448769], [-74.18433937666497, 40.64311796476475], [-74.18434858566671, 40.643112297917504], [-74.18435976254926, 40.64310572661003], [-74.18436802684603, 40.643100903275275], [-74.18437046773803, 40.643099576889924], [-74.18435901049814, 40.64309359136269], [-74.18433873506196, 40.64309207087385], [-74.18431210081681, 40.643098793271136], [-74.18429610585869, 40.64310369365737], [-74.18427562166455, 40.64311004282009], [-74.18425765815478, 40.64311584665501], [-74.1842418913177, 40.64311798043344], [-74.18422288569082, 40.64311874097171], [-74.18421444000022, 40.643116895769445], [-74.18420706556407, 40.64310555566647], [-74.18419895716902, 40.643087188404834], [-74.18419133566236, 40.6430735046343], [-74.18418955959052, 40.64306143317914], [-74.18419643752019, 40.64304333508839], [-74.18420079256292, 40.64302578282631], [-74.18419608628763, 40.643010351584735], [-74.18417961224515, 40.64299092000832], [-74.18415887514915, 40.642970174627315], [-74.18414539613768, 40.642949774959355], [-74.18413505499888, 40.642924863909315], [-74.184091145958, 40.6427959032249], [-74.1840764542197, 40.642744144608876], [-74.18406005446543, 40.64270074003321], [-74.18403744099062, 40.64266203570204], [-74.18402454295668, 40.64262745545523], [-74.18401176301163, 40.64260387076817], [-74.1840048060151, 40.642576789651365], [-74.18399139173812, 40.64255206410089], [-74.18398053223476, 40.64253700539437], [-74.18396961120783, 40.642526273215495], [-74.1839595333223, 40.64250881156425], [-74.18395490670638, 40.642494160701105], [-74.18394325132424, 40.64247640186284], [-74.18393263758804, 40.642463686318415], [-74.1839035841428, 40.64243244377228], [-74.18382380222785, 40.642374809424595], [-74.18372750164232, 40.6423071746499], [-74.18372084348564, 40.64229775643333], [-74.1837193912754, 40.642288806853855], [-74.18372612396574, 40.64227425476805], [-74.18372687316322, 40.6422733378588], [-74.18365067064063, 40.642239641050644], [-74.18365044228705, 40.64223993676863], [-74.18363300578271, 40.6422606383333], [-74.18362163771548, 40.64228018732448], [-74.18361500455805, 40.642300627378575], [-74.18361271660737, 40.642323160970086], [-74.18361672504857, 40.64234177790494], [-74.18362698370106, 40.6423659086179], [-74.18363631699478, 40.64239599025656], [-74.18365642485556, 40.64244034690106], [-74.18367268913023, 40.64246765022708], [-74.18368608635798, 40.64248727006792], [-74.18370501881277, 40.64251048159308], [-74.18372202672101, 40.64252516763561], [-74.1837372296301, 40.64254231856232], [-74.18379548529253, 40.64260125824884], [-74.18380665138265, 40.642614333109705], [-74.18380877202564, 40.64263463504347], [-74.18380378190209, 40.64265104760307], [-74.18379706677689, 40.6426707066114], [-74.18379225864186, 40.6426937865241], [-74.18378812880815, 40.642708574856904], [-74.18379068641373, 40.64271824287462], [-74.18380131774566, 40.64273606384266], [-74.18383019147466, 40.64276063838623], [-74.18386697631102, 40.64279180425758], [-74.18388937232362, 40.64281362860443], [-74.1839143096972, 40.64284001169558], [-74.18392865089538, 40.64285878833344], [-74.18394450544073, 40.64288218650995], [-74.18394945937301, 40.64289996042305], [-74.18394807926667, 40.64291143751274], [-74.18394489393962, 40.642925382843096], [-74.18395277585942, 40.64294651504454], [-74.18396522943465, 40.64296697618324], [-74.18398047012975, 40.642994342082446], [-74.18401938279032, 40.64307056303376], [-74.18402970542948, 40.643090368852036], [-74.18402611192003, 40.64310040856969], [-74.18401605931439, 40.64311280519824], [-74.18399969266284, 40.64312401309292], [-74.18397514388748, 40.64314082585007], [-74.18396037388047, 40.64315743733749], [-74.18394767177905, 40.6431790311709], [-74.18393449863362, 40.64320104625341], [-74.18392013590598, 40.64322156234678], [-74.18391586166544, 40.64323989565509], [-74.18391107965493, 40.643248435709474], [-74.18390346813281, 40.64325950611962], [-74.18389568292724, 40.64326509442863], [-74.18387557032597, 40.64326513580326], [-74.18384575936906, 40.643266156807364], [-74.18382325266997, 40.64325809176602], [-74.18380081037569, 40.6432457003549], [-74.18378349279038, 40.64323299986798], [-74.18376016910378, 40.643217125594596], [-74.18374322410038, 40.643198115373536], [-74.18373830844334, 40.643190555283866], [-74.18373047952299, 40.64318474281771], [-74.18372312230241, 40.643178509602784], [-74.18371078634262, 40.64316904165454], [-74.18369172686104, 40.64315448135174], [-74.1836746827703, 40.64312958314572], [-74.18366454111677, 40.6431164458238], [-74.1836556701408, 40.643105589385776], [-74.1836367564231, 40.64308748309497], [-74.18362054583271, 40.64307550169182], [-74.18360544172096, 40.643064235877986], [-74.18356172737563, 40.64303584822491], [-74.1835450268846, 40.643019179971134], [-74.18353931163176, 40.643008917570086], [-74.1835297885602, 40.64299181350821], [-74.18351569164177, 40.6429753814987], [-74.18349671423685, 40.64296160054375], [-74.18347427166451, 40.64294920957172], [-74.18345121209893, 40.64294078532028], [-74.18343384779197, 40.64293751705893], [-74.18340441023176, 40.642932228220374], [-74.18338450746943, 40.64292439934391], [-74.18337360336598, 40.64291877285582], [-74.18334541783328, 40.64291066031903], [-74.18332782755705, 40.64291015310812], [-74.18330953917729, 40.642912834241876], [-74.1832835379203, 40.64292069726795], [-74.18326590229488, 40.64292962399897], [-74.1832500261246, 40.642945517454685], [-74.18324212279416, 40.64296367810297], [-74.18324399975582, 40.64298163754487], [-74.18325446680502, 40.642997897456446], [-74.18326729257107, 40.64301205061985], [-74.1832835392777, 40.643034246578075], [-74.18330191653295, 40.6430571000389], [-74.1833180172884, 40.64308284079816], [-74.18332858412336, 40.64310498867681], [-74.18334351509431, 40.64313433802367], [-74.18335949769718, 40.64314908388614], [-74.18338103400569, 40.64317253121236], [-74.18339926624556, 40.643198930634746], [-74.183416700406, 40.64322262607264], [-74.18342822891086, 40.643249037857494], [-74.1834377710867, 40.64327124716789], [-74.18344583352973, 40.643299047261856], [-74.18345513064988, 40.64331891407218], [-74.18346315710941, 40.643336502004956], [-74.18346988667655, 40.64336634740397], [-74.18347382733964, 40.643404058370926], [-74.18347453640791, 40.64341085723141], [-74.18346987385277, 40.643430391652956], [-74.18346695058911, 40.643451785381494], [-74.1834630019431, 40.643473242228794], [-74.18345208754049, 40.643487263033094], [-74.18343248256524, 40.64349709540773], [-74.18341365855974, 40.64350452316628], [-74.18339963287723, 40.643508516133885], [-74.18338303939252, 40.643522488358435], [-74.18337764050345, 40.64353499561693], [-74.18337670464464, 40.64356059064253], [-74.18338476709732, 40.64358839073988], [-74.18339832798834, 40.643609571551366], [-74.18341590888673, 40.64362972268411], [-74.18343316208438, 40.64364675076355], [-74.18345736459925, 40.643666106790555], [-74.18348360638707, 40.643704985535884], [-74.18349031843778, 40.64372972401451], [-74.18349881775332, 40.643746889017144], [-74.18350690847798, 40.64376015158669], [-74.1835188912455, 40.64378103402031], [-74.18353011105356, 40.643809428653675], [-74.1835334205677, 40.643831230084984], [-74.18354232785633, 40.643852301370764], [-74.18354306065436, 40.64385932736155], [-74.18353584047904, 40.64386919543811], [-74.1835291889148, 40.643884529739495], [-74.18354289587367, 40.64390216505724], [-74.18356308699293, 40.643947303267495], [-74.18361739421937, 40.644027699885754], [-74.18364108756896, 40.64408166309698], [-74.18366485015768, 40.64415605230031], [-74.18368494209423, 40.644195303687], [-74.18369981985037, 40.644209332734825], [-74.18370845991122, 40.6442673503823], [-74.18371510901716, 40.64429641587898], [-74.18371629728709, 40.64429791537322], [-74.18372435273662, 40.6443009629189], [-74.18372970545593, 40.6442978879215], [-74.18373445919728, 40.644303885896356], [-74.18375686313779, 40.64435046212028], [-74.18379367344528, 40.64441148862553], [-74.18380044942428, 40.64443190122303], [-74.18379336287131, 40.64445787062245], [-74.18377555128016, 40.64448488143148], [-74.18376216878362, 40.644514767827275], [-74.18374425935657, 40.6445358918282], [-74.18372750215218, 40.644548301989275], [-74.18370426855401, 40.64455796178771], [-74.18368396408773, 40.644570978701886], [-74.18366981003881, 40.64458362459776], [-74.18367062573019, 40.644591433316485], [-74.1836716056141, 40.644600803140726], [-74.1836674587201, 40.64461048505006], [-74.18365974655892, 40.6446156674707], [-74.18363390887212, 40.64462509201518], [-74.1835946460288, 40.64462943657016], [-74.18356389126626, 40.64463130167616], [-74.1835395966405, 40.64463080955412], [-74.18350412369487, 40.644636888820415], [-74.18348441994567, 40.64464083490324], [-74.18344730522705, 40.64465094164889], [-74.18341751142201, 40.64465706945902], [-74.18339190186244, 40.644663730198744], [-74.18335950399491, 40.64466962442314], [-74.1833256914268, 40.64467678070959], [-74.18328823272974, 40.64467865860643], [-74.18326782928085, 40.64468578914795], [-74.18324492195774, 40.64469857240891], [-74.18322611315764, 40.64471110772638], [-74.18320935712921, 40.64472351663691], [-74.18318312917108, 40.64473414455775], [-74.1831516115432, 40.64474351990829], [-74.18310606948668, 40.64475688935173], [-74.18298000103783, 40.64479439161924], [-74.18288931621824, 40.644800282946065], [-74.18279757007667, 40.644796021805305], [-74.18272829668106, 40.644789613019086], [-74.18265318039725, 40.64478159346869], [-74.1825850137458, 40.64477590203921], [-74.18251092355709, 40.644767819786416], [-74.18242832172822, 40.644762218686346], [-74.18233563311568, 40.644758800169], [-74.18227968543268, 40.64475158276636], [-74.18222090790327, 40.64474689514837], [-74.1820090620485, 40.64470394355196], [-74.18198616489583, 40.64469708045339], [-74.18195246178735, 40.644690478202044], [-74.18192823190024, 40.64468566155351], [-74.18189998192533, 40.64468187234611], [-74.18186789280165, 40.644685782289194], [-74.18184219957065, 40.64469166162958], [-74.18179397192915, 40.64470401421466], [-74.18174935519656, 40.64471143304819], [-74.18171591587043, 40.64471228249049], [-74.18166763350055, 40.64470931427318], [-74.18162706434607, 40.644701164252986], [-74.18159074200629, 40.64468922136376], [-74.18154423129559, 40.64467357440472], [-74.18149439873258, 40.644655771213934], [-74.18144993953203, 40.644639999520756], [-74.18141652844002, 40.64462630683647], [-74.18138106656384, 40.64461274101333], [-74.18133327279313, 40.64459991030747], [-74.18129509360917, 40.6445896616227], [-74.18127078096558, 40.64458406327712], [-74.18124435360966, 40.64458291363371], [-74.18121659349188, 40.6445838111074], [-74.18118434243927, 40.64458615924491], [-74.18116006545075, 40.64459077303499], [-74.18113469914123, 40.644599774772665], [-74.18111257031259, 40.6446101536854], [-74.18108652313572, 40.644627448988174], [-74.18106508249205, 40.64465428616894], [-74.18104239123718, 40.64468395135612], [-74.1810329091121, 40.64470181345178], [-74.18100704872606, 40.64475052861926], [-74.18099528793684, 40.644771279645], [-74.18097651497148, 40.64479402725502], [-74.180963096418, 40.644813698818204], [-74.18094445062566, 40.64482779470521], [-74.18091579948613, 40.64484485483489], [-74.18089420751467, 40.64485048476192], [-74.18087212513213, 40.644851429678305], [-74.1808501901788, 40.64484883028889], [-74.1808286272017, 40.64483992218444], [-74.1808068653847, 40.64481923762298], [-74.18080183217685, 40.644800684193044], [-74.18080512854887, 40.64477298132178], [-74.18080739951004, 40.64474533936766], [-74.18080780137126, 40.64472449263601], [-74.18080699545725, 40.644697037287926], [-74.18080510372455, 40.64467397072121], [-74.18080491630867, 40.64464254933103], [-74.18080328559822, 40.64462693285812], [-74.18079999363216, 40.644610237599075], [-74.18079289381421, 40.64458669958499], [-74.18078836640565, 40.64457793875232], [-74.1807699430072, 40.64456451716834], [-74.18075192718292, 40.644555000705274], [-74.18071298324575, 40.64453771481314], [-74.18067579790016, 40.644527395171814], [-74.18062914205834, 40.64451529314595], [-74.18060507292968, 40.64451203666371], [-74.180581541482, 40.64450664166577], [-74.18052719090944, 40.64448469563185], [-74.18050323923352, 40.64445745168033], [-74.18049917835953, 40.644427145743954], [-74.18048997769914, 40.64435989958375], [-74.18048173953655, 40.64427892477651], [-74.18004077949224, 40.644288140673645], [-74.1800477217942, 40.644371680729826], [-74.17990671455671, 40.644378721669376], [-74.1799240477021, 40.64465501204932], [-74.17993656012413, 40.644654899158915], [-74.17996507274022, 40.645266982302964], [-74.17989568340026, 40.64526879647802]]], [[[-74.15945602438187, 40.641448333324036], [-74.15974815874293, 40.64141652579022], [-74.15997875699605, 40.641446480836365], [-74.1601497797168, 40.641494681697715], [-74.16032485634237, 40.641576178173665], [-74.16046972940245, 40.641643622512944], [-74.16058341643887, 40.64166664358547], [-74.16069638162347, 40.64168952114458], [-74.16077833143676, 40.64173903462685], [-74.16085388585509, 40.641754066524996], [-74.16094538402464, 40.641787239353285], [-74.16102490269706, 40.64179924136286], [-74.16111242522163, 40.64183545373729], [-74.16114204605088, 40.641911649849234], [-74.16117897104701, 40.642006612330825], [-74.161176743771, 40.642132057246144], [-74.16119293221192, 40.64225617213678], [-74.16120631312614, 40.64235864106052], [-74.16120187191912, 40.642663365028916], [-74.16123004765561, 40.6428116688183], [-74.16125491873757, 40.64294256044936], [-74.16134686994936, 40.64342646082162], [-74.16139280510836, 40.64366821292528], [-74.16143754262458, 40.64390369860129], [-74.16147514233892, 40.644101526465526], [-74.16146036002637, 40.644294969882374], [-74.15798759520317, 40.643861788983045], [-74.1579712715804, 40.643813021689596], [-74.15791348653059, 40.64374875882984], [-74.1578517562741, 40.64369662171829], [-74.15782088030684, 40.64366260786776], [-74.15777609272376, 40.64363087087955], [-74.15773024039142, 40.643573417844465], [-74.15762669194054, 40.643484246616204], [-74.1575055522123, 40.643379924476584], [-74.15743349200979, 40.643302857779], [-74.15755898947857, 40.6432347781657], [-74.15770553736039, 40.64321014591429], [-74.15782859631487, 40.643146392426516], [-74.15791589406754, 40.64308269445286], [-74.15799119129827, 40.64298873896387], [-74.15805052271504, 40.64286755565934], [-74.1580701364351, 40.64275853874279], [-74.1581057339996, 40.64268885333524], [-74.15813451603343, 40.642632530662986], [-74.1581989309435, 40.64260092422935], [-74.15827543717339, 40.64256338383835], [-74.15840347672427, 40.64254160388196], [-74.15848495839121, 40.642538466885725], [-74.15855043770534, 40.642497501306394], [-74.15857610598304, 40.642429351102614], [-74.15866125960514, 40.64230510545717], [-74.15869278737433, 40.64219001147032], [-74.15874378147612, 40.64200383984838], [-74.158810937453, 40.641758634740285], [-74.15920024534566, 40.6416504039774], [-74.15931925561976, 40.64155638657175], [-74.15945602438187, 40.641448333324036]]]]}}, {\"id\": \"156\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 157, \"Shape_Leng\": 0.131300217777, \"Shape_Area\": 0.000354370128323, \"zone\": \"Maspeth\", \"LocationID\": 157, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8978323689999, 40.73516035399992], [-73.89691187999993, 40.73508660199993], [-73.89651105199997, 40.73509674199992], [-73.89602931599993, 40.735017891999846], [-73.89510133499994, 40.73502118299989], [-73.89492421799993, 40.73500772199991], [-73.89474848099997, 40.73498281499997], [-73.89474827899986, 40.734982786999915], [-73.8947480899998, 40.73498275099996], [-73.89457557899993, 40.734946554999915], [-73.89457544899996, 40.7349465289999], [-73.89457531899991, 40.73494649199996], [-73.89440759699976, 40.73489932399987], [-73.89424644900001, 40.73484181299991], [-73.89415835499986, 40.734807791999884], [-73.89393780599978, 40.73472620999992], [-73.8937090569999, 40.7346574519999], [-73.89347408299997, 40.734602177999875], [-73.89323494499989, 40.7345608119999], [-73.89301088099992, 40.734523575999944], [-73.89278405699996, 40.73449596299988], [-73.89255554699984, 40.73447812599994], [-73.89232644100004, 40.73447014099991], [-73.8917501639998, 40.73448480999984], [-73.89138497899995, 40.734504988999895], [-73.89113852799989, 40.73451296999988], [-73.88993450699996, 40.734559955999906], [-73.88910318999984, 40.73459239599987], [-73.8885228599999, 40.734531159999904], [-73.88847908999993, 40.73415815399987], [-73.88844349199991, 40.73385475599986], [-73.88840065599999, 40.73344799199987], [-73.88837510099995, 40.73320535899991], [-73.8883492629999, 40.73294143199996], [-73.8883315609999, 40.73276054799993], [-73.88830747899983, 40.73251463099991], [-73.88828525999989, 40.732260539999864], [-73.88825349299994, 40.73189716199991], [-73.88817834099999, 40.73119478199992], [-73.88811223499987, 40.73059071099994], [-73.888059752, 40.730087229999896], [-73.88755962199991, 40.73019705599993], [-73.8874512889999, 40.729885994999904], [-73.88745122999997, 40.729885822999904], [-73.88745118199992, 40.72988564399985], [-73.88736094499983, 40.72957082699987], [-73.88736093299984, 40.72957078299989], [-73.887360922, 40.729570737999936], [-73.88728902599995, 40.72925284999991], [-73.88723573099988, 40.728932964999956], [-73.88720114099984, 40.728612069999926], [-73.88719765199994, 40.72818371199987], [-73.88720051699987, 40.72809239399991], [-73.88720430099983, 40.72801082999992], [-73.88720482599993, 40.72786882899991], [-73.88720524999994, 40.727792996999874], [-73.88720496699986, 40.727694363999916], [-73.88721164499994, 40.72761629699995], [-73.88722136099985, 40.727526533999864], [-73.88724113899983, 40.72740160899991], [-73.88720395699988, 40.72619323199994], [-73.8871325359999, 40.726142346999914], [-73.88709910799992, 40.72531867999989], [-73.8870771209998, 40.72463770199994], [-73.88897522300003, 40.724819902999876], [-73.89017449699986, 40.72494688899995], [-73.89030329399986, 40.724923886999875], [-73.891007685, 40.72461633099994], [-73.89181746299987, 40.724256213999865], [-73.89265591299983, 40.72388782499993], [-73.89411255499982, 40.72321425999989], [-73.89426494199992, 40.723143188999856], [-73.89451032599995, 40.72313194899989], [-73.89569743299992, 40.72341258599993], [-73.8958238069999, 40.72389882199994], [-73.8960873509999, 40.723811304999906], [-73.8983528509999, 40.72311228999994], [-73.89903191799989, 40.723856959999864], [-73.90010558699989, 40.72329624099992], [-73.899816102, 40.721236899999894], [-73.89739704899986, 40.7215512099999], [-73.89736598399989, 40.72046438299989], [-73.89774824399997, 40.72035734699994], [-73.89783709999988, 40.720470195999916], [-73.90009286399996, 40.720176852999934], [-73.90062644399997, 40.719764053999874], [-73.90033454499991, 40.71866913999984], [-73.90033904299986, 40.718019947999885], [-73.90005937999983, 40.71804637899991], [-73.89985149399986, 40.7183753979999], [-73.89834760099983, 40.719197677999894], [-73.897267579, 40.71814715699985], [-73.89882095599994, 40.71737222899991], [-73.89901821099994, 40.71738212699993], [-73.89820366399991, 40.71693000199986], [-73.89806016999988, 40.71686344399991], [-73.89805995699993, 40.71686334499989], [-73.89805976800002, 40.71686323599993], [-73.89792442199999, 40.71678547399989], [-73.89792430299995, 40.71678540199989], [-73.89792419799988, 40.71678533099995], [-73.89779899799984, 40.71669714999988], [-73.89779893899993, 40.71669710399988], [-73.89779888000004, 40.71669705899988], [-73.89768581399989, 40.71659970299993], [-73.89758660799987, 40.71649479699992], [-73.89758653699987, 40.716494714999904], [-73.89758646599992, 40.716494624999854], [-73.89750226399987, 40.716383867999916], [-73.89782401399992, 40.7162363809999], [-73.89860294399998, 40.71586712099985], [-73.89926420299997, 40.71554439099988], [-73.89989954399981, 40.71523272099996], [-73.90071775, 40.71483570899989], [-73.90115752599986, 40.714622314999886], [-73.902045084, 40.714197033999874], [-73.90243306099984, 40.714003411999904], [-73.90285443299987, 40.71379311999986], [-73.90380184499993, 40.71334837899985], [-73.90477244599997, 40.71286579999984], [-73.90489449600001, 40.712856814999924], [-73.90537582199983, 40.71287658099986], [-73.90556258999989, 40.71371463299989], [-73.90571002399983, 40.7143679749999], [-73.90574342600004, 40.71451601199994], [-73.90586816399987, 40.71506869299989], [-73.90602078299982, 40.7157426789999], [-73.90617600199982, 40.71642140399994], [-73.90639010099999, 40.71736369199989], [-73.90688152399996, 40.7173129149999], [-73.9069637079999, 40.717304416999866], [-73.90768387299997, 40.71737969499987], [-73.90850501099997, 40.71746899599985], [-73.90862256599989, 40.71748082299995], [-73.90943398699983, 40.717581772999885], [-73.90955515599987, 40.71769519699992], [-73.90962006699982, 40.71775153799996], [-73.90965980099996, 40.71779315499991], [-73.90980535299991, 40.717713699999926], [-73.91002374299995, 40.717588123999896], [-73.91048364399985, 40.71732796499988], [-73.91105946199994, 40.71684218099989], [-73.91134270799989, 40.716597027999974], [-73.9115182479999, 40.71641746599994], [-73.91156249200004, 40.71636931299988], [-73.9117893419999, 40.71612247799989], [-73.912108516, 40.71581262199985], [-73.91221070499992, 40.7157093569999], [-73.91232805399994, 40.71561530699993], [-73.91245827599991, 40.7155322899999], [-73.91283037699982, 40.71536090699993], [-73.91279689999989, 40.71534014299995], [-73.91274485099986, 40.71530376799991], [-73.91173862299988, 40.714723810999885], [-73.91159537799996, 40.71464113299989], [-73.91015817899998, 40.713821599999946], [-73.91069895799997, 40.713268520999875], [-73.91077807399988, 40.71318759999986], [-73.91303759100005, 40.71333207799989], [-73.91466958199995, 40.713437073999906], [-73.91521146700003, 40.71346123899988], [-73.91569881899996, 40.71348528899991], [-73.91633296499982, 40.71353115599987], [-73.91947977199997, 40.71373199799996], [-73.92083937599983, 40.713815139999966], [-73.92404011299993, 40.714008312999916], [-73.9240590971852, 40.71411155982241], [-73.92368352152519, 40.71408284140815], [-73.92374911534878, 40.714211276054876], [-73.9240522324595, 40.71487024514004], [-73.92412241575492, 40.71513804844516], [-73.92409931290926, 40.71518070819926], [-73.92406244021412, 40.715218064846695], [-73.92397375250823, 40.71529766471718], [-73.92389709645242, 40.71537152830813], [-73.92381513128568, 40.715441300341155], [-73.92376389043952, 40.715479717209874], [-73.92366895763148, 40.7155355684407], [-73.9235983444235, 40.71556125759519], [-73.92356011347093, 40.71557516241921], [-73.92354470735603, 40.71558267230683], [-73.92353323079922, 40.715590624560775], [-73.92352752100395, 40.71560508337099], [-73.92352747398412, 40.71560519089198], [-73.92352547251075, 40.71561660836783], [-73.92279196672435, 40.71580026374761], [-73.92232970167696, 40.716076545172385], [-73.92064258940593, 40.715773192028834], [-73.92056920092939, 40.71597116381606], [-73.92056821733091, 40.71597197396795], [-73.92056808910688, 40.71599107295813], [-73.92056905536089, 40.716003708340594], [-73.92057300316696, 40.71601877592619], [-73.92057861388705, 40.71602742469069], [-73.92060425100038, 40.716036007189885], [-73.92109482826223, 40.71614176633602], [-73.92127646277748, 40.716180378827495], [-73.92140714622649, 40.716209779971685], [-73.92170005987498, 40.71627377114592], [-73.92171847814689, 40.716277486351885], [-73.92216958810182, 40.71636848191687], [-73.9222376112434, 40.71638244049413], [-73.92228207681381, 40.71648327299211], [-73.92232773978448, 40.71659264406108], [-73.92153810999987, 40.71679515599987], [-73.91930947299991, 40.717417101999935], [-73.9180503519999, 40.71776542099987], [-73.91610398900005, 40.71829810299989], [-73.91516686599985, 40.71855456599992], [-73.914265253, 40.71881117399987], [-73.91339850999998, 40.719051086999876], [-73.9123211429999, 40.7193483859999], [-73.91152490699986, 40.71956777999985], [-73.91132095099978, 40.71962422999992], [-73.9118888929999, 40.720262830999886], [-73.91245967199984, 40.720930533999855], [-73.9129842039999, 40.721600847999845], [-73.91309781699987, 40.7217373449999], [-73.91309792299982, 40.72173746899996], [-73.91309801799984, 40.721737595999876], [-73.913199729, 40.72188004399986], [-73.91328886299985, 40.72202764999987], [-73.91331372599998, 40.7220773499999], [-73.9133647129999, 40.72217927199988], [-73.91336473699985, 40.72217933499993], [-73.9134268889998, 40.722333936999874], [-73.91355755899988, 40.72246288799988], [-73.91367234399998, 40.72260115799988], [-73.91376963799999, 40.722746977999854], [-73.91384828299984, 40.72289841199989], [-73.91316969799993, 40.72308152599989], [-73.91270773499983, 40.723284502999874], [-73.91257187699985, 40.723239634999864], [-73.91134506899984, 40.72418216999995], [-73.90889173800001, 40.726068424999916], [-73.90804680899986, 40.726733886999945], [-73.90740997399989, 40.72720873899992], [-73.90730013799994, 40.727294368999935], [-73.90716705199989, 40.72738865399988], [-73.90712253099991, 40.727424727999924], [-73.90706867299988, 40.7274665229999], [-73.90693049899986, 40.72757602999992], [-73.90673028099992, 40.72773268299985], [-73.90573360899982, 40.7285107579999], [-73.90557740099993, 40.72863668399986], [-73.905180579, 40.72895657699995], [-73.904218336, 40.730125830999945], [-73.9041685329999, 40.73018659099991], [-73.9025965079999, 40.73210442799992], [-73.90107970900003, 40.73391463399987], [-73.90091094899987, 40.7341260369999], [-73.9007113889999, 40.73435872699986], [-73.90062897999994, 40.734463231999946], [-73.90062893299998, 40.73446328599989], [-73.90062887299989, 40.73446333999988], [-73.90053014999994, 40.73456130199992], [-73.90053009099984, 40.73456135699991], [-73.90041661699985, 40.73465058799992], [-73.90041646300001, 40.73465070499992], [-73.90041629699991, 40.734650803999884], [-73.90029014899983, 40.734729471999906], [-73.90029003099994, 40.734729542999915], [-73.90028989999986, 40.73472960699994], [-73.90015343399983, 40.73479640499991], [-73.90000969399993, 40.73485031999996], [-73.89982513099983, 40.734931394999926], [-73.89963257999993, 40.73500235799992], [-73.89943348500005, 40.73506258999993], [-73.89922944799997, 40.73511159899988], [-73.89902209999995, 40.73514908399993], [-73.89878646099986, 40.73517679299992], [-73.89878634299994, 40.73517681099992], [-73.89878621299994, 40.73517681999992], [-73.89854787099992, 40.7351919009999], [-73.89854772799987, 40.735191909999855], [-73.8985475869999, 40.735191909999934], [-73.89830812699986, 40.73519418399992], [-73.89830799699995, 40.735194183999965], [-73.89830786699996, 40.73519417399996], [-73.89806910099988, 40.735183616999954], [-73.89806895999997, 40.73518360799988], [-73.89806882999987, 40.73518359899987], [-73.8978323689999, 40.73516035399992]]]}}, {\"id\": \"157\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 158, \"Shape_Leng\": 0.0548099905515, \"Shape_Area\": 0.000185568253002, \"zone\": \"Meatpacking/West Village West\", \"LocationID\": 158, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.0077911609999, 40.74197084499982], [-74.00697077799991, 40.741627126999916], [-74.00536591799992, 40.74094529499997], [-74.00515808199998, 40.74085808099991], [-74.00529178799992, 40.74002322299992], [-74.00539035999991, 40.73941197299994], [-74.00549494099988, 40.73875392299988], [-74.00559149599997, 40.73813833999991], [-74.0056870449999, 40.737551878999916], [-74.00579620299999, 40.7369216979999], [-74.00585387099987, 40.73653322599989], [-74.00596018000002, 40.73584500699992], [-74.00606869899997, 40.735133213999916], [-74.00618179699998, 40.734439131999885], [-74.0062942419999, 40.73374204199993], [-74.00640707599995, 40.73305393999986], [-74.00650135299995, 40.73242587499988], [-74.00662407199995, 40.73166937899992], [-74.00673713399992, 40.73094706599989], [-74.00685682099994, 40.73022823599989], [-74.00697298799992, 40.729494989999864], [-74.0070902079999, 40.72877225299993], [-74.00859272099996, 40.728915276999864], [-74.00934961999998, 40.728987555999865], [-74.01054441299988, 40.7291000979999], [-74.01067264999996, 40.729112666999896], [-74.01076791499999, 40.72817395799991], [-74.01087083299997, 40.72818049399996], [-74.01100472999994, 40.7281932389999], [-74.01138323602366, 40.728229272183555], [-74.014390699162, 40.728463047760016], [-74.01434461474159, 40.728622183781454], [-74.01421928543024, 40.7295234277556], [-74.0140774139113, 40.73064040918783], [-74.01408693559922, 40.730663173040284], [-74.01403796924805, 40.73069421884391], [-74.01401552295421, 40.73068283965609], [-74.01116195696464, 40.730450676349065], [-74.01108642940412, 40.73128902958222], [-74.01099350879154, 40.732320417940606], [-74.01094127374121, 40.73302031334127], [-74.01400220214602, 40.73306800179543], [-74.01399957240955, 40.733332277517704], [-74.01090389883275, 40.733297617139584], [-74.01089482435798, 40.73334817096212], [-74.0108673655772, 40.73334883439819], [-74.01084045026995, 40.73335302243394], [-74.01081485457662, 40.73336061430153], [-74.01079131658479, 40.73337139107892], [-74.01077051504689, 40.7333850420029], [-74.01075304980593, 40.733401173429414], [-74.01073942450006, 40.733419320185995], [-74.01073003203787, 40.73343895898451], [-74.01072514326883, 40.733459523511115], [-74.01072489917198, 40.73348042075681], [-74.01072930679061, 40.73350104811758], [-74.01073823902861, 40.733520810771275], [-74.01075143831419, 40.73353913883061], [-74.01076852402858, 40.73355550377658], [-74.0107890034796, 40.73356943369936], [-74.01081228611086, 40.733580526906486], [-74.01083770052972, 40.733588463507175], [-74.0108645138697, 40.73359301463594], [-74.01083284563217, 40.73400709451363], [-74.01201792010478, 40.734063744456314], [-74.01200124222409, 40.734338401668715], [-74.01081359479103, 40.734283443647534], [-74.01078798331716, 40.73454829384434], [-74.01070922910374, 40.73536287214014], [-74.01064292017875, 40.73604869770492], [-74.01067724659347, 40.73605627995678], [-74.01066986751215, 40.73610193192912], [-74.01068752603774, 40.73611386772274], [-74.01070404345339, 40.73612918443091], [-74.01071702975825, 40.73614635650884], [-74.01072613231958, 40.736164917666336], [-74.01073110396315, 40.73618436389367], [-74.01073180968515, 40.73620416714772], [-74.0107282303185, 40.73622378968969], [-74.01072046305407, 40.736242698687995], [-74.01070871880131, 40.73626038068563], [-74.01069377906512, 40.7362759493593], [-74.01067576035113, 40.73628950181188], [-74.01065512584192, 40.7363006896678], [-74.01065421033896, 40.73634774524587], [-74.01061910077436, 40.736345390721766], [-74.01056810573431, 40.736972220142476], [-74.01051581576753, 40.737596064953664], [-74.0104765981442, 40.73806394059504], [-74.01108706635938, 40.738092750081364], [-74.01106791934316, 40.73830409617404], [-74.01108874586753, 40.73831752469385], [-74.01110665713956, 40.73833319830264], [-74.01112123961192, 40.738350755119924], [-74.01113215659413, 40.73836978978442], [-74.01113915602558, 40.73838986281303], [-74.0111180025413, 40.7383926960924], [-74.01109653938134, 40.73839332225606], [-74.01107516272494, 40.73839172974585], [-74.01105426715444, 40.738387947957264], [-74.01105126196637, 40.738396425750636], [-74.01045432864609, 40.73836450622813], [-74.01042867884628, 40.738703364658704], [-74.0103930313456, 40.739174296059204], [-74.01049858988287, 40.7391734327704], [-74.01055473996806, 40.739186239049005], [-74.01074676135308, 40.73917593667961], [-74.01138798406548, 40.739216056088566], [-74.01148456734634, 40.739260462594075], [-74.01158045502316, 40.739264536112046], [-74.01158818724917, 40.73927159590028], [-74.01158715816628, 40.73947887033473], [-74.01163351895192, 40.73950648606618], [-74.01162792311773, 40.73956247410615], [-74.01155518156293, 40.7395567262273], [-74.0115274502041, 40.7398737714499], [-74.01159286133142, 40.73987376488411], [-74.01161258017336, 40.73970760054936], [-74.01234819241687, 40.739737565353636], [-74.0123691740004, 40.73975164637212], [-74.01237411577733, 40.73977886786768], [-74.01234943428379, 40.739793890761796], [-74.01228649627012, 40.73979296102123], [-74.01228897847172, 40.73989904212453], [-74.01231366348769, 40.739902793753366], [-74.01234328917008, 40.739920629792294], [-74.0123568713412, 40.73993658471745], [-74.01234947552231, 40.740016379815906], [-74.01232356118135, 40.740031403003705], [-74.01227049910142, 40.740034226965065], [-74.01226310747168, 40.74014218976771], [-74.01230877290088, 40.74014781277593], [-74.01232852356425, 40.74016283157744], [-74.01232975860133, 40.74018348844809], [-74.01231001212764, 40.74019475620399], [-74.01157686924239, 40.74016009950159], [-74.01158637887598, 40.739973847952534], [-74.01151491736529, 40.73997286369881], [-74.01146073955334, 40.74058421475249], [-74.01215741913083, 40.74060599813125], [-74.01215372624051, 40.740674256620736], [-74.01265742489983, 40.74069665223182], [-74.01265928208485, 40.74074763849748], [-74.0125737299691, 40.74074173184487], [-74.01257372874738, 40.74073525812516], [-74.01147486918853, 40.74068706027102], [-74.01137438820483, 40.740723947641335], [-74.01071840609639, 40.74068559392549], [-74.00963145571744, 40.74064536737427], [-74.00949685520366, 40.74072701178543], [-74.00949154621287, 40.74074270943978], [-74.0095162735821, 40.74077358367349], [-74.00951844195929, 40.740805722916896], [-74.00950137905853, 40.740829223745706], [-74.00946293860208, 40.74090233724026], [-74.00941140266131, 40.741154093312815], [-74.00958131822176, 40.74117496028477], [-74.0095769279162, 40.7412106566544], [-74.01216232909212, 40.74149188968265], [-74.01211745872139, 40.74175851621157], [-74.00951947145005, 40.741490302198606], [-74.00944784072523, 40.74189159122011], [-74.00927792756069, 40.741878610856126], [-74.00916608258481, 40.742416591231674], [-74.00902439199994, 40.742399742999964], [-74.008893399, 40.742384166999884], [-74.00879080599995, 40.742375143999915], [-74.00869902299993, 40.742361722999924], [-74.00820401899988, 40.74214751799986], [-74.0077911609999, 40.74197084499982]]]}}, {\"id\": \"158\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 159, \"Shape_Leng\": 0.064363233206, \"Shape_Area\": 0.00017124910387, \"zone\": \"Melrose South\", \"LocationID\": 159, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91189154999992, 40.822141481999914], [-73.91181615799985, 40.82203502199991], [-73.91180780599983, 40.822039769999925], [-73.91179935999997, 40.8220444549999], [-73.91179083099985, 40.82204905899985], [-73.911782208, 40.822053599999876], [-73.91177350099998, 40.8220580509999], [-73.91176469899987, 40.82206243799991], [-73.91175582700001, 40.82206673699992], [-73.91174685999992, 40.82207096199993], [-73.91173782099985, 40.82207510599991], [-73.9117286999999, 40.82207916899994], [-73.91171949699986, 40.8220831519999], [-73.91171023399987, 40.82208704299993], [-73.9117008879999, 40.82209084599988], [-73.91169147099988, 40.822094565999876], [-73.91168198299992, 40.82209820599994], [-73.91167243599985, 40.8221017459999], [-73.91166281800001, 40.82210519699987], [-73.91165313999997, 40.82210855799985], [-73.91164340399992, 40.8221118279999], [-73.91163361999985, 40.822115008999894], [-73.91162376599982, 40.82211808899989], [-73.91161386399986, 40.82212107999989], [-73.91160391499992, 40.82212397199989], [-73.91159391799995, 40.82212677499996], [-73.9115838739999, 40.82212947799995], [-73.91157378299995, 40.822132080999914], [-73.91156365699992, 40.822134584999894], [-73.91155348299996, 40.82213699099988], [-73.91154328499991, 40.82213929699984], [-73.91153305199994, 40.82214150499988], [-73.91152278399981, 40.82214361299986], [-73.91151249299988, 40.8221456219999], [-73.91150216499999, 40.822147531999875], [-73.91149182600002, 40.822149333999896], [-73.91148146299997, 40.822151046999885], [-73.91147107699982, 40.82215264999989], [-73.91146067899994, 40.82215415499997], [-73.91145026999986, 40.82215556099989], [-73.9114398599999, 40.82215685899992], [-73.91142942699986, 40.82215805799994], [-73.911418994, 40.822159156999845], [-73.91140856199999, 40.82216015799987], [-73.9113981169998, 40.82216105899986], [-73.91138768499987, 40.82216185299992], [-73.91137726399995, 40.822162555999945], [-73.91136683199993, 40.82216315199986], [-73.91135642300002, 40.82216364799994], [-73.91107298599978, 40.82224294299992], [-73.91069998799998, 40.8223241139999], [-73.90990894100004, 40.822551107999914], [-73.90945760899984, 40.82243240199992], [-73.90841854199991, 40.82218127099991], [-73.90742681699992, 40.82194506199989], [-73.90645722199996, 40.821715363999886], [-73.90565890299979, 40.821556532999864], [-73.90552466299984, 40.82152381299991], [-73.90545592300005, 40.821507058999906], [-73.90532770299998, 40.82147580599994], [-73.90525022600004, 40.82145692199989], [-73.90489653200002, 40.82137070999988], [-73.90478103799988, 40.82134144499991], [-73.90379310199981, 40.821091099999876], [-73.90266819199994, 40.82080315899992], [-73.9012927759998, 40.820475442999914], [-73.90160121199982, 40.81969485899989], [-73.90166319399991, 40.81955042599993], [-73.90210743299994, 40.81850519199991], [-73.90257859099984, 40.81736979399992], [-73.90300983199995, 40.81631779199992], [-73.90306443799986, 40.8161067619999], [-73.9034699719999, 40.81507578799996], [-73.90380412299992, 40.81413184199994], [-73.90418337799991, 40.81308645499989], [-73.9044655149999, 40.81228195999994], [-73.9054135989999, 40.812476195999906], [-73.90632991800003, 40.812667073999876], [-73.90724503699998, 40.81285516299991], [-73.90815917699986, 40.813043575999906], [-73.90907451799984, 40.81323307299991], [-73.90949684799988, 40.813322147999926], [-73.9095236779999, 40.813327805999975], [-73.91001840399991, 40.81343214799989], [-73.91013342199993, 40.81345551199997], [-73.91029615499986, 40.81348856499989], [-73.91033664499996, 40.81349678899988], [-73.91038742199996, 40.8135071039999], [-73.91043307699985, 40.81351637699987], [-73.91094779, 40.81362092099986], [-73.91181210799998, 40.813883314999885], [-73.91295450799997, 40.81426933499991], [-73.9149549369999, 40.81493646599995], [-73.91677117699996, 40.81569882899991], [-73.91768626999985, 40.8160812429999], [-73.91861739800005, 40.815523135999854], [-73.91946744899994, 40.81507370999987], [-73.92034892799985, 40.81462886399989], [-73.92048117399989, 40.8145636869999], [-73.9213038339999, 40.81547724799986], [-73.92158392200005, 40.815550631999955], [-73.92124519699988, 40.816219396999934], [-73.92309299899993, 40.81669772799995], [-73.92372701399987, 40.81581731999991], [-73.92376838899996, 40.81575986499996], [-73.92390804799993, 40.815565926999945], [-73.92395483099983, 40.8155009609999], [-73.92399951299991, 40.81543891199991], [-73.92480953899991, 40.81565901999993], [-73.92505519900006, 40.81572157599992], [-73.92550133099999, 40.81583301299993], [-73.92568897199988, 40.81587494499995], [-73.92662718199988, 40.816118958999915], [-73.92621971999989, 40.816758655999905], [-73.92518486499998, 40.818012668999884], [-73.92454726099986, 40.81871262399987], [-73.92402557199983, 40.8192853299999], [-73.92265261199995, 40.82049431499993], [-73.92194764099997, 40.82103154499987], [-73.92114408999996, 40.82156300199989], [-73.92066539199996, 40.82183689399995], [-73.92054346999996, 40.821933856999856], [-73.92048603299985, 40.82197953699992], [-73.92038676299994, 40.822058483999875], [-73.91993471699983, 40.82235220199986], [-73.91980424599993, 40.822436974999896], [-73.91944303599995, 40.82267166799987], [-73.91938041799983, 40.82271235199997], [-73.91929142000006, 40.82277017699986], [-73.91894597199995, 40.8229926729999], [-73.91888836899983, 40.82302977399992], [-73.91875570599991, 40.8231152179999], [-73.91833693999992, 40.82338493199989], [-73.91788272099983, 40.823678776999884], [-73.91782151699984, 40.82371837099987], [-73.9175659939999, 40.823883671999944], [-73.91746772799986, 40.82394610699989], [-73.9174045909999, 40.82398622199988], [-73.9173178239999, 40.82404134999992], [-73.91725010299993, 40.82408437699994], [-73.91719007399993, 40.824122516999914], [-73.917126938, 40.82416263099988], [-73.9168919579999, 40.8243119259999], [-73.91677400599986, 40.82438708699987], [-73.91587735799988, 40.824103731999855], [-73.91621588599988, 40.82346792999991], [-73.91450553699998, 40.82294185699991], [-73.91287633399998, 40.822441347999884], [-73.91189154999992, 40.822141481999914]]]}}, {\"id\": \"159\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 160, \"Shape_Leng\": 0.14151914297, \"Shape_Area\": 0.000571234977994, \"zone\": \"Middle Village\", \"LocationID\": 160, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.869707654, 40.72514643399992], [-73.86931025899987, 40.72446599099993], [-73.86929270499994, 40.72443592899991], [-73.8692721569999, 40.72441183599992], [-73.8687681729999, 40.723820906999904], [-73.86845305199996, 40.723485493999874], [-73.86821068499994, 40.72325474899987], [-73.86765927399983, 40.72266125599994], [-73.86730908299992, 40.72228902499991], [-73.86711404599988, 40.72207640399993], [-73.86657241099984, 40.721486636999884], [-73.86604176200004, 40.72090092299985], [-73.86547610900004, 40.72028093699992], [-73.86523870699996, 40.719976709999905], [-73.86468729499988, 40.719427075999874], [-73.86412297599996, 40.718844806999925], [-73.8635711119999, 40.71827667999986], [-73.86301850099984, 40.717707881999964], [-73.86272195199989, 40.71742801899989], [-73.86242540499987, 40.717148146999854], [-73.86213638199993, 40.71687677399989], [-73.86183368299996, 40.716592542999884], [-73.861528867, 40.716314936999915], [-73.86122406399997, 40.7160373289999], [-73.86062663399984, 40.71548716899991], [-73.86003789899996, 40.71493223599987], [-73.85992523600002, 40.714749735999945], [-73.85982746199984, 40.714561688999915], [-73.8597452479999, 40.71436917699986], [-73.85967906599994, 40.71417332399993], [-73.85962920999985, 40.713975294999926], [-73.8596317549999, 40.71359826399985], [-73.85965416799995, 40.713379960999866], [-73.85978566499996, 40.71331129599985], [-73.86207872699998, 40.71220105999993], [-73.86500981999998, 40.710776908999875], [-73.86557714699987, 40.71065921899992], [-73.86661823199995, 40.71040164899995], [-73.87016462499984, 40.70920774299987], [-73.86989582499986, 40.7085070939999], [-73.86949738599989, 40.70787206999989], [-73.86911007199988, 40.707089336999914], [-73.87153400399993, 40.70703676799997], [-73.87326053099987, 40.70692273699991], [-73.87425518099995, 40.706857030999906], [-73.87482359599984, 40.70681947799994], [-73.87532966799982, 40.70679838799992], [-73.87625034900005, 40.706744642999894], [-73.87720614800003, 40.70668883999991], [-73.87814452399996, 40.7066096259999], [-73.87900684300003, 40.706536838999924], [-73.88000104599993, 40.706492933999876], [-73.88088332699985, 40.7064539769999], [-73.88086188499989, 40.7065847289999], [-73.87761134199985, 40.70716770499993], [-73.87872783199988, 40.71002753499987], [-73.87926181200001, 40.70990534699992], [-73.87943495499985, 40.71028441899988], [-73.87907685299999, 40.71035938999992], [-73.88006267699978, 40.71265257399993], [-73.88205387399988, 40.712454149999864], [-73.88435098699988, 40.712230172999966], [-73.88507555699987, 40.71216105599992], [-73.88519763099994, 40.71216627399993], [-73.88622785599989, 40.7122023819999], [-73.88637649699994, 40.71220771499993], [-73.88687942999985, 40.71222318499987], [-73.88842000699994, 40.71226995799984], [-73.88877304399992, 40.71228017799994], [-73.89367393299992, 40.71243002499995], [-73.89401868699997, 40.7124405549999], [-73.89502461000001, 40.71247825299989], [-73.8959038609999, 40.71250695899986], [-73.89621914099989, 40.712522056999944], [-73.89710338699993, 40.71255773699993], [-73.89768322699987, 40.71258029799992], [-73.89935504399993, 40.71264701399986], [-73.90018380399995, 40.71268065799996], [-73.900595879, 40.712697384999906], [-73.90224267799981, 40.712759832999886], [-73.90293472899992, 40.71278607299988], [-73.9037694399998, 40.712817031999904], [-73.90477244599997, 40.71286579999984], [-73.90380184499993, 40.71334837899985], [-73.90285443299987, 40.71379311999986], [-73.90243306099984, 40.714003411999904], [-73.902045084, 40.714197033999874], [-73.90115752599986, 40.714622314999886], [-73.89989954399981, 40.71523272099996], [-73.89926420299997, 40.71554439099988], [-73.89860294399998, 40.71586712099985], [-73.89782401399992, 40.7162363809999], [-73.89750226399987, 40.716383867999916], [-73.89758646599992, 40.716494624999854], [-73.89758653699987, 40.716494714999904], [-73.89758660799987, 40.71649479699992], [-73.89768581399989, 40.71659970299993], [-73.89779888000004, 40.71669705899988], [-73.89779893899993, 40.71669710399988], [-73.89779899799984, 40.71669714999988], [-73.89792419799988, 40.71678533099995], [-73.89792430299995, 40.71678540199989], [-73.89792442199999, 40.71678547399989], [-73.89805976800002, 40.71686323599993], [-73.89805995699993, 40.71686334499989], [-73.89806016999988, 40.71686344399991], [-73.89820366399991, 40.71693000199986], [-73.89901821099994, 40.71738212699993], [-73.89882095599994, 40.71737222899991], [-73.897267579, 40.71814715699985], [-73.89834760099983, 40.719197677999894], [-73.89985149399986, 40.7183753979999], [-73.90005937999983, 40.71804637899991], [-73.90033904299986, 40.718019947999885], [-73.90033454499991, 40.71866913999984], [-73.90062644399997, 40.719764053999874], [-73.90009286399996, 40.720176852999934], [-73.89783709999988, 40.720470195999916], [-73.89774824399997, 40.72035734699994], [-73.89736598399989, 40.72046438299989], [-73.89739704899986, 40.7215512099999], [-73.899816102, 40.721236899999894], [-73.90010558699989, 40.72329624099992], [-73.89903191799989, 40.723856959999864], [-73.8983528509999, 40.72311228999994], [-73.8960873509999, 40.723811304999906], [-73.8958238069999, 40.72389882199994], [-73.89569743299992, 40.72341258599993], [-73.89451032599995, 40.72313194899989], [-73.89426494199992, 40.723143188999856], [-73.89411255499982, 40.72321425999989], [-73.89265591299983, 40.72388782499993], [-73.89181746299987, 40.724256213999865], [-73.891007685, 40.72461633099994], [-73.89030329399986, 40.724923886999875], [-73.89017449699986, 40.72494688899995], [-73.88897522300003, 40.724819902999876], [-73.8870771209998, 40.72463770199994], [-73.88709910799992, 40.72531867999989], [-73.8871325359999, 40.726142346999914], [-73.88720395699988, 40.72619323199994], [-73.88724113899983, 40.72740160899991], [-73.88722136099985, 40.727526533999864], [-73.88721164499994, 40.72761629699995], [-73.88720496699986, 40.727694363999916], [-73.88720524999994, 40.727792996999874], [-73.88689134399985, 40.727917365999915], [-73.8867298489999, 40.72797312799988], [-73.88661323299988, 40.72801339299988], [-73.8865683549999, 40.72802889099993], [-73.88623765899992, 40.72812701699997], [-73.88590069999995, 40.728211322999904], [-73.88559330999979, 40.728274440999954], [-73.88500729499987, 40.7283718949999], [-73.88444917100003, 40.72841912699995], [-73.88391980299994, 40.72846391299988], [-73.88387642499987, 40.7284675829999], [-73.8835332119999, 40.72849744799988], [-73.88297543399985, 40.728540950999914], [-73.88208519900004, 40.728610374999924], [-73.88172301799989, 40.72863525399986], [-73.88130917699985, 40.72868082699991], [-73.88089793699997, 40.728741311999876], [-73.88088484699995, 40.728743729999906], [-73.88049068199993, 40.72881654199995], [-73.88008877499988, 40.728906274999936], [-73.87969354099992, 40.729010158999884], [-73.87847478499992, 40.72946892199991], [-73.87836629199992, 40.72951588699991], [-73.87789402699987, 40.729715930999944], [-73.87699724199997, 40.73004835299989], [-73.87628302399997, 40.73031309399994], [-73.8753864729999, 40.73059067299993], [-73.87509154199992, 40.73067136299992], [-73.8749534949998, 40.73061869599996], [-73.87475475600003, 40.73054189099991], [-73.87408913199982, 40.73028279199987], [-73.87316942599995, 40.72995061899995], [-73.87235229600002, 40.72965548399991], [-73.87154749999993, 40.72936238299992], [-73.87146074199987, 40.729326373999854], [-73.87140980799985, 40.729178101999935], [-73.8711328349999, 40.7283717199999], [-73.87100156100003, 40.727924523999896], [-73.87093081299994, 40.72770320699995], [-73.87084539399993, 40.72748459299991], [-73.87074560999994, 40.72726953899994], [-73.87063185099991, 40.7270588539999], [-73.87021135899995, 40.726187325999916], [-73.869707654, 40.72514643399992]]]}}, {\"id\": \"160\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 161, \"Shape_Leng\": 0.0358039100611, \"Shape_Area\": 7.19130682992e-05, \"zone\": \"Midtown Center\", \"LocationID\": 161, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97534377999993, 40.7552996949999], [-73.97580739399986, 40.754677274999935], [-73.97589877700003, 40.75471422299989], [-73.9765156939999, 40.75497426299993], [-73.97697379099985, 40.75434867599992], [-73.97743259200001, 40.75371708799987], [-73.9778749419999, 40.75309839199991], [-73.9783696929999, 40.752416226999905], [-73.97928686199992, 40.752800496999946], [-73.98088709899987, 40.753480988999875], [-73.98411754799999, 40.75484205299995], [-73.98362433299988, 40.75551634599986], [-73.98316796099988, 40.75614022199993], [-73.9827120529998, 40.75677122799991], [-73.98225528700006, 40.75738736499995], [-73.98179404299992, 40.75802130299995], [-73.98134107099996, 40.75864529799986], [-73.98088574599987, 40.75927066999991], [-73.98042606399994, 40.75989908099994], [-73.97997188199987, 40.760524700999866], [-73.97951319199996, 40.7611485619999], [-73.97905731199987, 40.761770719999944], [-73.97860169399989, 40.762396071999945], [-73.97813931699997, 40.76302401399991], [-73.977686006, 40.76364440299992], [-73.97445730599989, 40.76229308399991], [-73.97285250499992, 40.76161435699986], [-73.97133777299992, 40.760977297999894], [-73.97124277300004, 40.76093641799984], [-73.97169609199985, 40.76031266099992], [-73.97215595499983, 40.759681239999985], [-73.9726127139999, 40.759059307999955], [-73.97306851999987, 40.758431493999915], [-73.97352419599999, 40.75780843999997], [-73.97398183199977, 40.75717975799992], [-73.97443866399978, 40.75655630599991], [-73.9748872719999, 40.7559237909999], [-73.97534377999993, 40.7552996949999]]]}}, {\"id\": \"161\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 162, \"Shape_Leng\": 0.0352698146219, \"Shape_Area\": 4.78936962112e-05, \"zone\": \"Midtown East\", \"LocationID\": 162, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9712170449999, 40.75519332399994], [-73.97167699199998, 40.754569630999946], [-73.97213617399994, 40.7539432349999], [-73.97258874999983, 40.753317887999856], [-73.97304502599985, 40.7526916219999], [-73.97350255899984, 40.75206493199988], [-73.97395914699996, 40.751441020999934], [-73.9744494109998, 40.75076644499991], [-73.97604824899987, 40.75144139299994], [-73.97685152399993, 40.75177649399989], [-73.97710781099983, 40.7518834059999], [-73.97754284999984, 40.75206355599991], [-73.97767660499981, 40.7521189369999], [-73.97779152099986, 40.7521675949999], [-73.9783696929999, 40.752416226999905], [-73.9778749419999, 40.75309839199991], [-73.97743259200001, 40.75371708799987], [-73.97697379000006, 40.754348675999886], [-73.9765156939999, 40.75497426299993], [-73.97589877700003, 40.75471422299989], [-73.97580739399984, 40.75467727399997], [-73.97534377999993, 40.7552996949999], [-73.9748872719999, 40.7559237909999], [-73.97443866299996, 40.756556305999894], [-73.97398183199977, 40.75717975799992], [-73.97352419599999, 40.75780843999997], [-73.97306851999987, 40.758431493999915], [-73.9726127139999, 40.759059307999955], [-73.97215595499983, 40.759681239999985], [-73.97169609199985, 40.76031266099992], [-73.97124277300004, 40.76093641799984], [-73.97075063899999, 40.761608701999926], [-73.97025814699983, 40.762288752999886], [-73.96980028899992, 40.76291351199995], [-73.96968801199992, 40.76286596199986], [-73.96818687799986, 40.76222682099993], [-73.9665833839999, 40.76155093499988], [-73.96657146999985, 40.761481879999955], [-73.96699471899983, 40.76090538299993], [-73.96746215199992, 40.76022290599993], [-73.96792193499995, 40.75959308699989], [-73.96802436899998, 40.759578139999896], [-73.96847979599983, 40.75895012799986], [-73.96893510299992, 40.75833009199993], [-73.96939411299984, 40.75769952799991], [-73.96984859899989, 40.75707595799985], [-73.97031393599994, 40.75645301899989], [-73.97075919399998, 40.7558248509999], [-73.9712170449999, 40.75519332399994]]]}}, {\"id\": \"162\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 163, \"Shape_Leng\": 0.0341768669752, \"Shape_Area\": 4.08567804196e-05, \"zone\": \"Midtown North\", \"LocationID\": 163, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98136213699995, 40.767836738999904], [-73.97907658799991, 40.76687608599991], [-73.97836113000002, 40.766573710999886], [-73.97623388799995, 40.76567463099986], [-73.9741922899999, 40.76480652799991], [-73.9736358469999, 40.7645699129999], [-73.97301487199995, 40.76427887899991], [-73.97141141699996, 40.76358747399994], [-73.96990489299992, 40.762958042999834], [-73.96980028899992, 40.76291351199995], [-73.97025814699983, 40.762288752999886], [-73.97075063899999, 40.761608701999926], [-73.97124277300004, 40.76093641799984], [-73.97133777299992, 40.760977297999894], [-73.97285250499992, 40.76161435699986], [-73.97445730599989, 40.76229308399991], [-73.977686006, 40.76364440299992], [-73.97813931699997, 40.76302401399991], [-73.97860169399989, 40.762396071999945], [-73.98143351999995, 40.7635943599999], [-73.9824697079998, 40.764033463999944], [-73.98428230099987, 40.76479181499994], [-73.98382185700001, 40.7654184749999], [-73.98336657899995, 40.766045241999926], [-73.9828771349999, 40.766715594999866], [-73.98236545099999, 40.7673921529999], [-73.98215157200003, 40.76768179699992], [-73.98205899699985, 40.76764257499984], [-73.9820588189999, 40.767642502999934], [-73.98205860599991, 40.76764244899991], [-73.98195445399982, 40.76761766099985], [-73.981954182, 40.76761759699986], [-73.98195388499981, 40.76761757899992], [-73.98184408100003, 40.767609708999885], [-73.98184380799981, 40.7676096909999], [-73.98184353599999, 40.767609717999925], [-73.98173502899988, 40.76761929099996], [-73.98173489899985, 40.76761929999993], [-73.98173478100003, 40.76761932799989], [-73.98163412099993, 40.76764472399989], [-73.98155301499993, 40.76768021699989], [-73.98147898799989, 40.7677250589999], [-73.98141469499981, 40.767777872999865], [-73.98136213699995, 40.767836738999904]]]}}, {\"id\": \"163\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 164, \"Shape_Leng\": 0.0357716119908, \"Shape_Area\": 5.56318237441e-05, \"zone\": \"Midtown South\", \"LocationID\": 164, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98362519999995, 40.7439431149999], [-73.98407485299995, 40.74332471799989], [-73.98413729099997, 40.74334959099992], [-73.98569571299991, 40.74400764099993], [-73.98729805500001, 40.74468149799989], [-73.98872905499988, 40.74528637399991], [-73.99051761299985, 40.746038637999895], [-73.99006749199991, 40.74665928299993], [-73.98961683100002, 40.747276422999875], [-73.98916651699989, 40.747895434999904], [-73.9887140509998, 40.74851327399993], [-73.98823761499983, 40.749130168999926], [-73.98797357699985, 40.74949077899986], [-73.98776905899994, 40.749787028999926], [-73.98729377099981, 40.75045160899988], [-73.986849911, 40.751081678999874], [-73.98639930899988, 40.751699985999934], [-73.98594653099995, 40.752316804999886], [-73.98548819099985, 40.752935179999945], [-73.98504363799995, 40.753554148999854], [-73.98459584300001, 40.754176941999916], [-73.98411754799999, 40.75484205299995], [-73.98088709899987, 40.753480988999875], [-73.9813714319999, 40.752813855999925], [-73.98182244299984, 40.75219712699987], [-73.98227490399982, 40.751577297999845], [-73.9827238749999, 40.750958943999926], [-73.9831741629998, 40.75034220999989], [-73.983621411, 40.749723770999914], [-73.98407636100002, 40.74910212499993], [-73.98247211999985, 40.74843150099986], [-73.98094249999997, 40.74779007099992], [-73.98085054499998, 40.74775036999995], [-73.98134068699997, 40.74707866399992], [-73.98177455300004, 40.74649679899989], [-73.98183365099989, 40.746416590999935], [-73.98227872699987, 40.74579756299986], [-73.98233039299987, 40.74572641199994], [-73.98272686499999, 40.74518041399988], [-73.98317411899995, 40.74456031099989], [-73.98362519999995, 40.7439431149999]]]}}, {\"id\": \"164\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 165, \"Shape_Leng\": 0.088608036213, \"Shape_Area\": 0.000353959317019, \"zone\": \"Midwood\", \"LocationID\": 165, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96014773499996, 40.628915184999876], [-73.95995565099992, 40.627778671999884], [-73.95899416299996, 40.62788559399996], [-73.95805293100001, 40.627971118999895], [-73.95684697299988, 40.62811987899991], [-73.95581207199996, 40.62823349199987], [-73.95488499199999, 40.62833554799986], [-73.95395983399986, 40.62843938999987], [-73.95303133199995, 40.62854011499994], [-73.9520690179999, 40.628645780999875], [-73.95110641899993, 40.62875046699989], [-73.95017656999988, 40.6288527719999], [-73.94925229799985, 40.628954764999925], [-73.94832386299989, 40.62905666099993], [-73.94732672199994, 40.629166566999835], [-73.94687439899997, 40.62677366799994], [-73.94642266100001, 40.6243855669999], [-73.94542442899984, 40.624495609999926], [-73.9449703129999, 40.62210451199989], [-73.94451833400001, 40.61971364699989], [-73.94551663599992, 40.61960561199991], [-73.94651373199997, 40.61949401899995], [-73.94606699600003, 40.617103738999894], [-73.94576359499983, 40.61550406399987], [-73.94573691999982, 40.61536340699989], [-73.94657366799994, 40.61478913099987], [-73.94742608799993, 40.61422460099989], [-73.94772682600002, 40.61416971599985], [-73.94834517399988, 40.61413649599991], [-73.94930527399987, 40.614030734999886], [-73.95026514899993, 40.61392435199983], [-73.94999598099999, 40.612491460999884], [-73.94996296699989, 40.612315726999924], [-73.94993049299988, 40.6121446509999], [-73.94978380499997, 40.611371749999854], [-73.95070512899999, 40.611250451999915], [-73.95163234999984, 40.61112353899989], [-73.95256553099986, 40.61098718399993], [-73.95258786900004, 40.611112239999905], [-73.9535988889999, 40.61094986499993], [-73.95479552700002, 40.61082076599994], [-73.95574066999986, 40.6107148269999], [-73.95670471399993, 40.61061187799987], [-73.95766801399986, 40.61050371499993], [-73.95804731899992, 40.610462761999926], [-73.95859278499987, 40.61040303099988], [-73.95952069599996, 40.61030205299987], [-73.9604436189999, 40.61020150699987], [-73.96137268399995, 40.61009887499992], [-73.96237947799978, 40.609986373999966], [-73.96285973999984, 40.61254948599989], [-73.9633609399999, 40.615204427999885], [-73.96356470599983, 40.61623872199992], [-73.9636224559999, 40.616589736999856], [-73.96375418699988, 40.61730243299991], [-73.96381145199987, 40.61758940199997], [-73.96388935599992, 40.61800054999993], [-73.96426798299994, 40.619983802999876], [-73.96471983399981, 40.62237296099993], [-73.96571874599992, 40.622264028999915], [-73.96664515699992, 40.62216223699988], [-73.96757137699979, 40.62206027499997], [-73.96864510599995, 40.6219416949999], [-73.96909724499984, 40.62433282499995], [-73.97019276999991, 40.62420324999987], [-73.97047171899992, 40.62417002999995], [-73.97080408199997, 40.62593266699988], [-73.97086150900003, 40.626237182999894], [-73.97092126300004, 40.62656998499993], [-73.97111421999993, 40.62758588399991], [-73.97120750499982, 40.62808971199992], [-73.97136622199992, 40.62892916199996], [-73.97109232299984, 40.628966140999886], [-73.97000109799986, 40.62911347399987], [-73.96892915899984, 40.62923162599992], [-73.96800084399993, 40.62933305799992], [-73.96707656900001, 40.629434681999946], [-73.96608191399986, 40.629542706999906], [-73.96508072799992, 40.629654674999856], [-73.96414862899985, 40.62973896499996], [-73.96322149299982, 40.629841060999865], [-73.96232599399991, 40.6299687169999], [-73.96197454299985, 40.629995040999866], [-73.96182099799995, 40.630004876999855], [-73.96163639899993, 40.63002222299986], [-73.96136921599995, 40.63005838799989], [-73.96046280599994, 40.630155660999904], [-73.96040734299989, 40.6301625309999], [-73.96025855099994, 40.62941826299991], [-73.96024902900002, 40.62937062199989], [-73.96022560099996, 40.62925450099989], [-73.96014773499996, 40.628915184999876]]]}}, {\"id\": \"165\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 166, \"Shape_Leng\": 0.0688237542871, \"Shape_Area\": 0.000255276270405, \"zone\": \"Morningside Heights\", \"LocationID\": 166, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95707738499995, 40.8142980939999], [-73.95626957999987, 40.813375022999864], [-73.95570690100003, 40.81414395999996], [-73.95520337899985, 40.8135541289999], [-73.95408058399987, 40.81228011299989], [-73.95354382999984, 40.81205082099992], [-73.95210125199992, 40.811442853999935], [-73.9525912579999, 40.810765332999935], [-73.95307869999975, 40.81009408499988], [-73.95327654899991, 40.809879603999875], [-73.9535757779999, 40.8094766979999], [-73.95496657099991, 40.81006455499994], [-73.95542974899983, 40.80942915299993], [-73.95588417599987, 40.80880224899986], [-73.95634493700001, 40.808173353999905], [-73.95680543799989, 40.80754570899993], [-73.95726581599985, 40.806916170999976], [-73.95772176, 40.8062909889999], [-73.95818167299996, 40.80559679099989], [-73.95825482599987, 40.804713877999916], [-73.958203008, 40.803894270999905], [-73.95824878299989, 40.80310648599987], [-73.95868839399998, 40.802469793999855], [-73.95914741099993, 40.80184375999991], [-73.95964685399987, 40.80115642299993], [-73.96107793999975, 40.801800357999845], [-73.96386126299988, 40.80297203799993], [-73.96659731599986, 40.804122764999946], [-73.96670106899995, 40.804168475999916], [-73.96680327299997, 40.80420756099992], [-73.96804849299988, 40.804683699999906], [-73.96849506699984, 40.80483226999993], [-73.96855913399985, 40.80485358399993], [-73.96860707399988, 40.804870949999895], [-73.96978722000001, 40.80530049699996], [-73.97069326499995, 40.80563026799993], [-73.97076362399993, 40.80565737299987], [-73.97084148399995, 40.80568534399987], [-73.97110765876137, 40.80579013958964], [-73.97045990226607, 40.80663109398826], [-73.97021400799224, 40.806935706153325], [-73.9698829061469, 40.80736478848125], [-73.96977288943326, 40.807525282643745], [-73.96955934797347, 40.80779550099897], [-73.96911173565826, 40.808406370068965], [-73.9690910324437, 40.80843016507841], [-73.9689167848404, 40.808630780559795], [-73.96885218645302, 40.80875198661712], [-73.96866264764923, 40.80896329267825], [-73.96855747771205, 40.80907289432358], [-73.96854420633765, 40.80914511039274], [-73.96846348611653, 40.8093187140199], [-73.9684247350703, 40.80939406033304], [-73.96823952732154, 40.80966268389443], [-73.96815337512054, 40.80976588140878], [-73.96811675018661, 40.80980683624045], [-73.96813186592715, 40.809844489225036], [-73.9679606531174, 40.810100010069306], [-73.96788312497532, 40.810201565182176], [-73.96788099174, 40.81024414732448], [-73.96779484834431, 40.810366997374004], [-73.96772160095219, 40.81043416111091], [-73.96770439052305, 40.810488199884375], [-73.96766778940892, 40.81055044594282], [-73.96762038824623, 40.810584850394285], [-73.9676204142259, 40.81063070153996], [-73.96754397270519, 40.810773199724096], [-73.96740185312369, 40.81102215539704], [-73.96713048758585, 40.81138742007399], [-73.9670411118029, 40.811520099626875], [-73.96700232144049, 40.81154794802964], [-73.96692265508437, 40.81169044653371], [-73.96688175014941, 40.811773976428185], [-73.96674930787943, 40.81198198551439], [-73.96663300760063, 40.812153970788046], [-73.96652962321858, 40.812285015524836], [-73.96639177720876, 40.81247337316197], [-73.9662776271676, 40.81263881250281], [-73.96615809223954, 40.81281570279124], [-73.96610639539495, 40.812863206936264], [-73.96588240945108, 40.81320062634857], [-73.96579194974646, 40.81332838170002], [-73.96565840161266, 40.81349873149241], [-73.96548975012963, 40.81374707236784], [-73.96548932424895, 40.81374769375629], [-73.96534706551687, 40.81393965189671], [-73.9651078426741, 40.814275476532295], [-73.96509511906311, 40.814319572659144], [-73.96432483391364, 40.81551273701064], [-73.96414082888295, 40.815771657453894], [-73.9634039510228, 40.816684197940255], [-73.96312702403796, 40.81695509533639], [-73.96308529694089, 40.81700611343312], [-73.96271913392074, 40.81736849075736], [-73.96203105797969, 40.81808633074085], [-73.9618998499999, 40.818047286999885], [-73.961818603, 40.818023153999924], [-73.96168806099998, 40.81796669399994], [-73.9616361359999, 40.81793985099995], [-73.96157544099992, 40.8179099709999], [-73.96138875999992, 40.81781807599991], [-73.9610201569998, 40.81764868599986], [-73.96068717899983, 40.81751051699993], [-73.9594932889999, 40.8170074739999], [-73.9583949659998, 40.81579688699991], [-73.95829447399994, 40.81569078999992], [-73.95819953200001, 40.81558041799985], [-73.95782821899992, 40.815156048999874], [-73.95716929999979, 40.81440312399995], [-73.95710689999994, 40.81433182099988], [-73.95707738499995, 40.8142980939999]]]}}, {\"id\": \"166\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 167, \"Shape_Leng\": 0.0908159737292, \"Shape_Area\": 0.000167529391193, \"zone\": \"Morrisania/Melrose\", \"LocationID\": 167, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89690017899987, 40.8310333359999], [-73.89717220999982, 40.830368832999966], [-73.89669632499997, 40.83041450499996], [-73.89560633499985, 40.830524382999876], [-73.89485843399979, 40.830601857999895], [-73.894692052, 40.83061909399988], [-73.893360418, 40.83015253099996], [-73.8928855849999, 40.8301475709999], [-73.89187594399993, 40.830137505999936], [-73.89087299799995, 40.83013600299989], [-73.8899110649999, 40.830124191999865], [-73.88898205299992, 40.829814800999905], [-73.88805361099989, 40.82950305199991], [-73.88761307899999, 40.829337419999945], [-73.88755325099989, 40.82929634299989], [-73.88601262699991, 40.828238530999926], [-73.88609624199987, 40.82765577699989], [-73.88639957199996, 40.82750337899987], [-73.88789510999993, 40.82674318499987], [-73.88885046199992, 40.82676859499989], [-73.88963895299987, 40.82679750999989], [-73.88969004999984, 40.826799383999834], [-73.88980564199993, 40.82680362199993], [-73.88983599199996, 40.826804733999936], [-73.88997942599995, 40.82680999199989], [-73.89090560099991, 40.82678778499992], [-73.89191271899998, 40.82679444999991], [-73.89293226099981, 40.82678703699992], [-73.89386719099983, 40.82651496799989], [-73.89483522899992, 40.82624684099993], [-73.89515277899991, 40.82783428999991], [-73.8951102089999, 40.82803689899988], [-73.89529156500001, 40.82814234999993], [-73.89548522799996, 40.82824854099993], [-73.89576668599983, 40.8282118029999], [-73.89700278099993, 40.82805045499992], [-73.89820469300003, 40.82788910999989], [-73.89932566999991, 40.82774501799988], [-73.89940994800001, 40.827734183999915], [-73.89979756799993, 40.827684353999906], [-73.90093510699997, 40.82792810199992], [-73.90164895599995, 40.82625697799991], [-73.90050540499992, 40.82600762199993], [-73.901176691, 40.8243889089999], [-73.90181687399985, 40.82285063499993], [-73.901846838, 40.822778632999885], [-73.90198758599993, 40.82244042299992], [-73.9006114109999, 40.822108807999875], [-73.90084561199987, 40.821541417999846], [-73.9012927759998, 40.820475442999914], [-73.90266819199994, 40.82080315899992], [-73.90379310199981, 40.821091099999876], [-73.90478103799988, 40.82134144499991], [-73.90489653200002, 40.82137070999988], [-73.90525022600004, 40.82145692199989], [-73.90532770299998, 40.82147580599994], [-73.90545592300005, 40.821507058999906], [-73.90552466299984, 40.82152381299991], [-73.90565890299979, 40.821556532999864], [-73.90645722199996, 40.821715363999886], [-73.90742681699992, 40.82194506199989], [-73.90841854199991, 40.82218127099991], [-73.90945760899984, 40.82243240199992], [-73.90990894100004, 40.822551107999914], [-73.91069998799998, 40.8223241139999], [-73.91107298599978, 40.82224294299992], [-73.91135642300002, 40.82216364799994], [-73.91136683199993, 40.82216315199986], [-73.91137726399995, 40.822162555999945], [-73.91138768499987, 40.82216185299992], [-73.9113981169998, 40.82216105899986], [-73.91140856199999, 40.82216015799987], [-73.911418994, 40.822159156999845], [-73.91142942699986, 40.82215805799994], [-73.9114398599999, 40.82215685899992], [-73.91145026999986, 40.82215556099989], [-73.91146067899994, 40.82215415499997], [-73.91147107699982, 40.82215264999989], [-73.91148146299997, 40.822151046999885], [-73.91149182600002, 40.822149333999896], [-73.91150216499999, 40.822147531999875], [-73.91151249299988, 40.8221456219999], [-73.91152278399981, 40.82214361299986], [-73.91153305199994, 40.82214150499988], [-73.91154328499991, 40.82213929699984], [-73.91155348299996, 40.82213699099988], [-73.91156365699992, 40.822134584999894], [-73.91157378299995, 40.822132080999914], [-73.9115838739999, 40.82212947799995], [-73.91159391799995, 40.82212677499996], [-73.91160391499992, 40.82212397199989], [-73.91161386399986, 40.82212107999989], [-73.91162376599982, 40.82211808899989], [-73.91163361999985, 40.822115008999894], [-73.91164340399992, 40.8221118279999], [-73.91165313999997, 40.82210855799985], [-73.91166281800001, 40.82210519699987], [-73.91167243599985, 40.8221017459999], [-73.91168198299992, 40.82209820599994], [-73.91169147099988, 40.822094565999876], [-73.9117008879999, 40.82209084599988], [-73.91171023399987, 40.82208704299993], [-73.91171949699985, 40.82208315099992], [-73.9117286999999, 40.82207916899994], [-73.91173782099985, 40.82207510599991], [-73.91174685999992, 40.82207096199993], [-73.91175582700001, 40.82206673699992], [-73.91176469899987, 40.82206243799991], [-73.91177350099998, 40.8220580509999], [-73.911782208, 40.822053599999876], [-73.91179083099985, 40.82204905899985], [-73.91179935999997, 40.8220444549999], [-73.91180780599983, 40.822039769999925], [-73.91181615799985, 40.82203502199991], [-73.91189154999992, 40.822141481999914], [-73.91287633399998, 40.822441347999884], [-73.91450553699998, 40.82294185699991], [-73.91621588599988, 40.82346792999991], [-73.91587735799988, 40.824103731999855], [-73.91677400599986, 40.82438708699987], [-73.91608782899984, 40.82483785199988], [-73.91630917499997, 40.82490286699987], [-73.91661368499985, 40.82499230799991], [-73.91642451899982, 40.82511491299993], [-73.9163666169999, 40.82515243999989], [-73.91630388, 40.825193101999915], [-73.91624748099997, 40.82522965499987], [-73.91618778499992, 40.82526834599989], [-73.91613011899993, 40.82530572099995], [-73.91592284999992, 40.8254400559999], [-73.91545061499994, 40.82574777899985], [-73.91541108199998, 40.82577390899993], [-73.91527590199993, 40.82565539499992], [-73.91475121099991, 40.82520047299993], [-73.91351191699981, 40.824804943999915], [-73.91316674499997, 40.82545645499989], [-73.91192435199989, 40.82504975899994], [-73.91130974499993, 40.824848557999886], [-73.91179361999997, 40.8264281559999], [-73.91169430299985, 40.82775871599993], [-73.91030770399989, 40.828772484999895], [-73.90948178399991, 40.82959211899987], [-73.908959903, 40.83022623299989], [-73.90772165099985, 40.829823367999914], [-73.907527588, 40.82976078199988], [-73.90747341000004, 40.829743308999916], [-73.90657083599997, 40.829452219999936], [-73.90553116799988, 40.82910413299989], [-73.90450621899981, 40.8306249629999], [-73.90397692599986, 40.83141130499991], [-73.90367608499999, 40.83185824799992], [-73.90343501499996, 40.83221638099998], [-73.90185605899981, 40.83462212499988], [-73.90134101199988, 40.83544132399986], [-73.90122534999992, 40.83549991099991], [-73.9012157839999, 40.8354101409999], [-73.89963414999981, 40.835016222999904], [-73.89845304899997, 40.83472024199993], [-73.89809341399994, 40.834628802999916], [-73.89697220299995, 40.8343452539999], [-73.89678082699997, 40.834126788999924], [-73.89630199999992, 40.833543217999896], [-73.89647296199996, 40.83304756799986], [-73.89669216099999, 40.83196614399987], [-73.89690017899987, 40.8310333359999]]]}}, {\"id\": \"167\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 168, \"Shape_Leng\": 0.114517434155, \"Shape_Area\": 0.000547058100444, \"zone\": \"Mott Haven/Port Morris\", \"LocationID\": 168, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.92480953899991, 40.81565901999993], [-73.92399951299991, 40.81543891199991], [-73.92395483099983, 40.8155009609999], [-73.92390804799993, 40.815565926999945], [-73.92376838899996, 40.81575986499996], [-73.92372701399987, 40.81581731999991], [-73.92309299899993, 40.81669772799995], [-73.92124519699988, 40.816219396999934], [-73.92158392200005, 40.815550631999955], [-73.9213038339999, 40.81547724799986], [-73.92048117399989, 40.8145636869999], [-73.92034892799985, 40.81462886399989], [-73.91946744899994, 40.81507370999987], [-73.91861739800005, 40.815523135999854], [-73.91768626999985, 40.8160812429999], [-73.91677117699996, 40.81569882899991], [-73.9149549369999, 40.81493646599995], [-73.91295450799997, 40.81426933499991], [-73.91181210799998, 40.813883314999885], [-73.91094779, 40.81362092099986], [-73.91043307699985, 40.81351637699987], [-73.91038742199996, 40.8135071039999], [-73.91033664499996, 40.81349678899988], [-73.91029615499986, 40.81348856499989], [-73.91013342199993, 40.81345551199997], [-73.91001840399991, 40.81343214799989], [-73.9095236779999, 40.813327805999975], [-73.90949684799988, 40.813322147999926], [-73.90907451799984, 40.81323307299991], [-73.90815917699986, 40.813043575999906], [-73.90724503699998, 40.81285516299991], [-73.90632991800003, 40.812667073999876], [-73.9054135989999, 40.812476195999906], [-73.9044655149999, 40.81228195999994], [-73.90418036399996, 40.81212482099991], [-73.90341720399988, 40.811506882999886], [-73.90324273499989, 40.81071288499991], [-73.90307457199998, 40.80988061199988], [-73.90307568299997, 40.80976898599992], [-73.90302967699985, 40.80970212099986], [-73.90302018099997, 40.80962342899996], [-73.9030087789999, 40.809550072999876], [-73.90299766999989, 40.809455582999924], [-73.90298795399984, 40.80936430099996], [-73.90293929200004, 40.80906270099992], [-73.90288034499996, 40.80874479299996], [-73.90222284749719, 40.80494811309474], [-73.90223410896625, 40.80493879715243], [-73.90223923407719, 40.80493413504008], [-73.90223822507791, 40.80492402279727], [-73.90222993494517, 40.80491593583728], [-73.90221178642066, 40.80488273472918], [-73.90220455168733, 40.80485230849677], [-73.90219185700263, 40.80482326082269], [-73.90219372776716, 40.80478869324347], [-73.9022265326984, 40.804754153035844], [-73.90225572775613, 40.804700252130665], [-73.90231956717278, 40.804597984779555], [-73.90231697601617, 40.80459644908813], [-73.90230320400495, 40.804588292281366], [-73.90229048802287, 40.804573070753094], [-73.90229231992774, 40.80456477614167], [-73.90244539136893, 40.80441695455712], [-73.90252298450754, 40.80446378809854], [-73.9027559359522, 40.80432853482149], [-73.90292770331085, 40.80419729170906], [-73.90287760500665, 40.80415758214557], [-73.90271278211219, 40.80417688793933], [-73.9027097310254, 40.804163663467186], [-73.90289540505104, 40.804143881212795], [-73.90305966667553, 40.80401743484976], [-73.90297311945014, 40.80393677187099], [-73.90300283171007, 40.803917353272666], [-73.9030860978034, 40.80399708789624], [-73.9032126886261, 40.80389963951358], [-73.90325563815311, 40.80392767616931], [-73.90333865261518, 40.80385541293835], [-73.90331727421179, 40.803775282455184], [-73.90333877121293, 40.80377374417664], [-73.90335633787748, 40.80386112361064], [-73.90339901556943, 40.80387490713975], [-73.90355158434579, 40.80383536869234], [-73.90356583811838, 40.80383102363099], [-73.90363555376418, 40.803809770759656], [-73.90368265865864, 40.80379503309547], [-73.90373284581686, 40.80377329611777], [-73.90374201335595, 40.80376780879085], [-73.90366539253351, 40.80370246203189], [-73.90368997946494, 40.803687704539584], [-73.90376599453884, 40.803753454678976], [-73.90377177524543, 40.803749995258485], [-73.90379945054785, 40.803722796073195], [-73.90382200372163, 40.803698703544086], [-73.90385991710878, 40.80366995724983], [-73.90389269713367, 40.80365209611302], [-73.90392732704238, 40.80363439130163], [-73.90385084302861, 40.803574282777895], [-73.90387236017021, 40.80355874470061], [-73.90394842506747, 40.80362130994786], [-73.9039674817376, 40.80360782449496], [-73.90400232330266, 40.8035790755945], [-73.90407055303483, 40.8035209771482], [-73.90401680971797, 40.80347097616888], [-73.9040977498009, 40.80341659812555], [-73.90414898834801, 40.80345565258671], [-73.90437865610475, 40.80327459751856], [-73.90445884046326, 40.80321138584579], [-73.90448088690111, 40.80319169531934], [-73.90449983884291, 40.80318343345092], [-73.9045295409574, 40.80317023565182], [-73.90455004089955, 40.80315003003258], [-73.90457566775626, 40.80312360745073], [-73.90462487688006, 40.80306920276429], [-73.90467203828253, 40.803014796698264], [-73.90476931117227, 40.80291284189395], [-73.90479864625082, 40.80288624391099], [-73.90485211233523, 40.80283522898893], [-73.90495358976553, 40.80272461344582], [-73.90500098071378, 40.80266796125853], [-73.90503014241452, 40.80263480045379], [-73.90504498642524, 40.8026003333841], [-73.90505752846576, 40.80257121666834], [-73.90506666976971, 40.80254080436112], [-73.90508491686398, 40.802504867879755], [-73.90511408874724, 40.80246479364503], [-73.90515780873844, 40.80243026086632], [-73.90518695888107, 40.802405396178024], [-73.90524527987044, 40.80234045671036], [-73.90531233908494, 40.80227472998734], [-73.90539107059976, 40.80218571009475], [-73.90543844595938, 40.80214011954579], [-73.90546395588993, 40.80211525129771], [-73.90549128546061, 40.80209038453586], [-73.90550588373289, 40.80206135891332], [-73.90552153240687, 40.80201995489853], [-73.9057605151772, 40.80179988126246], [-73.90601427821801, 40.80155293109594], [-73.90606336502219, 40.801583391449576], [-73.90620386757755, 40.80149299299668], [-73.90652042213162, 40.801175510519826], [-73.90638976821404, 40.801090791685844], [-73.90645171125526, 40.801036915201635], [-73.90658587767751, 40.8011074784454], [-73.90672893676945, 40.800969967551694], [-73.90681401688877, 40.800890639606216], [-73.9067500914722, 40.80085059452257], [-73.90670130044884, 40.80082002982634], [-73.9067685009217, 40.80074671046322], [-73.90676872957964, 40.80074541735579], [-73.90717872796303, 40.80032263355821], [-73.9072629877428, 40.800237448444896], [-73.90734272035664, 40.80015683842138], [-73.90739374242608, 40.80010433539613], [-73.90743384140607, 40.80005458974865], [-73.90745262874307, 40.80002824445095], [-73.9074703050283, 40.800003458417955], [-73.90749762847857, 40.79998135616736], [-73.90752658104599, 40.79996321969477], [-73.9075395155793, 40.79995511834565], [-73.90757045920829, 40.79994684581935], [-73.9076086725145, 40.7999454942057], [-73.90763052690512, 40.799931684882345], [-73.90764146516834, 40.799916482885465], [-73.90764331288219, 40.799895744109], [-73.90775808108162, 40.799799044286026], [-73.90790019737781, 40.79966088618894], [-73.90793663556686, 40.7996277292761], [-73.90795847677178, 40.799622215547046], [-73.9079912162604, 40.799631921275214], [-73.90802032355599, 40.79963609292482], [-73.90807397255668, 40.7995728191672], [-73.90802770464775, 40.799561430602715], [-73.90795130658027, 40.79954477778334], [-73.90787127636884, 40.799522589666495], [-73.90779307559532, 40.799493489537916], [-73.90773488799034, 40.7994657893005], [-73.90765852928469, 40.799420098716624], [-73.9076822280089, 40.79938831392528], [-73.90778950640637, 40.79944232607244], [-73.90784951426157, 40.79947002824018], [-73.90793317717973, 40.7994963676941], [-73.90799501744833, 40.79951439232811], [-73.90805140706364, 40.799526881912755], [-73.90810210034232, 40.799539646862826], [-73.9081679475427, 40.799461986193734], [-73.90813152403726, 40.79943130077593], [-73.90796076542273, 40.79928344926851], [-73.90802633276698, 40.79923787216213], [-73.90816814020228, 40.7993223306837], [-73.90825177132321, 40.79937217681461], [-73.90832086851556, 40.79940541663984], [-73.90842190768012, 40.79944209244497], [-73.90847556062435, 40.79938341538653], [-73.90855026742408, 40.79930742538958], [-73.90848846163397, 40.79926451139593], [-73.90837756004214, 40.799198052427066], [-73.90829031194237, 40.79913299559319], [-73.90825579566476, 40.799092868181084], [-73.90825513733583, 40.79909100008898], [-73.90824310840051, 40.79905690705975], [-73.90824133200054, 40.79902510304328], [-73.90830303542083, 40.798959135920065], [-73.9091565966024, 40.79830942792145], [-73.90963900093544, 40.79794222580131], [-73.90967334768548, 40.79796124955153], [-73.90967455525322, 40.79796191908996], [-73.90974077921025, 40.79791214367448], [-73.90979388517711, 40.797868495233274], [-73.90984607703047, 40.79790059833692], [-73.90991082881695, 40.79785886102841], [-73.90994644280848, 40.7978343059063], [-73.90996589145395, 40.797804822964714], [-73.91004369318306, 40.797679516552414], [-73.9101343870526, 40.79758863462839], [-73.9103254478973, 40.79742900177438], [-73.9104679052537, 40.79733078589606], [-73.91060709552251, 40.79725714741216], [-73.91071392139699, 40.79719577567378], [-73.91079168499027, 40.79709750926229], [-73.91089222265335, 40.79701779503629], [-73.91090501284451, 40.79701156009358], [-73.91095679669981, 40.796989475918444], [-73.9109859346481, 40.796969832998634], [-73.91105069334637, 40.796920719929254], [-73.91109827415714, 40.79688807133548], [-73.91126755502734, 40.796815184825704], [-73.91141644034329, 40.796746470854565], [-73.91149413374444, 40.79669982456395], [-73.9116228580015, 40.796641078496094], [-73.91168831241185, 40.79662376988872], [-73.9117465332402, 40.796626273244925], [-73.91178857213137, 40.79663367937281], [-73.91182091990503, 40.79663370412359], [-73.91188561236626, 40.79663375359552], [-73.91195998908077, 40.79664855949788], [-73.91203438162788, 40.796651075059444], [-73.91208612977921, 40.79665603089616], [-73.91214700601829, 40.79667811807366], [-73.91215402389724, 40.7966806644417], [-73.91223808592994, 40.79671022611216], [-73.91234481835205, 40.796717682448175], [-73.9126682602596, 40.79673513462004], [-73.91300786031593, 40.79676243196518], [-73.91317604910027, 40.79677239148837], [-73.91344774544464, 40.79678242853534], [-73.91372255986785, 40.79678943887929], [-73.91512767422381, 40.797362948754], [-73.91616947166384, 40.79783071846342], [-73.91880394085274, 40.79887972986115], [-73.9188909264878, 40.7989158947982], [-73.91895581703086, 40.79894691713742], [-73.91904978748826, 40.798993935843235], [-73.91912380285072, 40.79903245502605], [-73.92024068146364, 40.79961368793487], [-73.92044066990172, 40.7998041562453], [-73.92078133912115, 40.800275009727244], [-73.92095645282181, 40.800598814112334], [-73.92132730789253, 40.801284591815644], [-73.92168826986943, 40.80158572340205], [-73.9220376598457, 40.801877197786325], [-73.92259611000095, 40.80218820141712], [-73.9229271008289, 40.80237329482137], [-73.92358699072928, 40.80251572516189], [-73.92422706820558, 40.802545011178985], [-73.92665794788495, 40.80240980391469], [-73.92700359398965, 40.80251195043942], [-73.92762788658843, 40.80269566548149], [-73.92750834884211, 40.80293323371109], [-73.92731248358167, 40.80321177098279], [-73.92743689902498, 40.80327744169375], [-73.92728271943585, 40.80350826441703], [-73.92764500352583, 40.80398863284607], [-73.92768097596573, 40.80402420015814], [-73.92822904734729, 40.80397013297662], [-73.92836704046069, 40.80403270404446], [-73.92841615833625, 40.804100618781256], [-73.92848299256855, 40.80419302907304], [-73.92872607489814, 40.8041036252273], [-73.93005458890505, 40.80590234732321], [-73.93031950058612, 40.80619014620159], [-73.93053645059723, 40.806470276942896], [-73.93051758839161, 40.80659056356675], [-73.93116268129293, 40.807128295776856], [-73.93176476876161, 40.80776969347355], [-73.93181050747292, 40.80781841795046], [-73.93184564493039, 40.807855845481406], [-73.93187509626175, 40.80788945175409], [-73.93191523558487, 40.807935243710496], [-73.9322696077928, 40.808339527481216], [-73.93252708785968, 40.80882328177534], [-73.93264257115322, 40.80938286452631], [-73.93262095288786, 40.809770741773775], [-73.93259434400991, 40.81005315882681], [-73.93283247594536, 40.8101161156863], [-73.93286397083267, 40.81012443738852], [-73.93287412001943, 40.810414043429], [-73.93278688085368, 40.81049446019911], [-73.93269089700772, 40.81058293308123], [-73.93267921899451, 40.810799135187075], [-73.93256503302071, 40.81088597518972], [-73.93261225770267, 40.81139350530766], [-73.93262208681354, 40.811531179116784], [-73.93263229277598, 40.811904622131635], [-73.93247701798535, 40.813423607917436], [-73.93243119390786, 40.81401683319362], [-73.93242839923347, 40.81406068615789], [-73.93242632000944, 40.81411751897961], [-73.93236752499996, 40.81411173099992], [-73.93222960499986, 40.81409433299993], [-73.93139319399977, 40.81374459799993], [-73.93130722899988, 40.81369873899996], [-73.9312360049999, 40.813666728999884], [-73.93115852299998, 40.813635488999914], [-73.93104478900003, 40.813588441999876], [-73.93056964999984, 40.8133732979999], [-73.93039605099992, 40.81330374699994], [-73.93027812699984, 40.813248024999965], [-73.92921297999986, 40.81438195799993], [-73.92919204499995, 40.814496010999946], [-73.9289656749999, 40.81482825499994], [-73.92885642099992, 40.81487986599991], [-73.92821699000002, 40.81653564699991], [-73.92775576099996, 40.81757238099987], [-73.92731653099999, 40.81855963399991], [-73.92724662700006, 40.81854298499994], [-73.92635859099993, 40.818320691999936], [-73.92582070699982, 40.81818033699992], [-73.92538538499986, 40.818065414999865], [-73.92518486499998, 40.818012668999884], [-73.92621972000003, 40.816758654999944], [-73.92662718199988, 40.816118958999915], [-73.92568897199988, 40.81587494499995], [-73.92550133099999, 40.81583301299993], [-73.92505519900006, 40.81572157599992], [-73.92480953899991, 40.81565901999993]]], [[[-73.89833036270552, 40.80241282093997], [-73.89646668834577, 40.80079047089134], [-73.89798486686917, 40.79960428081981], [-73.89864038714974, 40.79910116462134], [-73.89902455567211, 40.799172199839965], [-73.90021004993142, 40.79926415589601], [-73.89978738800835, 40.799509554374524], [-73.8997156381744, 40.8007988085028], [-73.90003735815736, 40.800908742050225], [-73.89948919423654, 40.80090111592956], [-73.89938748571029, 40.80193566507032], [-73.89833036270552, 40.80241282093997]]], [[[-73.89680883223778, 40.795808445159786], [-73.89693872998787, 40.79563587285352], [-73.89723603843935, 40.795720037537116], [-73.89796839783742, 40.79564483916198], [-73.89857332665562, 40.79606914025958], [-73.89895261832532, 40.796227852579634], [-73.89919434249981, 40.796502456018175], [-73.89852052071456, 40.796936194189726], [-73.8978825324018, 40.79711653214704], [-73.89713149795635, 40.79679807772831], [-73.89678526341211, 40.796329166487105], [-73.89680883223778, 40.795808445159786]]]]}}, {\"id\": \"168\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 169, \"Shape_Leng\": 0.0601046580332, \"Shape_Area\": 0.000146027537733, \"zone\": \"Mount Hope\", \"LocationID\": 169, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89816577499982, 40.85514637599989], [-73.89863543699985, 40.85390697499993], [-73.89831118599984, 40.85406115099989], [-73.89709926799998, 40.855172290999874], [-73.89644052599982, 40.854907758999914], [-73.89560883299981, 40.85457036999986], [-73.89662446799987, 40.8527875899999], [-73.89708706499987, 40.85196827699985], [-73.89808598699996, 40.850213123999914], [-73.89887482600002, 40.84881118299993], [-73.89967790699988, 40.84742465799992], [-73.90020067899994, 40.84647356099991], [-73.90088731499989, 40.84524135999991], [-73.90107505699997, 40.84490497899987], [-73.90126635699991, 40.844562217999915], [-73.9013172989999, 40.844473940999926], [-73.901344046, 40.84442875099994], [-73.9026881009999, 40.84458147199991], [-73.90338651199983, 40.844655295999914], [-73.9041010739999, 40.8447308219999], [-73.90483121999989, 40.84480034199987], [-73.90483494099993, 40.844752618999976], [-73.90484219199982, 40.844665447999944], [-73.90484645099987, 40.84458935999987], [-73.90493735799986, 40.84337489399996], [-73.9050151499998, 40.842404697999875], [-73.90514893999989, 40.84254719499995], [-73.9058672189999, 40.84257998699996], [-73.90596061799985, 40.84262870399992], [-73.90684546599987, 40.842667510999966], [-73.90774413799994, 40.84270215599993], [-73.90865715699992, 40.84271936999993], [-73.90958727299986, 40.84275637499993], [-73.91046008499987, 40.842791642999885], [-73.91043026399994, 40.8431077299999], [-73.91096872799983, 40.84313621399987], [-73.91175379899997, 40.84318035699991], [-73.91193751499992, 40.843202387999916], [-73.91211222199998, 40.843260341999915], [-73.91293803199989, 40.8435975549999], [-73.91302876899995, 40.8436416209999], [-73.91385799499986, 40.843978793999945], [-73.91474419399994, 40.8443425039999], [-73.915866478, 40.84479392499991], [-73.91611927099989, 40.84485579299997], [-73.91684309399993, 40.8450707789999], [-73.91689747599989, 40.84511773399991], [-73.9156148529999, 40.845146555999875], [-73.9147938479999, 40.845164996999905], [-73.91416184199993, 40.845179187999925], [-73.91412710499995, 40.84522891399991], [-73.91397534899991, 40.84542943099992], [-73.9137516889998, 40.84573553999994], [-73.91360819800002, 40.8459414389999], [-73.91346470799996, 40.84614733699992], [-73.91315074799996, 40.846577003999876], [-73.91305693099993, 40.846708837999905], [-73.91236345699993, 40.84768333899996], [-73.91197263799982, 40.84822959499992], [-73.9116799109998, 40.84863951899994], [-73.91112232299982, 40.84941809799992], [-73.9106946369999, 40.85001909799993], [-73.910581699, 40.85018159499988], [-73.90921545599986, 40.851800944999894], [-73.9084173709999, 40.85264382199996], [-73.9073328969999, 40.85381641499987], [-73.90557007299988, 40.855787321999884], [-73.90472622199998, 40.85537039499996], [-73.90395154299999, 40.854973922999896], [-73.90298864599995, 40.854773108999936], [-73.90208001099982, 40.85462259899992], [-73.90185996099993, 40.85458614899994], [-73.90153703400001, 40.85499464299988], [-73.90130126599995, 40.85530348999985], [-73.90116590999993, 40.85549567499992], [-73.90085477099996, 40.85593743999986], [-73.90077922699986, 40.85604710999993], [-73.90060385999978, 40.85598723799995], [-73.89967155800004, 40.85566890799991], [-73.89877598599995, 40.85536324699995], [-73.89816577499982, 40.85514637599989]]]}}, {\"id\": \"169\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 170, \"Shape_Leng\": 0.0457690091051, \"Shape_Area\": 7.43150675972e-05, \"zone\": \"Murray Hill\", \"LocationID\": 170, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97203331099985, 40.745804967999945], [-73.97212958899988, 40.745680901999954], [-73.97257598299979, 40.74506785499992], [-73.97302670299986, 40.744450040999915], [-73.97350835000003, 40.7437820339999], [-73.97522419999986, 40.74450468199987], [-73.97588117699996, 40.74477922399991], [-73.97735988999995, 40.74540421599994], [-73.97812378199983, 40.745727172999885], [-73.97860713299984, 40.745057830999876], [-73.9790584949999, 40.744443551999886], [-73.97950791000001, 40.74382520099996], [-73.97995899799997, 40.74320520099991], [-73.98040998099987, 40.74258715999986], [-73.98085965399994, 40.74196976799989], [-73.98245520799999, 40.74264695999995], [-73.98400852, 40.743299017999945], [-73.98407485299995, 40.74332471799989], [-73.98362519999995, 40.7439431149999], [-73.98317411899995, 40.74456031099989], [-73.98272686499999, 40.74518041399988], [-73.98233039299987, 40.74572641199994], [-73.98227872699987, 40.74579756299986], [-73.98183365099989, 40.746416590999935], [-73.98177455300004, 40.74649679899989], [-73.98134068699997, 40.74707866399992], [-73.98085054399985, 40.74775036999991], [-73.98094249999997, 40.74779007099992], [-73.98247211999985, 40.74843150099986], [-73.98407636100002, 40.74910212499993], [-73.983621411, 40.749723770999914], [-73.98317416299989, 40.75034220899994], [-73.98272387399996, 40.75095894399989], [-73.98227490399982, 40.751577297999845], [-73.98182244299984, 40.75219712699987], [-73.9813714319999, 40.752813855999925], [-73.98088709899987, 40.753480988999875], [-73.97928686199992, 40.752800496999946], [-73.9783696929999, 40.752416226999905], [-73.97779152099986, 40.7521675949999], [-73.97767660499981, 40.7521189369999], [-73.97754284999984, 40.75206355599991], [-73.97710781099983, 40.7518834059999], [-73.97685152399993, 40.75177649399989], [-73.97604824899987, 40.75144139299994], [-73.9744494109998, 40.75076644499991], [-73.9749347119999, 40.75009962799993], [-73.97538295899986, 40.74948305799992], [-73.9758346669999, 40.74886314599996], [-73.97497000599988, 40.74850184899986], [-73.97359187599997, 40.74791866299991], [-73.9727379199999, 40.74755798999993], [-73.97121971999988, 40.74692154099996], [-73.97165740599984, 40.74630369399994], [-73.97174810699977, 40.74618613899989], [-73.97194428900002, 40.74593187199988], [-73.97203331099985, 40.745804967999945]]]}}, {\"id\": \"170\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 171, \"Shape_Leng\": 0.107353876244, \"Shape_Area\": 0.000520099538161, \"zone\": \"Murray Hill-Queens\", \"LocationID\": 171, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.79317944599988, 40.77040317699989], [-73.79321182299991, 40.77033648899987], [-73.793422277, 40.76859990799989], [-73.79361279599985, 40.76722335799996], [-73.7939591469999, 40.7646953199999], [-73.79302551399985, 40.76462119199986], [-73.79322483899982, 40.763142007999946], [-73.79334546300007, 40.76229534299991], [-73.79343673799991, 40.76159683099993], [-73.793438002, 40.7615135269999], [-73.79344299999985, 40.761185336999866], [-73.79344033499987, 40.76082252699995], [-73.79343991199988, 40.760764127999934], [-73.79255337999999, 40.76084029399992], [-73.791654662, 40.76091206299987], [-73.79076004599987, 40.76098576899991], [-73.79053450999999, 40.759373037999886], [-73.79031517, 40.75782345099988], [-73.79122151599987, 40.75779876599992], [-73.79201171599978, 40.757780149999896], [-73.79212059099993, 40.75777758199993], [-73.79303015799997, 40.75775619399999], [-73.79402473999977, 40.75773596899993], [-73.79493246199993, 40.75779802999994], [-73.79580437699985, 40.758091216999915], [-73.79621926899995, 40.75824167999989], [-73.79670467899993, 40.758417711999876], [-73.79760259799993, 40.758744178999905], [-73.798501602, 40.75907103599994], [-73.79940088899988, 40.759397218999936], [-73.80030038999979, 40.759702298999926], [-73.80125017499996, 40.75982823699995], [-73.80215291900004, 40.75998721799985], [-73.80307328899983, 40.76012624799988], [-73.80399971299993, 40.76027048799994], [-73.804916685, 40.760440411999895], [-73.80584005399999, 40.760615390999874], [-73.80676048599993, 40.7607877729999], [-73.80768023699983, 40.76095995899987], [-73.80859484099997, 40.76113246199993], [-73.80951372599986, 40.761299910999874], [-73.81044348399986, 40.761377438999894], [-73.81104963199985, 40.761417020999964], [-73.81226778099978, 40.761130021999854], [-73.8131322989999, 40.76089834899995], [-73.81399882899986, 40.760665033999906], [-73.8150873259998, 40.76037161099985], [-73.8172882139999, 40.75977554699992], [-73.82063593299985, 40.75887226799989], [-73.82080684899987, 40.7589417829999], [-73.82116404999985, 40.75968607499993], [-73.82155809699998, 40.76053432399992], [-73.82176735399993, 40.760979813999924], [-73.8218107709998, 40.76107223699987], [-73.82207166499991, 40.761625090999885], [-73.82260068399988, 40.76275987799985], [-73.82302513300002, 40.7636663409999], [-73.82348245199985, 40.76465179199993], [-73.82372658199988, 40.7658255159999], [-73.82395032599992, 40.76688720199994], [-73.82421202499988, 40.76816922799988], [-73.82444330299992, 40.7692855139999], [-73.82466908999987, 40.77040760799985], [-73.8255648519999, 40.77028295999992], [-73.82604927499997, 40.77021081399986], [-73.82655841999986, 40.770137083999906], [-73.82656578899986, 40.77064147599989], [-73.82657511799988, 40.771630352999935], [-73.82657481899986, 40.772345397999885], [-73.82647720799984, 40.77311644299986], [-73.82625348499994, 40.77452065899989], [-73.82583089999997, 40.77591527099986], [-73.82486800999989, 40.77654235499993], [-73.82433101899989, 40.776743069999924], [-73.8242835999999, 40.77698996099983], [-73.8242214729999, 40.777235226999935], [-73.82414480499983, 40.77747809299995], [-73.82405384599981, 40.7777178299999], [-73.82382108999985, 40.7783601379999], [-73.82376244899997, 40.77850455199988], [-73.82354681599996, 40.77903553799989], [-73.82331286899986, 40.77957865999986], [-73.82223999299987, 40.77949023699989], [-73.82132278499986, 40.779409258999856], [-73.82032141599986, 40.77933468099991], [-73.8198104219999, 40.77929698899986], [-73.81930985099999, 40.77926005999988], [-73.81728146999997, 40.7791055129999], [-73.81452997799975, 40.77888817499992], [-73.81466305899991, 40.77790468699987], [-73.81475882999987, 40.77720350099991], [-73.81478821799982, 40.77699573899987], [-73.81485850699985, 40.776485922999875], [-73.812686263, 40.77631279399991], [-73.8094525209999, 40.77605807899988], [-73.80659228499994, 40.77583186199997], [-73.8037902289999, 40.77561011199992], [-73.800989741, 40.77538911599989], [-73.7986519199999, 40.77520055899988], [-73.79853009399987, 40.775190728999924], [-73.79823295499997, 40.77484830499985], [-73.79785400499989, 40.77441566899992], [-73.79772190099995, 40.774272373999935], [-73.79757522899995, 40.77413684199995], [-73.79741530999986, 40.77401038199986], [-73.79741522800005, 40.774010318999885], [-73.79741514499983, 40.774010264999916], [-73.79730005599994, 40.773932285999884], [-73.79724339799986, 40.77389389699988], [-73.79672685099997, 40.77360922899989], [-73.79652490499979, 40.7735032779999], [-73.79571707399992, 40.77306964299988], [-73.79565702099983, 40.773036085999905], [-73.7952155069999, 40.77267490699989], [-73.79507190299978, 40.77254212499992], [-73.794936368, 40.77240452999993], [-73.79480546099995, 40.77225687499992], [-73.79415923499985, 40.77151603799994], [-73.79394905799995, 40.7712673749999], [-73.79323711899993, 40.77046772099994], [-73.79317944599988, 40.77040317699989]]]}}, {\"id\": \"171\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 172, \"Shape_Leng\": 0.118476116148, \"Shape_Area\": 0.000658402501406, \"zone\": \"New Dorp/Midland Beach\", \"LocationID\": 172, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08654858694766, 40.56963231913558], [-74.08691263653213, 40.56898584268365], [-74.0861924436643, 40.56856462059217], [-74.08621632224266, 40.56853269814713], [-74.08725078316107, 40.569081678192894], [-74.08734672272566, 40.569021891998524], [-74.08756362440468, 40.568850448177216], [-74.0879298186953, 40.56859328693609], [-74.08835699781724, 40.56826670759142], [-74.08852300985629, 40.56807032993391], [-74.089202951032, 40.567174036207575], [-74.08914045954154, 40.56713347467466], [-74.08924906075131, 40.56701557053471], [-74.08998259932834, 40.567380011533096], [-74.09054960106333, 40.567083597211294], [-74.09099905467036, 40.56673712909749], [-74.09123807762191, 40.56650661322813], [-74.09404968409731, 40.56298164761111], [-74.09399643661986, 40.56280605965054], [-74.09407071109435, 40.562731619461196], [-74.09511063361298, 40.56317327343457], [-74.09524145226949, 40.56314838653871], [-74.09542371796327, 40.5631546355686], [-74.09568304214078, 40.56291939729116], [-74.09566331996835, 40.56281931767249], [-74.09567198061367, 40.5627755982454], [-74.09571191999136, 40.562750869401256], [-74.09576141073522, 40.562732119593214], [-74.09582321827422, 40.56272227341536], [-74.09588687685212, 40.56272491684932], [-74.09611847135454, 40.56258757449008], [-74.09629202201867, 40.56242083077325], [-74.09661362272206, 40.562087145537326], [-74.09687036780507, 40.56171797265268], [-74.0969819943174, 40.56147621667032], [-74.09702483295521, 40.56133643899698], [-74.09708844300383, 40.561254568028545], [-74.09715495845641, 40.56118830750991], [-74.09723420757017, 40.561119586719194], [-74.09730732893384, 40.561015811089995], [-74.09741969570297, 40.56088364770623], [-74.09759824840336, 40.560612925033304], [-74.09788807970713, 40.56016942673052], [-74.09796392976403, 40.56003655008663], [-74.09797864152097, 40.55999301361571], [-74.09799348696792, 40.55996710535788], [-74.09804431644582, 40.55991316544581], [-74.09812034213034, 40.559892952543535], [-74.09819767261925, 40.559836476804655], [-74.09827265698341, 40.55954880879489], [-74.09829030823695, 40.559332614156816], [-74.09822154829855, 40.559283582038326], [-74.0980868632024, 40.559202743358156], [-74.09799902257589, 40.55918570617137], [-74.09783230303549, 40.5591204548634], [-74.09779665317956, 40.55908724277523], [-74.09782127023999, 40.55905346621769], [-74.09784815988272, 40.55903366909169], [-74.0979317271503, 40.559041462891436], [-74.09862204726852, 40.559387981860255], [-74.09865870468045, 40.55940756041859], [-74.09870617083808, 40.55942360149753], [-74.09876415480899, 40.559423551650866], [-74.09883262879086, 40.55938730897776], [-74.09892581152889, 40.55935005689283], [-74.09909547833853, 40.55925879063089], [-74.09932770361462, 40.559097413730214], [-74.09929593747901, 40.55900497241413], [-74.09938031432728, 40.5589641933612], [-74.09971784547332, 40.559141300031115], [-74.09992914200652, 40.55907068582615], [-74.10173732699991, 40.55990557799989], [-74.10199826099996, 40.560006518999955], [-74.10225503199992, 40.56011348999988], [-74.10250740099998, 40.56022639199988], [-74.10275513299999, 40.56034511799994], [-74.10299799599996, 40.56046955799994], [-74.10323576299994, 40.56059959599989], [-74.10346821399996, 40.560735110999865], [-74.10369512999992, 40.5608759759999], [-74.10391630199994, 40.56102206099994], [-74.10413152099991, 40.56117322799983], [-74.104846271, 40.56159523899992], [-74.10557645499996, 40.56198997299986], [-74.10617236999992, 40.56229633299993], [-74.10690692399996, 40.56266704599994], [-74.10710400899995, 40.56276988299986], [-74.1078679589999, 40.56314558799995], [-74.108615941, 40.56352836299992], [-74.10936214499999, 40.56390893799991], [-74.11013947199989, 40.56431066699985], [-74.11211239799994, 40.56550555799987], [-74.11369932899999, 40.56645974399986], [-74.11376161199998, 40.56649718899993], [-74.11430477299992, 40.56594740099993], [-74.11489902599997, 40.56536698599993], [-74.11546552899996, 40.56480595799997], [-74.11636336399992, 40.56391563799986], [-74.11641052699991, 40.56394351599993], [-74.1172129029999, 40.56441776399988], [-74.11804055399998, 40.56492164199989], [-74.11876038899996, 40.56535522399992], [-74.11948009699996, 40.5657795529999], [-74.12024008999991, 40.56622746499986], [-74.12166636299999, 40.56708845999986], [-74.1230928109999, 40.567966365999894], [-74.12329206199995, 40.56807586399985], [-74.12359663199994, 40.568239980999905], [-74.12395063699994, 40.568430718999934], [-74.12489904399992, 40.568949141999916], [-74.12418560399995, 40.569424537999964], [-74.12345096199988, 40.56992276099989], [-74.12278967199991, 40.57042984499987], [-74.12259644899996, 40.57059643299989], [-74.12216856599991, 40.57093949299983], [-74.12162563999989, 40.571501393999924], [-74.12150213699994, 40.571721558999904], [-74.12138731299996, 40.57209221099987], [-74.12138815399996, 40.57222366799989], [-74.1213575439999, 40.572372611999846], [-74.12135751999995, 40.572372710999936], [-74.12135750799999, 40.572372800999894], [-74.12134254799993, 40.57252387099992], [-74.12134352099994, 40.57267566199993], [-74.12134352099997, 40.57267573399987], [-74.12136044699997, 40.57282674399991], [-74.12139299299997, 40.57297554799992], [-74.12160439199994, 40.57355591999987], [-74.1217007549999, 40.57383202799988], [-74.12183079799992, 40.5742170889999], [-74.12185333599993, 40.57465318099989], [-74.12172623899997, 40.57490698499986], [-74.1215372899999, 40.57515325299986], [-74.1212571389999, 40.575254893999904], [-74.12112080299991, 40.5753068449999], [-74.11978567499995, 40.5758155559999], [-74.11910954899994, 40.57615725199988], [-74.11844855699992, 40.57665868099989], [-74.118033584, 40.577170408999955], [-74.1175761019999, 40.57770348399992], [-74.11702078599997, 40.57819692599992], [-74.11683019199994, 40.578254717999904], [-74.1166329409999, 40.57830047599992], [-74.1164310249999, 40.578333594999904], [-74.11622655999997, 40.57835373999991], [-74.11602165699998, 40.57836082599991], [-74.11588397199995, 40.5783836949999], [-74.11575021599991, 40.5784236329999], [-74.11562578199988, 40.57847975299993], [-74.11551544900001, 40.5785498789999], [-74.1154228409999, 40.57863086699996], [-74.11481941499994, 40.579176874999895], [-74.11422746699996, 40.579733907999916], [-74.11394959799999, 40.58004141599991], [-74.11367299899999, 40.5803129369999], [-74.11306954499995, 40.58076852199988], [-74.11220998099996, 40.581266237999905], [-74.11127706499995, 40.58179072899986], [-74.11028714299995, 40.58219874799994], [-74.10924835499999, 40.58267841299991], [-74.10790851599995, 40.58368563799992], [-74.10729031099996, 40.584222297999865], [-74.10649465099986, 40.58489534299985], [-74.10588393199993, 40.58540720699986], [-74.10525775499997, 40.58592143599986], [-74.10278862899995, 40.58422448699991], [-74.10265901099994, 40.58408819999994], [-74.10249695999993, 40.58401503299994], [-74.1006131089999, 40.58271119099989], [-74.09844000899997, 40.581241705999915], [-74.09908893799992, 40.58072522899993], [-74.09974044199997, 40.58020781799996], [-74.10039083499996, 40.579688665999925], [-74.09940090399988, 40.57918675599992], [-74.09851032499989, 40.5787306869999], [-74.09741191099992, 40.57817443199992], [-74.09613172499989, 40.57752610599994], [-74.09517725099997, 40.57702113899995], [-74.09483640599994, 40.57680653799987], [-74.094086718, 40.57641955099987], [-74.09335651099995, 40.576047193999926], [-74.09257214599997, 40.57567191299993], [-74.09182234099991, 40.57527493299995], [-74.0910942809999, 40.57489922899992], [-74.09032842599996, 40.57451533599988], [-74.08960544799997, 40.5741304699999], [-74.08886977599995, 40.573723237999936], [-74.08775041799994, 40.57311057999993], [-74.08769737599994, 40.57306188399989], [-74.08765791399992, 40.57301266499997], [-74.08597647499992, 40.57223697099986], [-74.08495715819193, 40.57125984002107], [-74.08595297385915, 40.57029822710432], [-74.08654858694766, 40.56963231913558]]]}}, {\"id\": \"172\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 173, \"Shape_Leng\": 0.0600639661414, \"Shape_Area\": 0.000178329851357, \"zone\": \"North Corona\", \"LocationID\": 173, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85540366099997, 40.75834030899989], [-73.85530200199995, 40.75829284199989], [-73.85505441, 40.758330370999914], [-73.85490872699997, 40.75833644899987], [-73.85476880299996, 40.758328129999896], [-73.85462918699993, 40.75830695499996], [-73.85449593299987, 40.75827863899985], [-73.85436796999998, 40.75811908299986], [-73.85405570299993, 40.75760316299988], [-73.85379813799992, 40.757177611999914], [-73.85377320199979, 40.75711640099996], [-73.8535413749999, 40.75656303299997], [-73.85339582599983, 40.75614235899991], [-73.85335273999982, 40.756017825999876], [-73.85318740799981, 40.755400028999944], [-73.85312635499996, 40.75510934299989], [-73.85323754899999, 40.75509560099989], [-73.85383081399995, 40.754958337999874], [-73.85478630699987, 40.75470744999992], [-73.85579909099987, 40.75440952799994], [-73.85538540499991, 40.75357105999993], [-73.85497762399996, 40.75278221699993], [-73.85455552299993, 40.7519619189999], [-73.855834956, 40.7515772719999], [-73.85547890700003, 40.75089123399986], [-73.85513928899988, 40.75022639499993], [-73.85479286699997, 40.74956323099989], [-73.8544701179998, 40.748939347999936], [-73.85442575299999, 40.748851725999884], [-73.85734765199993, 40.74797143499989], [-73.86027336099988, 40.74710172799993], [-73.86032027699984, 40.747195698999896], [-73.86253100999993, 40.74653380199991], [-73.86407236699985, 40.7460736519999], [-73.8660885059999, 40.745473952999944], [-73.86790963899986, 40.74493371599989], [-73.86808574299982, 40.745600582999884], [-73.86826502599992, 40.74631422299992], [-73.86843750999991, 40.747031196999934], [-73.86852925199995, 40.747588301999954], [-73.8689139579999, 40.748368876999955], [-73.86919188099995, 40.74876289399987], [-73.86942458599987, 40.74915685199991], [-73.86968551, 40.74961236199989], [-73.87084215599985, 40.75154469299985], [-73.87179244199989, 40.753325668999906], [-73.87207047299985, 40.75386198799987], [-73.87214232699986, 40.753986047999874], [-73.87266966999984, 40.75498679899989], [-73.872692407, 40.75502993899988], [-73.87271067399989, 40.75506352099989], [-73.87365721799983, 40.756803485999946], [-73.87311054799989, 40.756850881999895], [-73.87221856399995, 40.75694322399988], [-73.87132395899997, 40.75703633999996], [-73.87041371399981, 40.75713241199987], [-73.86950314799998, 40.75722801899992], [-73.86848374899984, 40.75733493699991], [-73.86746336500002, 40.757441539999895], [-73.86656997600001, 40.75753552099992], [-73.86567436999994, 40.75762958299988], [-73.86461956599985, 40.75769371599994], [-73.86371499900002, 40.757725670999875], [-73.86280704199999, 40.75775898399997], [-73.8619061719999, 40.757794863999926], [-73.86099882399996, 40.75782499299988], [-73.86046162299996, 40.75784373199986], [-73.85992746199989, 40.75786410199987], [-73.85951332399988, 40.75787858999995], [-73.85888932400003, 40.75789947599986], [-73.85856138999993, 40.75791243699992], [-73.85775351199999, 40.75802727499996], [-73.85695186799988, 40.7581331269999], [-73.85560819899995, 40.758305410999846], [-73.85540366099997, 40.75834030899989]]]}}, {\"id\": \"173\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 174, \"Shape_Leng\": 0.0678177559602, \"Shape_Area\": 0.000155928664422, \"zone\": \"Norwood\", \"LocationID\": 174, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87772817699984, 40.88345419499995], [-73.87812594399992, 40.88215382599989], [-73.87717005999983, 40.88199391299992], [-73.87625890099994, 40.88183203999991], [-73.87533699399987, 40.88167183199992], [-73.87467601499999, 40.88156285199993], [-73.87479240699987, 40.880075128999884], [-73.87494589799991, 40.87994766899988], [-73.87406751299994, 40.880038905999896], [-73.87307334600001, 40.88013438499995], [-73.87191803300003, 40.880237465999876], [-73.87187100700007, 40.88032811999996], [-73.87084077099992, 40.880438893999965], [-73.87024100099985, 40.88022935299995], [-73.87094367500002, 40.87851076699998], [-73.87160577400002, 40.87692192799989], [-73.87255250899983, 40.8751751829999], [-73.87436750799998, 40.872701008999876], [-73.87518802799978, 40.87157682799994], [-73.87619224999979, 40.87047671099987], [-73.87694351999987, 40.86995468199993], [-73.87772752999979, 40.86955533799994], [-73.87991019699996, 40.86836304099992], [-73.88135802599994, 40.86757898699991], [-73.8830900319999, 40.86659150899993], [-73.88362518100003, 40.86725758799991], [-73.88284356199999, 40.86773622599992], [-73.88188379899985, 40.86832083599991], [-73.88146889699985, 40.868573645999916], [-73.88174189599988, 40.8695527139999], [-73.88197649899989, 40.87038728399996], [-73.88222049399987, 40.871227247999855], [-73.88246984299988, 40.87212868499994], [-73.8825014509999, 40.872293623999944], [-73.8825014739999, 40.87229374999986], [-73.88250150999994, 40.87229386699992], [-73.88255385599989, 40.87245711599994], [-73.88255391499996, 40.872457287999914], [-73.88255399799999, 40.87245745799991], [-73.88262679199994, 40.872616855999865], [-73.88262687499989, 40.87261702799994], [-73.88262696899983, 40.87261718899993], [-73.88271951399993, 40.87277078099993], [-73.88283030599987, 40.87291636699989], [-73.88283042399992, 40.87291651999991], [-73.8828305549999, 40.87291666499986], [-73.882957783, 40.87305245199983], [-73.88308270399986, 40.87321891799995], [-73.88308273999986, 40.873218971999904], [-73.88308277499985, 40.87321902499988], [-73.8831904229999, 40.87339318999993], [-73.88319048200002, 40.87339327999999], [-73.88319052799997, 40.87339337899994], [-73.88327982500005, 40.87357380999987], [-73.88334994299991, 40.87375884999991], [-73.883400353, 40.87394662599995], [-73.88344008999987, 40.87479468399987], [-73.88343632799992, 40.87538431899991], [-73.88342472699996, 40.875742665999866], [-73.88341932399999, 40.87584701799994], [-73.88342964999991, 40.87595216999989], [-73.88345575699994, 40.87605593399992], [-73.88345579299984, 40.87605609499993], [-73.88345586399996, 40.876056256999945], [-73.88349725699989, 40.876156739999885], [-73.88355294399997, 40.87625222899987], [-73.88382977799998, 40.8767828419999], [-73.8846776199998, 40.87823547699989], [-73.88477074799978, 40.87839502799989], [-73.88489171499985, 40.87860601699993], [-73.88505419400002, 40.87889287599994], [-73.88527683199987, 40.87921766899991], [-73.88541068099991, 40.87910168399991], [-73.88567517999988, 40.87887251099986], [-73.88575540599997, 40.87931311799986], [-73.88578568400006, 40.879616873999964], [-73.88582680399999, 40.879907896999896], [-73.88595071699984, 40.88050741199993], [-73.88706962799996, 40.882476895999915], [-73.88716505299983, 40.882576510999904], [-73.88694246899995, 40.88268276699995], [-73.88697007500001, 40.8827956429999], [-73.8872319409999, 40.88305497799989], [-73.88718101599989, 40.88308394199995], [-73.88714697699992, 40.8831179379999], [-73.8871320659999, 40.88315094499989], [-73.88712303599982, 40.883200939999874], [-73.88710703399992, 40.8836739859999], [-73.88710004499984, 40.88372994399995], [-73.88708705799998, 40.88379702699996], [-73.88704304099994, 40.88404023199996], [-73.88705142599984, 40.884349575999906], [-73.88692889699986, 40.88435571499989], [-73.88683486699986, 40.88434202599992], [-73.88662412999999, 40.88426730299992], [-73.88516495799995, 40.883631611999924], [-73.88452615199998, 40.88331214699985], [-73.88437922299993, 40.88324772299995], [-73.88437905799985, 40.88324765099988], [-73.88437887899987, 40.88324759799992], [-73.88421696399996, 40.8831977279999], [-73.88421683399996, 40.883197691999904], [-73.88421671500001, 40.88319766499993], [-73.88404353399987, 40.88316500199992], [-73.88404341599993, 40.88316498399992], [-73.88404328600001, 40.88316497399997], [-73.88386391399983, 40.88315139399986], [-73.8838637479999, 40.883151384999884], [-73.88386358099999, 40.88315139399987], [-73.88368347599999, 40.88315761499991], [-73.883683369, 40.883157614999966], [-73.88368327399996, 40.88315763299994], [-73.88350810499995, 40.88318314699995], [-73.8835078799999, 40.88318318399992], [-73.88350764199998, 40.883183245999874], [-73.88334258699989, 40.88322637399988], [-73.88334234899999, 40.883226436999934], [-73.88334213499988, 40.883226517999944], [-73.88319088799989, 40.88328483399986], [-73.88274833399998, 40.88339738699993], [-73.882086237, 40.883132220999904], [-73.88124422499992, 40.88270963299992], [-73.8799579879998, 40.884390628999896], [-73.87936283299985, 40.88521823999992], [-73.87876041699998, 40.88604019899987], [-73.8781494459999, 40.88687380699994], [-73.878131292, 40.886912796999866], [-73.87803562399996, 40.887118321999885], [-73.87799008999986, 40.8870503589999], [-73.87783125899992, 40.88677275999987], [-73.87760676600001, 40.88452304399993], [-73.87772817699984, 40.88345419499995]]]}}, {\"id\": \"174\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 175, \"Shape_Leng\": 0.134897665102, \"Shape_Area\": 0.000504920726134, \"zone\": \"Oakland Gardens\", \"LocationID\": 175, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7562560639999, 40.7584304689999], [-73.75583713299987, 40.757742061999934], [-73.7555734369999, 40.75828095199988], [-73.75551718999988, 40.7583214729999], [-73.75539137399988, 40.758389247999915], [-73.75525632799999, 40.75844214599991], [-73.75511648399994, 40.758479497999915], [-73.75497607999992, 40.75850155999997], [-73.75422838200002, 40.75852489199992], [-73.75404744899984, 40.75851238599993], [-73.7538672559999, 40.75848565299992], [-73.75369069100002, 40.75844478799994], [-73.75352058499998, 40.75839033099988], [-73.75335958499983, 40.75832324899992], [-73.75320997199995, 40.7582448729999], [-73.752771716, 40.75887302799991], [-73.75230259399993, 40.759455198999916], [-73.75115582999986, 40.75906976799995], [-73.75057710699984, 40.75801489499991], [-73.75004764499985, 40.75840465799996], [-73.74809587899985, 40.75589954999988], [-73.74711113899991, 40.75417328599991], [-73.74774116499992, 40.75384994399989], [-73.74760569999998, 40.75350787699992], [-73.74737515000001, 40.752959619999935], [-73.7470988539999, 40.75234682899989], [-73.74687468099984, 40.75208864699991], [-73.74684219699999, 40.752058075999884], [-73.74680794599988, 40.75202841699993], [-73.74667146099986, 40.75193218699994], [-73.74652681399978, 40.7518426459999], [-73.74637489399997, 40.75176036499987], [-73.74621665699989, 40.75168584999995], [-73.74577948799998, 40.75156080499994], [-73.74507911699986, 40.75027389699992], [-73.74409668799984, 40.74820664299995], [-73.74384322099985, 40.74769009599988], [-73.74625984399985, 40.74671147499993], [-73.74593060399998, 40.74571565899988], [-73.74477587099986, 40.74328109699994], [-73.74567621100002, 40.7430272989999], [-73.74671975299987, 40.74312364799993], [-73.74784861300004, 40.743296077999936], [-73.74887697299984, 40.74328837399992], [-73.74979834499989, 40.74314244299993], [-73.75072790300003, 40.74299784699989], [-73.75163327499985, 40.742806759999915], [-73.75253331799986, 40.742611332999914], [-73.75347299999999, 40.74240484399992], [-73.75138967499991, 40.74019415999988], [-73.75126422199996, 40.740125368999884], [-73.75126399699991, 40.74012524299995], [-73.75126376099992, 40.74012514299993], [-73.75112759899997, 40.740067314999884], [-73.7511273519999, 40.74006720699986], [-73.75112706799989, 40.74006711699994], [-73.75098241899991, 40.74002149799987], [-73.75098230099987, 40.74002146199996], [-73.75098218299992, 40.74002143499992], [-73.75083193299997, 40.73998895299995], [-73.75083176700001, 40.73998891699989], [-73.75083158999992, 40.739988898999904], [-73.75067885099985, 40.73996996399992], [-73.75052540299986, 40.73995378199988], [-73.75052516599999, 40.73995375399992], [-73.75052495299994, 40.739953708999934], [-73.75037372900003, 40.7399224929999], [-73.75037358699991, 40.73992246699986], [-73.75037344499992, 40.73992242099994], [-73.75022795699984, 40.7398765299999], [-73.75022786299992, 40.73987649399992], [-73.75009192199988, 40.73981711699985], [-73.75009170899982, 40.73981702699994], [-73.75009150899994, 40.73981690799993], [-73.74996804299997, 40.73974575199988], [-73.74657796099996, 40.74034659199985], [-73.7439619359999, 40.74078912699986], [-73.74385264199996, 40.740093948999835], [-73.74371941299982, 40.73938314999984], [-73.74360280299993, 40.73901260899992], [-73.743469568, 40.73887010399992], [-73.74306820799991, 40.7384408179999], [-73.74222678199988, 40.738615671999874], [-73.74155202599987, 40.73876204099987], [-73.74142000699983, 40.73878380799991], [-73.74128113899988, 40.73878869199996], [-73.74114115799996, 40.73877533799986], [-73.74100638499996, 40.738743967999895], [-73.74100625499993, 40.73874393099987], [-73.7410061249999, 40.73874389499987], [-73.7408821769999, 40.73869623999991], [-73.740773448, 40.738635272999886], [-73.74055585499997, 40.73858718299989], [-73.74029081599988, 40.738516669999974], [-73.74016951799993, 40.73848189799986], [-73.741446287, 40.73693874699991], [-73.74151636899985, 40.73686036099986], [-73.74158944499997, 40.736782008999946], [-73.74228431199994, 40.736086407999935], [-73.742947859, 40.7354565519999], [-73.74472417799997, 40.734011029999905], [-73.74555368099982, 40.73340431799991], [-73.7461625949999, 40.73292721999995], [-73.74672140400006, 40.73247715099991], [-73.74704531499992, 40.73219365299985], [-73.74733126699995, 40.73193874099994], [-73.74749692899994, 40.73179105099994], [-73.7479342389999, 40.73140119599985], [-73.74917188599983, 40.7302990689999], [-73.74980987499995, 40.72974674999989], [-73.75048295699996, 40.72922129399996], [-73.75081066899993, 40.72899503899987], [-73.75106753999994, 40.72881769799992], [-73.75184238299993, 40.72832650199991], [-73.75221914099994, 40.728112343999925], [-73.75259471000005, 40.7279089859999], [-73.7529702879999, 40.72770561999986], [-73.75363109999996, 40.727393747999855], [-73.75380709300003, 40.72731553199989], [-73.75406469299989, 40.72720104599992], [-73.75442222499994, 40.727042147999924], [-73.75527584399988, 40.72671544099991], [-73.75612598799987, 40.7264302599999], [-73.75655253399987, 40.72627075399994], [-73.75663658099985, 40.7262464109999], [-73.75669771299987, 40.7262287109999], [-73.7567473959999, 40.72627289499993], [-73.75678272799996, 40.72630578399985], [-73.75684854799977, 40.72635581199993], [-73.75787141499994, 40.72710685099989], [-73.75831675599993, 40.727362406999916], [-73.75993818299987, 40.72815841199989], [-73.76024211099985, 40.728307611999874], [-73.76114083699993, 40.72879441399988], [-73.76160814399992, 40.72904753399989], [-73.76234209799992, 40.72953829099986], [-73.76327537799996, 40.730240540999915], [-73.76381866799998, 40.73068357399991], [-73.76435071499999, 40.73115119199987], [-73.76486853099996, 40.73166863999996], [-73.76528830399985, 40.73211466399991], [-73.76558856199988, 40.732467520999876], [-73.76564374500005, 40.732533126999954], [-73.76555086099985, 40.732544282999875], [-73.76491667699993, 40.73265467299992], [-73.76434014999984, 40.73275295499994], [-73.7636997919999, 40.73286411699993], [-73.76374010499987, 40.73291287299987], [-73.76470929199999, 40.73408485499987], [-73.76503275799992, 40.734466408999886], [-73.76510920699994, 40.73455658499992], [-73.76503749400005, 40.73457204399988], [-73.7642906499999, 40.73480260699987], [-73.76388970599994, 40.73492637999993], [-73.76318167899983, 40.73514494799989], [-73.76311240499989, 40.73516391399993], [-73.76273303499985, 40.73526216299995], [-73.76252875800003, 40.73600396099994], [-73.76248538699991, 40.73618169499987], [-73.76246334599998, 40.73636362499993], [-73.76246348399997, 40.73654750099992], [-73.76248609299982, 40.736730981999884], [-73.76253086199996, 40.73691167999987], [-73.7625969479999, 40.737087325999894], [-73.76268297199981, 40.737255835999854], [-73.763168165, 40.73826170799989], [-73.76360860199983, 40.73917476499987], [-73.76431495599994, 40.740672626999896], [-73.76515128899993, 40.74243734599987], [-73.76556849799985, 40.74325815499986], [-73.76617484799982, 40.743567271999886], [-73.76633756199995, 40.74365225899987], [-73.76649109699986, 40.74374842999987], [-73.76663352600005, 40.74385485299991], [-73.7667631489999, 40.74397036199991], [-73.76687853599992, 40.74409357899988], [-73.76697863000001, 40.74422296799987], [-73.76761507999991, 40.745526354999924], [-73.76770614599987, 40.74572274199989], [-73.76772991399994, 40.74577622599993], [-73.76729011899992, 40.74590381399993], [-73.76681551899995, 40.7460300569999], [-73.76635520899988, 40.74615249699984], [-73.7658920239999, 40.74628062699993], [-73.76501623199981, 40.746522881999894], [-73.76414324400004, 40.74676404599995], [-73.76297777799992, 40.74708162199994], [-73.76240045799992, 40.74724325399988], [-73.76151846699999, 40.74749054799987], [-73.76145571899991, 40.747507951999864], [-73.76138355799993, 40.74752917099989], [-73.76073612099987, 40.74773738499994], [-73.75972553599988, 40.748188660999894], [-73.75885932599982, 40.748584227999885], [-73.75845857499995, 40.748763092999916], [-73.75805782299985, 40.74894194899988], [-73.75802805599993, 40.74895781599986], [-73.7577393699999, 40.7490634429999], [-73.75744314899997, 40.74915673299992], [-73.75714080500002, 40.74923725899987], [-73.75684127399992, 40.74932207799991], [-73.75686552299992, 40.74936148999998], [-73.75698991500002, 40.74957500999989], [-73.75737714099989, 40.750228101999895], [-73.75782746199997, 40.75075826999989], [-73.75910049199996, 40.752253820999954], [-73.75948272499986, 40.75243381299991], [-73.75957000299998, 40.752474795999895], [-73.75957016899991, 40.75247487799994], [-73.75957032200002, 40.75247497699995], [-73.75971494599995, 40.75256269999991], [-73.75971508799988, 40.7525627919999], [-73.75971521799991, 40.75256288999989], [-73.7598465069999, 40.75266410199989], [-73.7598466729999, 40.75266422799993], [-73.75984680199994, 40.75266435499989], [-73.75996168499987, 40.75277718499993], [-73.7599618029999, 40.75277730199993], [-73.75996192099983, 40.752777446999865], [-73.76005787299981, 40.752899403999926], [-73.76005800199988, 40.752899574999894], [-73.76005811999997, 40.75289976499995], [-73.76013358599997, 40.753028199999925], [-73.76013364499984, 40.75302828899988], [-73.76013367999985, 40.75302838899988], [-73.76018801899988, 40.75316049899985], [-73.76048623699985, 40.75383961699987], [-73.76080194199993, 40.754547729999956], [-73.76090047000001, 40.75468513599987], [-73.76101539999992, 40.754816091999864], [-73.76114556499998, 40.7549389489999], [-73.76128938199999, 40.7550522149999], [-73.76144497499985, 40.75515465699987], [-73.7615672369999, 40.75523235199987], [-73.76167669899993, 40.75532347699988], [-73.76176970599987, 40.755426020999884], [-73.76184343999986, 40.75553726899989], [-73.76189611099981, 40.755654002999904], [-73.76192713800003, 40.755772852999876], [-73.76185572299987, 40.756516974999926], [-73.76176692899985, 40.757278801999846], [-73.76199745999992, 40.75810219099993], [-73.76239510599996, 40.759496997999854], [-73.76125540599999, 40.75971470799989], [-73.76031788499999, 40.75990455099988], [-73.7598774599999, 40.75999492299991], [-73.75800131799996, 40.76037986699988], [-73.75682062199998, 40.75973480599985], [-73.7562560639999, 40.7584304689999]]]}}, {\"id\": \"175\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 176, \"Shape_Leng\": 0.151995190703, \"Shape_Area\": 0.000657782126267, \"zone\": \"Oakwood\", \"LocationID\": 176, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.12187198599999, 40.57510757499989], [-74.12187183299987, 40.57510756699992], [-74.12187169099991, 40.57510757499995], [-74.12175740099993, 40.57511179399995], [-74.1217572469999, 40.5751118029999], [-74.12175709399996, 40.57511182099992], [-74.121644926, 40.57512723099993], [-74.12164482099995, 40.575127248999905], [-74.12164470299997, 40.57512727699991], [-74.1215372899999, 40.57515325299986], [-74.12172623899997, 40.57490698499986], [-74.12185333599993, 40.57465318099989], [-74.12183079799992, 40.5742170889999], [-74.1217007549999, 40.57383202799988], [-74.12160439199994, 40.57355591999987], [-74.12139299299997, 40.57297554799992], [-74.12136044699997, 40.57282674399991], [-74.12134352099997, 40.57267573399987], [-74.12134352099994, 40.57267566199993], [-74.12134254799993, 40.57252387099992], [-74.12135750799999, 40.572372800999894], [-74.12135751999995, 40.572372710999936], [-74.1213575439999, 40.572372611999846], [-74.12138815399996, 40.57222366799989], [-74.12138731299996, 40.57209221099987], [-74.12150213699994, 40.571721558999904], [-74.12162563999989, 40.571501393999924], [-74.12216856599991, 40.57093949299983], [-74.12259644899996, 40.57059643299989], [-74.1227896729999, 40.5704298449999], [-74.12345096199988, 40.56992276099989], [-74.12418560399995, 40.569424537999964], [-74.12489904399992, 40.568949141999916], [-74.12395063699994, 40.568430718999934], [-74.12359663199994, 40.568239980999905], [-74.12329206199995, 40.56807586399985], [-74.1230928109999, 40.567966365999894], [-74.12166636299999, 40.56708845999986], [-74.12024009, 40.56622746599986], [-74.11948009699996, 40.5657795529999], [-74.11876038799987, 40.56535522399988], [-74.11804055399998, 40.56492164199989], [-74.1172129029999, 40.56441776399988], [-74.11641052699991, 40.56394351599993], [-74.11636336299996, 40.56391563799985], [-74.11546552899996, 40.56480595799997], [-74.11489902599997, 40.56536698599993], [-74.11430477299992, 40.56594740099993], [-74.11376161199998, 40.56649718899993], [-74.11369932899999, 40.56645974399986], [-74.11211239799994, 40.56550555799987], [-74.11013947199989, 40.56431066699985], [-74.10936214499999, 40.56390893799991], [-74.108615941, 40.56352836299992], [-74.1078679589999, 40.56314558799995], [-74.10710400899995, 40.56276988299986], [-74.10690692399996, 40.56266704599994], [-74.10617236999992, 40.56229633299993], [-74.10557645499996, 40.56198997299986], [-74.104846271, 40.56159523899992], [-74.10413152099991, 40.56117322799983], [-74.10391630199994, 40.56102206099994], [-74.10369512999992, 40.5608759759999], [-74.10346821399996, 40.560735110999865], [-74.10323576299994, 40.56059959599989], [-74.10299799599996, 40.56046955799994], [-74.10275513299999, 40.56034511799994], [-74.10250740099998, 40.56022639199988], [-74.10225503199992, 40.56011348999988], [-74.10199826099996, 40.560006518999955], [-74.10173732699991, 40.55990557799989], [-74.09992914200652, 40.55907068582615], [-74.1002122584932, 40.558871151128955], [-74.10079963928099, 40.55832182111587], [-74.10124354139867, 40.557660831315516], [-74.10160202986727, 40.55680345871943], [-74.10184637674693, 40.55621698878], [-74.10184032964895, 40.55594463924127], [-74.10137477595708, 40.55569237210675], [-74.10134593796643, 40.55562567725728], [-74.10135423270516, 40.55558329994345], [-74.10140516910823, 40.555538526164405], [-74.10147440635062, 40.55553361099763], [-74.10265076360245, 40.5561344047852], [-74.10309839111532, 40.555904641451136], [-74.10385795454353, 40.55511816010134], [-74.10465943491393, 40.55412046731118], [-74.10508907947455, 40.55323546801066], [-74.10486249561575, 40.553080018006725], [-74.10484447636338, 40.55302581455626], [-74.10491463759158, 40.55301626711249], [-74.10509470498073, 40.55312902839251], [-74.1060364393664, 40.553640567381144], [-74.10642822065164, 40.553853373621195], [-74.10643031530263, 40.55390124353811], [-74.10642523429996, 40.55394898305795], [-74.10641650990111, 40.55397979168804], [-74.10641430769086, 40.554011266011194], [-74.10641868352174, 40.5540426079087], [-74.10642952644133, 40.554073022618624], [-74.1064465615053, 40.554101738889536], [-74.10647529759494, 40.55412764507523], [-74.1065092241863, 40.55414959740877], [-74.10654741892462, 40.554166999073985], [-74.10658884341498, 40.55417937697288], [-74.1066323714532, 40.554186394588086], [-74.1066768196442, 40.55418786113204], [-74.10704738449654, 40.55419545154716], [-74.10734831283652, 40.55414815203118], [-74.10768623464054, 40.553976671477514], [-74.10853922462532, 40.55325802567331], [-74.10909619942149, 40.55271912189415], [-74.10924476841656, 40.552575370439065], [-74.10924543154377, 40.5525747291735], [-74.1094704941342, 40.5523569644331], [-74.10984807430184, 40.55194528458063], [-74.11028457387536, 40.55149635741752], [-74.11097678212174, 40.55051237768511], [-74.11120326201794, 40.55019042824327], [-74.11166742567612, 40.549515178422254], [-74.11278846979026, 40.54788426261343], [-74.11291619901675, 40.54781765186015], [-74.11301046102231, 40.547780065214994], [-74.11338340608127, 40.54807337507532], [-74.11395149899992, 40.548725322999935], [-74.11444772299996, 40.549169449999944], [-74.11495831999993, 40.54958143899991], [-74.11656290099995, 40.54912774199988], [-74.11813477599993, 40.550592692999906], [-74.11876291999992, 40.55121611399988], [-74.11852157499992, 40.55143427699991], [-74.1194585929999, 40.5522219059999], [-74.12076288199997, 40.55308194999988], [-74.12269895799986, 40.55435850199992], [-74.12380031599992, 40.55508465799988], [-74.12402047099992, 40.555230827999914], [-74.12572675099993, 40.55632805699984], [-74.1272522789999, 40.557265490999896], [-74.12730057799996, 40.55729517199993], [-74.12811697199996, 40.556534435999886], [-74.12896141599992, 40.555769871999864], [-74.12975602299994, 40.555032624999974], [-74.13042459599988, 40.55443704199991], [-74.131086652, 40.5539737589999], [-74.13177828099997, 40.55348997099988], [-74.13316825899993, 40.552356178999915], [-74.13321002499998, 40.552383875999936], [-74.13376761399991, 40.55281510299993], [-74.13498726199998, 40.553810883999844], [-74.13609776299991, 40.55472110499991], [-74.137431255, 40.55581758299994], [-74.13754790999995, 40.555906577999956], [-74.13769985100001, 40.55602734599992], [-74.13885092699991, 40.5569511789999], [-74.13818000999993, 40.55759386599989], [-74.13739437899987, 40.558020936999874], [-74.13660029399993, 40.558388938999926], [-74.13574932399999, 40.55885004499987], [-74.13561300299993, 40.55898002699994], [-74.13549208699995, 40.559120344999876], [-74.13538846199997, 40.55926934199991], [-74.13530363499989, 40.55942508899991], [-74.13523859399992, 40.55958551299993], [-74.13519379599992, 40.55974841799983], [-74.13480973399993, 40.560828618999864], [-74.13396833899998, 40.56158699299987], [-74.13367116299995, 40.56184093399989], [-74.13340740099987, 40.5621298059999], [-74.1331097929999, 40.56286769099993], [-74.13264383999997, 40.563503334999915], [-74.13170009, 40.56453574899991], [-74.13240280499994, 40.564760999999905], [-74.13329816299988, 40.56497397399988], [-74.13459671299997, 40.5652709339999], [-74.13493919999993, 40.56538743599986], [-74.13527417299991, 40.565517760999874], [-74.13560020799989, 40.565661454999926], [-74.1359159259999, 40.56581795999991], [-74.13622008499992, 40.56598659899989], [-74.13651149199997, 40.566166621999876], [-74.13768350999999, 40.56671786699995], [-74.13889993799994, 40.56729303599993], [-74.13976413499992, 40.56771224499988], [-74.14026824299997, 40.56795018899985], [-74.14131233599991, 40.5684333479999], [-74.1415196949999, 40.56852030799994], [-74.1428731119999, 40.56908188399993], [-74.1443874499999, 40.56970609999989], [-74.14402153199993, 40.57033011999992], [-74.14387453599993, 40.570606593999905], [-74.1440387979999, 40.5712994959999], [-74.14418588199999, 40.57203740999988], [-74.141847646, 40.57233261399994], [-74.14110631299992, 40.57242255299991], [-74.1402400869999, 40.57252737199988], [-74.13846597099997, 40.57273988699994], [-74.13813873699996, 40.572779642999905], [-74.13704258199989, 40.57290845199991], [-74.1363534319999, 40.57299130999997], [-74.1357919339999, 40.57305194999985], [-74.13461855699988, 40.5731977129999], [-74.13378794099991, 40.573296974999934], [-74.1333012009999, 40.573372680999924], [-74.13295322899991, 40.573444774999885], [-74.13261022999991, 40.573530162999894], [-74.13227350299991, 40.573628501999885], [-74.13194427599997, 40.57373938299985], [-74.13094048799995, 40.57410601099989], [-74.12930379600002, 40.57467541099988], [-74.12914464399998, 40.57473005399986], [-74.12814797599997, 40.575056538999924], [-74.12784711499992, 40.575167575999885], [-74.12765358299998, 40.57523494299987], [-74.1275212319999, 40.57541053899995], [-74.12752110199995, 40.57541070999993], [-74.12752099599996, 40.57541088199988], [-74.12740306499991, 40.57559308999995], [-74.12740297099995, 40.5755932429999], [-74.12740288899992, 40.57559339599988], [-74.12730007399989, 40.57578144999996], [-74.12730004999996, 40.57578149499989], [-74.12730002599994, 40.5757815399999], [-74.12721306899995, 40.57597415099987], [-74.12721297499998, 40.57597434899988], [-74.12721289299994, 40.57597456599993], [-74.12714229799994, 40.57617048099992], [-74.1271422509999, 40.57617061599995], [-74.12714220399988, 40.57617076799987], [-74.12708814799998, 40.57636910599995], [-74.12665688899989, 40.576284932999926], [-74.12563664099993, 40.575973443999935], [-74.12513333899997, 40.575879987999905], [-74.12408417799992, 40.57583401999993], [-74.12349756499998, 40.57580793399989], [-74.12333972999994, 40.57578701199989], [-74.12333955299995, 40.57578698599991], [-74.12333936499992, 40.57578694099992], [-74.1231838069999, 40.57575185199992], [-74.12318366499987, 40.57575181599987], [-74.12318352299998, 40.57575177099994], [-74.12303336099995, 40.57570277299987], [-74.12303319599995, 40.57570271899991], [-74.12303304199995, 40.57570264699994], [-74.12289135999994, 40.57564058099989], [-74.12289127699994, 40.57564054599992], [-74.12289120599996, 40.57564049999989], [-74.12276083999994, 40.5755667879999], [-74.122760698, 40.5755667069999], [-74.12276057999999, 40.575566625999954], [-74.12264375399991, 40.5754830199999], [-74.12209602099993, 40.575132586999864], [-74.12198585999995, 40.575114666999845], [-74.12198569499998, 40.57511464099983], [-74.12198554099993, 40.575114630999884], [-74.12187198599999, 40.57510757499989]]]}}, {\"id\": \"176\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 177, \"Shape_Leng\": 0.0715805407056, \"Shape_Area\": 0.000198894762248, \"zone\": \"Ocean Hill\", \"LocationID\": 177, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90287079599995, 40.6786314149999], [-73.9023946, 40.67833239799992], [-73.90239451999992, 40.67833234699991], [-73.90236198299992, 40.67831191499987], [-73.90199214699999, 40.678079679999925], [-73.90198538099985, 40.67807543199993], [-73.90187352500006, 40.678005191999866], [-73.90144574099996, 40.67773656199995], [-73.90142169599987, 40.6777214619999], [-73.90140356899998, 40.6777099629999], [-73.90136992499991, 40.67768861999989], [-73.90099766899989, 40.67745246799996], [-73.9005649959999, 40.67717798499989], [-73.90014864499996, 40.67694931599987], [-73.90004626199993, 40.67689308999995], [-73.90079183699987, 40.67657245799992], [-73.90096234799994, 40.67650221199989], [-73.90245583099987, 40.67588204999992], [-73.90257716399984, 40.675831705999926], [-73.90267831799984, 40.6757897289999], [-73.90279745999989, 40.67575749299986], [-73.90291887299995, 40.675684580999864], [-73.90302076399996, 40.675656480999905], [-73.90313803899994, 40.67562221499988], [-73.90347422300006, 40.67550668299985], [-73.90374374599986, 40.67540683399991], [-73.90428786799986, 40.67522935599987], [-73.90438792800002, 40.67520674699993], [-73.90450912299991, 40.67505211099989], [-73.90524012899992, 40.674724945999856], [-73.90582626799984, 40.674469653999886], [-73.90609438799999, 40.674352931999906], [-73.90629662099994, 40.6742653329999], [-73.90695655499991, 40.67398525899991], [-73.90746842999984, 40.673761613999915], [-73.90787711599997, 40.6735668089999], [-73.90827800999982, 40.67337939399993], [-73.9083295419999, 40.673355302999894], [-73.90841931599986, 40.673313332999925], [-73.90925438099987, 40.672947997999856], [-73.90954171999984, 40.67282880299995], [-73.90989833499988, 40.67268086899986], [-73.90995832899995, 40.672655980999906], [-73.91014265099993, 40.6725795189999], [-73.91019287099996, 40.67255868499989], [-73.91037473899996, 40.672483237999884], [-73.910463053, 40.672446601999866], [-73.91104631099985, 40.672204641999954], [-73.91109867399982, 40.67218291999989], [-73.9111599549999, 40.67215535099993], [-73.91133533699998, 40.672076452999896], [-73.91221776899997, 40.67168716299988], [-73.91308574699985, 40.67131022599989], [-73.91403857199998, 40.67089683999989], [-73.9145448559999, 40.67067547299995], [-73.91461857800005, 40.67064323799989], [-73.91487246299994, 40.6705322279999], [-73.91553422600005, 40.67024338399992], [-73.91558467299987, 40.67022136299989], [-73.9157287449999, 40.67015847799994], [-73.91580361899986, 40.670126027999906], [-73.91586213199994, 40.670100668999964], [-73.9164716349998, 40.669836508999886], [-73.91652296199993, 40.66981426399983], [-73.91658459999987, 40.6697875489999], [-73.9171358899999, 40.66954376399992], [-73.91748266100001, 40.66939223099988], [-73.91838191199984, 40.66900224699985], [-73.91923062999997, 40.66863136499991], [-73.91999873699986, 40.6684094839999], [-73.92011464299999, 40.668376001999924], [-73.92014864299998, 40.66822948399992], [-73.9210198879999, 40.667852697999905], [-73.92187171199997, 40.667480689999906], [-73.92221564099988, 40.6673334459999], [-73.92275257199998, 40.667097332999894], [-73.9227271419999, 40.667365055999866], [-73.92265529999987, 40.668127114999905], [-73.92263210799995, 40.66837321099994], [-73.92261354299985, 40.668564106999874], [-73.92254432199987, 40.66927581099986], [-73.92246696799995, 40.6701169299999], [-73.92245355399994, 40.67026114899993], [-73.92238401799992, 40.67100876299986], [-73.92229950400001, 40.67190026199992], [-73.92221444299993, 40.67279163299992], [-73.92213153399979, 40.673683888999946], [-73.92204843299987, 40.67457439699991], [-73.92197495299999, 40.675348546999935], [-73.92190090500002, 40.67613162099988], [-73.92183232600003, 40.67684904199989], [-73.92182360999988, 40.67693835799988], [-73.92181538999996, 40.67701838999991], [-73.92171159899995, 40.67811571399989], [-73.92164666399992, 40.67886992299994], [-73.91905083899984, 40.67872847399986], [-73.91796830599993, 40.67866902199989], [-73.91720270699984, 40.67862681499991], [-73.9163011989999, 40.67857711199991], [-73.91639300100002, 40.678905848999904], [-73.916539987, 40.67963749699986], [-73.91668638399985, 40.680371378999965], [-73.91683011499998, 40.68110098099987], [-73.9169763279999, 40.68183415899995], [-73.91712138499985, 40.682568471999964], [-73.9172663269999, 40.6833032239999], [-73.91741294299999, 40.68403405199993], [-73.91755763299994, 40.68476719199988], [-73.91762630599987, 40.68511814899987], [-73.91763742499997, 40.68517497599987], [-73.91764776599992, 40.68522782299993], [-73.91765638699995, 40.68527188299992], [-73.91770109499986, 40.68550035899995], [-73.91776547599997, 40.68581854099988], [-73.91778426799988, 40.68591141199988], [-73.91783531199984, 40.686163668999875], [-73.91784937499983, 40.686233166999926], [-73.91789200399997, 40.68644915899996], [-73.917915484, 40.68656593599996], [-73.91799580199995, 40.686965380999894], [-73.91804607, 40.68721324799989], [-73.91768113899987, 40.68700409499997], [-73.91755226099991, 40.68693022299989], [-73.916798329, 40.68649719899987], [-73.9166103199999, 40.68639135299993], [-73.91604599299995, 40.68607206899985], [-73.91550033999997, 40.68576152399992], [-73.91528963499992, 40.685645535999875], [-73.91470083399992, 40.68531694499989], [-73.91453972399987, 40.68522691799994], [-73.91379700199988, 40.68479987199986], [-73.91330728099987, 40.6845208579999], [-73.91303721399989, 40.684367809999884], [-73.91227301199991, 40.68393496499991], [-73.911511929, 40.683504989999896], [-73.91076954399978, 40.68309270099994], [-73.91002460999994, 40.68265737099986], [-73.90927462899988, 40.68223432399993], [-73.90905526999994, 40.682105574999895], [-73.90886025599995, 40.681994035999914], [-73.90851642300002, 40.681803942999906], [-73.90777158799989, 40.68137784699997], [-73.90701820299995, 40.68094827999989], [-73.90671941199996, 40.68076742199985], [-73.90628287099989, 40.68050675599993], [-73.90565934199981, 40.68014069099989], [-73.905548368, 40.6800716019999], [-73.90549303499998, 40.68003715799995], [-73.90541416499981, 40.67999264299987], [-73.90479113399992, 40.67964096499992], [-73.90456891299982, 40.67951553699991], [-73.9040463979999, 40.67922059799985], [-73.90402950599994, 40.679213539999914], [-73.90400588299991, 40.67920367099988], [-73.903630796, 40.67904695599991], [-73.90333723699995, 40.678924302999874], [-73.90333666299999, 40.678923942999916], [-73.90324287499999, 40.67886505099995], [-73.9031631289999, 40.6788149779999], [-73.90287079599995, 40.6786314149999]]]}}, {\"id\": \"177\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 178, \"Shape_Leng\": 0.0682528770739, \"Shape_Area\": 0.000175805409143, \"zone\": \"Ocean Parkway South\", \"LocationID\": 178, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97080408199997, 40.62593266699988], [-73.97047171899992, 40.62417002999995], [-73.97019276999991, 40.62420324999987], [-73.96909724499984, 40.62433282499995], [-73.96864510599995, 40.6219416949999], [-73.96757137699979, 40.62206027499997], [-73.96664515699992, 40.62216223699988], [-73.96571874599992, 40.622264028999915], [-73.96471983399981, 40.62237296099993], [-73.96426798299994, 40.619983802999876], [-73.96388935599992, 40.61800054999993], [-73.96381145199987, 40.61758940199997], [-73.96375418699988, 40.61730243299991], [-73.9636224559999, 40.616589736999856], [-73.96356470599983, 40.61623872199992], [-73.9633609399999, 40.615204427999885], [-73.96285973999984, 40.61254948599989], [-73.96385332599989, 40.61243251899986], [-73.96478112799986, 40.61233092199988], [-73.96429649899991, 40.60977642599992], [-73.96522378099998, 40.60967440099992], [-73.96629835499999, 40.609555212999936], [-73.96739979599994, 40.60945960699995], [-73.96768635599992, 40.609417786999906], [-73.96792168899997, 40.60938918999989], [-73.96904086599989, 40.60925319299989], [-73.96996969299995, 40.60915048299991], [-73.9708953839999, 40.60904971899987], [-73.97191331299986, 40.60893592099993], [-73.97299433999983, 40.6088141419999], [-73.97316643599986, 40.60879557699993], [-73.97403097799986, 40.608702309999906], [-73.97418385499991, 40.609463501999905], [-73.974390729, 40.6105792129999], [-73.97451543899999, 40.61125680099987], [-73.97458407499984, 40.61161357599985], [-73.97477658, 40.6126384749999], [-73.97489295799981, 40.61318636799989], [-73.97497264699994, 40.61366810999988], [-73.97517176999993, 40.61472418599991], [-73.97399319500002, 40.61402989399992], [-73.97290326899996, 40.61415296799988], [-73.97335879399992, 40.616541714999904], [-73.97444328399989, 40.61642164299991], [-73.97548096999985, 40.616311924999906], [-73.975442767, 40.616114089999925], [-73.97537335500002, 40.6157547219999], [-73.97562954199996, 40.61594539099984], [-73.97785009299986, 40.617287605999906], [-73.97752301899989, 40.61767654099989], [-73.97682433599995, 40.61867579999989], [-73.97539380499998, 40.62076998699995], [-73.97705352899999, 40.62153252199987], [-73.97695070799993, 40.62163000699997], [-73.97656583899989, 40.621994902999894], [-73.97675209599991, 40.623013311999884], [-73.97683465199992, 40.623481041999895], [-73.976943824, 40.624052749999855], [-73.97710959299997, 40.62494825999989], [-73.97719665599993, 40.62510197899993], [-73.9772614999999, 40.625852772999906], [-73.97623453700001, 40.625976349999895], [-73.975635809, 40.62603520899995], [-73.97507611299997, 40.62611376699993], [-73.97414911500006, 40.626214897999894], [-73.97322001099991, 40.62631854299984], [-73.97229894799983, 40.626431167999876], [-73.97119209799986, 40.6265426959999], [-73.97108160299985, 40.62655370899987], [-73.97092126300004, 40.62656998499993], [-73.97086150900003, 40.626237182999894], [-73.97080408199997, 40.62593266699988]]]}}, {\"id\": \"178\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 179, \"Shape_Leng\": 0.0713808644377, \"Shape_Area\": 0.000183893478476, \"zone\": \"Old Astoria\", \"LocationID\": 179, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93290347544752, 40.77794449857707], [-73.93277477867667, 40.777922388777874], [-73.9327412296532, 40.77799395031759], [-73.93272210453951, 40.778034767117944], [-73.93264931173371, 40.77802197423286], [-73.93184646077484, 40.77788082503058], [-73.93178834305208, 40.77787060660231], [-73.9310496429894, 40.77744189731199], [-73.93107377146733, 40.777260594398136], [-73.93088304263134, 40.776973706679804], [-73.9306285655417, 40.7765909027496], [-73.92988058500097, 40.776221631901876], [-73.92903572143476, 40.77646741705894], [-73.92860515023345, 40.77659267510096], [-73.92839561141334, 40.776790529567684], [-73.92827730816353, 40.77689897516217], [-73.92782147400004, 40.776697526999946], [-73.92737641899997, 40.776466611999865], [-73.92658081899985, 40.7760392589999], [-73.92602208199996, 40.77573039899992], [-73.92531479799992, 40.77534385199994], [-73.92523246199984, 40.77529885399988], [-73.92398142900002, 40.77461511299991], [-73.92279264499986, 40.773962294999905], [-73.92216697499995, 40.77354687799987], [-73.92146878499983, 40.773051306999925], [-73.92076339599988, 40.77255254799995], [-73.920076088, 40.77206690899993], [-73.919421785, 40.77159517499992], [-73.91880312499995, 40.77114938899993], [-73.91870527799998, 40.77107888199988], [-73.91815346699994, 40.77070124299992], [-73.91770217299991, 40.7703931119999], [-73.91811686499992, 40.77002848599989], [-73.91828990899992, 40.76987856799984], [-73.91995596899991, 40.768440064999865], [-73.91911161599978, 40.76802545099991], [-73.91849336999996, 40.76773465699995], [-73.918906013, 40.76722677599987], [-73.91977764799998, 40.7661521839999], [-73.92058727399994, 40.76652069899995], [-73.92139066099995, 40.76689410499988], [-73.92218946199995, 40.767267356999895], [-73.92353865299998, 40.76560349799989], [-73.92425288799978, 40.76472456399995], [-73.92506062099999, 40.76509731499991], [-73.92643792699984, 40.765732814999936], [-73.92855636599988, 40.76671423899995], [-73.9298564789999, 40.76732655199985], [-73.93202371699994, 40.768256495999886], [-73.93344468499993, 40.768817789999915], [-73.93422865800001, 40.767854682999875], [-73.93493511099992, 40.76701302399987], [-73.93594942700004, 40.76757822599996], [-73.93618379699991, 40.767691800999934], [-73.93593177099983, 40.7677837029999], [-73.93582070900001, 40.76863091499992], [-73.9357713409999, 40.76905000999985], [-73.93522718199998, 40.76952572999988], [-73.9350398979999, 40.76968230199987], [-73.934869397, 40.76985073499985], [-73.93486925499987, 40.76985087799988], [-73.93486912400003, 40.76985103099991], [-73.93471718499994, 40.77002985799987], [-73.93458521699996, 40.770217420999906], [-73.93447449099997, 40.77041159699994], [-73.93424609600005, 40.771029275999894], [-73.93452730104707, 40.771176775051664], [-73.9344854830391, 40.77130652321843], [-73.93485790974898, 40.77138291296914], [-73.93487018163421, 40.77135427855315], [-73.93495398714984, 40.771372943016836], [-73.9349294331986, 40.77144023609009], [-73.93483715299811, 40.77142013530973], [-73.93484282697513, 40.771395793192276], [-73.93448015089042, 40.771323068200275], [-73.93443525743162, 40.771462362333395], [-73.93482622761688, 40.771576958073254], [-73.93480858897101, 40.77169958689862], [-73.93480681344163, 40.7717341388899], [-73.9351221671339, 40.77182525833333], [-73.93605335028684, 40.77210580162643], [-73.93746638537067, 40.77253149512518], [-73.93781335969511, 40.77356289464787], [-73.93785811822775, 40.77408756897306], [-73.93765483090819, 40.775085685383594], [-73.93704702153673, 40.775866487041334], [-73.93703012839919, 40.77589320499802], [-73.9363802281754, 40.77692145656271], [-73.93608919699105, 40.77718432207706], [-73.93585452641493, 40.777211684624234], [-73.93550938863626, 40.77753910328639], [-73.93508232492144, 40.77794423372776], [-73.93474353011273, 40.77807633537696], [-73.93441179004596, 40.778092545767635], [-73.93390840247848, 40.778117129818426], [-73.93290347544752, 40.77794449857707]]]}}, {\"id\": \"179\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 180, \"Shape_Leng\": 0.0805277671523, \"Shape_Area\": 0.000247682193353, \"zone\": \"Ozone Park\", \"LocationID\": 180, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84119605699982, 40.68335136599989], [-73.84012856099982, 40.6811466509999], [-73.8395206529999, 40.679961868999946], [-73.83794250999986, 40.676794208999866], [-73.83811946699984, 40.67662988899986], [-73.83784372299996, 40.67607812399992], [-73.83749571700004, 40.67540682399994], [-73.83713760399998, 40.67466513399992], [-73.83680894799993, 40.67396111699987], [-73.83641423899985, 40.673115588999956], [-73.83600752799984, 40.67226955899987], [-73.83527457099994, 40.6708004029999], [-73.83334819299995, 40.66677318699994], [-73.83464965599987, 40.667115307999936], [-73.83487654300001, 40.6672116649999], [-73.83528941499992, 40.667235978999955], [-73.83570108799996, 40.667260221999854], [-73.83667829499991, 40.667586491999906], [-73.83730072099985, 40.66770735899994], [-73.83744908699992, 40.667798349999885], [-73.83834358199996, 40.66798902599986], [-73.83927850999994, 40.66820184899992], [-73.84008991099994, 40.66843655599991], [-73.84013623800001, 40.66826802499991], [-73.84117399199987, 40.6685074429999], [-73.84215802199992, 40.66870144799986], [-73.84227237899998, 40.66873604599992], [-73.84229542199994, 40.66881391799989], [-73.84243585399989, 40.66928855699992], [-73.84258049099998, 40.66928053299986], [-73.84271367699986, 40.66927829299993], [-73.84288589399995, 40.66928507599986], [-73.84305613800001, 40.66930593799987], [-73.84322150199992, 40.66934041099991], [-73.84556670099984, 40.67008464499985], [-73.84593839099993, 40.67017897699987], [-73.84741378199982, 40.67062999499992], [-73.84787000899989, 40.67077949999991], [-73.84847765299996, 40.67091673899989], [-73.85096453700001, 40.671719510999885], [-73.85163460999999, 40.671930332999835], [-73.85283803899996, 40.6722791889999], [-73.85648739299987, 40.6733767739999], [-73.85711349199988, 40.673575054999915], [-73.85825153699983, 40.6741356109999], [-73.85928317999998, 40.674452715999934], [-73.85959262199998, 40.674556409999894], [-73.86029565599982, 40.67477189399992], [-73.86131863599985, 40.6750212689999], [-73.86139775899983, 40.67533658999988], [-73.86171238699991, 40.67661059599987], [-73.86202843099984, 40.67788611399987], [-73.86225943099994, 40.67881678599989], [-73.86234580499983, 40.67916478599994], [-73.86296470699996, 40.67911281699992], [-73.86328640800004, 40.67907719699985], [-73.86330964200002, 40.67916796999987], [-73.86334943599995, 40.67935163999989], [-73.86229626199986, 40.679631579999906], [-73.86156047099999, 40.679839967999875], [-73.861423295, 40.67987881299996], [-73.86066198699996, 40.680097537999856], [-73.86053300099987, 40.680134447999905], [-73.85977462599983, 40.6803507119999], [-73.85966271799995, 40.68038237199995], [-73.85929512400006, 40.680486365999855], [-73.85878576899987, 40.68063046299989], [-73.85819836099995, 40.67943084899987], [-73.85779268399989, 40.67945571699988], [-73.85721078599984, 40.67948586399992], [-73.85683137000001, 40.67950933799989], [-73.85653410299985, 40.67952772499995], [-73.85439330799991, 40.67966011399988], [-73.85392389199987, 40.679686155999875], [-73.85267726099994, 40.68007941499991], [-73.85223417099996, 40.68020789099992], [-73.85168205499998, 40.68036797699993], [-73.85099089299987, 40.68056836699987], [-73.85131773799996, 40.681240015999904], [-73.851639288, 40.68188627199993], [-73.85197864599989, 40.682564210999864], [-73.85087162799985, 40.68225954199993], [-73.85060478199985, 40.68219378299992], [-73.8497640749999, 40.681986613999925], [-73.84921620599982, 40.68184421099987], [-73.84863383999999, 40.68168402799994], [-73.84786817599985, 40.681486831999905], [-73.84757845099998, 40.68157986799987], [-73.84671464299996, 40.68185849299988], [-73.84583237399983, 40.68210644999986], [-73.84529042399988, 40.68223044499989], [-73.84520200299993, 40.68225066799988], [-73.84513349899997, 40.68226653299996], [-73.84505837899985, 40.682284450999866], [-73.84496190700004, 40.68230745499992], [-73.84451063800005, 40.68241508799991], [-73.84408914999997, 40.68252929899992], [-73.84323090399981, 40.682775196999884], [-73.84237029099977, 40.683018421999904], [-73.84151391999985, 40.683261680999884], [-73.84134091999981, 40.68331050299987], [-73.84119605699982, 40.68335136599989]]]}}, {\"id\": \"180\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 181, \"Shape_Leng\": 0.0895372399547, \"Shape_Area\": 0.000306890292807, \"zone\": \"Park Slope\", \"LocationID\": 181, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97583132299987, 40.66581507999991], [-73.97633386199989, 40.66521360399989], [-73.97684516799998, 40.66460638199993], [-73.97734252999986, 40.66401189699998], [-73.97784275099997, 40.66340176899994], [-73.97834945199986, 40.66279901499991], [-73.97885475899994, 40.66220183299989], [-73.97935059899989, 40.66159482099989], [-73.97965353499985, 40.661243232999915], [-73.97970643699993, 40.66126063599991], [-73.97976200499983, 40.66127225999991], [-73.97981915699984, 40.66127787799995], [-73.97987678099987, 40.661277381999895], [-73.97993375199998, 40.661270780999864], [-73.97998896000001, 40.66125820299989], [-73.98004133199993, 40.66123989299993], [-73.98008984499991, 40.66121620899992], [-73.98013355499987, 40.66118761199988], [-73.98017161099993, 40.66115465799992], [-73.98223034199995, 40.66214149299991], [-73.98273252399984, 40.66153948399988], [-73.98323712099993, 40.660936220999886], [-73.98391792699988, 40.66013748799985], [-73.98448024099987, 40.659598336999935], [-73.985086794, 40.65901328799989], [-73.98648200899994, 40.659909879999915], [-73.98660379800002, 40.66001202199991], [-73.98671251899984, 40.66012320299989], [-73.9868065409999, 40.66024190199991], [-73.9868846449999, 40.6603663889999], [-73.98713178899986, 40.660374367999935], [-73.98725068999993, 40.66037895599985], [-73.98729619199992, 40.660441852999895], [-73.98804399099996, 40.6614754359999], [-73.988716511, 40.66240493799995], [-73.9887680069999, 40.66235596399985], [-73.98888159799985, 40.66224717499995], [-73.98947000099999, 40.66169904099992], [-73.99005216500004, 40.66113864799992], [-73.9906354539999, 40.66057908799997], [-73.99156751399993, 40.66114158699991], [-73.99283830599988, 40.66190848099993], [-73.99502082499983, 40.66320906699988], [-73.99510907599998, 40.66326211199989], [-73.99521061899985, 40.66332127999991], [-73.99692689899985, 40.664374914999925], [-73.9963724839999, 40.66496190899988], [-73.99605311599986, 40.66528516599988], [-73.99593232999982, 40.66540742499992], [-73.99583369399986, 40.66550726199988], [-73.995810253, 40.665530988999926], [-73.99595866299985, 40.665704548999855], [-73.9958739449999, 40.66573228499992], [-73.99575957400002, 40.66578388799989], [-73.99530800399985, 40.66615799899995], [-73.9951247889999, 40.666381354999885], [-73.99502872600004, 40.666515284999946], [-73.99498642699994, 40.66662582999985], [-73.99496779899978, 40.66666374099987], [-73.99494918099985, 40.666701642999904], [-73.99490350199993, 40.66678543499988], [-73.99284786000001, 40.66555439199987], [-73.99277080699981, 40.665507280999876], [-73.9922346229999, 40.66613762499993], [-73.99173365199998, 40.66673659499987], [-73.99123399599989, 40.6673356529999], [-73.99072558999995, 40.66794617299989], [-73.99022252199988, 40.668548097999896], [-73.98972203499986, 40.66914780699988], [-73.98921685399992, 40.669753474999915], [-73.98907079799996, 40.66992861999993], [-73.9890700769999, 40.66992948399993], [-73.988713226, 40.67035690699993], [-73.98821210799997, 40.67096081399996], [-73.98771575999984, 40.671555380999976], [-73.98744496399998, 40.67188008599991], [-73.98721100499988, 40.67216061599988], [-73.98670905399989, 40.67276673299988], [-73.98622500899995, 40.67334636499989], [-73.98570017199982, 40.67397483299985], [-73.98520256000002, 40.67457380999991], [-73.98469872999982, 40.67517562999993], [-73.98420294599991, 40.67577119999985], [-73.98390048699991, 40.67614865299992], [-73.98347361499994, 40.67677916099998], [-73.98304811899982, 40.677406685999834], [-73.98261725399998, 40.678042429999905], [-73.98218939700001, 40.67867580399995], [-73.98175642299985, 40.67931462399987], [-73.98132413699994, 40.67995210099991], [-73.98088932299994, 40.6805933489999], [-73.98046155400003, 40.68122254699992], [-73.98003510099997, 40.68185262699988], [-73.97961592499983, 40.682493556999916], [-73.97918390599999, 40.68312883799991], [-73.97876022499986, 40.6837654439999], [-73.97842970699993, 40.684355612999866], [-73.97838862599984, 40.68442334199987], [-73.97804289599988, 40.68485661499987], [-73.97745722099988, 40.68405228699988], [-73.97709796699996, 40.6838697799999], [-73.97682267999997, 40.68375383799986], [-73.97676303199997, 40.68373178199996], [-73.97653928699988, 40.683649050999854], [-73.97624898299983, 40.68355587899987], [-73.97595299799993, 40.68347469799988], [-73.97483191299989, 40.68323523699998], [-73.97375598499991, 40.68303401999988], [-73.97374011799997, 40.68289203199989], [-73.973845731, 40.682586386999915], [-73.97395135399996, 40.68228072999996], [-73.97422671399997, 40.68150463899983], [-73.97439144199983, 40.68104616999989], [-73.9745039589999, 40.68073320299992], [-73.97469118399997, 40.68026661599992], [-73.97490473799986, 40.67995413099986], [-73.97495737999984, 40.679877112999925], [-73.97532192699995, 40.6793327759999], [-73.97574929499986, 40.67869970599991], [-73.97618051399984, 40.67806432099995], [-73.973686958, 40.67707800199987], [-73.97188596799982, 40.676394652999925], [-73.96949738899991, 40.67590637999996], [-73.96969317499985, 40.67536085399989], [-73.96954563799989, 40.67527700499989], [-73.96954556799999, 40.67527696899988], [-73.96940821999982, 40.67518167599996], [-73.96940813699987, 40.67518162199986], [-73.96928310699997, 40.67507596699993], [-73.96917236899992, 40.67496145599997], [-73.96917233299985, 40.67496140999985], [-73.96917229799983, 40.67496136499993], [-73.96907739, 40.67483976099988], [-73.96907735399995, 40.67483971599988], [-73.9690773179999, 40.674839670999916], [-73.96899918499997, 40.674712775999886], [-73.96891162400006, 40.674563054999915], [-73.96883910899994, 40.67440796699988], [-73.96878240699998, 40.67424884999986], [-73.96874205, 40.674087121999904], [-73.96871822199984, 40.67392423399988], [-73.96871164100001, 40.67380499399991], [-73.96871337599998, 40.67368565599987], [-73.96872342099994, 40.67356655699992], [-73.968741747, 40.67344802999989], [-73.96876830399998, 40.67333040999996], [-73.96880301499984, 40.6732140279999], [-73.96884578399981, 40.67309921099997], [-73.96889649099982, 40.67298628299992], [-73.96892332199981, 40.67293755299988], [-73.96927327499998, 40.67279742699989], [-73.96966394100005, 40.67264236699988], [-73.97000244600004, 40.67263114999985], [-73.9704128949999, 40.6725937229999], [-73.97085440699995, 40.67186436499991], [-73.9713252639998, 40.671248223999946], [-73.97181594699985, 40.67062947499989], [-73.97231720899988, 40.67002909799988], [-73.97281990699993, 40.669427177999864], [-73.97332284299985, 40.66882492299992], [-73.97382528499993, 40.66822308799986], [-73.97433093599996, 40.66762123499995], [-73.97483327599997, 40.66702209799992], [-73.97533714599997, 40.666419095999906], [-73.97583132299987, 40.66581507999991]]]}}, {\"id\": \"181\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 182, \"Shape_Leng\": 0.0393606369479, \"Shape_Area\": 9.10644156732e-05, \"zone\": \"Parkchester\", \"LocationID\": 182, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85465135700002, 40.83388802699987], [-73.85773226199993, 40.83355730699993], [-73.8590567219999, 40.833413224999894], [-73.85983937599993, 40.83334089599988], [-73.86033845899995, 40.833253165999935], [-73.86037967100003, 40.83333610299992], [-73.8604271769999, 40.83340383299985], [-73.86049053499993, 40.833466403999864], [-73.8605680069999, 40.83352073499988], [-73.86065651499995, 40.833564271999926], [-73.86075203199984, 40.833595374999916], [-73.86085019699983, 40.833613537999874], [-73.86129183699992, 40.833536420999934], [-73.86137654799992, 40.83349072299995], [-73.86142308999996, 40.83344741099989], [-73.86155834899989, 40.83333022099989], [-73.86271114399993, 40.83299150899997], [-73.86272700700006, 40.833072220999945], [-73.86284851199984, 40.833690230999885], [-73.86288113299992, 40.833889072999916], [-73.86289361599997, 40.83396426099987], [-73.86289761399996, 40.83400338399992], [-73.862942603, 40.83429960099993], [-73.86295952099994, 40.834420322999925], [-73.863156384, 40.835530821999924], [-73.86355551599988, 40.83786561999994], [-73.86393748099981, 40.84004456599994], [-73.86448011799995, 40.84108779699994], [-73.86436922099999, 40.84110901799995], [-73.863639382, 40.8412652749999], [-73.86358013999998, 40.84127795699989], [-73.86347232399994, 40.84130593299991], [-73.86176166600002, 40.841512719999926], [-73.86162833399996, 40.84152785999986], [-73.86045627399983, 40.841661664999954], [-73.86018491899982, 40.84168908999986], [-73.8597245239999, 40.84174401899984], [-73.85919593699994, 40.841804032999924], [-73.85835669199989, 40.84189928599988], [-73.85719247999995, 40.84202764799989], [-73.85618402199997, 40.84214591699992], [-73.85553579799995, 40.84221948999986], [-73.85520012799999, 40.84225761399988], [-73.85420745799989, 40.84237073899991], [-73.85349746599992, 40.84245166799994], [-73.85333524999986, 40.84247015499987], [-73.85309460199986, 40.8410469599999], [-73.85303510299988, 40.840735457999884], [-73.85286997899986, 40.839851303999936], [-73.85266480499992, 40.8387800839999], [-73.85247938999986, 40.83778428699986], [-73.85233148699994, 40.83701052699997], [-73.85220720999986, 40.836347494999934], [-73.85207005899989, 40.83562530299987], [-73.85185868799985, 40.834777746999876], [-73.85168459699997, 40.8342010519999], [-73.85188599699991, 40.83417942099992], [-73.85281379799993, 40.83407976199991], [-73.85347099099992, 40.83401491699993], [-73.85465135700002, 40.83388802699987]]]}}, {\"id\": \"182\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 183, \"Shape_Leng\": 0.0398262960348, \"Shape_Area\": 9.51929451066e-05, \"zone\": \"Pelham Bay\", \"LocationID\": 183, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83676643699985, 40.84338596999994], [-73.83688925199996, 40.84337478799992], [-73.83676876799983, 40.843542820999886], [-73.8365331269999, 40.84462069499994], [-73.83619907199994, 40.84582739599996], [-73.83615362999997, 40.84613117599993], [-73.83606934399998, 40.84716484899992], [-73.83593362199987, 40.84840635599993], [-73.83580031399995, 40.8496768159999], [-73.83555900199993, 40.85196401299994], [-73.83545671599992, 40.85226119199988], [-73.83533589099999, 40.85255469799993], [-73.83519691899983, 40.85284353099991], [-73.83504028899982, 40.853126736999876], [-73.83480132399997, 40.85349832599997], [-73.83473123599998, 40.853660186999896], [-73.83463970299987, 40.853817993999904], [-73.83452741999979, 40.85396932399989], [-73.83439574299983, 40.85411187099993], [-73.83424656499982, 40.854243551999915], [-73.83408232099997, 40.8543626399999], [-73.83390574099988, 40.854467779999915], [-73.83350017899996, 40.85467968399993], [-73.83295928499994, 40.85492410599992], [-73.832624647, 40.855056966999925], [-73.83228148899984, 40.85517744999994], [-73.83193102199998, 40.85528511599992], [-73.83157449000001, 40.855379596999946], [-73.83129157799985, 40.8554341049999], [-73.83056649299975, 40.85551244799992], [-73.82956640599991, 40.85562081699991], [-73.82908318700004, 40.8556760139999], [-73.82860197199996, 40.85571784799988], [-73.8280141599998, 40.85472248699992], [-73.82723026100001, 40.853350756999895], [-73.82721751200002, 40.853250512999935], [-73.82737835799986, 40.8528179589999], [-73.82713133299993, 40.85187108099993], [-73.82718423799987, 40.85171120499995], [-73.82742445699994, 40.850993646999946], [-73.82765823699988, 40.85027403399991], [-73.82776655799988, 40.85000758699991], [-73.82785470899994, 40.8497359229999], [-73.82792209199984, 40.84946036399986], [-73.82796831299994, 40.8491823059999], [-73.82799320299986, 40.84890315399986], [-73.82799676999998, 40.84862431399989], [-73.827677297, 40.848130680999894], [-73.82688311099984, 40.84690351499993], [-73.82672302600005, 40.84669743899988], [-73.826578403, 40.846483722999906], [-73.82645014900007, 40.84626339599992], [-73.82634836399981, 40.846056569999845], [-73.82633899899987, 40.846037546999916], [-73.8262455299999, 40.84580733899987], [-73.8261701409999, 40.845573987999934], [-73.82611304199997, 40.845338855999906], [-73.82611300699996, 40.84533869399991], [-73.82674715099992, 40.84507891799988], [-73.8275753599998, 40.84473187499988], [-73.8284527429999, 40.84445071699992], [-73.82870970999984, 40.84437387199991], [-73.8293154839999, 40.84419272599995], [-73.8296554989998, 40.844089122999925], [-73.83043186399985, 40.84385798399996], [-73.83108377099984, 40.843744568999924], [-73.83200795799995, 40.843630393999916], [-73.83294076399983, 40.84351915999992], [-73.83383204399993, 40.84351641299986], [-73.83474813499993, 40.84350280799986], [-73.83582911399996, 40.84347129499987], [-73.83676643699985, 40.84338596999994]]]}}, {\"id\": \"183\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 184, \"Shape_Leng\": 0.260815683043, \"Shape_Area\": 0.00198883363103, \"zone\": \"Pelham Bay Park\", \"LocationID\": 184, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.80728647099995, 40.885931813999875], [-73.80693113299988, 40.88592660899994], [-73.80657730199997, 40.88593752499988], [-73.80635884999995, 40.88594565899991], [-73.80614087499991, 40.88596540999986], [-73.80592498899998, 40.88599671799993], [-73.80571277199984, 40.8860393679999], [-73.80550576700001, 40.886092948999874], [-73.80520113699998, 40.88599995599985], [-73.803286821, 40.885415552999916], [-73.79937520399986, 40.88435196999995], [-73.79744825599991, 40.88384538399988], [-73.79449254399985, 40.88306825099994], [-73.79416555799988, 40.88299321599987], [-73.79322870891143, 40.88282259495222], [-73.79358741929856, 40.88251752388317], [-73.79379237490441, 40.881970326764964], [-73.79491705586672, 40.880543790984866], [-73.79554579908853, 40.880691015374964], [-73.79502596232832, 40.87974779721171], [-73.7962509753578, 40.87701199977848], [-73.79712701174678, 40.87619040090393], [-73.7972039970304, 40.87578579298386], [-73.79743815457286, 40.87513533295314], [-73.7985533078771, 40.87440142827448], [-73.7989124546362, 40.87464589314291], [-73.79981389580328, 40.87415984971623], [-73.79987713793564, 40.873484667292644], [-73.80027274375148, 40.87282095182416], [-73.80087270458245, 40.87159011989393], [-73.8013723540985, 40.87146104310755], [-73.80252284756597, 40.87050664469679], [-73.80317726501715, 40.871683155070606], [-73.80269939786251, 40.872214691294246], [-73.80339117523091, 40.87250503763108], [-73.80431849468228, 40.87115974106797], [-73.80373339399283, 40.87096260282379], [-73.80358314939251, 40.86994806102857], [-73.8028295989319, 40.87007629403041], [-73.80302203685277, 40.86915511979648], [-73.80422027051749, 40.869002148168136], [-73.80483122145964, 40.86931318178597], [-73.80554495995769, 40.86895600501215], [-73.80588410012888, 40.869068502184916], [-73.8057448608033, 40.869692576552794], [-73.80841596818787, 40.87118969336752], [-73.80838731046325, 40.87057162161172], [-73.80754063636172, 40.87037755217338], [-73.80780673194735, 40.86978752042846], [-73.80695275224407, 40.86965794817595], [-73.80673074905101, 40.868770013078354], [-73.80593179559793, 40.868371932067845], [-73.80487266169933, 40.86823330283225], [-73.80464604502912, 40.86664711643685], [-73.80385280781444, 40.86716648480004], [-73.80300388108948, 40.86704183656889], [-73.8030604148454, 40.86581513370676], [-73.80225816157765, 40.8656082064109], [-73.80069778002715, 40.86808147781558], [-73.79857728261854, 40.87119632057149], [-73.79780377219292, 40.87225613227397], [-73.79693375040256, 40.87262338150214], [-73.794790446816, 40.873200484033966], [-73.79397363837722, 40.874566594568385], [-73.79357210883583, 40.87612105830348], [-73.79359605330511, 40.8767207327259], [-73.79316703778706, 40.87705563906647], [-73.79422169990922, 40.877382370466336], [-73.79440294586227, 40.87764184299902], [-73.79384702788171, 40.87859445979086], [-73.79299880220005, 40.87914686267462], [-73.79288910674136, 40.8792640928223], [-73.79258921077623, 40.87936100860248], [-73.79218928228387, 40.879511883286014], [-73.79188890967126, 40.87976039082595], [-73.79174583941239, 40.879890071652405], [-73.7914602395418, 40.879976180765674], [-73.79127474265381, 40.87998667328982], [-73.79090368014784, 40.880029315198286], [-73.79068927478805, 40.880158864278194], [-73.790574766816, 40.88027776709313], [-73.79048858865667, 40.88046168983179], [-73.79047377660959, 40.88063491373199], [-73.79043046503072, 40.88079725802921], [-73.79023015226491, 40.88098097375842], [-73.79005864434492, 40.8810781162332], [-73.78986765156738, 40.881169184109275], [-73.78978716776687, 40.881207560463345], [-73.7895587294431, 40.881261284915404], [-73.78926161019805, 40.881205414623615], [-73.78931835780448, 40.880578670091815], [-73.78912020636906, 40.88008021099344], [-73.79006332860774, 40.87959466151484], [-73.789523093688, 40.87899812706048], [-73.78827313470907, 40.87887357315418], [-73.78775415489733, 40.87897323005543], [-73.78741169652315, 40.878994257129385], [-73.78712640734612, 40.87898290341336], [-73.78694087806699, 40.87900421726645], [-73.78673946461805, 40.879031707195836], [-73.78588496976647, 40.879067231752636], [-73.7843816810755, 40.878420531670514], [-73.7841773836883, 40.877721353928486], [-73.78470964026441, 40.87635799484768], [-73.78542459134646, 40.87548530202441], [-73.78596895445062, 40.8751367501274], [-73.78668493218761, 40.87429347554253], [-73.7870867847903, 40.8735362422161], [-73.78715932296441, 40.873157389360145], [-73.78719087560506, 40.872730553729134], [-73.78679051717079, 40.872507015313424], [-73.78624874452605, 40.87238284677213], [-73.7859914785657, 40.87257050546378], [-73.78573128788767, 40.87280383386965], [-73.78544759969013, 40.873132563370994], [-73.7852750909762, 40.87354371380044], [-73.78508139658601, 40.87358996322568], [-73.78517279045143, 40.87311937546341], [-73.78568099853844, 40.872387265057085], [-73.78555929452179, 40.87215276669471], [-73.78526152075709, 40.87174532233236], [-73.78439311743477, 40.872784094928754], [-73.7840215783953, 40.87369242209927], [-73.78380448210427, 40.873736688343506], [-73.78358738486025, 40.87378095468029], [-73.78333967922703, 40.873512621378566], [-73.78343142541378, 40.87317870194881], [-73.78403716971833, 40.87282660126358], [-73.7838090688668, 40.87253395291509], [-73.78342458843244, 40.8723274962909], [-73.78306853426027, 40.87215357592268], [-73.7830557962405, 40.87168067823897], [-73.78314218899503, 40.87143905270971], [-73.78324264074435, 40.87125516094618], [-73.78324331138717, 40.87104942747412], [-73.78318664230073, 40.8709302097453], [-73.78320175426292, 40.87067036085376], [-73.78350179966235, 40.87051932877749], [-73.78389734052207, 40.870426958369556], [-73.78391578371762, 40.870422651371904], [-73.78414454020398, 40.870260656235736], [-73.7842880274107, 40.87000104674132], [-73.78465917845698, 40.869915114965124], [-73.78490216077769, 40.869763973240644], [-73.78495998839328, 40.86952585943925], [-73.7852180023888, 40.86913652413008], [-73.78534746253581, 40.86880109054052], [-73.78560245305444, 40.86834165222368], [-73.78557645524876, 40.868563295841604], [-73.78551838553852, 40.86887720666757], [-73.78551744254952, 40.86916956571913], [-73.78568791368025, 40.86938644779148], [-73.78598679893318, 40.86959273828636], [-73.78657103927205, 40.86977789988556], [-73.78726970836335, 40.86986581499132], [-73.78795446145405, 40.86984541906357], [-73.78853953524599, 40.86977069609062], [-73.78901059796526, 40.869663275452666], [-73.78968197448623, 40.86936131077263], [-73.79038207692929, 40.86899442562479], [-73.79069663688344, 40.86875677621223], [-73.79139032586131, 40.868141307466146], [-73.79228914014455, 40.865814377219436], [-73.79265095767063, 40.86413665318477], [-73.79147317119875, 40.862207106249], [-73.79188014597733, 40.86125563220965], [-73.79190454927502, 40.86082956271207], [-73.79208362124983, 40.86077936805869], [-73.79226563531348, 40.86073569737327], [-73.79245017939147, 40.86069864952371], [-73.79263683568298, 40.86066830838418], [-73.7928251816082, 40.86064474264513], [-73.79301479076308, 40.860628005657965], [-73.7932052338849, 40.86061813531404], [-73.79332560925523, 40.86060616885451], [-73.79351029757724, 40.86055168129084], [-73.79358268830747, 40.86049090242635], [-73.79362302791495, 40.86041788353875], [-73.79363962082937, 40.86024127816975], [-73.79355967957899, 40.86014977213991], [-73.79355684101755, 40.86014838482353], [-73.79335942097596, 40.86005195896525], [-73.79339985516697, 40.859948486200935], [-73.79364088707253, 40.859833191400895], [-73.79379355287108, 40.85979623624946], [-73.79386568469461, 40.85977877482917], [-73.79401054069923, 40.859632852092865], [-73.79409935372796, 40.859450285063986], [-73.7941217124408, 40.85931765680062], [-73.79425610311195, 40.85917063977191], [-73.79439746946176, 40.859027443657986], [-73.79454562318031, 40.85888825916121], [-73.79470951415547, 40.858482315190734], [-73.794869838669, 40.85807555125805], [-73.79482082544892, 40.857893757266375], [-73.7947650749976, 40.85771308581656], [-73.79470263177444, 40.85753368081173], [-73.79463354556832, 40.857355685145066], [-73.7945578714582, 40.857179240586575], [-73.79447566976818, 40.85700448766908], [-73.79443324794623, 40.85698632143939], [-73.79439424825077, 40.856964171093466], [-73.79435931592499, 40.85693840310721], [-73.79432902891645, 40.85690944380959], [-73.79430388831402, 40.85687777232901], [-73.79428256679108, 40.85684019970247], [-73.79426848548165, 40.85680070960694], [-73.79426192953805, 40.85676010176765], [-73.79426303171186, 40.85671919854404], [-73.79427176966915, 40.85667882827606], [-73.79428796644193, 40.856639808509314], [-73.7942939635105, 40.85660753428865], [-73.79430448203574, 40.85657593402549], [-73.79435005934273, 40.85652702380392], [-73.7944721328208, 40.85646873133395], [-73.79451792446211, 40.8564520297211], [-73.79456667191796, 40.85644117384735], [-73.79461709047264, 40.85643644981377], [-73.79466785137188, 40.85643798211963], [-73.79471761683845, 40.8564457303821], [-73.79476507532874, 40.85645949039967], [-73.79480897609665, 40.85647889953374], [-73.79489023542949, 40.85656202421893], [-73.79558880172945, 40.85651323676668], [-73.79685461759917, 40.8562643582134], [-73.79700108875329, 40.85614016952222], [-73.79714070961904, 40.856011512603175], [-73.79727324402178, 40.85587860512185], [-73.79739846777979, 40.85574167193426], [-73.79751616908338, 40.85560094470548], [-73.79762614885384, 40.85545666151794], [-73.79772822107861, 40.85530906646866], [-73.79782221312759, 40.8551584092559], [-73.79790796604266, 40.855004944756715], [-73.79798533480839, 40.85484893259572], [-73.79824950227362, 40.85479598510957], [-73.79853274127018, 40.85500533260181], [-73.7987225560008, 40.85491440117259], [-73.79904776834826, 40.854051560550324], [-73.7990241939332, 40.85395726765013], [-73.79899253393481, 40.853864350104196], [-73.79895292443402, 40.85377320719197], [-73.7989055356695, 40.85368423056506], [-73.79885057130656, 40.85359780256494], [-73.79878826756028, 40.853514294579924], [-73.79871889218126, 40.85343406544942], [-73.79864274330347, 40.85335745992206], [-73.7985601481648, 40.8532848071745], [-73.79843216214293, 40.85323433361904], [-73.7983074626901, 40.85317932120461], [-73.79818632879257, 40.853119893012085], [-73.79806903145645, 40.8530561820021], [-73.79795583310242, 40.85298833071622], [-73.79780522990592, 40.85194959859714], [-73.7981936599684, 40.850592561406664], [-73.7993204316021, 40.849912775376495], [-73.79940672091065, 40.84983242018016], [-73.79948681481507, 40.849748448064666], [-73.79956045098254, 40.84966113408999], [-73.79962738823653, 40.84957076426158], [-73.79968740734603, 40.84947763459341], [-73.79974031174334, 40.8493820501384], [-73.79978592816823, 40.84928432398887], [-73.79982410723423, 40.849184776251136], [-73.79985472391844, 40.84908373299661], [-73.7998776779705, 40.848981525193935], [-73.79988337043154, 40.84891083980215], [-73.79989651109994, 40.848840728886024], [-73.79991618964858, 40.848773995563654], [-73.799942616454, 40.848708620252815], [-73.79997376027083, 40.84864824934954], [-73.8000106592219, 40.84858977848549], [-73.80005311397095, 40.84853352350942], [-73.80009912704746, 40.84829841440252], [-73.80012554455554, 40.84826373787393], [-73.8001566945148, 40.84823140574045], [-73.80019221958115, 40.84820178890101], [-73.80023888568967, 40.84817098129821], [-73.80029026716008, 40.848144856343545], [-73.80034555421923, 40.848123825764596], [-73.80040387554423, 40.84810822100142], [-73.80046431199588, 40.848098287983504], [-73.80058890199892, 40.8481403033575], [-73.80070945952177, 40.84818862865633], [-73.80082543115157, 40.848243042048736], [-73.80093628452296, 40.84830329375673], [-73.80104151076104, 40.84836910720121], [-73.80114062681764, 40.84844018027172], [-73.80123317768903, 40.848516186713255], [-73.80131873850269, 40.848596777623065], [-73.80139691647074, 40.84868158305278], [-73.80146735269054, 40.84877021370569], [-73.80152972379491, 40.84886226272415], [-73.80158374343453, 40.84895730755684], [-73.80162916359536, 40.84905491189835], [-73.80166577573556, 40.84915462769178], [-73.80169341174509, 40.849255997185665], [-73.80171194471764, 40.849358555034705], [-73.80172128953284, 40.84946183043671], [-73.80172140324993, 40.84956534929296], [-73.80171228530318, 40.84966863638507], [-73.80172173365993, 40.84976281463484], [-73.80173835351495, 40.849856418842926], [-73.80176208903741, 40.849949134395956], [-73.80179286047874, 40.850040649665964], [-73.80183056444126, 40.850130657058436], [-73.8020169417146, 40.85043302429988], [-73.80221116637931, 40.85073253056465], [-73.80241316240595, 40.85102905844784], [-73.80262285071488, 40.85132249171065], [-73.80284014920842, 40.8516127153258], [-73.8030649728024, 40.85189961552297], [-73.80329723345862, 40.852183079832955], [-73.80353684022062, 40.85246299713217], [-73.80378369924797, 40.85273925768593], [-73.80417557932397, 40.853340120189074], [-73.80454247276606, 40.854739377515024], [-73.80453818120623, 40.854796001154234], [-73.8045252673004, 40.854851864466276], [-73.80450390541125, 40.854906213096434], [-73.80447438398078, 40.8549583131414], [-73.80443710163813, 40.85500746106074], [-73.80439256181495, 40.855052993176656], [-73.80434136595021, 40.85509429463673], [-73.80428420536751, 40.85513080771725], [-73.80422185194051, 40.85516203935425], [-73.80415514766925, 40.85518756780286], [-73.80408499331024, 40.85520704833207], [-73.80401233621217, 40.85522021788068], [-73.80395066937308, 40.85521934650568], [-73.80388942576332, 40.85522489168396], [-73.80382975801699, 40.8552367490525], [-73.80377278910957, 40.85525469545025], [-73.8037195912262, 40.855278393116954], [-73.80367116558244, 40.85530739605073], [-73.80362842358109, 40.85534115840109], [-73.80359216965921, 40.85537904474202], [-73.80356308614826, 40.855420342030335], [-73.8035417204318, 40.855464273025774], [-73.80352847464194, 40.855510010918394], [-73.80352359809021, 40.85555669488954], [-73.80352718257473, 40.855603446313005], [-73.80353916065029, 40.85564938529117], [-73.80368603416521, 40.85613337035755], [-73.80377806309502, 40.85630145969351], [-73.80383737490631, 40.85648845650411], [-73.8038636768573, 40.856663144315135], [-73.80389056334138, 40.856900266329724], [-73.80395891628706, 40.85717462927908], [-73.80405246302159, 40.85750505118683], [-73.80413784756402, 40.857841760375635], [-73.80417354171651, 40.858141274646506], [-73.8042014225376, 40.858484538159374], [-73.80417856111792, 40.85867822182149], [-73.80417485011644, 40.858755691505884], [-73.80417167874228, 40.85882186816634], [-73.80420585726515, 40.85895904893252], [-73.80422034282891, 40.85897798290789], [-73.80430576415975, 40.859089632036785], [-73.80439744909205, 40.85922025991275], [-73.80449817480466, 40.859438252845365], [-73.80456624045203, 40.85968139696558], [-73.80464293882284, 40.85996820157236], [-73.804679688048, 40.86038010036601], [-73.8046898389874, 40.8605860934545], [-73.80468371760045, 40.86081090572282], [-73.80463487426351, 40.86100342015324], [-73.80462819630267, 40.86102974103276], [-73.80460527769026, 40.86121718099268], [-73.80464756427624, 40.86134183147036], [-73.80473234281749, 40.8614796406207], [-73.80478410547944, 40.86154633053462], [-73.80484333208106, 40.8616093388739], [-73.80490957126987, 40.86166818544455], [-73.8049823182458, 40.86172242176845], [-73.80506101860708, 40.861771634502], [-73.80514507257695, 40.86181544858605], [-73.80523383957318, 40.86185353010517], [-73.8053266430914, 40.86188558883212], [-73.80542277586062, 40.86191138044036], [-73.80552150523441, 40.86193070836608], [-73.80562207877394, 40.86194342530667], [-73.80572372998525, 40.86194943434326], [-73.80582568415839, 40.86194868967931], [-73.80592716427573, 40.86194119699028], [-73.80624571930763, 40.8614274844424], [-73.80582041215638, 40.860034612469434], [-73.80612390783546, 40.85981220661901], [-73.8065822930691, 40.85967223238393], [-73.80674259234422, 40.859919112272024], [-73.80677024150565, 40.859983230649476], [-73.80680626099317, 40.86004490702291], [-73.80685028406147, 40.86010351337308], [-73.80690186246515, 40.860158452939686], [-73.80696047102249, 40.860209166298226], [-73.80702551296353, 40.860255137056164], [-73.80709632600465, 40.860295897112316], [-73.80717218909406, 40.86033103142286], [-73.8072523297539, 40.86036018222822], [-73.80733593194458, 40.86038305269601], [-73.80742214437697, 40.86039940994423], [-73.80783343870087, 40.860318133305384], [-73.80778401428749, 40.86017934667235], [-73.8078480686893, 40.860147190619365], [-73.80790702442809, 40.86010983377677], [-73.80796015245868, 40.860067738104334], [-73.80800679580457, 40.86002142416208], [-73.80804637768432, 40.859971464672554], [-73.80807840864172, 40.85991847743785], [-73.80810249259939, 40.85986311770023], [-73.80811833175508, 40.85980607003865], [-73.80812573026354, 40.85974803990317], [-73.80805973431342, 40.85959661665057], [-73.80772902934285, 40.85909168620477], [-73.80751534772594, 40.85858417837486], [-73.8076037333222, 40.85853559904765], [-73.80768048107562, 40.858538063289515], [-73.80824887200353, 40.85956779276714], [-73.80851624551879, 40.85995638438492], [-73.8089217106452, 40.860287959995375], [-73.80936778566492, 40.8605679398505], [-73.80951163278827, 40.860703386508746], [-73.8096494113749, 40.86084242234357], [-73.8097809661435, 40.860984890694944], [-73.80990614882143, 40.86113063103378], [-73.81002481831423, 40.86127947914359], [-73.810136840863, 40.861431267304674], [-73.81024209019576, 40.86158582448369], [-73.81037746570142, 40.86183596998654], [-73.81039649387583, 40.861880591606585], [-73.81042399903666, 40.86192258832921], [-73.8104593771622, 40.861961037870834], [-73.81050185133067, 40.86199509584518], [-73.81055048877973, 40.86202401430757], [-73.81060422139166, 40.86204715818095], [-73.81066186915001, 40.862064019203295], [-73.81069091016548, 40.862127547810765], [-73.81072848577666, 40.86218844266045], [-73.81077418950343, 40.86224604497456], [-73.81082752692866, 40.86229973159373], [-73.81088792104822, 40.86234892171809], [-73.81095471851108, 40.86239308319121], [-73.81102719668907, 40.862431738257484], [-73.81110457149269, 40.86246446873085], [-73.81139501173809, 40.86251920286502], [-73.81166764690963, 40.86256837491089], [-73.81198840271249, 40.862623715215996], [-73.81228010874106, 40.862749512295586], [-73.81230290205069, 40.8628618385043], [-73.81256827895241, 40.863000876957884], [-73.81275751667133, 40.86309592859981], [-73.81303909715699, 40.863226219723316], [-73.81323099767701, 40.863196704303526], [-73.81329794398005, 40.86322312991144], [-73.81340632950182, 40.86337519555467], [-73.81356635930297, 40.863527725999255], [-73.8136222293541, 40.863631361315115], [-73.81368612350815, 40.86373500838618], [-73.81384566155016, 40.86390175802964], [-73.81411414337536, 40.86373145821164], [-73.81436118470883, 40.863249077889286], [-73.81447233903512, 40.86312307718502], [-73.81468686995449, 40.86306856019122], [-73.8146872979941, 40.86291495026301], [-73.81450667389406, 40.86290368723946], [-73.81458906516404, 40.86272643663105], [-73.81471457316347, 40.86263520277645], [-73.8148450715455, 40.862481801636655], [-73.8149054682641, 40.86241789369311], [-73.81503361392045, 40.86224437200941], [-73.81515628564217, 40.86230674492953], [-73.81515356048828, 40.86242012043739], [-73.81507393686068, 40.86246936737076], [-73.81501601995143, 40.86250767727128], [-73.81494351469583, 40.86259533943938], [-73.81493834063642, 40.86272333915636], [-73.81482452410822, 40.86294077217951], [-73.81494003249354, 40.862981188755924], [-73.8149399052731, 40.86302690653523], [-73.8149932178353, 40.86309420762519], [-73.81501897203428, 40.86312252536485], [-73.81510315384149, 40.863057465367774], [-73.81550189599471, 40.86248004810606], [-73.81641392102723, 40.86118062770394], [-73.81632908278753, 40.861152730522214], [-73.81587638257493, 40.8610038646114], [-73.8156820424257, 40.861109764543755], [-73.81549293707927, 40.860955286737195], [-73.81526321415275, 40.86111774712503], [-73.81479629757335, 40.86112895674676], [-73.81485206024097, 40.86090099030983], [-73.8138180258687, 40.86054827867073], [-73.81273146495035, 40.85887754194771], [-73.81230602708418, 40.854162026473176], [-73.81640870908014, 40.854082255114704], [-73.81649772827157, 40.85401620609713], [-73.81658051197839, 40.85394563638093], [-73.81665666301353, 40.85387088456941], [-73.81672581601713, 40.85379230933092], [-73.8167876392117, 40.85371028767776], [-73.81684183599076, 40.85362521315728], [-73.81688814634457, 40.85353749396302], [-73.81692634810447, 40.85344755097646], [-73.81695625801093, 40.85335581574689], [-73.81697773259017, 40.85326272842151], [-73.81699066884251, 40.8531687356329], [-73.81699500473643, 40.85307428835584], [-73.81699272677278, 40.85301646912296], [-73.81698689510887, 40.85298390587563], [-73.81697781322627, 40.85295177460578], [-73.81696688000619, 40.85289655533696], [-73.81696177954453, 40.85284084997985], [-73.8169625442986, 40.85278501317808], [-73.81696983732702, 40.8527112782992], [-73.81698577763542, 40.85263833311409], [-73.81701023768579, 40.85256676112483], [-73.81704302179158, 40.8524971348473], [-73.81708386768248, 40.85243001123146], [-73.81711166093481, 40.852389394246195], [-73.8171332651999, 40.852346662471184], [-73.81714840664624, 40.852302357551125], [-73.81715689336222, 40.85225704107011], [-73.81715861778896, 40.85221128743284], [-73.81713069323155, 40.852150098176615], [-73.81712633131487, 40.8520952739737], [-73.81713099465547, 40.852040462904455], [-73.81713976212659, 40.85201844005767], [-73.81715825633269, 40.85197198337275], [-73.81717191352776, 40.85192746497483], [-73.81719459659071, 40.851882961380404], [-73.81720828353117, 40.85182816577731], [-73.81719039626154, 40.85176989361128], [-73.81713189892409, 40.851711556070185], [-73.81710848616731, 40.85169856987072], [-73.81708234186158, 40.85168406927604], [-73.81702373247587, 40.851666845518096], [-73.81696953091003, 40.8516873157069], [-73.81692891227493, 40.851690677786415], [-73.81691542317321, 40.85167352556716], [-73.81693804031279, 40.85165300579098], [-73.8169742219704, 40.8516222269334], [-73.81701488702278, 40.851601735774366], [-73.81706907886695, 40.85158468973444], [-73.81708724951115, 40.85154017967316], [-73.81708739083649, 40.851488787961486], [-73.81708786881536, 40.851487704505494], [-73.81710555246347, 40.8514477037433], [-73.81712368537174, 40.85141689622223], [-73.81712386434577, 40.85135180078959], [-73.81712409042197, 40.85126957344483], [-73.81708814456702, 40.851214699159414], [-73.81705764722436, 40.85117859683575], [-73.81701179857696, 40.851084384969845], [-73.81696683811616, 40.85102606981367], [-73.81691731869432, 40.85098487771606], [-73.81685873870755, 40.85095737599869], [-73.8167820523949, 40.850950401837046], [-73.81670082809461, 40.850953698165675], [-73.81660611270236, 40.85093984385117], [-73.81650694371835, 40.85090542502866], [-73.81641233245735, 40.85085388255884], [-73.81632663289389, 40.85084346721819], [-73.81628595906908, 40.850867385157265], [-73.81623984150589, 40.85089957410617], [-73.81622718968529, 40.8509084062442], [-73.8161775944023, 40.85089462177078], [-73.81615793715294, 40.85087162930072], [-73.816110132014, 40.85081571307258], [-73.81604706927764, 40.85077792604364], [-73.81596136832383, 40.85076751093162], [-73.8159021999626, 40.85078437395156], [-73.81584723466285, 40.850796831744084], [-73.81579903851735, 40.85082111845265], [-73.81571882498523, 40.85082099034471], [-73.8157028324223, 40.85080269213893], [-73.81570289992754, 40.85077832859145], [-73.81578326522938, 40.85072363847701], [-73.81585752887507, 40.85069559143508], [-73.81587962275361, 40.85068724648958], [-73.81602424307003, 40.850602206267055], [-73.81602444523365, 40.85052911495124], [-73.8159925283698, 40.8504681550727], [-73.81593644656883, 40.850443702450676], [-73.81594460284292, 40.85039498764828], [-73.81603285347633, 40.850389038927], [-73.8161692493101, 40.850377073693906], [-73.81624162574909, 40.85031018953592], [-73.81624996708972, 40.85019447536701], [-73.8162100447246, 40.85012741179522], [-73.81589012272623, 40.84979190260364], [-73.81575414982423, 40.84965159606208], [-73.8156741733411, 40.84956619565584], [-73.81561022437111, 40.8494869118112], [-73.81558644724868, 40.849383328494824], [-73.81545040822886, 40.84926738413394], [-73.81537028110345, 40.84923680109016], [-73.81529809165959, 40.8492366855288], [-73.81521774467456, 40.84928528484982], [-73.8151455705632, 40.84927907754829], [-73.81514445571717, 40.849278312188204], [-73.81506551195956, 40.849224131912884], [-73.81507165904237, 40.84913361979451], [-73.816301192, 40.84857217199994], [-73.8172632259998, 40.848426566999926], [-73.81820226899997, 40.848311031999884], [-73.81915009199983, 40.84817017999989], [-73.82014043700003, 40.848020598999895], [-73.82048405599983, 40.84850102599995], [-73.82110355699993, 40.84819897999991], [-73.82203826799994, 40.84769036899989], [-73.82301297099994, 40.847161737999905], [-73.82395955500006, 40.84664124199988], [-73.82443760499989, 40.84637787399988], [-73.82495697699986, 40.846105444999864], [-73.8250848919999, 40.844697697999955], [-73.82507950000002, 40.84430400199992], [-73.82513836199985, 40.843343597999905], [-73.82517999, 40.842700058999874], [-73.82558155599993, 40.842818658999946], [-73.82567803699995, 40.84284715199994], [-73.82580705599993, 40.84288069299988], [-73.82624132899997, 40.84299359399996], [-73.82611300699996, 40.84533869399991], [-73.82611304199985, 40.84533885499995], [-73.8261701409999, 40.845573987999934], [-73.8262455299999, 40.84580733899987], [-73.82633899899987, 40.846037546999916], [-73.82634836399981, 40.846056569999845], [-73.82645014900007, 40.84626339599992], [-73.826578403, 40.846483722999906], [-73.82672302600005, 40.84669743899988], [-73.82688311099984, 40.84690351499993], [-73.827677297, 40.848130680999894], [-73.82799676999987, 40.8486243129999], [-73.82799320199993, 40.84890315399989], [-73.82796831299994, 40.8491823059999], [-73.82792209199984, 40.84946036399986], [-73.82785470799985, 40.84973592299989], [-73.82776655799988, 40.85000758699991], [-73.82765823699988, 40.85027403399991], [-73.82742445699994, 40.850993646999946], [-73.82718423799987, 40.85171120499995], [-73.82713133299993, 40.85187108099993], [-73.82737835799986, 40.8528179589999], [-73.82721751200002, 40.853250512999935], [-73.82723026100001, 40.853350756999895], [-73.8280141599998, 40.85472248699992], [-73.82860197199996, 40.85571784799988], [-73.82908318599985, 40.8556760139999], [-73.82956640499994, 40.855620816999895], [-73.83056649299975, 40.85551244799992], [-73.83129157799985, 40.8554341049999], [-73.83147565799987, 40.855450852999844], [-73.83165843199998, 40.855483447999916], [-73.83183634999988, 40.85553171199996], [-73.83200598799996, 40.855594876999916], [-73.83216420699983, 40.85567161299986], [-73.83229711899999, 40.85574641199985], [-73.83242972799992, 40.85582103999992], [-73.83281265300002, 40.85597971299992], [-73.83283502299986, 40.8560358449999], [-73.8327621999999, 40.856079539999925], [-73.83268903299997, 40.8561323829999], [-73.83238468599995, 40.856352210999916], [-73.8321371229998, 40.85654369099993], [-73.83193393799985, 40.856693516999876], [-73.83128338399985, 40.857349961999894], [-73.831114492, 40.8575243299999], [-73.83072266700006, 40.857994586999915], [-73.83055471299986, 40.85819039599993], [-73.82959409199992, 40.85935976399989], [-73.82872856899989, 40.860413327999915], [-73.82847346699995, 40.860731352999906], [-73.82834347399992, 40.86089202599988], [-73.82826796599981, 40.860837332999935], [-73.82799444999982, 40.860840185999905], [-73.82787032399982, 40.860678212999915], [-73.82759107999998, 40.86007855099994], [-73.82741458099993, 40.85995987299994], [-73.82732348799999, 40.859686886999896], [-73.82720794099991, 40.85941373899988], [-73.82714556299992, 40.85923239499992], [-73.82699653199998, 40.85908778599986], [-73.82632571099997, 40.85929799699995], [-73.82444012099991, 40.85950138399989], [-73.8245057509999, 40.85976402099993], [-73.82037552999996, 40.86103300499993], [-73.81884300999991, 40.862568209999885], [-73.8176165129999, 40.86362634299998], [-73.8168449589999, 40.864306792999926], [-73.81699760299986, 40.864459030999946], [-73.81641455899984, 40.8651514979999], [-73.81578764099999, 40.8657580609999], [-73.81938761399985, 40.86798690299985], [-73.81940759199996, 40.867999270999874], [-73.8209161719998, 40.86891693499996], [-73.82097601199978, 40.868949786999956], [-73.82103881099985, 40.868976079999875], [-73.82111615299983, 40.869008136999895], [-73.821266938, 40.869071605999885], [-73.82329599399986, 40.87309741799988], [-73.82110268599995, 40.878969244999915], [-73.81992586499996, 40.88211933299987], [-73.82023958399981, 40.88549105099992], [-73.82027432999985, 40.885550699999925], [-73.81983275399992, 40.885669253999886], [-73.81841703399992, 40.886036994999884], [-73.81742638899989, 40.88629430599987], [-73.81670778700003, 40.88648095099997], [-73.8154787309999, 40.88668744599995], [-73.81536464199993, 40.886701220999875], [-73.81528345499991, 40.886711023999936], [-73.81518316800005, 40.88671965999993], [-73.81485665899983, 40.88674823199989], [-73.81442762200005, 40.88677196499989], [-73.81399748399998, 40.8867821319999], [-73.813567347, 40.8867787089999], [-73.81209659199986, 40.88671500999993], [-73.811980437, 40.88670433899995], [-73.81140645199994, 40.88663558699991], [-73.81083611999995, 40.88655115199987], [-73.81027053199992, 40.88645124299992], [-73.8097107929999, 40.88633607499994], [-73.80915068099985, 40.88621447699993], [-73.80868459399994, 40.886113291999905], [-73.80834232399997, 40.88604431399994], [-73.8079939819999, 40.88599083199992], [-73.80764139499996, 40.885953252999904], [-73.80728647099995, 40.885931813999875]]], [[[-73.78650554049733, 40.880940134479225], [-73.78613451219806, 40.88074604087828], [-73.78598849185093, 40.88076643617943], [-73.78582190502614, 40.88075509610012], [-73.78572615707054, 40.88057541460278], [-73.78578002611279, 40.88036396266133], [-73.785981809052, 40.88031748451096], [-73.7860993049638, 40.880386571723264], [-73.7864073213083, 40.88041011377249], [-73.78664008476228, 40.88049983743608], [-73.78684884910366, 40.88044811129193], [-73.78692034479134, 40.880273350606394], [-73.78677238818248, 40.88013589624925], [-73.78679741033042, 40.87994707381406], [-73.7869998480519, 40.87982153242357], [-73.78726793709994, 40.87968270383201], [-73.78742039384403, 40.87977089028999], [-73.7873445170899, 40.88007294498983], [-73.78742531437602, 40.88044737396561], [-73.787127834652, 40.88092346317125], [-73.78691765981318, 40.88094479518859], [-73.78650554049733, 40.880940134479225]]], [[[-73.78103351104939, 40.87648400204775], [-73.78120649649391, 40.87628502546013], [-73.78132704118002, 40.87636132654635], [-73.78112767478156, 40.87665268050809], [-73.78103351104939, 40.87648400204775]]], [[[-73.78241811865315, 40.87492327042177], [-73.78265785360905, 40.87479265669398], [-73.78280207510406, 40.87494886620542], [-73.78266372854155, 40.87513656444491], [-73.78247604125525, 40.87516813337884], [-73.78241811865315, 40.87492327042177]]], [[[-73.77435244645233, 40.874169955874805], [-73.7744796961532, 40.874077670285956], [-73.77458142278213, 40.87410913390798], [-73.77460919866006, 40.874202172627676], [-73.77457138994558, 40.87433206752441], [-73.77451315255762, 40.87446495533304], [-73.77455324011393, 40.87454877066337], [-73.77453640510954, 40.87471001543833], [-73.77420264266921, 40.87488112230395], [-73.77409538292126, 40.87489618448013], [-73.77407613213158, 40.874814734007096], [-73.77415170184918, 40.87469267688607], [-73.77397610935063, 40.87459940192131], [-73.7740318153443, 40.874394774956215], [-73.77425306035093, 40.87428435764291], [-73.77435244645233, 40.874169955874805]]], [[[-73.78605394964902, 40.873782646192794], [-73.78618924358646, 40.87359972769974], [-73.78633325996694, 40.87367772488745], [-73.78632872330336, 40.87380492844096], [-73.7861946371647, 40.87388531941464], [-73.78605394964902, 40.873782646192794]]], [[[-73.78648510546586, 40.87320925495041], [-73.78662489062734, 40.873034648034434], [-73.7867268909535, 40.87307453926568], [-73.78665449887535, 40.87339363776097], [-73.786499866474, 40.87348161860739], [-73.78626202153426, 40.87339368663393], [-73.78632869488987, 40.873271144554096], [-73.78648510546586, 40.87320925495041]]], [[[-73.77080975398195, 40.871549946847985], [-73.76990710668544, 40.87047694447026], [-73.77109611675694, 40.87069300521904], [-73.77288042665229, 40.871245263430254], [-73.77217786492288, 40.87198111068936], [-73.77080975398195, 40.871549946847985]]], [[[-73.76668965841925, 40.86709778162448], [-73.76709907439783, 40.8667045727482], [-73.76779449382023, 40.86678117376544], [-73.76760072096499, 40.86749638654143], [-73.76668965841925, 40.86709778162448]]], [[[-73.76964988627088, 40.86548557242442], [-73.76979148260493, 40.86512823356445], [-73.77022931630273, 40.865138939396005], [-73.770068972998, 40.86552587618341], [-73.76964988627088, 40.86548557242442]]], [[[-73.78401249138903, 40.863131994074394], [-73.78395232143296, 40.86313026482231], [-73.78389534027481, 40.86313015811381], [-73.78387081440718, 40.86313941124133], [-73.78386118864937, 40.86315152208419], [-73.78385122027875, 40.86315600527144], [-73.78383612396978, 40.86316279486428], [-73.78381322544635, 40.863162751962605], [-73.78379568035903, 40.86315382459151], [-73.78377874150742, 40.86314595995071], [-73.78375633278242, 40.863135557293], [-73.78371746749217, 40.86313265463885], [-73.78369885872442, 40.8631348890298], [-73.78368763328497, 40.86313623765823], [-73.78366256726379, 40.863147915470144], [-73.78363488084096, 40.863146245278166], [-73.78360828124866, 40.86313811003793], [-73.78359127698181, 40.863126757908645], [-73.78356481886246, 40.863116312438635], [-73.78355166102793, 40.86310968179934], [-73.7835410529967, 40.863095506324825], [-73.78353843789932, 40.86308033467228], [-73.78353980995178, 40.8630682046868], [-73.78354252311605, 40.86305304353274], [-73.78354391074059, 40.86303585771484], [-73.78356394999163, 40.86301668454097], [-73.78358659292192, 40.863015715840476], [-73.78361720203176, 40.86302285078345], [-73.78364647814364, 40.8630299837214], [-73.783686441909, 40.86302702463132], [-73.78370779897656, 40.8630118990948], [-73.7837185262937, 40.86298967441828], [-73.78374256752757, 40.86296848754292], [-73.78377317859609, 40.862974611276094], [-73.78380117343677, 40.8629665745244], [-73.78379858786646, 40.862942303722576], [-73.783777332747, 40.86292608664389], [-73.78376405410458, 40.86291392903196], [-73.7837507750274, 40.862901770915116], [-73.7837202166094, 40.862879468852725], [-73.78369627734214, 40.862869313593905], [-73.78367633617697, 40.86285815465367], [-73.7836724235964, 40.862832869685526], [-73.7836804838154, 40.86281165168633], [-73.78368722462388, 40.862786387863814], [-73.78369528417225, 40.862765169359484], [-73.78371130197019, 40.86275407783133], [-73.78372199876624, 40.86274095347989], [-73.78372741840673, 40.86271265366126], [-73.78372750718738, 40.862685353849685], [-73.78373688873616, 40.86266717183979], [-73.7837569170726, 40.86265103263052], [-73.78375829499413, 40.862636879135145], [-73.78376770348979, 40.86261060846434], [-73.78379307646343, 40.86258942357059], [-73.78382641096285, 40.86257735330138], [-73.7838597152167, 40.862574381674385], [-73.78388769190262, 40.862571401231904], [-73.78389574150354, 40.86255321621144], [-73.78389450817735, 40.862522882081066], [-73.78391453469644, 40.862507752337876], [-73.78393983203142, 40.86250982255919], [-73.78397177545816, 40.862515948735506], [-73.78398912415656, 40.86250485966031], [-73.78401316513425, 40.862483671051514], [-73.78403314760357, 40.862481686111146], [-73.7840757656106, 40.86248176587435], [-73.78410369695696, 40.862492940367574], [-73.78413030583008, 40.86250107869518], [-73.78416090425225, 40.862511247494524], [-73.78418483033813, 40.86252544665239], [-73.78420876955407, 40.862535602480506], [-73.78423274486083, 40.862534636647986], [-73.78425940296712, 40.86252760845761], [-73.78430733389965, 40.86253174259114], [-73.78432861001713, 40.862541893918575], [-73.78435921788362, 40.86254902916577], [-73.78440446211489, 40.86256023527143], [-73.78444173884387, 40.86256434843543], [-73.78447101878139, 40.86257046999975], [-73.78449766115588, 40.8625684974192], [-73.78452699591001, 40.86255743014714], [-73.7845523718973, 40.86253523391987], [-73.78458968070599, 40.86252923721539], [-73.78463228171132, 40.862534371932846], [-73.78466024197866, 40.86253644596145], [-73.78469165850701, 40.86254278990822], [-73.78469476742806, 40.86256785438942], [-73.78468005226556, 40.86258804870091], [-73.78467197341119, 40.86261533360188], [-73.78465196734292, 40.86262439547496], [-73.78464133697305, 40.862622861723345], [-73.78463066822562, 40.8626213228734], [-73.784602707706, 40.86261924783206], [-73.7845773613989, 40.862632344914815], [-73.78454667485435, 40.862649476039216], [-73.78454392939913, 40.86267474804281], [-73.7845398085792, 40.86271316238302], [-73.78453969720576, 40.862747538515556], [-73.78454650079381, 40.86277376480672], [-73.78456087516446, 40.862788022439965], [-73.78456744948141, 40.862814322346935], [-73.7845673217424, 40.86285375481233], [-73.78455255057246, 40.86289113759078], [-73.78452979565624, 40.86292648311366], [-73.78451638211648, 40.8629557792117], [-73.78449501214256, 40.86297495006532], [-73.78447228643577, 40.86300119642071], [-73.78444824869851, 40.86302137278443], [-73.78441622565997, 40.86303951254886], [-73.78439355912623, 40.86304755890363], [-73.78435811804623, 40.863073517960906], [-73.78432954841307, 40.863073728077026], [-73.78430301799293, 40.863041323316814], [-73.7842937482678, 40.86302512924163], [-73.78427247111449, 40.863014979079615], [-73.78425378453589, 40.863028086866834], [-73.78422440861301, 40.863051288394196], [-73.78419106476987, 40.86306639110431], [-73.78416702363967, 40.86308757924548], [-73.78416827678372, 40.86311184855092], [-73.78420682075179, 40.8631361866003], [-73.78425470292191, 40.86315548573396], [-73.78427596567165, 40.86316968106346], [-73.7842878476007, 40.86318533054603], [-73.7842865050008, 40.86320508878048], [-73.78426981898309, 40.86321401741422], [-73.78425664943187, 40.86321614027492], [-73.78423747960105, 40.863216104429355], [-73.78421458891884, 40.86321323131151], [-73.78418478751463, 40.863207111388306], [-73.7841643411375, 40.86319940006104], [-73.78414276900331, 40.863191264967305], [-73.78410980767785, 40.86317422294242], [-73.78408589281415, 40.86315921900894], [-73.78404441686627, 40.86314014036228], [-73.78401249138903, 40.863131994074394]]], [[[-73.78312589594455, 40.862856167300684], [-73.78310886092432, 40.86283947489251], [-73.78308802390593, 40.86283943572183], [-73.78307171542822, 40.86284009291337], [-73.78305814187065, 40.86283525259774], [-73.7830355109771, 40.86282970756513], [-73.78301654119176, 40.862813165124244], [-73.7830102575906, 40.86279527100244], [-73.78300940115759, 40.86278013832683], [-73.78299672466267, 40.86277805109834], [-73.78297408257099, 40.86277594495667], [-73.78296318511775, 40.86278417784314], [-73.7829558947177, 40.86279723281158], [-73.78293775624341, 40.86280338851873], [-73.7829278359814, 40.862789614494375], [-73.7829161867475, 40.86275038837365], [-73.78291897441508, 40.862729072054314], [-73.78292445681086, 40.86271464032812], [-73.782911867233, 40.862685728539645], [-73.78290014705881, 40.86266851203447], [-73.78289118224538, 40.86263960823638], [-73.78289306367382, 40.86261829037946], [-73.7829021856519, 40.86259904953427], [-73.78290043669504, 40.86257978890438], [-73.78290505414367, 40.862552973547736], [-73.7828996766172, 40.86253508112389], [-73.78289883055704, 40.862516509374444], [-73.78289979786095, 40.862497941037], [-73.78290619366129, 40.86248144697154], [-73.78290536622264, 40.8624573732796], [-73.78289634979612, 40.86244428763147], [-73.78288188624921, 40.862434631997054], [-73.78288192666943, 40.8624222512419], [-73.7828819783119, 40.862406433091266], [-73.7828974421713, 40.86238720417943], [-73.78289839828749, 40.8623713865579], [-73.78289575072031, 40.8623500601781], [-73.78289032081526, 40.862317193725445], [-73.78290913725111, 40.86231772801563], [-73.78294302128242, 40.86232274158374], [-73.78296910843481, 40.86231982029562], [-73.78297566695083, 40.86230795353378], [-73.78299917499668, 40.862296119823], [-73.78302916358793, 40.862296176219715], [-73.78305399875154, 40.86227741454213], [-73.78305278875975, 40.86224870475323], [-73.78304897317251, 40.86221900100553], [-73.78301584621748, 40.86219472719624], [-73.78303721101206, 40.862173883880736], [-73.7830432358342, 40.86216800507987], [-73.78308042401895, 40.8621705556905], [-73.78308699864493, 40.862153739132445], [-73.78308839527007, 40.86212394684794], [-73.78310610268183, 40.86210667042478], [-73.7831450428813, 40.86206867890555], [-73.78315761269366, 40.86209051912838], [-73.78317971460484, 40.8621103587707], [-73.7831965836362, 40.86213513839591], [-73.78319519264798, 40.862161862664934], [-73.7831885989095, 40.862184617569675], [-73.78318983477908, 40.86220540722526], [-73.78319889402896, 40.862226212083876], [-73.78320536214063, 40.862242062253294], [-73.78320916011666, 40.86227671645224], [-73.78321820970366, 40.862300490479086], [-73.78323379517936, 40.86231932763464], [-73.78323505687578, 40.862332198832505], [-73.7832284861612, 40.86234802450364], [-73.78321931974057, 40.86235988638011], [-73.78321923261986, 40.86238661309795], [-73.78322046681964, 40.86240839298142], [-73.78322562653649, 40.86242523159285], [-73.78322558458962, 40.862438100341336], [-73.78322682047566, 40.86245888999567], [-73.78320980870355, 40.862477666409546], [-73.78319152877746, 40.86248555106949], [-73.7831745376775, 40.862498387968046], [-73.78317316493471, 40.8625191727205], [-73.78317178126751, 40.86254391687242], [-73.78315909842297, 40.86257268615447], [-73.783159423968, 40.86258437153654], [-73.78313379182323, 40.86259829064132], [-73.78312068770268, 40.862618064115644], [-73.783142831741, 40.862625035015235], [-73.78316056266705, 40.86262521709972], [-73.78318585915916, 40.86262511587645], [-73.78321713842091, 40.862629133566635], [-73.783230148944, 40.86263806691489], [-73.78324441706522, 40.86266086151539], [-73.78325863057353, 40.862700483922836], [-73.7832585692753, 40.86271929218303], [-73.78323114283971, 40.86273309921567], [-73.78321153015568, 40.862749890111346], [-73.78323629034399, 40.86275389621756], [-73.78324935830236, 40.86274501253318], [-73.78326241313623, 40.86274008723735], [-73.78327150474334, 40.86275099251027], [-73.78328057699571, 40.86276783796488], [-73.78328053505923, 40.86278070671258], [-73.78326872514111, 40.86280345182183], [-73.78325309247046, 40.862799463539304], [-73.78324136063814, 40.86279845059823], [-73.78322307739761, 40.86280732615861], [-73.78320868897492, 40.862821157178246], [-73.78318905689552, 40.86284388758188], [-73.78316714445815, 40.86286121709221], [-73.78312589594455, 40.862856167300684]]], [[[-73.77460156350928, 40.86206904745972], [-73.77458187384889, 40.86204989320576], [-73.77457833789663, 40.862033528953766], [-73.77457032723042, 40.86201306519905], [-73.77457307831463, 40.861996030906475], [-73.77457403579132, 40.86197831128814], [-73.77457410741187, 40.861957182173015], [-73.77458855460326, 40.86193267270721], [-73.7745975950102, 40.86191428772824], [-73.77461024990188, 40.86188909343952], [-73.77462471322956, 40.86185981426719], [-73.77462479638463, 40.86183527673783], [-73.77463383677214, 40.861816890583064], [-73.77465542355712, 40.86180466498845], [-73.77466976742646, 40.86181082688054], [-73.7746895827087, 40.86179178111501], [-73.77471472524364, 40.861790466567356], [-73.77473987426, 40.86178710790325], [-73.77476056449404, 40.86177488053859], [-73.77475796560724, 40.86174692933691], [-73.77477862120502, 40.861744924864844], [-73.7748029483004, 40.86171907168416], [-73.77481468588563, 40.86169932881202], [-73.77481833935344, 40.861680933301336], [-73.77482736581815, 40.861666637193984], [-73.77481933138, 40.8616529891088], [-73.7748364463393, 40.86163598284136], [-73.77484549586673, 40.861614871747584], [-73.77484645161898, 40.861597833943854], [-73.77484472768988, 40.861576700653266], [-73.7748743968104, 40.86156380846539], [-73.77490045511236, 40.86155704310423], [-73.77493188967381, 40.861552302864105], [-73.77492928975407, 40.861525746632736], [-73.77492845924617, 40.86150597923115], [-73.77492942815577, 40.86148485136198], [-73.77492858842466, 40.86146780938477], [-73.77492862302375, 40.861457585985775], [-73.7749268524907, 40.86145008618316], [-73.77492058397516, 40.86144530154081], [-73.7749179068177, 40.86144052591441], [-73.77491794834289, 40.86142825656256], [-73.77492784743957, 40.86142145967052], [-73.7749287867994, 40.861409193242494], [-73.77493509683372, 40.86140170869999], [-73.7749432115575, 40.861391499889486], [-73.77495940759927, 40.86138062619217], [-73.77497201524237, 40.861369064342945], [-73.7749854888804, 40.86136704532386], [-73.77500438446661, 40.86135413142096], [-73.77501337164381, 40.86135142350354], [-73.77502326610856, 40.861345989566836], [-73.77503944137759, 40.8613412496968], [-73.77505473165346, 40.86133310083514], [-73.77506102234307, 40.86133106830559], [-73.77507096157976, 40.86131200321902], [-73.77508178153292, 40.86129839240863], [-73.77510007843742, 40.86128378399367], [-73.77510436129421, 40.861258222032106], [-73.77510711704002, 40.861239824761384], [-73.77510807711035, 40.861221423320174], [-73.77511532368617, 40.86120235298102], [-73.77512255323983, 40.86118805451983], [-73.7751378266812, 40.86118467534818], [-73.77516742171098, 40.86117626888893], [-73.77519080850668, 40.8611806895072], [-73.77521233827294, 40.86118550286396], [-73.77523206568029, 40.861192356494044], [-73.77525088505807, 40.86120261712371], [-73.77526521436944, 40.86121286900464], [-73.77527954829246, 40.86122175741748], [-73.77529297955127, 40.86123200821588], [-73.77530191604177, 40.86124429388045], [-73.77530545442977, 40.86125997746839], [-73.7753054060716, 40.8612742904286], [-73.77530443892576, 40.86129473665108], [-73.77530528862722, 40.86130905136012], [-73.77530522414428, 40.861328136870526], [-73.77530517118274, 40.861343812291516], [-73.77530333426999, 40.86135607814727], [-73.77529520995593, 40.861369012406726], [-73.77529156793233, 40.861383999686666], [-73.77527711947938, 40.86140919039076], [-73.7752770457749, 40.861431002497994], [-73.77527340232487, 40.86144667092545], [-73.77525273004983, 40.86145344635982], [-73.7752311396364, 40.861466354885955], [-73.77522481965403, 40.86147724786151], [-73.77519783801257, 40.8614915085157], [-73.77518699402977, 40.86151261714614], [-73.77517525592451, 40.861532360558094], [-73.77516262896701, 40.86154937562171], [-73.77515720845145, 40.861559588524955], [-73.77515266437278, 40.86157593818725], [-73.77513862733319, 40.86159428114062], [-73.7751048844543, 40.86163446159275], [-73.7750931670466, 40.86164807115778], [-73.7750620542956, 40.86166197226264], [-73.77503496253725, 40.861685922567034], [-73.77503283049055, 40.86170316279228], [-73.77501840657534, 40.86172085498463], [-73.77499681899609, 40.861733082320825], [-73.7749743266488, 40.86174735150942], [-73.7749688747791, 40.86176642600857], [-73.77495447620205, 40.86177662139126], [-73.77494010924667, 40.8617772756918], [-73.77491036622386, 40.861811978326244], [-73.77488877354436, 40.861825567431886], [-73.77487254969277, 40.86184462090928], [-73.77486709592127, 40.86186437655067], [-73.77486614039097, 40.8618814143542], [-73.77487686060509, 40.86189711196679], [-73.77487680292438, 40.861914151522086], [-73.77487587189496, 40.861923691349936], [-73.7748685983656, 40.861950940961435], [-73.77486136712257, 40.861965241579504], [-73.77485681624742, 40.86198363433426], [-73.77483701776669, 40.86199791042756], [-73.7748091371016, 40.86201216856552], [-73.7747857457861, 40.862027117783626], [-73.77477043454223, 40.86204140094879], [-73.77474885077193, 40.862052264100676], [-73.77472549317646, 40.86205698990684], [-73.77470842425795, 40.86206036501546], [-73.7746796809878, 40.86206439847501], [-73.77466262361408, 40.862064365163285], [-73.77464467344878, 40.862062967635495], [-73.77462582015613, 40.862062930810644], [-73.77460156350928, 40.86206904745972]]], [[[-73.77290231992438, 40.86120858327854], [-73.77323150244695, 40.86074716979338], [-73.77351264165688, 40.861241184087085], [-73.77310632921937, 40.861614575032775], [-73.77290231992438, 40.86120858327854]]], [[[-73.8022229535527, 40.84163481314409], [-73.80263811156135, 40.841081153267076], [-73.80529495910673, 40.8411349248502], [-73.80694608641589, 40.84146244718637], [-73.80680801372397, 40.84248913998752], [-73.80393073307404, 40.84259407254998], [-73.80367560026521, 40.84225849291599], [-73.8022229535527, 40.84163481314409]]]]}}, {\"id\": \"184\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 185, \"Shape_Leng\": 0.0861619681636, \"Shape_Area\": 0.000228537100218, \"zone\": \"Pelham Parkway\", \"LocationID\": 185, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.84004614299985, 40.857806268999866], [-73.83911116499995, 40.85765790099986], [-73.838176221, 40.85750772399996], [-73.83724089099992, 40.85735732299986], [-73.8368191849999, 40.85727680899989], [-73.83690582299991, 40.85718069599995], [-73.83691212799982, 40.85700810699997], [-73.83667557299988, 40.85665266299989], [-73.83499552599994, 40.85657120099989], [-73.8353782199999, 40.85644896499984], [-73.835623356, 40.8563700229999], [-73.83562457299986, 40.85636963099995], [-73.83646667999994, 40.856098438999915], [-73.83690771399995, 40.85599174899995], [-73.83734249299998, 40.85586686399994], [-73.83776909799984, 40.855724111999905], [-73.83818561199983, 40.8555640099999], [-73.83859024199985, 40.85538719199999], [-73.83898125999988, 40.85519446599994], [-73.83935709000001, 40.854986753999924], [-73.83971628999994, 40.85476509399998], [-73.83991984599993, 40.85462633599991], [-73.84011803799997, 40.854483171999924], [-73.84031070099996, 40.85433571999993], [-73.84049767399992, 40.8541841049999], [-73.84067880099991, 40.85402845199995], [-73.84085393099993, 40.85386889199994], [-73.84102291699999, 40.85370555799991], [-73.84118561999999, 40.85353858499997], [-73.84134190199984, 40.853368113999906], [-73.84149163399988, 40.85319428599991], [-73.84194933699985, 40.852512679999904], [-73.84361735799992, 40.85008033999988], [-73.84428070699988, 40.84912269899996], [-73.84494053799982, 40.8482182209999], [-73.84500013599994, 40.84845821899989], [-73.84503845499987, 40.84870153999991], [-73.8450550669999, 40.84894655899991], [-73.84504985900004, 40.84919158499989], [-73.84502303799998, 40.84943496099992], [-73.84496954099991, 40.85009703099991], [-73.84475779899996, 40.850468460999906], [-73.84441301999993, 40.85104299699991], [-73.84430846999992, 40.85119046399989], [-73.84453077199993, 40.85113289899989], [-73.8466850049998, 40.8511180779999], [-73.84721027699993, 40.851036838999946], [-73.84729781299993, 40.85108686899989], [-73.8482145439999, 40.85109088899991], [-73.84861308099981, 40.851079957999964], [-73.84968869799998, 40.851058906999874], [-73.85049849099983, 40.85069600599994], [-73.85136789699986, 40.85030370399994], [-73.85224152599983, 40.8499109959999], [-73.85304526899994, 40.849547331999865], [-73.85457388299992, 40.85150834599995], [-73.855383516, 40.85114371899989], [-73.85619083999988, 40.85077955999987], [-73.85700079699997, 40.850415596999916], [-73.85780815, 40.85005115699993], [-73.8586172349999, 40.84968784799988], [-73.85942539999989, 40.84932393799991], [-73.86023344799996, 40.84896024699987], [-73.8604583409998, 40.84885014599992], [-73.86069456699983, 40.84875339399986], [-73.86094001199987, 40.848670934999895], [-73.86119240899995, 40.848603467999936], [-73.86160276499992, 40.84854631199996], [-73.86250923799993, 40.84836228099987], [-73.86278872899996, 40.84923798299993], [-73.86303938099996, 40.850611306999944], [-73.86473360999999, 40.8502745589999], [-73.86518787199995, 40.8501643859999], [-73.86553546999978, 40.850288634999906], [-73.86612130799995, 40.85052722199985], [-73.86680474499987, 40.85094982699991], [-73.86780004799985, 40.85158929399994], [-73.86860515599989, 40.85206864599991], [-73.86872263199984, 40.85242385199994], [-73.86916668699996, 40.85368604599991], [-73.86937675499983, 40.854265722999855], [-73.86981576299978, 40.855483773999914], [-73.86985960399987, 40.855576727999924], [-73.86988939599996, 40.855673861999925], [-73.86990421600002, 40.85577329299987], [-73.86990379599995, 40.85587302199991], [-73.86988842299996, 40.85597106699987], [-73.86927391299994, 40.856688982999884], [-73.86907624599999, 40.85694556099986], [-73.86887858999994, 40.85720213799993], [-73.86876752999989, 40.85734077999995], [-73.86841539699994, 40.85778035799994], [-73.8681883059999, 40.858063833999964], [-73.86820126399982, 40.85801835599993], [-73.86819723700003, 40.857967986999896], [-73.86817334099987, 40.85791854899991], [-73.86813140299985, 40.857876592999915], [-73.86807772399996, 40.85784684199991], [-73.86802029299992, 40.85783059499988], [-73.86758752499986, 40.85783008999988], [-73.8665833139999, 40.85785535399993], [-73.86564157899991, 40.85788154599987], [-73.86467028199996, 40.85790523699997], [-73.86455073499997, 40.857908462999944], [-73.863957134, 40.857924478999976], [-73.863870478, 40.85792681599985], [-73.86326398600002, 40.85794317599989], [-73.86318907800002, 40.85794519699991], [-73.8629585349999, 40.857951413999956], [-73.86287784399997, 40.85795359099992], [-73.8617906949999, 40.857982903999854], [-73.85882515999995, 40.85806135699987], [-73.85663774599989, 40.858119171999924], [-73.85601437899996, 40.85815817999989], [-73.85585355899978, 40.858171764999966], [-73.85405423899992, 40.85832373099985], [-73.85310947900003, 40.8584097979999], [-73.85217337899977, 40.858499155999944], [-73.8512286699999, 40.85857584399987], [-73.85028234799984, 40.85865726599995], [-73.8493089339999, 40.85871750299987], [-73.84833709099989, 40.85870184799994], [-73.84739746999989, 40.85868254599995], [-73.84645854099998, 40.8586363739999], [-73.8455246059999, 40.85853848699988], [-73.84458712899988, 40.858445809999914], [-73.8432955319999, 40.85829037599991], [-73.84191078799992, 40.858091531999925], [-73.8409714479999, 40.85794907899985], [-73.84004614299985, 40.857806268999866]]]}}, {\"id\": \"185\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 186, \"Shape_Leng\": 0.0246963902234, \"Shape_Area\": 3.70729416953e-05, \"zone\": \"Penn Station/Madison Sq West\", \"LocationID\": 186, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.99096832799995, 40.74542088999985], [-73.99141858599994, 40.74480268199988], [-73.99426258599996, 40.74600398099986], [-73.99709902899994, 40.74720510199991], [-73.99664704399989, 40.74782272899995], [-73.99619655499993, 40.74843932799997], [-73.99574855599987, 40.74905219599989], [-73.99529821499979, 40.749671682999924], [-73.99484363799989, 40.75029330899991], [-73.99437946899984, 40.75093038999993], [-73.99394825699984, 40.751522221999934], [-73.99346417699986, 40.75219005499987], [-73.99211708899992, 40.751622077999876], [-73.99062219999986, 40.75099175399992], [-73.98793687499989, 40.74985496699991], [-73.98776905899994, 40.749787028999926], [-73.98797357699985, 40.74949077899986], [-73.98823761499983, 40.749130168999926], [-73.9887140509998, 40.74851327399993], [-73.98916651699989, 40.747895434999904], [-73.98961683100002, 40.747276422999875], [-73.99006749199991, 40.74665928299993], [-73.99051761299985, 40.746038637999895], [-73.99096832799995, 40.74542088999985]]]}}, {\"id\": \"186\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 187, \"Shape_Leng\": 0.126868431324, \"Shape_Area\": 0.000421195755741, \"zone\": \"Port Richmond\", \"LocationID\": 187, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-74.13435269573621, 40.64188679740515], [-74.13430281185364, 40.641753794909285], [-74.13349719766661, 40.64171690148104], [-74.13346740034589, 40.641595789936815], [-74.13341539233275, 40.64154711655101], [-74.13331577284222, 40.64150390667397], [-74.13322799616068, 40.64145707358684], [-74.13314502680257, 40.64144453266317], [-74.13299172393485, 40.64141693666465], [-74.13288903113595, 40.64141775096135], [-74.13282505053002, 40.641419630335584], [-74.13274216389047, 40.64144860847784], [-74.13266166572117, 40.64148661175371], [-74.13256222253688, 40.64153185578872], [-74.13251504013276, 40.64158922672978], [-74.1324204314397, 40.64162342108231], [-74.1324028357712, 40.64170438898458], [-74.13172794959777, 40.641641280200254], [-74.13134468029385, 40.64160543743175], [-74.13125581180263, 40.64159683076381], [-74.13146199764823, 40.64131010407507], [-74.13108083937513, 40.64102656060494], [-74.13073845051632, 40.6409383118305], [-74.13018370013205, 40.64072382608305], [-74.13012782872325, 40.64079470651256], [-74.12967785166224, 40.64138531255285], [-74.12949057637357, 40.64135483519591], [-74.1299774026272, 40.64072987285558], [-74.12987750736707, 40.640686818458704], [-74.12992088579598, 40.64062898297609], [-74.1293071318684, 40.64033741121096], [-74.12922977261063, 40.64048437235243], [-74.12893301209895, 40.640413929487785], [-74.12891779304783, 40.64049800620997], [-74.12899496309494, 40.64118931324101], [-74.12890729033458, 40.64119482758617], [-74.12882766993523, 40.640401922667834], [-74.12881201847124, 40.64022697750388], [-74.12813214919126, 40.640249999438126], [-74.12810900485859, 40.640139674293174], [-74.12783485946173, 40.6401534508588], [-74.1275267724088, 40.64015530471651], [-74.12739568551268, 40.63997472215616], [-74.12739119380265, 40.63982979896201], [-74.12734770317398, 40.63980148638105], [-74.12724481838394, 40.63976689085693], [-74.12714774563888, 40.6393110568502], [-74.12702714583776, 40.6393157849717], [-74.12702091299994, 40.63927376299988], [-74.12699049099994, 40.63910816399989], [-74.12698007099992, 40.638609711999905], [-74.12697697999995, 40.63843172199987], [-74.1269820749999, 40.63839636999991], [-74.1269897449999, 40.63837057999993], [-74.12700759399989, 40.63833452499991], [-74.12705476399996, 40.638197599999934], [-74.12707923599991, 40.6381224369999], [-74.1268887269999, 40.637210087999925], [-74.12784669199989, 40.63727742999988], [-74.12867047299994, 40.63734826499993], [-74.12872489299997, 40.63708883699995], [-74.12878738899998, 40.636964953999936], [-74.128831656, 40.63683527499989], [-74.12885649499997, 40.636702185999916], [-74.12886142599999, 40.63656824599986], [-74.12884672499999, 40.636436003999975], [-74.12885588799993, 40.63557396099989], [-74.128844165, 40.63532458499988], [-74.12885281899997, 40.6350744189999], [-74.128881913, 40.63482489799992], [-74.12893129399997, 40.63457748699988], [-74.12900054099997, 40.63433359399993], [-74.12944148699994, 40.633671720999885], [-74.1296229589999, 40.63342476399984], [-74.12978688499999, 40.63317008399987], [-74.129932476, 40.632908786999906], [-74.1300590599999, 40.632642026999896], [-74.13016617699994, 40.63237102799989], [-74.13046976799993, 40.6316763209999], [-74.13060220399996, 40.63135782699987], [-74.13047428999985, 40.6308855089999], [-74.13059499799988, 40.6308802829999], [-74.13107460299987, 40.631147729999896], [-74.1316183449999, 40.630583688999884], [-74.13228531899989, 40.630018989999925], [-74.13343731199996, 40.629074345999854], [-74.13408698299992, 40.62954168099984], [-74.13586108399996, 40.62806152799989], [-74.13604609, 40.628188012999864], [-74.13674237799992, 40.62866253699986], [-74.13730784599991, 40.62819029599987], [-74.13785800099993, 40.62769048799992], [-74.13806393799992, 40.627568470999904], [-74.13789126699994, 40.62745287999997], [-74.13805849499992, 40.62706851799986], [-74.13793305099995, 40.626956696999834], [-74.13581008299997, 40.62506433999991], [-74.13618173899991, 40.62483789599987], [-74.13629823599996, 40.62475557699985], [-74.13641924499991, 40.62467712699986], [-74.13654454499994, 40.6246026899999], [-74.13667390599991, 40.6245324029999], [-74.13680709299996, 40.62446639299985], [-74.13694386099996, 40.624404781999914], [-74.13708396099996, 40.624347681999915], [-74.137227136, 40.624295196999846], [-74.13742903499994, 40.624255017999886], [-74.13763642599997, 40.62422942999995], [-74.13784655299993, 40.62421900999986], [-74.13805654699992, 40.62422387299995], [-74.13826358199992, 40.624243769999936], [-74.1385045579999, 40.622633797999875], [-74.13861274099999, 40.62177287699995], [-74.13836666899986, 40.621737116999896], [-74.13853418099995, 40.62112823399985], [-74.13873783599995, 40.62037370299991], [-74.13900214099995, 40.619394642999886], [-74.13923849299987, 40.61851915699989], [-74.13944209899994, 40.617765813999895], [-74.13964602399992, 40.61701454899985], [-74.13982184799993, 40.61629982499988], [-74.14072587899987, 40.616414554999864], [-74.14161689099993, 40.61652246999996], [-74.1417342679999, 40.61653721899988], [-74.14233611399993, 40.616608643999925], [-74.14326214899995, 40.616740238999924], [-74.14419491799998, 40.61687026199996], [-74.14512419099998, 40.6169659789999], [-74.14603562099998, 40.61701277699996], [-74.1464070389999, 40.61427315199995], [-74.14645944899998, 40.6138914249999], [-74.14654340699992, 40.61326141199989], [-74.14670113499992, 40.61207152099992], [-74.14680105499993, 40.6113349189999], [-74.14689344899988, 40.61063170899994], [-74.14696864199988, 40.610151355999896], [-74.14774036799996, 40.609999249999866], [-74.14893620699995, 40.60975083699985], [-74.14954343599997, 40.60962717199986], [-74.14998477099994, 40.6095384179999], [-74.15018666499996, 40.60949781099991], [-74.15031900699995, 40.60959281499989], [-74.15120348199993, 40.61026499499993], [-74.15317275299992, 40.611761521999874], [-74.15371834199986, 40.612206280999864], [-74.15379116699992, 40.612255611999906], [-74.15375060999993, 40.61239939199994], [-74.15371281699991, 40.61254675899991], [-74.15368213999994, 40.612659589999915], [-74.15321287899991, 40.61438556299991], [-74.15311484199995, 40.61474617099987], [-74.15295619099997, 40.61532973399988], [-74.15290527299994, 40.615516992999865], [-74.15281391799995, 40.61582562099988], [-74.1525926639999, 40.61657308099992], [-74.15251285699996, 40.616773003999924], [-74.15241908699993, 40.61696983899991], [-74.15231170399994, 40.61716275599988], [-74.15219115799992, 40.61735094499989], [-74.15205801099998, 40.61753361999995], [-74.15183858099991, 40.61782347299985], [-74.15165757899996, 40.6180405579999], [-74.1513546609999, 40.6183499979999], [-74.15093894399993, 40.618774638999916], [-74.15071865699997, 40.61899965099989], [-74.15037556699993, 40.619352764999896], [-74.15019521699998, 40.61953838199983], [-74.149823327, 40.6199211269999], [-74.14926443299994, 40.620484252999915], [-74.1492283809999, 40.62052057799983], [-74.14911237299995, 40.62063746199988], [-74.14859494699988, 40.62116762499993], [-74.14831982299995, 40.62144951599987], [-74.14825244599989, 40.621529045999864], [-74.14818936799996, 40.621601078999944], [-74.14772830399991, 40.62213661899995], [-74.14753387799992, 40.62241637399988], [-74.14745462999993, 40.622530399999896], [-74.14728835499992, 40.62276964599989], [-74.14710957699992, 40.623032259999924], [-74.14678442599998, 40.62360755099992], [-74.14650354399993, 40.62420010099991], [-74.14622812499992, 40.6249253619999], [-74.145917453, 40.625843585999846], [-74.1457599779999, 40.62630135499995], [-74.14571443699994, 40.62643373899992], [-74.14563382799992, 40.62672514999989], [-74.1456079119999, 40.62689450899992], [-74.14559821999993, 40.62698877599997], [-74.14554348099992, 40.62752119099989], [-74.14554071999994, 40.62783252199995], [-74.14565757199988, 40.62782953799991], [-74.14593544599994, 40.62782411499994], [-74.14607595499996, 40.62875294999986], [-74.14607246899999, 40.628834198999854], [-74.14608756399993, 40.62891590299991], [-74.14612099599996, 40.628994393999896], [-74.14617085499992, 40.62906625599992], [-74.14622202799995, 40.62910695699992], [-74.14628562599995, 40.62913738599987], [-74.14635596699999, 40.629154846999924], [-74.14752618999992, 40.62938928799988], [-74.1473730989999, 40.6298270839999], [-74.14691085899993, 40.631156230999856], [-74.146830166, 40.6313826539999], [-74.14666005299989, 40.63195271099987], [-74.14659947199989, 40.63212929899992], [-74.1464212309999, 40.632719835999836], [-74.14583905999997, 40.634574971999925], [-74.14566932899993, 40.635051931999946], [-74.14556478899993, 40.63545901799992], [-74.14478228999994, 40.637953779999904], [-74.143943612, 40.638348130999915], [-74.14365267699999, 40.638484927999876], [-74.14359998299996, 40.638514125999905], [-74.14312802471521, 40.63968037306379], [-74.14307730726739, 40.639667586793905], [-74.14293728788314, 40.639632316382276], [-74.14292244780515, 40.63959699783795], [-74.14285974698427, 40.63956930452341], [-74.14278730139041, 40.6395620093365], [-74.14268320996482, 40.63954429908314], [-74.1425776933604, 40.63953413750176], [-74.14253970701276, 40.63953752561878], [-74.14247549160602, 40.63956988881193], [-74.14238795149322, 40.63964121082989], [-74.14235881393219, 40.63967020775366], [-74.14232244041622, 40.63974825529746], [-74.14231969707282, 40.63979226696526], [-74.14238558314106, 40.640055010567046], [-74.14240463835897, 40.64019825128138], [-74.1423806145685, 40.64025245611129], [-74.14229766980965, 40.64030580709358], [-74.14215420029252, 40.64034324777036], [-74.14209864155765, 40.64032805206493], [-74.14205431483457, 40.64029675970731], [-74.14155622972348, 40.640366768816975], [-74.1414804740345, 40.640092194295505], [-74.14128624022685, 40.6401452751609], [-74.14119221839894, 40.64038867508108], [-74.14102430552119, 40.64050666884392], [-74.14076253645274, 40.64058939691122], [-74.14028892343684, 40.64066702411254], [-74.13993785315502, 40.64073373910059], [-74.13929745763882, 40.64080583775188], [-74.13925744852796, 40.64073572722679], [-74.13920382377303, 40.64074596817812], [-74.13918328035892, 40.640854892019256], [-74.1387924907907, 40.64089044986864], [-74.13757574838482, 40.64100115237115], [-74.13752644766952, 40.64101365447568], [-74.13748013999505, 40.64103160861471], [-74.13743785761464, 40.64105461457024], [-74.13740054305566, 40.64108215951298], [-74.13736902811014, 40.641113629433725], [-74.13734401529285, 40.64114832283013], [-74.13737001749985, 40.64129546348895], [-74.13570742110944, 40.64162131057876], [-74.13570958979983, 40.64164552310736], [-74.13499892317141, 40.64178536058637], [-74.13499655243614, 40.64176062240995], [-74.13435269573621, 40.64188679740515]]], [[[-74.12117246275159, 40.64166914557247], [-74.12128074797168, 40.64143249599354], [-74.12131194814832, 40.64145253116248], [-74.12124033529696, 40.641586639758586], [-74.12123492143331, 40.64158487954768], [-74.12119488957957, 40.641675010065406], [-74.12117246275159, 40.64166914557247]]], [[[-74.1215534360538, 40.64161162832474], [-74.12176052602824, 40.64157431506241], [-74.12172375309429, 40.641428664221905], [-74.12175764698422, 40.64142342422718], [-74.12178754859733, 40.641556621564185], [-74.12177132201069, 40.64155958222511], [-74.12177911197513, 40.641592547117035], [-74.12176442929801, 40.64159491878664], [-74.12175977879318, 40.64158785752315], [-74.12155886560592, 40.64162163225002], [-74.1215534360538, 40.64161162832474]]], [[[-74.12107727405213, 40.64160742058869], [-74.12118039503834, 40.64138344518971], [-74.12121470277087, 40.64139683999368], [-74.12110124737936, 40.64161328229901], [-74.12107727405213, 40.64160742058869]]], [[[-74.12130529083001, 40.64160070313003], [-74.12137332919554, 40.64147647512165], [-74.12138690694671, 40.641485332526244], [-74.12132617111645, 40.64160656922885], [-74.12130529083001, 40.64160070313003]]], [[[-74.12185329597821, 40.64158069227959], [-74.12184550812958, 40.64154890545135], [-74.12183391805557, 40.64155186120821], [-74.12180661091023, 40.64141215903768], [-74.1218414704778, 40.641405894387034], [-74.12184390745297, 40.64146529598151], [-74.12186561799325, 40.64155594982686], [-74.12194830995264, 40.64154644035803], [-74.12194282613203, 40.64150640851948], [-74.12198146380658, 40.64150106676945], [-74.12199083807539, 40.64155463949942], [-74.12187182772014, 40.64157007444868], [-74.12187416191144, 40.64157831572086], [-74.12185329597821, 40.64158069227959]]], [[[-74.12270319595869, 40.64137664893253], [-74.12269555893829, 40.641353026836214], [-74.12272114424832, 40.64134762377306], [-74.1227256087455, 40.641374859096395], [-74.12274570372185, 40.64137307106], [-74.12276203509289, 40.641426635297144], [-74.12271103075855, 40.641433755252706], [-74.12269546793483, 40.6413772457061], [-74.12270319595869, 40.64137664893253]]], [[[-74.12613276551227, 40.6412643880282], [-74.1259954246122, 40.640231242814394], [-74.12606982881191, 40.64025447334623], [-74.12619996291647, 40.64126431430287], [-74.1263418044479, 40.641252784199516], [-74.12636808688976, 40.64133238048696], [-74.12564026870598, 40.64141849208226], [-74.12563265806332, 40.64134171819285], [-74.12613276551227, 40.6412643880282]]]]}}, {\"id\": \"187\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 188, \"Shape_Leng\": 0.0977983968367, \"Shape_Area\": 0.000313003032771, \"zone\": \"Prospect-Lefferts Gardens\", \"LocationID\": 188, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95729172699996, 40.6643026319999], [-73.95720935499996, 40.663517396999914], [-73.95384851099975, 40.66372780899991], [-73.9509741009999, 40.6639048679999], [-73.9494063899999, 40.66400530799991], [-73.94816604199978, 40.66408182199995], [-73.94546293699983, 40.66422610399989], [-73.94489227700002, 40.66417414399993], [-73.94408198699985, 40.664128572999914], [-73.94269628999989, 40.664055128999934], [-73.94282892199992, 40.6628474859999], [-73.94002976999997, 40.66302074299991], [-73.93726377600001, 40.66319183299991], [-73.93455236699992, 40.6633622859999], [-73.93437459299999, 40.663372810999896], [-73.93160743400001, 40.66355672299988], [-73.93068465499985, 40.66362047699992], [-73.92854068399994, 40.66168068599989], [-73.92828249199995, 40.6614470669999], [-73.92646424299983, 40.659814361999956], [-73.92725505399994, 40.65933756399987], [-73.92819257899993, 40.660177173999905], [-73.92834017599988, 40.660060035999884], [-73.92817215400002, 40.658477344999945], [-73.92799465199988, 40.65680210099988], [-73.92896419499985, 40.6567426129999], [-73.92989802399997, 40.65668443599994], [-73.93090354899984, 40.65661035599996], [-73.93190743899989, 40.656537652999944], [-73.93284657399983, 40.65647848199988], [-73.93381012099998, 40.65642299299988], [-73.93478437399983, 40.65636039499992], [-73.93571813299981, 40.656303296999894], [-73.936179278, 40.6562739719999], [-73.936684695, 40.65624182299995], [-73.93712063899989, 40.65621590299989], [-73.93765923499997, 40.656183875999886], [-73.93859348999989, 40.65612237899993], [-73.93955925399985, 40.656062752999915], [-73.94053214699989, 40.65600153899992], [-73.94146260999986, 40.655944248999944], [-73.94240147899986, 40.655884810999964], [-73.94335488899993, 40.655828334999896], [-73.94446083699984, 40.65576016599992], [-73.94487284399997, 40.655734761999966], [-73.94507909299995, 40.65572204699989], [-73.94715896299984, 40.65559378699993], [-73.9500666489999, 40.65542299699989], [-73.94993008899999, 40.65415032699993], [-73.94976875000005, 40.65264232299987], [-73.94966851199989, 40.651697519999914], [-73.94957113100006, 40.65080789999991], [-73.95137845899984, 40.65071574499988], [-73.95246901399994, 40.650669454999914], [-73.95295084000004, 40.650655559999954], [-73.95480071999985, 40.65056930299989], [-73.9558251839998, 40.6505298079999], [-73.9587078989999, 40.65038727299989], [-73.95892527299989, 40.65121154999988], [-73.95915000999992, 40.65214455699985], [-73.95932926499988, 40.652928178999915], [-73.959451838, 40.65357538799987], [-73.95969665899997, 40.65483921499991], [-73.95986103199981, 40.65563478899989], [-73.96069809399995, 40.6553409239999], [-73.96156461500001, 40.655004116999905], [-73.96190028499997, 40.65487096199992], [-73.9624446589999, 40.658315463999884], [-73.96281137499982, 40.660487278999916], [-73.96307724199983, 40.66213019499987], [-73.96306232099982, 40.662253370999935], [-73.96306230799986, 40.66225346199992], [-73.96306228499995, 40.66225355099985], [-73.96302689399995, 40.66237571299994], [-73.96302687099994, 40.66237578499985], [-73.96302683499988, 40.6623758479999], [-73.96297127299991, 40.66249420299992], [-73.96289695799999, 40.66260575299985], [-73.96289685099988, 40.66260591499986], [-73.9628967209998, 40.66260605899985], [-73.96280577299981, 40.66270845499989], [-73.96242894199995, 40.663125207999876], [-73.96193896099986, 40.66322359899989], [-73.96095595499996, 40.66328505499992], [-73.96061544799986, 40.663302749999936], [-73.96033682899981, 40.66410869799995], [-73.95934422499985, 40.66417564699991], [-73.95787061899992, 40.66427335799993], [-73.95729172699996, 40.6643026319999]]]}}, {\"id\": \"188\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 189, \"Shape_Leng\": 0.0486743306502, \"Shape_Area\": 0.000101427594033, \"zone\": \"Prospect Heights\", \"LocationID\": 189, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96070974499996, 40.673121393999985], [-73.96102898299978, 40.67224318399995], [-73.96126442099991, 40.67156359099989], [-73.96131877999991, 40.67140667599988], [-73.96258784899996, 40.671711596999884], [-73.9631432779999, 40.671860108999894], [-73.96468350799994, 40.67226296899986], [-73.96746491799992, 40.67292729799987], [-73.96769809699997, 40.67297045299991], [-73.96793529099995, 40.673001883999866], [-73.96817496399977, 40.673021321999876], [-73.96839545899999, 40.67302801799983], [-73.96841552899997, 40.6730286279999], [-73.96865541400003, 40.673023838999924], [-73.96892332199981, 40.67293755299988], [-73.96889649099982, 40.67298628299992], [-73.96884578399981, 40.67309921099997], [-73.96880301499984, 40.6732140279999], [-73.96876830399998, 40.67333040999996], [-73.968741747, 40.67344802999989], [-73.96872342099994, 40.67356655699992], [-73.96871337599998, 40.67368565599987], [-73.96871164100001, 40.67380499399991], [-73.96871822199984, 40.67392423399988], [-73.96874205, 40.674087121999904], [-73.96878240699998, 40.67424884999986], [-73.96883910899994, 40.67440796699988], [-73.96891162400006, 40.674563054999915], [-73.96899918499987, 40.67471277699991], [-73.9690773179999, 40.674839670999916], [-73.96907735499994, 40.674839715999944], [-73.96907739, 40.67483976099988], [-73.96917229799983, 40.67496136499993], [-73.96917233299985, 40.67496140999985], [-73.96917236899992, 40.67496145599997], [-73.96928310699997, 40.67507596699993], [-73.96940813699987, 40.67518162199986], [-73.96940821999982, 40.67518167599996], [-73.96954556799999, 40.67527696899988], [-73.96954563799989, 40.67527700499989], [-73.96969317499985, 40.67536085399989], [-73.96949738899991, 40.67590637999996], [-73.97188596799982, 40.676394652999925], [-73.973686958, 40.67707800199987], [-73.97618051399984, 40.67806432099995], [-73.97574929499986, 40.67869970599991], [-73.97532192699995, 40.6793327759999], [-73.97495737999984, 40.679877112999925], [-73.97490473799986, 40.67995413099986], [-73.97469118399997, 40.68026661599992], [-73.9745039589999, 40.68073320299992], [-73.97439144199983, 40.68104616999989], [-73.97422671399997, 40.68150463899983], [-73.97395135399996, 40.68228072999996], [-73.973845731, 40.682586386999915], [-73.97374011799997, 40.68289203199989], [-73.97375598499991, 40.68303401999988], [-73.97278445899987, 40.68285242199992], [-73.97264904900005, 40.68282745499995], [-73.97186424199982, 40.68267028699993], [-73.97067424999982, 40.68242096299987], [-73.96969385699984, 40.68221680499988], [-73.96868961400001, 40.68200525799988], [-73.96753196799997, 40.68176878399992], [-73.96657247099982, 40.68156481399995], [-73.965510434, 40.6813440199999], [-73.96447930699999, 40.681131147999956], [-73.96442184899986, 40.681119287999934], [-73.96329205499984, 40.680881585999884], [-73.96134655899995, 40.6804635239999], [-73.9614104629999, 40.680413152999925], [-73.96153961999995, 40.68008625499992], [-73.96167998599988, 40.67970220599991], [-73.96195644399998, 40.67893384299989], [-73.96222841199993, 40.67816620799992], [-73.96254635399981, 40.67728661699993], [-73.9628644419999, 40.67640534099992], [-73.96318238899983, 40.675529503999876], [-73.96007428300001, 40.67487807099994], [-73.96039301899985, 40.67399860199986], [-73.96070974499996, 40.673121393999985]]]}}, {\"id\": \"189\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 190, \"Shape_Leng\": 0.0743294660829, \"Shape_Area\": 0.000270873666522, \"zone\": \"Prospect Park\", \"LocationID\": 190, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96307724199983, 40.66213019499987], [-73.96281137499982, 40.660487278999916], [-73.96244465949985, 40.65831546399984], [-73.96190028499997, 40.65487096199992], [-73.96414422599983, 40.653994264999916], [-73.96621321999996, 40.653189404999885], [-73.96575121499994, 40.65249922899995], [-73.9652410359999, 40.65174204699989], [-73.96465438699984, 40.650887727999894], [-73.96526281999996, 40.65064244999991], [-73.96614503299999, 40.650297679999866], [-73.9670280919999, 40.649947273999885], [-73.96790948399995, 40.649599373999884], [-73.96879367199982, 40.649248262999926], [-73.96967173699989, 40.64889793599989], [-73.96998500799992, 40.64877493499993], [-73.97069344099991, 40.648496775999945], [-73.97139631700004, 40.64825778599986], [-73.97155930399998, 40.648822828999876], [-73.9719089039999, 40.65005140999992], [-73.97210978599996, 40.650664143999904], [-73.97234114899992, 40.65065904799994], [-73.97252930399988, 40.65075307999995], [-73.9726347479999, 40.650862013999934], [-73.9726576979999, 40.650912039999874], [-73.97267303699994, 40.65096372799991], [-73.97268056399999, 40.651016407999855], [-73.97268018299987, 40.6510693989999], [-73.97267189899989, 40.65112201399995], [-73.97265581899993, 40.65117357099994], [-73.97251721299988, 40.65131415399989], [-73.97232846299997, 40.65139792899986], [-73.97268288599984, 40.65263927799989], [-73.97305103699989, 40.6539125069999], [-73.97326628099982, 40.65464926099985], [-73.97355493999991, 40.6556439389999], [-73.97379189999987, 40.656392323999945], [-73.97404458299992, 40.65734725299994], [-73.97407278699997, 40.65752097799993], [-73.97407281199997, 40.657521085999896], [-73.97407283499992, 40.657521192999916], [-73.97411953499982, 40.657693751999915], [-73.97418455199994, 40.6578635049999], [-73.97418457599994, 40.657863576999915], [-73.97426735599991, 40.658028786999886], [-73.97436701299992, 40.65818790399992], [-73.97448233999991, 40.65833944999988], [-73.97515738099997, 40.65872279399991], [-73.97751171399987, 40.65986275599987], [-73.97957870099991, 40.66085324199988], [-73.97961986699994, 40.66080721499986], [-73.9796200439999, 40.66080700799993], [-73.97962025699994, 40.66080686399987], [-73.97968019899987, 40.66076673799995], [-73.97968033000002, 40.6607666569999], [-73.97968049599983, 40.660766593999895], [-73.9797565759999, 40.66073860199992], [-73.97975683499992, 40.66073850199989], [-73.97975708399997, 40.660738466999874], [-73.9798419799999, 40.660727449999854], [-73.9798422759999, 40.66072741499992], [-73.97984261899994, 40.660727440999956], [-73.97992656299992, 40.66073468699993], [-73.9799267049999, 40.66073469699993], [-73.97992682399993, 40.66073473199991], [-73.98000088999997, 40.66075749199991], [-73.98000113899982, 40.660757563999894], [-73.98000138699987, 40.660757698999866], [-73.98006025399978, 40.66079075799991], [-73.98011964299997, 40.66082279099989], [-73.98011991500002, 40.66082293599988], [-73.98012012799995, 40.66082312499993], [-73.98017186199999, 40.660867618999895], [-73.98017201599991, 40.660867753999845], [-73.98017209799997, 40.660867879999905], [-73.98020992299999, 40.660923132999876], [-73.98021005299988, 40.66092332199989], [-73.98021012299994, 40.660923555999936], [-73.980228943, 40.660985055999895], [-73.98022902599985, 40.660985325999896], [-73.980229014, 40.66098557799995], [-73.98022716299992, 40.66104784799991], [-73.98022716299988, 40.66104797499988], [-73.98022710399982, 40.6610481279999], [-73.98020654600003, 40.66110546099991], [-73.98020645099992, 40.66110571199987], [-73.98020628599997, 40.66110595499992], [-73.98017161099993, 40.66115465799992], [-73.98013355499987, 40.66118761199988], [-73.98008984499991, 40.66121620899992], [-73.98004133199993, 40.66123989299993], [-73.97998896000001, 40.66125820299989], [-73.97993375199998, 40.661270780999864], [-73.97987678099987, 40.661277381999895], [-73.97981915699984, 40.66127787799995], [-73.97976200499983, 40.66127225999991], [-73.97970643699993, 40.66126063599991], [-73.97965353499985, 40.661243232999915], [-73.97935059899989, 40.66159482099989], [-73.97885475899994, 40.66220183299989], [-73.97834945199986, 40.66279901499991], [-73.97784275099997, 40.66340176899994], [-73.97734252999986, 40.66401189699998], [-73.97684516799998, 40.66460638199993], [-73.97633386199989, 40.66521360399989], [-73.97583132299987, 40.66581507999991], [-73.97533714599997, 40.666419095999906], [-73.97483327599997, 40.66702209799992], [-73.97433093599996, 40.66762123499995], [-73.97382528499993, 40.66822308799986], [-73.97332284299985, 40.66882492299992], [-73.97281990699993, 40.669427177999864], [-73.97231720899988, 40.67002909799988], [-73.97181594699985, 40.67062947499989], [-73.97132526399996, 40.671248224999914], [-73.97085440699995, 40.67186436499991], [-73.97041289500005, 40.67259372399991], [-73.97000244600004, 40.67263114999985], [-73.96966394100005, 40.67264236699988], [-73.96927327499998, 40.67279742699989], [-73.96892332199981, 40.67293755299988], [-73.96883836699993, 40.67296491399987], [-73.96865541400003, 40.673023838999924], [-73.96841552899997, 40.6730286279999], [-73.96839545899999, 40.67302801799983], [-73.96817496399977, 40.673021321999876], [-73.96793529099995, 40.673001883999866], [-73.96769809699997, 40.67297045299991], [-73.96746491799992, 40.67292729799987], [-73.96468350799994, 40.67226296899986], [-73.96314327899981, 40.67186010899995], [-73.96258784899996, 40.671711596999884], [-73.96254108199987, 40.67146011899995], [-73.96252326599993, 40.67137973899986], [-73.96219447199988, 40.66964102999985], [-73.96211021099997, 40.66919569699996], [-73.96199657599993, 40.66866286999989], [-73.96180658699991, 40.66768122699987], [-73.96161627599987, 40.66669521599988], [-73.96125791699997, 40.66480182299993], [-73.96110886899996, 40.664062323999914], [-73.96095595499996, 40.66328505499992], [-73.96193896099986, 40.66322359899989], [-73.96242894199995, 40.663125207999876], [-73.96280577299981, 40.66270845499989], [-73.9628967209998, 40.66260605899985], [-73.96289685099988, 40.66260591499986], [-73.96289695799999, 40.66260575299985], [-73.96297127299991, 40.66249420299992], [-73.96302683499988, 40.6623758479999], [-73.96302687099994, 40.66237578499985], [-73.96302689399995, 40.66237571299994], [-73.96306228499995, 40.66225355099985], [-73.96306230799986, 40.66225346199992], [-73.96306232099987, 40.66225337199995], [-73.96307724199983, 40.66213019499987]]]}}, {\"id\": \"190\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 191, \"Shape_Leng\": 0.130932637664, \"Shape_Area\": 0.000688460603185, \"zone\": \"Queens Village\", \"LocationID\": 191, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.73016587199996, 40.72395859599988], [-73.72990512499977, 40.723272494999954], [-73.72836415399992, 40.72361435199985], [-73.72818574899999, 40.723145636999924], [-73.72816233910697, 40.7230841398913], [-73.72849600617099, 40.72299077906028], [-73.72882383511435, 40.72288364309897], [-73.72914435403575, 40.722763070762504], [-73.7294561428778, 40.722629528205644], [-73.72965186981291, 40.7225348580731], [-73.72975788476148, 40.722483579204564], [-73.73004830901388, 40.72232591492649], [-73.73032628984046, 40.722157296892334], [-73.73030132022359, 40.72209451699486], [-73.72966006089679, 40.72048124695883], [-73.72966017623874, 40.72048187721397], [-73.72965947411386, 40.72047999397092], [-73.72965916916328, 40.72047916440093], [-73.7296587481738, 40.72047801913239], [-73.72965817488387, 40.72047644199296], [-73.72965728480857, 40.72047403518619], [-73.72964935890133, 40.72045251304386], [-73.72964541390358, 40.72044178737715], [-73.72964242877289, 40.720433685154234], [-73.72964196075878, 40.72043241392386], [-73.72963556812131, 40.72041503730895], [-73.72947667309306, 40.71998333630181], [-73.72917629507717, 40.71916721478088], [-73.72883757476156, 40.717933019006566], [-73.72856951516104, 40.71695629288504], [-73.72855295497688, 40.71689594605374], [-73.72852236515118, 40.71678448081893], [-73.72850152477136, 40.71670554586146], [-73.72736801308001, 40.712411131811415], [-73.72735961821309, 40.71238894105729], [-73.72718876297576, 40.71169261784399], [-73.72698520205972, 40.710776311826805], [-73.7269714679152, 40.710714504016735], [-73.72696515545228, 40.71066071005933], [-73.72690004194364, 40.70999683099981], [-73.72689487490652, 40.709978573809686], [-73.7270469657635, 40.70949875315655], [-73.72696135646343, 40.7075490870481], [-73.72689497712992, 40.70580017280212], [-73.72683525330599, 40.70440569982361], [-73.72677759516957, 40.7030592911994], [-73.72713924599991, 40.70304627199988], [-73.72725657499997, 40.703041498999916], [-73.7273634669998, 40.70303714099988], [-73.7277679189999, 40.70302063799987], [-73.73036969799986, 40.70309853299989], [-73.732314832, 40.70315970599992], [-73.73245633499985, 40.70316405099986], [-73.7333187329999, 40.70319051999997], [-73.73426455299999, 40.70321954199994], [-73.73426739299985, 40.70316624699992], [-73.73430403699984, 40.702478130999936], [-73.73624941199995, 40.702536977999884], [-73.73819579999989, 40.70259533599989], [-73.74072128199992, 40.70267801899989], [-73.74077675799997, 40.70267983699994], [-73.74051703299997, 40.70338761599989], [-73.74050336499984, 40.70343355599992], [-73.74055102499982, 40.70344338099995], [-73.74069300199989, 40.70346611399988], [-73.74089188600004, 40.70348164599992], [-73.74109293999989, 40.7034810629999], [-73.74129277999985, 40.70346424899991], [-73.74148805299996, 40.70343161199993], [-73.74167563399995, 40.70338409999992], [-73.74287389299987, 40.703074942999876], [-73.74330723899996, 40.70295979699989], [-73.74386175999997, 40.70281109599994], [-73.7441343549998, 40.7027421119999], [-73.74502395299994, 40.70251829799986], [-73.74590871000004, 40.70228727399988], [-73.74500119100004, 40.70026221299988], [-73.74588751799996, 40.70003041699988], [-73.74677542799994, 40.69980088799986], [-73.74767063100002, 40.699568575999876], [-73.74773012199991, 40.69955313699991], [-73.74741867999983, 40.69885902799989], [-73.74712436899999, 40.698198034999926], [-73.74681909299984, 40.697525451999844], [-73.74649367299993, 40.696799873999936], [-73.74618157999997, 40.696121925999904], [-73.74624875999989, 40.69610314499993], [-73.74744768999992, 40.69577042099991], [-73.74774501699984, 40.695688540999946], [-73.74829993899988, 40.69553724399987], [-73.74863325999986, 40.6954447609999], [-73.74928080799988, 40.696894076999904], [-73.7501929579999, 40.69891208199988], [-73.7493054709999, 40.69914231299992], [-73.7496211959998, 40.69982977999993], [-73.74991098899996, 40.70047962799991], [-73.75022047099984, 40.70116599699989], [-73.75053038699998, 40.701854896999876], [-73.75082756899988, 40.70251465499992], [-73.75113533300001, 40.70320334299989], [-73.75143698199983, 40.703864323999895], [-73.75174489199993, 40.70454615499992], [-73.75205066499986, 40.70522802799995], [-73.75052149099987, 40.70562662299987], [-73.75047058199985, 40.70563993099994], [-73.7507743589998, 40.706321836999905], [-73.75121246999993, 40.707306579999965], [-73.75139828700003, 40.70774871999987], [-73.7518426989999, 40.708822920999886], [-73.75245965599989, 40.71031656399992], [-73.7531839939999, 40.71205517599985], [-73.7535751219999, 40.712992296999886], [-73.75360661900005, 40.71307112199987], [-73.75370834599995, 40.71332463599994], [-73.75401024499985, 40.71383490099993], [-73.7540478399999, 40.713877538999945], [-73.75407540999991, 40.7139046949999], [-73.75470078399988, 40.714710052999905], [-73.75588877999999, 40.7162986439999], [-73.75725671499985, 40.71813860199992], [-73.75721025899989, 40.7181586389999], [-73.75638211900001, 40.718515726999954], [-73.75556553099986, 40.7188668699999], [-73.75474912799989, 40.71921669499994], [-73.75393063599992, 40.71957218199985], [-73.75313314299989, 40.71991531799997], [-73.7523343839998, 40.72026246099993], [-73.75247511899987, 40.72048373099989], [-73.75263582400002, 40.7206979659999], [-73.75281552300001, 40.720903624999885], [-73.75301295399987, 40.721099278999915], [-73.7532266449999, 40.721283649999876], [-73.75433064399985, 40.72230878699987], [-73.7546271559998, 40.72258094099987], [-73.75491106099987, 40.72284152399987], [-73.75546163099992, 40.723352762999916], [-73.75577120999998, 40.72364021599991], [-73.755035606, 40.72411532099988], [-73.75496919499987, 40.72415964699993], [-73.75508625099991, 40.72449005999985], [-73.75524449499997, 40.724783126999924], [-73.75548073699997, 40.72511980699992], [-73.75566637499988, 40.725319169999906], [-73.75581422999997, 40.725430010999965], [-73.75603126999992, 40.72559271799984], [-73.75670987299993, 40.72617907299986], [-73.75669771299987, 40.7262287109999], [-73.75663658099985, 40.7262464109999], [-73.75655253399987, 40.72627075399994], [-73.75612598799987, 40.7264302599999], [-73.75527584399988, 40.72671544099991], [-73.75442222499994, 40.727042147999924], [-73.75406469299989, 40.72720104599992], [-73.75380709300003, 40.72731553199989], [-73.75363109999996, 40.727393747999855], [-73.7529702879999, 40.72770561999986], [-73.75259471000005, 40.7279089859999], [-73.75221914099994, 40.728112343999925], [-73.75184238299993, 40.72832650199991], [-73.75106753999994, 40.72881769799992], [-73.75081066899993, 40.72899503899987], [-73.75048295699996, 40.72922129399996], [-73.74980987499995, 40.72974674999989], [-73.74917188599983, 40.7302990689999], [-73.7479342389999, 40.73140119599985], [-73.74749692899994, 40.73179105099994], [-73.74733126699995, 40.73193874099994], [-73.74704531499992, 40.73219365299985], [-73.74672140400006, 40.73247715099991], [-73.74671314399986, 40.73236436899988], [-73.74669850599986, 40.73221018799995], [-73.74667404599982, 40.73195264999993], [-73.7466568949999, 40.73181492399991], [-73.74662058899987, 40.73167780499995], [-73.74659290699987, 40.73161070999988], [-73.74656523699987, 40.73154360499996], [-73.74649162199995, 40.731414588999925], [-73.74640108899996, 40.7312928389999], [-73.74629555799991, 40.731180177999875], [-73.74555667199989, 40.73063689799994], [-73.74543152500001, 40.73061791799988], [-73.74488800699997, 40.73048310599994], [-73.74417482499997, 40.730322763999936], [-73.74411105400003, 40.73030792499984], [-73.74398330999986, 40.73027821199988], [-73.74385189999987, 40.73024764599996], [-73.74383878399982, 40.7302445999999], [-73.74376398999986, 40.730227197999966], [-73.74242143799988, 40.72991886499994], [-73.74230785999983, 40.72989362199998], [-73.74200329399991, 40.72982592399989], [-73.74080920999988, 40.729560507999956], [-73.74040469999989, 40.72946953699989], [-73.73968059999994, 40.72930668299992], [-73.73904904899976, 40.72916463999989], [-73.73840175399988, 40.729011523999866], [-73.73816144099989, 40.728958091999914], [-73.73715501999992, 40.728732010999884], [-73.73632483599988, 40.72854566199993], [-73.73545493899995, 40.7283456249999], [-73.73498719899999, 40.72824220599991], [-73.73450816499994, 40.7281364099999], [-73.73424214199999, 40.7280753799999], [-73.73361727799993, 40.727933573999955], [-73.73272577999994, 40.727734069999904], [-73.73161735999996, 40.72744912699996], [-73.7314721509999, 40.727411794999924], [-73.73144806399984, 40.72734384999995], [-73.73121918199986, 40.72669819599986], [-73.73095653299988, 40.726014939999835], [-73.73069257599997, 40.72532826699992], [-73.73043003899987, 40.724640254999876], [-73.73016587199996, 40.72395859599988]]]}}, {\"id\": \"191\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 192, \"Shape_Leng\": 0.11196461971, \"Shape_Area\": 0.000373808019026, \"zone\": \"Queensboro Hill\", \"LocationID\": 192, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82239598999983, 40.74986422299986], [-73.82131898199997, 40.7487581829999], [-73.82108427599998, 40.748558445999954], [-73.82052281799989, 40.74835684999989], [-73.81961962699988, 40.748042042999934], [-73.81873142799999, 40.747711900999875], [-73.81782025899999, 40.74738541599988], [-73.81691898599989, 40.74720466799995], [-73.81657996199982, 40.74713778799989], [-73.81575069499985, 40.7469794799999], [-73.8153731739999, 40.747218337999946], [-73.81477152699999, 40.74668237099996], [-73.81416171999987, 40.74613857999992], [-73.81341871899986, 40.74658444499988], [-73.81316297299992, 40.74673791899988], [-73.81235411899996, 40.74722328299991], [-73.81161754599998, 40.74766484299989], [-73.80976035399996, 40.74877922299991], [-73.80750599299991, 40.750136586999936], [-73.8072707609999, 40.75011897499993], [-73.80633253899988, 40.750047665999936], [-73.80539953699991, 40.7499727289999], [-73.80446953099995, 40.74989927599989], [-73.80353070499987, 40.7498254129999], [-73.80295670699991, 40.74977052699987], [-73.80287666499983, 40.749765149999895], [-73.80280784099993, 40.74906666099989], [-73.80277033699984, 40.748847537999936], [-73.80274910899995, 40.74860675199987], [-73.80274770699985, 40.74836447599992], [-73.80276629299996, 40.74812214499988], [-73.80280481399994, 40.7478811629999], [-73.80286300399987, 40.7476429709999], [-73.80294034199999, 40.74740891599989], [-73.80302898599999, 40.747194800999935], [-73.8033521419999, 40.74643196499987], [-73.80327809299986, 40.74643950999996], [-73.80017472200002, 40.746755577999906], [-73.79921673599998, 40.7468562809999], [-73.79826235399989, 40.746965655999894], [-73.7946756879998, 40.74721192799994], [-73.79452019399987, 40.74590015899989], [-73.79448123999998, 40.745669693999915], [-73.79448121599987, 40.74566953199994], [-73.79448120499993, 40.74566936999989], [-73.79445869199989, 40.74543695399997], [-73.79445282799998, 40.74520375699995], [-73.79445281699995, 40.74520343199987], [-73.79445282999983, 40.7452030899999], [-73.7944636839999, 40.74497002199991], [-73.7944911469999, 40.744737937999965], [-73.79449117099993, 40.74473774799991], [-73.7944912069999, 40.744737567999955], [-73.79453507400001, 40.74450763799989], [-73.79467327199988, 40.74424460699987], [-73.79467340799984, 40.74424434599993], [-73.79467356599999, 40.7442440949999], [-73.79483295499979, 40.74398694199992], [-73.79501285899983, 40.743737044999854], [-73.79501298499999, 40.74373688099991], [-73.79501312, 40.74373670699995], [-73.7952125779999, 40.74349556899989], [-73.79543059299992, 40.74326428999993], [-73.79566566899987, 40.74304423599989], [-73.79589302899998, 40.742803857999924], [-73.79589319199995, 40.74280368799987], [-73.79589335199982, 40.742803526999886], [-73.79613624999985, 40.74257152399986], [-73.79613631199986, 40.74257146699995], [-73.79613638599983, 40.742571410999865], [-73.79639414999993, 40.74234849999986], [-73.79639423599997, 40.74234842499992], [-73.79639432299986, 40.74234834999986], [-73.79666581899986, 40.742135522999895], [-73.79666601799985, 40.742135371999886], [-73.79666622899994, 40.74213522099994], [-73.79695065300002, 40.741933034999896], [-73.79723084999992, 40.74169550699993], [-73.79726795099995, 40.74056448599986], [-73.79727885499992, 40.74044812399993], [-73.79727887899985, 40.74044786199991], [-73.79727886799999, 40.74044760099988], [-73.79727517699983, 40.74033034899986], [-73.79727517699985, 40.740330185999895], [-73.79727515299997, 40.740330033999896], [-73.79725674299983, 40.740213241999896], [-73.79725670899992, 40.74021304299991], [-73.79725664999995, 40.74021283499994], [-73.79722380199985, 40.740098277999905], [-73.79722373099987, 40.74009803499988], [-73.79722362499994, 40.740097800999884], [-73.7971768669999, 40.739986684999934], [-73.79640833699986, 40.738593419999894], [-73.79629297299985, 40.73855230599992], [-73.79672497900003, 40.738416642999894], [-73.79643940099992, 40.73843894799993], [-73.79641178399994, 40.738325325999895], [-73.79640218700004, 40.73828235799987], [-73.79684795799999, 40.73823554099991], [-73.79729260599996, 40.73818542899995], [-73.79822058899992, 40.7381746309999], [-73.79911195699995, 40.738206139999875], [-73.80000292499997, 40.73823762399993], [-73.80089151300002, 40.7382690159999], [-73.80132155499992, 40.738284201999946], [-73.80180834299986, 40.738302654999885], [-73.80268632099998, 40.73833105199989], [-73.80363600499999, 40.738360851999886], [-73.80461553500005, 40.738391577999906], [-73.80466543299995, 40.73839298499992], [-73.80471291600003, 40.738393948999914], [-73.80564551, 40.73841865199988], [-73.8065788389999, 40.73844721099994], [-73.80752082000002, 40.738480847999924], [-73.80805258499988, 40.738500293999905], [-73.80844891299986, 40.73851478699992], [-73.80893016799997, 40.73853237999986], [-73.809411422, 40.73854997199989], [-73.8104043559999, 40.73858964799991], [-73.81139729099982, 40.738629306999904], [-73.81227737799998, 40.7386644509999], [-73.813224018, 40.7387022539999], [-73.81407381999989, 40.738780901999874], [-73.81489497099994, 40.73891574299995], [-73.81499755699998, 40.738932585999926], [-73.81504998799994, 40.7389431329999], [-73.8150386279998, 40.73889366799993], [-73.81497208299996, 40.738560715999874], [-73.81501663899991, 40.738558544999876], [-73.81518583099992, 40.73857117099989], [-73.8160767679998, 40.73885793399994], [-73.81633109399993, 40.73844640099989], [-73.81647443699993, 40.73840887099993], [-73.81662531599994, 40.73838631099992], [-73.81677959599985, 40.73837979399986], [-73.81693283800001, 40.738389509999934], [-73.81708075599988, 40.7384147349999], [-73.81837989599988, 40.73859373999992], [-73.82006431399985, 40.73882813999993], [-73.82089716499988, 40.73900211099992], [-73.82177826400005, 40.739193646999894], [-73.82266847299987, 40.739384992999916], [-73.82355883499997, 40.73957874499993], [-73.8243501109999, 40.73975360599992], [-73.82481178499988, 40.739630959999886], [-73.82494800899991, 40.739567104999914], [-73.82497439999995, 40.73944042599991], [-73.8254265879999, 40.7394851379999], [-73.82559030299986, 40.739447591999905], [-73.825853684, 40.73912377999988], [-73.826349658, 40.738843263999904], [-73.82686746499992, 40.738628911999854], [-73.82718901699991, 40.7385498709999], [-73.82759017299992, 40.73849421699993], [-73.82788031099987, 40.73848979699991], [-73.8279436029999, 40.73848998199987], [-73.82794969099994, 40.73781959899991], [-73.82809152599998, 40.73708317499995], [-73.82946751999994, 40.737024481999896], [-73.83053551299994, 40.737520194999895], [-73.83143025500006, 40.73913573299994], [-73.8313011729999, 40.739561062999904], [-73.83126834299986, 40.74026227599991], [-73.83122032399993, 40.74095321999989], [-73.8311729489998, 40.74163764499991], [-73.83113122799989, 40.74233799999994], [-73.83207064899982, 40.74245093799993], [-73.83301515399995, 40.742600957999905], [-73.83382572699993, 40.742762420999874], [-73.83586131899989, 40.743227984999926], [-73.83590196999995, 40.74337980599992], [-73.83591564899993, 40.74343089299987], [-73.83592530399984, 40.7435005709999], [-73.83595071499995, 40.74373056899992], [-73.83584967799999, 40.7441757629999], [-73.83567553899992, 40.74485051799991], [-73.83556958700001, 40.745150630999916], [-73.83543777199996, 40.74552397399991], [-73.835065747, 40.746204108999926], [-73.83469042599992, 40.746902697999936], [-73.83460369599997, 40.747020998999886], [-73.834603649, 40.747021071999875], [-73.83460358899993, 40.747021133999986], [-73.83450317399993, 40.74713383299994], [-73.83450311499995, 40.74713390399994], [-73.83450304299996, 40.747133967999886], [-73.83438980999983, 40.74723985799992], [-73.834265193, 40.74733759999988], [-73.8342650619998, 40.74733769799992], [-73.83426490799998, 40.747337797999926], [-73.83413036799998, 40.7474264389999], [-73.83392232599984, 40.747532001999936], [-73.83392218299996, 40.74753207499989], [-73.83392204099982, 40.74753213699994], [-73.83370364199992, 40.74762634799989], [-73.83364897199982, 40.74764609899995], [-73.83300301999995, 40.74783363099989], [-73.83125479799999, 40.74810635899991], [-73.83128299699986, 40.7482212609999], [-73.83129377999995, 40.74833944999994], [-73.83128640899984, 40.74845858499988], [-73.83126084299984, 40.748576271999895], [-73.83121778599984, 40.748690121999914], [-73.83115861499981, 40.748797997999915], [-73.83109206499991, 40.74886061199991], [-73.8310116789999, 40.748916018999886], [-73.83091972499989, 40.74896171099987], [-73.83081942899994, 40.748995791999924], [-73.83071466599985, 40.74901721399987], [-73.83060941299986, 40.74902588399991], [-73.82965157399984, 40.749171992999905], [-73.82943632699994, 40.74920530899993], [-73.82922511999989, 40.74925608099989], [-73.82902159099983, 40.74932374899987], [-73.82882914999982, 40.74940718399992], [-73.82865083499996, 40.749504731999856], [-73.82846651899989, 40.74959657899997], [-73.82827349999988, 40.749678714999966], [-73.82807314199995, 40.74975046599994], [-73.82786693899982, 40.74981128499992], [-73.82765641899984, 40.749860830999886], [-73.82751792599984, 40.749874789999886], [-73.82737756299996, 40.74987437099992], [-73.82723921499992, 40.749859602999905], [-73.82673694299987, 40.74967723499986], [-73.82584440999999, 40.74934839099993], [-73.82574065099986, 40.750274963999864], [-73.82569664700004, 40.75039824999985], [-73.8256669259999, 40.7505247439999], [-73.82565201299991, 40.75065299899992], [-73.82565209100002, 40.75078150099995], [-73.82566698599992, 40.7509087479999], [-73.82576557199987, 40.750969023999865], [-73.82518837299989, 40.75128858499995], [-73.82492269399982, 40.75144029499993], [-73.8246103029999, 40.751626747999865], [-73.82426493799994, 40.75183288799992], [-73.82321319899984, 40.75070793199988], [-73.82239598999983, 40.74986422299986]]]}}, {\"id\": \"192\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 193, \"Shape_Leng\": 0.0655299584801, \"Shape_Area\": 0.000145591657944, \"zone\": \"Queensbridge/Ravenswood\", \"LocationID\": 193, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93522718099996, 40.76952572899989], [-73.9357713409999, 40.76905000999985], [-73.93582070900001, 40.76863091499992], [-73.93593177099983, 40.76778370199991], [-73.93618379699997, 40.767691801999945], [-73.93709253699996, 40.767360392999905], [-73.93836125899998, 40.76691075599995], [-73.93858976899998, 40.766817358999916], [-73.93846096299997, 40.766768302999935], [-73.93763535299998, 40.76647487199988], [-73.93682006599995, 40.76618273599989], [-73.93600532399995, 40.76588489499987], [-73.93700478599986, 40.76423995399997], [-73.93687031099991, 40.76417791699989], [-73.93684249199987, 40.764165082999924], [-73.9362557499999, 40.7638943969999], [-73.93621639299991, 40.76387581299986], [-73.93619184999997, 40.763864223999846], [-73.93616861799997, 40.76385325299988], [-73.93614654199985, 40.763842828999906], [-73.93541372899996, 40.763496787999884], [-73.93398092199995, 40.76282598199987], [-73.93404264799992, 40.762761377999915], [-73.9341081699999, 40.76269280199993], [-73.93429712599992, 40.76249503399991], [-73.9343717929999, 40.7624168849999], [-73.93457323599995, 40.762206044999935], [-73.93463877499988, 40.762137449999926], [-73.93472503099987, 40.762047169999875], [-73.93480791700001, 40.761960415999894], [-73.93538955299994, 40.76135538399994], [-73.93543323399999, 40.76130585199992], [-73.93546712899983, 40.761270327999945], [-73.93550408999991, 40.76123158999994], [-73.93556528699992, 40.76116745099992], [-73.93560854899978, 40.761122109999846], [-73.93566314299997, 40.76106489099995], [-73.93569960399994, 40.7610266769999], [-73.93577489099982, 40.7609477719999], [-73.93581185199994, 40.7609090339999], [-73.93584407599987, 40.760875259999935], [-73.93589256999996, 40.7608244349999], [-73.93593276099996, 40.7607823119999], [-73.93601701999981, 40.760694001999894], [-73.93605584499984, 40.76065330999995], [-73.93679091699993, 40.75988288099991], [-73.93791705599988, 40.7587066049999], [-73.93910030499991, 40.7574941399999], [-73.94004034700002, 40.756486750999926], [-73.94123861599985, 40.755234414999954], [-73.94243020699994, 40.75399761299994], [-73.94355728100004, 40.75282055199991], [-73.94374837099986, 40.75262098099987], [-73.9438353679999, 40.75253012299991], [-73.94391620499988, 40.75245068799991], [-73.94392521099985, 40.752441104999924], [-73.94396284399986, 40.752404075999905], [-73.94401823399977, 40.75234992099993], [-73.944052652, 40.75231325999992], [-73.94458401999987, 40.752534888999946], [-73.94481659399985, 40.75263169799989], [-73.94504698699998, 40.75273571599994], [-73.94552905199996, 40.752957406999876], [-73.9457177839999, 40.753044195999934], [-73.94597511599991, 40.75315619699993], [-73.94635771599988, 40.75332271699992], [-73.94655794699989, 40.75340986099995], [-73.94745112999996, 40.75379680499989], [-73.9482744449999, 40.75415377799993], [-73.94911295099988, 40.75452361999986], [-73.95080763809156, 40.75526368005708], [-73.95077122679736, 40.75530796907786], [-73.95073481497832, 40.7553522591792], [-73.95066341780087, 40.75543632291635], [-73.94915021804697, 40.75732894544871], [-73.94844487681875, 40.75821799715228], [-73.94783189558463, 40.75898919290095], [-73.9459642420993, 40.76129260680508], [-73.94580554802424, 40.761449567060474], [-73.94570525620351, 40.76148815702824], [-73.9447304898605, 40.76285654700463], [-73.94471460801532, 40.76287884021323], [-73.94350282032728, 40.76449904074385], [-73.94158523271753, 40.76661608765766], [-73.9413117494144, 40.76691800476961], [-73.94046256885548, 40.76753046980284], [-73.94035103784385, 40.7676533759874], [-73.93983559283014, 40.76798800593895], [-73.93953321663712, 40.76805874435426], [-73.9393658722388, 40.768159701376995], [-73.93895502453339, 40.76795818621701], [-73.9382038348605, 40.76872757186401], [-73.93757632517331, 40.76892399418493], [-73.93681460654729, 40.76890394747839], [-73.93668539119408, 40.76913305469088], [-73.93670737989197, 40.769252609074115], [-73.93640381736212, 40.769474884014315], [-73.93616913919553, 40.76951611465445], [-73.93574021432624, 40.76936722986653], [-73.93524016831192, 40.769679325462995], [-73.93489533894558, 40.77004186288951], [-73.93469499066988, 40.77042533852511], [-73.93514715402594, 40.77054195605843], [-73.9350081388611, 40.77116239443329], [-73.93494027200222, 40.771164020966346], [-73.93493892978363, 40.77116405283739], [-73.93490481291309, 40.77127064109334], [-73.93452730104707, 40.771176775051664], [-73.93424609599985, 40.771029274999925], [-73.93447449099997, 40.77041159699994], [-73.93458521599987, 40.770217420999856], [-73.93471718499994, 40.77002985799987], [-73.93486912299994, 40.76985103099989], [-73.93486925499987, 40.76985087799988], [-73.934869397, 40.76985073499985], [-73.93503989699988, 40.76968230199992], [-73.93522718099996, 40.76952572899989]]]}}, {\"id\": \"193\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 194, \"Shape_Leng\": 0.0817938353532, \"Shape_Area\": 0.000273481702719, \"zone\": \"Randalls Island\", \"LocationID\": 194, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91639852914044, 40.79768484203896], [-73.91545470966419, 40.79706346603092], [-73.91443827233297, 40.79566551786384], [-73.91380428276793, 40.79451184645597], [-73.91378385370595, 40.793836285482065], [-73.9145920846776, 40.79277660195909], [-73.91514134426104, 40.792012482994856], [-73.91535698910359, 40.79184304950765], [-73.91552117846275, 40.79178325024936], [-73.91630577076101, 40.79111736995261], [-73.91616965652562, 40.79101537285409], [-73.9168544196006, 40.79035844788467], [-73.91707800430397, 40.79045626926613], [-73.91710622312881, 40.79046861723939], [-73.91784071922032, 40.79017131993736], [-73.91812330871126, 40.79004023993423], [-73.91843338465661, 40.78980396433666], [-73.9185629977498, 40.78969533382588], [-73.91859001366782, 40.78947410390862], [-73.91850876959094, 40.78942508976939], [-73.92188898370831, 40.785701197162865], [-73.92199195097098, 40.78575394991961], [-73.92218831157655, 40.78550927813926], [-73.92285203280137, 40.783555693417696], [-73.92306473139391, 40.78334288477424], [-73.92401145782644, 40.78266068982501], [-73.9240136473876, 40.78265764398254], [-73.92406208829156, 40.78259027845207], [-73.92412113996787, 40.78252307401082], [-73.9242362187582, 40.78249691558926], [-73.92424754901657, 40.782494339126565], [-73.92431077958685, 40.78245595655429], [-73.92448052280419, 40.782320490645894], [-73.92467329671872, 40.78224486000101], [-73.92488394771702, 40.78222258277354], [-73.9250608790881, 40.78221629397282], [-73.92514021519148, 40.78224376759571], [-73.92515351840345, 40.78224837480777], [-73.92524195308084, 40.78227404884853], [-73.9253893908798, 40.78227414439881], [-73.92571808678538, 40.782176557944545], [-73.92713754068623, 40.78119426862356], [-73.92732422574854, 40.781090629324254], [-73.92754410768514, 40.781050635401634], [-73.9276039547454, 40.78103974834903], [-73.92769650608254, 40.78101478526652], [-73.92778746547549, 40.78098985891759], [-73.92810118993664, 40.78092032002105], [-73.92849269580674, 40.78100835324047], [-73.92917352795327, 40.78141057877102], [-73.9303098816472, 40.782077335895295], [-73.93063819695584, 40.78226141302441], [-73.93105352803921, 40.782396593783226], [-73.9326708060521, 40.782693639812365], [-73.93502345117236, 40.783006382014335], [-73.93540566292114, 40.78339449913405], [-73.935812997993, 40.78371008126878], [-73.93598226003293, 40.7839423680532], [-73.93607064658492, 40.78421721581008], [-73.93605332720759, 40.784512368076776], [-73.93600271220353, 40.78467537090368], [-73.93591823870035, 40.78480687998174], [-73.93579345324424, 40.785254781152005], [-73.93576384806458, 40.785446039146336], [-73.93571265998847, 40.78568048924624], [-73.93314749383154, 40.788691282071184], [-73.93270707318968, 40.789167192182106], [-73.93240673563922, 40.789718503046494], [-73.93224324588684, 40.78969251520576], [-73.93147595314876, 40.79047915912887], [-73.93084178719639, 40.79099143000079], [-73.93072940501884, 40.79095511421538], [-73.92978443004152, 40.79134147536451], [-73.92972400790947, 40.79136617817153], [-73.92966610521292, 40.79135578622649], [-73.92871948963382, 40.790943532756195], [-73.92859459823978, 40.790966924151896], [-73.92807273587826, 40.790823391141544], [-73.92737326899098, 40.79081082516722], [-73.9269225827856, 40.790788693001986], [-73.92680823697295, 40.79076444018542], [-73.92666714373019, 40.79073451389356], [-73.92641834021488, 40.7906933761935], [-73.92625431474322, 40.79069453300847], [-73.9261041024332, 40.79073160730996], [-73.92606242622999, 40.79082793616362], [-73.92606571253833, 40.79093668275996], [-73.9260860778176, 40.79100401293874], [-73.92608938664702, 40.791092046136995], [-73.92604500498163, 40.79117745932122], [-73.92604831400958, 40.791265491932684], [-73.92608708144499, 40.79135413308455], [-73.9261128771905, 40.791413114098766], [-73.92615360747125, 40.791547774940106], [-73.92616033990507, 40.79162027517715], [-73.92616023377902, 40.791716073510884], [-73.92627260981645, 40.791757571034445], [-73.92644290176645, 40.79179651769726], [-73.9265484745292, 40.79182765421353], [-73.9265688801793, 40.79185873681861], [-73.92651433019059, 40.791897539443355], [-73.92627928808834, 40.791879265875274], [-73.92614646074732, 40.7918481101675], [-73.92592839987515, 40.791876450921905], [-73.92576489923744, 40.79185822180665], [-73.92563608586745, 40.79182315574161], [-73.9255877914775, 40.79182185999814], [-73.92558094001407, 40.79185551410814], [-73.92560814781015, 40.79189695828146], [-73.9261702508689, 40.79189990812937], [-73.9264290966088, 40.791957035080074], [-73.92663010365783, 40.791946806944416], [-73.92666775380432, 40.79194057213957], [-73.92673913886138, 40.791928751852296], [-73.92670177076633, 40.791832930109756], [-73.92664049578529, 40.79179146422634], [-73.92657917174049, 40.79179401517148], [-73.92650765413156, 40.79177325647902], [-73.9263681028016, 40.791661834192126], [-73.92633080392882, 40.79150387311421], [-73.92634471493115, 40.79124755705965], [-73.9264266261548, 40.79111038629358], [-73.92648804311864, 40.791022393999704], [-73.92656584807987, 40.79097950185657], [-73.92670272281141, 40.790968158883764], [-73.92673885594114, 40.79097412083719], [-73.92682874582606, 40.790988952076695], [-73.92694456108919, 40.79099938201269], [-73.92701950689816, 40.790999429752766], [-73.92711487713235, 40.79101502423024], [-73.92716599683943, 40.79099693345351], [-73.92724094286736, 40.790996980621685], [-73.9273158787935, 40.791007383961684], [-73.92738061595081, 40.79099706866594], [-73.92743856860018, 40.7909608571228], [-73.9275101080249, 40.790960902131225], [-73.92756120582968, 40.79096352366368], [-73.92765319089136, 40.79095840286455], [-73.9277840996424, 40.790941603827065], [-73.9278688922052, 40.79095483831721], [-73.9278932871547, 40.79097884634049], [-73.92790202759703, 40.791009818668336], [-73.9279105358314, 40.79103914679122], [-73.92789374499282, 40.791091111210484], [-73.92765982803098, 40.79134833426527], [-73.92757830264749, 40.79146047486957], [-73.927320657043, 40.7917431602298], [-73.92727664462073, 40.791767327255954], [-73.92723286076209, 40.79179136818892], [-73.92713563764022, 40.7919293068015], [-73.92780141486193, 40.791997479131176], [-73.92800053586241, 40.79213364095345], [-73.92826720298079, 40.792328523974035], [-73.9283194359912, 40.79309126608271], [-73.9282921999328, 40.79315082224056], [-73.92826221599341, 40.79320287515347], [-73.927352245788, 40.79446084678257], [-73.92724362202914, 40.79477971085496], [-73.92709775283825, 40.79510394223802], [-73.92705117746826, 40.79517878406459], [-73.9270115547382, 40.79525590389574], [-73.92697907687112, 40.79533492894325], [-73.92695390083249, 40.79541547781581], [-73.9269361484504, 40.7954971609405], [-73.92692590462262, 40.79557958400356], [-73.92692321960746, 40.79566234890315], [-73.92690492714154, 40.795983851767936], [-73.92689066980925, 40.796234524362376], [-73.9268896786395, 40.79625201143132], [-73.92695192377981, 40.79670843214217], [-73.92703845685821, 40.79698879520749], [-73.9271115141623, 40.79719788006736], [-73.92718514800379, 40.797349896718686], [-73.9272029981222, 40.797386747692755], [-73.92721689278596, 40.7974154332488], [-73.9272772028445, 40.79740745307425], [-73.92743922192645, 40.79771053810798], [-73.92747588601745, 40.79812289983246], [-73.92726293295631, 40.798918645061036], [-73.92691995657893, 40.799837590489915], [-73.92690105422145, 40.79988823667607], [-73.92688261777174, 40.79994076875161], [-73.92685410186323, 40.799987559023], [-73.92680607671765, 40.800035900768776], [-73.92669572614771, 40.80014697707624], [-73.92666565490181, 40.80018271401784], [-73.92655351676811, 40.80031517296066], [-73.92653846445016, 40.80033389815394], [-73.9265025028775, 40.80037491715338], [-73.9266778180094, 40.8004496894687], [-73.9265524180424, 40.80060482504614], [-73.92650954883642, 40.80065703126224], [-73.92588046318872, 40.801551741962996], [-73.92582494619354, 40.80164248988908], [-73.92577551805864, 40.80173060099813], [-73.92569826189312, 40.801810871426575], [-73.9254929354651, 40.80195616882528], [-73.92529030602597, 40.801995586077716], [-73.9252752102012, 40.801998522743354], [-73.92436044107556, 40.80198555341901], [-73.92263378594883, 40.80186873814113], [-73.92215538898822, 40.80162646997485], [-73.92191092229565, 40.80150266625349], [-73.92133752419274, 40.8008521075022], [-73.92113594709092, 40.80047546009652], [-73.92103317645132, 40.800265281858074], [-73.92083898713122, 40.80001259800666], [-73.92057878116655, 40.79966814846022], [-73.92035479893732, 40.79942285236055], [-73.9203146552122, 40.79937545929002], [-73.91923785764347, 40.79888106768282], [-73.91914573748252, 40.79883898734595], [-73.91906560282143, 40.79880279731935], [-73.91899496843433, 40.79877184320446], [-73.9189872445785, 40.798769712219844], [-73.91891076100565, 40.79874013836822], [-73.9179799205819, 40.79838020192752], [-73.9179027981093, 40.79836265323148], [-73.91662985247798, 40.79785813943717], [-73.91639852914044, 40.79768484203896]]]}}, {\"id\": \"194\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 195, \"Shape_Leng\": 0.131308244067, \"Shape_Area\": 0.000801651953985, \"zone\": \"Red Hook\", \"LocationID\": 195, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.0071710969999, 40.68635700999991], [-74.00549714599994, 40.6858477269999], [-74.00485241599992, 40.685668130999886], [-74.0051790759999, 40.68499722499988], [-74.0054980969999, 40.68432595799987], [-74.00581925399995, 40.683656554999914], [-74.00596041099988, 40.683362870999936], [-74.00567492299993, 40.68292992799995], [-74.00560068499993, 40.68281599899994], [-74.00503641499995, 40.681910287999884], [-74.00456896599997, 40.68114997599986], [-74.00447389899988, 40.680993011999846], [-74.00378241799994, 40.6800860019999], [-74.00371762999994, 40.68002103199993], [-74.00345372099991, 40.679789886999885], [-74.0033891689999, 40.679721305999905], [-74.00329231299999, 40.67964328799992], [-74.00310560799991, 40.67933188599993], [-74.00255365299998, 40.678658683999856], [-74.00221991299993, 40.678260321999836], [-74.00215698499991, 40.67818268099997], [-74.00153194399996, 40.67727039699989], [-74.00102822999993, 40.67641403999991], [-74.0006740919999, 40.67555578899988], [-74.00083254999997, 40.67561225999985], [-74.00090355299992, 40.67563696899988], [-74.00112481699996, 40.67569755399992], [-74.00105973699986, 40.675555677999945], [-74.00077979399994, 40.67529390099993], [-74.00045011499992, 40.674769425999926], [-73.99994578200001, 40.67392615699986], [-73.99991051299993, 40.673872729999836], [-73.99983349199978, 40.673756049999945], [-73.99934534299987, 40.67300654899992], [-73.99903998999996, 40.67250767899989], [-73.99898242799983, 40.67241358399986], [-73.9986188799999, 40.67183017199989], [-73.99868108199985, 40.67173915799989], [-73.99876941079444, 40.671602047052545], [-73.99892591269524, 40.67141490575386], [-73.99931101149453, 40.67077263902337], [-73.99962139936635, 40.67012716794642], [-73.9996281368291, 40.67011307640566], [-73.99991082003736, 40.66949866821084], [-74.00030337854434, 40.66871689224847], [-74.0010566411318, 40.668945681717275], [-74.00141527103162, 40.66835941434697], [-74.00221506816538, 40.668578637441165], [-74.00289483876944, 40.66734846594933], [-74.00319075274831, 40.667263199814926], [-74.00369462000533, 40.66742761120842], [-74.00363026351538, 40.66756752335649], [-74.00398842006275, 40.66766564164089], [-74.0044025343906, 40.66777908632258], [-74.00510111198017, 40.66797532110033], [-74.00510227346263, 40.667975647728206], [-74.00547017208598, 40.66795127142809], [-74.00552727453992, 40.667899204711475], [-74.0058636618068, 40.66799376258568], [-74.00597850124247, 40.667882004170494], [-74.00600812868151, 40.66788538149734], [-74.00612925948894, 40.66761146714417], [-74.00625704452314, 40.66733351369736], [-74.0062287607646, 40.667325973125315], [-74.00631442722586, 40.66713963653244], [-74.00656004211396, 40.66663591439601], [-74.0065952902589, 40.666561084187165], [-74.00677062553615, 40.6661888624435], [-74.00683085521844, 40.66620895670721], [-74.00686713038442, 40.66612878941262], [-74.00686084931154, 40.6661261855656], [-74.00685037903985, 40.666120565663505], [-74.00684130973484, 40.66611367686599], [-74.00683390334237, 40.66610571814059], [-74.00682837377936, 40.666096919357486], [-74.00682488075381, 40.666087534649485], [-74.00682352515308, 40.66607783507295], [-74.00682434612966, 40.66606810077804], [-74.00682731997071, 40.66605861291754], [-74.00683236078251, 40.666049645526776], [-74.00683932297235, 40.66604145760807], [-74.0068480054524, 40.666034285650504], [-74.00685815744939, 40.666028336799286], [-74.00686948574585, 40.6660237828732], [-74.00688166315058, 40.66602075540171], [-74.00689433794756, 40.666019341826235], [-74.0069071440549, 40.6660195829746], [-74.00691971159857, 40.66602147188169], [-74.00693167759442, 40.66602495399107], [-74.00694269643263, 40.66602992872995], [-74.00695244985948, 40.66603625241477], [-74.00696065616967, 40.66604374240062], [-74.00696707834247, 40.666052182356665], [-74.00697153088751, 40.66606132851423], [-74.00697388520256, 40.666070916707675], [-74.00697407328752, 40.66608067000397], [-74.00697208970928, 40.66609030670144], [-74.00696799175799, 40.666099548465645], [-74.00696189779265, 40.66610812836936], [-74.00695398382344, 40.66611579860115], [-74.00694447842689, 40.66612233762348], [-74.00693365614498, 40.66612755657132], [-74.00692182955493, 40.666131304706845], [-74.00690934024162, 40.66613347377333], [-74.00689654893131, 40.666134001122344], [-74.0068962959208, 40.666133995609044], [-74.00688141944153, 40.666132467137324], [-74.00686975908653, 40.666159079647606], [-74.00684452428645, 40.66621244834966], [-74.00689626622095, 40.6662233826561], [-74.00675023792375, 40.666538299107025], [-74.00630635035533, 40.667495533618066], [-74.00625662073615, 40.667595754713815], [-74.00604596458896, 40.66805324348046], [-74.00531907552738, 40.66955428536638], [-74.00475388692331, 40.67072135894501], [-74.00543926529353, 40.6709214682384], [-74.0060285141717, 40.66978474627907], [-74.0066260987057, 40.66863190113328], [-74.00680345938906, 40.668591375668285], [-74.00754821618384, 40.66705174573358], [-74.00779653349619, 40.66715977077724], [-74.00708724221813, 40.668591358535764], [-74.00985411855632, 40.668523628335336], [-74.01154017543327, 40.6650859197566], [-74.01574872882654, 40.6645683362916], [-74.01690267361069, 40.66484660233888], [-74.01744446051003, 40.665267996351716], [-74.01788719149395, 40.66585046628534], [-74.01857605010865, 40.67039241386994], [-74.01876336878675, 40.67107587449437], [-74.01933946871777, 40.671624948422625], [-74.01888283363327, 40.67160166620541], [-74.01751037086694, 40.671034530984144], [-74.01668768368882, 40.66543706726946], [-74.01657536097834, 40.66503853783983], [-74.01648665554487, 40.664930508953816], [-74.01602551776006, 40.66486304878802], [-74.0159369853455, 40.66547079172142], [-74.01227967872792, 40.66573241415882], [-74.01087664028981, 40.66860161576353], [-74.0103568366731, 40.6686428353885], [-74.01033845947926, 40.669078195448], [-74.0105598064476, 40.66919052506965], [-74.01091024340404, 40.66919049236947], [-74.01087343978188, 40.66971011555842], [-74.01154288031563, 40.669784342750255], [-74.01162173116997, 40.67011434594473], [-74.01151443463287, 40.67040702338137], [-74.01162527521356, 40.670581028275876], [-74.01406219828176, 40.66970583843646], [-74.01424926931875, 40.66972860473708], [-74.01424180477498, 40.66980837053881], [-74.01175350509699, 40.670657757578326], [-74.01177191195623, 40.67070305441849], [-74.01179735454902, 40.67074633196385], [-74.01182946197498, 40.670786959294624], [-74.01186776616825, 40.6708243441261], [-74.01191170872066, 40.67085794144325], [-74.01196290733463, 40.67088844675834], [-74.01201873929931, 40.67091380572068], [-74.0120783175301, 40.67093361541317], [-74.01214069541709, 40.670947561088035], [-74.01220488186557, 40.670955421168514], [-74.01226985704388, 40.67095707076885], [-74.01233458858765, 40.67095248367923], [-74.0123980480043, 40.67094173278214], [-74.01245922701378, 40.67092498889416], [-74.01251715357101, 40.670902518051534], [-74.01257090731009, 40.67087467728396], [-74.01261963416778, 40.670841908940496], [-74.01355198671448, 40.6712569343771], [-74.0135983928744, 40.67122691731589], [-74.01365024646175, 40.67120261272598], [-74.01370634525973, 40.67118458410269], [-74.0137628109139, 40.67117359469209], [-74.01382077477412, 40.67116897109751], [-74.01387900570414, 40.67117081152283], [-74.01393626689504, 40.671179076877884], [-74.01399134213433, 40.67119359160873], [-74.01427407257738, 40.67137929918188], [-74.01508006471072, 40.670685673295516], [-74.0151998057007, 40.6707540276644], [-74.01436920648234, 40.67145479665874], [-74.01412302856023, 40.67166933413735], [-74.01387273763235, 40.67188076176044], [-74.01423289452782, 40.67213000051297], [-74.01559644489554, 40.67097047751736], [-74.01568625497515, 40.671038834928844], [-74.01428722178993, 40.67224118299989], [-74.01422923190452, 40.672207205492896], [-74.0129758779219, 40.67332791872093], [-74.01336378218133, 40.67359153693441], [-74.0139510936524, 40.6731115881933], [-74.01511918663962, 40.67215699596904], [-74.01632099922307, 40.6729082695637], [-74.0145794672095, 40.67441401707156], [-74.01496392184576, 40.67467763029424], [-74.01543956638726, 40.674272882158625], [-74.01670765236591, 40.67319377442169], [-74.01816839280792, 40.67193924060107], [-74.0188009477692, 40.6722507648487], [-74.01727138470127, 40.67360864580548], [-74.0172928251957, 40.67362821502458], [-74.01775247096357, 40.67404772289249], [-74.01779805190311, 40.674015898973224], [-74.01781515223432, 40.674030357995235], [-74.01789301471658, 40.673965264550155], [-74.01787971614986, 40.67395658985093], [-74.01794428080251, 40.673882820496836], [-74.01801077320536, 40.67391318105236], [-74.01795760572409, 40.67398694866508], [-74.01792341013338, 40.67397538419332], [-74.01783604921178, 40.674040479131776], [-74.01785694945181, 40.67405783059562], [-74.01780187203198, 40.674095441412426], [-74.01811536390225, 40.67434270168223], [-74.01813640515077, 40.67435864540075], [-74.01815361380635, 40.67437706226807], [-74.01816649698902, 40.67439742480279], [-74.01817468570437, 40.67441914979691], [-74.01817794541266, 40.67444161501923], [-74.01817618274653, 40.674464177036725], [-74.01804519537194, 40.67476791076817], [-74.01802399440471, 40.67485275836311], [-74.01803004073714, 40.67492410782759], [-74.01803574975753, 40.6749588168435], [-74.01800916427273, 40.67499353142102], [-74.01801676669672, 40.675009439505125], [-74.01792370591399, 40.67507598132641], [-74.01798640472803, 40.67512514362106], [-74.01783447572689, 40.67525822301435], [-74.01814986189117, 40.67546787941486], [-74.01819544347482, 40.67544039313992], [-74.01822964241144, 40.67546208190084], [-74.01829802655324, 40.675459179181395], [-74.0184123582175, 40.675365449934176], [-74.01832457843103, 40.67530442504134], [-74.01826950310674, 40.67534926808163], [-74.01818210632884, 40.67529143140757], [-74.0183929089602, 40.675107724147864], [-74.01847650481606, 40.67516266940931], [-74.01842523089353, 40.67521474157496], [-74.01851887130543, 40.6752738896401], [-74.01875814086333, 40.67506448439689], [-74.01876801869415, 40.675060661401396], [-74.01877860226983, 40.675058162481996], [-74.01878958796179, 40.6750570593293], [-74.01880066060488, 40.67505738359122], [-74.01881150253966, 40.675059125965184], [-74.01882180272558, 40.67506223646492], [-74.01883126566364, 40.67506662585417], [-74.01883961987457, 40.67507216820711], [-74.0188466256867, 40.675078704520935], [-74.01885208211192, 40.67508604727744], [-74.01885583261196, 40.675093985822286], [-74.0188577695892, 40.6751022924089], [-74.01885783747345, 40.67511072873192], [-74.01885603431671, 40.67511905276391], [-74.0188524118484, 40.67512702569873], [-74.0184086952087, 40.675524021071425], [-74.01841657994261, 40.67554307574418], [-74.01842802392619, 40.67556105358067], [-74.01844278335878, 40.67557757157944], [-74.01846054380597, 40.67559227783935], [-74.01848092689923, 40.67560485905655], [-74.01853619417705, 40.67577301703708], [-74.01853402430305, 40.6757915353948], [-74.01852767505498, 40.675809487866694], [-74.01851733829146, 40.67582633196449], [-74.0185033263665, 40.67584155869196], [-74.01848606269128, 40.67585470792659], [-74.01846606893977, 40.675865382323295], [-74.01844394928432, 40.6758732593221], [-74.01842037213926, 40.67587810089463], [-74.01839604996148, 40.675879760737644], [-74.018018007396, 40.67588952276334], [-74.01797535616167, 40.67595301480272], [-74.01794001437204, 40.6760190579041], [-74.01791224249098, 40.67608716538086], [-74.01789224519982, 40.676156835331945], [-74.01788016988853, 40.67622755434057], [-74.017876105569, 40.676298801257545], [-74.01788008221803, 40.676370051041765], [-74.01789207055593, 40.67644077862896], [-74.01791198226178, 40.676510462801936], [-74.01797569982641, 40.67651692060813], [-74.01797996235578, 40.676571898356606], [-74.01910985389938, 40.67654584509902], [-74.01912261786643, 40.676616989159406], [-74.01976925613225, 40.67661322491722], [-74.01995197864673, 40.67710301044993], [-74.01908257386326, 40.67713956404753], [-74.01914183574368, 40.67772313432773], [-74.01918927313335, 40.6777411827303], [-74.01919098767956, 40.677743271179814], [-74.01921892737656, 40.677777288546444], [-74.01917743946889, 40.677826949007], [-74.01908763812094, 40.677831236398745], [-74.01908257644466, 40.67783147881331], [-74.01906541502083, 40.67785015211888], [-74.01904108711257, 40.677876624017756], [-74.01903387384746, 40.677876998462985], [-74.01843040661609, 40.67790832186485], [-74.01842965849707, 40.677927584823045], [-74.01841103183156, 40.67840712127842], [-74.0185950422009, 40.67839777362912], [-74.01862532606161, 40.678369581525644], [-74.01862787070775, 40.678369581113714], [-74.01862981462028, 40.67836958079898], [-74.01864991332955, 40.678384875837374], [-74.01866235777157, 40.67839434600305], [-74.01895405840415, 40.67838148630463], [-74.01898322115336, 40.678352440235145], [-74.01898995247258, 40.678353293180585], [-74.01902137413651, 40.67837720491473], [-74.0194723889745, 40.67835833705062], [-74.01953856537544, 40.67830195259354], [-74.01966200306937, 40.678385638958936], [-74.01966312651498, 40.67838820127059], [-74.01955208101631, 40.67848217760074], [-74.019518417501, 40.67845912061588], [-74.01942864377901, 40.67840019839305], [-74.01942574949798, 40.67840028231049], [-74.0193650147007, 40.67840202997831], [-74.01927942798392, 40.67840449494622], [-74.01927671763752, 40.67840677243678], [-74.01927485273436, 40.678408339468156], [-74.01924689963592, 40.678431833373594], [-74.01924241195326, 40.6784318341242], [-74.0192065036037, 40.6784079243722], [-74.01912079360895, 40.67841256190949], [-74.01906401917516, 40.6784156345743], [-74.0190326112239, 40.678442972755384], [-74.01903013778114, 40.678442658547496], [-74.01902587989507, 40.67844211914231], [-74.01899445934593, 40.67841991686206], [-74.01899029164359, 40.67842001639134], [-74.01898714122163, 40.67842009213119], [-74.01885085228994, 40.678423357206974], [-74.01884757078854, 40.6784260174143], [-74.01882433869434, 40.67844485367609], [-74.01881607949576, 40.67845155010387], [-74.01881159159161, 40.678451550837714], [-74.01878017201017, 40.67843020272259], [-74.01877503939653, 40.67843032099709], [-74.01871751760083, 40.67843166874764], [-74.01863319772804, 40.67843364217266], [-74.01863191295386, 40.678434762303326], [-74.01860179179259, 40.67846098140842], [-74.01859998188733, 40.67846098170102], [-74.01859505982546, 40.67846098249659], [-74.01856476167906, 40.67843963246608], [-74.01851059175075, 40.67844051183506], [-74.01840544610997, 40.67844222006226], [-74.01841786210231, 40.67854955189745], [-74.01825267004557, 40.67855288672587], [-74.01828314900892, 40.67872828941208], [-74.01832227122634, 40.67872166500742], [-74.01838313797529, 40.67874482169806], [-74.01857882788774, 40.678983079472324], [-74.01879335612368, 40.678976333138394], [-74.0192135121457, 40.67896311902049], [-74.01943955737198, 40.678933293624226], [-74.01945620040587, 40.67893245234802], [-74.01947279936257, 40.67893370013941], [-74.01948890488148, 40.67893700321873], [-74.01950408096025, 40.6789422721663], [-74.01951791675745, 40.678949364343204], [-74.01953003771537, 40.67895808775279], [-74.01954011569893, 40.67896820623785], [-74.01954787787973, 40.67897944587447], [-74.01955311412145, 40.678991502387134], [-74.0195556826687, 40.679004049385874], [-74.01955551398514, 40.67901674720213], [-74.01955261263586, 40.67902925208458], [-74.01928128101008, 40.67964814029763], [-74.01906833414864, 40.67976074748824], [-74.01892813994154, 40.6798443050555], [-74.01899105402694, 40.67990499335191], [-74.01772532586205, 40.68066190283674], [-74.01764735092732, 40.680601890920784], [-74.01757049414064, 40.68065064806215], [-74.01746928868556, 40.68061473185043], [-74.01684837001166, 40.680985758993785], [-74.01678833975089, 40.68102196539755], [-74.01520082289557, 40.68197941921397], [-74.01532059799361, 40.682110728890265], [-74.0150829519353, 40.68225409916025], [-74.01486782800362, 40.6823832140236], [-74.01282000316418, 40.68362241624865], [-74.01262428295385, 40.683468696195796], [-74.01212285893645, 40.68296217785077], [-74.01213481688791, 40.68290592802506], [-74.01410549647976, 40.68171251937541], [-74.01360863270277, 40.681221946728165], [-74.01387398777976, 40.681009812816335], [-74.013742066444, 40.68091770134298], [-74.013018354221, 40.68041237401427], [-74.01186281377171, 40.6813984795506], [-74.01060858082327, 40.68246876026793], [-74.00967108558879, 40.683268727715], [-74.0100860394785, 40.683655431043526], [-74.01077312339376, 40.68435861554084], [-74.01178272201199, 40.68375999549927], [-74.01193259955522, 40.683887749077535], [-74.01092841300002, 40.684491472999824], [-74.0086007239999, 40.68590956499989], [-74.00851294299997, 40.68596254899994], [-74.00842516199995, 40.68601553199992], [-74.00816320571415, 40.68617364485845], [-74.00816181924652, 40.686174423074014], [-74.00798157801087, 40.68627511006105], [-74.00793423715932, 40.686281348969096], [-74.00788683480657, 40.68624536814394], [-74.00782671596491, 40.68624036700248], [-74.00776820497654, 40.68624901912176], [-74.00771707547891, 40.68626442900919], [-74.00753083199993, 40.68607448699991], [-74.0071710969999, 40.68635700999991]]]}}, {\"id\": \"195\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 196, \"Shape_Leng\": 0.0832421106332, \"Shape_Area\": 0.0001974713336, \"zone\": \"Rego Park\", \"LocationID\": 196, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85762923599995, 40.728606921999884], [-73.85771592500002, 40.728505460999926], [-73.85741240699991, 40.72832247999986], [-73.85775002599989, 40.72844560099985], [-73.8580876569999, 40.728568721999864], [-73.85812618599992, 40.728508245999905], [-73.85820066499991, 40.72839135199992], [-73.85860783699987, 40.72775228399992], [-73.85901374300002, 40.7271089069999], [-73.8594198799999, 40.726464837999885], [-73.85727108100001, 40.72568129799988], [-73.85776316799985, 40.72490571199994], [-73.85805416899993, 40.72444540599986], [-73.85960517199996, 40.7250223999999], [-73.86015205599992, 40.725227040999876], [-73.86203833799982, 40.72595678499994], [-73.85990796499992, 40.72434623399985], [-73.8586809759999, 40.72261243999988], [-73.85852214099987, 40.72153379399993], [-73.85823757099998, 40.719612410999936], [-73.85777168499992, 40.71645629299992], [-73.85744857899988, 40.7136927999999], [-73.85729287299985, 40.711467676999874], [-73.8581874529999, 40.7115995459999], [-73.85964591099994, 40.711827343999886], [-73.85975932900001, 40.711845058999856], [-73.8597131359999, 40.71260553899993], [-73.85966066799995, 40.71329464599993], [-73.85965416799995, 40.713379960999866], [-73.8596317549999, 40.71359826399985], [-73.85962920999985, 40.713975294999926], [-73.85967906599994, 40.71417332399993], [-73.8597452479999, 40.71436917699986], [-73.85982746199984, 40.714561688999915], [-73.85992523600002, 40.714749735999945], [-73.86003789899996, 40.71493223599987], [-73.86062663399984, 40.71548716899991], [-73.86122406399997, 40.7160373289999], [-73.861528867, 40.716314936999915], [-73.86183368299996, 40.716592542999884], [-73.86213638199993, 40.71687677399989], [-73.86242540499987, 40.717148146999854], [-73.86272195199989, 40.71742801899989], [-73.86301850099984, 40.717707881999964], [-73.8635711119999, 40.71827667999986], [-73.86412297599996, 40.718844806999925], [-73.86468729499988, 40.719427075999874], [-73.86523870699996, 40.719976709999905], [-73.86547610900004, 40.72028093699992], [-73.86604176200004, 40.72090092299985], [-73.86657241099984, 40.721486636999884], [-73.86711404599988, 40.72207640399993], [-73.86730908299992, 40.72228902499991], [-73.86765927399983, 40.72266125599994], [-73.86821068499994, 40.72325474899987], [-73.86845305199996, 40.723485493999874], [-73.8687681729999, 40.723820906999904], [-73.8692721569999, 40.72441183599992], [-73.86929270499994, 40.72443592899991], [-73.86931025899987, 40.72446599099993], [-73.869707654, 40.72514643399992], [-73.87021135899995, 40.726187325999916], [-73.87063185099991, 40.7270588539999], [-73.87074560999994, 40.72726953899994], [-73.87084539399993, 40.72748459299991], [-73.87093081299994, 40.72770320699995], [-73.87100156100003, 40.727924523999896], [-73.8711328349999, 40.7283717199999], [-73.87140980799985, 40.729178101999935], [-73.87146074199987, 40.729326373999854], [-73.87154749999993, 40.72936238299992], [-73.87235229600002, 40.72965548399991], [-73.87316942599995, 40.72995061899995], [-73.87408913199982, 40.73028279199987], [-73.87475475600003, 40.73054189099991], [-73.8749534949998, 40.73061869599996], [-73.87509154199992, 40.73067136299992], [-73.87481299, 40.73075339399985], [-73.87415476600002, 40.73095008599991], [-73.87315527199992, 40.731243969999866], [-73.87192058399992, 40.731613928999884], [-73.87149800699996, 40.73175058699994], [-73.87134309799984, 40.7317993569999], [-73.87128940500001, 40.7318162619999], [-73.87112231399985, 40.73187152899993], [-73.87054725300005, 40.73203701799992], [-73.86955585299982, 40.73235284299989], [-73.8690483869999, 40.73251391799992], [-73.8686094929999, 40.732653176999854], [-73.86845203499989, 40.732713375999886], [-73.86835853299982, 40.732740158999874], [-73.8683180679999, 40.73275361999993], [-73.86827761499993, 40.732767071999874], [-73.86810582299985, 40.73282625499992], [-73.86792586499989, 40.73287256899993], [-73.86682801299989, 40.733245779999926], [-73.8661043939999, 40.73349176299991], [-73.86524772599986, 40.73376224899993], [-73.86491748899991, 40.733866509999864], [-73.86447086199989, 40.73400727399998], [-73.86450230199999, 40.73407539899988], [-73.864572486, 40.73418839799996], [-73.86462193199988, 40.73428275699994], [-73.86284000199998, 40.734799493999965], [-73.86106394499986, 40.73531449099991], [-73.85942834899984, 40.73578873199989], [-73.85886555799988, 40.73593913899987], [-73.85877141199997, 40.7357852899999], [-73.85872465799989, 40.73570864399988], [-73.85867078199992, 40.73561895999997], [-73.8585833349998, 40.73547194199987], [-73.85834345899994, 40.73500123999988], [-73.85799293699985, 40.73433365699993], [-73.85765182999988, 40.73366788599986], [-73.85730927099998, 40.73300411099996], [-73.85704937999998, 40.732512433999894], [-73.85659673999993, 40.731623059999926], [-73.85625665099991, 40.73095912299993], [-73.85851097899999, 40.73028680199993], [-73.85837971299996, 40.73003567799985], [-73.85816500199988, 40.72962493499991], [-73.857827674, 40.72897471999988], [-73.85762923599995, 40.728606921999884]]]}}, {\"id\": \"196\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 197, \"Shape_Leng\": 0.108568532229, \"Shape_Area\": 0.000504689160432, \"zone\": \"Richmond Hill\", \"LocationID\": 197, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83696237799982, 40.70421245199992], [-73.83661882199989, 40.70342001799988], [-73.83644057000006, 40.70345494699991], [-73.83633362899995, 40.70342921199993], [-73.83621686499995, 40.703418726999864], [-73.8360961639999, 40.703425946999886], [-73.83597857799987, 40.70345153499989], [-73.8358710909998, 40.703493947999874], [-73.83577932099996, 40.70354977499989], [-73.83577924899996, 40.70354982099987], [-73.83577918999993, 40.70354987399992], [-73.83570634099988, 40.703614678999934], [-73.83548255899987, 40.70370111499994], [-73.83475458000002, 40.7038869019999], [-73.83460244999996, 40.70392746699988], [-73.83391709699988, 40.70234736699992], [-73.83302084599993, 40.702570325999915], [-73.83213204199994, 40.7027940809999], [-73.83152109599999, 40.70138962899994], [-73.830629147, 40.70161654599988], [-73.82973796799996, 40.7018403959999], [-73.82946917899983, 40.701910183999864], [-73.82884805700002, 40.702065798999875], [-73.82795308, 40.702289961999895], [-73.82706578399991, 40.702515056999886], [-73.82607186999988, 40.70277207899992], [-73.82517586799999, 40.70299178799996], [-73.82426993499999, 40.70321932799986], [-73.82387070899986, 40.703322241999906], [-73.82338938599995, 40.703443280999885], [-73.822708114, 40.70362481299992], [-73.82245890399994, 40.70365553099989], [-73.821478249, 40.703415603999936], [-73.82053608399994, 40.70322714699992], [-73.8170868249999, 40.702570330999876], [-73.81686030399996, 40.70244433099987], [-73.81647531200002, 40.70245042599993], [-73.81637824499992, 40.70245196399987], [-73.81613853799999, 40.70193756999992], [-73.81593703000003, 40.701505116999904], [-73.81551777599985, 40.70060539199991], [-73.81546252299998, 40.70049286399993], [-73.81540727999986, 40.700380337999874], [-73.81537190699999, 40.70031031099995], [-73.81513940199982, 40.69985000899991], [-73.81498727099982, 40.699548830999966], [-73.8149234329999, 40.69942243999986], [-73.81466945899993, 40.69897071199993], [-73.81441548799988, 40.698518972999885], [-73.81430922799986, 40.69833533199988], [-73.81415191799988, 40.69803996999988], [-73.8140705039998, 40.69788712999989], [-73.81369419799996, 40.697266795999894], [-73.81365634399988, 40.69720105099985], [-73.81361850799995, 40.69713673899985], [-73.81338900499996, 40.6967577539999], [-73.81315381599988, 40.69636935899991], [-73.81299801799996, 40.6961120629999], [-73.81245207999991, 40.69519812499991], [-73.81224188600005, 40.69483223599984], [-73.8120913559999, 40.694559620999875], [-73.81202952199988, 40.69444762999992], [-73.81176990199997, 40.69399390499992], [-73.81172663199997, 40.6939192989999], [-73.81139791299987, 40.69332231399992], [-73.81102158699986, 40.69262288699985], [-73.81075433199987, 40.69215870099987], [-73.81063298299996, 40.69194738299988], [-73.81072587499985, 40.69192242199995], [-73.8109881059999, 40.691855044999926], [-73.81186315099997, 40.69163007999991], [-73.81271762199997, 40.69140780299994], [-73.81309863099992, 40.6913181019999], [-73.81354014199995, 40.691211367999905], [-73.81438438, 40.69097195299984], [-73.81522826699991, 40.69073347699986], [-73.81607487599989, 40.690492269999886], [-73.8169264139999, 40.69025014699988], [-73.81818710999987, 40.68989207499987], [-73.81906075299987, 40.689642441999894], [-73.81993922399998, 40.689393691999925], [-73.82085264199988, 40.68913438299991], [-73.82189346699995, 40.688838225999866], [-73.82293216299993, 40.688543000999836], [-73.82398034299996, 40.68824526799993], [-73.82501128899997, 40.68795231799991], [-73.82599347399987, 40.689953661999894], [-73.82685467499992, 40.689709535999974], [-73.82768417699987, 40.68947479999991], [-73.82851136799994, 40.689239055999934], [-73.82753021399995, 40.687239482999914], [-73.82835620899998, 40.6870028089999], [-73.82918242299996, 40.68677080599985], [-73.83003934099995, 40.68652935899991], [-73.83088556299994, 40.686286547999885], [-73.83172298299989, 40.6860507049999], [-73.83257747700002, 40.685806345999914], [-73.833421907, 40.68556778199997], [-73.83426545999994, 40.685327337999894], [-73.8351108629999, 40.6850876819999], [-73.83595082799985, 40.68484354599989], [-73.83717651699999, 40.68449539199987], [-73.83807769099997, 40.68423890999988], [-73.83697486299991, 40.6819880179999], [-73.837115114, 40.681933891999954], [-73.83713815499993, 40.68192499999988], [-73.83772037399996, 40.681700306999915], [-73.8377888039999, 40.681673896999904], [-73.8377973829999, 40.681670585999875], [-73.83865065799998, 40.68143823699989], [-73.83952877099979, 40.68126341499991], [-73.84012856099982, 40.6811466509999], [-73.84119605699982, 40.68335136599989], [-73.84217464599998, 40.6853535589999], [-73.84300297499996, 40.6870379159999], [-73.84356447799996, 40.68821396099985], [-73.84386364499983, 40.688832111999936], [-73.84415010299988, 40.6893932859999], [-73.84419150899994, 40.689474407999924], [-73.84430714400003, 40.68944166799988], [-73.84500533199989, 40.689243910999885], [-73.84504194999991, 40.689318946999904], [-73.84596733099984, 40.69116596499989], [-73.84661793899996, 40.692495182999885], [-73.84773325500002, 40.69477527999988], [-73.84894128999987, 40.697556847999905], [-73.84947763499991, 40.69819580699987], [-73.84913837299978, 40.69828274099995], [-73.84820912899981, 40.69851840899985], [-73.84789669799994, 40.69860032899992], [-73.84700405300002, 40.698825210999914], [-73.84502472899999, 40.6993593769999], [-73.84427352200001, 40.69978050599992], [-73.84359398099988, 40.70017235199988], [-73.84293789599988, 40.700554428999894], [-73.84275678399989, 40.70065761499992], [-73.84275664199988, 40.70065769499992], [-73.8427564879998, 40.70065776799991], [-73.8425654379999, 40.700750565999925], [-73.84256525999987, 40.70075065499986], [-73.84256506999995, 40.70075073599988], [-73.84236544299989, 40.7008324279999], [-73.84236528999996, 40.700832489999904], [-73.842365123, 40.700832543999915], [-73.84215844799995, 40.700902563999875], [-73.84156378099983, 40.70110059799985], [-73.83947815200001, 40.70269019399994], [-73.83856418299983, 40.70329696899991], [-73.83696237799982, 40.70421245199992]]]}}, {\"id\": \"197\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 198, \"Shape_Leng\": 0.134094242763, \"Shape_Area\": 0.000499809882564, \"zone\": \"Ridgewood\", \"LocationID\": 198, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90768387299997, 40.71737969499987], [-73.9069637079999, 40.717304416999866], [-73.90688152399996, 40.7173129149999], [-73.90639010099999, 40.71736369199989], [-73.90617600199982, 40.71642140399994], [-73.90602078299982, 40.7157426789999], [-73.90586816399987, 40.71506869299989], [-73.90574342600004, 40.71451601199994], [-73.90571002399983, 40.7143679749999], [-73.90556258999989, 40.71371463299989], [-73.90537582199983, 40.71287658099986], [-73.90489449600001, 40.712856814999924], [-73.90477244599997, 40.71286579999984], [-73.9037694399998, 40.712817031999904], [-73.90293472899992, 40.71278607299988], [-73.90224267799981, 40.712759832999886], [-73.900595879, 40.712697384999906], [-73.90018380399995, 40.71268065799996], [-73.89935504399993, 40.71264701399986], [-73.89768322699987, 40.71258029799992], [-73.89710338699993, 40.71255773699993], [-73.89621914099989, 40.712522056999944], [-73.8959038609999, 40.71250695899986], [-73.89502461000001, 40.71247825299989], [-73.89401868699997, 40.7124405549999], [-73.89367393299992, 40.71243002499995], [-73.88877304399992, 40.71228017799994], [-73.88842000699994, 40.71226995799984], [-73.88687942999985, 40.71222318499987], [-73.88637649699994, 40.71220771499993], [-73.88622785599988, 40.712202380999926], [-73.88519762999981, 40.71216627399991], [-73.88507555699987, 40.71216105499991], [-73.88435098699988, 40.712230172999966], [-73.88205387399988, 40.712454149999864], [-73.88006267699978, 40.71265257399993], [-73.87907685299999, 40.71035938999992], [-73.87943495499985, 40.71028441899988], [-73.87926181200001, 40.70990534699992], [-73.87872783199988, 40.71002753499987], [-73.87761134199985, 40.70716770499993], [-73.88086188499989, 40.7065847289999], [-73.88088332699985, 40.7064539769999], [-73.8817801769999, 40.706404547999966], [-73.88266573799994, 40.70635551699991], [-73.88377296899992, 40.70629420199989], [-73.88456244600002, 40.70636218699992], [-73.88553214199999, 40.70644568299984], [-73.88554946899997, 40.70615461899994], [-73.88652826799989, 40.706225205999864], [-73.8873957249999, 40.70601812999995], [-73.88826271699995, 40.70580991299987], [-73.88911917799989, 40.705598876999915], [-73.88984330199986, 40.704972401999896], [-73.89105316999985, 40.70381641299988], [-73.8912683769999, 40.70359657599989], [-73.89194157299991, 40.70288622799989], [-73.89283458099982, 40.70191027699992], [-73.89384018399997, 40.70085972599988], [-73.8942016529999, 40.70050730899993], [-73.89500322699999, 40.69971237699985], [-73.89484994999988, 40.698589622999876], [-73.89578005099996, 40.69851096199989], [-73.89558457599986, 40.697127641999884], [-73.89677249999983, 40.696799116999884], [-73.89756262999987, 40.696480707999896], [-73.89838185199996, 40.69613599799993], [-73.89888585799994, 40.695389474999935], [-73.89954093099989, 40.69428573099985], [-73.900030392, 40.693437450999895], [-73.90038667799995, 40.69313995999987], [-73.90103661799981, 40.691845252999904], [-73.90123290699994, 40.69144227899997], [-73.90148893599996, 40.6915568839999], [-73.90170525399989, 40.69166024399989], [-73.90207277399993, 40.691999230999905], [-73.9027906169999, 40.69242104699992], [-73.90354057099982, 40.69284878199992], [-73.90386600499991, 40.693035711999904], [-73.90428792799995, 40.6932780679999], [-73.90504295399994, 40.69370400599987], [-73.90579597099993, 40.69412715499987], [-73.90491424299982, 40.69507357099997], [-73.90480435399992, 40.695148168999914], [-73.9042601839999, 40.69570037099989], [-73.90466384799993, 40.69593224699988], [-73.90500484799985, 40.69611874699991], [-73.9054012769999, 40.69635030599995], [-73.90576535999999, 40.696548555999925], [-73.906116261, 40.6967505569999], [-73.90652350699987, 40.69697614399997], [-73.9068458349999, 40.69716354899987], [-73.90727452399994, 40.69739918999991], [-73.90756985199991, 40.69757174999993], [-73.9080224919999, 40.69782540899991], [-73.90877366099991, 40.69825522699988], [-73.90953273199997, 40.69867510499995], [-73.91103489599992, 40.6995321139999], [-73.91180820099989, 40.69993800299986], [-73.91067882699998, 40.701045968999914], [-73.91142242699996, 40.70148658899994], [-73.9121616979999, 40.70192316499989], [-73.91290404099983, 40.70236189199989], [-73.91180710099992, 40.703434952999864], [-73.91254913899994, 40.70387341999991], [-73.91329162399998, 40.70431263899994], [-73.91403097499997, 40.704749112999885], [-73.91477063799996, 40.70518943699992], [-73.91551250499982, 40.705626444999965], [-73.91589738599998, 40.705854816999846], [-73.91625251999979, 40.70606554299993], [-73.91660892400002, 40.70627549499992], [-73.91664948199995, 40.706299387999955], [-73.9169950679999, 40.706502964999885], [-73.91736520300005, 40.7067224689999], [-73.91773662899993, 40.70694273899988], [-73.91808167700003, 40.70714732999993], [-73.918102996, 40.707159968999896], [-73.91847670899989, 40.70738156299989], [-73.91884342599998, 40.70759799799987], [-73.91922162699984, 40.707815133999844], [-73.91956473400002, 40.708023977999915], [-73.91995952899983, 40.708257287999885], [-73.92070065899999, 40.708694629999876], [-73.92143475999988, 40.7091287659999], [-73.92189184699987, 40.709396095999836], [-73.92074519699985, 40.71052968599989], [-73.92122073500002, 40.71079055899986], [-73.92154601699998, 40.71104328399984], [-73.92168663599989, 40.71189431699987], [-73.92223294699991, 40.712854424999904], [-73.92313936299986, 40.713394727999905], [-73.92404011299993, 40.714008312999916], [-73.92083937599983, 40.713815139999966], [-73.91947977199997, 40.71373199799996], [-73.91633296499982, 40.71353115599987], [-73.91569881899996, 40.71348528899991], [-73.91521146700003, 40.71346123899988], [-73.91466958199995, 40.713437073999906], [-73.91303759100005, 40.71333207799989], [-73.91077807399988, 40.71318759999986], [-73.91069895799997, 40.713268520999875], [-73.91015817899998, 40.713821599999946], [-73.91159537799996, 40.71464113299989], [-73.91173862299988, 40.714723810999885], [-73.91274485099986, 40.71530376799991], [-73.91279689999989, 40.71534014299995], [-73.91283037699992, 40.71536090799986], [-73.91245827599991, 40.7155322899999], [-73.91232805399994, 40.71561530699993], [-73.91221070499992, 40.7157093569999], [-73.912108516, 40.71581262199985], [-73.91178934199993, 40.716122478999914], [-73.91156249200004, 40.71636931299988], [-73.9115182479999, 40.71641746599994], [-73.9113427079999, 40.716597028999935], [-73.91105946199994, 40.71684218099989], [-73.91048364399985, 40.71732796499988], [-73.91002374299995, 40.717588123999896], [-73.90980535299991, 40.717713699999926], [-73.90965980099996, 40.71779315499991], [-73.90962006699982, 40.71775153799996], [-73.90955515599987, 40.71769519699992], [-73.90943398699983, 40.717581772999885], [-73.90862256599989, 40.71748082299995], [-73.90850501099997, 40.71746899599985], [-73.90768387299997, 40.71737969499987]]]}}, {\"id\": \"198\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 199, \"Shape_Leng\": 0.0778085005169, \"Shape_Area\": 0.000288747549303, \"zone\": \"Rikers Island\", \"LocationID\": 199, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88885148496314, 40.79870632895875], [-73.88860021869878, 40.798650985918684], [-73.88858562507336, 40.798706072297065], [-73.88821348851273, 40.79866530463852], [-73.8882523074438, 40.79857718039829], [-73.88835861919844, 40.79859565119033], [-73.88839250519672, 40.798566297269915], [-73.88823353313644, 40.79824288428591], [-73.88794420603693, 40.798343336683786], [-73.88766940705628, 40.79838105015445], [-73.88755967580657, 40.79836873243706], [-73.88745451160935, 40.79835365457647], [-73.88744413439005, 40.7983519333435], [-73.88734380764394, 40.79833532909415], [-73.88728568013728, 40.798324358684575], [-73.88723161846639, 40.798312265785], [-73.88718674168426, 40.798299650460244], [-73.88713264426238, 40.79828110190251], [-73.88707655961188, 40.79825992072696], [-73.8870335290872, 40.79824148180351], [-73.88699057468439, 40.79822080915984], [-73.88693756858686, 40.798193823176966], [-73.8868262842498, 40.798134208744465], [-73.88677992738062, 40.79810681476005], [-73.88673071139102, 40.79807487986166], [-73.88672123646727, 40.79807189078293], [-73.88670122347439, 40.79806432478119], [-73.88668224426033, 40.798055399991554], [-73.88665108524579, 40.79803819669988], [-73.88660440944788, 40.798010334600754], [-73.88653645218785, 40.79796863725461], [-73.88648571061235, 40.79793482714909], [-73.88646193517582, 40.79791690200779], [-73.8864476279971, 40.797906117730015], [-73.88639344440251, 40.7978612554141], [-73.88631837125503, 40.79779607530592], [-73.88628680056247, 40.797770028532206], [-73.88618605645821, 40.79769315237726], [-73.88609112726316, 40.79761856993663], [-73.88598921889239, 40.797543566056994], [-73.88586675456301, 40.79745397107574], [-73.8857087958639, 40.79733929778954], [-73.88557218565714, 40.797241097444775], [-73.8854196281804, 40.79713548837185], [-73.8853600288644, 40.79709249327738], [-73.88531358206146, 40.79705652568051], [-73.88526897378183, 40.79701915569956], [-73.8852237547494, 40.79697882190967], [-73.88511038267097, 40.796874107822205], [-73.88500486751619, 40.796775352620216], [-73.8848710637576, 40.79664615907997], [-73.88479288320566, 40.79657631045863], [-73.88470366866711, 40.79649503207026], [-73.88465812670354, 40.796457129525415], [-73.8845759214609, 40.79639244454064], [-73.88450450963163, 40.796335570279545], [-73.88446288938577, 40.79630395695948], [-73.88436938249504, 40.79623858587312], [-73.88428576654385, 40.79618176992692], [-73.88424498708406, 40.79615101275439], [-73.88422937027987, 40.79613685074921], [-73.88421619472517, 40.79612287999498], [-73.88415665725522, 40.79605151864877], [-73.88408036837752, 40.79595839282077], [-73.88403220776783, 40.795900316364765], [-73.88400985604741, 40.79587539478737], [-73.88398001309021, 40.795845486149574], [-73.88395093197968, 40.79581988366578], [-73.88391718206042, 40.795795021959435], [-73.88388038316793, 40.79577162552899], [-73.88384160297444, 40.79575001944093], [-73.88375741590156, 40.79570856541983], [-73.88371602364624, 40.795689839259396], [-73.88368606502722, 40.79567830910385], [-73.88365916952804, 40.79567073562462], [-73.8836472632472, 40.79566874245467], [-73.88363602032433, 40.795667848415356], [-73.88363491877148, 40.79566786539586], [-73.88363371018107, 40.795667954304264], [-73.88363113716696, 40.79566837537766], [-73.88362866045301, 40.795669066263606], [-73.88362651450376, 40.7956699467867], [-73.88362566059602, 40.79567041415722], [-73.88362493671484, 40.79567089924916], [-73.88362434350388, 40.795671412449714], [-73.88362389181384, 40.79567192529058], [-73.8836235824944, 40.79567245636789], [-73.88362347511048, 40.79567272630989], [-73.8836234160024, 40.795672996300745], [-73.88362339107707, 40.79567325761467], [-73.88362339060498, 40.79567352766474], [-73.8836234384092, 40.79567379726083], [-73.88362352038044, 40.79567406739406], [-73.88363082548062, 40.79569131997075], [-73.88364118305184, 40.7957106730427], [-73.88366120810024, 40.79574454237241], [-73.88368570715002, 40.79578851244839], [-73.8836917781387, 40.795799405863015], [-73.88372136446775, 40.79585411288936], [-73.88372923494416, 40.79587318468204], [-73.88373136425903, 40.795882336213054], [-73.8837322847909, 40.7958911801211], [-73.88373179288648, 40.79590815460777], [-73.88372908649971, 40.795924225575405], [-73.88372552582665, 40.79593466784329], [-73.88372319549673, 40.79593887975372], [-73.88372041619786, 40.7959425506077], [-73.88371433895306, 40.7959484790532], [-73.88370684004401, 40.795954063140215], [-73.88369799188168, 40.795959241124685], [-73.8836879120701, 40.79596394963297], [-73.88368124897234, 40.79596630233258], [-73.88367448015707, 40.795967844606196], [-73.88366792669436, 40.79596851261583], [-73.88366478678273, 40.79596850944679], [-73.88366175261613, 40.79596829128218], [-73.88365516756649, 40.79596723006511], [-73.88364828586806, 40.79596550380874], [-73.88363294995125, 40.79595985998664], [-73.88362568067883, 40.795956394259576], [-73.88361818816536, 40.79595213675056], [-73.88360044806825, 40.79593998935041], [-73.88357745705342, 40.795920964932016], [-73.8835487025507, 40.79589245377118], [-73.88352132338426, 40.79586230375447], [-73.88349059449249, 40.79582446077591], [-73.88343778229446, 40.79574969303035], [-73.88341030322668, 40.795716860808724], [-73.88338637422893, 40.79569276525554], [-73.8833475098465, 40.795658309237865], [-73.88332622624746, 40.79563943140555], [-73.88326600487356, 40.79558727649096], [-73.88324624360256, 40.79557194803204], [-73.88322052550512, 40.79555484865766], [-73.88320394303746, 40.79554499746232], [-73.88318903725748, 40.79553750855989], [-73.88317473357542, 40.795531856843624], [-73.88316406288516, 40.7955288558722], [-73.88316079453163, 40.795527944070635], [-73.8831435822138, 40.795525044687366], [-73.88309890770998, 40.79552013680324], [-73.8829831164623, 40.79550055976044], [-73.88294565229103, 40.795492362242605], [-73.88289156223459, 40.79547848530561], [-73.8828337519772, 40.795463937644165], [-73.88281685343809, 40.79545799459988], [-73.88280269382362, 40.795451298967265], [-73.88273381151981, 40.79541236241356], [-73.88264736334979, 40.795363837058424], [-73.88256589218054, 40.79532175419615], [-73.88251548178499, 40.79529729944238], [-73.88245647333433, 40.795270099375806], [-73.88241189497742, 40.795250943982204], [-73.88237750527836, 40.79523946391297], [-73.88228246152997, 40.79521398197161], [-73.88218546415864, 40.795187615265064], [-73.88209633687475, 40.79515986104309], [-73.8820730731555, 40.79515530736489], [-73.88202447821953, 40.79514958423101], [-73.88200405621475, 40.79514623125208], [-73.88189640244067, 40.79512278910242], [-73.88179719700753, 40.79509817697692], [-73.88170835287256, 40.795078706359206], [-73.88166054417218, 40.795070831516], [-73.88159381157867, 40.7950628383747], [-73.88153903087833, 40.79505787510881], [-73.88148620045226, 40.79505534394159], [-73.88138137953057, 40.79505729851871], [-73.88136502712474, 40.79505760551507], [-73.88130870043936, 40.79505603460755], [-73.8812896725956, 40.79505425130767], [-73.88127276943145, 40.795051531382505], [-73.88119676142422, 40.79503589266156], [-73.88110369782537, 40.79501712870741], [-73.88106414287465, 40.79501153209566], [-73.88097705479039, 40.79500379702309], [-73.88087685804106, 40.794991311397396], [-73.88084019560638, 40.7949851335379], [-73.88075892952138, 40.79496851476577], [-73.8807225522413, 40.79496202975529], [-73.8806622449003, 40.7949531610383], [-73.88061282089043, 40.794947229556996], [-73.8805007720725, 40.79493616249444], [-73.88038031546304, 40.7949231428035], [-73.88031188472486, 40.794917723358644], [-73.88023542873849, 40.79491445627383], [-73.88009881438347, 40.79491159363486], [-73.87998700129121, 40.79490802055457], [-73.87991508464467, 40.794904315549594], [-73.87991449155209, 40.79490426216128], [-73.8798539645331, 40.79489884900334], [-73.87980493376996, 40.79489225117035], [-73.87975745821161, 40.79488368367066], [-73.87967478298773, 40.79486649659612], [-73.8795627221185, 40.794842390630144], [-73.87946001096141, 40.79481857259242], [-73.87934893739114, 40.79479262132381], [-73.87920446532179, 40.79475827779217], [-73.8790765980677, 40.794728274486744], [-73.87899352766185, 40.794706529656715], [-73.87887252713152, 40.794673606333994], [-73.87874662234786, 40.794639444237234], [-73.87864663726008, 40.794608839552254], [-73.87852685920608, 40.79457407923452], [-73.87847233873828, 40.79455671457331], [-73.87841281195189, 40.79453577816251], [-73.87837437812739, 40.794520806767316], [-73.87834370276049, 40.79450663694727], [-73.87824960606055, 40.79445685734027], [-73.87820787186249, 40.79443256180638], [-73.87818989044003, 40.794423195906724], [-73.8781774587458, 40.79441798697566], [-73.87816302352833, 40.794413145988514], [-73.87811831475943, 40.79440155322258], [-73.878041848809, 40.79438437271278], [-73.87801071682776, 40.79437429898058], [-73.87797523216784, 40.794359790094425], [-73.87793243517783, 40.7943397169792], [-73.87789422394009, 40.79431924480303], [-73.87778072525026, 40.79425333319388], [-73.8776219539544, 40.794160476244315], [-73.87753980202646, 40.79411136643943], [-73.87744715599825, 40.79405969551736], [-73.87733499061781, 40.793997297718484], [-73.87726571206598, 40.7939634917726], [-73.87715743688905, 40.79390651051256], [-73.87705474512775, 40.79385314443542], [-73.87695902423698, 40.79380537015927], [-73.87690341417037, 40.7937754584675], [-73.8768533497296, 40.79374495929963], [-73.87683239271796, 40.793730195529655], [-73.87678543251936, 40.79369322480133], [-73.87676325835338, 40.79367725370378], [-73.87667149377093, 40.79361710079402], [-73.87657771056746, 40.793553191409615], [-73.87653382908425, 40.79352398548307], [-73.8764934794008, 40.793500700983486], [-73.87640255055362, 40.79345691060194], [-73.87636541846646, 40.79343652853076], [-73.87633147326575, 40.79341632158013], [-73.87631755762334, 40.79340643656754], [-73.87628751906915, 40.79338163204678], [-73.87627936699432, 40.793375904489544], [-73.87627147160718, 40.79337118704908], [-73.87624509705624, 40.793358479757956], [-73.87620619071437, 40.79334278693764], [-73.87614385924937, 40.793321242408474], [-73.87612202605544, 40.79331278220923], [-73.87609618491513, 40.79329952547196], [-73.87606235014438, 40.79327750811568], [-73.87604786716726, 40.793266686496935], [-73.87601532254274, 40.79323949180808], [-73.87599744174818, 40.79322720855622], [-73.87596602598026, 40.79321042474757], [-73.87591748808131, 40.793188031513274], [-73.87586407715739, 40.79316731596839], [-73.87584483848269, 40.79315850721027], [-73.87583075485645, 40.79315011728718], [-73.87581681561578, 40.79313983599114], [-73.87580272711051, 40.793127448573365], [-73.87578703333045, 40.79311169937122], [-73.87575830870135, 40.793075522506925], [-73.87574836430458, 40.79306453503411], [-73.8757250017987, 40.79304350168518], [-73.87569604980123, 40.793021110207654], [-73.87561279450044, 40.792967801153914], [-73.87557896865843, 40.792941181914216], [-73.87555275917276, 40.79291615567656], [-73.87553042327113, 40.79289078231575], [-73.87551232737925, 40.79286725898751], [-73.87547123144924, 40.79280636776735], [-73.87544969810897, 40.79277715038665], [-73.87543203665531, 40.79275543862973], [-73.87538576406376, 40.792706834279535], [-73.87530659534345, 40.792628989825324], [-73.87522354103142, 40.792550726651875], [-73.8751279512564, 40.792465183933835], [-73.87507395872458, 40.79242077547183], [-73.87498356453385, 40.79235155521852], [-73.87491771753012, 40.79230347842358], [-73.87485267905681, 40.792260723906786], [-73.87481081338947, 40.79223740047804], [-73.87471064054327, 40.79218973574527], [-73.87466759149237, 40.79216665405675], [-73.87463269006614, 40.79214496832345], [-73.87454446880673, 40.7920870781407], [-73.87449318523954, 40.79205745058919], [-73.87440998103716, 40.7920153600434], [-73.87429326006679, 40.79195306184479], [-73.8742000330937, 40.79190437831606], [-73.87417526889703, 40.79189245565297], [-73.87415314054883, 40.79188356220571], [-73.87413213350335, 40.7918771181924], [-73.87411193956802, 40.79187302627992], [-73.87407064434674, 40.79186803733165], [-73.87401755922818, 40.791863783147164], [-73.87392742618482, 40.79185865900265], [-73.8738851008293, 40.79185338122018], [-73.87386851936819, 40.79184991406616], [-73.87361978246376, 40.7917728187123], [-73.87351081387887, 40.79172536073438], [-73.87343783594675, 40.79168303863182], [-73.87337134112268, 40.791646207041694], [-73.87335904106946, 40.79164097963956], [-73.87334448778849, 40.79163647882992], [-73.87329524220436, 40.79162598804911], [-73.87327840110588, 40.79162162895527], [-73.87320017614613, 40.79159741895402], [-73.87314608947096, 40.791578358725786], [-73.87309734778617, 40.791558187298946], [-73.8730353749119, 40.7915296270255], [-73.87297284856062, 40.79149959004363], [-73.87292724191762, 40.791476261133425], [-73.87288457445655, 40.79145210820714], [-73.87284269361069, 40.791425586823344], [-73.87273896798422, 40.79135358558459], [-73.87264706604645, 40.791294140769466], [-73.87256990394003, 40.7912478072913], [-73.87246121603856, 40.79118066172868], [-73.87235602503631, 40.791119887235126], [-73.8722629891168, 40.79106568272001], [-73.87222386671776, 40.79104017317083], [-73.87221878971064, 40.7910368617664], [-73.87216399871963, 40.7909957211357], [-73.87208390642414, 40.790932830573716], [-73.87197367245845, 40.79084501775286], [-73.8718453860334, 40.790736977019165], [-73.87175020506085, 40.79065850952962], [-73.87171094798327, 40.79062407591866], [-73.87168210639884, 40.790594704777234], [-73.87160773568894, 40.790507993800524], [-73.87153422429404, 40.79042463390263], [-73.87143773747367, 40.79032368749403], [-73.87136151517977, 40.790245430797164], [-73.87134366346872, 40.790224888522246], [-73.8713252263043, 40.79020128356386], [-73.87126782455506, 40.79011992145684], [-73.87123092167869, 40.790068732251946], [-73.87121154143483, 40.7900361217371], [-73.87120394439336, 40.79001908547554], [-73.87119724046612, 40.789999896673955], [-73.87118336516131, 40.78994531968979], [-73.87115500629291, 40.78981952374308], [-73.87114607484399, 40.78978815788253], [-73.87113409505912, 40.78974607319266], [-73.87110291637639, 40.78966364252112], [-73.87107215941816, 40.78956494041246], [-73.87105158005147, 40.78950368287369], [-73.87102842677966, 40.789431688937135], [-73.87101839580241, 40.78939257797725], [-73.87101587054431, 40.789375259066304], [-73.87101491849326, 40.78935891423255], [-73.87101411499657, 40.78927214896465], [-73.87101378121669, 40.789230545525015], [-73.87101364700264, 40.78921403910673], [-73.87101127918723, 40.789194414888], [-73.87100344385925, 40.789165859806744], [-73.87098474146693, 40.789108152873915], [-73.87098333940936, 40.78910382881657], [-73.870972637136, 40.789057134951975], [-73.87096530682801, 40.789005188329035], [-73.8709644492219, 40.78896249015653], [-73.87096821802318, 40.78891988308357], [-73.87097658644602, 40.78887765611325], [-73.87096652040277, 40.78874201301874], [-73.87094904721623, 40.78860680496223], [-73.87092419700147, 40.78847226522805], [-73.87089201274493, 40.78833862702214], [-73.87085254976718, 40.788206120771726], [-73.87080132248498, 40.78806324195944], [-73.87079045056352, 40.78803026893092], [-73.8707857397605, 40.787996465122724], [-73.87078728267711, 40.78796249289199], [-73.87079504870778, 40.78792901624749], [-73.87080888583708, 40.78789669184152], [-73.87107316722266, 40.78757484012508], [-73.87122538745567, 40.78744541709071], [-73.87137398779542, 40.787270914413675], [-73.87148752123443, 40.78705256623682], [-73.8716306538668, 40.78679790542149], [-73.871875621225, 40.78662350897697], [-73.87210572314568, 40.78646881706841], [-73.8722480610516, 40.786428740218575], [-73.87226518948736, 40.78642391838861], [-73.87247298656574, 40.78630300714108], [-73.87268454313948, 40.78615392699541], [-73.87278847084744, 40.78607797617648], [-73.8728719590193, 40.785975027902346], [-73.8728849699182, 40.785962633107935], [-73.87298121191189, 40.785894452232135], [-73.87308761694455, 40.78585495239161], [-73.87320234357092, 40.785824969447575], [-73.87328576323272, 40.78581238408133], [-73.87335461041052, 40.785788689164015], [-73.87342763293343, 40.78576182947974], [-73.87351452479308, 40.78575009134951], [-73.87368934684048, 40.78575523993384], [-73.87386380700414, 40.78576521513464], [-73.87403767496974, 40.78578000485868], [-73.87421072081537, 40.785799588830336], [-73.87435603384517, 40.78580981338755], [-73.87459603198725, 40.785836219299306], [-73.87483701214678, 40.78585682040508], [-73.87507873184192, 40.78587159631924], [-73.8753209462172, 40.78588053209308], [-73.87556341216793, 40.78588361822196], [-73.87580588496253, 40.7858808521502], [-73.87604811986586, 40.785872236272205], [-73.87628987276307, 40.78585777993131], [-73.87653089978205, 40.785837496915406], [-73.87677095791717, 40.78581140794937], [-73.87697326901583, 40.78577771407224], [-73.877173407119, 40.78573721118406], [-73.87737097598894, 40.785689979903516], [-73.87756558393352, 40.785636113014206], [-73.87775684519505, 40.785575717871666], [-73.87794438183145, 40.785508913798175], [-73.87812782109263, 40.785435832963415], [-73.8783068007881, 40.785356620754236], [-73.88522186791747, 40.78663048709767], [-73.88537442205505, 40.78666912689923], [-73.885770293862, 40.786677867067525], [-73.88596200190945, 40.786752519242356], [-73.88612955519179, 40.78670799337734], [-73.88631726186046, 40.78673287031077], [-73.88659282785129, 40.78673817520633], [-73.88692199760122, 40.78673311412677], [-73.8871531644244, 40.786826182014984], [-73.8872778340932, 40.786806673375665], [-73.88770126353202, 40.78687227428625], [-73.88830725683066, 40.78700634439277], [-73.88856541983498, 40.78716095016911], [-73.88905212421602, 40.78737256012899], [-73.88940460886543, 40.787791280288864], [-73.88952254814944, 40.78805882484885], [-73.88961097071468, 40.78819999100402], [-73.88979724043043, 40.78865239234611], [-73.88987049722098, 40.788895497992435], [-73.88988482198518, 40.78900996596757], [-73.89012684985323, 40.78963627949646], [-73.89030135549454, 40.78987672716731], [-73.8905697158751, 40.79010763610641], [-73.89079999904915, 40.79020757609917], [-73.89091952981933, 40.79022555413888], [-73.89115649549046, 40.790173855852174], [-73.89140452851856, 40.79019391914968], [-73.89160587107678, 40.79016070047654], [-73.89177016601688, 40.79022204338593], [-73.89197448913517, 40.790236274120744], [-73.89212803173004, 40.7903154821244], [-73.89218214128833, 40.790394508060714], [-73.89282283610926, 40.79281708195147], [-73.89261805166494, 40.79384099530879], [-73.89179289686568, 40.79677524575864], [-73.89146412571839, 40.79706970970708], [-73.89138566506293, 40.79713592160785], [-73.89131158868767, 40.797232471180436], [-73.89126861053201, 40.79729759336434], [-73.89073139028322, 40.797660899806495], [-73.89074667860793, 40.797674655538785], [-73.89072763263877, 40.79768290300246], [-73.89080903156564, 40.7977862944989], [-73.89082535336988, 40.79778217694919], [-73.89087421678985, 40.79782974822637], [-73.89080346891845, 40.79786274095017], [-73.89075732720465, 40.797813106639126], [-73.89078725984355, 40.79779867072494], [-73.89071401711358, 40.79769528744903], [-73.89068952645523, 40.797707661749016], [-73.89067322907707, 40.79769731502421], [-73.89062148266537, 40.797748923448545], [-73.89067572679147, 40.79783162639602], [-73.89068932626901, 40.797829573039735], [-73.89083042169466, 40.79800740607706], [-73.89079505126988, 40.79802183785564], [-73.89065123202073, 40.79784606676986], [-73.89065939632937, 40.797841941504636], [-73.89060785824246, 40.79776750713776], [-73.89016916368543, 40.79796465893519], [-73.89011894402582, 40.79788745725576], [-73.89008344066364, 40.79790090498261], [-73.89004647710622, 40.797903116574965], [-73.89000362263465, 40.797891840957554], [-73.8897656404151, 40.797869142979295], [-73.88961192353301, 40.7978476501687], [-73.88936510050216, 40.79780809138587], [-73.88921283898088, 40.79780120410412], [-73.88914925293622, 40.79781013085769], [-73.88906339518935, 40.797833878023496], [-73.88886162193924, 40.797873057398476], [-73.88871105363494, 40.797938248730176], [-73.88856224600559, 40.79802249938153], [-73.88846004385208, 40.79808501552154], [-73.88835248098191, 40.79813935970092], [-73.88831097224373, 40.798184185370175], [-73.88851389271126, 40.79824315578625], [-73.88846064025032, 40.79829085878359], [-73.88856204587574, 40.79835340382445], [-73.88850873250031, 40.79843783999455], [-73.88839282545187, 40.7983752808678], [-73.88850364697177, 40.79858844391536], [-73.88860998976942, 40.7985885467283], [-73.88859543371058, 40.79862159405709], [-73.88899654222763, 40.79868075499042], [-73.88897708980068, 40.79875053085948], [-73.88883691629131, 40.79874672177404], [-73.88885148496314, 40.79870632895875]]]}}, {\"id\": \"199\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 200, \"Shape_Leng\": 0.112661735435, \"Shape_Area\": 0.000744643168558, \"zone\": \"Riverdale/North Riverdale/Fieldston\", \"LocationID\": 200, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.896411335, 40.90450452099991], [-73.89635785999998, 40.9032001319999], [-73.89640895699986, 40.9029611079999], [-73.89661002199993, 40.90220351299991], [-73.89687844599996, 40.90104057999988], [-73.89697054199992, 40.90061116799993], [-73.89702895099995, 40.90031520399991], [-73.8970336369998, 40.900278620999885], [-73.89704651399992, 40.90022470299987], [-73.89707329199993, 40.90008389099992], [-73.89713947299998, 40.8996194349999], [-73.89715717199994, 40.89937182699985], [-73.89709149099991, 40.89859331799993], [-73.896994929, 40.897878723999916], [-73.89670747099987, 40.89580502899987], [-73.89643170799987, 40.89365527899989], [-73.89675790099987, 40.89264544399993], [-73.89705763499997, 40.89214045799989], [-73.89728112499986, 40.89171068399993], [-73.89731936199985, 40.891615860999885], [-73.89769601099988, 40.890681823999934], [-73.89776669099993, 40.890509389999906], [-73.89787887499983, 40.89027610099993], [-73.89801518, 40.89031454599989], [-73.89820200999998, 40.890415934999965], [-73.89825201999993, 40.890434969999944], [-73.89828904299992, 40.890442980999914], [-73.89834896199993, 40.890444996999896], [-73.89842801199981, 40.89043097399988], [-73.89853911899992, 40.89038907399991], [-73.89959425899988, 40.88981699899988], [-73.89987570700002, 40.88954945799986], [-73.90006286299982, 40.889458615999885], [-73.90024058900005, 40.88935606899991], [-73.90040701199986, 40.88924269799986], [-73.90056048999982, 40.88911965299994], [-73.9006996359998, 40.88898822099989], [-73.90079375199993, 40.88894284499989], [-73.90090301399997, 40.888910548999895], [-73.90102228799991, 40.888895126999884], [-73.9011444999999, 40.888898679999926], [-73.90126187899993, 40.88892096799987], [-73.90136756999983, 40.888959499999935], [-73.90145684499987, 40.88901019299989], [-73.9015597889999, 40.8890681819999], [-73.90167746699993, 40.88911356799987], [-73.90180553799988, 40.889143419999876], [-73.90193848899985, 40.88915603999993], [-73.90207033599997, 40.88915127199993], [-73.90219556299989, 40.8891304149999], [-73.90232738499991, 40.889111381999946], [-73.90246297399985, 40.88910514799994], [-73.90259906499982, 40.88911217899994], [-73.90273228799997, 40.88913231799993], [-73.9028594909999, 40.88916474399989], [-73.90300926099984, 40.888815111999875], [-73.9031806419999, 40.88847059499993], [-73.903373145, 40.88813234499985], [-73.90358612699993, 40.8878014589999], [-73.90381882699994, 40.88747900899994], [-73.90413642399992, 40.887080708999896], [-73.90533517099998, 40.88735459899987], [-73.90626252500002, 40.88757033799994], [-73.90738830199982, 40.88781969199987], [-73.90759731899983, 40.888200979999965], [-73.9079634, 40.88885975299993], [-73.90831585200002, 40.88911589699994], [-73.90837360299984, 40.889116015999896], [-73.90843131799984, 40.889124993999914], [-73.90848550899992, 40.88914254199987], [-73.90853308099986, 40.889167388999915], [-73.90855714199998, 40.88928966699994], [-73.90877717899996, 40.88937017299986], [-73.90881651000004, 40.889390590999895], [-73.90887023199998, 40.889421753999876], [-73.90900536, 40.88950014099986], [-73.90929422499988, 40.88928229699996], [-73.90975930299997, 40.889250027999886], [-73.90981767099994, 40.888996568999886], [-73.91075165299989, 40.88849245499989], [-73.91115061799995, 40.88827710599987], [-73.91130760799977, 40.888236029999874], [-73.91173089099983, 40.88821871299993], [-73.91183800999981, 40.887935438999904], [-73.91196427499982, 40.887656339999914], [-73.91210920699979, 40.88738250499992], [-73.91227221199996, 40.88711502299987], [-73.91243612599989, 40.88686844899993], [-73.91261653600003, 40.88662819299988], [-73.91281277499982, 40.88639518699987], [-73.91302404599989, 40.88617032399992], [-73.91347365499988, 40.885771970999926], [-73.91386874899985, 40.88543763799994], [-73.91393522099992, 40.8853813979999], [-73.91440838499985, 40.8849988979999], [-73.91489276999991, 40.8846247709999], [-73.91586527699988, 40.88485810699987], [-73.91630438600004, 40.884909510999954], [-73.91876151499997, 40.88556968399996], [-73.9187098259998, 40.88578600799991], [-73.91867419999998, 40.8860048999999], [-73.91865495699979, 40.88622532599991], [-73.91865219599985, 40.88644622199995], [-73.91866590399985, 40.8866665349999], [-73.91869590499994, 40.88688520399987], [-73.920398527, 40.88712930499984], [-73.92058641888602, 40.88718267830985], [-73.9203928019909, 40.88764850022696], [-73.91978510749304, 40.88928644430759], [-73.91915943838116, 40.89089435397414], [-73.91888014391608, 40.89160572195276], [-73.91853688876205, 40.892545682915404], [-73.91816405458411, 40.89356661453002], [-73.91800263585222, 40.89400861634836], [-73.91782218257843, 40.894422288084016], [-73.91781851992614, 40.89454242132222], [-73.91786770394255, 40.89460801163131], [-73.9179087010731, 40.89467676640003], [-73.91763488207482, 40.89494190535218], [-73.91694161078398, 40.896644551702416], [-73.91685138033517, 40.896930782857055], [-73.91670849171784, 40.89738405527042], [-73.916720710036, 40.89740870362791], [-73.91672732153596, 40.89743455567752], [-73.91672813983982, 40.89746088267584], [-73.91672314187362, 40.897486942489934], [-73.91667343212758, 40.897527053119], [-73.9151845587166, 40.901584282862146], [-73.91523397419424, 40.90169679724567], [-73.91543172958416, 40.90181268368833], [-73.91501449586046, 40.90308129910641], [-73.9145903924332, 40.90411900219645], [-73.91429080087573, 40.904093207462616], [-73.91418416752522, 40.90413933380093], [-73.91386708918738, 40.90495583544221], [-73.91286540202547, 40.907535122202916], [-73.91287309701401, 40.90756169578182], [-73.91288586402978, 40.907587120537485], [-73.91290342493161, 40.90761084255639], [-73.91292539713486, 40.90763234502045], [-73.91295130194734, 40.907651159467385], [-73.91299834908081, 40.90767597498969], [-73.91304024119393, 40.907705634872606], [-73.91307611900355, 40.90773953074303], [-73.91310524658566, 40.90777696733996], [-73.91311160642807, 40.9077899602662], [-73.91311517923928, 40.90780355253811], [-73.91311587518278, 40.90781740239073], [-73.9131136767581, 40.907831161582145], [-73.91310863924069, 40.90784448415], [-73.91310088929278, 40.90785703510997], [-73.91309062177854, 40.90786849887935], [-73.91307809486497, 40.907878587211094], [-73.91299604135064, 40.907916930054235], [-73.91290961086187, 40.90794926755081], [-73.91281956481328, 40.907975314819865], [-73.91272669647384, 40.90799484239435], [-73.91264932172878, 40.90807081195536], [-73.91234051167459, 40.90905819476917], [-73.91235918448416, 40.90913673357688], [-73.9123221765585, 40.909188844939514], [-73.91228107016744, 40.9091933610948], [-73.91226868851436, 40.90923014850241], [-73.9122758401167, 40.90926351655544], [-73.91181952393785, 40.91114833486351], [-73.91177495821104, 40.91125218736267], [-73.91157895749342, 40.912135959319954], [-73.91150475394844, 40.91216987987947], [-73.91149190703744, 40.912233769312], [-73.9115361890538, 40.91230206277718], [-73.91144882473377, 40.912754682588634], [-73.91143213027841, 40.91282382621967], [-73.91147332099729, 40.9129109035367], [-73.91150045783573, 40.91296826932599], [-73.91155029028242, 40.913069812694616], [-73.911654724022, 40.913417851896696], [-73.91160330729478, 40.91352414535434], [-73.91154690203327, 40.91362897266789], [-73.91148558068319, 40.91373219915614], [-73.91141942200743, 40.913833692194345], [-73.91136627234506, 40.91387072355504], [-73.91131788006946, 40.91391133540743], [-73.91127466250195, 40.913955177533495], [-73.91123699234173, 40.91400187185856], [-73.91120519445255, 40.91405101571086], [-73.91121589433219, 40.91409544274206], [-73.91122114380083, 40.9141404292674], [-73.91122089804242, 40.91418559131997], [-73.91118065842387, 40.91425918163407], [-73.91113412832105, 40.91433060593092], [-73.91108150903531, 40.91439955517914], [-73.91102302821666, 40.91446573105548], [-73.91095893887852, 40.914528847235125], [-73.91088951830272, 40.91458863063123], [-73.9108342742287, 40.91462338429265], [-73.91078405014537, 40.91466227514883], [-73.9107393818088, 40.91470488834581], [-73.91070074571455, 40.91475076932193], [-73.91066855401304, 40.91479942865664], [-73.9103325682445, 40.91553277700258], [-73.91011276612555, 40.91539873407221], [-73.90851155008805, 40.91500226401317], [-73.90245663952454, 40.912958933212956], [-73.90066409305153, 40.912463877835], [-73.89991648381654, 40.912370689980314], [-73.89810489644351, 40.91179060000506], [-73.89725326377472, 40.911655519969216], [-73.89663333881637, 40.911417374923055], [-73.8966306859999, 40.91129892699992], [-73.89663068599988, 40.91129887799991], [-73.89660613699981, 40.91020224899991], [-73.89659583799997, 40.909627658999916], [-73.89656464500003, 40.9082858239999], [-73.89654653299982, 40.90750665599994], [-73.89649229399996, 40.90633965199994], [-73.89643503499988, 40.905092998999855], [-73.896411335, 40.90450452099991]]]}}, {\"id\": \"200\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 201, \"Shape_Leng\": 0.130404117686, \"Shape_Area\": 0.000619350747819, \"zone\": \"Rockaway Park\", \"LocationID\": 201, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.81849901800001, 40.5873623529999], [-73.81829590099994, 40.58735573699989], [-73.81812861099992, 40.5873700519999], [-73.81800188699995, 40.58740172099992], [-73.81788322299984, 40.587448468999916], [-73.81771610099989, 40.587150957999924], [-73.81739092999989, 40.58656943199988], [-73.81733678199987, 40.58648319299993], [-73.81693289499991, 40.58582331099997], [-73.81637844599986, 40.58491100499989], [-73.81576138799994, 40.58395921499993], [-73.81570076199996, 40.58382964799994], [-73.8156731639999, 40.58377646399992], [-73.81563939099986, 40.58372220799987], [-73.81550882099985, 40.583475793999924], [-73.81544517899997, 40.58337349199993], [-73.81567618499987, 40.58330770499987], [-73.81670273099988, 40.5830153519999], [-73.81751059199993, 40.58278527099993], [-73.81730795916116, 40.58233475897], [-73.8176010354653, 40.582260543915815], [-73.81820861021012, 40.58211739081629], [-73.81838189820859, 40.58207656004311], [-73.81886971738469, 40.581932715311716], [-73.81895632282897, 40.58190067592644], [-73.81917967481245, 40.58180414823225], [-73.81928812015272, 40.581766974177114], [-73.81940325144654, 40.58173798797363], [-73.81964430603263, 40.58168608220069], [-73.82036608810552, 40.581498083891034], [-73.82048260084306, 40.58146454207295], [-73.82050198699154, 40.58145867602859], [-73.82134165094914, 40.58126212722807], [-73.82194916338798, 40.5811711179728], [-73.82214136703563, 40.58113557552871], [-73.82222751244, 40.581115032161954], [-73.82228991494195, 40.581096073193926], [-73.82242185945248, 40.581041805400375], [-73.82264897973464, 40.58093751426907], [-73.82320233589388, 40.58074798778896], [-73.82321130220612, 40.58074496142334], [-73.82372487494459, 40.58057131417577], [-73.82397556699816, 40.58048076353873], [-73.82419170047795, 40.580425253616895], [-73.82436118230778, 40.58037836962266], [-73.8245996868574, 40.58030150058319], [-73.82473701900733, 40.5802572377077], [-73.82495905834286, 40.580138139202575], [-73.82516976808388, 40.58006973278708], [-73.82562469406477, 40.57996870979248], [-73.82622485462862, 40.579787027917014], [-73.82670299479278, 40.579659794905], [-73.82687139012124, 40.57959730905056], [-73.82717378884985, 40.57944451295523], [-73.82755616404528, 40.57933987898739], [-73.82798819062813, 40.57924391773673], [-73.82837544428159, 40.579139550197], [-73.82868597099545, 40.5790558594242], [-73.82878289896757, 40.579038479966385], [-73.82911066345382, 40.5789260059822], [-73.8292744991744, 40.57886524292074], [-73.82946175273133, 40.57880456046041], [-73.82956637549206, 40.578780356045904], [-73.82961974131516, 40.57877358256277], [-73.82967941933, 40.578769546132996], [-73.82973373970096, 40.578768249890736], [-73.82990284920685, 40.57877585645819], [-73.83002699980847, 40.57876835913071], [-73.8301726281433, 40.578748672456854], [-73.83032282201474, 40.578703715949544], [-73.83048178791245, 40.57865015244701], [-73.83066083402844, 40.57856791075439], [-73.83090077277144, 40.57849590721292], [-73.83106502235128, 40.57847312154713], [-73.83124796291203, 40.578461665667504], [-73.83146093331435, 40.57841943762274], [-73.83162601375413, 40.57836656653714], [-73.83180355871933, 40.57827893511694], [-73.83203538969488, 40.57820887106939], [-73.83219724462421, 40.57815361250897], [-73.8322688248142, 40.57812917409044], [-73.83269598986004, 40.5780257143778], [-73.83292788645088, 40.57794283360204], [-73.83304274011873, 40.57790372019579], [-73.83338855905824, 40.57778594737806], [-73.8336543297874, 40.577715514812624], [-73.83386923172188, 40.5776498976089], [-73.8340874013796, 40.577552497787394], [-73.83450274193689, 40.57734903736139], [-73.83455087542131, 40.57732545851957], [-73.83465388054621, 40.577284957888324], [-73.83506278580667, 40.57711626452328], [-73.83542129219823, 40.576962041781314], [-73.83575900817624, 40.57681443276685], [-73.83592239041295, 40.57676697468214], [-73.83600181324205, 40.576743908161994], [-73.8362312572482, 40.57665570551991], [-73.83632284878783, 40.576611088002586], [-73.83641045643289, 40.57656841076554], [-73.83703852680397, 40.57628429349713], [-73.83710829476571, 40.57625273333054], [-73.8376569816259, 40.576012375816035], [-73.8377849130498, 40.575923683475416], [-73.83800935914823, 40.575848707928195], [-73.83825877960838, 40.57579457648635], [-73.83843049074804, 40.575700534010586], [-73.8386147788839, 40.575636260900176], [-73.83888275267506, 40.57555196026982], [-73.83961762770561, 40.5753070268271], [-73.84043821431337, 40.575033517951056], [-73.84138297830171, 40.57471860841416], [-73.84206938902219, 40.57448096166989], [-73.84288906671075, 40.574197168991205], [-73.84378359075176, 40.57388745204173], [-73.84451460324345, 40.57361053577377], [-73.84526900267889, 40.57332475203252], [-73.84615755209354, 40.573034265606616], [-73.84622515923755, 40.57300882173353], [-73.84699487325773, 40.57271912813605], [-73.84776924944421, 40.57240533287795], [-73.84850444718795, 40.57214844758093], [-73.84862868351767, 40.57209635461399], [-73.84877296803536, 40.57205630900948], [-73.84883931672952, 40.57201892477135], [-73.84890933304747, 40.57200424812317], [-73.84900064117859, 40.57195839648904], [-73.84915963634387, 40.57187790910384], [-73.84930014476147, 40.5718368950852], [-73.8493710236191, 40.571806108948984], [-73.8494932823788, 40.571762725512706], [-73.84957881218321, 40.571732376262254], [-73.84969529478947, 40.57168265828363], [-73.84978693455582, 40.57165195232748], [-73.84995330636089, 40.571588385290596], [-73.85024821434561, 40.571492207765424], [-73.8503876633553, 40.571451035065664], [-73.85053462925642, 40.5714076426579], [-73.85071291963756, 40.571342585945395], [-73.85085482979719, 40.571301292407654], [-73.85110217430288, 40.57121318098269], [-73.8511958562786, 40.57118706150655], [-73.85128141360782, 40.5711721872626], [-73.85153838483797, 40.57106626704578], [-73.85168071856945, 40.570972138147624], [-73.85176922098792, 40.57092755592738], [-73.8518407838795, 40.570891505698], [-73.8522020239966, 40.57074510300148], [-73.8527416269569, 40.57060530564907], [-73.85312102850051, 40.57050635573066], [-73.85332207852214, 40.57043243577316], [-73.85357525479438, 40.57034136437587], [-73.85381160480503, 40.57033599183698], [-73.85384605365918, 40.570330746420076], [-73.85396556577076, 40.570312548872685], [-73.8540958517921, 40.57025512554546], [-73.85423306127085, 40.57013716925942], [-73.8544210818527, 40.57006354647071], [-73.85471701291569, 40.56993606470247], [-73.85487827913525, 40.569913844513444], [-73.85493371329643, 40.569896876915344], [-73.85499076009883, 40.56986537710774], [-73.8550557024824, 40.56981498367037], [-73.85524079248215, 40.56975142438558], [-73.85551038432557, 40.56965884706633], [-73.85574599043328, 40.569597358678195], [-73.85585072247942, 40.56955516524605], [-73.85596762967016, 40.569523253046185], [-73.85612202437312, 40.56950549051674], [-73.85623100571082, 40.569436358403735], [-73.8564559882803, 40.569344434524716], [-73.85664094497899, 40.56929855877574], [-73.85681285637894, 40.56924595756865], [-73.8568830828207, 40.5692194160495], [-73.85713422195636, 40.56912687559003], [-73.85727709576854, 40.569022962381474], [-73.85734798265783, 40.56899135998706], [-73.8576707951522, 40.568901338701], [-73.85775087478515, 40.56887026511511], [-73.85777929081, 40.56885923813767], [-73.85791711481181, 40.56879210324576], [-73.858271174134, 40.568618144918446], [-73.85852684289735, 40.56852014971852], [-73.85909318999227, 40.56831079825676], [-73.85922718254851, 40.56826204617406], [-73.85942657606537, 40.568205007884444], [-73.8595035986373, 40.568186055816184], [-73.85957813852878, 40.5681677128154], [-73.85968856304649, 40.56813429470619], [-73.85977216149385, 40.56810427358402], [-73.85985829196693, 40.56807184339545], [-73.86004153002389, 40.56799843894272], [-73.86022276711229, 40.56792109732444], [-73.86116232688367, 40.56750423205129], [-73.86202965574294, 40.56710844193087], [-73.86214592196148, 40.567045948811234], [-73.86226388529106, 40.56697404896458], [-73.86239006797445, 40.566889892810075], [-73.86255286029154, 40.56681001230082], [-73.86265927675198, 40.56680029670612], [-73.86283290775316, 40.56681866826843], [-73.86286892288632, 40.56683057177312], [-73.86308767499982, 40.5675977289999], [-73.86316930299984, 40.56773927199992], [-73.86406802399988, 40.569297978999934], [-73.86410310099986, 40.569354608999916], [-73.86454176099993, 40.56924292199994], [-73.86522555399998, 40.57045847199989], [-73.86633836399983, 40.572381674999846], [-73.86662812000002, 40.573057648999914], [-73.86449302999984, 40.5735427759999], [-73.86361132999987, 40.57382305799996], [-73.86340629400003, 40.574114330999855], [-73.86394267899998, 40.5749881079999], [-73.86396642099993, 40.57502678699988], [-73.86341227899989, 40.5753164769999], [-73.86333068899987, 40.57538325199989], [-73.86275061599991, 40.575857984999864], [-73.86278001199997, 40.57589856099986], [-73.86280361999994, 40.57593303499988], [-73.86288301945424, 40.57605976162732], [-73.8606003068899, 40.57727158725376], [-73.85984470698162, 40.57767269432985], [-73.85908862067566, 40.57807404880597], [-73.85833844346155, 40.578468346055026], [-73.85758465057629, 40.578864533874594], [-73.8568386972949, 40.57925659271925], [-73.85607299805861, 40.57966151657458], [-73.85532244619522, 40.58005842202096], [-73.85456020425893, 40.58046149731443], [-73.85381681956373, 40.580853655037664], [-73.85305549585115, 40.581255267855006], [-73.85218976840359, 40.58154553231257], [-73.85134716820436, 40.581827725643166], [-73.8504438514919, 40.582130243581396], [-73.84930667192806, 40.58216616119363], [-73.84861053538168, 40.58218814264622], [-73.84819630556807, 40.58211494985829], [-73.84725841773745, 40.58194922099892], [-73.84624074421872, 40.58176938413859], [-73.84521016865186, 40.58157982626369], [-73.84505670628532, 40.58158655664317], [-73.8441571217811, 40.58162600643133], [-73.8432082793877, 40.58166760860317], [-73.84233979552249, 40.581705680693865], [-73.84225010404165, 40.58171023534407], [-73.84129969123791, 40.58175849046554], [-73.84033325297445, 40.58180755277048], [-73.83935385899039, 40.58185726280555], [-73.83929597982016, 40.581863545478186], [-73.83925941300275, 40.58196846164904], [-73.83921592662652, 40.58207183188689], [-73.83916563218244, 40.58217339109339], [-73.83910865862318, 40.58227287881347], [-73.83904515203258, 40.58237003990318], [-73.83902799504563, 40.58240605117917], [-73.83900671021873, 40.58244075812745], [-73.83897179252091, 40.58248499140797], [-73.83893018678124, 40.58252572328265], [-73.83888249024051, 40.58256236904268], [-73.83882938757823, 40.58259440263505], [-73.83879155856049, 40.58261287528828], [-73.83875192458103, 40.582628998727245], [-73.8386630081304, 40.58266830737766], [-73.83868793405183, 40.58270107700946], [-73.83866495939662, 40.5827112290482], [-73.83860223382548, 40.58271924637285], [-73.83853891608467, 40.5827273424076], [-73.83855306451113, 40.582791714299255], [-73.83848216240924, 40.58280078170419], [-73.83846883217524, 40.5827402026538], [-73.83830919856948, 40.58276060922555], [-73.83829873026785, 40.58271302862347], [-73.83819123533861, 40.58271787503073], [-73.83819230033566, 40.58273150172095], [-73.83749042864683, 40.58277110741209], [-73.83649754841137, 40.582819740382604], [-73.8364974655717, 40.58281974026471], [-73.83649535606483, 40.582827571454274], [-73.83584132334414, 40.58287524942916], [-73.83122690074686, 40.584113882116725], [-73.83122819886285, 40.584151572342854], [-73.83115765903744, 40.584170216785935], [-73.83035863109153, 40.58438148239096], [-73.82995875885679, 40.584492313959174], [-73.82964511265376, 40.58457923636072], [-73.82923330280437, 40.58469222542379], [-73.82922322222436, 40.58467614618565], [-73.82921665703279, 40.58466566280551], [-73.82852471261212, 40.584854220382454], [-73.82850475950465, 40.584929329682865], [-73.82846704297499, 40.58506317216264], [-73.82844883541067, 40.585125263817595], [-73.82836036871306, 40.58542692421065], [-73.82827661199255, 40.585711103981126], [-73.82826267442242, 40.5857520669423], [-73.82824469493104, 40.58579445389308], [-73.82822250440167, 40.58583865492305], [-73.82819566531695, 40.58588552243649], [-73.82817935896809, 40.585910367667346], [-73.82817257831394, 40.58592069829708], [-73.8281347354472, 40.585968868443416], [-73.82808927885533, 40.586013046934106], [-73.82803692404548, 40.586052538366324], [-73.82797849511182, 40.58608672111368], [-73.82791491176476, 40.58611505711239], [-73.8278471748534, 40.58613710033012], [-73.82668838981161, 40.58648979899999], [-73.82524882706767, 40.58690086942337], [-73.82495652892945, 40.586984326713804], [-73.82476427875102, 40.587039215772236], [-73.82477620978837, 40.58706383302067], [-73.82477099301649, 40.58706515072738], [-73.82481333907114, 40.587151322035965], [-73.82480116623435, 40.587154615369926], [-73.82475708090976, 40.58706910398411], [-73.8247544742627, 40.58706910000852], [-73.82474352289884, 40.58704514197362], [-73.823261572582, 40.587468241280696], [-73.82320743426241, 40.58747831481585], [-73.8231555877068, 40.587487970348164], [-73.82310704583925, 40.587490120589194], [-73.82306926486032, 40.58749178998467], [-73.82302002675071, 40.587493966089156], [-73.82298001388828, 40.58749111290447], [-73.82293688631508, 40.58749164972748], [-73.8228937850444, 40.58748201986876], [-73.82286311305268, 40.58747562388299], [-73.82278668904684, 40.58745286624583], [-73.82271074897726, 40.587425806137084], [-73.82264303518465, 40.58740168458186], [-73.82256733467796, 40.58736009808132], [-73.82246212330784, 40.58730228474667], [-73.82199421976468, 40.587053330406484], [-73.82145971991768, 40.58676992492123], [-73.8211617779835, 40.58692469023413], [-73.82039765036815, 40.58732160621366], [-73.82003299074137, 40.58751102017107], [-73.81968658634672, 40.587693686886944], [-73.81934686541226, 40.58787282886022], [-73.81929743026505, 40.587894965897796], [-73.81923430777522, 40.5879332933974], [-73.8191882946688, 40.58797371677861], [-73.81912916381579, 40.588038892972584], [-73.81912652299997, 40.58799676499997], [-73.81912265299988, 40.58793502599987], [-73.81909407399996, 40.58783196099988], [-73.81904410199994, 40.587733741999934], [-73.81897507799992, 40.58764405899989], [-73.81893219199988, 40.58759907299989], [-73.81890104000004, 40.58757409799995], [-73.81880351800001, 40.58749592199988], [-73.81849901800001, 40.5873623529999]]]}}, {\"id\": \"201\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 202, \"Shape_Leng\": 0.0699215712618, \"Shape_Area\": 0.000106175396322, \"zone\": \"Roosevelt Island\", \"LocationID\": 202, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94180032729437, 40.76904692662474], [-73.9425680614214, 40.76850590573147], [-73.94285528620689, 40.768303494409295], [-73.94321923520103, 40.76789264385325], [-73.94381427322762, 40.76722091204986], [-73.94422343089231, 40.76666497788984], [-73.9448402665221, 40.765823628390834], [-73.94623787959823, 40.76428501454967], [-73.94662141408685, 40.76386277208089], [-73.94682460228385, 40.763617633234475], [-73.94708876929518, 40.7632989210867], [-73.94788268389203, 40.76236569759203], [-73.9479297966422, 40.76231031625976], [-73.94832459525026, 40.76184623000117], [-73.94833197228185, 40.76183755882172], [-73.94896770106449, 40.761090236621925], [-73.94943175497116, 40.76054861736118], [-73.94971226700028, 40.760328917384776], [-73.95033358814477, 40.7598567555536], [-73.95126005908766, 40.758810116371734], [-73.95148626283033, 40.75866645585446], [-73.9517119041103, 40.75846249420143], [-73.95177822026001, 40.75840254987696], [-73.9520923302263, 40.75799314034483], [-73.95217663423693, 40.757831336440056], [-73.9522114634206, 40.75758363983293], [-73.95254079336881, 40.757095314537246], [-73.95312156546179, 40.75652380482708], [-73.95322009963138, 40.75642684232034], [-73.95326888611, 40.756381044103335], [-73.95330696035853, 40.7563443271275], [-73.95334556462886, 40.75631137566744], [-73.95340583051967, 40.75625853187726], [-73.95409527736291, 40.75565492894537], [-73.95507326928144, 40.75479868255477], [-73.95519503183546, 40.75477452347619], [-73.95579415237704, 40.75416154740866], [-73.9569240430464, 40.75296115989124], [-73.95721003531298, 40.75271535683171], [-73.95791452493181, 40.7521098545253], [-73.95860755756419, 40.75161332398696], [-73.95903975987336, 40.75139248511159], [-73.95902757442488, 40.751297369850946], [-73.95941235107571, 40.750947663845146], [-73.95955875930795, 40.750814597761945], [-73.95990187081871, 40.75055851535353], [-73.9603917057254, 40.750256186117326], [-73.96082903392376, 40.749910550705266], [-73.9610401926698, 40.74967084452351], [-73.96117372960865, 40.74949844307387], [-73.9614247793652, 40.749401291404006], [-73.96158304969282, 40.74941430105318], [-73.9616079947122, 40.74956431675508], [-73.96155080052249, 40.74977388985281], [-73.96132003404372, 40.750074671967425], [-73.96111060170934, 40.750600469450376], [-73.9607094085506, 40.75114849812205], [-73.9602676618637, 40.75158071381144], [-73.96001132397946, 40.751831517167524], [-73.95949352891616, 40.752972403252826], [-73.95901616663733, 40.75359505947865], [-73.95892671110458, 40.75356249218071], [-73.95845836852824, 40.75394923675239], [-73.95768072880023, 40.754821961082754], [-73.9568947698001, 40.75570399516188], [-73.95646348786491, 40.75618798277283], [-73.95598563455, 40.756724225403765], [-73.95565513182049, 40.75709510586291], [-73.95559286286709, 40.75722280008676], [-73.95556724783259, 40.75725766684306], [-73.95553464388514, 40.757296034234145], [-73.9555042551224, 40.75733345603627], [-73.95546455390387, 40.7573773681059], [-73.95544589007469, 40.757399251221365], [-73.9549940600116, 40.75792901932858], [-73.95365747474159, 40.75949610137514], [-73.95332321608949, 40.75974596434368], [-73.9533914071698, 40.7597967621788], [-73.95312451552458, 40.76005275821471], [-73.95297788208019, 40.75996210793835], [-73.95231257072545, 40.7606337551973], [-73.9520863228326, 40.760862154202385], [-73.95187667349164, 40.76124558289901], [-73.95162504986979, 40.76167886380285], [-73.95097191647194, 40.762494507753686], [-73.95093734914026, 40.76257049511886], [-73.95073068791342, 40.76275822521683], [-73.95037548087188, 40.763173845290204], [-73.94997191725122, 40.76364603015847], [-73.94920376011383, 40.7642696439164], [-73.94908628795767, 40.76426402713745], [-73.94881315308189, 40.7646107542343], [-73.94770109006382, 40.76602239716257], [-73.94655273946526, 40.767368205283255], [-73.94591458852791, 40.768213939009335], [-73.94585112316526, 40.76824775011774], [-73.94584451411889, 40.76832122812564], [-73.94575361316765, 40.76830167121358], [-73.94570357284314, 40.76832793415568], [-73.94559658517926, 40.76847086500249], [-73.94548114294636, 40.76862477921883], [-73.94562721096725, 40.768692953089385], [-73.94556302024347, 40.76880177601348], [-73.94540491875006, 40.76873217301927], [-73.94538208189473, 40.768763203023916], [-73.94524510320667, 40.76877326815514], [-73.94515704210755, 40.769118317875765], [-73.9450906579704, 40.76921372115898], [-73.94472353210328, 40.76973405894194], [-73.94472478498723, 40.76978627176733], [-73.94467293291667, 40.76981996802662], [-73.94456161508772, 40.769886395963674], [-73.9442700703175, 40.77014053537861], [-73.94400423978152, 40.77029398207531], [-73.94320414157107, 40.77099239987431], [-73.94281710980138, 40.77141086085112], [-73.94257830300492, 40.7717018568592], [-73.94212674623404, 40.77203470075466], [-73.94136990721195, 40.77230915091384], [-73.94088707205813, 40.77237251588802], [-73.94056957410288, 40.77261475898201], [-73.94027460960578, 40.77294743744903], [-73.94017959802049, 40.77295931579713], [-73.94007665725826, 40.77292618617896], [-73.9400115727917, 40.77284236077629], [-73.94018451224508, 40.772564906804156], [-73.94028207924202, 40.772301406991225], [-73.9402776672235, 40.77171314173954], [-73.94020021779653, 40.77112830566273], [-73.94022870389077, 40.770852261664444], [-73.94027985829325, 40.77067071100977], [-73.94035533477506, 40.77049657437108], [-73.94082605300244, 40.76991215726808], [-73.9411177871802, 40.76941440857463], [-73.94121934173008, 40.76933434312226], [-73.94125975382602, 40.769308879392725], [-73.94180032729437, 40.76904692662474]]]}}, {\"id\": \"202\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 203, \"Shape_Leng\": 0.189937815868, \"Shape_Area\": 0.000615129861659, \"zone\": \"Rosedale\", \"LocationID\": 203, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7275350030123, 40.673930284146124], [-73.72779414890223, 40.67145467623331], [-73.72788358613498, 40.670796436811784], [-73.72792572875012, 40.670097148261284], [-73.72799445690038, 40.669426693791536], [-73.7281284031003, 40.66870817078305], [-73.72826385201665, 40.66807287581589], [-73.72822217020318, 40.667313639278575], [-73.72826620792841, 40.666606286172005], [-73.7282727227748, 40.66642959204175], [-73.72821911979167, 40.66591458472654], [-73.72814252019296, 40.665514374138105], [-73.72811351398208, 40.66534531496973], [-73.72787486779977, 40.664695132359554], [-73.72783352392932, 40.66397643722227], [-73.72781392396702, 40.66322396923896], [-73.72808466920584, 40.663071188790134], [-73.72833071887973, 40.66304394386548], [-73.72815281678892, 40.662454142752736], [-73.7279312440573, 40.66174883505236], [-73.7275917590304, 40.66109448811916], [-73.72720802015712, 40.660422320847296], [-73.72680978675194, 40.65979485606475], [-73.72644438687217, 40.65916161496058], [-73.7262113990149, 40.658478816948495], [-73.72602875988939, 40.65799899893644], [-73.72592687720594, 40.65773131917636], [-73.72580343602628, 40.657191252818514], [-73.72575924916657, 40.65699795914443], [-73.72570240303517, 40.65645873584837], [-73.72555284202292, 40.65503995829803], [-73.72533218511886, 40.65430869419415], [-73.7250556910977, 40.65350789382087], [-73.72509934591031, 40.652345748155845], [-73.72522608103498, 40.652001000073724], [-73.7264979013976, 40.65166351420005], [-73.73028652986073, 40.65036852283714], [-73.73051457300515, 40.65031031200321], [-73.73207263098428, 40.650087474361364], [-73.73398259890436, 40.64985220914876], [-73.73513501486147, 40.64973204023334], [-73.73673508618299, 40.649099816248636], [-73.7390448211571, 40.64820118017219], [-73.74143706021165, 40.646889178160706], [-73.74143890933054, 40.64634484395938], [-73.74143979613383, 40.64608557613655], [-73.74144760054372, 40.64594171784392], [-73.74146811233385, 40.64556352703525], [-73.74148523636285, 40.645247833122994], [-73.74162258204508, 40.64271602721339], [-73.74166141308763, 40.64219806209119], [-73.74205084926373, 40.6414742047628], [-73.74142279307473, 40.64050339087667], [-73.74227387212949, 40.64012338122567], [-73.74125900083672, 40.63889749946873], [-73.74033103314679, 40.63726030097883], [-73.73955759615876, 40.63586253413046], [-73.739471216134, 40.63570643191466], [-73.73986921396012, 40.63524791286182], [-73.7399595753528, 40.6351438091866], [-73.74001376732642, 40.6353455867249], [-73.74020461124273, 40.635264471195924], [-73.740656399135, 40.63520548812592], [-73.7406680700561, 40.63520396413286], [-73.7406637791744, 40.63518252913801], [-73.740773490474, 40.635163200064824], [-73.74076026693612, 40.63512664736521], [-73.74074429132365, 40.63512998806589], [-73.74075056900965, 40.63515580273157], [-73.74073204713044, 40.63515833840738], [-73.74065547157522, 40.63516881823407], [-73.74065635495072, 40.63518273637971], [-73.74053168112293, 40.6352039754206], [-73.74052856928526, 40.635170700437456], [-73.74044226722103, 40.63518263812667], [-73.74041969158273, 40.63517860666841], [-73.74042047354324, 40.63514429844044], [-73.74043676987378, 40.635141673858776], [-73.7404395370067, 40.63515257587345], [-73.74074057242717, 40.63509750865651], [-73.74091563209127, 40.635065483979695], [-73.74091396115182, 40.63506080464041], [-73.74097436060475, 40.63504828823865], [-73.74097754781047, 40.63505721601665], [-73.7409838783064, 40.63505590512063], [-73.74099288434387, 40.635081128174654], [-73.74098314718391, 40.63508404587671], [-73.7409892725328, 40.63510119942609], [-73.74092361851515, 40.63511480617143], [-73.74091479001564, 40.635090079733835], [-73.74089879802692, 40.635093394619396], [-73.74091019852334, 40.63512532307322], [-73.74080494840042, 40.63514713489751], [-73.74079465283724, 40.635118301681715], [-73.74077422547461, 40.63512253533369], [-73.74078781200114, 40.6351610348698], [-73.74102145003503, 40.635162790993924], [-73.74133506687683, 40.63511399889662], [-73.74164858697307, 40.63505389381684], [-73.74164684544989, 40.63504715472515], [-73.74163556420449, 40.63501375972658], [-73.7416959670252, 40.63500169042439], [-73.74167886270997, 40.63495617450636], [-73.74163328571251, 40.634961456830304], [-73.74163098437188, 40.6349470908835], [-73.74144863296593, 40.63497839545674], [-73.74144240592258, 40.634964020678765], [-73.74170882729821, 40.634926322434815], [-73.74170957088873, 40.634937095231436], [-73.7417693079242, 40.63492645827809], [-73.7417731743439, 40.6349420243985], [-73.74170635665378, 40.634955637353514], [-73.7417193582388, 40.634997161881145], [-73.74176899243555, 40.63498866620964], [-73.74178775322268, 40.635021719479006], [-73.74178960649937, 40.6350214041624], [-73.74192006040933, 40.63499916334794], [-73.74200012051077, 40.63498325953497], [-73.74221252506776, 40.63501581057908], [-73.74231218319328, 40.63504370812134], [-73.74246642615259, 40.635119228252094], [-73.7425014825021, 40.63516891578415], [-73.74252931736748, 40.635221188481914], [-73.74254960752418, 40.63527543937579], [-73.74255145972872, 40.63528340869711], [-73.74255152692179, 40.63529150181095], [-73.74254980719093, 40.635299488343584], [-73.7425463494877, 40.63530714095494], [-73.74254125223675, 40.63531424181038], [-73.74253466053294, 40.635320588780786], [-73.74252676201193, 40.63532600119686], [-73.74251778150813, 40.63533032499192], [-73.74250797465538, 40.63533343708692], [-73.74249590726653, 40.63533541538815], [-73.74248185224103, 40.63533948252295], [-73.74246887116452, 40.635345261159905], [-73.74245732011887, 40.63535259278653], [-73.74244751595937, 40.63536127628996], [-73.74243972762315, 40.635371073474545], [-73.74238800510525, 40.63549865225259], [-73.7423654773219, 40.63555422299664], [-73.7424158673349, 40.635569117161985], [-73.7423912395262, 40.635630300126756], [-73.7424290274033, 40.6356395206516], [-73.74243665568173, 40.63564138156584], [-73.74255329130374, 40.63544146022971], [-73.74258539843598, 40.63545335666109], [-73.74243638020491, 40.63571275510745], [-73.7424012521535, 40.635698675159766], [-73.74239929447099, 40.635697890757854], [-73.74240752450443, 40.63568531819995], [-73.74242717666695, 40.635655297928054], [-73.74242234728739, 40.63565374835393], [-73.74238343507966, 40.63564126234088], [-73.74236603847486, 40.635696126549824], [-73.74231340408035, 40.63568798445787], [-73.74224399981287, 40.635825094654194], [-73.74217084528762, 40.63596535231388], [-73.7420809643295, 40.63609706192347], [-73.74194304350247, 40.636262257290824], [-73.74190682047218, 40.63629123963369], [-73.74176405727077, 40.636438638677255], [-73.74173263035995, 40.6364285803368], [-73.74149147298868, 40.636653631785016], [-73.741457402659, 40.636686544370384], [-73.7414418260182, 40.636701590327284], [-73.7412429544081, 40.63693385749198], [-73.74117860782604, 40.6370641976703], [-73.74113327509292, 40.63715533874024], [-73.74096487686366, 40.63730477503007], [-73.74109421928736, 40.637646355176074], [-73.74112989506905, 40.637714333579964], [-73.74115278454505, 40.637742857535336], [-73.74117306495566, 40.63776270681417], [-73.74118644354527, 40.63778238257523], [-73.74120095386145, 40.63779949885748], [-73.74121658796302, 40.63781576293301], [-73.74123335092197, 40.63783032058112], [-73.74125345959186, 40.63784915842032], [-73.74129078685053, 40.63788603316224], [-73.74130372491103, 40.63789881291847], [-73.7413056658439, 40.637900414173444], [-73.74131825515447, 40.63791080300416], [-73.74133615185963, 40.63792194877484], [-73.74134845884922, 40.63792880880198], [-73.7413540421291, 40.63793180606096], [-73.74135964846242, 40.637934813255875], [-73.74136748331289, 40.63793824813897], [-73.74137195611459, 40.63794167378464], [-73.74137979362102, 40.63794425343616], [-73.7413829865627, 40.63794426061731], [-73.74138764391616, 40.637944271091875], [-73.74139661688328, 40.63794343821319], [-73.7413999840108, 40.6379425910516], [-73.74140223045113, 40.63794174187227], [-73.74141346178008, 40.63793664257871], [-73.74141683155504, 40.63793494186131], [-73.74142020000122, 40.63793409470204], [-73.74142244203142, 40.63793409974375], [-73.74142468384161, 40.63793410478491], [-73.74144486341844, 40.637935004894906], [-73.74145382711917, 40.63793673334487], [-73.74146279147763, 40.637938462465826], [-73.74148518992935, 40.63794534516711], [-73.74150869561394, 40.63795564761388], [-73.74158142517841, 40.637995102881575], [-73.74160828610616, 40.63800797654757], [-73.74172246881945, 40.63805606667883], [-73.74187696472138, 40.638117911883164], [-73.74189040305612, 40.638122213540264], [-73.74193747964232, 40.638126589519885], [-73.74202490121498, 40.63813618222593], [-73.74211119816731, 40.63814662540193], [-73.74219748524882, 40.63815963051505], [-73.74228377060031, 40.63817349079602], [-73.74238797185457, 40.6381933699328], [-73.74249440927368, 40.63821496345038], [-73.74260419719744, 40.638239979954456], [-73.74271621824597, 40.63826756337385], [-73.74272966303361, 40.63827015598163], [-73.74276445497053, 40.63826169200664], [-73.74289350196115, 40.638235500952234], [-73.74302365615813, 40.63821273015811], [-73.74310107004379, 40.63820094456608], [-73.74318072177344, 40.6381900181516], [-73.74326148890701, 40.63818080414269], [-73.74334449521898, 40.63817244712844], [-73.74350517659867, 40.638083972966285], [-73.74373884101573, 40.63796918108366], [-73.74397128496219, 40.63788001261484], [-73.74403815450607, 40.637986076622575], [-73.7437853783843, 40.63842873057221], [-73.7436606468547, 40.638647155902675], [-73.74352389261917, 40.6388866326333], [-73.74352226238186, 40.63888948637474], [-73.74356355009007, 40.63894168237593], [-73.74364746537917, 40.63898884834804], [-73.743666112419, 40.638995999520056], [-73.7437342150463, 40.6390221176639], [-73.74376618359526, 40.639020322461185], [-73.74378684907364, 40.63900571121325], [-73.74383148928239, 40.63895978102217], [-73.74384645847971, 40.638955394111186], [-73.74402450885484, 40.63866200607285], [-73.74402804795417, 40.638656173046186], [-73.74402997541277, 40.6386529966277], [-73.74402564937427, 40.638651566852985], [-73.74401611368735, 40.63864841433777], [-73.74396627971575, 40.638631939474664], [-73.74395937455591, 40.638628768184866], [-73.74395663263608, 40.63862033552935], [-73.74398391923127, 40.6385585575775], [-73.74403572252056, 40.63844127682693], [-73.74404183532316, 40.63843549777715], [-73.74404519787957, 40.63843509464784], [-73.74406202167812, 40.6384330734959], [-73.7440786978075, 40.63843492109936], [-73.74408094395514, 40.638434946369344], [-73.74409080311706, 40.63843505894466], [-73.74411229309374, 40.638379370311505], [-73.74412476786969, 40.638347042992116], [-73.74418327814672, 40.63819542255729], [-73.74416787097152, 40.63819157414296], [-73.74415629764292, 40.63818422033203], [-73.74415773134824, 40.63818058998131], [-73.74422124418085, 40.638019704396356], [-73.74426333512342, 40.63795441304668], [-73.74445896739225, 40.637655119311184], [-73.74450503104404, 40.63758053346876], [-73.74453083731126, 40.637576320486055], [-73.74455328893065, 40.63756953684862], [-73.74457350396703, 40.63756103994328], [-73.74458136924316, 40.63755678709512], [-73.7445892362507, 40.6375516795163], [-73.74460273481121, 40.63754060566321], [-73.74461624163654, 40.63752696879766], [-73.74465002034718, 40.63749031504757], [-73.7446939616317, 40.63743489143131], [-73.74472891886717, 40.63738286461201], [-73.74473117199703, 40.63738030809243], [-73.74473230486352, 40.63737689384945], [-73.74473345626623, 40.63736920882553], [-73.74473236368084, 40.637361518149994], [-73.7447279674387, 40.637338446655924], [-73.74472799357945, 40.637331613197915], [-73.74467585741199, 40.63731383199284], [-73.74468657946085, 40.637295753264695], [-73.74470130596333, 40.63730140455609], [-73.74470379258364, 40.63729391951125], [-73.74467432936497, 40.63728511526765], [-73.74470895990751, 40.63722838970686], [-73.7447138760486, 40.63722840062137], [-73.7447955814851, 40.63707502907608], [-73.74483896213275, 40.63708698507472], [-73.74471275511705, 40.63730704689421], [-73.74473447464202, 40.637313227015916], [-73.74474271381361, 40.6372940630451], [-73.74475964729425, 40.63726420538296], [-73.74477993647058, 40.63723606245654], [-73.74480358440826, 40.63720963695133], [-73.74481145943241, 40.637202820409584], [-73.74482045163967, 40.637196861080625], [-73.74484067698694, 40.63718580162469], [-73.74486538068541, 40.637176460396994], [-73.74489231636274, 40.63716968731257], [-73.74493270409721, 40.63716379762361], [-73.74496859387851, 40.637161314702475], [-73.74500110986895, 40.637161386815116], [-73.7450302490282, 40.63716486818898], [-73.74504257301133, 40.63716745804168], [-73.74505601113682, 40.6371717598314], [-73.74508287457526, 40.637183776282356], [-73.74508334793234, 40.63718397987455], [-73.7451007854299, 40.6371915035677], [-73.74511757909548, 40.63719837481242], [-73.7451276632474, 40.63720010495726], [-73.74513662540275, 40.637201833786996], [-73.74514559368646, 40.63720270672301], [-73.7451466706038, 40.63720259217439], [-73.74515344517405, 40.637201871065756], [-73.74516353911312, 40.63720103870244], [-73.7451725195585, 40.637198496580545], [-73.74519496966812, 40.63719171231205], [-73.7452257881082, 40.637163630500375], [-73.74528780756869, 40.63712970385058], [-73.74530950051825, 40.63711657137385], [-73.74531990806949, 40.63711175534441], [-73.74533233070176, 40.63710664792052], [-73.74534457149676, 40.63710182003374], [-73.7453854460563, 40.637080882618484], [-73.74540060046327, 40.63707600136936], [-73.74541408813883, 40.637067489420794], [-73.74553205659532, 40.63700539751762], [-73.74563429196256, 40.636952665561225], [-73.74567138084416, 40.63692968469265], [-73.7457354656978, 40.63688370227758], [-73.74576131977507, 40.636866675821324], [-73.74586695037763, 40.63680540935348], [-73.74592203058502, 40.63676880130039], [-73.74597825393182, 40.6367262181368], [-73.74600637720442, 40.63670236312222], [-73.74603225626699, 40.63667850381399], [-73.74605477568909, 40.63665378335162], [-73.74607359389105, 40.63662948844446], [-73.74607393134647, 40.63662905495397], [-73.7461014404632, 40.63660412436256], [-73.74612214127606, 40.636579489331794], [-73.74614353062222, 40.636555200089944], [-73.7461655987983, 40.63653126365129], [-73.74618833562963, 40.63650769540581], [-73.7462117333877, 40.63648450354488], [-73.74623652817685, 40.636462655330966], [-73.7462638571202, 40.63644221911155], [-73.74629219000661, 40.6364372679824], [-73.74636290096177, 40.63645664684356], [-73.74643086896202, 40.63648106607109], [-73.7464954765674, 40.63651030383521], [-73.74655613686292, 40.63654409453334], [-73.74661229878836, 40.63658213120217], [-73.74668073499127, 40.636647518865125], [-73.74674269086134, 40.63671655457172], [-73.74676319099969, 40.636734919779585], [-73.74677936191642, 40.6367556420797], [-73.74679074637297, 40.63677813554669], [-73.74679702246733, 40.636801764174585], [-73.74679801273618, 40.63682585985995], [-73.7467936891742, 40.63684974129233], [-73.74678417402505, 40.636872733218965], [-73.74676973632697, 40.63689418553679], [-73.7466521856608, 40.63703299024159], [-73.74658986937197, 40.637113263923226], [-73.74648181054782, 40.63720049205395], [-73.74645973515285, 40.637227978646926], [-73.74645394301926, 40.6372598208812], [-73.74635020373697, 40.63732924134759], [-73.74620675133012, 40.63740451164501], [-73.74619472121903, 40.637399626243266], [-73.74613888619244, 40.63743378549239], [-73.74614917206596, 40.63744973737823], [-73.7460362068704, 40.63752183642386], [-73.74598653863283, 40.637536844549714], [-73.74596746181402, 40.6375211449612], [-73.7457196111919, 40.637652335898274], [-73.74553630736398, 40.63776963121703], [-73.7454922640949, 40.63779598978515], [-73.74545481335466, 40.637798617678136], [-73.74527138642912, 40.63791590124532], [-73.74527129986721, 40.637938577469136], [-73.7452379014121, 40.637961719505704], [-73.74524280313427, 40.63797738781067], [-73.74524489519087, 40.637986031423075], [-73.74519377749161, 40.63800913536163], [-73.74516813350769, 40.63804309215177], [-73.74515886775069, 40.63807741225209], [-73.74516219973704, 40.63811218797305], [-73.74519182005321, 40.638139770278514], [-73.74521617222685, 40.638170197028785], [-73.74523479225815, 40.638202888489566], [-73.74524732536197, 40.63823722177521], [-73.74525353272955, 40.63827254271655], [-73.74525371526053, 40.63827490477969], [-73.74525473246845, 40.63828445716307], [-73.7452576164747, 40.63829378570828], [-73.74526230220606, 40.638302679927996], [-73.74526868393507, 40.63831093913482], [-73.74527661766675, 40.63831837696979], [-73.745285924386, 40.63832482560704], [-73.74529639409818, 40.63833013954073], [-73.74532770333599, 40.63837221196975], [-73.74537980558401, 40.63829246200337], [-73.74567316004243, 40.63812465742794], [-73.74551058251748, 40.63794388103343], [-73.7456650449037, 40.6378363254409], [-73.74571895695183, 40.63782402657131], [-73.74602015126122, 40.63764382151664], [-73.74614300250167, 40.63758092290193], [-73.74615726236783, 40.63758185220314], [-73.7461715382726, 40.63758107889671], [-73.74618548204819, 40.63757862184229], [-73.74619875362757, 40.63757454096377], [-73.74621102933739, 40.63756893578738], [-73.74622200979226, 40.63756194301498], [-73.74628878934672, 40.63752640089844], [-73.74635102409644, 40.63748635705709], [-73.74635785086164, 40.63747925597855], [-73.74638148401334, 40.63745944882353], [-73.7464089339973, 40.63744274032665], [-73.74643950901621, 40.63742955157548], [-73.74644709594338, 40.63742325126385], [-73.74645339578426, 40.63741617550551], [-73.74645832276613, 40.63740838099433], [-73.74646168622706, 40.63740011801954], [-73.74648289889547, 40.63736994602533], [-73.74650999202005, 40.637342642155396], [-73.74654231123998, 40.637318865854134], [-73.74657199571628, 40.63727132712017], [-73.74660886413301, 40.63722680220138], [-73.74665239719059, 40.63718591822405], [-73.74667907572703, 40.637162951453284], [-73.74673378674966, 40.63710887499921], [-73.74678162543171, 40.63705114067182], [-73.74683530688736, 40.63699321074431], [-73.74689392766133, 40.63693812930759], [-73.74690287397662, 40.63692627973814], [-73.7469142713852, 40.636915721600566], [-73.74692780549884, 40.636906746130904], [-73.74694310299202, 40.63689960090885], [-73.7469597418971, 40.6368944830285], [-73.74697726324622, 40.63689153366156], [-73.74699518386497, 40.63689083408166], [-73.7470100138752, 40.63689898110525], [-73.74702300418711, 40.63690878292149], [-73.74703384090704, 40.63692000177473], [-73.74704226024105, 40.636932365240085], [-73.74704805917632, 40.636945575016966], [-73.74705109695319, 40.63695931005968], [-73.74705129976839, 40.636973237954514], [-73.74705091277869, 40.6370135677895], [-73.74704201885555, 40.63705332507568], [-73.74702484920296, 40.63709147605736], [-73.74699985119135, 40.6371270278801], [-73.74699391228548, 40.63715064484292], [-73.74699980678282, 40.63735439195552], [-73.74694320619099, 40.637529202088494], [-73.74693721618013, 40.63755610186007], [-73.74692334076764, 40.6375689439732], [-73.74691204524537, 40.637583178221206], [-73.74690356808544, 40.637598506801226], [-73.74689808620667, 40.63761460693872], [-73.74689571510714, 40.637631141760515], [-73.7468842769982, 40.63771943376995], [-73.74688172727956, 40.63773345181411], [-73.74686888875607, 40.63774554680735], [-73.74686422903173, 40.63773983099183], [-73.74683698181362, 40.6377666318109], [-73.74681465385633, 40.6378012512264], [-73.74678203982828, 40.637831866230115], [-73.7467805007807, 40.63784627104142], [-73.74676044982169, 40.637853007083805], [-73.74677257200611, 40.63788542222379], [-73.74679822393384, 40.63790165107325], [-73.7468198747719, 40.63792096983206], [-73.74683689197154, 40.63794281399733], [-73.74684877911791, 40.63796654619522], [-73.74685518918301, 40.637991473990446], [-73.74685593509076, 40.638016869000054], [-73.74685099495117, 40.6380419900177], [-73.74684051287743, 40.638066103836], [-73.74682479488087, 40.638088505952965], [-73.74679919514226, 40.63814425825788], [-73.74676951657275, 40.638198824417046], [-73.74673585173714, 40.638252031390294], [-73.74671166797484, 40.63826841107], [-73.74668498569231, 40.6382823580292], [-73.74665622285652, 40.638293653779684], [-73.74662525716256, 40.63830224980312], [-73.74660506399749, 40.638329575857945], [-73.74659068995052, 40.63835896011138], [-73.74658248411997, 40.638389689146535], [-73.74658064617921, 40.63842101621456], [-73.74654132522075, 40.63869678799164], [-73.74653979797041, 40.63871136266965], [-73.74653920203187, 40.63875876643843], [-73.74653311778164, 40.638805945225926], [-73.7465215919483, 40.63885253100168], [-73.74650471498472, 40.638898160047624], [-73.74649644705079, 40.6389161598987], [-73.74646260310949, 40.638942102090844], [-73.7464342260009, 40.63897160479255], [-73.74641195210027, 40.63900400500942], [-73.7463962818071, 40.63903857623228], [-73.74638756594, 40.639074543094445], [-73.74639333412867, 40.639085084998094], [-73.74639688005601, 40.63909618197724], [-73.74639811707112, 40.639107563992184], [-73.74639701512068, 40.639118954034174], [-73.7463936011126, 40.63913007499625], [-73.74638150617538, 40.63914702776413], [-73.74637306612549, 40.639165219143706], [-73.74636849123884, 40.63918419622715], [-73.74636789607887, 40.639203485810235], [-73.74637129507286, 40.639222605796995], [-73.74637860382623, 40.63924107920137], [-73.74637770384747, 40.63925189211933], [-73.74637447296097, 40.63926244327974], [-73.74636899999037, 40.63927244342559], [-73.7463614350201, 40.63928161835958], [-73.74635198583576, 40.63928971620863], [-73.74634091107603, 40.63929651437757], [-73.74632991201233, 40.639315030219635], [-73.74632293238811, 40.639334645482435], [-73.74632015215035, 40.639354858274714], [-73.74632164143556, 40.639375150052935], [-73.74632736315525, 40.639395000261615], [-73.74633716998578, 40.639413900998285], [-73.74635081024664, 40.63943136703504], [-73.7463525348708, 40.63943207118218], [-73.7463540851698, 40.63943298011282], [-73.74635542092985, 40.639434069185825], [-73.74635650581013, 40.639435309851685], [-73.74635731081932, 40.639436668041014], [-73.74635781420385, 40.639438108107875], [-73.74635800323406, 40.639439591813925], [-73.74635787288386, 40.63944107884276], [-73.74635742589957, 40.63944252982913], [-73.74635667475202, 40.639443905890126], [-73.74635563897762, 40.639445170142515], [-73.7463543469099, 40.63944628919412], [-73.7463528328097, 40.63944723309525], [-73.746339761774, 40.63947941688861], [-73.74632968004435, 40.63951221822162], [-73.74632769317725, 40.63952292313115], [-73.74632774309308, 40.63953373423414], [-73.74632982877726, 40.639544428197794], [-73.74633390615892, 40.63955478390221], [-73.74633989205708, 40.639564586229305], [-73.7463337720348, 40.63967796479996], [-73.74633694803521, 40.639791412877585], [-73.74633811184864, 40.63987714617596], [-73.74616200777528, 40.64041774225677], [-73.74616735915122, 40.640430848147076], [-73.74617495614544, 40.64044329391234], [-73.74622488120059, 40.64048906604952], [-73.74626850888167, 40.64053843194672], [-73.74630539588146, 40.640590890195135], [-73.74633516808112, 40.6406459091977], [-73.74635752395051, 40.640702930868216], [-73.74635908816687, 40.640731022895444], [-73.74635425902625, 40.640766735886004], [-73.74634189979751, 40.64080138113564], [-73.74632233618877, 40.6408340441297], [-73.74629608604022, 40.6408638620328], [-73.74628411090862, 40.64088982724007], [-73.74627759987665, 40.640916898037254], [-73.74627671216511, 40.64094441098869], [-73.74628147014262, 40.640971692242736], [-73.74629175705896, 40.64099807478083], [-73.74630732097097, 40.641022911095824], [-73.74632778082714, 40.64104559410167], [-73.74634653518861, 40.64109389680149], [-73.74635900398914, 40.64114336394255], [-73.74636506704749, 40.64119352277494], [-73.74636466683809, 40.6412438922395], [-73.74635780677009, 40.64129399176526], [-73.74635155089871, 40.64129737283889], [-73.74634606106508, 40.64130145375825], [-73.74634147100916, 40.641306134911474], [-73.74633789203455, 40.64131130288634], [-73.74633541115193, 40.6413168318543], [-73.74633408902065, 40.641322587098266], [-73.74633496524163, 40.641324142179286], [-73.74633551187432, 40.64132578290818], [-73.74633571419191, 40.641327468086686], [-73.7463355682, 40.641329156963344], [-73.74633507719585, 40.64133080724513], [-73.746334253225, 40.64133237910914], [-73.74633311693245, 40.64133383320319], [-73.74633169581273, 40.641335134122336], [-73.74633002486517, 40.64133624984959], [-73.74632814516256, 40.64133715215073], [-73.74631998880159, 40.641355633771084], [-73.74631546715995, 40.64137482484783], [-73.74631467883957, 40.641394313090785], [-73.7463263960144, 40.64139993015429], [-73.74633897087129, 40.64140434222867], [-73.74635437383768, 40.641407856925035], [-73.74635320161447, 40.64142234107921], [-73.74634883268592, 40.641436466113134], [-73.74634139288489, 40.64144982587809], [-73.74635254190866, 40.64148725614551], [-73.74637170395165, 40.64152275416235], [-73.74639834592102, 40.64155533205001], [-73.7464317261564, 40.64158408305852], [-73.74650806618486, 40.64163919898093], [-73.74654320079884, 40.64169085889284], [-73.74657654809101, 40.6418254975971], [-73.74664070001944, 40.64198935977372], [-73.74667588614882, 40.64206492995331], [-73.74670489405828, 40.64214200794834], [-73.746720374971, 40.64221253186798], [-73.74674116403365, 40.642282259021144], [-73.74675343215667, 40.64230856119203], [-73.74677148184261, 40.64233285382945], [-73.74679476917676, 40.64235440513798], [-73.74682259497264, 40.64237256803435], [-73.74685412180307, 40.64238679589501], [-73.74686279407507, 40.64239063175947], [-73.74687053986388, 40.64239548384163], [-73.74687715501426, 40.64240122496013], [-73.74688246620484, 40.64240770476527], [-73.74688633422684, 40.64241475221656], [-73.746888657815, 40.64242218381513], [-73.74688937497864, 40.642429803031426], [-73.74688846780323, 40.6424374108656], [-73.74688593494254, 40.64244485988226], [-73.74688184514358, 40.642451895899086], [-73.74688032576586, 40.64248366095721], [-73.74688465111727, 40.642515275935864], [-73.74689473721212, 40.64254611979221], [-73.74690499708855, 40.64256327719392], [-73.74691884764626, 40.64257890305549], [-73.74693590494275, 40.64259256489], [-73.7469556986995, 40.64260388485852], [-73.74697768082069, 40.64261254999068], [-73.74698636565677, 40.64261449711887], [-73.74699477003442, 40.64261705996882], [-73.74700930015858, 40.642623327841115], [-73.74702218202302, 40.64263142109823], [-73.74702879315804, 40.64264421183799], [-73.74718790386437, 40.64287869159084], [-73.7471908910085, 40.642942985903574], [-73.74720293919907, 40.6430066628579], [-73.74722390905603, 40.643068983064566], [-73.74725355694629, 40.643129224001946], [-73.7472915388935, 40.64318668671111], [-73.74733741407343, 40.6432407038501], [-73.74739065087017, 40.6432906490615], [-73.7473963210573, 40.64329656386502], [-73.7474029040544, 40.643301901860134], [-73.74741029980449, 40.64330658195646], [-73.74741453307503, 40.64330878282975], [-73.74743277987598, 40.64331131486161], [-73.74745026095411, 40.643316034898085], [-73.74746652776999, 40.643322821811076], [-73.74748116188306, 40.64333150092601], [-73.74749378782086, 40.64334184976103], [-73.74750408090786, 40.64335360189041], [-73.74751177693027, 40.64336645586005], [-73.74751667803395, 40.643380081064656], [-73.74751865876323, 40.64339412798986], [-73.74751695111142, 40.64340254926681], [-73.74751699780106, 40.64341107091824], [-73.74751879675959, 40.643419481158844], [-73.74752230376089, 40.643427573072834], [-73.74752743197418, 40.64343514618109], [-73.7475340550353, 40.643442013412646], [-73.74754200911231, 40.6434480054163], [-73.7475489641527, 40.64344793740736], [-73.747555843039, 40.64344872269587], [-73.74756246658518, 40.64345034153919], [-73.74756866093848, 40.64345275133291], [-73.74757426583201, 40.64345588916065], [-73.74757913469017, 40.64345967311334], [-73.74758314053385, 40.64346400435616], [-73.74758617863681, 40.64346877041464], [-73.74758816988634, 40.64347384714796], [-73.74758906281693, 40.64347910186851], [-73.74758883378736, 40.64348439706855], [-73.747587488784, 40.643489595206034], [-73.74760994492256, 40.64352963498251], [-73.74763920510865, 40.643567038059885], [-73.74767474790812, 40.64360113788255], [-73.74771593886167, 40.643631325854756], [-73.74776204313743, 40.64365706376416], [-73.74776870697815, 40.64365720497771], [-73.74777524297676, 40.64365820395108], [-73.74778146223453, 40.643660031756575], [-73.74779288977031, 40.64367507318876], [-73.74781185615768, 40.643691114899866], [-73.74782855597535, 40.64370855719547], [-73.74784784286973, 40.64373505696274], [-73.74786191116263, 40.643763397485436], [-73.74787046781157, 40.64379298755474], [-73.74787635413617, 40.643814913439236], [-73.74788698622861, 40.643835776334306], [-73.74790206240706, 40.64385498506315], [-73.74792115600381, 40.64387199439488], [-73.74794372492359, 40.643886323260695], [-73.74796912931046, 40.643897565087016], [-73.74799665011672, 40.64390540105188], [-73.74801489111466, 40.64390792003266], [-73.7480322809754, 40.64391281315845], [-73.74804829105341, 40.64391993210214], [-73.74806243515813, 40.64392906022952], [-73.74807387893388, 40.64393947877813], [-73.7480828762219, 40.643951206788415], [-73.74808917505155, 40.64396391491117], [-73.74808916506115, 40.64398687302886], [-73.74809398823074, 40.644009535205576], [-73.74810352014437, 40.64403131516901], [-73.7481175139531, 40.644051649065965], [-73.74813560706815, 40.6440700107574], [-73.74815733184688, 40.644085924129165], [-73.74818212515288, 40.644098978241296], [-73.74820934613322, 40.64410883515944], [-73.7482245059317, 40.644114708946006], [-73.74823846022485, 40.644122115129996], [-73.74825068985453, 40.64413070789769], [-73.74826128919973, 40.64414047595863], [-73.74827006681191, 40.64415124193082], [-73.7482768618246, 40.644162809852915], [-73.74829894571495, 40.644172048091995], [-73.74832252972762, 40.644184060649096], [-73.74834849194528, 40.64419277370574], [-73.74840059785282, 40.644214571104705], [-73.74842757009225, 40.6442319818407], [-73.74845082016503, 40.644252269201175], [-73.74846633091204, 40.644279846138794], [-73.7485294907717, 40.64431811192], [-73.74855450017645, 40.644322651223185], [-73.74875121280796, 40.644471103175114], [-73.74876828394348, 40.64451640006256], [-73.74876528278705, 40.6445582518254], [-73.74876839311068, 40.644600098772194], [-73.74874242087304, 40.644667680807544], [-73.74871341519051, 40.64473454305686], [-73.74869723280953, 40.644789907198735], [-73.74868947609633, 40.64484631887667], [-73.748690248044, 40.64490303580214], [-73.74869953808962, 40.64495931217762], [-73.74871722418027, 40.64501440909765], [-73.74874307407524, 40.64506760191111], [-73.7487767478731, 40.64511819098248], [-73.74880038307, 40.64509693646107], [-73.74886499005787, 40.645141931957184], [-73.74884575010155, 40.64516880380206], [-73.74889242315085, 40.645165182763606], [-73.74892326194116, 40.64516602190768], [-73.74895379812877, 40.6451694079589], [-73.74896028077396, 40.645170463214406], [-73.74897280699656, 40.64517130174732], [-73.74898534781406, 40.64517059711792], [-73.74899757763455, 40.64516836895693], [-73.74900917889802, 40.64516467415515], [-73.74903252966776, 40.645158083124876], [-73.74905431995698, 40.64514890192631], [-73.749074058821, 40.64513733718795], [-73.74909129997339, 40.64512365093547], [-73.74910565481635, 40.645108151145216], [-73.74911679883317, 40.64509118825734], [-73.74912448024499, 40.645073145226604], [-73.74916792702578, 40.64504933986303], [-73.74921285809943, 40.645027190750184], [-73.74928320807959, 40.644953444829426], [-73.74930651309568, 40.64494161115258], [-73.74933203905464, 40.644932830114456], [-73.74935910296438, 40.64492733619018], [-73.74938698075869, 40.64492527687788], [-73.74941492598496, 40.64492670623158], [-73.74942668120917, 40.64492753299102], [-73.7494380991099, 40.64492981816267], [-73.74944886976486, 40.64493350023758], [-73.74945870003683, 40.64493847975471], [-73.74946732403991, 40.644944620155954], [-73.74947450775919, 40.64495175611966], [-73.74948005522563, 40.64495969382197], [-73.74948381716177, 40.64496821706417], [-73.74948569102449, 40.64497709519505], [-73.74950334599635, 40.645017244820245], [-73.74952942093164, 40.64505463897659], [-73.74956320702307, 40.645088262037596], [-73.74960378617716, 40.64511720012023], [-73.74965005599087, 40.64514066700189], [-73.74970075823686, 40.64515802480679], [-73.74971708505481, 40.64515953522574], [-73.74973294810908, 40.645162842976504], [-73.74974800895761, 40.645167876230175], [-73.74975879009766, 40.645172827850715], [-73.74976877794585, 40.645178665144904], [-73.74978096511317, 40.64518798386466], [-73.74979113708127, 40.64519861292796], [-73.74979905407045, 40.64521030217187], [-73.74991395709945, 40.64533036651034], [-73.75006516810481, 40.64544395362289], [-73.75011226494563, 40.64544742003597], [-73.75023266906744, 40.64553178494395], [-73.75039425820727, 40.64562633104435], [-73.75048097018974, 40.64566576731067], [-73.75052512012377, 40.645670347910375], [-73.7505382608302, 40.64569953203489], [-73.75062636605905, 40.64576027709227], [-73.75067194915144, 40.6457749541129], [-73.75107287523264, 40.64603934501582], [-73.75114929577599, 40.64607539489828], [-73.75125051822864, 40.64617205100801], [-73.75133567996826, 40.64623279044436], [-73.75135766788735, 40.646258628756826], [-73.75139443879556, 40.64626767984411], [-73.75141643490626, 40.64629127572598], [-73.75145614255564, 40.64630257591342], [-73.75147474305422, 40.646327524537206], [-73.75154567506182, 40.64637565823735], [-73.751607403107, 40.64640382522379], [-73.75166474521832, 40.64642413475903], [-73.75167487578338, 40.64647125390893], [-73.75171156796117, 40.646501609859], [-73.75186438589381, 40.64658043587709], [-73.75196590098793, 40.646598596964076], [-73.75220824210494, 40.64675386910737], [-73.75252557187562, 40.646935092227764], [-73.75277838595592, 40.64704553084226], [-73.75303393403244, 40.64721316513301], [-73.75308099198507, 40.64722784317051], [-73.75327090546294, 40.64733376275958], [-73.75340365378038, 40.647162374803656], [-73.75321112423842, 40.64707225193361], [-73.75323328284715, 40.64705211484303], [-73.7535419181792, 40.647196312159615], [-73.75352565808248, 40.64721422007991], [-73.75342130018778, 40.647168019235885], [-73.7532921068744, 40.647343402860926], [-73.75335085096773, 40.64735694895325], [-73.75340638296309, 40.64737686011339], [-73.75345752108981, 40.647402713017634], [-73.75350317781748, 40.647433957134396], [-73.7535423809316, 40.64746992807568], [-73.75355803278856, 40.647481761973665], [-73.7535758409182, 40.64749167017348], [-73.75359540119143, 40.647499428211255], [-73.75361627009028, 40.64750486003058], [-73.75363797324661, 40.64750784198052], [-73.7536600188211, 40.64750830621547], [-73.75371952093603, 40.64750950786568], [-73.75377829206127, 40.647516691037595], [-73.7538353001764, 40.647529730160265], [-73.75388954423967, 40.64754839624723], [-73.75406462691677, 40.64775918987505], [-73.75428169216147, 40.64788534600618], [-73.75439420696925, 40.64788558617587], [-73.75450655308, 40.647931533072594], [-73.75475454803566, 40.647806369149464], [-73.75487502521172, 40.64768093284719], [-73.7549204502295, 40.647566763850904], [-73.75460006764922, 40.6472007730313], [-73.75522386899995, 40.64744446899989], [-73.75502113300001, 40.6478138189999], [-73.75483677999976, 40.64807090299996], [-73.7545491879999, 40.64834992799991], [-73.75416443199985, 40.648654094999856], [-73.7541108669999, 40.648697664999915], [-73.75465758099988, 40.64901209699987], [-73.7579393749999, 40.6508533029999], [-73.76406290099993, 40.65435626599989], [-73.76404059399991, 40.65439329499984], [-73.76319000499986, 40.65589805399988], [-73.76299380499994, 40.65592780799994], [-73.762799522, 40.655973306999876], [-73.76261039799991, 40.65603444099988], [-73.76242965099998, 40.6561106019999], [-73.76226027499987, 40.656200664999915], [-73.76210493899991, 40.656303065999936], [-73.76196576499984, 40.656415891999856], [-73.76184429799997, 40.656536949999925], [-73.76177605899989, 40.65687448599989], [-73.76177603499984, 40.65687459399989], [-73.76177602300001, 40.65687469299997], [-73.76172316099992, 40.65721386399987], [-73.76172311299987, 40.65721414199993], [-73.76172307699989, 40.657214420999935], [-73.76168566599995, 40.65755484799992], [-73.76168565299982, 40.657554974999876], [-73.76168563999983, 40.657555109999876], [-73.76166374199983, 40.6578960729999], [-73.76166371699979, 40.65789649699987], [-73.76166370199995, 40.65789692799995], [-73.76165732899996, 40.658238085999855], [-73.76003600699998, 40.658073440999885], [-73.75922652199985, 40.657902235999906], [-73.75845384399994, 40.657647902999955], [-73.75819056099982, 40.65755494699994], [-73.75763169899993, 40.6573576289999], [-73.75680959499985, 40.65709328399992], [-73.75606608399983, 40.65648646099987], [-73.75371894599986, 40.65536538099989], [-73.75294892899993, 40.65488504099994], [-73.752088853, 40.65451770399988], [-73.75149741999986, 40.65433688199992], [-73.75109427999986, 40.654212440999906], [-73.74937142799992, 40.653034834999914], [-73.74920814100003, 40.652933863999934], [-73.74867194699979, 40.65267808799992], [-73.74654917999989, 40.65255348199986], [-73.74332546799988, 40.652537803999905], [-73.74331805599995, 40.65328120399988], [-73.74331194199989, 40.65399441999995], [-73.74330720799989, 40.65470855899987], [-73.74329846199996, 40.65542352699992], [-73.74328714099991, 40.65613473199991], [-73.74377018899996, 40.65613902399991], [-73.74487420299995, 40.656144557999944], [-73.74492278899996, 40.65647897499991], [-73.74506037799983, 40.65740714999988], [-73.74618048999982, 40.657439074999914], [-73.74679970800003, 40.657471280999914], [-73.74705195899993, 40.657488288999915], [-73.74807112499987, 40.65754849199991], [-73.74853177199994, 40.65757569699993], [-73.74779788899987, 40.65852210299988], [-73.74733230899986, 40.65912452099992], [-73.74679260099995, 40.65890762199992], [-73.74684750599981, 40.660795540999864], [-73.7461369799999, 40.6617167199999], [-73.74450922599979, 40.663821277999865], [-73.74405313899989, 40.66365807099993], [-73.74374970899983, 40.66409039299991], [-73.74340973899989, 40.6644041699999], [-73.74281884699987, 40.66530554899986], [-73.73954958300003, 40.66525602199989], [-73.739371342, 40.66545222799987], [-73.73920855099999, 40.66562918899995], [-73.73891022900001, 40.66595449199986], [-73.73856469500002, 40.66639689799991], [-73.73993110699995, 40.66640155399987], [-73.74010286899983, 40.666402139999896], [-73.74021946399998, 40.6664022769999], [-73.74042437599985, 40.66640252299988], [-73.74117817299991, 40.666402953999885], [-73.74080404599995, 40.666843967999924], [-73.73993331699991, 40.66776382799986], [-73.73921986399984, 40.66851888399989], [-73.73900041699997, 40.66874928599985], [-73.73869946900007, 40.6690682359999], [-73.73850086499984, 40.66928477299994], [-73.73823608799982, 40.66953170599987], [-73.73798806, 40.6697896359999], [-73.73775794399985, 40.67005754699992], [-73.7375467379998, 40.67033434399988], [-73.73735527499991, 40.67061884099995], [-73.73718424499995, 40.670909812999895], [-73.73593438599976, 40.67243297099988], [-73.73624941800001, 40.67263060199991], [-73.73597469399992, 40.67256742099995], [-73.73561853899999, 40.67313052699989], [-73.73509933199999, 40.673924256999896], [-73.73505915299991, 40.67398974999989], [-73.73502751399988, 40.67402828999989], [-73.73403489799982, 40.6752376569999], [-73.73384204199998, 40.67546337699988], [-73.733322441, 40.676132565999914], [-73.73303223399991, 40.67649502599989], [-73.73187383000001, 40.67756892099986], [-73.73106142499994, 40.6783066209999], [-73.73047301200002, 40.67934793699989], [-73.72950555799994, 40.68005572599989], [-73.72924739999999, 40.68043098599987], [-73.72898998699992, 40.68068281299988], [-73.72874616700001, 40.680942857999874], [-73.72851656799985, 40.68121047299993], [-73.72830180799986, 40.68148497499995], [-73.72810238799995, 40.681765636999934], [-73.72720084199985, 40.68356123499992], [-73.72718272499995, 40.68359732999987], [-73.72675245899993, 40.68346409399994], [-73.72666460699985, 40.683436888999964], [-73.72658306499989, 40.68341482999993], [-73.72586264191175, 40.683241080973865], [-73.7259066008032, 40.68265660481204], [-73.72582367682487, 40.681635750806386], [-73.72575419113436, 40.680945914047264], [-73.72568736813386, 40.680250454931496], [-73.72563005109944, 40.67958795089065], [-73.72601116044585, 40.678078338165896], [-73.72619806197862, 40.67738088986485], [-73.72654317309771, 40.67648672981019], [-73.72714134234852, 40.67509587592294], [-73.72744080479352, 40.674428741175404], [-73.72753358410341, 40.674213863808255], [-73.72755616378213, 40.674161588095764], [-73.7275350030123, 40.673930284146124]]]}}, {\"id\": \"203\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 204, \"Shape_Leng\": 0.101912418311, \"Shape_Area\": 0.000641384265972, \"zone\": \"Rossville/Woodrow\", \"LocationID\": 204, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.2032648999999, 40.55562847699989], [-74.2029103849999, 40.55494466599992], [-74.20262828599995, 40.554343511999946], [-74.20234825099999, 40.55375861799991], [-74.20216129599993, 40.55335994099991], [-74.20185089099988, 40.552689430999926], [-74.20097039299995, 40.55079013899987], [-74.20018262899993, 40.548906645999864], [-74.20012791899997, 40.54875987399988], [-74.19985031599995, 40.54813241899989], [-74.19912044699996, 40.546708706999894], [-74.19860457899993, 40.545679908999915], [-74.19840857099993, 40.545306446999895], [-74.19730759099996, 40.54326204499992], [-74.19702175199988, 40.54252684899992], [-74.19671334599992, 40.541846973999895], [-74.19640185999997, 40.54117319999993], [-74.19609298399995, 40.540497735999914], [-74.19576408099995, 40.53983810199987], [-74.19542518899992, 40.53917847399987], [-74.19507682399993, 40.53848646999991], [-74.19476262699999, 40.5378314079999], [-74.19468316799997, 40.53766189199994], [-74.19444077699995, 40.5371565929999], [-74.19409661699994, 40.53650724499989], [-74.19377169299999, 40.53583384699993], [-74.19356025199987, 40.53540617199993], [-74.19341548699992, 40.53512049899993], [-74.19335436399993, 40.53499987599987], [-74.19329057000002, 40.53487778199998], [-74.19300900999994, 40.53432605099987], [-74.19267129299996, 40.533633488999904], [-74.19237403399998, 40.533040610999905], [-74.19174368799999, 40.53191669699986], [-74.19250928499991, 40.53149740799989], [-74.19313382999997, 40.53119491499988], [-74.1939845689999, 40.53087104699991], [-74.19453974799998, 40.530619741999885], [-74.19478532600002, 40.53051789799983], [-74.19608390199986, 40.52991174699987], [-74.19779445399999, 40.52903208399988], [-74.19909783799994, 40.52829964499994], [-74.19927554899992, 40.52819690099989], [-74.19984360499993, 40.52785744099988], [-74.2005284199999, 40.52736440399985], [-74.20071057499999, 40.52722595199986], [-74.200879328, 40.52707723199989], [-74.20103324199994, 40.5269194149999], [-74.20117111699992, 40.52675382699987], [-74.20124767199997, 40.526626658999895], [-74.20133258999991, 40.526502620999935], [-74.201425651, 40.52638203299995], [-74.20152661599995, 40.526265205999934], [-74.20217525999989, 40.52606646299992], [-74.20276973299998, 40.5259072179999], [-74.203687632, 40.525782940999896], [-74.20525952799987, 40.525576179999874], [-74.20546090199988, 40.525544336999936], [-74.20666645799994, 40.52535435799996], [-74.20674433099997, 40.52534553599984], [-74.20730165099995, 40.5252824219999], [-74.207426748, 40.525262338999916], [-74.20755306499991, 40.52524734899991], [-74.20768024599988, 40.52523749299995], [-74.20780793299994, 40.52523279999989], [-74.20793576699992, 40.52523328199992], [-74.20806338799994, 40.52523893799989], [-74.20819043499995, 40.52524975299988], [-74.20831655199999, 40.52526569499987], [-74.20844138199995, 40.525286719999926], [-74.20856457599996, 40.52531276899991], [-74.20864934299995, 40.525341516999916], [-74.208737317, 40.525363960999854], [-74.20882769699993, 40.52537989899983], [-74.2089196559999, 40.525389182999895], [-74.20901235799994, 40.525391729999946], [-74.209104955, 40.52538751599993], [-74.20919660199999, 40.525376579999865], [-74.20928646399996, 40.52535902099987], [-74.20937372099999, 40.525334998999874], [-74.20945757499997, 40.52530473499994], [-74.20953726399988, 40.52526850299993], [-74.20961205799995, 40.52522663499995], [-74.20971385199991, 40.5251045449999], [-74.20983392599999, 40.5249908669999], [-74.20983398499993, 40.524990811999864], [-74.20983404399989, 40.52499076799997], [-74.20997053199989, 40.52488777599995], [-74.2101209019999, 40.52479737599988], [-74.21028201899998, 40.52472106699988], [-74.21142518399989, 40.52441410199989], [-74.21201514099992, 40.52431623099991], [-74.21229294499989, 40.524276852999876], [-74.2135190009999, 40.524148838999885], [-74.21409806799993, 40.52409719499991], [-74.2156723039999, 40.52395762799989], [-74.21559699299993, 40.52406106999992], [-74.21559935899988, 40.52437390099987], [-74.21560783599998, 40.52549473899995], [-74.215644706, 40.52598184799993], [-74.21574341099989, 40.52646400599987], [-74.21586638599997, 40.526647941999926], [-74.2161939549999, 40.52735009899994], [-74.21648646999992, 40.527925626999924], [-74.21661081199987, 40.52817658599992], [-74.21666369900001, 40.52828331599991], [-74.21672062699999, 40.5284057529999], [-74.216845171, 40.52864208699986], [-74.21701386699995, 40.52912208399992], [-74.21728324499995, 40.52987328299983], [-74.21752355299995, 40.53064379899991], [-74.21775551799986, 40.53144884899995], [-74.21800676499996, 40.53229858999995], [-74.2182685219999, 40.534525247999966], [-74.21829305299998, 40.53476500299996], [-74.2183151519999, 40.535312078999915], [-74.21828207299988, 40.53548704299991], [-74.21819886099995, 40.53585901999993], [-74.21806225399989, 40.536496512999875], [-74.21791053499996, 40.53702886099995], [-74.21786779799996, 40.53736657899988], [-74.21781643899992, 40.537655948999934], [-74.21779984299988, 40.53774949499994], [-74.21773895399996, 40.53803020699992], [-74.21770976099997, 40.5383119929999], [-74.21767785799992, 40.54013154699988], [-74.21775655499991, 40.54098679399993], [-74.21776941599994, 40.54112662999992], [-74.21800676099997, 40.54182852099993], [-74.21803946299991, 40.541913359999924], [-74.21817195699992, 40.54225028499995], [-74.21844432499992, 40.54296409499987], [-74.21873688699995, 40.54366800399986], [-74.21888584299994, 40.544070039999895], [-74.21899304799996, 40.54430991699989], [-74.21929684599996, 40.54505270699986], [-74.21954309399989, 40.5456993839999], [-74.21981449799995, 40.54643839699984], [-74.2198714179999, 40.54657677099993], [-74.21997344699999, 40.54683441599984], [-74.22008210999998, 40.547138270999874], [-74.22036701999993, 40.54783386999984], [-74.22056994699994, 40.5483906379999], [-74.22060624599999, 40.54848651999988], [-74.22064536099997, 40.54859150899989], [-74.22083496899992, 40.54909173999988], [-74.22087751999999, 40.54920399899988], [-74.22109340399999, 40.54977355599989], [-74.22131562699998, 40.55025307399992], [-74.2212741239999, 40.550336848999926], [-74.22121804999989, 40.55041641999991], [-74.2211486979999, 40.55048965799989], [-74.22106803199995, 40.55055473299996], [-74.21931406699997, 40.552055134999875], [-74.21911576499998, 40.55223231399988], [-74.21877630399993, 40.552473628999955], [-74.21778377699997, 40.55304918799994], [-74.21777232599997, 40.55324488499987], [-74.21783047799995, 40.553438685999886], [-74.21787055399989, 40.55363617699986], [-74.21789204399995, 40.55383585499985], [-74.21789469299988, 40.554036199999906], [-74.21787855399997, 40.55423566199991], [-74.21784397799998, 40.55443271799988], [-74.21721338199997, 40.554714858999915], [-74.21525777299996, 40.55543783799987], [-74.21509339699995, 40.55549893299997], [-74.21491900999992, 40.555545456999894], [-74.21473815899992, 40.555576114999965], [-74.21473802999996, 40.55557613299985], [-74.21473791199993, 40.555576142999875], [-74.21455430499994, 40.5555903089999], [-74.21455422299992, 40.55559031799991], [-74.21455413899993, 40.55559031799987], [-74.21437138999993, 40.55558806599986], [-74.21350175499995, 40.55566761799987], [-74.21220033999998, 40.55644342999993], [-74.21050789400002, 40.557064181999934], [-74.20880330699998, 40.557308688999896], [-74.20699885299997, 40.55756748899989], [-74.20671355499994, 40.557576935999876], [-74.20642597299995, 40.55756934899989], [-74.20613872499992, 40.55754450599989], [-74.205854485, 40.55750245699993], [-74.20557588399996, 40.5574435929999], [-74.20530544099992, 40.557368621999956], [-74.20520041099995, 40.5573322209999], [-74.20506288299997, 40.5572778759999], [-74.2049282399999, 40.5572054889999], [-74.20398449999993, 40.55661748499994], [-74.20384426499996, 40.55648572799986], [-74.20369556299995, 40.55634601699991], [-74.20363978099992, 40.55630271299988], [-74.20358106099994, 40.55626173299989], [-74.20351956699993, 40.55622318999985], [-74.20344936699998, 40.556183969999886], [-74.20337629799997, 40.55614792799985], [-74.20330246299997, 40.55611593799991], [-74.2032263689999, 40.55608719199989], [-74.20314825999996, 40.55606178499986], [-74.20318032199998, 40.5559919049999], [-74.20320742499993, 40.555920814999894], [-74.20322948999994, 40.55584871799988], [-74.20324645499988, 40.55577582299988], [-74.20325826899995, 40.55570233899991], [-74.2032648999999, 40.55562847699989]]]}}, {\"id\": \"204\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 205, \"Shape_Leng\": 0.141953388327, \"Shape_Area\": 0.000766216513505, \"zone\": \"Saint Albans\", \"LocationID\": 205, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.766369252, 40.70256339699992], [-73.76612489399999, 40.702334485999884], [-73.76544147799987, 40.702512314999886], [-73.76455359299997, 40.702742996999866], [-73.76366846199983, 40.70297297499996], [-73.76278023599987, 40.70320597499994], [-73.76262946699983, 40.70324155099991], [-73.76126359299998, 40.703598939999914], [-73.76095546600006, 40.702909577999954], [-73.76006675299989, 40.70314071099992], [-73.75918228200003, 40.70337203399986], [-73.75829554399995, 40.7036033809999], [-73.75740580399986, 40.70383535499992], [-73.75651887399991, 40.70406425799995], [-73.75561453899982, 40.70430046399987], [-73.75471127099998, 40.70453789099988], [-73.75382395799984, 40.704765834999876], [-73.752937772, 40.70499820599988], [-73.75205066499986, 40.70522802799995], [-73.75174489199993, 40.70454615499992], [-73.75143698199983, 40.703864323999895], [-73.75113533300001, 40.70320334299989], [-73.75082756899988, 40.70251465499992], [-73.75053038699998, 40.701854896999876], [-73.75022047099984, 40.70116599699989], [-73.74991098899996, 40.70047962799991], [-73.7496211959998, 40.69982977999993], [-73.7493054709999, 40.69914231299992], [-73.7501929579999, 40.69891208199988], [-73.74928080799988, 40.696894076999904], [-73.74863325999986, 40.6954447609999], [-73.74829993899988, 40.69553724399987], [-73.74774501699984, 40.695688540999946], [-73.74744768999992, 40.69577042099991], [-73.74624875999989, 40.69610314499993], [-73.74618157999997, 40.696121925999904], [-73.74612956299988, 40.69613573299988], [-73.74523747599996, 40.696374658999936], [-73.74507479999994, 40.69639094899984], [-73.7449099489999, 40.69639459999991], [-73.74474545399995, 40.69638546499987], [-73.74458386999987, 40.69636378199985], [-73.74365178399992, 40.69622415699989], [-73.7435928609999, 40.69621532499986], [-73.74386913499988, 40.695400117999945], [-73.74415874099992, 40.69454989999984], [-73.74426648799988, 40.69426010099992], [-73.74438939, 40.69397343399991], [-73.7445271819999, 40.69369059899986], [-73.74467955599985, 40.693412262999885], [-73.74479652999989, 40.693220371999864], [-73.74484610600003, 40.69313905699987], [-73.74487951500002, 40.69309307799993], [-73.74510485099985, 40.69278293499986], [-73.74624345599987, 40.6912794439999], [-73.74728617699986, 40.689737172999834], [-73.7475517539999, 40.68934733599984], [-73.74856993199998, 40.687803737999914], [-73.74884601399998, 40.68744748399996], [-73.74944546799985, 40.686672021999854], [-73.75008991199991, 40.68592344599988], [-73.75025615899999, 40.685613109999906], [-73.75103020599988, 40.68422286699994], [-73.75125793499984, 40.683734125999926], [-73.751542155, 40.68304604199996], [-73.75180409200001, 40.68240332399987], [-73.75225828699998, 40.6814914109999], [-73.75247059899984, 40.68118051499989], [-73.75268290999999, 40.6808696169999], [-73.75321057900005, 40.68015428799992], [-73.75323817199994, 40.68011688599991], [-73.75328689999988, 40.68005114499995], [-73.75345604299982, 40.67982295599992], [-73.75372596199996, 40.67945742799991], [-73.75399395199992, 40.679094506999895], [-73.7541785699999, 40.67884448599991], [-73.75484909699989, 40.67792883799996], [-73.75523457799996, 40.677288409999925], [-73.75549392499991, 40.67662967299988], [-73.75562236499997, 40.67625847299991], [-73.75563646999986, 40.67621517899991], [-73.75576169199995, 40.67588590899988], [-73.75648885799987, 40.67602276199989], [-73.75662976499991, 40.676116912999916], [-73.75672742400002, 40.67620876499991], [-73.75680933199992, 40.67630971499991], [-73.75687349699994, 40.67641731799989], [-73.756918817, 40.676528870999924], [-73.75789961600002, 40.67612340999989], [-73.75801266799994, 40.676077675999835], [-73.75950918099984, 40.67544720999995], [-73.76031463999993, 40.67510997099984], [-73.76114911299982, 40.67476361999996], [-73.76248600399998, 40.67419498199988], [-73.76339040599994, 40.674681117999924], [-73.76345120599991, 40.67471618299986], [-73.76352325899985, 40.67473995099995], [-73.76360070799988, 40.67474980899989], [-73.76367677599995, 40.67474552499992], [-73.76371825199992, 40.67431589199985], [-73.76380484799996, 40.67376993299995], [-73.76561573099998, 40.6747790199999], [-73.7683733069999, 40.67655024099987], [-73.76932752799988, 40.677143788999906], [-73.77030295199995, 40.67775050099993], [-73.77109570599985, 40.678243582999905], [-73.77177843899987, 40.678668299999906], [-73.77346815399997, 40.67971949999986], [-73.77469730199986, 40.680486045999906], [-73.77408130199989, 40.68076015399986], [-73.77356539499996, 40.68096392599992], [-73.77299539399996, 40.6812088249999], [-73.77232211299984, 40.68149096799993], [-73.77220275299989, 40.681610338999896], [-73.77256209599992, 40.682209496999846], [-73.77297749199987, 40.68293379299992], [-73.7732870399999, 40.683454731999866], [-73.773499153, 40.68348222699995], [-73.77418414899984, 40.68323792099991], [-73.77441975399977, 40.68365352199991], [-73.7745579039998, 40.683885621999906], [-73.77462061100005, 40.68399096999984], [-73.775012315, 40.68427557899987], [-73.77567289899983, 40.68474158899993], [-73.77632402199986, 40.6852165559999], [-73.7767991239999, 40.68596846499985], [-73.77697494099996, 40.686246713999935], [-73.77699767599997, 40.68628268899992], [-73.77784971099992, 40.68598927299989], [-73.77870794599994, 40.685700419999876], [-73.77957347199998, 40.68541503299992], [-73.77966850599998, 40.685377122999945], [-73.77976509199982, 40.68574360299993], [-73.78051569899993, 40.687036520999875], [-73.78148675699995, 40.6886971829999], [-73.78234452799995, 40.68840743799994], [-73.7832662499999, 40.68999429299992], [-73.7824271229998, 40.69034436899994], [-73.78160296599982, 40.69068690399989], [-73.78078155499992, 40.69102994299994], [-73.77977295099997, 40.689281169999916], [-73.77891249599985, 40.689570646999925], [-73.77796055199985, 40.689894809999956], [-73.77789337399993, 40.68991768999992], [-73.779198385, 40.6916505679999], [-73.7802372379999, 40.692972334999894], [-73.78018646099991, 40.6930152289999], [-73.77944081299977, 40.69364510899989], [-73.77863399799988, 40.694013787999914], [-73.77783593399984, 40.69432133499986], [-73.77725734299997, 40.69506404199992], [-73.77703882099993, 40.69536433699991], [-73.77659775499988, 40.695969113999865], [-73.77618932299991, 40.69657434099989], [-73.77608749199995, 40.6967252309999], [-73.7757628779999, 40.697182862999924], [-73.77567626099993, 40.69730502899993], [-73.77532842099995, 40.69778566699989], [-73.77513633799991, 40.69805046599988], [-73.774892245, 40.698387447999906], [-73.7745937749998, 40.69879949799986], [-73.77446186199991, 40.69899514599993], [-73.77395659499993, 40.69980578099988], [-73.77359394899996, 40.69999438299987], [-73.7734383289999, 40.70007532199988], [-73.77318719099986, 40.70018646399989], [-73.77293506599993, 40.70029491999987], [-73.77274622199978, 40.70037864699992], [-73.77230768699997, 40.70057150699989], [-73.77272432599987, 40.70126221599989], [-73.77312771699984, 40.70194727699994], [-73.7733174479998, 40.70226169599991], [-73.7735339129999, 40.702620427999854], [-73.77394353599978, 40.70331150399988], [-73.7743477909998, 40.7039863969999], [-73.77507462299982, 40.70523998399989], [-73.77559218799982, 40.70532930899986], [-73.77561355999983, 40.70551157799985], [-73.77561486299999, 40.705697221999834], [-73.77559530099992, 40.705884318999956], [-73.77555451099992, 40.706070835999924], [-73.77549258399985, 40.7062546719999], [-73.77541010599997, 40.70643377599989], [-73.77530808899986, 40.70660617899991], [-73.77518799499988, 40.706770116999955], [-73.77505158100001, 40.70692403599988], [-73.7749009329999, 40.707066661999875], [-73.77461091499995, 40.70723572499985], [-73.77430843599988, 40.70739366799994], [-73.77399462499987, 40.707539717999914], [-73.77373764399991, 40.707645606999904], [-73.77367069099986, 40.70767319499985], [-73.77333799499985, 40.70779351599987], [-73.77299790100001, 40.70790021499987], [-73.77278262899993, 40.707957893999925], [-73.77265189, 40.707992923999875], [-73.77230141699995, 40.70807142299988], [-73.76918606499989, 40.70933634199989], [-73.76844546499984, 40.709637026999886], [-73.76825969099991, 40.709526366999945], [-73.76789085799992, 40.70840640299988], [-73.767762698, 40.70799573499992], [-73.7675558879999, 40.707321957999895], [-73.76734431899995, 40.70663634599993], [-73.76725075899985, 40.70631255799994], [-73.76705298500002, 40.705470301999895], [-73.76701441799999, 40.7052681309999], [-73.76679155699992, 40.704008526999885], [-73.76665413499987, 40.70333761699992], [-73.766572825, 40.70275162099996], [-73.766369252, 40.70256339699992]]]}}, {\"id\": \"205\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 206, \"Shape_Leng\": 0.212756793386, \"Shape_Area\": 0.000944392507762, \"zone\": \"Saint George/New Brighton\", \"LocationID\": 206, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08221272914942, 40.64828016229008], [-74.08212394387884, 40.64827811218601], [-74.08204629493248, 40.64830775311508], [-74.08186603545586, 40.64837339135716], [-74.08165246388559, 40.648424260217816], [-74.08142228203803, 40.64850472594945], [-74.08135603833722, 40.648490943530966], [-74.08095598247468, 40.648348676459456], [-74.08072838762372, 40.648274873846276], [-74.0805182378835, 40.6482455147327], [-74.07980996428698, 40.64838331298796], [-74.07923252724933, 40.64818632548863], [-74.07899546333263, 40.64814255442241], [-74.0770865800269, 40.647199861267865], [-74.07699341088887, 40.647205851839054], [-74.07682895680941, 40.64708052243883], [-74.07650657152854, 40.646968818183346], [-74.07620922502595, 40.64672388272722], [-74.0759922276155, 40.646552023758986], [-74.07530767022044, 40.64591302206679], [-74.07445282563698, 40.645067488723235], [-74.07407812091208, 40.64508432891062], [-74.07395839976466, 40.64519320544552], [-74.07387258053362, 40.645147816347084], [-74.07359107919267, 40.644998928043016], [-74.07349851367614, 40.645083373475124], [-74.07375861295428, 40.64531335227553], [-74.07380088322195, 40.645285707094914], [-74.07382911232169, 40.645311772142705], [-74.07377880342949, 40.64535629846428], [-74.07383419382343, 40.64540738068856], [-74.0738565368472, 40.6454248160996], [-74.07382143191991, 40.6454434833772], [-74.0733381385699, 40.64578311616232], [-74.07324539464845, 40.64570492526951], [-74.07348694129136, 40.64554980774531], [-74.07341838754051, 40.645488479178326], [-74.07369011806544, 40.645305725059956], [-74.07346429816413, 40.645107943609474], [-74.07333328763886, 40.645034380734074], [-74.07334938819302, 40.64502209632997], [-74.0732385381076, 40.6449638639098], [-74.07305744965362, 40.645145027374426], [-74.07302520507207, 40.645131238320104], [-74.07320225954693, 40.64494394124359], [-74.07301240029486, 40.64483867970176], [-74.07305559881726, 40.64480791784095], [-74.07341383827794, 40.64501591847193], [-74.07342590747642, 40.64500056851527], [-74.0732727290158, 40.644916279978126], [-74.07334517133606, 40.64485025792902], [-74.07312184183515, 40.64471479946889], [-74.07292585037442, 40.64486407396003], [-74.07287964133008, 40.64484417943485], [-74.0727985034149, 40.64491012376873], [-74.0725877555387, 40.64479086317363], [-74.07270308267607, 40.64464559562752], [-74.07269226553024, 40.64463737028705], [-74.07259617125489, 40.64473547246686], [-74.07256468232941, 40.64471974885816], [-74.07261232795737, 40.64467529058511], [-74.072599349268, 40.644667064925294], [-74.07257263940097, 40.644693803403015], [-74.07252587521035, 40.6446700400171], [-74.07257551719216, 40.64461768538858], [-74.07262742408486, 40.644643997634525], [-74.07261662439397, 40.64465058915858], [-74.07262311218182, 40.6446538788378], [-74.07266630771204, 40.644620920659506], [-74.07255383884463, 40.64456007230925], [-74.07254303492037, 40.6445633712257], [-74.07253222025724, 40.64455679217022], [-74.07258623043283, 40.64453041424987], [-74.07259920774122, 40.644536992437715], [-74.07258408702005, 40.64454688046275], [-74.07272467145069, 40.64462088376608], [-74.07279042215042, 40.64455201831282], [-74.07274586000851, 40.64452403106858], [-74.07278260612932, 40.64449184001752], [-74.07269828799016, 40.64444127346728], [-74.07233100894283, 40.64459987318499], [-74.07170671648227, 40.64502264983108], [-74.07171641184448, 40.645031865721236], [-74.07168737376726, 40.64505032820053], [-74.07165829759784, 40.64503374643502], [-74.07168248051121, 40.64500421939304], [-74.07168733413398, 40.645013439476294], [-74.07230919046846, 40.644577754057316], [-74.07225588436278, 40.64454643110188], [-74.07216148380606, 40.644581534806505], [-74.07201843213228, 40.644411932940834], [-74.07211281812974, 40.644363917444466], [-74.07209826186593, 40.64433810424378], [-74.07223218396409, 40.64428826979357], [-74.07229190281811, 40.64426604782037], [-74.07218488601674, 40.644110465395165], [-74.07197701849, 40.64418139905795], [-74.07196247757301, 40.644170341733236], [-74.0718608087088, 40.64420176177014], [-74.07175648087771, 40.64401184549864], [-74.07187025099051, 40.643974885029756], [-74.07186296798388, 40.64395828969226], [-74.07193800055079, 40.64392688581522], [-74.07194769578626, 40.64393610218965], [-74.07205817506032, 40.64390042769858], [-74.07200080155476, 40.64378113296951], [-74.0719062401355, 40.643797109178514], [-74.07179259243398, 40.643816308578934], [-74.07177075212515, 40.6437720548353], [-74.07164969408515, 40.64378688572462], [-74.07158891657957, 40.643591315373655], [-74.07162787633769, 40.64357834116639], [-74.07159679201153, 40.64346015748861], [-74.07135612176627, 40.64349730455597], [-74.0713410941792, 40.643456225521575], [-74.07149090063898, 40.643428740585875], [-74.07147284705017, 40.643360272718176], [-74.07150206405164, 40.64335486600386], [-74.0714820243262, 40.64329262506356], [-74.07125341439202, 40.643330949683474], [-74.07125775124943, 40.643344119757], [-74.0709832823077, 40.64338874400836], [-74.07097244806427, 40.64336405315123], [-74.07095083519633, 40.64336571396242], [-74.07093784471476, 40.64334596384062], [-74.07096809463135, 40.64333441989067], [-74.07095942168603, 40.643309727028274], [-74.07125765981509, 40.64325850172703], [-74.07127283144267, 40.643296361122495], [-74.07144331706321, 40.6432705613306], [-74.07141659716521, 40.64319310959575], [-74.07130487290098, 40.64321200375994], [-74.07130787990505, 40.643221132697754], [-74.0711430928901, 40.643250909742], [-74.07113105720924, 40.64320526367827], [-74.07150263147396, 40.64312930419292], [-74.07147777322605, 40.64280829377494], [-74.07164540901456, 40.642800359680585], [-74.07164892198989, 40.64267993167963], [-74.07170286121513, 40.6426798980603], [-74.07170351967257, 40.64279633528003], [-74.0721843496904, 40.64271409328902], [-74.07210952444638, 40.64246321679278], [-74.07037430990586, 40.642691403053135], [-74.07032923715386, 40.64252612813239], [-74.07323829183592, 40.64218473608239], [-74.07314400586021, 40.641852974480116], [-74.07024658698661, 40.64220697429687], [-74.07019399559213, 40.6420075028053], [-74.07308689819374, 40.64163681496966], [-74.07289005609624, 40.64092267287602], [-74.07196280961668, 40.64099557976069], [-74.07198176865172, 40.641012401156196], [-74.07180500367038, 40.64102934375624], [-74.07181141822922, 40.641123124201854], [-74.07001849148746, 40.64127813223749], [-74.06998354653952, 40.64106894311676], [-74.07177648067534, 40.64092595946847], [-74.07177665732668, 40.6409203595471], [-74.07177804354045, 40.64091485843836], [-74.0717806013794, 40.64090960669362], [-74.07178426084167, 40.64090474803977], [-74.07178892177699, 40.64090041544562], [-74.071794456627, 40.64089672748326], [-74.07180071391738, 40.6408937850827], [-74.07180752240184, 40.6408916687699], [-74.0718146957496, 40.64089043646289], [-74.07182203764496, 40.64089012188676], [-74.07182934715873, 40.64089073365054], [-74.0718364242488, 40.640892255011934], [-74.07183643460411, 40.64090187382061], [-74.07285041194942, 40.64080368844508], [-74.07277690011115, 40.64052896314733], [-74.07307725837738, 40.64051331179637], [-74.07309035578784, 40.638618999685846], [-74.07303265958848, 40.638590610271926], [-74.07292933384748, 40.63859264983825], [-74.07292601136344, 40.638441154560006], [-74.07301618029149, 40.638426633932355], [-74.07301444562252, 40.6384093846214], [-74.07293199871835, 40.63841448843019], [-74.07292278393903, 40.63837622975343], [-74.07287265506746, 40.63837998786964], [-74.07279336887404, 40.63838593056913], [-74.07272076458273, 40.63838597650762], [-74.07267024642057, 40.6383763896267], [-74.07261338131848, 40.63833554523842], [-74.07258809054142, 40.63830189417055], [-74.07258173009109, 40.63825861400699], [-74.0726093386763, 40.63822689532102], [-74.0723885295029, 40.638245265996716], [-74.07237941129377, 40.63816272607411], [-74.0726063279754, 40.63813630834696], [-74.07258133886755, 40.63794187429597], [-74.07329742299999, 40.63790350799987], [-74.07373948799989, 40.637861233999885], [-74.07469451399994, 40.637805179999916], [-74.07486917499988, 40.6377928099999], [-74.07493115799993, 40.63778151099995], [-74.07496304100002, 40.637773349999854], [-74.075060529, 40.637748399999936], [-74.07513243599989, 40.63772719999991], [-74.07530550099996, 40.637729959999895], [-74.07601776799994, 40.637748104999915], [-74.0768399779999, 40.6377502599999], [-74.07698362199989, 40.63779569399989], [-74.0778990219999, 40.63809465199991], [-74.078131634, 40.63817145299989], [-74.07818273599996, 40.63820992299987], [-74.07823997999992, 40.63824299499991], [-74.07830238999998, 40.638270105999865], [-74.0783689039999, 40.638290794999854], [-74.07843838799988, 40.63830470799997], [-74.07850965999987, 40.63831160899986], [-74.07858150399997, 40.638311379999884], [-74.07865269699992, 40.63830402499987], [-74.078722026, 40.63828966999992], [-74.07878831099995, 40.63826855899991], [-74.08055168499997, 40.63731999999989], [-74.08177940499988, 40.63662301399993], [-74.08302091199991, 40.63602507099989], [-74.08373397299994, 40.63573714399991], [-74.08513463099995, 40.63635959899995], [-74.08568983299989, 40.6366072799999], [-74.08587882899995, 40.63669158999991], [-74.0869059279999, 40.635857242999904], [-74.08759775199998, 40.636262023999926], [-74.08870582299987, 40.63684520599986], [-74.08891735099989, 40.636528383999874], [-74.08910236299987, 40.63624702699983], [-74.08915919799995, 40.63616632399986], [-74.08922167799987, 40.63608807999989], [-74.089289621, 40.63601252499995], [-74.08936282699987, 40.63593987999993], [-74.08944108199995, 40.63587035599987], [-74.08952415699996, 40.635804156999875], [-74.08961180999991, 40.63574147799996], [-74.08970378299989, 40.63568250099992], [-74.09027738599993, 40.63549902999994], [-74.09090698799996, 40.635485728999875], [-74.09182025899999, 40.63555529799987], [-74.09242513499994, 40.63558181299992], [-74.09338942299995, 40.63659955499988], [-74.09362634099998, 40.63728369499989], [-74.09386276399991, 40.63802491699991], [-74.09409312599998, 40.639266913999975], [-74.09570924799992, 40.63909864099989], [-74.09637906199993, 40.63903536999992], [-74.09686242099997, 40.63900747099991], [-74.09767290799995, 40.63896067899989], [-74.09787969199995, 40.64035805499988], [-74.0992883379999, 40.64020472299996], [-74.09990409599993, 40.64016295499994], [-74.10042458199996, 40.6400410299999], [-74.10129653399997, 40.639827157999925], [-74.10216582799997, 40.63961813399992], [-74.10305261099988, 40.639401245999856], [-74.10393828599992, 40.63919020399988], [-74.10480990899997, 40.638976700999876], [-74.10542286599998, 40.63889896699986], [-74.10690506699991, 40.63872593199991], [-74.10696781199987, 40.63871878699992], [-74.10702926799993, 40.63870678799987], [-74.10708878899997, 40.638690059999874], [-74.10714575, 40.63866877999992], [-74.10719955199997, 40.63864316999993], [-74.10724963099997, 40.63861349899986], [-74.10729545999992, 40.63858007999987], [-74.1073365579999, 40.63854326299991], [-74.10746323199996, 40.638438487999956], [-74.10822759399989, 40.638314039999905], [-74.10868871399997, 40.63823996099989], [-74.10887095399988, 40.638220561999894], [-74.1088711309999, 40.6382205439999], [-74.10887129699995, 40.638220507999904], [-74.10905105199996, 40.638184892999895], [-74.10905120499997, 40.638184865999904], [-74.10905134699988, 40.63818482099989], [-74.10922508199995, 40.638133370999896], [-74.10922521199993, 40.638133334999935], [-74.109225354, 40.63813328099991], [-74.10938954799997, 40.63806700899991], [-74.10938961799994, 40.63806698199992], [-74.10938968899991, 40.638066945999896], [-74.10954135499992, 40.637987429999896], [-74.10974833299994, 40.63786583299993], [-74.11080857499996, 40.637746199999874], [-74.11174806399994, 40.63764295499989], [-74.11197916199993, 40.63761817599988], [-74.11279390199998, 40.637530809999916], [-74.1137364029999, 40.63742912299987], [-74.1149955969999, 40.63729222599993], [-74.11519147399999, 40.63834768099987], [-74.11588274199994, 40.63826439099994], [-74.11641377299995, 40.63820248399986], [-74.11709261399987, 40.63812694699992], [-74.11765722499996, 40.638062466999905], [-74.11754902299995, 40.63695363199987], [-74.11753352399988, 40.63674776299993], [-74.11752704599989, 40.63666172399995], [-74.11751731899994, 40.6365325329999], [-74.11751136899993, 40.636441801999894], [-74.1175003539999, 40.63630719399988], [-74.11749409699998, 40.63622409099987], [-74.11748576400001, 40.6361134059999], [-74.11747983199996, 40.63602690599986], [-74.11747417299995, 40.635943517999884], [-74.11746820599997, 40.63588019599996], [-74.1174598659999, 40.63576940499993], [-74.1173836129999, 40.63539539899992], [-74.1171994709999, 40.634568097999924], [-74.11707785799997, 40.634007107999906], [-74.11704470999996, 40.633816609999904], [-74.11695838499996, 40.633351407999896], [-74.11683619599992, 40.63267481799986], [-74.11666642599987, 40.6317101019999], [-74.11650496399994, 40.63075571399985], [-74.11634635499996, 40.629835002999876], [-74.11618948699994, 40.62894119699992], [-74.11614321199995, 40.62870226399988], [-74.11595162399986, 40.628230133999864], [-74.11588524999998, 40.62808893699991], [-74.11560933499997, 40.62754898299993], [-74.11544777599991, 40.62722061699989], [-74.115275528, 40.626889055999875], [-74.11526026599996, 40.62685967799992], [-74.11524316599996, 40.62682656499992], [-74.11503453399992, 40.62642257599991], [-74.11475499299996, 40.625889287999904], [-74.11473772799998, 40.6258563459999], [-74.1147144859999, 40.625811685999935], [-74.11407428699995, 40.62458176999998], [-74.11393395699994, 40.62433020399988], [-74.11358478199992, 40.62369716899985], [-74.1132624959999, 40.62305442499995], [-74.11541921799997, 40.62391096299995], [-74.11627412799992, 40.62423942099988], [-74.11701375799994, 40.62452357699987], [-74.11724781899987, 40.62459150899986], [-74.11747606599991, 40.62467350199995], [-74.11769631299995, 40.62476906499987], [-74.11790654899988, 40.624877460999876], [-74.11810485699996, 40.624997729999926], [-74.11828960499999, 40.62512868299993], [-74.11845939699997, 40.62526898899994], [-74.11916467799988, 40.62582382799994], [-74.11967435899994, 40.62619142199987], [-74.12012887999998, 40.62653615299989], [-74.12056862499992, 40.6269886999999], [-74.12125677899994, 40.62782274999989], [-74.12216225499999, 40.627476891999876], [-74.12270654899999, 40.627407395999896], [-74.12365042899994, 40.627286876999904], [-74.12424800199987, 40.62721164199989], [-74.12513287199992, 40.62710260099988], [-74.12666303799995, 40.626889617999915], [-74.12826227099993, 40.6266735039999], [-74.12920250099991, 40.62654060599987], [-74.13007946899988, 40.62641859599995], [-74.13069767299996, 40.626340956999854], [-74.1313098529999, 40.62632834299998], [-74.13099290699998, 40.627374110999874], [-74.13069219499997, 40.62833690199986], [-74.13056374599996, 40.628707729999896], [-74.13033008299996, 40.6293843399999], [-74.13024788999991, 40.62960913099992], [-74.13014141499997, 40.62996520899991], [-74.13047428999988, 40.63088550999992], [-74.13060220399996, 40.63135782699987], [-74.13046976899993, 40.63167632099987], [-74.13016617699994, 40.63237102799989], [-74.1300590599999, 40.632642026999896], [-74.129932476, 40.632908786999906], [-74.12978688499999, 40.63317008399987], [-74.1296229589999, 40.63342476399984], [-74.12944148699994, 40.633671720999885], [-74.12900054099997, 40.63433359399993], [-74.12893129399997, 40.63457748699988], [-74.128881913, 40.63482489799992], [-74.12885281899997, 40.6350744189999], [-74.128844165, 40.63532458499988], [-74.12885588799993, 40.63557396099989], [-74.12884672499999, 40.636436003999975], [-74.12886142599999, 40.63656824599986], [-74.12885649499997, 40.636702185999916], [-74.128831656, 40.63683527499989], [-74.12878738899998, 40.636964953999936], [-74.12872489299997, 40.63708883699995], [-74.12867047299994, 40.63734826499993], [-74.12784669199989, 40.63727742999988], [-74.1268887269999, 40.637210087999925], [-74.12707923599991, 40.6381224369999], [-74.12705476399996, 40.638197599999934], [-74.12700759399989, 40.63833452499991], [-74.1269897449999, 40.63837057999993], [-74.1269820749999, 40.63839636999991], [-74.12697697999995, 40.63843172199987], [-74.12698007099992, 40.638609711999905], [-74.12699049099994, 40.63910816399989], [-74.12702091299994, 40.63927376299988], [-74.12702714583776, 40.6393157849717], [-74.12706108014973, 40.639513642158754], [-74.1270762008027, 40.63957569431474], [-74.12709862022669, 40.63962602221111], [-74.12710264794042, 40.63968075296103], [-74.12710164386345, 40.63971966695505], [-74.12710456677688, 40.63976646139726], [-74.12712443004504, 40.639863183880756], [-74.12709776289094, 40.63997981165702], [-74.12708754066087, 40.64004284927891], [-74.12708156085438, 40.6401610290877], [-74.12705362202625, 40.64024173781949], [-74.12651529296663, 40.640371752940354], [-74.12606982881191, 40.64025447334623], [-74.1259954246122, 40.640231242814394], [-74.12593026603197, 40.64021089814807], [-74.12580812521344, 40.64017984793913], [-74.12569811616551, 40.640155933093226], [-74.12557833584698, 40.64014732822179], [-74.12546249896081, 40.64015483952424], [-74.12533094012922, 40.64018202506978], [-74.12520491374713, 40.64019329877803], [-74.12487769898755, 40.64024024996559], [-74.12469728890524, 40.64023935103757], [-74.1246151276807, 40.64022237751123], [-74.12452923299043, 40.64020540789036], [-74.12448065891869, 40.640182710745435], [-74.12438357096798, 40.64016859594369], [-74.12425872953952, 40.64014708733402], [-74.12415578422493, 40.640134716820036], [-74.1239506352287, 40.6402287828927], [-74.12384245180269, 40.640271556074076], [-74.12374918070343, 40.640302937917255], [-74.12363726012842, 40.64034287119688], [-74.1235775928573, 40.64037706046953], [-74.12354404655959, 40.64040553481481], [-74.12354039801825, 40.640451038990626], [-74.12354793753005, 40.640490843191294], [-74.12351439698959, 40.64052216109504], [-74.12340726999689, 40.640588957305845], [-74.12318343872492, 40.640673189868664], [-74.12285630577614, 40.64082050817312], [-74.12277976988646, 40.64077054990097], [-74.12265174080149, 40.64062676581546], [-74.12260363650799, 40.64059707395136], [-74.12257280489571, 40.64062234004702], [-74.12274476463523, 40.64084347425283], [-74.12279688379691, 40.640900519904164], [-74.12287959621975, 40.64095636778684], [-74.12290724899286, 40.641019266308525], [-74.12290581795914, 40.64107287288534], [-74.12285698406907, 40.64113818393525], [-74.12286625478826, 40.641188283220224], [-74.12285868322375, 40.6412302431834], [-74.12282672976266, 40.641291594234964], [-74.1228256540515, 40.641293875757476], [-74.12282049121923, 40.641299078217855], [-74.12279544107928, 40.64132432597279], [-74.12277438475277, 40.64133737539297], [-74.12272114424832, 40.64134762377306], [-74.12269555893829, 40.641353026836214], [-74.12195850081171, 40.64144698304061], [-74.12194129710181, 40.64144190806065], [-74.12190205880334, 40.64140120212528], [-74.1218414704778, 40.641405894387034], [-74.12180661091023, 40.64141215903768], [-74.12175764698422, 40.64142342422718], [-74.12172375309429, 40.641428664221905], [-74.12144702079314, 40.64150237930765], [-74.12138690694671, 40.641485332526244], [-74.12137332919554, 40.64147647512165], [-74.12131194814832, 40.64145253116248], [-74.12128074797168, 40.64143249599354], [-74.12121470277087, 40.64139683999368], [-74.12118039503834, 40.64138344518971], [-74.12071317594332, 40.64104578209544], [-74.12064634499572, 40.64104409562102], [-74.1205742902843, 40.6410563284083], [-74.12052811811581, 40.64106278260175], [-74.12016783688034, 40.64113899598833], [-74.11993480088911, 40.64119760438391], [-74.11997111213574, 40.641314814562264], [-74.11960716747862, 40.64138030935416], [-74.11962565609885, 40.641442628676074], [-74.1199569987576, 40.642209453429054], [-74.11985443942083, 40.642235613904816], [-74.1195298610575, 40.64146991432194], [-74.11946561296826, 40.64147887417194], [-74.11944988177683, 40.64143464839656], [-74.11942631292477, 40.641423524687255], [-74.11932302547099, 40.64143919866271], [-74.11909720493722, 40.64147346652342], [-74.11948087155078, 40.64235510170977], [-74.11940029532462, 40.64237937853275], [-74.11900348200304, 40.64146461451002], [-74.11861635236262, 40.641530365028935], [-74.11860486153472, 40.64156325117877], [-74.11868834100237, 40.64179764355324], [-74.1186517159865, 40.641808846624336], [-74.1188750592626, 40.64238736597404], [-74.1188994831258, 40.64238361874216], [-74.11896077402346, 40.64250451599316], [-74.11883624616948, 40.64254186406039], [-74.11864479718022, 40.642039609106014], [-74.11858364618543, 40.641996871650264], [-74.11842117592663, 40.64157604952014], [-74.11840185083544, 40.64152980054283], [-74.11815536520912, 40.64159368426173], [-74.11820831549385, 40.64175023537594], [-74.1180076709366, 40.64180549753333], [-74.11815635743852, 40.642131298298736], [-74.11804891156352, 40.64215746101188], [-74.11791728683136, 40.64182587695215], [-74.11785534854104, 40.64183757949359], [-74.11780150303653, 40.641809469261624], [-74.11773478385697, 40.64179669853558], [-74.1176284161481, 40.641621582901344], [-74.11745689604214, 40.641644801773694], [-74.11740303374847, 40.64163333969023], [-74.1173492642659, 40.6416203672168], [-74.1173028282626, 40.64160924948545], [-74.11728814480372, 40.64159437720457], [-74.1171709328468, 40.64162427104175], [-74.11711235699552, 40.64165596713886], [-74.117134061225, 40.641696163949156], [-74.11722122301936, 40.64187096523019], [-74.11750702523284, 40.64242040843011], [-74.11741912492029, 40.642448412156504], [-74.11713706403349, 40.64189806463106], [-74.11672187663888, 40.6420317560966], [-74.11673703407642, 40.64205948561654], [-74.11688396076235, 40.64236707592774], [-74.1170334812623, 40.64264978574938], [-74.1172247008255, 40.64302735996259], [-74.1171490052741, 40.64304976865223], [-74.11705580706132, 40.6428433002025], [-74.11695775345594, 40.642653584774536], [-74.11684258188978, 40.642454581782964], [-74.11673967258692, 40.64228162026454], [-74.11666607646228, 40.64210676633395], [-74.11660991949574, 40.6421254323637], [-74.11659975924091, 40.64210998296402], [-74.1163000008274, 40.64220266510229], [-74.11633166977425, 40.64227086677678], [-74.11630481365927, 40.6422801988119], [-74.1167534287466, 40.64316182670229], [-74.11667041382688, 40.64318982489282], [-74.11624629709846, 40.64234725163883], [-74.11621943792055, 40.6423547232525], [-74.1161572274712, 40.64224490178976], [-74.11614208635716, 40.642216454150336], [-74.11599693311375, 40.64226229373323], [-74.1157191704229, 40.64235181921401], [-74.11622691905411, 40.643307105862036], [-74.11614843005724, 40.64333111259185], [-74.11564823433872, 40.642374683334474], [-74.11535082182925, 40.64247054015812], [-74.11511714752054, 40.64278499267784], [-74.11480735067566, 40.64297696808479], [-74.11478076428318, 40.642957615928985], [-74.11429538722346, 40.64327914734321], [-74.11379666086005, 40.64360951580788], [-74.11404053146022, 40.64404250211974], [-74.11395094068477, 40.64413659167517], [-74.11288806510967, 40.64453244463756], [-74.1128296089354, 40.64446242868866], [-74.11379152893625, 40.64407522183085], [-74.11380493647306, 40.644042735507895], [-74.11385205211577, 40.64404098058403], [-74.11389010824344, 40.643989669126725], [-74.11381590334797, 40.643895741340636], [-74.11376653215497, 40.643890663485635], [-74.11369030102539, 40.64392321192986], [-74.11363867687, 40.643913008367164], [-74.11354450863914, 40.643954120067306], [-74.11333023192623, 40.64372349758636], [-74.1123037090662, 40.64431880043215], [-74.11180799136828, 40.6446106744768], [-74.11190333685228, 40.64470602724296], [-74.11116129469252, 40.64509642318122], [-74.111116336666, 40.645046902323045], [-74.11176646650024, 40.64470616034501], [-74.11172529995703, 40.644659361726134], [-74.11167635063406, 40.64468818279128], [-74.11149571146441, 40.64477816472747], [-74.11101110999861, 40.645053095282016], [-74.1097393256449, 40.64546372818479], [-74.1095380858384, 40.64549583253676], [-74.10886489895957, 40.64557912880393], [-74.10869553743693, 40.64557472334869], [-74.1082362049677, 40.645562773203665], [-74.1077967414638, 40.645561984289195], [-74.1068034270611, 40.64556019594091], [-74.10472221599757, 40.645563657399386], [-74.10464577782204, 40.645630470315766], [-74.10439613555202, 40.64566994107998], [-74.10434058744978, 40.64572116640804], [-74.1041235630174, 40.64574043220019], [-74.1040550911333, 40.64566616481553], [-74.10402237217032, 40.64563805214804], [-74.10398644791051, 40.64562270284431], [-74.10391463250335, 40.64561422213156], [-74.10383609558916, 40.64561258528449], [-74.10376428818483, 40.6456092311764], [-74.10372839854041, 40.64561609971925], [-74.10369926142748, 40.645636634996855], [-74.10365048802035, 40.645654813250864], [-74.10307296677558, 40.645638149422595], [-74.102316793374, 40.645542600180114], [-74.10168264441663, 40.64540673298673], [-74.10167855232588, 40.64541576035736], [-74.1016648034949, 40.6454153833438], [-74.10162941761631, 40.64548895894026], [-74.10170826781193, 40.6455186317306], [-74.10169906586987, 40.64553648533391], [-74.10162518955083, 40.64550951288895], [-74.10161828954732, 40.64552653887154], [-74.10157833974772, 40.645514421316165], [-74.10164420314467, 40.64540783027231], [-74.10163782873113, 40.64540559839609], [-74.10164266441156, 40.6453965489262], [-74.10060542005353, 40.64513233331709], [-74.10010007356692, 40.64505941609462], [-74.09971112877103, 40.645055710851395], [-74.09880066885808, 40.64504703431087], [-74.09849981625507, 40.64537036195378], [-74.09604555281575, 40.64537274744267], [-74.09468500921257, 40.645555717734815], [-74.09362656930101, 40.64583865879389], [-74.09303960268036, 40.64596199782461], [-74.09298828187653, 40.6459727807525], [-74.09300507296297, 40.64607631481677], [-74.0913569794201, 40.64670072410049], [-74.09096024080317, 40.64681894305663], [-74.08992334047647, 40.64717613260689], [-74.08940351561799, 40.64730577665889], [-74.08845521748377, 40.6478434710791], [-74.08836043489221, 40.64790790532765], [-74.08827444370826, 40.64792065007824], [-74.08824112474028, 40.64790165648607], [-74.08793804157592, 40.64808009375914], [-74.0876744268149, 40.64823529361473], [-74.08634467630301, 40.648699609386675], [-74.08570754378212, 40.648880553370454], [-74.08522698411666, 40.648925551773466], [-74.08484403215712, 40.64889114733939], [-74.08482973925649, 40.64888665756863], [-74.08476074851815, 40.6488747850029], [-74.08469394761234, 40.648857080111114], [-74.08463018353473, 40.64883376738129], [-74.08457026477315, 40.6488051424052], [-74.08446817789952, 40.64875390179318], [-74.08436179703365, 40.648708007510365], [-74.08425160405804, 40.64866766744467], [-74.08413809812035, 40.64863306432455], [-74.08402179337246, 40.64860435489161], [-74.0839032166416, 40.6485816691905], [-74.08256585807113, 40.64831581935348], [-74.08221272914942, 40.64828016229008]]]}}, {\"id\": \"206\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 207, \"Shape_Leng\": 0.0283756831492, \"Shape_Area\": 4.47230848441e-05, \"zone\": \"Saint Michaels Cemetery/Woodside\", \"LocationID\": 207, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90137640299986, 40.75964203799989], [-73.90147053099994, 40.759633049999884], [-73.90175373499979, 40.76077547499991], [-73.90225241299993, 40.762643825999895], [-73.902767937, 40.76387945699992], [-73.90299205599985, 40.76441183899989], [-73.90331045299985, 40.76516814899989], [-73.90356915599997, 40.765781788999924], [-73.90351227799994, 40.76584945999989], [-73.90341888199997, 40.765957306999944], [-73.90255919500005, 40.766940786999896], [-73.90217951699996, 40.76731584199997], [-73.90202516599992, 40.767416783999884], [-73.90065875099992, 40.7672114339999], [-73.89595958499986, 40.76643103399986], [-73.89432371999989, 40.76614647099994], [-73.89349605899986, 40.765950929999924], [-73.89377741500005, 40.765867198999956], [-73.89384201999984, 40.765842335999885], [-73.89390458899999, 40.765818256999836], [-73.89403763099985, 40.76575891999986], [-73.8941491849999, 40.765694356999866], [-73.89428031700004, 40.76560677599987], [-73.89437624799997, 40.7655265849999], [-73.89446044299983, 40.76544639099992], [-73.89453879999992, 40.7653572669999], [-73.8946171639999, 40.7652562569999], [-73.89470543599994, 40.76506903299987], [-73.89477219899987, 40.76488921799994], [-73.89483535100004, 40.764655315999974], [-73.89499458899998, 40.76445315399994], [-73.89505758700001, 40.76434653799987], [-73.89506657699998, 40.76433132799987], [-73.8951683709999, 40.7641973619999], [-73.89527479399989, 40.764085598999905], [-73.89542447799998, 40.76394724499989], [-73.89557704399981, 40.76382796799991], [-73.89585991899992, 40.763628473999916], [-73.89598406499984, 40.76354310999991], [-73.89608492499991, 40.763473754999914], [-73.89624744699991, 40.76333274699986], [-73.89641138399999, 40.7631500829999], [-73.8966818839999, 40.762777588999874], [-73.89700714599998, 40.76235119299988], [-73.89720062399994, 40.7620732819999], [-73.89723108699998, 40.76202952499994], [-73.89735543399995, 40.76185091299988], [-73.897895281, 40.761044505999905], [-73.89791145199985, 40.76102034899989], [-73.89825240299986, 40.76051448699995], [-73.89866137899992, 40.75992101699997], [-73.89877250799998, 40.75991557699993], [-73.89886718099987, 40.759904206999835], [-73.89901804799997, 40.7598912129999], [-73.90127448199993, 40.7596514789999], [-73.90137640299986, 40.75964203799989]]]}}, {\"id\": \"207\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 208, \"Shape_Leng\": 0.214721009581, \"Shape_Area\": 0.00202032598969, \"zone\": \"Schuylerville/Edgewater Park\", \"LocationID\": 208, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.825986869, 40.84067575199988], [-73.82596550399984, 40.840326137999924], [-73.82596226799994, 40.83997641699986], [-73.82590798500001, 40.83874481799989], [-73.8258866379999, 40.8381214889999], [-73.8258498599999, 40.83706086699989], [-73.82580066799997, 40.83685202199996], [-73.82573525099993, 40.83664539499994], [-73.82565389199993, 40.836442047999874], [-73.82555705999988, 40.836243043999914], [-73.82549108799998, 40.83610415299995], [-73.82544296399993, 40.83595917099997], [-73.82541397499979, 40.835810148999876], [-73.82540482599977, 40.83565927499994], [-73.82541568999991, 40.83550885499988], [-73.82544607399977, 40.83536112899984], [-73.8254949649998, 40.83521822399992], [-73.82560350399983, 40.83503014699991], [-73.82573143499985, 40.8348481889999], [-73.8258777559998, 40.83467404699993], [-73.82604118599988, 40.83450929499991], [-73.82622087800003, 40.834354707999864], [-73.82612415599984, 40.834214164999864], [-73.82601252600003, 40.83402077799985], [-73.82515883900005, 40.83461447199994], [-73.82489104199995, 40.83422775199989], [-73.82463346500003, 40.833855807999925], [-73.82379383199985, 40.8343001469999], [-73.82368336699984, 40.83431888899992], [-73.8232749899999, 40.8343881879999], [-73.82319485499993, 40.83440178799987], [-73.82309210499989, 40.834421378999934], [-73.82265928699988, 40.834469871999886], [-73.822580515, 40.83447870199988], [-73.82223130099996, 40.8345560129999], [-73.82150930799996, 40.83473357299988], [-73.82062699699992, 40.83492251499993], [-73.81977578099998, 40.835132839999915], [-73.81890004099992, 40.83534343599992], [-73.8180458349999, 40.8355423629999], [-73.81720434599991, 40.83574058099987], [-73.81606068217324, 40.83603019622843], [-73.81598036912192, 40.835996800713865], [-73.81592410268253, 40.835953886206994], [-73.81588948558863, 40.83592418251605], [-73.81585050609584, 40.83590435561084], [-73.81578540086508, 40.835920722120726], [-73.81571163104587, 40.83593378105565], [-73.81565956542137, 40.83594028574039], [-73.81562480304432, 40.83596329021814], [-73.81556838162061, 40.83597637629091], [-73.81552500922504, 40.83597630692601], [-73.81550338608844, 40.83595321396952], [-73.81550346830674, 40.83592356566442], [-73.81554257777411, 40.83589727565455], [-73.81561200118198, 40.83588750290262], [-73.81567705144427, 40.83589090107792], [-73.81572047919107, 40.835871205799], [-73.81579426718272, 40.83585155782468], [-73.81590709059596, 40.83583197328933], [-73.81599390874709, 40.83580575876495], [-73.81605249592852, 40.83577807787728], [-73.81606339574223, 40.835772927473634], [-73.81607658058105, 40.835710359104645], [-73.81607673539797, 40.83565435717539], [-73.81607256205915, 40.83559505531963], [-73.8160451953109, 40.83555326614345], [-73.81603801016679, 40.83554229340519], [-73.81600773070701, 40.835512597838644], [-73.81598173584557, 40.83550267310644], [-73.8159253513381, 40.83550258312969], [-73.8158776139851, 40.83551239018991], [-73.81579086935717, 40.83551225167565], [-73.81572147374337, 40.8355121408174], [-73.81567804623124, 40.83553183609791], [-73.81564334731141, 40.83553178063829], [-73.81559997608781, 40.83553171130299], [-73.81557397060101, 40.8355250818311], [-73.81557836264213, 40.83550532417767], [-73.81560880514263, 40.835475724413065], [-73.81569554972249, 40.835475863064254], [-73.81575628058062, 40.835472665900184], [-73.81585173701401, 40.83545964089439], [-73.81596024072202, 40.835433461016734], [-73.81602538032784, 40.835403917006275], [-73.8160399882678, 40.835392869803144], [-73.81608188313594, 40.83536118240022], [-73.81609076772055, 40.83528543088401], [-73.81610833453743, 40.835206397840395], [-73.81610422427254, 40.83512403720764], [-73.81609138692201, 40.83506142783821], [-73.81604377742968, 40.835025115076654], [-73.81602744008109, 40.83500641676502], [-73.81600054186795, 40.83497563349135], [-73.81600068763353, 40.83492292691015], [-73.8160225096872, 40.834873548602516], [-73.81602309893778, 40.834872709073466], [-73.8160617003037, 40.834817608928276], [-73.81607054775074, 40.83475503363249], [-73.81609245230891, 40.83467600867484], [-73.81608401401107, 40.83459034626523], [-73.8160408429049, 40.83451780424187], [-73.81601058212841, 40.83448152080838], [-73.81601039495825, 40.83448148817769], [-73.81597156677697, 40.834474870668984], [-73.81591952032647, 40.83447478760986], [-73.81583274903504, 40.834484531171], [-73.81577201908263, 40.834487728375876], [-73.81571124482586, 40.834507395960735], [-73.8156808845104, 40.83450734744412], [-73.81566358139244, 40.834490849306995], [-73.81566361790196, 40.8344776720744], [-73.81570270666533, 40.834457969193664], [-73.81577214678626, 40.83444160965314], [-73.81588930522146, 40.834422030886216], [-73.81599347092263, 40.83439584455293], [-73.81600703303408, 40.83437790641613], [-73.81602829523239, 40.834349781181], [-73.81602840454025, 40.83431025082213], [-73.81603732548106, 40.83422132189588], [-73.81603747121723, 40.83416861480526], [-73.81602034962135, 40.834086233902646], [-73.81601183016511, 40.83403021929284], [-73.8160119668146, 40.83398080521078], [-73.81599215149129, 40.83391958016274], [-73.8158948330437, 40.83393979513451], [-73.81588801171014, 40.833925191540615], [-73.81598684286708, 40.833901590474646], [-73.81593920982324, 40.83367686641552], [-73.81589473438342, 40.83360640172238], [-73.81587121741998, 40.8335691424606], [-73.81584739072106, 40.833534785057424], [-73.81583040473905, 40.833501403513104], [-73.81581094367175, 40.83348099075928], [-73.81578412892186, 40.833473534981586], [-73.81575969200951, 40.833488320170616], [-73.8157328297484, 40.833497542353456], [-73.81570846020288, 40.83348823831639], [-73.8156987722173, 40.83346228086111], [-73.8157110123089, 40.833447476698666], [-73.81574031307859, 40.83343825858438], [-73.81577942432305, 40.833410526271784], [-73.81580394394058, 40.83336609443652], [-73.81580595261715, 40.83335557175826], [-73.81581632050575, 40.83330125985278], [-73.81581405126511, 40.8332401071805], [-73.81580692689249, 40.83316968282723], [-73.81578517553721, 40.83309552913763], [-73.81576099019283, 40.83301951722804], [-73.81575868443079, 40.83297133606776], [-73.81573931591329, 40.832917569023245], [-73.81570045065659, 40.83285635786962], [-73.81566150552952, 40.832824796199446], [-73.8156396258708, 40.83279696542463], [-73.81564455576138, 40.83277844427898], [-73.81565687208857, 40.83273584511138], [-73.81561069891927, 40.83267091744393], [-73.81558896401555, 40.83259120460979], [-73.8155262657416, 40.83244031603444], [-73.81552445059262, 40.83242168308638], [-73.81551415742112, 40.83238887725339], [-73.81550148340233, 40.832340216626605], [-73.81548695785708, 40.83229989105653], [-73.81545415504173, 40.832252587162245], [-73.81541947560144, 40.8322219576365], [-73.81538475036352, 40.83220800390095], [-73.81536778603407, 40.83220016860216], [-73.81536178096943, 40.8321821799423], [-73.81535012883155, 40.83215652838934], [-73.81534473691308, 40.8321217765506], [-73.81534851485645, 40.8320787003674], [-73.81533585027684, 40.832027260003535], [-73.81531774665558, 40.83195774409858], [-73.81528686987531, 40.83187570053322], [-73.81523578077928, 40.8318241986078], [-73.81519564015055, 40.83178383195661], [-73.81515181352549, 40.83175318718669], [-73.81513389129239, 40.83173271194589], [-73.81511886641742, 40.83170941705416], [-73.81509887669334, 40.83170863014218], [-73.81507143847716, 40.83170580742162], [-73.81504219117492, 40.831696032715996], [-73.81503125254112, 40.83168211701787], [-73.81503311704115, 40.831669612305724], [-73.815047781245, 40.83165990852288], [-73.81505696355028, 40.83164741504313], [-73.81508468995636, 40.83163755931672], [-73.81508809871505, 40.831636347215365], [-73.81511007088926, 40.8316308238175], [-73.81511743613852, 40.831614158744074], [-73.81511017129878, 40.83159469031741], [-73.81510110443273, 40.83156549119746], [-73.81508473776748, 40.83152933179735], [-73.81507019720061, 40.83149456527241], [-73.8150483259507, 40.83146395566524], [-73.81502276436507, 40.83144445906091], [-73.8150063166291, 40.831437483601015], [-73.81498986889517, 40.83143050864124], [-73.8149845118381, 40.831426932448565], [-73.81497525883807, 40.83142075639332], [-73.81497297607608, 40.83140267752356], [-73.81496845670792, 40.831378871930774], [-73.81495728330384, 40.83130537949727], [-73.8149519676384, 40.83124283312105], [-73.81494661465638, 40.83119418316462], [-73.81491210654471, 40.83110240535299], [-73.8148793738859, 40.8310300864914], [-73.81485346914792, 40.831008199729396], [-73.81449221012457, 40.83117114022429], [-73.8144703368803, 40.831141458252], [-73.81484313432419, 40.83098612051357], [-73.81476387691956, 40.830816799993485], [-73.81471418896022, 40.83078935088011], [-73.81468315863783, 40.830762896332885], [-73.81465211672611, 40.830740610937674], [-73.81461196249836, 40.83070580318127], [-73.81457912170136, 40.83067239594655], [-73.8145444348584, 40.83064454553257], [-73.8145200857769, 40.830622875794205], [-73.8141029873901, 40.8308245004488], [-73.81411027925253, 40.83083424000771], [-73.81391425388868, 40.830924257035456], [-73.81389783360487, 40.83090755415827], [-73.81409205231331, 40.83080919639034], [-73.81409935105553, 40.83081615673173], [-73.81451016521454, 40.83061404705105], [-73.81450063252919, 40.83060556269102], [-73.81447146717453, 40.830566602710356], [-73.8144453576965, 40.830527533052255], [-73.81440990918259, 40.83049533863696], [-73.81438383728965, 40.83049697476876], [-73.8143581843146, 40.83051083113236], [-73.81433805012041, 40.83051357794952], [-73.8143124359147, 40.83051353671036], [-73.81429416266316, 40.83050516943084], [-73.81428691048255, 40.83048153229186], [-73.8142869880651, 40.83045373660404], [-73.81430350529655, 40.83043569753944], [-73.8143209286821, 40.83041452806312], [-73.81431644801685, 40.830387077507226], [-73.81430920681511, 40.83035927120747], [-73.81386585293004, 40.83056701679292], [-73.81385671977257, 40.83056144292272], [-73.81429959295038, 40.83035010370264], [-73.81427271193803, 40.83032446922214], [-73.81422894014088, 40.83027436739528], [-73.81415593309119, 40.83021032195971], [-73.81410480065877, 40.830175496330924], [-73.81404635686468, 40.83013787848808], [-73.8139984413421, 40.83012165186594], [-73.81396593536506, 40.83010856419353], [-73.81393851750967, 40.830098792155056], [-73.8139038412147, 40.83006816166204], [-73.81388380732328, 40.83003477606283], [-73.81386793788137, 40.8300031297187], [-73.81385179398815, 40.82998846873031], [-73.81382087385553, 40.82996758146256], [-73.81382094990106, 40.829940404437345], [-73.81383081071448, 40.829903360608874], [-73.8138677501533, 40.829857368782605], [-73.81370296001816, 40.8299330331285], [-73.81366831284713, 40.82989128485377], [-73.81372876508037, 40.82986497781873], [-73.81373601310588, 40.829890004436386], [-73.81388709503958, 40.829821032466576], [-73.81388971246565, 40.82977745220226], [-73.81385742042924, 40.82969339899325], [-73.81382829192314, 40.829641467990086], [-73.81376330779892, 40.829616656230705], [-73.81371136287345, 40.82958198390346], [-73.81369726279111, 40.82955233460176], [-73.81358494476267, 40.82960764421972], [-73.81356669955272, 40.829589547888396], [-73.81368881870836, 40.829534579120164], [-73.81368549299344, 40.82952758730918], [-73.81366931947859, 40.829495443566074], [-73.81365646178284, 40.829441067987815], [-73.81363379708303, 40.82940397161257], [-73.8136111049418, 40.829376758961125], [-73.81357536743714, 40.82936187696005], [-73.81354614192863, 40.829344534658915], [-73.8135135811422, 40.82935683542821], [-73.81350052277399, 40.82937410835893], [-73.81346143480168, 40.829393810340115], [-73.8134126382601, 40.82939620272017], [-73.81337687307196, 40.829391202710475], [-73.81336719226249, 40.829364009893176], [-73.81337049362966, 40.829346721355385], [-73.8133900852401, 40.82931957590819], [-73.81343239701275, 40.829309761766766], [-73.81345847285763, 40.8292900387481], [-73.81349434254014, 40.829257978479866], [-73.81350420409575, 40.82922093467621], [-73.81349778929821, 40.82918880618543], [-73.81348161595967, 40.829156661241505], [-73.81345250898103, 40.8290973188922], [-73.8133876638913, 40.82902309447355], [-73.81334223828975, 40.82898349038659], [-73.81329027328144, 40.82895622911309], [-73.81324478514992, 40.82893886157594], [-73.81319927513273, 40.828928905266224], [-73.813143988668, 40.82892634488135], [-73.81309840343981, 40.828943564892654], [-73.81304629884026, 40.828965717125605], [-73.81299420898003, 40.828982926550694], [-73.81298393018706, 40.82898290988223], [-73.81295192380885, 40.828982857973706], [-73.81292591739589, 40.82897787532666], [-73.81290321918617, 40.82895313216851], [-73.81293061658954, 40.82891763412877], [-73.81296517107852, 40.828898877852495], [-73.81299450062069, 40.828879159544265], [-73.81304009189216, 40.8288594687658], [-73.81308242468461, 40.82884224350567], [-73.81312472298374, 40.82883737042055], [-73.8131605294834, 40.82882754583679], [-73.81318011408904, 40.82880287122954], [-73.81317040437462, 40.82878556044469], [-73.81315741540367, 40.82877812812], [-73.81312813540961, 40.8287805514953], [-73.81309886147174, 40.828780504055096], [-73.81307603789213, 40.82880023176302], [-73.81305326894844, 40.828800194854914], [-73.81305005842925, 40.828785365413324], [-73.81306314544747, 40.82875820947431], [-73.81311848034197, 40.82874347609437], [-73.81314783102388, 40.82871634649997], [-73.81313163715156, 40.82869161392435], [-73.81308614880481, 40.82867424515056], [-73.8130373736062, 40.82866922461706], [-73.81297875814674, 40.82869383691489], [-73.81293970515073, 40.8287011848559], [-73.81288120079817, 40.82868626570542], [-73.81286608233691, 40.8286720696113], [-73.81283902833232, 40.82864666669566], [-73.81280015000063, 40.82859225045511], [-73.81281003898482, 40.828545324148955], [-73.81281664954315, 40.82850827461945], [-73.81282141350364, 40.82850210002995], [-73.81283950089782, 40.8284786642376], [-73.81285913372197, 40.828436694705566], [-73.81287548028875, 40.828407073927764], [-73.81287233170012, 40.828370009061004], [-73.81283989701991, 40.82833783763401], [-73.812797717407, 40.82830070941289], [-73.81275469994516, 40.82824823625135], [-73.81241743288683, 40.82826833757757], [-73.81207967398305, 40.82828291616279], [-73.81174157998137, 40.828291965241405], [-73.81140330778518, 40.82829548061387], [-73.81106501438015, 40.82829346064893], [-73.81072685676286, 40.82828590628369], [-73.81038899186557, 40.828272821024285], [-73.81005157648552, 40.82825421094328], [-73.81003364415581, 40.828255483883105], [-73.81001568318555, 40.8282544703552], [-73.809998195326, 40.8282511986734], [-73.80998166911195, 40.828245760234], [-73.80996656621346, 40.82823830696328], [-73.80995330853902, 40.828229047073116], [-73.80994226644887, 40.828218239244435], [-73.80993374840969, 40.82820618540038], [-73.80992799237673, 40.828193222272205], [-73.80992515914596, 40.82817971199259], [-73.80992532786387, 40.8281660319787], [-73.80993254336377, 40.82810445899091], [-73.80994476641965, 40.8280433437766], [-73.80996195006828, 40.827982921083546], [-73.80996243584207, 40.827957280459984], [-73.80996812293998, 40.82793200366772], [-73.80997887521065, 40.82790769581818], [-73.8099944352462, 40.82788493882607], [-73.81001443054461, 40.82786427747887], [-73.81003838242802, 40.82784620639564], [-73.81006571750203, 40.82783115818527], [-73.81009578138243, 40.827819493091425], [-73.81014751972339, 40.827844819626264], [-73.81019428969105, 40.82787521415669], [-73.81023523700281, 40.82791012151226], [-73.81026961372608, 40.82794890409298], [-73.81029679194002, 40.827990853514564], [-73.81039434055502, 40.82800269705201], [-73.81049268747769, 40.828009770889935], [-73.8105914289227, 40.82801204598525], [-73.81069015948461, 40.82800951299696], [-73.81078847380294, 40.82800218232476], [-73.81092191600172, 40.82799169144544], [-73.81105578108608, 40.82798499568344], [-73.81104291722448, 40.8278399863693], [-73.81025342354486, 40.82783413271453], [-73.81025357303955, 40.82779717689454], [-73.81003932398117, 40.82779708065319], [-73.81003940033308, 40.827778138786755], [-73.81005896686955, 40.82777817051337], [-73.81014675074867, 40.82777831339122], [-73.81016578820226, 40.827778344731236], [-73.81025463156841, 40.827778489270926], [-73.81027578336358, 40.82777852407166], [-73.81027562551783, 40.827817716832385], [-73.81087159975854, 40.8278207208212], [-73.81096784388194, 40.82782120579908], [-73.81098317961307, 40.827821282533804], [-73.8110730794472, 40.82782173558194], [-73.81109158610842, 40.82782182905629], [-73.81117672513535, 40.827822257807334], [-73.81119523466404, 40.82782235126974], [-73.81128460343885, 40.827822801608264], [-73.81129835436784, 40.82782286967677], [-73.81139036654251, 40.82782333364003], [-73.81140305953447, 40.82782339778795], [-73.8114913701604, 40.82782384188008], [-73.81150987969025, 40.82782393529195], [-73.81159977820816, 40.82782438792738], [-73.8116172284823, 40.82782447573839], [-73.81170712744435, 40.827824927788264], [-73.81172246317686, 40.82782500442467], [-73.81180813106737, 40.8278254357508], [-73.81182240952798, 40.827825507298016], [-73.8118223300786, 40.82784541192274], [-73.81106294234102, 40.827843650439824], [-73.81107326796136, 40.82798440231472], [-73.81123211581864, 40.82798198267601], [-73.8113909490106, 40.827984905487654], [-73.81154945629142, 40.82799316502208], [-73.81170732705306, 40.828006745094235], [-73.81193103041976, 40.8279578538898], [-73.81215213424491, 40.8279025594855], [-73.81221065157155, 40.82788214696051], [-73.81227167478252, 40.827866531878975], [-73.81233452441525, 40.82785588810631], [-73.81239850067199, 40.827850334155386], [-73.81246289121182, 40.82784993186659], [-73.81249874269284, 40.827852725500776], [-73.81253472144937, 40.8278511350799], [-73.81256990430887, 40.827845201412416], [-73.8126033885206, 40.82783507674915], [-73.81263431492006, 40.827821020877174], [-73.81266188997422, 40.82780339445332], [-73.81268540614411, 40.82778264975009], [-73.81270426003739, 40.827759319051076], [-73.81271796789207, 40.82773400099277], [-73.81271966987296, 40.8277011527119], [-73.81271421812419, 40.827662935101856], [-73.81270087766386, 40.82762585345084], [-73.81267998431142, 40.82759084117899], [-73.81265206400504, 40.8275587796147], [-73.81279793343212, 40.827488004691915], [-73.81294049778498, 40.827413445256624], [-73.81307958697005, 40.82733519027174], [-73.81322297909767, 40.827248350084865], [-73.81327731601434, 40.827219873642846], [-73.81332733265431, 40.827187146920544], [-73.81337245546887, 40.82715054520459], [-73.81341746821232, 40.827104396661156], [-73.81345470467141, 40.827054409336945], [-73.81347397102728, 40.82702117349016], [-73.81350706829701, 40.826979444546694], [-73.81354780924852, 40.8269418194804], [-73.81359532607138, 40.82690909972042], [-73.8136486066307, 40.82688198220948], [-73.81370651602658, 40.82686104455957], [-73.81376782076717, 40.82684673274936], [-73.81379818054299, 40.82683587613544], [-73.81382574499777, 40.82682134022102], [-73.81384974851248, 40.8268035287504], [-73.81386952437761, 40.82678293644884], [-73.81388452331008, 40.82676013527992], [-73.81389432871104, 40.826735758559764], [-73.81389866823578, 40.826710483365126], [-73.81389742135792, 40.82668501172801], [-73.81389062271647, 40.82666005113571], [-73.81387846115314, 40.826636294880494], [-73.81386127446578, 40.82661440280253], [-73.81383954002627, 40.82659498296336], [-73.81381386152117, 40.82657857475635], [-73.81377896438124, 40.82656185725203], [-73.81374816945775, 40.82654100133766], [-73.81372231818152, 40.82651657687666], [-73.81370233766253, 40.82648961622273], [-73.81368839582821, 40.826460546040394], [-73.81368086394365, 40.8264301404723], [-73.81367994257492, 40.82639920922203], [-73.8136856562489, 40.82636857599055], [-73.81369785280111, 40.826339056541826], [-73.81370737098624, 40.826304787117266], [-73.8137220768662, 40.826271592515035], [-73.8132835686149, 40.826325091587336], [-73.81328485368158, 40.82633405572625], [-73.81292293225123, 40.8263762874159], [-73.81291776131222, 40.826350389120634], [-73.81327444814539, 40.82630516167435], [-73.8132783556247, 40.82631413005962], [-73.8137278785009, 40.826261219723214], [-73.81375293191677, 40.82622537729861], [-73.8137841180907, 40.8261924278859], [-73.81382086755549, 40.82616297313953], [-73.81386250926184, 40.82613755090088], [-73.813899734164, 40.8261268030922], [-73.8139341692345, 40.82611160861488], [-73.81396490141368, 40.826092370357664], [-73.81399111582792, 40.82606959843132], [-73.81401211739598, 40.826043896643164], [-73.81402734925915, 40.82601594648597], [-73.81403640754664, 40.825986489068086], [-73.81403905208353, 40.82595630546266], [-73.81403521275747, 40.82592619599652], [-73.8140230126393, 40.82587493820311], [-73.81400472461513, 40.82582473600506], [-73.81398050035672, 40.82577600571638], [-73.8139505407614, 40.825729151443596], [-73.81342885926273, 40.82593498727305], [-73.81342102197195, 40.82592501676828], [-73.81394161953392, 40.82571703172848], [-73.81391025451528, 40.825679123760466], [-73.8138789643853, 40.82566500151753], [-73.81385119924143, 40.82564712537406], [-73.81382773137503, 40.82562599255938], [-73.81380921354477, 40.82560219088689], [-73.8137961608223, 40.82557638240445], [-73.81378893626466, 40.825549284978734], [-73.81378774081614, 40.82552165232791], [-73.813792607721, 40.825494253056405], [-73.81380340159868, 40.82546784927636], [-73.81382905764472, 40.825425599468396], [-73.8138415853742, 40.825390214389905], [-73.81382927557102, 40.82534770845477], [-73.81379523517661, 40.82529808600147], [-73.81373629018793, 40.8252649465927], [-73.81366797129029, 40.825250673361865], [-73.8136442164446, 40.82524725081229], [-73.81356857982078, 40.825236350529536], [-73.8134909326115, 40.82522442437556], [-73.81342574760329, 40.82520071539207], [-73.81333522606086, 40.82517094670755], [-73.81303096572003, 40.82525908365103], [-73.8130340268445, 40.82527561072113], [-73.81296590521575, 40.825294457496376], [-73.81295635248642, 40.82527312484057], [-73.81212588761348, 40.82556917811196], [-73.81211350556435, 40.82555263465937], [-73.81288179895613, 40.825265923633026], [-73.81289110191149, 40.825273018985], [-73.81294709154035, 40.82525186737314], [-73.81293781512854, 40.825235329045846], [-73.81300000697752, 40.82521890780793], [-73.81301238918823, 40.825235449993684], [-73.81327735515306, 40.82516132170753], [-73.81324198658949, 40.825150384612044], [-73.81320885874432, 40.8251359560348], [-73.81317856561382, 40.82511829468673], [-73.81314867141118, 40.82509506889877], [-73.81312368493748, 40.82506870878606], [-73.81310416796303, 40.825039807008935], [-73.81309055928445, 40.82500901337185], [-73.81308316485993, 40.82497702021324], [-73.81308215093007, 40.82494454683971], [-73.81311298890206, 40.824906756999546], [-73.81314913024612, 40.82487178418835], [-73.81319013315124, 40.824840055924795], [-73.81323549637867, 40.82481196006491], [-73.813359465185, 40.824748433206416], [-73.8134870982995, 40.82468924163074], [-73.81361813317884, 40.82463450709191], [-73.8138271853649, 40.82455801014524], [-73.81403745843352, 40.82448346949385], [-73.81404084486043, 40.824423343736534], [-73.81403579807272, 40.82436328520822], [-73.81402237528658, 40.824303974529165], [-73.81400072863937, 40.82424608384259], [-73.81397110346312, 40.8241902691979], [-73.81393383550632, 40.82413716311618], [-73.81388934712506, 40.82408736742192], [-73.8138381424988, 40.82404144642347], [-73.81378080191402, 40.82399992051836], [-73.81327417399014, 40.82396666581955], [-73.81319339244563, 40.82413798079497], [-73.81307687892688, 40.824158431179974], [-73.81304282374128, 40.82419059867518], [-73.81300354722481, 40.82421912588165], [-73.8129597174336, 40.82424352757411], [-73.81291207987398, 40.8242633886987], [-73.81286144482182, 40.82427837143275], [-73.81283277124159, 40.824307238545934], [-73.81281028704977, 40.824339113692574], [-73.81279452789245, 40.82437323751552], [-73.81278586920962, 40.8244087970866], [-73.81278451728717, 40.82444494527273], [-73.81279050434462, 40.82448082091667], [-73.81280368776335, 40.82451556935282], [-73.81281734431211, 40.824569687868866], [-73.81282365660886, 40.82462458268783], [-73.81282255867912, 40.82467968021854], [-73.81281406197633, 40.82473440475073], [-73.81280769858893, 40.8247447677268], [-73.81279911513803, 40.82475416151878], [-73.81278855997932, 40.8247623143231], [-73.81277633851847, 40.82476899024393], [-73.8127628043757, 40.824773996117656], [-73.81274834915234, 40.82477718710233], [-73.8127333911017, 40.82477847086892], [-73.81271836302544, 40.82477781027236], [-73.81267565026901, 40.82474855356526], [-73.81263921487141, 40.82471475042109], [-73.81260988545361, 40.82467716960712], [-73.81258832902596, 40.82463666580286], [-73.81254791394832, 40.824583427341565], [-73.81250030740723, 40.824533737704755], [-73.81244604433085, 40.82448815521099], [-73.8123857344377, 40.82444719202963], [-73.81232005538445, 40.824411308426235], [-73.81224974515277, 40.824380907590935], [-73.81217559375642, 40.82435633110877], [-73.8120984343665, 40.82433785512182], [-73.81201913394872, 40.8243256872264], [-73.81195839613518, 40.82433527463219], [-73.81189679599501, 40.824340846242244], [-73.81183479277101, 40.82434236051894], [-73.81177284871205, 40.8243398061728], [-73.8117418282521, 40.82435530930591], [-73.81171416958549, 40.82437412898191], [-73.81169048180882, 40.824395850757654], [-73.81167128657385, 40.82441999628017], [-73.81165700660038, 40.8244460338207], [-73.81158752719122, 40.82449259986495], [-73.81164355619617, 40.824601662268584], [-73.81161100459448, 40.82461149170246], [-73.81155844768891, 40.82451209023507], [-73.81145450110279, 40.82455077798017], [-73.8113923576503, 40.824550676355635], [-73.8113333076674, 40.82455530009842], [-73.81126489672882, 40.82457407017873], [-73.81122136266126, 40.824585800688645], [-73.81119333024951, 40.82460935833414], [-73.81118706943631, 40.8246258701925], [-73.81119009570455, 40.82465419902697], [-73.81121173987003, 40.82469200019445], [-73.81121163272373, 40.82472976574951], [-73.81121152557873, 40.82476753080167], [-73.81120379684033, 40.82478211456257], [-73.81119903005481, 40.824791113380655], [-73.81116795788567, 40.824791062503195], [-73.81116561831503, 40.824789871760714], [-73.81114003255594, 40.824776854413095], [-73.81112145616825, 40.82475322061574], [-73.8111029139569, 40.82471778543648], [-73.81105945414954, 40.82470355254859], [-73.81103147525586, 40.824708227042], [-73.81102206082721, 40.82474125700513], [-73.8109752916904, 40.82479782810827], [-73.81091717931496, 40.82484035124951], [-73.81084728678657, 40.82487249096481], [-73.81087496294379, 40.82511394844661], [-73.81078485286451, 40.82511380062512], [-73.81076192559172, 40.82490157071415], [-73.81068601764443, 40.82492378484928], [-73.81064242541402, 40.82493654096534], [-73.81052734503115, 40.82497647758409], [-73.81046825426856, 40.824995264180224], [-73.81040914396259, 40.825021130469], [-73.81036867618812, 40.82504702788842], [-73.81033123518831, 40.825101253647155], [-73.81031870564868, 40.825136637675875], [-73.81029687386898, 40.82516492681319], [-73.81025025884574, 40.825167210036845], [-73.81020678524449, 40.825157696642776], [-73.8101601978205, 40.82515053852707], [-73.81006078637894, 40.825143294569564], [-73.80990841221151, 40.82518552905511], [-73.80980576678344, 40.82522312553289], [-73.80968028036179, 40.82526297277709], [-73.8096548114174, 40.825275535621614], [-73.8095722864966, 40.82537616136589], [-73.80944144497772, 40.82549396224491], [-73.80935119369303, 40.825543380566586], [-73.80930446315837, 40.82558578886219], [-73.80924839089201, 40.82563526361869], [-73.80922018271399, 40.82572018858208], [-73.80919816842238, 40.82581220521553], [-73.80916700777829, 40.825842837949104], [-73.80909234633528, 40.82587339816415], [-73.80898675442158, 40.825854341283], [-73.80888744462783, 40.82581169067729], [-73.8087849392145, 40.82579971898293], [-73.80870719042498, 40.82582319357266], [-73.80859213406444, 40.82585368695095], [-73.80853309675437, 40.825853588943254], [-73.80846922650872, 40.82586033865238], [-73.80843370571013, 40.82583926168702], [-73.80838405054162, 40.825817937244864], [-73.8082753447703, 40.825801233282924], [-73.8081634785855, 40.825803407159285], [-73.80807959655907, 40.825798547247096], [-73.80799778085272, 40.82579036736252], [-73.80798431158676, 40.82582757491857], [-73.80795249362446, 40.82582557697375], [-73.80796309299755, 40.825781742732275], [-73.80789323724042, 40.82577227298295], [-73.80783421319295, 40.825767454803035], [-73.80775957168109, 40.825790932759794], [-73.80772524164837, 40.82584280293801], [-73.80770957571971, 40.825887623972285], [-73.80766604765235, 40.825896992059], [-73.80764748561123, 40.82586863722247], [-73.80763826655806, 40.82583321720863], [-73.80757311562436, 40.82579770320492], [-73.80752039504299, 40.82576220991271], [-73.80746459514764, 40.82571727077681], [-73.80741500869622, 40.82567234199043], [-73.80734677251718, 40.825629742447454], [-73.80730969098856, 40.82555886937343], [-73.80727567651664, 40.82550216477541], [-73.80722875093615, 40.82545562424451], [-73.80716084578457, 40.82545476690021], [-73.80710187725667, 40.82543106423241], [-73.80702113005182, 40.825416767444196], [-73.80697771152202, 40.825388370897386], [-73.80697779368697, 40.82536004598342], [-73.80699654610872, 40.82532231281291], [-73.80697191452626, 40.82524438070721], [-73.80691927004251, 40.825182923301696], [-73.80686672775298, 40.82508606190425], [-73.80681727303701, 40.82499628591176], [-73.80680811003201, 40.82494198271789], [-73.8068081786768, 40.82491833925311], [-73.8068082607937, 40.82489005554783], [-73.8067898298337, 40.82481685479483], [-73.80675587036083, 40.82474126692514], [-73.80673428399315, 40.82468458298442], [-73.80669402711496, 40.8246373085901], [-73.80666416253398, 40.8246307726419], [-73.80665055458626, 40.82462779486007], [-73.80659773251966, 40.8246277062807], [-73.80656361554976, 40.82460640543176], [-73.8065730253077, 40.82457573741435], [-73.80663305925727, 40.824551977740555], [-73.80663835217979, 40.82454988300981], [-73.80671607190382, 40.824535851615266], [-73.80675345538235, 40.824502870046146], [-73.80677536296291, 40.82444861773684], [-73.80677866215677, 40.824382534315184], [-73.80675402432223, 40.824306962560634], [-73.80670755354015, 40.824259677586625], [-73.80665477291527, 40.82424542690896], [-73.80659267091683, 40.824231161075886], [-73.80653986288887, 40.82422635215272], [-73.80653971012154, 40.82422625523476], [-73.80652125397441, 40.824214519159305], [-73.80651819407517, 40.82419799191265], [-73.80652975478517, 40.82417364586319], [-73.80658800807505, 40.82417240690529], [-73.80669991403106, 40.82415607293682], [-73.80673104693946, 40.824134881495034], [-73.80673534271885, 40.824127044858386], [-73.80676853145432, 40.82406649477149], [-73.80679049436186, 40.823993361689105], [-73.80676901809413, 40.82389891219252], [-73.80675058060002, 40.82382807065859], [-73.80670714973103, 40.823804394488], [-73.80664809366584, 40.82381137674557], [-73.8066002188434, 40.82381649005903], [-73.80653930859727, 40.82382299606159], [-73.80648339363748, 40.823818181899156], [-73.80646480610469, 40.82379926749916], [-73.80646486788679, 40.823778025148485], [-73.80650842971878, 40.82375685462484], [-73.80657994221666, 40.8237404524844], [-73.80660475663629, 40.823736307329185], [-73.8066638763997, 40.823726431548245], [-73.80669197728993, 40.82367927106416], [-73.80670458478495, 40.82361792408326], [-73.80668314901997, 40.82350931293968], [-73.80666166000621, 40.823419584237065], [-73.80659975771904, 40.82333686868608], [-73.80656262901503, 40.82331927529271], [-73.80650977491813, 40.82329423114611], [-73.80642582751348, 40.823312973294115], [-73.80634189402245, 40.82332699405835], [-73.806264174748, 40.82334102570462], [-73.80623630693701, 40.823307934650714], [-73.8063016326953, 40.82328208043145], [-73.80637938534883, 40.82325624754443], [-73.80646027418055, 40.82322097751142], [-73.80651018671007, 40.82315261172039], [-73.80650507976743, 40.82313628859853], [-73.80649171471407, 40.82309357302469], [-73.80644539068587, 40.82299672066007], [-73.80638355020798, 40.82289276212969], [-73.80629982379307, 40.82283597473965], [-73.80622518370484, 40.82285945171243], [-73.80612875487965, 40.82289705487307], [-73.80604792752236, 40.82291108114822], [-73.80596714429264, 40.82291094524419], [-73.80595169124446, 40.82288259534941], [-73.80601079450372, 40.822859091244155], [-73.80608232613048, 40.82283560918261], [-73.80614763085906, 40.82281683527484], [-73.80620054772561, 40.822783879486664], [-73.80622242082086, 40.822741429989435], [-73.80622264763869, 40.82266354012839], [-73.80621136474035, 40.822609352914775], [-73.80604677047366, 40.82227392069623], [-73.8058623716858, 40.8218981288477], [-73.80582465580834, 40.821767789793014], [-73.80581748219579, 40.82167150380358], [-73.80585139220263, 40.82154697180627], [-73.80590748413401, 40.82148477084862], [-73.80595822489663, 40.82142804377334], [-73.80599142834016, 40.8208266279261], [-73.80599159848934, 40.82082355849007], [-73.80572802989728, 40.820177338695], [-73.80571008397013, 40.82015714401185], [-73.80547386434988, 40.81990665057202], [-73.80523559070691, 40.819657282881714], [-73.80507202199193, 40.819487976380735], [-73.80461931205352, 40.819027114213704], [-73.80451800414906, 40.81892768751854], [-73.80441057109248, 40.81883204375195], [-73.80429725730733, 40.818740400491954], [-73.8041783205906, 40.81865296621511], [-73.80405403152841, 40.81856993982216], [-73.80392467287915, 40.818491510185964], [-73.80379053893057, 40.81841785572151], [-73.80365193483067, 40.81834914398075], [-73.80350917589287, 40.818285531271194], [-73.80348320649304, 40.81827698546656], [-73.8033684017511, 40.818251071675974], [-73.8032484670736, 40.81824411399359], [-73.80313418501758, 40.818238128099956], [-73.80301362934604, 40.81823305962992], [-73.80288339224205, 40.81823100031589], [-73.80276008083707, 40.81823144647568], [-73.80267407634148, 40.818238575091385], [-73.80262323567511, 40.81826744875063], [-73.80259400940082, 40.81833793435092], [-73.80258966124985, 40.81838925599237], [-73.8025867881536, 40.81842309141102], [-73.80257434029812, 40.818451606434195], [-73.8025613548568, 40.818481373285515], [-73.80252163401762, 40.81852598830582], [-73.80246811886731, 40.818540907528245], [-73.80241341714243, 40.818516527906276], [-73.80233422077421, 40.81849426583149], [-73.80230024253227, 40.81848471629447], [-73.80226765659415, 40.81849714235332], [-73.80224832249247, 40.81850952629914], [-73.80220279006076, 40.81853869768866], [-73.80207892871874, 40.818556736968574], [-73.802045607213, 40.818545541715466], [-73.80203991576224, 40.81846644869598], [-73.80197305662682, 40.818412980266366], [-73.80193233179318, 40.818388128712954], [-73.80188453083599, 40.818358950889284], [-73.80181251656832, 40.81833695442739], [-73.80171011541323, 40.81830567455004], [-73.80161087523832, 40.81826749431404], [-73.80149789289162, 40.818242761504756], [-73.80139138775395, 40.81822023599131], [-73.8012874403864, 40.81818294714325], [-73.80118517550201, 40.818129687279416], [-73.80111526287806, 40.81810654009357], [-73.8010504292954, 40.818101701868414], [-73.80102437218616, 40.81810285470764], [-73.80092598956914, 40.81810722246574], [-73.80081280395602, 40.818091195900735], [-73.80070762635494, 40.81804509879783], [-73.80061491137691, 40.81796563948056], [-73.80058906737115, 40.81793542809662], [-73.80058276188893, 40.8179114283121], [-73.80056705886217, 40.817851680240636], [-73.8005663755435, 40.81776719362641], [-73.80057291210369, 40.81770009533798], [-73.80051419814835, 40.817667880441114], [-73.80041511587255, 40.81761465286758], [-73.8003233797814, 40.81756283391447], [-73.80022175941981, 40.81752168902743], [-73.80012056556171, 40.81748365673034], [-73.80001179104401, 40.81744353070525], [-73.79989909088276, 40.817404667434865], [-73.79980333641196, 40.817370904118405], [-73.79968878220974, 40.81733351415372], [-73.79956940847994, 40.81729861114655], [-73.79945250870378, 40.81726094732123], [-73.79934721608105, 40.81721849427456], [-73.79849679070325, 40.81690614778245], [-73.79841487478996, 40.816878328494084], [-73.79832995437337, 40.816856315037775], [-73.79824274065875, 40.816840291774895], [-73.79820268141646, 40.816836311739145], [-73.79816398385059, 40.81682749481479], [-73.79812765294623, 40.81681406997936], [-73.79809463222344, 40.816796385879], [-73.7980657792349, 40.81677490177375], [-73.79798482995983, 40.81665547358205], [-73.7979480421167, 40.816622801752466], [-73.79786829159976, 40.816592990977135], [-73.79775500596446, 40.816564308576964], [-73.79765347182345, 40.81653368480568], [-73.79759032777322, 40.816507532070055], [-73.79740587876204, 40.81585153613393], [-73.79738112573526, 40.8157635048956], [-73.79744080996129, 40.81571872968728], [-73.79771465236787, 40.81551329274344], [-73.79811945293483, 40.815698785003185], [-73.798642206466, 40.81546401593515], [-73.79902816314, 40.81554113493866], [-73.7999467818626, 40.814449565689415], [-73.8001114686428, 40.814024525560605], [-73.80048824791561, 40.81386700886416], [-73.80091593237641, 40.813761402506564], [-73.80110647514584, 40.81353949952995], [-73.80131991856463, 40.81326090053891], [-73.801458005173, 40.812940947203145], [-73.80174561215799, 40.81286501791826], [-73.80151992552115, 40.81255404711591], [-73.80142634447695, 40.812424421536164], [-73.8013738591895, 40.81244424632369], [-73.80133906141198, 40.81239373439237], [-73.80139154446515, 40.812373909616674], [-73.80138806882769, 40.81236726411357], [-73.80140380968845, 40.8123633086586], [-73.80155119869595, 40.812569809594166], [-73.8017593218082, 40.81286139698733], [-73.80195659132498, 40.81280931612244], [-73.80201310827444, 40.81280396492847], [-73.80206824965333, 40.81279313454525], [-73.80212109920605, 40.81277700493558], [-73.80216842849373, 40.81275699420263], [-73.80221217944805, 40.81273273386117], [-73.80225169457027, 40.81270458850318], [-73.8022863800205, 40.81267298110569], [-73.80235405959267, 40.8126237038675], [-73.80242779292391, 40.81257972584967], [-73.80250686428273, 40.81254147394104], [-73.8025905061269, 40.81250931944726], [-73.80267790655299, 40.812483574486876], [-73.8027682171769, 40.81246448896156], [-73.80286056136796, 40.81245224813112], [-73.80295404275847, 40.8124469708149], [-73.80304775394252, 40.812448708238854], [-73.80314078528485, 40.81245744353789], [-73.80323223374798, 40.81247309192046], [-73.80332121165738, 40.81249550149052], [-73.80340685531826, 40.81252445472245], [-73.80343385755597, 40.812539387697015], [-73.80346320797932, 40.81255539280687], [-73.80347049567364, 40.81255936585385], [-73.80351243525232, 40.81255943733641], [-73.80352685926309, 40.812548007065814], [-73.80354921841081, 40.812530290037316], [-73.8035555484133, 40.81252779548782], [-73.80357865673949, 40.81251868448007], [-73.80361390158114, 40.812521106621354], [-73.80362783442257, 40.81253706297839], [-73.80364525098142, 40.812557007836645], [-73.80365392095106, 40.8125795935696], [-73.80365034781317, 40.81260614051942], [-73.80364013123159, 40.81263514480156], [-73.80355021877938, 40.81285740190122], [-73.80351997906509, 40.81292187125158], [-73.80362063328043, 40.81296474081772], [-73.80373757162269, 40.81279360802259], [-73.80377912019877, 40.81280990604305], [-73.80361429261028, 40.81304451492809], [-73.80368492820668, 40.813108696077066], [-73.80370415669779, 40.81312753398959], [-73.80377171351132, 40.81319372114632], [-73.80379173882275, 40.813213340563365], [-73.80385705015105, 40.81327732809868], [-73.8038800682624, 40.813299878414014], [-73.80394693022657, 40.81336538363001], [-73.803981205966, 40.813398963808304], [-73.80399360549882, 40.813411112183594], [-73.80417389009331, 40.81358773932113], [-73.80423653201755, 40.813646527993306], [-73.80425238349035, 40.813650440311555], [-73.80434433653338, 40.81359676461407], [-73.804358103467, 40.813608538674], [-73.80432127867917, 40.81363188929205], [-73.80419004986149, 40.81370794409653], [-73.80415365301042, 40.8137306728651], [-73.80414237460322, 40.813718304106935], [-73.80422899591787, 40.81366330521166], [-73.80415866820894, 40.813597522048475], [-73.80395040123676, 40.81339773172134], [-73.80394338723917, 40.813391002381074], [-73.80389636709297, 40.81334589524031], [-73.80387189663088, 40.81332242123863], [-73.80384850093282, 40.813299977327844], [-73.80379871138618, 40.813252213348065], [-73.80377802413402, 40.813232369495026], [-73.80372561929765, 40.813182096199625], [-73.80370612798548, 40.81316339722423], [-73.80360189093186, 40.81305645256864], [-73.80355921193633, 40.81304271217233], [-73.80361455805945, 40.81297219119157], [-73.80351370643069, 40.81293524328937], [-73.80345921365522, 40.81305142207793], [-73.80338820741223, 40.8131299665213], [-73.8033003046772, 40.81316068489423], [-73.80325294454988, 40.81319677354991], [-73.8032354331691, 40.81329636637064], [-73.8033443955358, 40.813320082824134], [-73.8033496817059, 40.813305156157924], [-73.80346101380117, 40.813329243757465], [-73.80345177160017, 40.81335213167163], [-73.80336795116905, 40.81333207238363], [-73.80336399216976, 40.81334102720876], [-73.80323201378704, 40.813315809673725], [-73.80322449366612, 40.81335858052121], [-73.80329365293572, 40.813375858613625], [-73.80335641541957, 40.81339202176537], [-73.80335865160461, 40.813384214730824], [-73.8034551929976, 40.813402690990465], [-73.80345039819939, 40.81341422712503], [-73.80351937559368, 40.81348500937568], [-73.80352565577122, 40.813491655739035], [-73.80355921961917, 40.81352717208712], [-73.80358389635967, 40.81355328519059], [-73.80363362401805, 40.81360590670701], [-73.8036399068593, 40.813612554743706], [-73.80369692748782, 40.813672893625], [-73.80370197676172, 40.8136782368844], [-73.80396704457125, 40.8139665532554], [-73.80404393782305, 40.81394089451769], [-73.80405436278971, 40.813954506879455], [-73.80407143404403, 40.81396600950239], [-73.80409073408268, 40.813951524759666], [-73.80418428440666, 40.8140304573973], [-73.80417379955037, 40.814038159973684], [-73.80419080253513, 40.81405279568053], [-73.80417701117337, 40.814060768219996], [-73.80419191642532, 40.81407447713419], [-73.80421121425614, 40.814092224321634], [-73.80424913836396, 40.814127100720036], [-73.8043747213263, 40.81424259074794], [-73.80443465897424, 40.814293196487256], [-73.80445407178813, 40.81429219092678], [-73.80452231465456, 40.81424730125912], [-73.80453081609834, 40.81425461992442], [-73.80443367590911, 40.814318886601356], [-73.8042593736155, 40.8141605587425], [-73.80423729676549, 40.81414087779582], [-73.80420271980937, 40.814110052301054], [-73.80417015187051, 40.814081016525826], [-73.80412441310237, 40.81404024000103], [-73.80409350211168, 40.814012682577705], [-73.80403602000344, 40.81396304862275], [-73.80395955345398, 40.813988085558265], [-73.80392654060662, 40.81395130327641], [-73.80387071965497, 40.81389195243698], [-73.80384261688646, 40.81386207162922], [-73.80383895795188, 40.81385818117211], [-73.8038036632947, 40.81382065551699], [-73.80379652765603, 40.813813068401444], [-73.80375932423901, 40.813773511260365], [-73.80375575455594, 40.81376971577157], [-73.80371743434281, 40.813728971477154], [-73.80370996088197, 40.81372102427258], [-73.80367439908748, 40.813683213329725], [-73.8036701640525, 40.81367871192803], [-73.80363275064953, 40.81363893141183], [-73.8036267551889, 40.813632556762215], [-73.80359077080303, 40.813594295436374], [-73.8035847810641, 40.81358792732769], [-73.8035482997436, 40.813549137605676], [-73.80354113734674, 40.81354152194913], [-73.80350374365683, 40.813501763536735], [-73.80349891935015, 40.813496633575944], [-73.8034634349478, 40.81345890445032], [-73.80346026692781, 40.813452907857055], [-73.8034304211, 40.81342211984284], [-73.80335485916852, 40.81340642122662], [-73.80335745131454, 40.81340112448984], [-73.80329255692037, 40.81338806964399], [-73.80322176299286, 40.81337410994221], [-73.80319870538818, 40.81350524057211], [-73.80351667337368, 40.81395196364859], [-73.80356595538284, 40.813999297010604], [-73.80388842505995, 40.81430900826339], [-73.80418033130343, 40.814589962811475], [-73.80431466904517, 40.814592160480984], [-73.80436017115758, 40.814569668414656], [-73.80435321971795, 40.8145563792685], [-73.8043917200654, 40.81453785620168], [-73.80443003333578, 40.814583062613096], [-73.80438628236728, 40.814604232053576], [-73.80436535843762, 40.81458826393251], [-73.80432977892738, 40.81460314258582], [-73.80435128904165, 40.81461877682578], [-73.80436154415412, 40.81469712818207], [-73.80440501682179, 40.81477022485211], [-73.80447282080358, 40.814889832153675], [-73.80460283523365, 40.81504422819805], [-73.80461752374775, 40.815059013201875], [-73.80464897472457, 40.81512510503409], [-73.8046869621925, 40.81518918424174], [-73.80471812332812, 40.8152305050939], [-73.80475934689049, 40.81527223036974], [-73.80480730478178, 40.81530957125489], [-73.8048611970971, 40.81534190492099], [-73.80488528910851, 40.81538273995896], [-73.80491612834518, 40.81542086565561], [-73.80495319709797, 40.81545564196675], [-73.80499587307426, 40.81548648507594], [-73.80501544179707, 40.81549785189193], [-73.80503248781558, 40.81551136188459], [-73.80504660879434, 40.815526696180044], [-73.80505747143667, 40.815543492845656], [-73.80506481935052, 40.815561355432756], [-73.8050859782673, 40.81560005424783], [-73.80511465526511, 40.81563585002811], [-73.80515017994553, 40.815667905940956], [-73.80519172182187, 40.81569547258355], [-73.80523830973368, 40.81571790550182], [-73.80528885455031, 40.81573468025808], [-73.8058461181963, 40.81601415814744], [-73.80596026416015, 40.8160424492395], [-73.80619919500408, 40.816287070772816], [-73.80628451748318, 40.816268486891076], [-73.80641593595035, 40.81651369035763], [-73.80654546539031, 40.81661485334422], [-73.8065724726017, 40.81660635539236], [-73.80656664218886, 40.81660043734657], [-73.80658358725697, 40.81659458816552], [-73.80654668247519, 40.8165270499433], [-73.80654330780881, 40.8165278973175], [-73.80630968841193, 40.81614680715884], [-73.80623465856394, 40.816069679988125], [-73.8061506996536, 40.81598585820601], [-73.80611761373622, 40.81595220932606], [-73.80608837514092, 40.815931147015604], [-73.8060233194087, 40.8158868540877], [-73.8059940724492, 40.81586694129428], [-73.80597884247236, 40.815856571344206], [-73.80594103581132, 40.81583082968136], [-73.80592904941429, 40.81582266981061], [-73.8058889626789, 40.8157953756932], [-73.80587372833129, 40.815785003543915], [-73.80583410312232, 40.8157580231207], [-73.80581997726753, 40.81574840552005], [-73.8057992148723, 40.81573426914921], [-73.80578506371808, 40.81572463491661], [-73.80573575544753, 40.81569106179364], [-73.80571940783229, 40.815679931875984], [-73.80567946981297, 40.815652738953], [-73.80566605855705, 40.815643607689424], [-73.8056135268847, 40.81560783986604], [-73.80558653716153, 40.81558946396059], [-73.80556394546241, 40.81557408176333], [-73.80554519963388, 40.81556131842981], [-73.80551227589093, 40.81553890218644], [-73.80545310964646, 40.81549861684705], [-73.80543412145816, 40.815499000937876], [-73.80541005396141, 40.815505275337735], [-73.8053787878482, 40.8154826202826], [-73.8052821809176, 40.81541117334463], [-73.80508103466697, 40.81519796401256], [-73.80506739580052, 40.81518318854919], [-73.80504496718342, 40.81515889219237], [-73.80498138535133, 40.815090012868914], [-73.80495091220341, 40.81505700198083], [-73.80493788840948, 40.815042891275674], [-73.80488927230539, 40.814990226651936], [-73.8048746512254, 40.81497438672526], [-73.80483702030837, 40.814933620526354], [-73.80482922720184, 40.81492517815443], [-73.80464185020507, 40.81472374153498], [-73.80455440518865, 40.8146298422371], [-73.80455902583564, 40.814626139728986], [-73.80454209268699, 40.81460836963893], [-73.80465942815042, 40.81454498912198], [-73.80468358440314, 40.814571940402026], [-73.80458924572582, 40.81462151114396], [-73.80464142045771, 40.8146808472396], [-73.80465309753234, 40.81469412845984], [-73.80468790441603, 40.81473371206963], [-73.80470143568597, 40.81474910104061], [-73.80473920794104, 40.814792059084844], [-73.80475413212416, 40.81480903083777], [-73.80479626573847, 40.814856948282994], [-73.80481419768991, 40.81487734178961], [-73.80498880311676, 40.81506725560579], [-73.80503703935383, 40.815119720859826], [-73.80505018523058, 40.81513402005794], [-73.8050985180555, 40.815186590822016], [-73.80511150262885, 40.815200713001595], [-73.80516130844828, 40.81525488475045], [-73.80517384860575, 40.815268524202764], [-73.80521866258665, 40.81531726722484], [-73.8052425045649, 40.81534064421272], [-73.80527878825075, 40.8153772647762], [-73.805295359951, 40.815390285150784], [-73.80535574933889, 40.81543773750667], [-73.80538058746234, 40.81545725436465], [-73.80539844430555, 40.81547064745827], [-73.80543623459857, 40.81545953364263], [-73.80554514789834, 40.815534273468856], [-73.80556384511613, 40.81554714376871], [-73.80562333543435, 40.81558809333045], [-73.80564435234865, 40.815602558846166], [-73.8056960781262, 40.81563816447364], [-73.80571555266238, 40.815651568452466], [-73.80574845644439, 40.81567421796571], [-73.80588246586188, 40.815766460022346], [-73.80589470306954, 40.8157748833344], [-73.80591819777833, 40.81579105535799], [-73.8060592785209, 40.81589012535946], [-73.80610141797764, 40.815919717456396], [-73.80621827961798, 40.81602684183027], [-73.80632932566019, 40.81613795932021], [-73.80639654597478, 40.81624528289643], [-73.8064191686194, 40.816281403043625], [-73.80642825943642, 40.816293747629395], [-73.8064518701059, 40.81632580611449], [-73.8064628307186, 40.81634068882139], [-73.80648254634458, 40.81636746054749], [-73.80649071162662, 40.816378548646405], [-73.80651302341059, 40.81640884391771], [-73.8065188961155, 40.816416817901775], [-73.80654164576909, 40.816454878174135], [-73.80655014924943, 40.81646910357052], [-73.80657281840718, 40.8165070281737], [-73.80658167058805, 40.81652183764143], [-73.80661216122775, 40.81657794543156], [-73.80670737798759, 40.816552942534614], [-73.80671967987072, 40.81657517074692], [-73.8065868955687, 40.81661680080234], [-73.80658464685322, 40.8166167970302], [-73.80657923631605, 40.81661807521463], [-73.80660603189658, 40.81666449404111], [-73.80692933177552, 40.816568379076706], [-73.8068860922784, 40.81648781099846], [-73.80685196012053, 40.81642687966935], [-73.80679665584827, 40.816332788342365], [-73.80679174430693, 40.81632443285374], [-73.80678668905028, 40.816315831197706], [-73.80676068779914, 40.81627159591996], [-73.80675616662523, 40.8162639019693], [-73.80673208315861, 40.816222928423784], [-73.80672672445573, 40.8162138112851], [-73.80669928739789, 40.816167131911655], [-73.80669511006323, 40.81616002537431], [-73.80666804914216, 40.816113983888016], [-73.80659999819879, 40.81599820476765], [-73.80659257259929, 40.81598557099249], [-73.80656839498864, 40.81594443525617], [-73.80653572493938, 40.81589198638627], [-73.80655435560061, 40.81588909014948], [-73.80671792148993, 40.816160117391824], [-73.8068992956903, 40.81648403113643], [-73.80694199535742, 40.81656461422068], [-73.80694308670084, 40.816564290547376], [-73.80695662026791, 40.81656026698044], [-73.80704600153508, 40.816533693875236], [-73.8069468699786, 40.816214078576806], [-73.80679777288515, 40.81602191978747], [-73.80679144235306, 40.81600857420437], [-73.80676593813136, 40.8159756145133], [-73.80662873422997, 40.8156838407102], [-73.80658671974201, 40.81569923127239], [-73.80657844298469, 40.81573031518814], [-73.80640177811462, 40.81570203064956], [-73.80641230578708, 40.81566444938959], [-73.80656368335477, 40.81568760275698], [-73.80661773802603, 40.81566855824344], [-73.80659328471774, 40.815616706154735], [-73.80635184999934, 40.81553347110787], [-73.80567498487898, 40.81521229218864], [-73.80531902580653, 40.81478131299817], [-73.80522949485474, 40.81420780309348], [-73.80519435133107, 40.813919367911], [-73.80516580583347, 40.81368508069282], [-73.80517536383367, 40.813331475425606], [-73.80518396072651, 40.813235892206166], [-73.80519224369245, 40.813143821116356], [-73.80505213646254, 40.81289496243412], [-73.80479686676797, 40.812727893160904], [-73.80454814914059, 40.81243148190015], [-73.80438719791255, 40.8122297932157], [-73.80421491507515, 40.812162265824774], [-73.80390281789289, 40.81214399837654], [-73.80361382016022, 40.81193828205798], [-73.80334845213235, 40.811632068063865], [-73.80333236911667, 40.81159984216772], [-73.803166607394, 40.811267696376106], [-73.80295467022943, 40.810965154494426], [-73.8027259539976, 40.81063016439714], [-73.80252595880887, 40.8104780428706], [-73.80224245334347, 40.81019739366033], [-73.802049646329, 40.81016071487725], [-73.8014783291995, 40.810606279088695], [-73.80135552746913, 40.81071084489494], [-73.80115707624924, 40.81058776561507], [-73.80101172909842, 40.81071339964729], [-73.80099517994405, 40.81070707633148], [-73.80124032530553, 40.8104793397404], [-73.80146670007328, 40.8101417179631], [-73.80010949916637, 40.8097109540783], [-73.7985376596495, 40.809587075529706], [-73.79575752264316, 40.80834332114413], [-73.79376346976116, 40.807451167872856], [-73.79367568886633, 40.807423787976035], [-73.79361464719491, 40.80739206270611], [-73.79228907337423, 40.80684900557948], [-73.79172376985717, 40.80687811405359], [-73.79166240651075, 40.80688291239258], [-73.79160253202036, 40.806894192487505], [-73.79154532622087, 40.80691173206286], [-73.79149191636198, 40.806935185499], [-73.79144335489711, 40.80696409064274], [-73.79140059874486, 40.80699787791307], [-73.79136449043281, 40.80703588152564], [-73.79134207789949, 40.80705431575743], [-73.79132391894659, 40.80707527899012], [-73.79131049914889, 40.80709821066667], [-73.79130217735737, 40.807122497592914], [-73.7912991761033, 40.80714749033481], [-73.79130157564589, 40.80717252058411], [-73.7913086602745, 40.80719538969613], [-73.7913202557913, 40.8072171691925], [-73.79133609046531, 40.80723734868176], [-73.79135579322265, 40.807255455267466], [-73.79137890234121, 40.8072710646301], [-73.79140863370712, 40.80728532035197], [-73.79144116789494, 40.807295442968716], [-73.7914851191538, 40.80731907962212], [-73.79152496573849, 40.807346598313664], [-73.79156011736815, 40.807377591389475], [-73.79159005331002, 40.807411599726976], [-73.79158656700483, 40.807427422057586], [-73.79157971415242, 40.80744259605886], [-73.79156968017568, 40.80745671115149], [-73.79155673657297, 40.807469385407586], [-73.7915412335731, 40.807480275885624], [-73.79152359065651, 40.80748908790947], [-73.79150428520737, 40.80749558304202], [-73.79148383959571, 40.80749958553697], [-73.79146280704165, 40.807500987094166], [-73.7914417566478, 40.8074997497899], [-73.79130225120352, 40.807450699934506], [-73.79115926599255, 40.80740782405771], [-73.79101327380823, 40.8073712639305], [-73.79086475738406, 40.80734114044052], [-73.79082222375607, 40.80725097780618], [-73.79078700896973, 40.80715901136414], [-73.79075924419679, 40.80706558375999], [-73.79073903285, 40.80697104308165], [-73.79072645019967, 40.80687574156306], [-73.79072154309202, 40.80678003427088], [-73.79072432977641, 40.806684277782416], [-73.79073479983785, 40.80658882885657], [-73.79042839442938, 40.806326649910694], [-73.79025263728276, 40.80601919752717], [-73.79012463294092, 40.80577155988573], [-73.7900820425006, 40.80552272773616], [-73.79006256328053, 40.80540894183436], [-73.7901519260013, 40.8050799885924], [-73.79028410874172, 40.80481710368888], [-73.79056823807788, 40.80455801129176], [-73.79065958257804, 40.8044857925821], [-73.79103717515576, 40.804263031702], [-73.7916001293646, 40.80414342358668], [-73.7916944800095, 40.80412191786076], [-73.7917684066589, 40.804126967721764], [-73.7918426294635, 40.804127377124786], [-73.79191664479967, 40.804123143292045], [-73.79198995045107, 40.80411429495127], [-73.79206204901791, 40.804100892141136], [-73.79215607116778, 40.80405105809916], [-73.79224529219171, 40.80399636431488], [-73.79235892851956, 40.80404046613813], [-73.79247568741523, 40.804079575992155], [-73.7925951894607, 40.80411356678456], [-73.79271704632252, 40.80414232805787], [-73.79284911683807, 40.80416712806936], [-73.7929829261505, 40.804185785704455], [-73.79311797997443, 40.80419823204247], [-73.7932537794263, 40.80420442110713], [-73.7933898228672, 40.804204330036214], [-73.79351389643507, 40.80426120031478], [-73.79358593394123, 40.80437214572877], [-73.79365540502808, 40.80438665196931], [-73.79373474810107, 40.804443515477054], [-73.79400631451523, 40.804637952667925], [-73.79401383625986, 40.80482686129688], [-73.79451648626257, 40.80500321868419], [-73.79480016551285, 40.80535723507229], [-73.79492122399245, 40.80548518672159], [-73.795049024771, 40.80560929506641], [-73.79518335679418, 40.80572935511478], [-73.79532316872111, 40.8058445112684], [-73.79546898916594, 40.805955281637246], [-73.79562058006752, 40.80606148535965], [-73.79577769394001, 40.80616294902918], [-73.79584187394705, 40.806168721202106], [-73.79601037569107, 40.805921031866944], [-73.79618823923639, 40.805996877675135], [-73.79620903473185, 40.80596859046182], [-73.79650268526339, 40.806104435989965], [-73.79636945999776, 40.8062772027666], [-73.79651535630283, 40.80629227429097], [-73.79702945544223, 40.80634537771899], [-73.79715349910241, 40.80639398668263], [-73.79791967769252, 40.80669422755318], [-73.79885945699559, 40.807136473955254], [-73.80057684375755, 40.808311824761496], [-73.80135746259248, 40.80867466115674], [-73.80149578047421, 40.80876493036836], [-73.80167342725866, 40.80888086774807], [-73.80176206677199, 40.80892708771134], [-73.80185524738366, 40.808967826140574], [-73.80195238321679, 40.80900282688759], [-73.80205286352215, 40.80903186988071], [-73.80215605651892, 40.80905477250789], [-73.80226131336802, 40.809071390765304], [-73.80235859690633, 40.80909493385668], [-73.80250350466216, 40.80911406594121], [-73.80266088412849, 40.80911748306436], [-73.80281002721175, 40.80910515042796], [-73.80295921748117, 40.8090770837295], [-73.8030628852357, 40.80903634957932], [-73.80319979764344, 40.808957906757655], [-73.80333253213347, 40.80889204732416], [-73.80343625359559, 40.80883242967637], [-73.80352332574783, 40.80880110869286], [-73.80359386615068, 40.80875717047788], [-73.80361883887213, 40.80871630005789], [-73.80365216061136, 40.808653415818696], [-73.80366472540902, 40.808606229502374], [-73.80367523090003, 40.808585268998904], [-73.80369016241391, 40.8085659210127], [-73.8037091070832, 40.80854872052492], [-73.80373154107626, 40.80853414313723], [-73.80375684408128, 40.80852259192061], [-73.8037843164576, 40.80851438627146], [-73.80381319858235, 40.80850975307941], [-73.80384269185261, 40.808508820453795], [-73.80387198076747, 40.808511614182414], [-73.8039002554765, 40.80851805701731], [-73.80392673417204, 40.80852797081171], [-73.80397063436935, 40.808579489109675], [-73.80400670107925, 40.80863440527091], [-73.80403448452356, 40.8086920344802], [-73.80405363821481, 40.80875165808883], [-73.80413994427607, 40.808823645840725], [-73.8043246055215, 40.80897027231311], [-73.80451556897218, 40.80911216221173], [-73.8046780836507, 40.80922579881701], [-73.80484464130906, 40.809336004315774], [-73.80501511652449, 40.809442695706984], [-73.8050323653641, 40.80941769848613], [-73.80502190996822, 40.809407723453226], [-73.80506135232707, 40.80936497193477], [-73.80511892265707, 40.80939693436299], [-73.80507948389705, 40.80943868913328], [-73.80506378406419, 40.80942970050321], [-73.80503639918926, 40.809455602310734], [-73.80521045489398, 40.80955787150303], [-73.80536120432706, 40.80964190475309], [-73.80551448228043, 40.80972325474557], [-73.80567020604411, 40.809801877578565], [-73.80584889757822, 40.80988733217924], [-73.80603047812365, 40.80996919585503], [-73.80621482271201, 40.81004741226069], [-73.80661359765998, 40.81031221261748], [-73.80713987884187, 40.81066167645373], [-73.80724175664754, 40.81081524051], [-73.80735019739234, 40.810966187898856], [-73.80746907410685, 40.811119307908385], [-73.80759470369259, 40.81126928328998], [-73.80773421272856, 40.81142370404314], [-73.80788088051818, 40.8115742469754], [-73.80799293413388, 40.81168212066458], [-73.808108664426, 40.81178772984366], [-73.80822799216608, 40.811891002197086], [-73.80839689705721, 40.81202835512165], [-73.80857216375779, 40.81216102742721], [-73.80875356774892, 40.81228884913055], [-73.80894087664379, 40.8124116564625], [-73.80913385048646, 40.81252929207718], [-73.80933224205826, 40.81264160525432], [-73.80953579719565, 40.812748452092066], [-73.80972609888273, 40.8128411911631], [-73.8099202908178, 40.81292915226176], [-73.80998314944907, 40.812946959403504], [-73.81006463809194, 40.812968335663946], [-73.810164771254, 40.812987973839114], [-73.81026812262404, 40.81299948037223], [-73.81034693698221, 40.81299208843081], [-73.81042488566062, 40.81298055690011], [-73.81050159246415, 40.81296494143388], [-73.81059832209317, 40.81293882510617], [-73.81066931287849, 40.812764366537614], [-73.81078477868347, 40.81277095135498], [-73.81078967105614, 40.812793192496365], [-73.81079927013214, 40.812814530157794], [-73.81081331909884, 40.8128343934649], [-73.81083144208924, 40.81285225098786], [-73.81085315423755, 40.81286762496024], [-73.81087787465059, 40.812880104060845], [-73.8109049419503, 40.812889354418935], [-73.81093363196798, 40.81289512854651], [-73.81096317711896, 40.81289727196018], [-73.81099021524315, 40.81288837429818], [-73.81101873877009, 40.812882759406634], [-73.81104806187903, 40.812880562290125], [-73.81107747952412, 40.81288183577624], [-73.81111203579378, 40.812895872048706], [-73.81114311206214, 40.81291400659979], [-73.81116987432775, 40.81293575275008], [-73.81119160436302, 40.812960526894834], [-73.81121212830357, 40.81297907149981], [-73.81124703768323, 40.81299329087466], [-73.81133318625713, 40.813014675039135], [-73.81142401094972, 40.81303075624895], [-73.81147990483964, 40.81303969933488], [-73.8115477260317, 40.813050955975505], [-73.81156607359787, 40.813054001914104], [-73.81166625396294, 40.813057705737904], [-73.81169687915175, 40.813056924669226], [-73.81173149756725, 40.81305604205964], [-73.81177347317572, 40.813043718612995], [-73.81179920804846, 40.81300658601698], [-73.81181560739665, 40.812974748716485], [-73.81183200304997, 40.812944680637685], [-73.81185065747438, 40.8129394000402], [-73.81187624715334, 40.81295360401523], [-73.81189483219015, 40.81297310709516], [-73.81191569106036, 40.813012086637585], [-73.8119412707777, 40.81302983071734], [-73.81199714539541, 40.81304585446229], [-73.81207166602559, 40.813060138171565], [-73.81216251439967, 40.81306736648839], [-73.81225801881348, 40.813076374393425], [-73.8123442043051, 40.813085365376885], [-73.81241175315147, 40.81309255607458], [-73.81248159793859, 40.81311214244907], [-73.81255145273762, 40.81312818866176], [-73.81260732758811, 40.81314421093821], [-73.81268886647128, 40.81314788474566], [-73.81276572662638, 40.81315863066586], [-73.8128331909565, 40.81319591631858], [-73.81290297626033, 40.81323674432575], [-73.81297045202062, 40.81327048877551], [-73.81304259684661, 40.81330070011885], [-73.81313381796316, 40.81333105424298], [-73.81346638785192, 40.813395649261096], [-73.8134754982378, 40.813380409354224], [-73.81348806610542, 40.813359385883324], [-73.81358218143802, 40.81332206814458], [-73.81359600879816, 40.81333731331731], [-73.81357590139099, 40.813363040098935], [-73.81359278876356, 40.8133864853681], [-73.81358329077027, 40.81339597939251], [-73.81354951521904, 40.81342973983681], [-73.8135956622967, 40.81346777917577], [-73.81364716804077, 40.81350161598164], [-73.81370337355365, 40.813530817385136], [-73.8137635598112, 40.81355500981668], [-73.81382695686109, 40.81357388378528], [-73.81389275367364, 40.813587197838025], [-73.81397755117555, 40.81355992115672], [-73.81406532924294, 40.81353877645404], [-73.81415532160705, 40.813523948314405], [-73.81424674267123, 40.81351556618141], [-73.81433879436773, 40.81351370322762], [-73.81443067312435, 40.81351837571565], [-73.81446017919359, 40.812963102256596], [-73.81437165013429, 40.8129594186148], [-73.8143693843687, 40.81293640140598], [-73.81443694808743, 40.812938281073016], [-73.81446963225495, 40.812915320279316], [-73.81446965693051, 40.81290646864015], [-73.81449994116008, 40.81290828760446], [-73.81450223252202, 40.812922453508186], [-73.81452546792079, 40.81294550378021], [-73.8146186626928, 40.812947424068234], [-73.81461860845134, 40.812966897238795], [-73.81449745463826, 40.812964931466944], [-73.81447727191016, 40.81351854184116], [-73.81469475694276, 40.81351931861387], [-73.81470560966756, 40.813519355824006], [-73.814857000871, 40.81354084268805], [-73.814890079811, 40.81354089580195], [-73.81501777001836, 40.81354110074136], [-73.81533237506966, 40.813522132317814], [-73.81551416367422, 40.81350472099332], [-73.81566326325769, 40.81351203974346], [-73.81576169237755, 40.8135718498211], [-73.81587919093577, 40.81376705536385], [-73.81620056896291, 40.81384747647932], [-73.81648015546546, 40.81385323288393], [-73.81667835319837, 40.8138004404734], [-73.81674850706827, 40.81370849827385], [-73.81686078332412, 40.81368842878229], [-73.81724755216894, 40.81360783264775], [-73.8173548154798, 40.81357790828957], [-73.81737201626979, 40.81357434380445], [-73.81735988456492, 40.81337572053985], [-73.81737692242342, 40.81337574752845], [-73.81738942091515, 40.81357073625065], [-73.8174480681646, 40.813558583226055], [-73.81759488728466, 40.8135481935175], [-73.81760101809583, 40.813548387661534], [-73.81777195284765, 40.81355378455104], [-73.81789779310118, 40.81354690310725], [-73.81792084127432, 40.813544680603236], [-73.81793125037362, 40.813400521983574], [-73.81793912728281, 40.81339555608654], [-73.81794185814788, 40.81335572885523], [-73.81793138112988, 40.813352725510505], [-73.81793546303673, 40.81329796423283], [-73.81797608887013, 40.813299024146396], [-73.81797068908527, 40.81335677131783], [-73.81795889327483, 40.81335675269384], [-73.81795616014033, 40.81339757451685], [-73.81796401253763, 40.81340156948312], [-73.81795807110602, 40.81354108952852], [-73.81804229161202, 40.81353296785627], [-73.81807961497645, 40.813517095309244], [-73.8180796827218, 40.8134923116244], [-73.81808440601249, 40.81346930619032], [-73.81811703024695, 40.8134675867559], [-73.81814963601113, 40.81347294922809], [-73.818200857166, 40.81348719171769], [-73.81845432163811, 40.81345990059012], [-73.81854113857591, 40.813450553177184], [-73.8187439158347, 40.81342608835089], [-73.81893043021248, 40.81338389527925], [-73.81916590911875, 40.813329386734786], [-73.81938042142716, 40.81327307513322], [-73.81946672338206, 40.81323957533384], [-73.81956231040617, 40.81321848131106], [-73.81968117345602, 40.813206276464676], [-73.81981638454968, 40.81317993315589], [-73.81990270879284, 40.813137581890224], [-73.82001227716462, 40.813092173502625], [-73.82006955436661, 40.81304473962543], [-73.82009596025118, 40.81297308472311], [-73.820107593403, 40.81286406966539], [-73.82039207280113, 40.81284401450982], [-73.8204130295436, 40.81297161162655], [-73.82047810145939, 40.813018766338544], [-73.82055521517324, 40.81301765507684], [-73.82049024326342, 40.81283644919561], [-73.82044595032455, 40.81284523139931], [-73.82040420162947, 40.81277435630248], [-73.82046249831421, 40.81275674407706], [-73.82043933256568, 40.812707141586564], [-73.82042068366609, 40.812710652675996], [-73.8202909114851, 40.812448452493214], [-73.82025827890693, 40.812453712664365], [-73.82024436122036, 40.81243067757398], [-73.8210651207081, 40.812203592467405], [-73.8210813694438, 40.81222663057411], [-73.82032355515894, 40.81243965219494], [-73.82042087950583, 40.81263807250523], [-73.82112969804922, 40.81244975686678], [-73.82113664072328, 40.812467471161504], [-73.82044179624151, 40.81265757783616], [-73.82046264635122, 40.8127018673558], [-73.82048115677324, 40.81274969286064], [-73.82053476598453, 40.81274269437526], [-73.82056254831423, 40.8128082378218], [-73.82052755199602, 40.812825885700505], [-73.82058709143999, 40.813019446608095], [-73.82084163522836, 40.81299808882692], [-73.82096282827042, 40.812985885107636], [-73.82105838069432, 40.81297718277624], [-73.8211306443835, 40.81296490242737], [-73.82117960695317, 40.81295258706845], [-73.82123791376058, 40.81293143382984], [-73.82125429431079, 40.812904906204345], [-73.82125259793112, 40.81290133764064], [-73.82124502768364, 40.812885418582646], [-73.82122873118149, 40.812880082275], [-73.82121710519647, 40.81287121357524], [-73.82121713848622, 40.81285882114425], [-73.82122415255714, 40.81284998084558], [-73.82124512277771, 40.81285001336021], [-73.82126609620738, 40.812848276143576], [-73.82130103684717, 40.81285187094222], [-73.82133130764494, 40.81285899812601], [-73.82136621348968, 40.81287498483091], [-73.82141253439893, 40.81287295247801], [-73.82139426361685, 40.81284139332036], [-73.82145723512515, 40.81281847796686], [-73.8214039410239, 40.81270863984824], [-73.82142259915253, 40.81270158730636], [-73.82151718570897, 40.81284732123544], [-73.82162030083451, 40.81283112168252], [-73.82168925261551, 40.81276832745507], [-73.82170205886845, 40.812753357807445], [-73.82166033195143, 40.81267363135489], [-73.8216953054991, 40.81266483427238], [-73.82175599067492, 40.812768536024244], [-73.82197234447621, 40.812750234169734], [-73.8221099447492, 40.81270088014639], [-73.82209149942082, 40.81262827121211], [-73.82212646342491, 40.81262301462178], [-73.82217757126159, 40.812679740811724], [-73.82228290825742, 40.81266149978754], [-73.82213413969171, 40.812366339030234], [-73.8221714521873, 40.81235400527647], [-73.82227114362938, 40.81254003518731], [-73.82225248068919, 40.81254885759145], [-73.82230863560132, 40.812658215078635], [-73.82263918380447, 40.812576007316075], [-73.82272558091357, 40.81250532945067], [-73.82285185101995, 40.81249017339933], [-73.82290738835799, 40.81247905532738], [-73.82298666429756, 40.81245793391372], [-73.82307756008963, 40.81244745252517], [-73.82318481038556, 40.81242106285956], [-73.82324776141463, 40.81240522676669], [-73.82332937582741, 40.8123805692376], [-73.82339468020483, 40.8123558854955], [-73.82345068402797, 40.81232410603195], [-73.8234551334144, 40.812309865024616], [-73.82346008270234, 40.81229402676287], [-73.82346484675018, 40.81225508855102], [-73.82346492657634, 40.812224993839294], [-73.82346265768916, 40.81220197747663], [-73.82347433550338, 40.81219137377909], [-73.82349762473447, 40.81219494957007], [-73.82352317906111, 40.81222331261734], [-73.82355798874956, 40.81227647361686], [-73.82356680906102, 40.81227648711492], [-73.8236208973294, 40.81227656987352], [-73.8237211278877, 40.812260790715214], [-73.82381669226842, 40.81223637848479], [-73.82387267753718, 40.81222207732535], [-73.82406387432498, 40.81216926152623], [-73.82421778221027, 40.81211992869825], [-73.82435772900804, 40.812063494629044], [-73.82453550423843, 40.81198695467631], [-73.82456543017311, 40.81193635175813], [-73.82459116681233, 40.81189567567694], [-73.82458438257436, 40.81181777383605], [-73.82459615811634, 40.81176999506733], [-73.82463817629335, 40.811739964149915], [-73.8247105112495, 40.811699359411215], [-73.82476875524654, 40.81170121718032], [-73.82479433788365, 40.81171895958462], [-73.82481754381085, 40.81175439904833], [-73.82483138373236, 40.811807527801534], [-73.82487321949907, 40.81184653686491], [-73.82494076044937, 40.81185726151931], [-73.82502001572509, 40.81184321961663], [-73.82505782162761, 40.81182991838929], [-73.82480905359992, 40.81143928159993], [-73.82483004135572, 40.81143223154372], [-73.82487876522168, 40.81151019708602], [-73.82492762380458, 40.81153682481714], [-73.82491588091877, 40.81157221235922], [-73.82507659171799, 40.811823313387535], [-73.82515527774036, 40.811795628141596], [-73.82524350467826, 40.81176134024005], [-73.82493047923725, 40.811336790727246], [-73.82492115121352, 40.81134031669775], [-73.82487938938073, 40.8112729838282], [-73.8249167005345, 40.811260649183524], [-73.82496314503149, 40.81131913746994], [-73.82494681756405, 40.811326193956454], [-73.82526664318074, 40.81175234789844], [-73.8254048364168, 40.811698642062694], [-73.82549075318562, 40.81165933042339], [-73.82520803886291, 40.81122391613742], [-73.82523137433176, 40.81120978979894], [-73.82551303817333, 40.811649133409404], [-73.82559378915639, 40.81161218535687], [-73.82565070298091, 40.81159229091601], [-73.82540857362882, 40.81116226158856], [-73.82539691090074, 40.811167553292265], [-73.82532502163363, 40.811038216045276], [-73.82535766859549, 40.811027643923794], [-73.82567229084775, 40.81158474373034], [-73.82572515052027, 40.811566265408715], [-73.8257664503311, 40.81155182855334], [-73.82559709277552, 40.81124043676879], [-73.82562508972042, 40.81122631740209], [-73.82559029284799, 40.81116784599779], [-73.8255739747615, 40.81117136261917], [-73.82549979349373, 40.81102785912637], [-73.82553710461548, 40.81101552428061], [-73.8256229491065, 40.8111537336798], [-73.8256066217491, 40.81116079042739], [-73.8256390924955, 40.81121748740355], [-73.82565308444987, 40.811212197522536], [-73.82571569203296, 40.811327358815056], [-73.82569235530275, 40.8113414852494], [-73.82581255718893, 40.811533704520265], [-73.82592714984648, 40.81148650139662], [-73.82576724152807, 40.81121591023323], [-73.82579755905094, 40.811205335639734], [-73.82595641099927, 40.811474448435035], [-73.82600062730944, 40.81146439488271], [-73.82604238618818, 40.81144946038733], [-73.8260807602305, 40.81142997662357], [-73.82611489720574, 40.81140637629834], [-73.82617764834127, 40.81129354241382], [-73.82618812289604, 40.81128842509033], [-73.82619956778117, 40.81128469564015], [-73.82621166726312, 40.811282456948845], [-73.82622408754939, 40.81128177077568], [-73.82623648599804, 40.81128265605054], [-73.82639533594696, 40.81128591951342], [-73.82653684940894, 40.81131957851293], [-73.82667984776913, 40.8113493955589], [-73.82686869217912, 40.8113825130345], [-73.82705935717021, 40.81140893138716], [-73.8272514312881, 40.811428593606124], [-73.82744450003561, 40.81144145726028], [-73.82763814676892, 40.811447494589565], [-73.82783195359622, 40.81144669256545], [-73.82802550227991, 40.8114390529188], [-73.82821837513981, 40.811424592135864], [-73.8284101559549, 40.811403341423336], [-73.8286004308618, 40.81137534664016], [-73.82878878924798, 40.8113406681993], [-73.82897482463858, 40.81129938093681], [-73.82915813557413, 40.811251573950464], [-73.82933832647683, 40.81119735040733], [-73.82951500850447, 40.811136827321235], [-73.82968780038979, 40.811070135299786], [-73.82985632926382, 40.81099741826285], [-73.8300202314608, 40.81091883313155], [-73.83017915330238, 40.81083454948945], [-73.83033275186162, 40.81074474921704], [-73.83048069570309, 40.81064962609816], [-73.83064261080314, 40.81053874295317], [-73.8307983038718, 40.81042282984924], [-73.83094750447907, 40.810302088145704], [-73.83108995347739, 40.810176727588285], [-73.83122540345188, 40.810046965945396], [-73.83135361915025, 40.8099130286293], [-73.83147437789091, 40.80977514830486], [-73.83158746995016, 40.80963356448454], [-73.83169269892545, 40.80948852311304], [-73.83178988207746, 40.80934027613904], [-73.83187885064571, 40.80918908107865], [-73.83195945014288, 40.80903520056658], [-73.83203154062217, 40.80887890190087], [-73.83208458645953, 40.80875510153657], [-73.83212949979134, 40.808629457389095], [-73.8321661704888, 40.80850227767013], [-73.8321945086448, 40.80837387435644], [-73.83221444479388, 40.80824456242455], [-73.83222593008111, 40.80811465907792], [-73.83222893638282, 40.80798448296906], [-73.8322234563745, 40.80785435341753], [-73.83220950354776, 40.807724589627036], [-73.83218711217783, 40.80759550990188], [-73.83215633723854, 40.80746743086719], [-73.83211725426618, 40.80734066669107], [-73.8320699591752, 40.80721552831496], [-73.83201456802169, 40.80709232269033], [-73.83195121671774, 40.80697135202641], [-73.83188006069922, 40.806852913048715], [-73.83180127454277, 40.80673729627106], [-73.83177925796888, 40.80673927729034], [-73.83175709637996, 40.80673880231731], [-73.83173526176998, 40.806735881467716], [-73.83171421916902, 40.80673057694927], [-73.83169221556712, 40.806721980905195], [-73.83167234011336, 40.80671078497406], [-73.83165511998523, 40.80669728611805], [-73.83164101192945, 40.806681842382105], [-73.83163039014659, 40.80666486339696], [-73.83162353636713, 40.80664679951423], [-73.8316206323777, 40.80662812986096], [-73.83162175520002, 40.80660934963126], [-73.8316268750495, 40.806590956951965], [-73.83163585612412, 40.80657343966979], [-73.83164846020705, 40.806557262412305], [-73.83166435298635, 40.80654285426413], [-73.83168311292104, 40.806530597385354], [-73.83170424242344, 40.80652081687644], [-73.8317271810562, 40.806513772154354], [-73.83177455546043, 40.80651765512604], [-73.83182212583493, 40.806515573081356], [-73.83186860015144, 40.806507582569296], [-73.83191271615236, 40.806493900615095], [-73.83195327563462, 40.80647489882502], [-73.83198917699312, 40.80645109329341], [-73.83201944514244, 40.80642313058547], [-73.8320432579997, 40.80639177017507], [-73.83205996881212, 40.80635786381794], [-73.83204659919782, 40.80624614815205], [-73.83202558717514, 40.806135113554596], [-73.83199698986499, 40.80602506167319], [-73.8319608849947, 40.80591629148391], [-73.8319173706858, 40.80580909847973], [-73.83186656518768, 40.80570377386722], [-73.83180860655449, 40.80560060377632], [-73.83174365227164, 40.805499868482066], [-73.83167187882668, 40.80540184164405], [-73.83164408179083, 40.80531084250263], [-73.83162387312709, 40.80521869493028], [-73.83161133284693, 40.80512576391069], [-73.83160651058895, 40.80503241752972], [-73.831615744777, 40.80493825599657], [-73.83168804452391, 40.804952878228214], [-73.83195319643457, 40.804942528187055], [-73.83221866123425, 40.80493916821706], [-73.83248411940502, 40.8049428028372], [-73.83274925293276, 40.80495342686368], [-73.83301374199807, 40.804971027912465], [-73.8332772691688, 40.80499558490685], [-73.83353951809008, 40.805027068090645], [-73.83380017317468, 40.80506544004523], [-73.8340589217921, 40.8051106542127], [-73.83431545195694, 40.805162656923386], [-73.83456945601598, 40.80522138492878], [-73.83482062883236, 40.80528676793842], [-73.83506866846948, 40.80535872716308], [-73.83531327587227, 40.80543717686122], [-73.83589256622265, 40.80566144824104], [-73.83643061278549, 40.80581642195253], [-73.83667080310849, 40.80587431601088], [-73.83691225676597, 40.80592909361344], [-73.83703585287998, 40.80598104011442], [-73.83715549184235, 40.80603808520558], [-73.83727080965522, 40.806100056110594], [-73.83735487277579, 40.80615004018267], [-73.83743612112924, 40.80620264259155], [-73.83800223017066, 40.80696132815477], [-73.83802727502474, 40.806984940091354], [-73.8380469598718, 40.80701132094272], [-73.83806076514087, 40.80703977484668], [-73.83806795307203, 40.807067107027194], [-73.83806970692224, 40.80709494793898], [-73.83806598715289, 40.80712267695709], [-73.83805687719341, 40.807149676200964], [-73.83804258001825, 40.80717534491687], [-73.83803522099961, 40.80730290612347], [-73.8380372510487, 40.80743058091789], [-73.83804866402264, 40.807557969956754], [-73.83806942412893, 40.80768467534121], [-73.83809946697961, 40.80781030223833], [-73.83813869819386, 40.80793445799861], [-73.83818699604151, 40.80805675426877], [-73.8382442091364, 40.80817681009697], [-73.83831015916998, 40.80829425002933], [-73.83838464019071, 40.80840870719504], [-73.83848257584289, 40.80849122024229], [-73.83852374282493, 40.8085951328486], [-73.83853400696889, 40.80865045311928], [-73.83854418408738, 40.80870531022513], [-73.83861835811027, 40.80886277004793], [-73.83868023517216, 40.80896671211981], [-73.83873795606092, 40.80907694238961], [-73.83881633282607, 40.80920923009083], [-73.83889477021641, 40.80931634198769], [-73.8389152291753, 40.80942022504676], [-73.83897298815991, 40.809514719474485], [-73.83908458592941, 40.80961243627587], [-73.83918782313233, 40.80974161221812], [-73.83929500564086, 40.80995261819515], [-73.83938574944685, 40.8101101008016], [-73.83944757694832, 40.81023607121156], [-73.83957150390646, 40.8103747178132], [-73.83966635991298, 40.810544794157764], [-73.83975711385058, 40.81069912905494], [-73.83986029409101, 40.81085348104485], [-73.83996760888154, 40.81101098591626], [-73.84000050350662, 40.8111117391841], [-73.84008711700521, 40.81126606807056], [-73.84017805294744, 40.811344872467366], [-73.84025952898116, 40.8114430368914], [-73.84029763042703, 40.811571630005304], [-73.84030147044926, 40.81169752058669], [-73.84028871196404, 40.81183597523997], [-73.84026339022252, 40.812031061945056], [-73.84028789614791, 40.81217586311286], [-73.84032899386818, 40.81231124600105], [-73.84039504843176, 40.812402604566756], [-73.84041550491237, 40.81247929982902], [-73.84043199520461, 40.81254112811847], [-73.84040031307201, 40.81263430143549], [-73.84039027980772, 40.812663808012246], [-73.84034035406837, 40.81275500628137], [-73.84029454885903, 40.81285564990053], [-73.84028603821864, 40.812950051059545], [-73.84030654525426, 40.81303505222116], [-73.8403095227641, 40.81304970479806], [-73.84032701577196, 40.81313578806891], [-73.84026867690187, 40.8132804750118], [-73.84017126693041, 40.81341311815477], [-73.84014400211656, 40.81345024593872], [-73.84002346996, 40.8136200230888], [-73.83993631613669, 40.81368913875994], [-73.83988754547781, 40.81375800402594], [-73.83987397095441, 40.8137771711536], [-73.83985463009746, 40.813789923199934], [-73.83974535687518, 40.81386196575971], [-73.83966236691798, 40.81392164504031], [-73.83962901684902, 40.814009717774574], [-73.83964836001682, 40.81408176316949], [-73.8396577547437, 40.814116760046424], [-73.83964922798799, 40.81421745603259], [-73.83959097056858, 40.81432752377343], [-73.83956806564427, 40.814491248804075], [-73.8395309152364, 40.815183454974985], [-73.8396174257576, 40.81538184391267], [-73.8396044608103, 40.815605271036794], [-73.83953766684768, 40.815819182111085], [-73.83959385575336, 40.81598823283838], [-73.83963940814726, 40.81604397588822], [-73.83967772293016, 40.816102777914374], [-73.83970844688206, 40.81616409798452], [-73.83973129719243, 40.81622737208514], [-73.83974606475891, 40.816292018214845], [-73.8397526121542, 40.81635744206133], [-73.83974683007641, 40.81638162194286], [-73.83974355921865, 40.816406071751366], [-73.83895834000002, 40.81636899399993], [-73.83696070299992, 40.81634054099991], [-73.83667856399991, 40.81758876299995], [-73.83666068699988, 40.81797116099989], [-73.8365014589999, 40.81871065899989], [-73.83639638699991, 40.81889136599989], [-73.83614239499985, 40.81919890299987], [-73.83613216399988, 40.82033578499992], [-73.83613095099999, 40.8209565959999], [-73.83612954299997, 40.821677827999984], [-73.83612811599986, 40.82240889399991], [-73.83612598999986, 40.82349743199991], [-73.83619033399988, 40.824205449999916], [-73.83626003599986, 40.82497188099989], [-73.8365046079999, 40.826403116999906], [-73.83661819699992, 40.82711414899993], [-73.83664057999991, 40.827215044999896], [-73.83678907899998, 40.827884395999895], [-73.83685097699993, 40.828163394999876], [-73.83686087299982, 40.82819847399989], [-73.83691608899997, 40.828451825999885], [-73.83703204399986, 40.828809450999906], [-73.83707424299983, 40.82892307199996], [-73.83709061399992, 40.82896714699996], [-73.83715124799993, 40.82895951599989], [-73.83747911899988, 40.82892810099993], [-73.8403229479999, 40.828823950999926], [-73.84058337699986, 40.82880567999992], [-73.84102340199989, 40.82877480599996], [-73.84122680199987, 40.82876053399988], [-73.84143734599988, 40.82874131099985], [-73.8415995109999, 40.828726494999906], [-73.84223265299995, 40.828652630999954], [-73.84228428387955, 40.828646604991356], [-73.84229237322901, 40.828699807532125], [-73.84230753757464, 40.828799468721485], [-73.84231435750092, 40.828922980996836], [-73.84233077477266, 40.8290739624537], [-73.84223013648109, 40.82983069124885], [-73.84191892734383, 40.83062975863897], [-73.84188643774428, 40.83071317653441], [-73.8408458452518, 40.831748524329065], [-73.84075047236948, 40.831779863609995], [-73.84024396311139, 40.83220402332119], [-73.83943411928455, 40.83298338028858], [-73.83918095199299, 40.833156118752484], [-73.83899809143148, 40.833382455358056], [-73.83888164699847, 40.833601154233584], [-73.83885232384739, 40.83369066784824], [-73.83882717280252, 40.83381022387015], [-73.83881438332645, 40.833958119303475], [-73.83883473556496, 40.83410920866094], [-73.83887989543763, 40.83428236293754], [-73.83892926676718, 40.834427198085116], [-73.8389413081543, 40.834587717131384], [-73.83893684463153, 40.83471988981303], [-73.83895304620648, 40.83487412044188], [-73.83896515672897, 40.83500631575781], [-73.83899797777033, 40.83514168759397], [-73.83904319206685, 40.83529281108476], [-73.8390594007588, 40.835443894584465], [-73.839046635205, 40.83558234988273], [-73.83909187186077, 40.835724033221126], [-73.83911300797067, 40.83591795498703], [-73.83920869951596, 40.83656934682847], [-73.83923392302755, 40.83674104063165], [-73.83918642082618, 40.83710741571121], [-73.83926310309631, 40.83720488523882], [-73.83922069428176, 40.837392472808716], [-73.8392458501793, 40.8375925459246], [-73.83928722789221, 40.83783158616525], [-73.83928671894768, 40.83804224481621], [-73.83921295808717, 40.83824771181022], [-73.83920223885042, 40.83827756978396], [-73.83915984296951, 40.83845984582744], [-73.83919255217855, 40.83871362802882], [-73.83936180401575, 40.83880178485791], [-73.83958953782852, 40.83879848591282], [-73.83972014457761, 40.83956540599305], [-73.83956700299987, 40.84054722799986], [-73.83936452799996, 40.84087706699995], [-73.83880813300001, 40.84076703399992], [-73.83849424699997, 40.840699219999934], [-73.83839056299992, 40.840676813999934], [-73.83830524999978, 40.84064271899997], [-73.83821037099989, 40.84060479799988], [-73.8381237629999, 40.84057018199993], [-73.83771374899992, 40.840406309999885], [-73.83768901199981, 40.8408111819999], [-73.83767001599998, 40.841121907999856], [-73.83736984299993, 40.84204138399985], [-73.83714579999987, 40.8427276329999], [-73.83688925199996, 40.84337478799992], [-73.83676643799986, 40.84338596999988], [-73.83582911399996, 40.84347129499987], [-73.83474813499988, 40.84350280699991], [-73.83383204499994, 40.84351641299992], [-73.832940765, 40.8435191599999], [-73.83200795799995, 40.843630393999916], [-73.83108377199986, 40.84374456899991], [-73.83043186399985, 40.84385798399996], [-73.82965549999982, 40.84408912299994], [-73.8293154839999, 40.84419272599995], [-73.82870971100007, 40.84437387199991], [-73.8284527429999, 40.84445071699992], [-73.8275753599998, 40.84473187499988], [-73.82674715099992, 40.84507891799988], [-73.82611300699996, 40.84533869399991], [-73.82619627799995, 40.84381693599995], [-73.82624132999996, 40.84299359399991], [-73.82608382099987, 40.841371456999894], [-73.82602634299997, 40.84102446399987], [-73.825986869, 40.84067575199988]]], [[[-73.83960098545435, 40.83567071869858], [-73.83961672561374, 40.835667752627316], [-73.83962599488439, 40.83569971218477], [-73.83960850400152, 40.83570805084221], [-73.83960098545435, 40.83567071869858]]], [[[-73.83979488581072, 40.835619207246324], [-73.83967394092852, 40.835601208664066], [-73.8396148723857, 40.83561056836579], [-73.83961491790704, 40.835591685275496], [-73.83969886642244, 40.83557527991654], [-73.83968659420826, 40.83550917341248], [-73.83959955569631, 40.83551849422488], [-73.83959649407355, 40.83549960798568], [-73.83966798828091, 40.835492625480214], [-73.83966494873249, 40.83546429744529], [-73.83957792735119, 40.83546653685204], [-73.8395779671898, 40.83545001483886], [-73.83966499992795, 40.83544305394731], [-73.83965271639111, 40.83538166942538], [-73.83956877446222, 40.83539571434488], [-73.8395563898934, 40.83537681396285], [-73.83963722333964, 40.835362764227675], [-73.83963111649646, 40.83531791035987], [-73.83954719172159, 40.835324873874214], [-73.83954412443111, 40.83530834753788], [-73.83961561208373, 40.83530372664378], [-73.8396033299459, 40.83524234044183], [-73.83953492671942, 40.83525640744828], [-73.83953185811168, 40.83523988110952], [-73.8396189076475, 40.83522584003561], [-73.83960659574578, 40.83517625553322], [-73.83951955898183, 40.835185576287614], [-73.8395133712029, 40.83517376593597], [-73.83960353479867, 40.835157369294045], [-73.83958501303327, 40.8351054162397], [-73.8395104064141, 40.83511475377276], [-73.83950733344858, 40.835100587842255], [-73.8395943764547, 40.835088907192976], [-73.83957586742348, 40.835032232653354], [-73.83949815155427, 40.83504156635618], [-73.83949818571932, 40.8350274042466], [-73.8394577753272, 40.83503206833108], [-73.83945783797591, 40.83500610569194], [-73.83950134541065, 40.83500616624555], [-73.83950138527082, 40.83498964305826], [-73.83956666839917, 40.834980293734695], [-73.83955747137699, 40.834928353644905], [-73.83948288203746, 40.83493060977055], [-73.83948291051087, 40.834918808068124], [-73.83955130770877, 40.834907101499226], [-73.83954830742066, 40.83486225144243], [-73.83947681454364, 40.83486923220015], [-73.83946132669129, 40.83484796824395], [-73.83961986353515, 40.83482930565607], [-73.839619931812, 40.834800981938365], [-73.83947693983029, 40.834817305077515], [-73.83947386622093, 40.83480314031713], [-73.8396137738246, 40.83477736989052], [-73.83960141648534, 40.834746668476726], [-73.83947085592368, 40.83476301057198], [-73.83946467367059, 40.83474883863638], [-73.83960456474958, 40.83473015128698], [-73.83959530972285, 40.83470181460232], [-73.83946474201342, 40.83472051491829], [-73.83944316529578, 40.83464731518525], [-73.83941519123677, 40.834649636151376], [-73.8394122031086, 40.834600065107395], [-73.83948681004493, 40.83459072830833], [-73.83948374192397, 40.83457420079436], [-73.83959266810578, 40.83456047582325], [-73.83964780499996, 40.83484114599992], [-73.83964694899996, 40.83484286999988], [-73.83962906799987, 40.834878885999935], [-73.83961961899993, 40.834930799999924], [-73.8396350259999, 40.834985108999916], [-73.83963165699996, 40.83509367899995], [-73.83964701399998, 40.83516923099997], [-73.83967775499997, 40.83530853299991], [-73.83970551900002, 40.835393542999896], [-73.83972396100002, 40.835478541999876], [-73.83974862799982, 40.83555882699993], [-73.83979488581072, 40.835619207246324]]]]}}, {\"id\": \"208\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 209, \"Shape_Leng\": 0.0306953113473, \"Shape_Area\": 3.88703351209e-05, \"zone\": \"Seaport\", \"LocationID\": 209, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00499805099996, 40.71201189699987], [-74.00490674599992, 40.711972278999944], [-74.00442860399994, 40.7117862259999], [-74.00418143899992, 40.7116871419999], [-74.00382358799995, 40.71155186199991], [-74.00366118299998, 40.71148728299994], [-74.00336407499992, 40.71126941399996], [-74.0033244169999, 40.711237482999884], [-74.0025796699999, 40.71063789699985], [-74.00223869799997, 40.71036393199988], [-74.00173688899993, 40.70996016999993], [-74.00155908199999, 40.70981710799992], [-74.00147140299997, 40.709746553999864], [-74.00134439799994, 40.709646292999906], [-74.00055004899994, 40.709017916999905], [-74.00026566699997, 40.70879294999988], [-74.000191031, 40.708733911999865], [-74.000123591, 40.708684518999924], [-74.00000765499989, 40.70859961899986], [-73.9999293639999, 40.708536816999896], [-73.99981193899988, 40.70844261399988], [-73.99956273399984, 40.70823648299992], [-73.99944281399992, 40.70813655699987], [-73.99939954999992, 40.70810033799996], [-73.99936195599989, 40.70807228699991], [-73.99928734399988, 40.708016626999886], [-73.99928731099999, 40.708016602999926], [-73.99919451154408, 40.707947376175724], [-73.99925945470767, 40.7079140213104], [-73.9993263267542, 40.70787967649388], [-73.99938225177355, 40.707841549321465], [-73.99942129138608, 40.70781893705527], [-73.99946033075176, 40.707796326451025], [-73.99952324941056, 40.70776262865822], [-73.99951211364748, 40.70774870660909], [-73.99949062474623, 40.707721845224945], [-73.99953839659445, 40.70769347002565], [-73.99955121325492, 40.70769435277648], [-73.99957149513774, 40.70771719818587], [-73.99960948039234, 40.70775997317624], [-73.99988297085214, 40.70762408603529], [-73.99994855949784, 40.707591496371016], [-73.99994505655117, 40.707582626983275], [-73.99994855952315, 40.70755869155377], [-73.99995671262306, 40.70753119885585], [-73.99997222648798, 40.70750639882202], [-73.99999645016015, 40.707491647898465], [-74.00013868859138, 40.70740505440754], [-74.00023380525755, 40.70734714248687], [-74.0004002783203, 40.70726027957068], [-74.0005524793205, 40.70718427592291], [-74.00077175219094, 40.707080148200696], [-74.00084973367683, 40.70704311913907], [-74.00088259097619, 40.70702525330048], [-74.00118685262835, 40.70685982577176], [-74.0011313057719, 40.70679905068455], [-74.00122367538816, 40.70674526164777], [-74.00123213691963, 40.70674033693093], [-74.00122701299958, 40.706734869906434], [-74.0011603902257, 40.70666372083431], [-74.00117521610974, 40.70664874526916], [-74.00116913341533, 40.70663949766774], [-74.00117643567683, 40.70663505879534], [-74.0011760207967, 40.70662757435901], [-74.00098276691729, 40.70641671299287], [-74.00140533024056, 40.706180135207944], [-74.00140666691955, 40.706179388020644], [-74.00140581495832, 40.706178612883114], [-74.0005829704798, 40.7054339336489], [-74.0005924020934, 40.7054100794135], [-74.00082417549918, 40.70526711251571], [-74.00143661179398, 40.70487217770521], [-74.00145437409415, 40.704873303215976], [-74.0014608578281, 40.704873717906025], [-74.00183509991501, 40.70521225891118], [-74.00205517713985, 40.70541134993458], [-74.002061425319, 40.7054170053203], [-74.0020628214227, 40.70541607717396], [-74.00212278743204, 40.70537900283618], [-74.00215818324273, 40.70535711905917], [-74.00256965079724, 40.705523579761724], [-74.00295417200002, 40.706027114999934], [-74.00296019099989, 40.70603499799994], [-74.0029653009999, 40.706041688999925], [-74.00299474599991, 40.70608024599991], [-74.0030261529999, 40.70610867499988], [-74.00306957, 40.706142802999885], [-74.00316110399999, 40.706232014999884], [-74.00360537999994, 40.706780592999905], [-74.00372234799997, 40.70693234399992], [-74.00392127499991, 40.70723974499993], [-74.00405660199985, 40.70743678099992], [-74.00432857299997, 40.70782349599986], [-74.00478782699994, 40.70840189299991], [-74.00505672799989, 40.70878876899994], [-74.00526217199995, 40.708920710999955], [-74.00551252899992, 40.709093407999916], [-74.00643549199992, 40.70955732199991], [-74.00705335799996, 40.7098719509999], [-74.00774466499993, 40.71022077299991], [-74.00906264699988, 40.71088670399993], [-74.00860085399992, 40.711392239999945], [-74.00834760999993, 40.71135854599987], [-74.00796787799999, 40.71144733199986], [-74.00746749399993, 40.71156690299989], [-74.00738921499999, 40.711642268999874], [-74.00698584099997, 40.711737498999845], [-74.00607752399999, 40.7119623529999], [-74.00589174999999, 40.71200715999994], [-74.00573499299995, 40.71197868799989], [-74.00543986499997, 40.7120785329999], [-74.00538198399992, 40.712076742999876], [-74.00522812799994, 40.71207345199985], [-74.00518548699988, 40.71207123299992], [-74.00510989899993, 40.712055593999914], [-74.00499805099996, 40.71201189699987]]]}}, {\"id\": \"209\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 210, \"Shape_Leng\": 0.114851983372, \"Shape_Area\": 0.000453155131253, \"zone\": \"Sheepshead Bay\", \"LocationID\": 210, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.92759936700004, 40.5950607789999], [-73.92653305100005, 40.59410431499987], [-73.9254714779999, 40.59314878799995], [-73.92440413399991, 40.59218425199987], [-73.92405498699993, 40.591871702999875], [-73.92363677299986, 40.5914953699999], [-73.92332177899988, 40.591211915999935], [-73.922307925, 40.590308207999875], [-73.92095992099988, 40.58909565099986], [-73.91972510799992, 40.587987277999886], [-73.91819087799986, 40.58660685499996], [-73.91809293895267, 40.58657062112203], [-73.91812555866335, 40.586507597473656], [-73.91805571865743, 40.58643685311621], [-73.91808352411303, 40.58643247412283], [-73.9181149830405, 40.58645427989567], [-73.91813980819195, 40.58644586716909], [-73.91817127183086, 40.58641955110692], [-73.9181758618163, 40.58639521720816], [-73.91820323428233, 40.5863871235826], [-73.91823846308047, 40.58640794224147], [-73.91826250403696, 40.58640428626349], [-73.91831572119233, 40.586390681349776], [-73.91839025484447, 40.586353648155765], [-73.91843893816063, 40.58632239140944], [-73.91848607837932, 40.58630967637502], [-73.91855338716753, 40.5862999345243], [-73.91858038529267, 40.58626338684265], [-73.91862753793326, 40.58624024037158], [-73.91871030551792, 40.586191673708484], [-73.91877052073922, 40.586167328699766], [-73.91881005548656, 40.586159244699886], [-73.91884807293582, 40.586147681871296], [-73.91888988807571, 40.58613479655335], [-73.91894085082454, 40.586107184130526], [-73.91896974793147, 40.586095615497186], [-73.91900017409613, 40.58607709462553], [-73.91904637464812, 40.58605874083082], [-73.91919033172351, 40.5859624941477], [-73.91923435968188, 40.58600888150374], [-73.91927750147164, 40.58601472693519], [-73.9193270820224, 40.586014741486196], [-73.91940300812695, 40.586082012771534], [-73.91943489358087, 40.58611332695053], [-73.9195054592814, 40.586133692869545], [-73.91951394035418, 40.58611454064769], [-73.91961116419868, 40.58616907989991], [-73.91973129637576, 40.586135554300206], [-73.91983470833878, 40.58610085810948], [-73.9198930012924, 40.58609162299616], [-73.91997613293829, 40.586059235165], [-73.92005215131567, 40.58605001667059], [-73.92013922923803, 40.5860569452636], [-73.92013970735708, 40.58602500929304], [-73.92018236154185, 40.58602503893172], [-73.92018236611428, 40.586021213232726], [-73.92018863964579, 40.58602026183536], [-73.92018491195509, 40.585990609887176], [-73.92004940419412, 40.586006774240296], [-73.92004691580856, 40.58598860025634], [-73.92017489750616, 40.585972430677764], [-73.92017117347483, 40.585939908785015], [-73.92020630052706, 40.585938976926904], [-73.92021129118353, 40.58596193460525], [-73.92025268900592, 40.58596291960334], [-73.92025517678752, 40.58598109358259], [-73.92020248303365, 40.58598392644289], [-73.92020745883667, 40.586019317475234], [-73.92021749158332, 40.58602219388615], [-73.92021874083053, 40.58602697788635], [-73.92027770472494, 40.58602606138264], [-73.92027733491835, 40.58606377702658], [-73.92029535641579, 40.586065251893686], [-73.9203607206783, 40.586067614317194], [-73.92040498231091, 40.586089779645754], [-73.92047021116144, 40.58603408174129], [-73.92055080816567, 40.58600979932682], [-73.92062531128488, 40.58599594403365], [-73.92071501761441, 40.58598209801122], [-73.92082156941888, 40.58599577420628], [-73.92082145662427, 40.58595783440858], [-73.92086842926197, 40.585921080321086], [-73.9207586767622, 40.58581296160033], [-73.92076503207028, 40.5858097375235], [-73.92072837493505, 40.58577797077197], [-73.9206973025787, 40.58579731724048], [-73.9206698086596, 40.585773626519675], [-73.92073901660137, 40.58573009669357], [-73.92076368725981, 40.585753785451], [-73.92073896679761, 40.58577205976687], [-73.92077774028736, 40.58580382847679], [-73.9207819787998, 40.58580060411212], [-73.92089145806791, 40.58590306230562], [-73.92092948644265, 40.58587330627243], [-73.92090217579444, 40.585830406473775], [-73.92097520370773, 40.585778304978994], [-73.92098905633135, 40.585769048620286], [-73.92091104290725, 40.58565005223212], [-73.92087843027372, 40.58559946336436], [-73.92090603836381, 40.5855905815336], [-73.92094410567492, 40.58564899408573], [-73.92101218266103, 40.585753595858236], [-73.92101224379708, 40.58575355586073], [-73.92103757896898, 40.585736626332256], [-73.92108167330595, 40.585728545186846], [-73.92110266358986, 40.585741450321855], [-73.9211770776566, 40.58574646780548], [-73.92128230738994, 40.585749542744566], [-73.92147080849864, 40.58574619516197], [-73.9215142802177, 40.58578133752638], [-73.92153288620224, 40.58576690098837], [-73.9217626135607, 40.58579970544528], [-73.92178240271222, 40.585776540719046], [-73.92175809215335, 40.58576609277944], [-73.92178148758434, 40.58573348620472], [-73.92181131953878, 40.585747585682746], [-73.92180978578286, 40.585759174325084], [-73.92183560839585, 40.58577657692656], [-73.92189955122011, 40.58569317567347], [-73.9219481743886, 40.58571175204423], [-73.92184819107767, 40.5858384214364], [-73.9218669875511, 40.58583203777862], [-73.9218992855612, 40.58584461338751], [-73.92188902702776, 40.585821125656516], [-73.92211851289852, 40.585810514152975], [-73.92216035285345, 40.58585611721177], [-73.92230460012115, 40.5859925662072], [-73.92230200146534, 40.586018025739904], [-73.92235337377305, 40.5860239365623], [-73.92246128010449, 40.58601617525401], [-73.92255248710735, 40.58600938129622], [-73.92256267742543, 40.5860087299414], [-73.92262827740261, 40.58600453563984], [-73.92271691647166, 40.585996761618], [-73.92282358156667, 40.585950807953715], [-73.92284926622335, 40.58595572141795], [-73.92296409826356, 40.58589445706378], [-73.92302013749718, 40.58592645722694], [-73.92313312983316, 40.58596276647311], [-73.92334118079789, 40.585995220555404], [-73.92353515890866, 40.585977723872205], [-73.92369833045592, 40.585941599249466], [-73.92385770220568, 40.585858466135356], [-73.92389501884094, 40.58580071381235], [-73.92392584582923, 40.5858007342269], [-73.92397204036718, 40.58584189461741], [-73.92401693919693, 40.58589284589225], [-73.92404262147038, 40.5858997188474], [-73.92406317839455, 40.585894835717646], [-73.92406705274973, 40.58587721116697], [-73.92404909560508, 40.58585467597212], [-73.92400933327512, 40.585804706311194], [-73.92401064509494, 40.58578022576851], [-73.92402182009499, 40.58577353533212], [-73.92429576856087, 40.5856827589019], [-73.9243231453673, 40.585705408319676], [-73.92456754948067, 40.58564637736859], [-73.9246794528378, 40.58563774645343], [-73.92475709799683, 40.58563257509932], [-73.92480731263153, 40.58565349922682], [-73.92483669450533, 40.585683013012336], [-73.924860511513, 40.58571339815731], [-73.92493840743964, 40.58570024105191], [-73.9249933142654, 40.585712845971294], [-73.92507664964324, 40.58575987157345], [-73.92520361514241, 40.58580627115536], [-73.92522694757393, 40.585839824560125], [-73.9252225445941, 40.58587904435365], [-73.92528378572597, 40.58589815289526], [-73.92546644505285, 40.58609236910719], [-73.9254183568754, 40.58619651406238], [-73.92554277632239, 40.58619329007061], [-73.92559446148006, 40.58625289064708], [-73.92564689237776, 40.58633474880302], [-73.92570847285627, 40.58640268462252], [-73.92570600948345, 40.58646749161629], [-73.92579061919318, 40.586458447268434], [-73.9259343533844, 40.58657518269955], [-73.92600279804184, 40.586630936862214], [-73.92598398427617, 40.58668024004204], [-73.92607581839894, 40.586679729897554], [-73.9261305753202, 40.58672328774673], [-73.92621271663319, 40.5867842729822], [-73.9262880091751, 40.586843513469674], [-73.92628231953488, 40.58688625846585], [-73.92630373489585, 40.58690211797234], [-73.92637018571116, 40.58687142093822], [-73.92647284761263, 40.58696201521453], [-73.92657322890143, 40.587049125842086], [-73.92649781833158, 40.58709608235675], [-73.92653260420076, 40.5871209110176], [-73.92658537503537, 40.58713308560274], [-73.92670333122614, 40.587204091517165], [-73.92673977671522, 40.587188506716075], [-73.9268423273604, 40.58728968995381], [-73.92677874563445, 40.587330385064845], [-73.92683150549894, 40.587373622998186], [-73.92687946572465, 40.58740958238608], [-73.92695782460427, 40.58745196857885], [-73.92700073232174, 40.58750153462451], [-73.92706884928583, 40.5875666615426], [-73.92708799146784, 40.58755640263128], [-73.92709462919922, 40.58756045703374], [-73.92712785833076, 40.58754022531675], [-73.92694208311515, 40.58736896953345], [-73.92696733834302, 40.58735379586573], [-73.92718761078223, 40.587559503277795], [-73.92717032872173, 40.587571644572165], [-73.92714112938128, 40.587551373264745], [-73.92710524303517, 40.58757160329974], [-73.92711984469311, 40.587579713111076], [-73.9270993562571, 40.587591037163], [-73.92711083019412, 40.58761769150011], [-73.92715928733517, 40.58766114718808], [-73.92720774188899, 40.587722759186676], [-73.92725820254594, 40.58770577334006], [-73.92729373170549, 40.58768666129003], [-73.92734003426652, 40.587656890786164], [-73.92738589449549, 40.58763029299064], [-73.92740972113143, 40.587615721094], [-73.92743892751554, 40.58759785829185], [-73.9274700309266, 40.5875788350308], [-73.92750198619638, 40.587559290779936], [-73.92753045676024, 40.587541877641584], [-73.92756029198215, 40.58752362921737], [-73.92757304438481, 40.58751583005147], [-73.92762064976755, 40.587486713955094], [-73.92765006933621, 40.587466693287276], [-73.92766845967759, 40.58735941719748], [-73.92768287747319, 40.58736160013402], [-73.92766599128431, 40.58747501343975], [-73.9276119876809, 40.58750759321528], [-73.92738744939162, 40.58764812864987], [-73.9273705958315, 40.587658677208076], [-73.92735333435394, 40.58766988322635], [-73.92734349185156, 40.58767564118687], [-73.92732601574134, 40.5876865790183], [-73.9272994485146, 40.58770320726212], [-73.92727720149654, 40.587738464989634], [-73.92724750521369, 40.58775327471978], [-73.92724312575965, 40.58776905070255], [-73.92731915293669, 40.5878499299063], [-73.92726483869893, 40.58788299850715], [-73.92727932188905, 40.58790916939882], [-73.9273568092889, 40.588022176276155], [-73.92739188174689, 40.587998800117184], [-73.92746402494045, 40.58795453583472], [-73.92749456893686, 40.58793579550671], [-73.9275058017439, 40.5879289015562], [-73.927568469414, 40.58789045187032], [-73.92758002886069, 40.587883258408326], [-73.92764640357963, 40.587842633804335], [-73.92765584309352, 40.587836842728464], [-73.92772506988965, 40.587794365994355], [-73.92773463890906, 40.58778849541703], [-73.92779832443665, 40.587749419853346], [-73.92794598351136, 40.58765882000514], [-73.92787426020541, 40.58758815885385], [-73.92787393831544, 40.58757550348134], [-73.9279162574105, 40.587548736392], [-73.92807906817849, 40.58770246052319], [-73.92803041429266, 40.587731484346115], [-73.92795555226466, 40.58766660355717], [-73.92785030679849, 40.587732985371986], [-73.92782113152668, 40.58775138740754], [-73.92767581765723, 40.58784303881275], [-73.92767538027489, 40.5878433141234], [-73.92760667965281, 40.587886644721095], [-73.92759326264297, 40.58789506450303], [-73.92746925071934, 40.587973060512624], [-73.92747502650556, 40.58797967687832], [-73.92736935753528, 40.58804459880297], [-73.92743823840641, 40.588147934742985], [-73.92739013146551, 40.58817864281224], [-73.92740450151646, 40.58820650448981], [-73.92742552260684, 40.58821939139702], [-73.92744551755536, 40.58820630423171], [-73.92750396203982, 40.58822136013031], [-73.92763603930852, 40.588140492014944], [-73.92777244932779, 40.58805697180035], [-73.92790482500476, 40.58797592078823], [-73.92789056855496, 40.587960335629944], [-73.9279017574099, 40.58795340927988], [-73.92802185276429, 40.588068756350886], [-73.92800676518989, 40.58807699939624], [-73.9279231804831, 40.58799618727679], [-73.92785876092728, 40.5880345564793], [-73.9278484736784, 40.58804068410847], [-73.9277909208587, 40.58807496400387], [-73.92778443159247, 40.588078829016816], [-73.92776892978343, 40.588088061534094], [-73.92769483165648, 40.58813219695235], [-73.92768070085903, 40.58814061294461], [-73.92764121499813, 40.588164131325144], [-73.92755543879122, 40.58821522114957], [-73.92756724259812, 40.58822895259825], [-73.92755751586084, 40.58823550272868], [-73.92754721531419, 40.588227906817906], [-73.92754016723876, 40.5882331438653], [-73.92752569167634, 40.588221654449846], [-73.92751741750983, 40.588225965625256], [-73.9275225816279, 40.588232089736294], [-73.92751674803628, 40.58823639156172], [-73.92752878258517, 40.58824788949042], [-73.9275032776782, 40.58826938434163], [-73.92747241644948, 40.5882891175855], [-73.92745467790976, 40.588299564074035], [-73.92748612581521, 40.588320574518725], [-73.92743489681918, 40.588369014793585], [-73.92753942773564, 40.58849382883325], [-73.92755035972286, 40.58851353182066], [-73.92763372065457, 40.58845892052031], [-73.92776143639104, 40.58866060975881], [-73.92711186317565, 40.589081472259394], [-73.92695102774618, 40.589182075210665], [-73.92665299278437, 40.58936849665159], [-73.92662861634162, 40.58938412866682], [-73.9266528016335, 40.58941087961657], [-73.9263611882989, 40.5895927189504], [-73.9260602421983, 40.58978037770557], [-73.92575347866315, 40.589971659927386], [-73.92544308131156, 40.59016520731762], [-73.92513959885657, 40.59035444080562], [-73.92483070193938, 40.59054704767478], [-73.92454610016368, 40.590739259052654], [-73.92424735748756, 40.59094101895615], [-73.92395467165778, 40.591138687336496], [-73.9239529485009, 40.59113985052502], [-73.92407257175674, 40.5912486814702], [-73.9241996339265, 40.59136428018873], [-73.92425997013223, 40.59132144670316], [-73.9242824753091, 40.59133793033905], [-73.9243502582044, 40.59129226862413], [-73.9243635723439, 40.59130074180863], [-73.92443135419055, 40.59125592422532], [-73.92444799020575, 40.591272016976234], [-73.92451798743379, 40.5912314344323], [-73.92452686320941, 40.591238211798434], [-73.9246012993856, 40.591199325003174], [-73.9246568634645, 40.591157885874786], [-73.92463689474579, 40.59114179046576], [-73.92477134199518, 40.591056389011506], [-73.92475692077481, 40.59104452971626], [-73.92491025194087, 40.5909515213958], [-73.92492023633986, 40.590959145814324], [-73.92497245564526, 40.59092870884722], [-73.92495914742958, 40.59091515674973], [-73.92508693133408, 40.59082975057345], [-73.92509358650521, 40.59083567992229], [-73.92536357858715, 40.59067334024729], [-73.92588801019414, 40.59035203456001], [-73.92587504247338, 40.590337151599265], [-73.92602962522365, 40.59024379563498], [-73.92615947414338, 40.59016537791108], [-73.92635608508674, 40.59003791473551], [-73.92665232672293, 40.589845861094936], [-73.92696173942281, 40.589645266707684], [-73.92725962405339, 40.5894521425307], [-73.92777636022275, 40.58910220009743], [-73.92776465896623, 40.58907718211487], [-73.92815905123096, 40.58882524958995], [-73.92836271266806, 40.58896901609184], [-73.92836783373292, 40.588965824834474], [-73.92841885809028, 40.5889340350981], [-73.92841297161772, 40.58892903419734], [-73.92839694020006, 40.58894062377454], [-73.92836947692139, 40.5889182156869], [-73.92839594752826, 40.58889803316905], [-73.92837977293736, 40.58888637359861], [-73.92836438201132, 40.58888289346586], [-73.92835661259966, 40.588889246193986], [-73.92821616994844, 40.588783912903885], [-73.92822539531493, 40.58877625470065], [-73.9283361999493, 40.58886253531396], [-73.92835866230406, 40.588865431329005], [-73.92842900670269, 40.58881272670056], [-73.92847329475424, 40.58877954325495], [-73.92849709626647, 40.58876171082953], [-73.92852165883288, 40.588743206409575], [-73.92854348148691, 40.58872676541511], [-73.92855975046837, 40.58871530010423], [-73.92861252736496, 40.58867810184751], [-73.928673449338, 40.5886351648781], [-73.92867656173935, 40.58862566755949], [-73.92862139018179, 40.58858029583018], [-73.92863061415638, 40.58857263642073], [-73.92873407743991, 40.588655223094904], [-73.92874926804639, 40.58866558532836], [-73.92875761553671, 40.58867429700575], [-73.92874984838602, 40.588680648086715], [-73.92869567082197, 40.588638434100844], [-73.92864656679113, 40.58867455913807], [-73.92862784709543, 40.58868833082378], [-73.92859836726453, 40.58871001895457], [-73.92856305715722, 40.588735995723496], [-73.92853560735229, 40.588755646267806], [-73.92851423024513, 40.58877095118616], [-73.92849655735782, 40.58878414921092], [-73.92845089882498, 40.588818250208014], [-73.92839036856563, 40.588863457828595], [-73.92839582804494, 40.58887850436322], [-73.92846910760312, 40.58893255510825], [-73.92844312664116, 40.5889529207223], [-73.92843184870172, 40.58894403263148], [-73.92838268998239, 40.58897773722424], [-73.92837745971521, 40.58898132361609], [-73.92844090493661, 40.589030428141996], [-73.92862026499205, 40.58912043629614], [-73.92865380058103, 40.58909874079444], [-73.92865104293678, 40.58909642248604], [-73.92876308761618, 40.58902100394114], [-73.9287803739474, 40.589010501518786], [-73.92880353610383, 40.588994608624006], [-73.92882969525398, 40.588976660152724], [-73.92885099369292, 40.58896217032521], [-73.92891351845398, 40.58891963277442], [-73.92891343177546, 40.58890568422545], [-73.92886464540858, 40.588867356316136], [-73.92888139684439, 40.58885371957536], [-73.92895102993238, 40.588909642409575], [-73.92881443150283, 40.589004326549414], [-73.92879080445435, 40.5890192925533], [-73.928631587027, 40.589126361638975], [-73.92866593842896, 40.589143264066465], [-73.92869126291087, 40.58912688184337], [-73.92873426467298, 40.58914716516767], [-73.92887113950044, 40.589057657450184], [-73.9291344038264, 40.588879241681944], [-73.92925835283317, 40.58893237343767], [-73.92924314818033, 40.588950691621356], [-73.92913312005682, 40.58889660396917], [-73.92888297663987, 40.58906895960711], [-73.92884820698325, 40.58909128683477], [-73.92885578380277, 40.589104795901086], [-73.92881653421958, 40.58912792348119], [-73.92884055219514, 40.58915012510956], [-73.9288240915015, 40.58915976125539], [-73.92878615917873, 40.58913272783028], [-73.92875703499415, 40.589152966925], [-73.9288784483006, 40.58920995672331], [-73.92884679091428, 40.58923212387582], [-73.9288859965192, 40.58925240529493], [-73.9289165622552, 40.58926317842035], [-73.92906310125774, 40.589295923803746], [-73.9290841723937, 40.5892610669131], [-73.92921570403128, 40.589285631091556], [-73.92934866543209, 40.58930381597756], [-73.929645245279, 40.58936184684617], [-73.92976491819753, 40.58940061950793], [-73.92990258163894, 40.58946466728639], [-73.92998187866192, 40.589515997878614], [-73.930013991319, 40.589461997600054], [-73.92998364350571, 40.58944139963777], [-73.92986055070119, 40.58937444315987], [-73.92975766786877, 40.58934351118515], [-73.92975094353098, 40.58932035518648], [-73.92961092161859, 40.589308693552695], [-73.92936128354346, 40.589250662982494], [-73.9293731224163, 40.589223659165306], [-73.92957384158932, 40.58927265737984], [-73.92976615773313, 40.589292069447644], [-73.93003427450557, 40.58942599650075], [-73.93015902101656, 40.589525109214854], [-73.93032759859494, 40.58965768991761], [-73.93039500720478, 40.58973232891241], [-73.93042363784221, 40.5897812224387], [-73.93043882862878, 40.58977608680935], [-73.93057705718242, 40.589890641203176], [-73.93054835816316, 40.58990734463013], [-73.93038990330076, 40.58977348527009], [-73.93034074246636, 40.58980768423797], [-73.9302284661763, 40.58968813602466], [-73.9302712478111, 40.589653163603685], [-73.93016235501895, 40.589563697105724], [-73.93010325988745, 40.58960610507142], [-73.93018342737281, 40.58967665569107], [-73.9303106518833, 40.5898185272553], [-73.93037543247318, 40.5899404597104], [-73.93029788091665, 40.59003516763001], [-73.93032208619142, 40.59004903480978], [-73.93044041745254, 40.59002041123875], [-73.93045501847267, 40.59003763271232], [-73.93056682228968, 40.590016943606095], [-73.93055281235699, 40.5899568607897], [-73.93058940447355, 40.589952591132516], [-73.93062849364412, 40.590252990337696], [-73.93058909225768, 40.590252966543545], [-73.93056960951894, 40.59004269153593], [-73.93046768917159, 40.59006137258804], [-73.93053560681557, 40.590263661461684], [-73.93052888333264, 40.590307988677466], [-73.93062960712314, 40.59030549085388], [-73.93063135116711, 40.59029360754254], [-73.93065213846425, 40.59029097949227], [-73.93067434421376, 40.590590721315586], [-73.93062757826256, 40.59059069309133], [-73.93063131413255, 40.59032925910514], [-73.93052507228651, 40.59033311342975], [-73.93048078930119, 40.59062507739256], [-73.93047314150454, 40.590740679401925], [-73.93047135080229, 40.590859601197096], [-73.93047481956741, 40.59090787720931], [-73.93047712293603, 40.59092070012568], [-73.93049906406719, 40.59092071339867], [-73.93049905444629, 40.59092995732656], [-73.93052330262896, 40.590929971990285], [-73.9305250476421, 40.59091676838311], [-73.93062344246574, 40.59091869427622], [-73.93062639159919, 40.59061900114781], [-73.93064749948525, 40.59061968417434], [-73.93064106700565, 40.59094456641932], [-73.93052154756077, 40.590952416341985], [-73.93052502016778, 40.59094317601575], [-73.93049384083261, 40.59094579825993], [-73.93049556105831, 40.5909576826588], [-73.93046958507563, 40.590956429572074], [-73.93046013661869, 40.59100121902088], [-73.93069297189139, 40.59100005480597], [-73.93069641658052, 40.59101854140028], [-73.9304563207586, 40.59102541699407], [-73.93044970323461, 40.59107887126437], [-73.93044960172921, 40.59107969192914], [-73.93072580784407, 40.591070054959765], [-73.93072578729826, 40.59108986026513], [-73.9305059228273, 40.591093850738034], [-73.93050596616035, 40.59111112823183], [-73.93044497861847, 40.5911170272311], [-73.93037197719016, 40.591211400875885], [-73.93042667103755, 40.591247715831585], [-73.93053169280374, 40.59119009299107], [-73.93052474129993, 40.5912112155772], [-73.93056977645742, 40.59120992249866], [-73.93057152827818, 40.59119143786804], [-73.9306044430696, 40.59118617622098], [-73.93060066906222, 40.59148458204167], [-73.93056428429375, 40.591495123288], [-73.93057495301012, 40.59122973161395], [-73.93052645879092, 40.5912257404034], [-73.93052297842125, 40.59124158305579], [-73.93046536765816, 40.591273407596056], [-73.93049945183151, 40.59129603763979], [-73.93047480590835, 40.5914619843199], [-73.93047423397412, 40.59146420239729], [-73.93040541662201, 40.59173094214291], [-73.93046185647538, 40.59172084849407], [-73.93057117820943, 40.59152813743205], [-73.93060580656592, 40.591541362508444], [-73.93050516364971, 40.591716913965314], [-73.930465294205, 40.59174725889172], [-73.93045964272001, 40.59198513253282], [-73.9304546522737, 40.59213519976274], [-73.93045091922694, 40.59224503726173], [-73.93042493568416, 40.59224634183091], [-73.93042696672613, 40.59212508383385], [-73.93043411755326, 40.59193279181264], [-73.93043758093522, 40.59174724211833], [-73.93039905602124, 40.59175559642899], [-73.93038380855803, 40.591829629289585], [-73.93035545570713, 40.59202813803916], [-73.93034584310841, 40.59209544130455], [-73.93033106509768, 40.59213223736405], [-73.93031136732715, 40.59219396315606], [-73.93030161927013, 40.5922991797263], [-73.93034433277133, 40.59230138551743], [-73.93034462939919, 40.59229806777917], [-73.93036426041914, 40.59229908703112], [-73.9303692274756, 40.59224352542571], [-73.93038850696499, 40.59225223975195], [-73.93038744801052, 40.592262268085314], [-73.93038686981008, 40.59226774745139], [-73.930383388218, 40.59230069936355], [-73.93038116977189, 40.59232169631124], [-73.93037650336757, 40.59236586778102], [-73.93037497475126, 40.5923803407173], [-73.93037402412372, 40.59238932825651], [-73.93037011701409, 40.59242632138718], [-73.93036902388113, 40.59243666532477], [-73.93036570007465, 40.592468131346436], [-73.93036443603306, 40.59248009351359], [-73.93034189370505, 40.59246497306263], [-73.93036221568056, 40.59231177060903], [-73.93030109951036, 40.59230803685339], [-73.93030751617428, 40.59235672359121], [-73.93028656348375, 40.5924455754653], [-73.93020441819301, 40.59237817647126], [-73.930176166342, 40.592406220970396], [-73.93021785135721, 40.59244085717815], [-73.93020149452236, 40.59244831201492], [-73.9302060581915, 40.59245166621076], [-73.93026531169379, 40.59249522914789], [-73.93026943351029, 40.592491972534006], [-73.93035364458565, 40.59255495029407], [-73.93034305347926, 40.592562660609445], [-73.93031400512083, 40.592583807644964], [-73.93030388752189, 40.59259117346872], [-73.93027668921863, 40.592610975016264], [-73.93026510927315, 40.59261940493299], [-73.93025320174314, 40.592610495198684], [-73.93032634060388, 40.59255368983698], [-73.93027813759224, 40.59251762420758], [-73.93026579193081, 40.592512439390404], [-73.93023285891259, 40.592487798622386], [-73.93023781228278, 40.59248395113589], [-73.93021718600727, 40.59246851938164], [-73.93021322788651, 40.59247159348604], [-73.9301897065827, 40.592454118288785], [-73.93012526400538, 40.59254669753296], [-73.93010583866844, 40.59257460414377], [-73.9300230102621, 40.592683046384884], [-73.93002724398043, 40.592685016427154], [-73.93002811427151, 40.59268542154037], [-73.93002958401452, 40.59268610478215], [-73.9300297092209, 40.592686162488484], [-73.93011593181335, 40.59272627808418], [-73.93009404144675, 40.59275361995975], [-73.93009108240771, 40.59275224358162], [-73.93008905262103, 40.59275478125344], [-73.9301019165656, 40.592760766189706], [-73.93018660630729, 40.5926656203169], [-73.93021251489361, 40.59267837029242], [-73.9301788650343, 40.59271858033942], [-73.93016410331174, 40.59273621991156], [-73.9301469470833, 40.59275672193976], [-73.93012845255014, 40.59277882171618], [-73.93011754836337, 40.592791849535615], [-73.930106582116, 40.59280495538488], [-73.93007616098994, 40.592789509941284], [-73.93008682543812, 40.592777303208564], [-73.93007558219115, 40.592771606402145], [-73.92999950016439, 40.592733054474145], [-73.92999915553243, 40.59273288070382], [-73.92999827166076, 40.59273243219206], [-73.92999750442164, 40.592732044731974], [-73.92999008815492, 40.59272828588208], [-73.92996263703681, 40.59281717042391], [-73.9299125015645, 40.592853394872975], [-73.92991885786864, 40.592855092303424], [-73.92992176281749, 40.59285586755785], [-73.92992534072587, 40.59285682164133], [-73.92999065309387, 40.5928742568567], [-73.92998741450496, 40.59288126732527], [-73.93005338592835, 40.592898976314295], [-73.93007018426357, 40.592902776553785], [-73.93007514030563, 40.59289515681875], [-73.93008710848919, 40.59286925033288], [-73.9300989932412, 40.592872440452346], [-73.93009701128348, 40.592876731012446], [-73.93009377470048, 40.59288456489173], [-73.93007945243008, 40.592919227372825], [-73.93006659641668, 40.59295033912306], [-73.93006258490884, 40.59296448436384], [-73.930052445394, 40.59300024352078], [-73.9300326034588, 40.59300023144112], [-73.93004645355526, 40.59295858421449], [-73.93006353529177, 40.59291716846797], [-73.93000857751039, 40.59290489163024], [-73.93001166376469, 40.592896866008296], [-73.92998800211831, 40.592891580452246], [-73.92998549410797, 40.59289810218026], [-73.92995835875485, 40.592892041018835], [-73.9299605293827, 40.59288639744193], [-73.92990668955943, 40.59287437035284], [-73.92990039140619, 40.592872963955664], [-73.92989511559179, 40.59287178484883], [-73.92989066964346, 40.592870791368064], [-73.92983201351177, 40.59290364860498], [-73.92971282036608, 40.593120048248075], [-73.92973850612175, 40.59312346748153], [-73.92974639689021, 40.59312451752122], [-73.92975537733638, 40.59312571263681], [-73.92980336402717, 40.59313209785502], [-73.92980260586405, 40.59313540760326], [-73.92988841971216, 40.59314682836411], [-73.92988961346813, 40.59314161893123], [-73.9299812081446, 40.59315888690685], [-73.93001524679518, 40.593054397167826], [-73.93004238419583, 40.59306087931805], [-73.93002689378028, 40.59309073114711], [-73.93002033232533, 40.59310546889237], [-73.92997911251686, 40.593236563438055], [-73.92997510439876, 40.593245975137066], [-73.92995591909563, 40.5933085723266], [-73.92993575938482, 40.5933053216952], [-73.92997593629296, 40.593173815413515], [-73.92989436237971, 40.59315842166691], [-73.92979816541931, 40.59314505122782], [-73.92974147322658, 40.593143922445535], [-73.92976735400573, 40.59318654967935], [-73.9297785203565, 40.593247437824104], [-73.92981465540538, 40.59329882823662], [-73.92982317336855, 40.59337269559881], [-73.92989320208056, 40.59336641696097], [-73.92990069737226, 40.59335881134764], [-73.93003922396665, 40.59334843088376], [-73.9300354613713, 40.59336745389019], [-73.92990442337926, 40.593375935617374], [-73.92980068404492, 40.59339144196071], [-73.92978081742058, 40.5934367414716], [-73.92985305834146, 40.593525079228115], [-73.92985678414759, 40.59358423525881], [-73.92984368238119, 40.59364613235038], [-73.92985416775136, 40.59369933684081], [-73.92986032030679, 40.59378210123926], [-73.92991887262683, 40.593876311711035], [-73.92994377175573, 40.5939324518857], [-73.92996116581368, 40.594005710816745], [-73.93001650952081, 40.5940945360621], [-73.93021519470311, 40.59397294418455], [-73.93019837897519, 40.59393884360408], [-73.93018896715142, 40.593941078758476], [-73.93018541414243, 40.59393201544326], [-73.93016703888816, 40.59388515270215], [-73.93016374518345, 40.59387674944167], [-73.9301458561132, 40.59383112654151], [-73.93012135534387, 40.593768640140134], [-73.93011026067549, 40.59373904051628], [-73.93010626301084, 40.59372837618566], [-73.93009029511923, 40.593697570902805], [-73.93007832030693, 40.593674468866475], [-73.9300743711095, 40.593666848571374], [-73.93005017634239, 40.59360230642058], [-73.93008159453139, 40.59359546287128], [-73.93013632556473, 40.593710960028424], [-73.93013899067863, 40.59372212915136], [-73.93016493129313, 40.59378428069545], [-73.9301826176162, 40.593826654574144], [-73.93018748124085, 40.59383790527812], [-73.93022756280047, 40.59393062089687], [-73.9302876039036, 40.59390604192331], [-73.93029155870589, 40.593911698603414], [-73.93030764773724, 40.59390237882368], [-73.93045934564594, 40.593814498247355], [-73.93052488661866, 40.593880240700756], [-73.93038136884562, 40.59396338222485], [-73.93035968941524, 40.59397594132398], [-73.93044330312577, 40.594059811558544], [-73.9305433316513, 40.594158593216584], [-73.93057032638413, 40.59418567053487], [-73.93054679070711, 40.59419930408557], [-73.93052537125826, 40.594177819579606], [-73.93052029290217, 40.59417284072187], [-73.9304873099403, 40.59413914320047], [-73.93048105499315, 40.59413284065055], [-73.93044180984458, 40.59409329506805], [-73.9304339641136, 40.594085389134996], [-73.93042039868939, 40.59407172008142], [-73.9303662465998, 40.59401715247634], [-73.93034985747646, 40.59400063806112], [-73.93034386205119, 40.59399459750923], [-73.93033684931885, 40.59398770328114], [-73.93034434256145, 40.59398271226768], [-73.93032694746384, 40.59396544788088], [-73.93029934404385, 40.59393805451023], [-73.93004011947689, 40.59410488733049], [-73.93009827326867, 40.594160850194044], [-73.93020176823151, 40.59423891758602], [-73.93027533487947, 40.59429698953186], [-73.93043246129608, 40.59440267494988], [-73.93052974770505, 40.59445124927902], [-73.93058089609403, 40.59446745145527], [-73.9306134962184, 40.5944768105665], [-73.93075181378947, 40.59438762434922], [-73.93076122839551, 40.59438190052074], [-73.93074807610716, 40.59436877907354], [-73.93070936725903, 40.594330163861734], [-73.93070268256008, 40.59432349434325], [-73.93066439573107, 40.59428529819391], [-73.93061151254695, 40.59423254145781], [-73.93059803646442, 40.59421933301731], [-73.9306230568455, 40.59420449765984], [-73.93064177530424, 40.594222842526435], [-73.9306495693869, 40.594230577701616], [-73.93068733320767, 40.59426805333232], [-73.93069541073764, 40.594276066941504], [-73.9307406167828, 40.59432092721014], [-73.9307491347329, 40.59432937984008], [-73.93078832527874, 40.59436827024408], [-73.93079506249981, 40.59437495570452], [-73.93083446436513, 40.59441357130414], [-73.93080693832646, 40.594429892356175], [-73.93076839841689, 40.594392122116396], [-73.93066303510112, 40.59446395093277], [-73.93070567804722, 40.59447704025298], [-73.93076901026963, 40.59449301481567], [-73.93082526679706, 40.59444675750643], [-73.93081953536732, 40.594441650444146], [-73.93083206587438, 40.59443065566926], [-73.93093516114615, 40.59445994986099], [-73.93095924897628, 40.594441570290854], [-73.93099362203101, 40.594416917392465], [-73.93102812893794, 40.59439371871824], [-73.93103616514564, 40.59438796003147], [-73.93110437164648, 40.594343699334104], [-73.93119016326017, 40.5942875652701], [-73.93126301572376, 40.59424418611863], [-73.93121284213825, 40.594188409219285], [-73.93122055938905, 40.59418331726279], [-73.93122677419932, 40.59419110690569], [-73.93127371485065, 40.59423310576509], [-73.93130987534305, 40.59426545835942], [-73.9313251214607, 40.594276256190035], [-73.93131702522788, 40.59428143789023], [-73.9312765055576, 40.5942567649142], [-73.93124036875787, 40.59427804397499], [-73.93119990483329, 40.594303600137046], [-73.93112950421332, 40.59434979638224], [-73.93112087099405, 40.5943554614039], [-73.93104573382053, 40.59440476393444], [-73.93103520310456, 40.59441167456396], [-73.93099809750406, 40.5944366689339], [-73.93097341657615, 40.59445329362182], [-73.9309546342639, 40.59446594520627], [-73.93100842980581, 40.59448226571793], [-73.93111095510909, 40.5944890129843], [-73.93117079151007, 40.59449753084694], [-73.93116797698178, 40.59450340357535], [-73.93117266234964, 40.59450400965462], [-73.93118315992675, 40.59450537024825], [-73.9312471362245, 40.59451366053259], [-73.93125615625141, 40.594514828582156], [-73.93132562373592, 40.594523830253785], [-73.93133571171825, 40.59452513731441], [-73.93136684826814, 40.59452917209447], [-73.9314197928919, 40.59452099074845], [-73.931430230235, 40.59451937864412], [-73.93151134999133, 40.59450684322595], [-73.93158758229917, 40.59449506291083], [-73.9316156203621, 40.59436596310946], [-73.93173994773721, 40.59437086790281], [-73.93174223368128, 40.59430090316993], [-73.93174280308598, 40.59428351246114], [-73.9317638740002, 40.59428409876283], [-73.93175908241429, 40.59436700800082], [-73.9317580816717, 40.594384328761244], [-73.93163177589155, 40.59437662255863], [-73.93162407058445, 40.594429080940465], [-73.93161313737198, 40.59450351841438], [-73.93153064222903, 40.59451955377668], [-73.93146047172367, 40.594533194229214], [-73.93136806343367, 40.59454645078467], [-73.93117195590496, 40.59452009321482], [-73.93102713500376, 40.59450468985097], [-73.93098771679708, 40.594493421284724], [-73.93084626727263, 40.594451525297195], [-73.93078131379613, 40.594498955276784], [-73.93080547079994, 40.59451610120178], [-73.93081918377538, 40.594529428370684], [-73.93085036660067, 40.59454371510518], [-73.9308847319492, 40.59454889166053], [-73.93101903473392, 40.59461691000353], [-73.93113820375036, 40.59464019048005], [-73.9313017105705, 40.594654481089464], [-73.93143018596695, 40.594686617062685], [-73.93156252515298, 40.59469897618456], [-73.93169652225768, 40.594709929731266], [-73.9318772173972, 40.59468868237025], [-73.93190503589658, 40.59468541176617], [-73.9319336498399, 40.594692404302386], [-73.93199915298685, 40.59472828630575], [-73.93208972932096, 40.59479438694401], [-73.9321296485849, 40.594828742469076], [-73.93216666204167, 40.59486309676736], [-73.93221820952614, 40.5948919217798], [-73.93225233912358, 40.594904678318606], [-73.93230826395404, 40.59491634048718], [-73.93234749083399, 40.59491691693656], [-73.93239677902233, 40.594912014048674], [-73.93245217688444, 40.59491496766572], [-73.93241277639828, 40.59484250702065], [-73.93239815454939, 40.594845842136536], [-73.93237188265157, 40.59480236388957], [-73.93239089433519, 40.59479568931841], [-73.93240548656371, 40.59482355818788], [-73.93256343839313, 40.59476347229703], [-73.93257803499358, 40.59478576979387], [-73.9324376290322, 40.59484363603982], [-73.93247702888684, 40.594916096675945], [-73.93255597836588, 40.59491168558764], [-73.93260420556844, 40.59492954452495], [-73.93267147327069, 40.59491175331411], [-73.93279727104483, 40.59484273251847], [-73.93282654852884, 40.59480374496913], [-73.93281932831556, 40.59471570244102], [-73.93280736080116, 40.594669158995245], [-73.93280121604168, 40.59464525651379], [-73.93278001234762, 40.59455966061023], [-73.93271241665666, 40.59433387116148], [-73.93267653975757, 40.594236417967586], [-73.93259834108942, 40.594112228741984], [-73.93252661189227, 40.59404001871799], [-73.93244863039499, 40.59395996141996], [-73.93235089999555, 40.59386846912427], [-73.93216554505263, 40.59356300857742], [-73.93200062731798, 40.59328764992159], [-73.93190580795942, 40.59308588397505], [-73.93182549996209, 40.592991109930814], [-73.9317589074308, 40.59283010489918], [-73.9317446553905, 40.59283411261459], [-73.9316741082931, 40.59285395021889], [-73.9317501541548, 40.5930111033848], [-73.931760735344, 40.59303709413345], [-73.93173338203975, 40.59304452121348], [-73.9317109079429, 40.59305030052047], [-73.93168977728054, 40.59300774573801], [-73.9316885567518, 40.59300489316119], [-73.93153495293322, 40.59269283051891], [-73.93155057104204, 40.592690093347755], [-73.93162526000405, 40.59282794876039], [-73.93166486127859, 40.59281645518014], [-73.931729250829, 40.59279776786523], [-73.93174087511966, 40.592794393694405], [-73.93168985287654, 40.59268679297557], [-73.93167672233834, 40.592661153688], [-73.93166657890721, 40.59263548434449], [-73.93161598469497, 40.59260222732161], [-73.93156462560293, 40.59255174077168], [-73.93153229503947, 40.59247981491751], [-73.93148861724065, 40.592300710940165], [-73.93146597430784, 40.59226351413933], [-73.93146571541051, 40.59226309951732], [-73.93145367368668, 40.592238386451925], [-73.931431598401, 40.592225419412266], [-73.9314367561379, 40.5921671345181], [-73.93130932293144, 40.59216058122609], [-73.93127197116024, 40.59213076681868], [-73.9312210463182, 40.59208280989741], [-73.93115993293195, 40.59202707583501], [-73.93113956057566, 40.59200892874752], [-73.93120075354456, 40.59198824051708], [-73.93125838564315, 40.59196519566691], [-73.93131727481894, 40.59194678741044], [-73.93136406792838, 40.5919283730722], [-73.93139997986378, 40.59180489056213], [-73.9310098934316, 40.59193104248098], [-73.93100451460829, 40.59183074145787], [-73.9310001292229, 40.591830738824655], [-73.93099465973228, 40.591818198986076], [-73.93105716582592, 40.59180820585461], [-73.9310604356206, 40.59182743247608], [-73.93115143543241, 40.59182999411142], [-73.93115360172413, 40.59185506969025], [-73.93116347041968, 40.59185424013377], [-73.93135300954727, 40.591791193961825], [-73.93130130855009, 40.591692970610374], [-73.93135740555536, 40.59166968924601], [-73.93139310403653, 40.59165416618969], [-73.93140775751667, 40.59164400925833], [-73.93142565439746, 40.591604184147364], [-73.9312371374292, 40.59165368814867], [-73.93124260951417, 40.591662885934895], [-73.93119983469255, 40.59167790513536], [-73.93119326187228, 40.591671213764315], [-73.93107262221186, 40.59170541142871], [-73.9310704045976, 40.59172881226959], [-73.93104847875684, 40.59172629086602], [-73.9310760222574, 40.59159759276817], [-73.93109794805815, 40.59160011299371], [-73.93107702484298, 40.591688697156016], [-73.9312305704303, 40.591641983132995], [-73.93122728942248, 40.59163445876775], [-73.93128458380468, 40.59136118117852], [-73.9313054163602, 40.59136035866831], [-73.93124592494814, 40.5916369769958], [-73.93142252650098, 40.59158857039047], [-73.93144647757205, 40.59156975807161], [-73.93142232187743, 40.59150663469118], [-73.93149291902427, 40.591481039286585], [-73.93148064834168, 40.591431790745574], [-73.93151343730557, 40.59141298100165], [-73.93154334667179, 40.5913504711831], [-73.93148084879925, 40.5913529410094], [-73.9314862870528, 40.59139640655309], [-73.9314665489462, 40.59139890185896], [-73.93143272420807, 40.59123924130791], [-73.93145684761484, 40.5912367486238], [-73.93147538548219, 40.591334549229884], [-73.93147647503521, 40.59134207228152], [-73.93154664980304, 40.591337935423475], [-73.93155939441614, 40.59129296407077], [-73.93155272211995, 40.59119979703636], [-73.93149418291642, 40.59116653304395], [-73.93142883104395, 40.59113638100172], [-73.9313961055893, 40.59080940786934], [-73.93140697512008, 40.590699910482144], [-73.93141600360825, 40.59060910592292], [-73.93134125982561, 40.59061546375555], [-73.93131907299008, 40.590527528418114], [-73.93113301959801, 40.58954611892402], [-73.93121294858955, 40.58953130811921], [-73.9310917816459, 40.589326565572115], [-73.93105313024402, 40.589330303923184], [-73.93101236544717, 40.58931170698334], [-73.93092739827031, 40.58925625816704], [-73.9308888007358, 40.58920734017373], [-73.93079601726942, 40.58907815135634], [-73.93070488530003, 40.58894206677664], [-73.93068568059813, 40.58882195422682], [-73.93062438721954, 40.58884907156901], [-73.93056889562718, 40.58877843681944], [-73.93010137658842, 40.588969317781576], [-73.93009138279199, 40.58898886237901], [-73.9300700476359, 40.58895409308378], [-73.93053614268541, 40.58876212495992], [-73.93054468184351, 40.5887708202054], [-73.93058602090872, 40.58875129283311], [-73.9305344412823, 40.58867959008412], [-73.9306502192753, 40.588669869770776], [-73.93063532324204, 40.58862358012012], [-73.93054733435007, 40.5886285422966], [-73.93050707979359, 40.588593413015495], [-73.93043726960487, 40.588514386369845], [-73.93039967597488, 40.58846552041533], [-73.93037983195714, 40.58847090635756], [-73.93038328514656, 40.588477447121264], [-73.92974410780818, 40.58864107972962], [-73.92972846380192, 40.58861391582969], [-73.93036500375167, 40.58844282514472], [-73.9303680674251, 40.588448629548814], [-73.93038545256262, 40.58844161607885], [-73.9303596123009, 40.588404298162175], [-73.93037062807417, 40.58836086664855], [-73.93036760201352, 40.58832110138197], [-73.93036150933793, 40.588278995714134], [-73.93033698653332, 40.5882579299451], [-73.9302909816294, 40.58824152872789], [-73.93026338157462, 40.58822981692194], [-73.92970255020441, 40.587607355474205], [-73.92972232646723, 40.587560082903835], [-73.92990371101061, 40.58752912420285], [-73.93021881179953, 40.58747683272641], [-73.93073284303382, 40.58740698378469], [-73.93134917824065, 40.58732816722948], [-73.93136556192967, 40.58716922512903], [-73.93135135966136, 40.587124684315924], [-73.9313172217923, 40.58707144166877], [-73.93130299966687, 40.58704753744361], [-73.93127751247582, 40.58689328743528], [-73.93125191267133, 40.58684982521422], [-73.93118073044302, 40.58679873371334], [-73.9310452875478, 40.58681416003715], [-73.93105809300631, 40.58690727637346], [-73.9308144495186, 40.586923422495275], [-73.93080593035211, 40.58689626302754], [-73.93055771935786, 40.58692511765015], [-73.92964322159177, 40.58703132696806], [-73.92964181549499, 40.58701394761244], [-73.93055096924029, 40.586907278390484], [-73.93082019774093, 40.58687563400684], [-73.93082871260312, 40.58690822494172], [-73.93091135186273, 40.586902843190764], [-73.93091279265593, 40.58688763841874], [-73.93095696421327, 40.58688223467156], [-73.93095410532507, 40.58689092186597], [-73.93101679851631, 40.586885528219156], [-73.93101218068466, 40.58683048768304], [-73.93052821173194, 40.586857215894916], [-73.92966894624747, 40.586955311676846], [-73.92961575482575, 40.58697984176409], [-73.92956343312808, 40.58703562312643], [-73.92928895400574, 40.587055262041325], [-73.92919437666475, 40.587087532114374], [-73.92908894003958, 40.58709506974444], [-73.92888983084922, 40.587083818719364], [-73.92879236548926, 40.58709060159946], [-73.92869359490844, 40.587081851936325], [-73.92857393936458, 40.58706150237135], [-73.92849798568434, 40.587032490303066], [-73.92843153421316, 40.58700058835756], [-73.92836889002496, 40.586959999834725], [-73.92829674888085, 40.586917956054535], [-73.92824930479918, 40.586875928826544], [-73.9281857599151, 40.58682528921666], [-73.9281430268027, 40.586781728308814], [-73.92811462171161, 40.58669916226926], [-73.92808232971585, 40.586697693937616], [-73.92806347188096, 40.5865687917075], [-73.92807588962786, 40.58648036137611], [-73.92808640183115, 40.58644425900183], [-73.92809974275899, 40.58640371685694], [-73.92810548598204, 40.58636172267004], [-73.92811957884383, 40.58632915019955], [-73.92808962925501, 40.58632649638931], [-73.92803894403345, 40.58632383025664], [-73.92804358383414, 40.58629397188225], [-73.9280965704473, 40.58629927452315], [-73.92811039892527, 40.58629489206262], [-73.92814969379499, 40.58617810776291], [-73.92813819706498, 40.58615614316272], [-73.92791933785853, 40.586135806046116], [-73.92792514440899, 40.58609365392655], [-73.9281647386095, 40.5861140034952], [-73.92816239374059, 40.58615176718001], [-73.92817273929886, 40.586172852701445], [-73.92819501248111, 40.58613440857037], [-73.92827672497616, 40.58610694357004], [-73.92815700695613, 40.58586452306607], [-73.92801568895456, 40.58587068011404], [-73.92785999074735, 40.585917424191905], [-73.92774311153144, 40.58605006520928], [-73.92764071689658, 40.586045316480664], [-73.92751386933554, 40.58602514187572], [-73.92744400140172, 40.58601571445453], [-73.92740528162324, 40.585992354154506], [-73.92738822580927, 40.5860154974507], [-73.92736955785888, 40.58600658531642], [-73.92733683471778, 40.58604038446054], [-73.92753271376783, 40.5862594418086], [-73.92758641829884, 40.58625413533235], [-73.9275955985745, 40.58640009709845], [-73.92751621995603, 40.58639826671549], [-73.92751168880422, 40.586270107541274], [-73.92734144038619, 40.586099125029804], [-73.9272807972705, 40.586043909008545], [-73.9273065040188, 40.58602078544186], [-73.92732050626583, 40.58602613463509], [-73.92735088516017, 40.586001234348885], [-73.92733921875872, 40.58599410697143], [-73.92737247661461, 40.585959205318275], [-73.92735853538503, 40.58592100147285], [-73.92731976953024, 40.58589260668801], [-73.92730433823876, 40.585865928419864], [-73.9272133428372, 40.58581247231648], [-73.9271409664433, 40.58581242644442], [-73.92711203473591, 40.585803552824544], [-73.92697296751643, 40.585719762297714], [-73.92686097045829, 40.5856573934011], [-73.92676297506527, 40.58560037198057], [-73.92669065751838, 40.58554692736528], [-73.92666268980562, 40.58550241135616], [-73.92666272889834, 40.585466812382734], [-73.92657139551324, 40.5854185317093], [-73.92638041881065, 40.585295756347826], [-73.92630563576202, 40.585361567241996], [-73.92633697458788, 40.58537928196403], [-73.92660195912502, 40.58552907123918], [-73.92656690671208, 40.58555752783965], [-73.92654820963332, 40.585575315387594], [-73.92610161910716, 40.58533987466663], [-73.92559227190102, 40.58507134304677], [-73.92549354358316, 40.58501929200321], [-73.92552860874584, 40.58497837657265], [-73.92565445909003, 40.5850414583568], [-73.92627529542231, 40.58535264744197], [-73.92634542074707, 40.585276154753124], [-73.92628242042889, 40.585242294513314], [-73.92623109243672, 40.58521022257151], [-73.92618443073194, 40.58518171360485], [-73.92620547282668, 40.585155028458026], [-73.92604075506927, 40.585065634720145], [-73.92583335908428, 40.584953077110605], [-73.92678292699999, 40.58486143799989], [-73.92712081899994, 40.58487935699989], [-73.92762689299991, 40.58503505599989], [-73.92840477199982, 40.58526206699991], [-73.92858911999988, 40.585207258999965], [-73.92942869199987, 40.585488201999915], [-73.93131301499997, 40.58605420599988], [-73.93139947599984, 40.58607989499989], [-73.93145727300002, 40.58610127199996], [-73.93210850499995, 40.586286202999865], [-73.93244497799995, 40.586360289999874], [-73.93344399399984, 40.58653743799994], [-73.9344471859999, 40.586652074999925], [-73.93477406299994, 40.586639969999965], [-73.93541271099996, 40.58658605499993], [-73.93588537499998, 40.58653208999991], [-73.93633646899995, 40.58648058799982], [-73.93677580200001, 40.58639859399994], [-73.9372471729998, 40.586310628999904], [-73.93816896699991, 40.58614987199996], [-73.93908601799993, 40.58599196799987], [-73.93914504099992, 40.58597982499988], [-73.93920635200004, 40.58597168899993], [-73.94105998299987, 40.585661083999916], [-73.94191610399989, 40.585520172999914], [-73.94291014799995, 40.58540563799992], [-73.94386629099992, 40.5852983619999], [-73.9449308669999, 40.58517250599993], [-73.94555221999997, 40.585139491999946], [-73.9466911319999, 40.585073071999915], [-73.94763574499989, 40.584998861999956], [-73.94866230599997, 40.58489385599993], [-73.94986974599985, 40.58485639599989], [-73.95123865600004, 40.5849246329999], [-73.95287969899992, 40.58519841999987], [-73.95289540899992, 40.58529875399992], [-73.95300767399999, 40.585274583999926], [-73.95312439499999, 40.585261132999904], [-73.95324286999993, 40.58525898299989], [-73.9533602969999, 40.58526818099989], [-73.95347394799992, 40.585288245999884], [-73.95357366099982, 40.585309187999876], [-73.953671153, 40.585348497999895], [-73.95375778800003, 40.585405959999875], [-73.95382536099993, 40.58547826199986], [-73.9538683109999, 40.58555947099993], [-73.95388521699984, 40.58564255099988], [-73.95387860700005, 40.58572124599991], [-73.95398040799994, 40.5857118479999], [-73.954399849, 40.585676247999885], [-73.95485543100001, 40.58562477799989], [-73.95576287300001, 40.585500097999905], [-73.95626771099987, 40.58546620499987], [-73.95641613599993, 40.585399010999915], [-73.95657432099985, 40.585344345999864], [-73.95673945299994, 40.5853033079999], [-73.95690846199993, 40.58527652799987], [-73.95757916399988, 40.5852008269999], [-73.95761659699984, 40.58519750799994], [-73.95765292099998, 40.58519361399986], [-73.95777933699982, 40.58518352899989], [-73.95880175499998, 40.58518715199989], [-73.95917287700001, 40.58513910599985], [-73.96007463199999, 40.58505031599991], [-73.96021742199999, 40.58503625499989], [-73.96025825899996, 40.58502930799994], [-73.9603253929998, 40.585812722999954], [-73.96034190799985, 40.585968365999854], [-73.96034614999985, 40.58612477099993], [-73.9603380369999, 40.58628108999996], [-73.96031766399983, 40.5864364679999], [-73.96031712099983, 40.58651472399986], [-73.9603495359999, 40.58730628599994], [-73.95978030299993, 40.58736947499985], [-73.9593473669999, 40.58741753699992], [-73.9581954319999, 40.587544331999865], [-73.95712566399992, 40.58766096099986], [-73.95619313699986, 40.58776226899994], [-73.95622721599997, 40.587995914999944], [-73.95639200299993, 40.58892867499993], [-73.95658790599994, 40.58988411499994], [-73.95665683399993, 40.59010165299992], [-73.9567157929999, 40.590498733999866], [-73.95700330399997, 40.592002255999894], [-73.95607720399984, 40.59210143199992], [-73.95655540499982, 40.59464772499994], [-73.956927197, 40.596612075999865], [-73.95600127899989, 40.59671338599991], [-73.95545313399988, 40.59677338399993], [-73.95507631399984, 40.59681539099995], [-73.95411036199994, 40.596920738999856], [-73.95314947399989, 40.5970279189999], [-73.95220370799984, 40.59713035099986], [-73.9510058549999, 40.5972626189999], [-73.94997334599992, 40.59737744199985], [-73.94904681599999, 40.59747936199991], [-73.94855032699992, 40.59753296399993], [-73.948120131, 40.597579399999866], [-73.94719207000004, 40.59768208699993], [-73.94622941199994, 40.59778899199995], [-73.94526519199995, 40.5978936269999], [-73.94479667499994, 40.59794473299995], [-73.94433922599985, 40.597994625999945], [-73.943875965, 40.59804616299987], [-73.94341159099989, 40.598097814999896], [-73.94248473299992, 40.598200364999855], [-73.94278945999999, 40.599794549999956], [-73.94283742399996, 40.60004569399995], [-73.94288954099984, 40.600318649999934], [-73.94189057599992, 40.60042741899986], [-73.9411660979999, 40.600508015999885], [-73.94089263599994, 40.60053783599993], [-73.93996699299998, 40.60063944799986], [-73.93952543799986, 40.600688096999896], [-73.93904134699991, 40.60074143199989], [-73.93811308699989, 40.60084375699994], [-73.93715062999982, 40.6009488659999], [-73.93619686399984, 40.60105506999984], [-73.93525981999983, 40.60114234799988], [-73.93447158899997, 40.60123151199989], [-73.93503274599996, 40.60173378999984], [-73.93426266699994, 40.60223071099988], [-73.9335511719998, 40.60268728099991], [-73.93173390199983, 40.601062068999894], [-73.92992224899993, 40.59943738199992], [-73.92813139699999, 40.59777792799987], [-73.92879010099999, 40.59732329499991], [-73.929124371, 40.5971073039999], [-73.92956858299992, 40.5968202649999], [-73.92866850999998, 40.59602045099991], [-73.92759936700004, 40.5950607789999]]]}}, {\"id\": \"210\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 211, \"Shape_Leng\": 0.0252345082132, \"Shape_Area\": 3.97291966087e-05, \"zone\": \"SoHo\", \"LocationID\": 211, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.001523638, 40.719820797999944], [-74.00188406299992, 40.71939757099991], [-74.00260001899987, 40.719903425999874], [-74.0032595509999, 40.72037709599994], [-74.00351186399998, 40.72055635199989], [-74.00392928399991, 40.72085443499993], [-74.00463516199997, 40.72135363299985], [-74.00522684899992, 40.72174422199989], [-74.0053098609999, 40.721824191999865], [-74.00535249999994, 40.72189013499991], [-74.00538048599999, 40.721962173999934], [-74.00539186699989, 40.722037365999896], [-74.00538616799996, 40.722112360999894], [-74.00536435999994, 40.722183951999966], [-74.00505887099996, 40.722977863999894], [-74.00478574899996, 40.72362483799989], [-74.00457474399995, 40.72417313099994], [-74.00427457099987, 40.72491210899993], [-74.00403067899997, 40.7255124959999], [-74.003730175, 40.726250058999945], [-74.00312206399995, 40.72687712799991], [-74.00251324599992, 40.72760092699992], [-74.00221100999997, 40.727961316999874], [-74.00214891499996, 40.728035366999904], [-74.00139033499991, 40.72766000199989], [-74.00063239199994, 40.72728661199988], [-73.99991579199988, 40.72693471299986], [-73.9991182469999, 40.72654231299988], [-73.998361193, 40.726167467999865], [-73.99761019599993, 40.72579713799992], [-73.9967717579999, 40.7254319439999], [-73.99684295899995, 40.725347837999905], [-73.9977850889999, 40.724234893999906], [-73.9988117219999, 40.72302140899991], [-73.99952163199998, 40.72218334699987], [-73.99985330399981, 40.72179178799992], [-74.000660725, 40.72083979399993], [-74.00142450999995, 40.71993719799993], [-74.001523638, 40.719820797999944]]]}}, {\"id\": \"211\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 212, \"Shape_Leng\": 0.0671290653429, \"Shape_Area\": 0.000161276191858, \"zone\": \"Soundview/Bruckner\", \"LocationID\": 212, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88421976599987, 40.82299645799987], [-73.8843985489999, 40.822965575999866], [-73.88465102199984, 40.823822897999925], [-73.8846216589999, 40.824956205999904], [-73.88461335999997, 40.825276502999905], [-73.88460938899989, 40.82530353599988], [-73.88438705399992, 40.826817283999965], [-73.88378414099992, 40.828274291999946], [-73.88365759099987, 40.828278785999885], [-73.88295104599979, 40.82830386599987], [-73.88198096599997, 40.828355979999856], [-73.88104510399991, 40.82843181299996], [-73.88011211299997, 40.82851166599995], [-73.87917067099991, 40.828582182999874], [-73.87870283599995, 40.82862110599992], [-73.8782350109998, 40.82866001899993], [-73.87729930499995, 40.8287427739999], [-73.87640134500002, 40.82897729399989], [-73.8755070249999, 40.82923413499987], [-73.87460867899986, 40.8295029049999], [-73.87372870299997, 40.829752885999945], [-73.87332596899981, 40.82987665899985], [-73.8732728849999, 40.8298937199999], [-73.87322660099987, 40.82990616699988], [-73.8727554649999, 40.83002788199985], [-73.87179422399991, 40.830305438999915], [-73.87090345699994, 40.83056365399988], [-73.87000670000006, 40.83082034099988], [-73.86907935400002, 40.83108592799988], [-73.86814994399988, 40.83136038499988], [-73.86725770399998, 40.83162391799994], [-73.86635554699981, 40.83187448399987], [-73.86546044899993, 40.83213164199989], [-73.86456547299998, 40.832389234999916], [-73.86366561299998, 40.832695324999904], [-73.86271114399993, 40.83299150899997], [-73.86155834899989, 40.83333022099989], [-73.86142308999996, 40.83344741099989], [-73.86137654799992, 40.83349072299995], [-73.86129183699992, 40.833536420999934], [-73.86085019699983, 40.833613537999874], [-73.86075203199984, 40.833595374999916], [-73.86065651499995, 40.833564271999926], [-73.8605680069999, 40.83352073499988], [-73.86049053499993, 40.833466403999864], [-73.8604271769999, 40.83340383299985], [-73.86037967100003, 40.83333610299992], [-73.86033845899995, 40.833253165999935], [-73.86032508999983, 40.83319863999985], [-73.86008270300002, 40.833111456999866], [-73.85934693599994, 40.83288902199993], [-73.859119143, 40.83282022499991], [-73.85847699499993, 40.83262592199991], [-73.85766180999998, 40.832379254999864], [-73.857391213, 40.83229737199986], [-73.85738951499987, 40.83225420099993], [-73.85738092499992, 40.832135234999924], [-73.85737529499985, 40.83204654599997], [-73.857323208, 40.83182148399989], [-73.85715675899989, 40.83110226799994], [-73.85698995199976, 40.83038523999988], [-73.85682187599986, 40.82966476899987], [-73.85665560399987, 40.82894491299988], [-73.85649133899996, 40.8282250779999], [-73.85641767299995, 40.82790051999994], [-73.85638435699991, 40.82775374199988], [-73.85632851899997, 40.82750768399989], [-73.85615112900003, 40.82677004599987], [-73.85635677499994, 40.82675208499993], [-73.856436988, 40.82674508099989], [-73.85672105899991, 40.82670825199993], [-73.85700204899985, 40.82665967699993], [-73.85727871200001, 40.826599624999886], [-73.85867194099998, 40.826424076999956], [-73.85970333299981, 40.826277348999874], [-73.86120690899979, 40.8260877949999], [-73.86216687599976, 40.82596516299993], [-73.86354738899982, 40.82577932899992], [-73.86361696799989, 40.82576996199992], [-73.86395604499984, 40.82572431499992], [-73.86406913799999, 40.825709089999854], [-73.86446141999993, 40.82565627799987], [-73.86455514499985, 40.825643659999834], [-73.86474490899985, 40.82561811299995], [-73.86490949499989, 40.8255959549999], [-73.86529634299995, 40.82554131799993], [-73.86538050299997, 40.82552943099993], [-73.86567536799996, 40.8254877849999], [-73.86577274899996, 40.82547402999994], [-73.86653521299999, 40.82536633399993], [-73.86662745199993, 40.82535330499995], [-73.86707924599993, 40.825289487999896], [-73.86718504299985, 40.825274542999935], [-73.86753174000002, 40.82522556999991], [-73.86761742399982, 40.825213376999926], [-73.8677031089998, 40.82520117399986], [-73.86779104499986, 40.82518945099986], [-73.86787899199994, 40.82517771999988], [-73.86840249599986, 40.82510980799988], [-73.86849386399993, 40.825097955999894], [-73.8691174439999, 40.8250170579999], [-73.86922974599999, 40.82500248899989], [-73.86984170799977, 40.824923093999864], [-73.87051877799988, 40.824840902999895], [-73.87145992799994, 40.82472020899991], [-73.87216881599993, 40.82461073999989], [-73.87253656999984, 40.82456891299994], [-73.87258431899996, 40.82456256399997], [-73.872630712, 40.824553357999925], [-73.87288306599994, 40.82451595999987], [-73.87356808499986, 40.824419845999884], [-73.87423302099994, 40.824348586999946], [-73.87437004400003, 40.82433051099987], [-73.87528548299996, 40.82420788399992], [-73.87621305999994, 40.82408474999996], [-73.8771361349999, 40.82395802999987], [-73.87793513899997, 40.82385161099992], [-73.87806435799995, 40.82383439499989], [-73.87898855699989, 40.82371109099993], [-73.879930023, 40.82364106199991], [-73.88178621799986, 40.823401699999856], [-73.8827603039999, 40.82324854799993], [-73.88421976599987, 40.82299645799987]]]}}, {\"id\": \"212\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 213, \"Shape_Leng\": 0.159958783822, \"Shape_Area\": 0.000904077906339, \"zone\": \"Soundview/Castle Hill\", \"LocationID\": 213, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.83667856399991, 40.81758876299995], [-73.83696070299992, 40.81634054099991], [-73.83895834000002, 40.81636899399993], [-73.83974355921865, 40.816406071751366], [-73.83974417046625, 40.81645136054549], [-73.83975336710861, 40.81649610979542], [-73.83977095732818, 40.81653938475521], [-73.83979657380425, 40.81658028171356], [-73.83982968160092, 40.816617946617086], [-73.839869589337, 40.81665159290646], [-73.83991546362404, 40.816680517942245], [-73.83996634646947, 40.81670411767843], [-73.84003308726955, 40.81682214096213], [-73.84009272315726, 40.816942339762356], [-73.84014513168316, 40.81706446735513], [-73.84019020523155, 40.81718827305633], [-73.84007129934879, 40.817548908704175], [-73.84008925878433, 40.817610472310655], [-73.84009887585671, 40.81767310502613], [-73.8401000506261, 40.817736156128184], [-73.84009277086297, 40.817798970546214], [-73.84007711217745, 40.817860895666705], [-73.84005323723338, 40.817921288113666], [-73.84002139406009, 40.81797952043317], [-73.83998191347625, 40.81803498761287], [-73.83993520565333, 40.81808711336741], [-73.83988175585432, 40.8181353561267], [-73.83982211939366, 40.81817921466233], [-73.83921022101013, 40.818888356624264], [-73.83892733295289, 40.81900121010709], [-73.83879133810842, 40.81952858868995], [-73.83868347700722, 40.819591381519366], [-73.83880789875982, 40.819741936146926], [-73.83906398923855, 40.81980243234166], [-73.83905466437488, 40.81998400139827], [-73.83899692969895, 40.82016459270339], [-73.83899820982785, 40.8204051183965], [-73.83899977773552, 40.820699606823], [-73.83895314598209, 40.821132403037275], [-73.83893197127122, 40.82132893403106], [-73.83896049851461, 40.82152409506767], [-73.83901168067719, 40.82195017052962], [-73.83891701760722, 40.822039722582545], [-73.83888231646733, 40.822140489856025], [-73.83891082837913, 40.82221526137366], [-73.8389946271518, 40.822234976265875], [-73.83907059472884, 40.82225284946007], [-73.8390622301788, 40.82232134057081], [-73.83903519242583, 40.822337030032806], [-73.83901290013846, 40.82238043416579], [-73.83906086613176, 40.82240162060206], [-73.83905358783915, 40.822505042589334], [-73.83905338450177, 40.822589113139955], [-73.83904904481194, 40.82268874814577], [-73.8391226306331, 40.82276980922682], [-73.83921255059413, 40.822875803426996], [-73.83925243425638, 40.822883564489686], [-73.83945427382974, 40.822922846558036], [-73.8394991378315, 40.823016322345936], [-73.83952777714363, 40.82303815864931], [-73.83952358080823, 40.82307863242266], [-73.83946189414036, 40.82316261788686], [-73.83949864965886, 40.82321871738543], [-73.83944934873239, 40.823265355083436], [-73.83958012564537, 40.82342745369348], [-73.83963278927277, 40.823685970476205], [-73.83961874262435, 40.82380510087933], [-73.83961186638571, 40.823863426519054], [-73.83968199656573, 40.823964227354374], [-73.83974013877227, 40.824110493036706], [-73.83977886908967, 40.82413803538902], [-73.83982204750704, 40.824161452037075], [-73.83986891799215, 40.82418033296484], [-73.83991398769338, 40.82418360136331], [-73.83995782576656, 40.82419219616364], [-73.83999934830801, 40.82420590485802], [-73.84004144009445, 40.82422666962721], [-73.84007814578007, 40.82425267122907], [-73.84010835009009, 40.82428311962925], [-73.84013113528368, 40.824317089680015], [-73.84013859078757, 40.824345001281685], [-73.84014119006315, 40.824373413143014], [-73.84013888869323, 40.824401839826066], [-73.84027247787519, 40.82476197802363], [-73.84067027156776, 40.82547609463137], [-73.84088074239587, 40.82581942096713], [-73.8411078091212, 40.82615332674874], [-73.84114422392041, 40.82634916720478], [-73.84132088542798, 40.826764735218475], [-73.84146448487586, 40.826964271825], [-73.84167732111614, 40.82737567321982], [-73.84190270994053, 40.82781133116808], [-73.84194160834433, 40.82788512569251], [-73.84200819971358, 40.8279782195929], [-73.84207575103679, 40.82812327445312], [-73.84222047999322, 40.82834188794376], [-73.84222728410815, 40.82837171423911], [-73.84223389473829, 40.82840361740347], [-73.84227572757854, 40.82860533187137], [-73.84228428387955, 40.82864660449135], [-73.84223265299995, 40.828652630999954], [-73.8415995109999, 40.828726494999906], [-73.84143734599988, 40.82874131099985], [-73.84122680199987, 40.82876053399988], [-73.84102340199989, 40.82877480599996], [-73.84058337699986, 40.82880567999992], [-73.8403229479999, 40.828823950999926], [-73.83747911799995, 40.82892810099991], [-73.83715124799986, 40.828959514999895], [-73.83709061399992, 40.82896714699996], [-73.83707424299983, 40.82892307199996], [-73.83703204399986, 40.828809450999906], [-73.83691608899997, 40.828451825999885], [-73.83686087299982, 40.82819847399989], [-73.83685097699993, 40.828163394999876], [-73.83678907899998, 40.827884395999895], [-73.83664057899998, 40.827215044999875], [-73.83661819699992, 40.82711414899993], [-73.8365046079999, 40.826403116999906], [-73.83626003599986, 40.82497188099989], [-73.83619033399988, 40.824205449999916], [-73.83612598999986, 40.82349743199991], [-73.83612811599986, 40.82240889399991], [-73.83612954299997, 40.821677827999984], [-73.83613095099999, 40.8209565959999], [-73.83613216399988, 40.82033578499992], [-73.83614239499985, 40.81919890299987], [-73.83639638699991, 40.81889136599989], [-73.8365014589999, 40.81871065899989], [-73.83666068699988, 40.81797116099989], [-73.83667856399991, 40.81758876299995]]], [[[-73.84231712480313, 40.82718468468027], [-73.84207620647236, 40.8265377353675], [-73.84202727066095, 40.826547447745], [-73.8417046629585, 40.825767381987035], [-73.84175457615964, 40.825755420396746], [-73.84138206178059, 40.825147433398946], [-73.84136106744249, 40.82508503349905], [-73.84127392633422, 40.82490899785533], [-73.84120250732005, 40.82481685808501], [-73.8410740038712, 40.82452611342756], [-73.8410237668874, 40.824341165873584], [-73.84100637880968, 40.8242410271096], [-73.84086821945728, 40.82415350720819], [-73.840581564588, 40.823594836156204], [-73.84044970316315, 40.823345566496606], [-73.8404204444649, 40.823268525056605], [-73.84043234895627, 40.82311304429765], [-73.84046603526154, 40.82308002358374], [-73.84041765401979, 40.8229980294567], [-73.84012785556982, 40.82242142701466], [-73.84011253729896, 40.82229940621904], [-73.840017529957, 40.82219024312548], [-73.84002656012701, 40.82206773291291], [-73.8399595945973, 40.821999860742416], [-73.83998726019149, 40.821921932554226], [-73.83992345125985, 40.82173953844155], [-73.83982926241688, 40.82157150149603], [-73.83983703184639, 40.82143133110785], [-73.83990886675211, 40.82115479859534], [-73.83985813718604, 40.82100998191922], [-73.83988142932418, 40.82089686751444], [-73.83998460083185, 40.82079477636269], [-73.83999403392806, 40.82068256106775], [-73.84005150923332, 40.820614445414265], [-73.8400535103904, 40.820512079962526], [-73.84010069135866, 40.82032276172762], [-73.8400605997814, 40.82023007173984], [-73.83996730528736, 40.82017908200399], [-73.83999860407422, 40.81993494681306], [-73.84014625733194, 40.81981924855244], [-73.84024605965783, 40.81982602352281], [-73.8402891194478, 40.81980770505641], [-73.84023155497631, 40.819735530771034], [-73.84028066974183, 40.81967849755755], [-73.84028506253651, 40.819626266126434], [-73.84036528525624, 40.8195777500243], [-73.84052340849311, 40.81953879722471], [-73.84061881496048, 40.819549500778415], [-73.84070339531256, 40.81947762296206], [-73.84081344956941, 40.81944469119037], [-73.84084352269801, 40.819355043389606], [-73.84091856215316, 40.81934212492081], [-73.84104454654545, 40.81923016959915], [-73.84117574214497, 40.81918308181378], [-73.84124966376413, 40.819161015081995], [-73.841289746031, 40.819156509570085], [-73.8413111172789, 40.81913614403746], [-73.84134926355487, 40.819122974148705], [-73.84142475199273, 40.81904188082877], [-73.84156703725515, 40.81894043221164], [-73.84170330169793, 40.81890421224721], [-73.84188441800475, 40.81885001810424], [-73.84192413895823, 40.8188323729219], [-73.84194908907305, 40.81880494783188], [-73.84198684834051, 40.818774373908084], [-73.84212585817498, 40.818458222415856], [-73.84213733844804, 40.818259033778745], [-73.84209366311856, 40.818059542951566], [-73.84237534420508, 40.81748984708056], [-73.84279875811097, 40.81693700864936], [-73.84289216450948, 40.816730453928784], [-73.84306379033725, 40.81655864502672], [-73.84304301763714, 40.816377320053974], [-73.8431528346121, 40.81627342796553], [-73.8431699276372, 40.81605105501337], [-73.84344776972225, 40.815698065257756], [-73.843580014338, 40.81546834659724], [-73.84374053000356, 40.81532169954207], [-73.84377226781889, 40.81513907127948], [-73.8439550510123, 40.814796012445896], [-73.84408472247657, 40.814571052924975], [-73.84417911440977, 40.814357177091956], [-73.84417920712757, 40.81418667106152], [-73.84438736896749, 40.81364513082717], [-73.84451791370577, 40.813425152899185], [-73.84464128728749, 40.81320279885393], [-73.8447574149284, 40.81297820360319], [-73.84486622624331, 40.81275150341687], [-73.84496132121113, 40.81253768005686], [-73.84504990735974, 40.81232224900447], [-73.84513193773182, 40.812105324608545], [-73.84520736885104, 40.811887022009515], [-73.84528019236548, 40.81177811674559], [-73.84536842801629, 40.81175143836059], [-73.845465386758, 40.81176496816685], [-73.84576561702221, 40.81158227884192], [-73.845854121851, 40.811429471534375], [-73.84591166916594, 40.81133011244603], [-73.84602600366073, 40.811023714421545], [-73.84617302872044, 40.81067666443039], [-73.84688871208358, 40.810521325178314], [-73.84719684509642, 40.81058048761422], [-73.84746772168266, 40.810903942854274], [-73.84758283399765, 40.81128175025538], [-73.84763099298789, 40.81135174127132], [-73.84770256622504, 40.811455764866025], [-73.84821486515582, 40.81246290949828], [-73.84876286855327, 40.812904686119495], [-73.84938556147972, 40.81323686534508], [-73.8500504981515, 40.81348902434006], [-73.85020486696234, 40.81353429551917], [-73.85041066820106, 40.81358402388847], [-73.85052470545138, 40.813830315346614], [-73.85060246203854, 40.81393020004212], [-73.85090731401337, 40.81419237663027], [-73.85091029734019, 40.814194942945846], [-73.8511447226069, 40.814396548876], [-73.8512981940995, 40.81438294371766], [-73.85246819393365, 40.81427920741919], [-73.85242403627308, 40.814420084234854], [-73.85262058756422, 40.814505749279334], [-73.85279369903853, 40.81443515054267], [-73.85291177501854, 40.81450099671153], [-73.8528188301477, 40.814335174744016], [-73.85265013335618, 40.814363146473674], [-73.85261262519265, 40.81396258284018], [-73.85288616298551, 40.81388171252951], [-73.85294165170168, 40.81381958893503], [-73.85294266398027, 40.81375024639095], [-73.85286936942734, 40.813692824923024], [-73.8529799523554, 40.8136738226034], [-73.85336857868805, 40.813593903136336], [-73.85374358027236, 40.81364583315394], [-73.85434655584949, 40.81386385791404], [-73.85484235409912, 40.814091518012425], [-73.8549550400126, 40.81419341717086], [-73.85514695315437, 40.81436899429632], [-73.85531511789715, 40.81441214218983], [-73.85538030243693, 40.81428888208278], [-73.8553196589575, 40.814161567150194], [-73.85504860133678, 40.81407318962293], [-73.85471008210905, 40.813844401415395], [-73.85443075596594, 40.81371791906975], [-73.85411268734923, 40.81363514632314], [-73.85385011057137, 40.813571412209946], [-73.85346774058524, 40.813477711608684], [-73.85335258746518, 40.81349036056817], [-73.85323900924007, 40.8135095422681], [-73.85312764700011, 40.813535148438184], [-73.85301912932853, 40.81356703454497], [-73.85291406875528, 40.81360502060807], [-73.852847930473, 40.8136326572358], [-73.85278364056278, 40.813662706287985], [-73.8527482834161, 40.81368364638444], [-73.85271000151495, 40.8137013710889], [-73.85266133386995, 40.81371795413689], [-73.85261014676433, 40.81372934610966], [-73.85255739434923, 40.813735334655185], [-73.85251652128306, 40.81373619352789], [-73.85247575393879, 40.81373380190724], [-73.85206357413182, 40.81372606016076], [-73.8519141729744, 40.81367205792277], [-73.85171353554064, 40.813646744048604], [-73.85148124228404, 40.813590682107], [-73.85136357625558, 40.81355006169712], [-73.85119997725175, 40.81355314437617], [-73.85109767952623, 40.81351388512053], [-73.85099218802844, 40.81347985266294], [-73.85083926181652, 40.81339204169612], [-73.85069123473296, 40.81329951159002], [-73.85054835887382, 40.8132024199404], [-73.8504108775591, 40.81310093211135], [-73.85034923468585, 40.81306335794553], [-73.85028418150665, 40.81302926391442], [-73.85021605769938, 40.81299882803898], [-73.85014521897382, 40.812972209238794], [-73.85009783305256, 40.81295700273269], [-73.84997954066965, 40.81291414443821], [-73.84986516854471, 40.8128655302934], [-73.849755201201, 40.81281136624995], [-73.8496501044987, 40.81275188177085], [-73.84955032365964, 40.8126873288585], [-73.84945628138209, 40.812617980986204], [-73.84936837604964, 40.812544131940214], [-73.84928698004323, 40.81246609457441], [-73.84921243816422, 40.81238419948505], [-73.84914506617369, 40.81229879360987], [-73.84908514945569, 40.81221023875826], [-73.84903294180673, 40.81211891007852], [-73.84903822877388, 40.812048716252406], [-73.84904536052214, 40.81198632429126], [-73.84906301850037, 40.8119053582321], [-73.84908240942389, 40.81183103394668], [-73.84911051614033, 40.81176601299328], [-73.84914037531507, 40.81169834023067], [-73.84917019428875, 40.81164925528569], [-73.84920883946317, 40.81156035034881], [-73.84924043533324, 40.811497990365154], [-73.84927896005829, 40.81146219294545], [-73.84934201516832, 40.81139721878769], [-73.84937358980308, 40.8113441523431], [-73.84938420255818, 40.81128707476753], [-73.84937559536057, 40.81122997375722], [-73.84929711666227, 40.81116083088631], [-73.8492482752338, 40.81112226347101], [-73.84921346822934, 40.81105981664474], [-73.84919088767829, 40.81100004157146], [-73.84917358424858, 40.810924340783984], [-73.84919469827568, 40.81086063912465], [-73.8492490511082, 40.81077972051286], [-73.84930338063882, 40.81070942442398], [-73.84935768132704, 40.810652405114325], [-73.84941898714325, 40.810587427448674], [-73.84945411997754, 40.810506484134905], [-73.84947877479328, 40.81042154456284], [-73.84949990037568, 40.810352532991686], [-73.84952977419123, 40.810278221124975], [-73.84957706330714, 40.81023048616367], [-73.84964356064835, 40.810188086771475], [-73.84971355923824, 40.810143036636525], [-73.84976960468839, 40.810086018230216], [-73.84979804522004, 40.81003287788506], [-73.84980999337282, 40.809998443815324], [-73.84981189671988, 40.80992940596733], [-73.84983485578132, 40.80982189365155], [-73.84987171574225, 40.80974891825656], [-73.84990892306315, 40.809660122089426], [-73.84976705103755, 40.809669120008586], [-73.84976011634066, 40.809645213220435], [-73.84978633908524, 40.80963993635042], [-73.84979155453436, 40.80965189202137], [-73.84988767869739, 40.80964405091255], [-73.84988421698621, 40.80962944226101], [-73.84991938701842, 40.809626665926885], [-73.84995635860089, 40.80955534779747], [-73.84998510045496, 40.80948187341391], [-73.85000539859047, 40.80940678988159], [-73.8500171018922, 40.809330656286136], [-73.85002012324549, 40.80925403953004], [-73.849848339326, 40.80926693630328], [-73.84984662518073, 40.80925232875296], [-73.85003001109644, 40.80923510332365], [-73.85004241612901, 40.809215408487624], [-73.85004422603993, 40.8091875293482], [-73.85004255373175, 40.809154334639324], [-73.85002340188676, 40.809123772963886], [-73.85000535107207, 40.80909363437765], [-73.84999554853006, 40.80907726851891], [-73.84997993834935, 40.80902546728386], [-73.84997477933598, 40.80898828546775], [-73.84996962763664, 40.808948448391995], [-73.84996732905364, 40.808875512454414], [-73.84974778878306, 40.80891364015451], [-73.84972865358634, 40.80887511163889], [-73.84995749465193, 40.80883006842509], [-73.84995116937989, 40.8087147201722], [-73.84955595679385, 40.80874477203535], [-73.84955076934938, 40.808720867510885], [-73.84995355606375, 40.80869340607634], [-73.84995975669729, 40.80867625909903], [-73.84997030101277, 40.80864971862477], [-73.84997211370978, 40.80862051218814], [-73.84996696802818, 40.808578018843576], [-73.84994260631682, 40.808532846727324], [-73.84990773115474, 40.80850093609059], [-73.84988093264295, 40.80847625242864], [-73.84990019134227, 40.80840965538153], [-73.84986663269066, 40.808337009760564], [-73.84978989800263, 40.8083073408305], [-73.84975944944871, 40.80826933666985], [-73.84975452880082, 40.808249803726646], [-73.84974410135968, 40.808224562735134], [-73.84973191508689, 40.80820463166331], [-73.84970227028309, 40.808178039490635], [-73.84968135249555, 40.808155441760775], [-73.84964993901428, 40.80813814087777], [-73.84961166420382, 40.808124185051255], [-73.8491657211891, 40.80822513593612], [-73.84917093986466, 40.808235764342896], [-73.84876007745984, 40.80833745779307], [-73.84870962479349, 40.80824046991644], [-73.84912748807834, 40.80813347459895], [-73.84913269939788, 40.808146758769695], [-73.84952218256736, 40.8080552777202], [-73.84943455377403, 40.80802546098172], [-73.8494264181715, 40.80795166988994], [-73.84937135407831, 40.80788026251642], [-73.8492610092135, 40.80783396016135], [-73.84908995466621, 40.80777079426387], [-73.84894530707565, 40.80770990905589], [-73.84876995488845, 40.807636097377575], [-73.84864314771036, 40.80754361587297], [-73.84852183536408, 40.8074637299373], [-73.8484164096164, 40.80736877579531], [-73.84836776547775, 40.807218051494516], [-73.84832891885054, 40.80703150497563], [-73.84828024018286, 40.806852405047735], [-73.84834937076192, 40.80667345962455], [-73.84839879797016, 40.80652432778419], [-73.8483795372141, 40.80636018633455], [-73.84828163452254, 40.80624069971052], [-73.84811499148918, 40.806128582675655], [-73.84798630581214, 40.80610305062379], [-73.84785003790938, 40.806076014266296], [-73.84763420691263, 40.80600859041145], [-73.84753621975607, 40.8059264027951], [-73.84751706251517, 40.80571750200647], [-73.847488226419, 40.805448909149185], [-73.84752776906166, 40.8053296047884], [-73.84766805165862, 40.80516644922731], [-73.8479029865273, 40.80500217939724], [-73.84807624599382, 40.80491964114916], [-73.8482536483152, 40.80490100697116], [-73.8484027288335, 40.804858076828026], [-73.84855540464778, 40.804793588072606], [-73.84860518188748, 40.80474244174422], [-73.8486478460621, 40.80469937148038], [-73.84870097964972, 40.80473178627716], [-73.84870289538134, 40.804832908365114], [-73.84870423878404, 40.8048584729291], [-73.8487431721285, 40.80489625968614], [-73.84879992124709, 40.804899029440456], [-73.84883551481951, 40.80484516792806], [-73.84883560644094, 40.80480482452005], [-73.84883562500292, 40.80479665116928], [-73.84887115183398, 40.804772439297615], [-73.84890663580971, 40.80476709519247], [-73.84894604005997, 40.8047621689947], [-73.84898109667138, 40.804780670479516], [-73.84903071942021, 40.80479690682731], [-73.84908379915261, 40.80485357982767], [-73.84910521575515, 40.80489775186137], [-73.84911910597006, 40.804926401396955], [-73.84916510390543, 40.80497767329903], [-73.84921466578409, 40.80502086441957], [-73.84933382307932, 40.80505121826175], [-73.8493178894178, 40.804661465703084], [-73.84941036337706, 40.804667976275645], [-73.849428527277, 40.80505982814053], [-73.84952672182332, 40.80506439844526], [-73.84961180552062, 40.80508607261303], [-73.84970756101629, 40.80509428291423], [-73.84982107183843, 40.80509443084343], [-73.84997722709161, 40.80505959498189], [-73.85006279732194, 40.80486563853722], [-73.85008420001887, 40.80477933154937], [-73.85011291202908, 40.80466355117527], [-73.85012150751717, 40.80463332861889], [-73.85013436531689, 40.804588107229755], [-73.85014867564297, 40.80453421881293], [-73.85019839036403, 40.80451002497395], [-73.85033560101455, 40.80451584359033], [-73.85036247088496, 40.80451256399025], [-73.8506495461609, 40.804477517135346], [-73.8509207057748, 40.80447786811723], [-73.85118552925947, 40.8044925861015], [-73.85126338998977, 40.80453037668598], [-73.85132411180251, 40.80455984994899], [-73.85147537230502, 40.804598378325544], [-73.8517780752273, 40.804593976711864], [-73.85204922599476, 40.80459911624106], [-73.85234565390326, 40.80458032926472], [-73.85257375622835, 40.80455088870426], [-73.85399349144966, 40.804461913159045], [-73.85407120427065, 40.80445746241942], [-73.85412473375457, 40.80446566886551], [-73.8542184149609, 40.80448002866804], [-73.85435087482364, 40.804513765251485], [-73.85460114248228, 40.80454764982743], [-73.85479994155997, 40.804547900395356], [-73.85503561758694, 40.804520223181086], [-73.85516083679211, 40.8044980005837], [-73.85517631512307, 40.80449574470605], [-73.85538916176097, 40.80446471807749], [-73.85561003777734, 40.80447059094388], [-73.8558549779329, 40.80451515019537], [-73.85611177205843, 40.80460355474822], [-73.85636513336867, 40.80469750496242], [-73.856614854502, 40.80479692395033], [-73.85686073107355, 40.80490173034878], [-73.85706828071218, 40.80499579164016], [-73.85727273269194, 40.80509369201627], [-73.85737000879435, 40.80515521818442], [-73.85746117275458, 40.80522193022442], [-73.85754574687121, 40.80529347857727], [-73.85762328796818, 40.8053694883417], [-73.8576933897171, 40.80544956123754], [-73.85775568476672, 40.805533277693534], [-73.85780984666786, 40.80562019904475], [-73.858015037578, 40.805774103446026], [-73.8580861892141, 40.80584179204565], [-73.85814131055963, 40.805918375210574], [-73.85817792087721, 40.806000420677286], [-73.85822115694067, 40.80607157611543], [-73.85825004218658, 40.80614727157391], [-73.85826360022342, 40.80622496518059], [-73.8582770672542, 40.80642348767274], [-73.85827985213197, 40.80664037718531], [-73.85866549456955, 40.806833088596065], [-73.85857242494637, 40.80731467797298], [-73.85841832709596, 40.808112040631514], [-73.85838996228809, 40.808258809148796], [-73.85861930569855, 40.808846836576414], [-73.85877452654408, 40.809233975638094], [-73.85895141221772, 40.810271417586556], [-73.85900066509616, 40.81019763659577], [-73.85900493674171, 40.81010470912203], [-73.85903336121947, 40.810046091275865], [-73.85907662566406, 40.80997555810217], [-73.85915209523831, 40.80995076988108], [-73.85931976564537, 40.809941462543314], [-73.85953938146619, 40.80989044997531], [-73.8596661655864, 40.809806564723154], [-73.85983152493267, 40.80973347730822], [-73.8601269523559, 40.80960549989125], [-73.86011826350311, 40.80956996152848], [-73.86008049117329, 40.80955908337169], [-73.86003529913395, 40.80951393486113], [-73.86003172846314, 40.809490322094156], [-73.8600088165894, 40.80948747650811], [-73.85995984693308, 40.809481397869405], [-73.85992715349192, 40.80944118855356], [-73.85992341159556, 40.80940215340927], [-73.85987123219722, 40.809384883359144], [-73.85977839204479, 40.809286757914435], [-73.85974669043343, 40.809277881916294], [-73.85971928075661, 40.809239285698126], [-73.85972355427504, 40.8092184023976], [-73.85967072961763, 40.80919825389722], [-73.85967606941364, 40.809172552709995], [-73.85962537388161, 40.80914597801559], [-73.8595405986246, 40.809019702446385], [-73.85948183810812, 40.80900634284723], [-73.85942148359356, 40.808998044457205], [-73.85936028769267, 40.80899491075598], [-73.85935625734362, 40.80898803428646], [-73.8593537005947, 40.808980761754704], [-73.8593526810898, 40.808973274193605], [-73.85935322420667, 40.80896575798885], [-73.85935531642495, 40.80895840023866], [-73.8593589056636, 40.808951384097334], [-73.85936390257604, 40.80894488421528], [-73.85940334758394, 40.80890502265233], [-73.8594494636649, 40.80886953437551], [-73.85932115264183, 40.80821573386948], [-73.85932421093707, 40.808206473413854], [-73.85932916230536, 40.808197698334624], [-73.85933588156571, 40.80818963048179], [-73.85934419884254, 40.8081824738257], [-73.85935390385922, 40.8081764092995], [-73.85936475125517, 40.80817159022571], [-73.85937646678792, 40.80816813843904], [-73.85938875426787, 40.80816614120708], [-73.85940108963192, 40.80816811325363], [-73.85941285110802, 40.80817155895378], [-73.85942373589205, 40.80817638959654], [-73.85943346375063, 40.80818248081511], [-73.85944178423567, 40.80818967578855], [-73.85944848313243, 40.80819778927942], [-73.85945338797475, 40.808206612402444], [-73.85945637248398, 40.80821591800244], [-73.85945735982212, 40.80822546650247], [-73.85950797756688, 40.80848949522924], [-73.8595139474048, 40.80856595204331], [-73.85952928167883, 40.808641651893545], [-73.8595538465831, 40.80871593407296], [-73.85958742774582, 40.808788150245576], [-73.85962973209718, 40.8088576701059], [-73.85967632676821, 40.808945447257834], [-73.85973120620523, 40.80903041481562], [-73.85979407932838, 40.80911212206512], [-73.85986461265254, 40.80919013558493], [-73.85994243205343, 40.80926404154603], [-73.86002712475386, 40.809333447906674], [-73.86011824151163, 40.809397986492776], [-73.86021529900421, 40.80945731495083], [-73.86031778239017, 40.80951111856422], [-73.86042514804214, 40.80955911192294], [-73.86053682643018, 40.809601040437975], [-73.86065222514291, 40.80963668169183], [-73.86077073203082, 40.80966584661852], [-73.86089171845343, 40.80968838050702], [-73.86228604582757, 40.80994591122738], [-73.86264680664812, 40.81001254001527], [-73.86440333379416, 40.81027598694017], [-73.86765646284456, 40.81058376193711], [-73.86868890960871, 40.81185427245648], [-73.86976955850041, 40.813423699757195], [-73.87077804885932, 40.814486895487775], [-73.87129421599971, 40.814729536037845], [-73.871716064501, 40.814922335490174], [-73.8765199603362, 40.81613106342415], [-73.87663459981849, 40.81571150804957], [-73.8780628329999, 40.81618263199993], [-73.88098029299991, 40.81823784499989], [-73.88244779599998, 40.81926832299992], [-73.88378786099997, 40.82193341299996], [-73.88423986499997, 40.82262562999994], [-73.88429488199988, 40.82271820499992], [-73.884332124, 40.82281014199988], [-73.88436128599993, 40.82287380099997], [-73.8843985489999, 40.822965575999866], [-73.88421976599987, 40.82299645799987], [-73.8827603039999, 40.82324854799993], [-73.88178621799986, 40.823401699999856], [-73.879930023, 40.82364106199991], [-73.87898855699989, 40.82371109099993], [-73.87806435799995, 40.82383439499989], [-73.87793513899997, 40.82385161099992], [-73.8771361349999, 40.82395802999987], [-73.87621305999994, 40.82408474999996], [-73.87528548299996, 40.82420788399992], [-73.87437004400003, 40.82433051099987], [-73.87423302099994, 40.824348586999946], [-73.87356808499986, 40.824419845999884], [-73.87288306599994, 40.82451595999987], [-73.872630712, 40.824553357999925], [-73.87258431899996, 40.82456256399997], [-73.87253656999984, 40.82456891299994], [-73.87216881599993, 40.82461073999989], [-73.87145992799994, 40.82472020899991], [-73.87051877799988, 40.824840902999895], [-73.86984170799977, 40.824923093999864], [-73.86922974599999, 40.82500248899989], [-73.8691174439999, 40.8250170579999], [-73.86849386399993, 40.825097955999894], [-73.86840249499991, 40.82510980799992], [-73.86787899199994, 40.82517771999988], [-73.86779104499986, 40.82518945099986], [-73.86770310799987, 40.82520117399989], [-73.86761742299993, 40.825213376999926], [-73.86753174000002, 40.82522556999991], [-73.86718504299985, 40.825274542999935], [-73.86707924599993, 40.825289487999896], [-73.86662745199993, 40.82535330499995], [-73.86653521299999, 40.82536633399993], [-73.86577274899996, 40.82547402999994], [-73.86567536799996, 40.8254877849999], [-73.86538050299997, 40.82552943099993], [-73.86529634199991, 40.825541317999864], [-73.86490949499989, 40.8255959549999], [-73.86474490899985, 40.82561811299995], [-73.86455514499985, 40.825643659999834], [-73.86446141899997, 40.82565627799988], [-73.86406913799999, 40.825709089999854], [-73.86395604399995, 40.82572431499988], [-73.86361696799989, 40.82576996199992], [-73.86354738899982, 40.82577932899992], [-73.86216687599976, 40.82596516299993], [-73.86120690799989, 40.82608779499987], [-73.85970333199991, 40.82627734899994], [-73.85867193999985, 40.82642407699992], [-73.85727871200001, 40.82659962399986], [-73.85700204899985, 40.82665967699993], [-73.85672105899991, 40.82670825199993], [-73.856436988, 40.82674508099989], [-73.85635677499994, 40.82675208499993], [-73.85632684699985, 40.82662383499993], [-73.85629023899993, 40.82648152999993], [-73.85626925499986, 40.826393776999936], [-73.856260876, 40.82632936699985], [-73.85621834499996, 40.82616062899991], [-73.85619656499995, 40.82604099499984], [-73.855971624, 40.82606387799991], [-73.85588881599983, 40.82567371799994], [-73.85564695999979, 40.82463286199989], [-73.85548601399985, 40.82391214599993], [-73.855312208, 40.82319243099992], [-73.85513639699997, 40.822436188999866], [-73.85207829099994, 40.82284723799988], [-73.849017675, 40.823259957999895], [-73.84595550900002, 40.823670982999886], [-73.84289858800003, 40.82408305899986], [-73.84306938499985, 40.82481661099991], [-73.84324615099987, 40.82555394499987], [-73.84341053499996, 40.82628047299992], [-73.84358503199985, 40.82700294399991], [-73.843728169, 40.82766817299993], [-73.84374214799988, 40.8277331529999], [-73.84375757299995, 40.82778278299986], [-73.84379428399991, 40.827890342999865], [-73.84381208799992, 40.827962324999895], [-73.8438380529999, 40.82810246899986], [-73.843851037, 40.82817200499993], [-73.84385743399987, 40.828199343999906], [-73.84389248199977, 40.82834904499988], [-73.84390624999999, 40.828407828999936], [-73.84370821, 40.8284387989999], [-73.843104377, 40.82853286499992], [-73.84289527503903, 40.828569176060846], [-73.842874508125, 40.828527571808635], [-73.84276131433994, 40.828327725005714], [-73.84272187785814, 40.82827562091454], [-73.84266998835984, 40.82804005329698], [-73.84265027487082, 40.8279089858428], [-73.84260103680262, 40.82780573035185], [-73.84256508338187, 40.8277392607169], [-73.84253934488183, 40.827599837775246], [-73.84253325730938, 40.82752730414838], [-73.84248198442683, 40.827477571696605], [-73.84246322388854, 40.82737496051706], [-73.84231712480313, 40.82718468468027]]]]}}, {\"id\": \"213\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 214, \"Shape_Leng\": 0.135300314246, \"Shape_Area\": 0.000811976523012, \"zone\": \"South Beach/Dongan Hills\", \"LocationID\": 214, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.08491019899996, 40.59570472799985], [-74.08406968699998, 40.59549626999994], [-74.08376862299998, 40.595520956999934], [-74.08330235099992, 40.59471632799989], [-74.0832075799999, 40.59453373499988], [-74.08304163599998, 40.59422972099989], [-74.08239591899998, 40.59319116199994], [-74.08177066499995, 40.59221445099991], [-74.08173342999991, 40.59215628599987], [-74.08029682099996, 40.59266864299983], [-74.07936282299987, 40.59299407499985], [-74.0784669379999, 40.5933000119999], [-74.07888191699993, 40.59394279199988], [-74.07802261099994, 40.59425482399996], [-74.07714922399995, 40.594534080999885], [-74.07626202999995, 40.5948225699999], [-74.07539899099994, 40.59510414799992], [-74.07453730499996, 40.595386779999885], [-74.073679591, 40.59566839599994], [-74.07281699299993, 40.59595221499992], [-74.07118634999996, 40.59648999299982], [-74.07043976099992, 40.59673474699988], [-74.06970478499989, 40.59537635999994], [-74.06964532799996, 40.59526401099989], [-74.06831810799993, 40.59278576599994], [-74.06814585499988, 40.59245576399995], [-74.06795227299993, 40.59208368299986], [-74.06751477499988, 40.591334642999925], [-74.06728410699992, 40.590986202999865], [-74.06666723999994, 40.590382761999905], [-74.06662559499993, 40.59034201999989], [-74.06651182899995, 40.59034492199989], [-74.0658293969999, 40.59005916499991], [-74.06577682899997, 40.590021390999915], [-74.06573690899995, 40.589969516999936], [-74.06571834999994, 40.58990810299984], [-74.06572536499986, 40.589845295999915], [-74.06575530299992, 40.58978965399993], [-74.06580036099992, 40.58974660899988], [-74.06572336299989, 40.58969105399995], [-74.06563611899992, 40.589628104999896], [-74.06503332378131, 40.58899099843518], [-74.06503309568002, 40.588909314195945], [-74.06454186133764, 40.58843399239428], [-74.06436003721248, 40.58826789117477], [-74.06440881666158, 40.588238924315746], [-74.06458752315092, 40.58840315741567], [-74.06470615524998, 40.58847254683954], [-74.06477833032984, 40.58848831072566], [-74.06593797099987, 40.58760609779563], [-74.06722276816986, 40.58673445026267], [-74.06933503355576, 40.584776520152325], [-74.06951435164832, 40.58458551737245], [-74.0698079096077, 40.58430404221734], [-74.07021154289465, 40.583981357711515], [-74.07096310182041, 40.58318199040263], [-74.07144837595611, 40.5824340679645], [-74.07149180521219, 40.58216878265253], [-74.07059533733295, 40.58169054342512], [-74.07065781537806, 40.5816256250378], [-74.07146670932998, 40.58202724389815], [-74.07177638431192, 40.581876535163985], [-74.07335389590794, 40.58067893913892], [-74.07383321073327, 40.58027722873235], [-74.07489264154759, 40.579317436177746], [-74.07499413073505, 40.57923655741631], [-74.07492909104904, 40.57917126032183], [-74.07482318784565, 40.579098932964285], [-74.07480676226291, 40.57909202569626], [-74.07478997100377, 40.5790856510728], [-74.07477284308196, 40.57907981912662], [-74.07475540795494, 40.57907454290568], [-74.07473769880751, 40.57906982892193], [-74.07471974443408, 40.57906568720836], [-74.0747015773599, 40.579062124109605], [-74.07468800489849, 40.57906013916494], [-74.07467460447475, 40.579057560047055], [-74.07466142311903, 40.57905439510209], [-74.0746485023713, 40.57905065519236], [-74.07463588772693, 40.57904635184811], [-74.07462361809372, 40.57904149995423], [-74.07461173941236, 40.57903611623391], [-74.07460028663816, 40.579030217920156], [-74.07458930110094, 40.579023824754714], [-74.07457881710285, 40.579016959667165], [-74.07456887026298, 40.57900964407804], [-74.07455949444534, 40.57900190208976], [-74.07454666036014, 40.57898720651877], [-74.07453317848523, 40.57897285446701], [-74.07451906179989, 40.57895885966328], [-74.07450432746322, 40.578945240692114], [-74.07448899504465, 40.578932010272695], [-74.07447307994613, 40.57891918749324], [-74.07445660063782, 40.57890678457116], [-74.07443957713203, 40.57889481724106], [-74.07442499929435, 40.57888615200647], [-74.07440993884515, 40.57887798229987], [-74.07439442348492, 40.57887032167316], [-74.07437848377208, 40.578863186189], [-74.07436215136022, 40.57885658839164], [-74.074345454168, 40.578850540827375], [-74.074328426925, 40.57884505620595], [-74.07431110216156, 40.578840145395624], [-74.07429351108699, 40.57883581675249], [-74.07427568952455, 40.578832078294624], [-74.07426066418708, 40.57882996792144], [-74.074245812177, 40.578827234392044], [-74.07423117920446, 40.57882388471294], [-74.07421680900784, 40.57881993108587], [-74.07420274268362, 40.57881538102322], [-74.07418902551538, 40.57881025325894], [-74.07417569662549, 40.578804558154914], [-74.07416279646655, 40.578798317128836], [-74.07415036416721, 40.57879154657362], [-74.07413808612743, 40.578783600027656], [-74.07412628574865, 40.57877524373007], [-74.07411498458016, 40.57876649291226], [-74.07410420131794, 40.57875736515281], [-74.0740939627892, 40.578747879700536], [-74.07408428615156, 40.578738053967584], [-74.07407519230351, 40.5787279107247], [-74.07406669862316, 40.578717468388966], [-74.07405882380822, 40.57870674621436], [-74.07404582592638, 40.57869557115831], [-74.0740322084776, 40.57868482956383], [-74.07401800049168, 40.578674545201395], [-74.07400322703648, 40.57866473597995], [-74.07398791515524, 40.57865541796498], [-74.07397209167445, 40.57864660956713], [-74.07395578869183, 40.57863832802113], [-74.07393903412809, 40.578630588051375], [-74.07392185897929, 40.578623403877636], [-74.07390268867573, 40.5786161315241], [-74.07388314529085, 40.578609454974696], [-74.07386326355447, 40.57860338693935], [-74.07384307577854, 40.57859793996175], [-74.07382261580653, 40.57859312005089], [-74.07380191528604, 40.5785889348929], [-74.07378101047931, 40.57858539267311], [-74.07375993347098, 40.578582498899436], [-74.0737466204695, 40.57857738841345], [-74.07373369993405, 40.57857172047078], [-74.07372121429081, 40.578565513137605], [-74.07370920047458, 40.57855878565612], [-74.0736976967387, 40.578551558273126], [-74.07368673738576, 40.57854385458835], [-74.07367635847415, 40.57853569652539], [-74.07366659189495, 40.57852711271125], [-74.07365746469746, 40.57851812490771], [-74.07364900987565, 40.578508765594584], [-74.07363842048949, 40.57849702688657], [-74.07362722877563, 40.57848561507887], [-74.07361545145487, 40.578474550096225], [-74.07360310700233, 40.57846384884735], [-74.07359021389335, 40.57845352840819], [-74.07357679631478, 40.5784436046784], [-74.07356287208148, 40.578434093561526], [-74.07354846340168, 40.578425010288534], [-74.07353359424235, 40.578416370926654], [-74.073518289883, 40.57840818584633], [-74.0735059130219, 40.578401751752644], [-74.07349309093149, 40.578395844320134], [-74.0734798634038, 40.578390484464535], [-74.07346627066252, 40.57838568589742], [-74.07345235029628, 40.57838146400724], [-74.07343698511879, 40.57839568418803], [-74.07342228481102, 40.57841030802299], [-74.07340826824962, 40.578425317909726], [-74.07339495255152, 40.57844069457177], [-74.07338235571389, 40.57845641940221], [-74.073377518446, 40.57845929629342], [-74.0733722356986, 40.578461676073886], [-74.07336659685528, 40.578463519819515], [-74.07336069570323, 40.57846479714735], [-74.07335462647943, 40.578465487223596], [-74.07334849309805, 40.57846557791998], [-74.07334239355201, 40.578465066828215], [-74.073336432439, 40.57846396393344], [-74.07333070316146, 40.57846228894463], [-74.0733253059379, 40.57846006525176], [-74.07332032496554, 40.57845733351098], [-74.07331584532228, 40.578454135885565], [-74.07331194199433, 40.57845052844999], [-74.07330867612431, 40.57844656645006], [-74.07330610403751, 40.578442320212325], [-74.07330426821355, 40.57843785672159], [-74.07330319905394, 40.5784332495038], [-74.07330291158699, 40.5784285779585], [-74.0733034140223, 40.57842391529095], [-74.07330469502064, 40.578419340079556], [-74.07331420346311, 40.578404592082975], [-74.07332463446693, 40.57839021038645], [-74.07333596236282, 40.57837622985229], [-74.07334815906086, 40.578362681993774], [-74.07334703225152, 40.578352798637894], [-74.07334515012121, 40.578342982776036], [-74.07334252105652, 40.57833326841131], [-74.07334090736205, 40.578327197500876], [-74.07334007906655, 40.57832103696331], [-74.07334004853416, 40.578314841907954], [-74.0733408134175, 40.57830867549499], [-74.07334152248278, 40.57830590226093], [-74.07334236696877, 40.57830259586141], [-74.07334469475148, 40.57829666232186], [-74.07334777133924, 40.57829093084747], [-74.07335157262345, 40.57828545640316], [-74.0733560566966, 40.578280292792655], [-74.07336118472098, 40.578275488289144], [-74.07336690379651, 40.57827109016974], [-74.07337316189489, 40.57826713967968], [-74.07337989556261, 40.578263678071885], [-74.07338704462865, 40.57826073436712], [-74.07339453332253, 40.57825833826658], [-74.07341048150207, 40.57825641220917], [-74.07342624827145, 40.57825375834969], [-74.0734417751959, 40.57825038660975], [-74.07345960990149, 40.5782328968504], [-74.07347805846352, 40.578215779951464], [-74.07349710639637, 40.578199050329474], [-74.0735026389054, 40.5781966400607], [-74.07350852334758, 40.57819477755539], [-74.073514668353, 40.57819349286008], [-74.0735209774885, 40.57819280714508], [-74.07352734903544, 40.578192730527164], [-74.07353368126489, 40.57819326390909], [-74.07353987705248, 40.57819439897652], [-74.07354583684445, 40.578196116695], [-74.07355146657468, 40.57819839300049], [-74.07355667809294, 40.57820118874749], [-74.07356138742034, 40.57820446193997], [-74.07356551979396, 40.57820815935125], [-74.07356901187005, 40.578212222722044], [-74.07357180930835, 40.57821658842577], [-74.0735738634751, 40.57822118797406], [-74.0735751468232, 40.578225946666414], [-74.07357563510095, 40.57823078980031], [-74.07357532163505, 40.57823564333189], [-74.07356610854268, 40.57824976192024], [-74.07355619533521, 40.57826360319215], [-74.07354559978785, 40.57827714736749], [-74.07353433462039, 40.57829037299427], [-74.0735224180513, 40.57830326364267], [-74.0735199987435, 40.578305678956475], [-74.07350986543725, 40.578315797858764], [-74.07349669675457, 40.578327959714436], [-74.07353135170506, 40.57834784557702], [-74.07356653230441, 40.578367186621364], [-74.07360222645721, 40.57838597330579], [-74.07363841745605, 40.57840419810166], [-74.0736996934535, 40.57843498022361], [-74.07376053874324, 40.578466256634016], [-74.07382094607229, 40.57849802465768], [-74.0738809055482, 40.578530279275824], [-74.07394041277513, 40.578563018314014], [-74.07406033780588, 40.57862739788969], [-74.07418094820548, 40.578691029715685], [-74.07430257821694, 40.57875771130072], [-74.07430790841188, 40.57876125411844], [-74.0743840161996, 40.57880248179907], [-74.07446028819841, 40.57883936042356], [-74.07454539912841, 40.57888481194712], [-74.07462334758766, 40.57892820057826], [-74.07470110047464, 40.57896342623951], [-74.07481070714091, 40.57902120169393], [-74.07508288192615, 40.579160158755506], [-74.07538416887525, 40.57880385281149], [-74.07579819540713, 40.57830925065062], [-74.07523453617956, 40.57799880325315], [-74.07534622582394, 40.57788796957173], [-74.07601596977891, 40.57824491229019], [-74.07611008044724, 40.578173861682544], [-74.07620499324025, 40.578126613398204], [-74.0762467632469, 40.57810441582899], [-74.07626037024005, 40.578085008718794], [-74.07628942409059, 40.578065592523316], [-74.07633845483512, 40.578036462249464], [-74.076366604199, 40.578021202205804], [-74.07639928398518, 40.57799485505246], [-74.07640834714078, 40.577975450946134], [-74.07639844892991, 40.57796977408893], [-74.07630470430556, 40.57792009669967], [-74.07629015242426, 40.57790763630175], [-74.07629741018434, 40.577898625093304], [-74.07631467297796, 40.5778986136275], [-74.07642748022657, 40.5779501201487], [-74.07646467115256, 40.57796709998583], [-74.07649283077285, 40.577960846125386], [-74.07651642995224, 40.57794004736422], [-74.07653911250937, 40.57791162845722], [-74.07655452620624, 40.5778825204822], [-74.07658356267683, 40.57784924785423], [-74.07662183569524, 40.577818535946605], [-74.07672273866132, 40.57773054541072], [-74.07688226472128, 40.57763678991154], [-74.076915528964, 40.577579500012654], [-74.07673747616732, 40.5773725093254], [-74.07674837365107, 40.57736695916287], [-74.07689787379739, 40.57751630829064], [-74.07693881528753, 40.57753434559136], [-74.0773747252254, 40.577282159408114], [-74.07830169156932, 40.57646471865795], [-74.07877412894697, 40.5760108222456], [-74.07926321256913, 40.57567129220574], [-74.07867933938911, 40.575081825223414], [-74.078751853163, 40.575042125637175], [-74.07933640538252, 40.57562720401683], [-74.08017867442044, 40.57518041825954], [-74.08050289147513, 40.57498045871067], [-74.08090683283069, 40.574689804751486], [-74.08133770718658, 40.57431998930679], [-74.08173725351443, 40.57393857852344], [-74.08230682682282, 40.57342250310401], [-74.08259185204045, 40.57314951590206], [-74.08300620740229, 40.57278557004956], [-74.08359916061018, 40.57237147537055], [-74.08459134860527, 40.57155929050495], [-74.08495715819193, 40.57125984002107], [-74.08597647499992, 40.57223697099986], [-74.08765791399992, 40.57301266499997], [-74.08769737599994, 40.57306188399989], [-74.08775041799994, 40.57311057999993], [-74.0888697769999, 40.57372323799989], [-74.08960544799997, 40.5741304699999], [-74.09032842599996, 40.57451533599988], [-74.0910942809999, 40.57489922899992], [-74.09182234199987, 40.57527493299988], [-74.09257214599997, 40.57567191299993], [-74.09335651099995, 40.576047193999926], [-74.094086718, 40.57641955099987], [-74.09483640599994, 40.57680653799987], [-74.09517725099997, 40.57702113899995], [-74.09613172499989, 40.57752610599994], [-74.09741191099992, 40.57817443199992], [-74.09851032499989, 40.5787306869999], [-74.09940090399988, 40.57918675599992], [-74.10039083499996, 40.579688665999925], [-74.09974044199997, 40.58020781799996], [-74.09908893799992, 40.58072522899993], [-74.09844000899997, 40.581241705999915], [-74.1006131089999, 40.58271119099989], [-74.10249695999993, 40.58401503299994], [-74.10265901099994, 40.58408819999994], [-74.10278862899995, 40.58422448699991], [-74.10525775499997, 40.58592143599986], [-74.10454864599998, 40.58650064899995], [-74.10392537999996, 40.58707587399993], [-74.10357323099993, 40.587393759999884], [-74.10344974399993, 40.587505230999945], [-74.10328237999992, 40.58765630999993], [-74.10317352499999, 40.58775457099988], [-74.10292541199989, 40.58797853999988], [-74.10282423999995, 40.58806986599994], [-74.10256758199995, 40.588301540999964], [-74.10246256599999, 40.588396335999924], [-74.10207990099998, 40.58874175099988], [-74.10149482299994, 40.58934426099992], [-74.10088251699995, 40.59017921599992], [-74.10089056999995, 40.59042994899993], [-74.10090957399993, 40.59102161399986], [-74.10097579699995, 40.591832233999845], [-74.10083052299994, 40.592632305999864], [-74.100817394, 40.592668734999904], [-74.10074056699987, 40.592844566999894], [-74.10074047299992, 40.592844782999926], [-74.10074036799995, 40.592844988999914], [-74.1006473839999, 40.59301653999994], [-74.10064730199993, 40.59301670199992], [-74.10064720699992, 40.59301685499988], [-74.10053851599993, 40.5931832049999], [-74.10041498999992, 40.59334309299988], [-74.10006671900001, 40.59364027099991], [-74.09979588999995, 40.59387136599988], [-74.09949192299995, 40.5940477809999], [-74.09908179799994, 40.59428707099988], [-74.0987378739999, 40.59448771499993], [-74.0980683909999, 40.59488151999991], [-74.09798651199996, 40.594929686999876], [-74.09722072899997, 40.59536515499987], [-74.09586192699999, 40.59595088899989], [-74.09490220899987, 40.59629562299991], [-74.09427668499993, 40.59644927099987], [-74.09375453799998, 40.59674488999988], [-74.09336426799996, 40.5970330689999], [-74.09272305199994, 40.59746426399992], [-74.09228640999991, 40.59808239999993], [-74.09192827899989, 40.598504050999914], [-74.09178257499993, 40.59871923899987], [-74.09175787199997, 40.59880142199993], [-74.09173885999999, 40.59888447899991], [-74.09172558899995, 40.59896818299989], [-74.09171809699998, 40.5990523009999], [-74.09171640399988, 40.59913660299988], [-74.09172051499995, 40.59922085699985], [-74.0916133049999, 40.59926156199984], [-74.09147742399989, 40.599408489999895], [-74.09083323299996, 40.59998852399986], [-74.09060390399996, 40.60019882399991], [-74.09036073899993, 40.60040036399994], [-74.09010463499996, 40.600592348999896], [-74.08983655899989, 40.60077409499996], [-74.08947630599992, 40.60090432199991], [-74.08920169499996, 40.600963789999916], [-74.0883067039999, 40.60110777999993], [-74.08666235099997, 40.6012335389999], [-74.08647535899999, 40.60126896199992], [-74.08625420399993, 40.601324188999854], [-74.085837971, 40.60142777299994], [-74.08520894199994, 40.60160095799991], [-74.08457153399995, 40.601822479999946], [-74.08432821299992, 40.6018726829999], [-74.08383802499996, 40.60192576699993], [-74.082815161, 40.60197418199989], [-74.08218729299988, 40.601941007999876], [-74.08184498199992, 40.601535196999926], [-74.0813965889999, 40.60092495199989], [-74.081318208, 40.60080532899989], [-74.08125785899989, 40.60067884799985], [-74.081216903, 40.600547931999905], [-74.08119595999997, 40.600415172999874], [-74.08116420899995, 40.600206553999946], [-74.08130343799999, 40.59938765599991], [-74.08144216599995, 40.5991592199999], [-74.0814942209999, 40.599081990999935], [-74.08160410699992, 40.59902445899997], [-74.08199990199995, 40.59868243799987], [-74.0823092379999, 40.59845665499994], [-74.08248303399992, 40.59832980799991], [-74.08360775099997, 40.597912413999936], [-74.08469129699996, 40.597507131999926], [-74.08476803199989, 40.597463562999955], [-74.08476811499993, 40.59746351699994], [-74.08476818599993, 40.59746346299987], [-74.08497132899998, 40.59732510999985], [-74.0851581859999, 40.59717283699992], [-74.08515824499993, 40.597172791999874], [-74.08515829199987, 40.59717273799993], [-74.08532650099991, 40.59700832199995], [-74.08547413999999, 40.596833816999876], [-74.08585350599992, 40.5959409809999], [-74.08587511899998, 40.59589012099985], [-74.08578780799998, 40.59588407999995], [-74.08491019899996, 40.59570472799985]]]}}, {\"id\": \"214\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 215, \"Shape_Leng\": 0.103556453625, \"Shape_Area\": 0.000395647022725, \"zone\": \"South Jamaica\", \"LocationID\": 215, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.7769103769999, 40.70252488499994], [-73.77638291300003, 40.7020949279999], [-73.77586892899987, 40.70165517099991], [-73.77536886199987, 40.701205982999966], [-73.77488318599994, 40.7007477519999], [-73.77441230099998, 40.70028087699991], [-73.77395659499986, 40.699805781999906], [-73.77446186199991, 40.69899514599993], [-73.7745937749998, 40.69879949799986], [-73.774892245, 40.698387447999906], [-73.77513633799991, 40.69805046599988], [-73.77532842099995, 40.69778566699989], [-73.77567626099992, 40.6973050299999], [-73.7757628779999, 40.697182862999924], [-73.77608749199995, 40.6967252309999], [-73.77618932399996, 40.696574341999955], [-73.77659775499988, 40.695969113999865], [-73.77703882099993, 40.69536433699991], [-73.77725734399985, 40.695064041999885], [-73.77783593399984, 40.69432133499986], [-73.77863399799988, 40.694013787999914], [-73.77944081299995, 40.693645109999856], [-73.78018646099989, 40.6930152299999], [-73.7802372379999, 40.692972334999894], [-73.77919838599988, 40.691650567999915], [-73.77789337399993, 40.68991768999992], [-73.77796055199985, 40.689894809999956], [-73.77891249599985, 40.689570646999925], [-73.77977295099997, 40.689281169999916], [-73.78078155499992, 40.69102994299994], [-73.78160296599982, 40.69068690399989], [-73.7824271229998, 40.69034436899994], [-73.7832662499999, 40.68999429299992], [-73.78409903999982, 40.68964775499987], [-73.78479240699981, 40.689361599999856], [-73.78580499899994, 40.68899689699988], [-73.78643950599985, 40.688781710999905], [-73.78698344499996, 40.68860847699995], [-73.78764209899992, 40.68843484999997], [-73.78867911999995, 40.68818018399991], [-73.78920529899993, 40.688053212999904], [-73.78962631299987, 40.68793905899987], [-73.79038468299987, 40.68775361399986], [-73.79210699399984, 40.687047535999945], [-73.79248043499994, 40.68688962999991], [-73.79317220799987, 40.68659668599987], [-73.793908417, 40.68630469099985], [-73.79437087299995, 40.68691258199987], [-73.79477837399985, 40.68747068399995], [-73.7952109489999, 40.688094665999884], [-73.79561486799987, 40.688656890999916], [-73.7966293329999, 40.68822145199994], [-73.79746057199985, 40.68787230199989], [-73.79828588299998, 40.68752401899992], [-73.79913531299994, 40.687181958999915], [-73.79995042799996, 40.68682771899991], [-73.79898209099994, 40.68485873799988], [-73.79943031499988, 40.684731988999935], [-73.79986119199985, 40.684610142999944], [-73.80077394899995, 40.684349927999904], [-73.80168572400002, 40.68409195599986], [-73.80256430899988, 40.68384243899986], [-73.80344528099988, 40.68359185699996], [-73.80432351099998, 40.683341794999855], [-73.80519209299995, 40.683096057999876], [-73.80568769599995, 40.68295563299993], [-73.80576840199991, 40.682932390999916], [-73.80606202699997, 40.6835543599999], [-73.80660608799995, 40.68468114599993], [-73.80709063399985, 40.68563031099991], [-73.80733621899985, 40.68612544699994], [-73.80787469700006, 40.68709833799987], [-73.80807342399996, 40.687452823999884], [-73.80857655199986, 40.688330889999925], [-73.80874358599979, 40.6886258839999], [-73.80892417199986, 40.68894480899991], [-73.809131198, 40.68930752699989], [-73.80954787499984, 40.690037545999886], [-73.809723267, 40.69034634399991], [-73.80995601299992, 40.69075611299995], [-73.8102442619999, 40.69126358699989], [-73.81063298299996, 40.69194738299988], [-73.81054216699988, 40.69197245899991], [-73.81026957999993, 40.692045248999925], [-73.80971017199981, 40.692199383999885], [-73.80890216900004, 40.69256434899993], [-73.80828132399985, 40.69214738599988], [-73.80803640399986, 40.69198289499991], [-73.80723957200003, 40.69143198799988], [-73.80642479399984, 40.690873857999904], [-73.80555849799987, 40.691608704999965], [-73.80468792999994, 40.6923455719999], [-73.80381770599995, 40.69307963199988], [-73.80295940099998, 40.69381747099987], [-73.80211252599995, 40.69454449899996], [-73.80119617899997, 40.69533648899994], [-73.80053204200006, 40.69585669199992], [-73.79985927599985, 40.69634707999995], [-73.79918226899999, 40.6968714759999], [-73.79876814999977, 40.69725955099994], [-73.79854640199993, 40.69746653299995], [-73.79786193399997, 40.69799340399991], [-73.79753672799987, 40.69819892499987], [-73.79722374699978, 40.6983594569999], [-73.7964760649998, 40.698693084999924], [-73.79572173799994, 40.69901473499993], [-73.79449920899998, 40.699385776999854], [-73.79357708699986, 40.699756471999876], [-73.79284356499997, 40.700243682999876], [-73.79260506799987, 40.70043209999988], [-73.79246111600004, 40.70055378899984], [-73.79246106899983, 40.70055382499993], [-73.79232916999992, 40.70068422399992], [-73.79232905199986, 40.70068434099988], [-73.79232894499991, 40.70068447699989], [-73.79221043999986, 40.700822508999885], [-73.7922102969999, 40.7008226709999], [-73.79221016699994, 40.700822850999934], [-73.7921061119999, 40.70096735699988], [-73.7921059579999, 40.70096757299994], [-73.79210582699996, 40.700967788999954], [-73.7920169119999, 40.70111772699987], [-73.79194372099994, 40.701271725999945], [-73.79186916599991, 40.70136613699988], [-73.79235282099985, 40.70209793999988], [-73.79239170299988, 40.7021665299999], [-73.79219799199994, 40.702200687999905], [-73.79200046599996, 40.70222080899988], [-73.7918017669999, 40.7022266369999], [-73.791637999, 40.7022205499999], [-73.7914732489998, 40.70222990499984], [-73.79131126499983, 40.70225465399992], [-73.79115569699985, 40.70229407599992], [-73.79055566899994, 40.702567795999904], [-73.78999405799982, 40.7028714169999], [-73.78977597299989, 40.70297886399991], [-73.78966391499986, 40.703039488999934], [-73.78919309, 40.70320438499993], [-73.78835413999998, 40.70346023099993], [-73.78804161499981, 40.70355559599991], [-73.78751689999984, 40.703715081999896], [-73.78597976500005, 40.704186615999916], [-73.7848487629999, 40.70458556999992], [-73.78442027299995, 40.7047624669999], [-73.78256582599995, 40.70542486399989], [-73.78147887399984, 40.705455250999854], [-73.78034491200003, 40.70541241699986], [-73.7798182799998, 40.70492789799992], [-73.77916009800003, 40.70438896699998], [-73.77849131099994, 40.70383322599987], [-73.77791103799981, 40.70328238899989], [-73.7769103769999, 40.70252488499994]]]}}, {\"id\": \"215\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 216, \"Shape_Leng\": 0.116797554681, \"Shape_Area\": 0.000816076647781, \"zone\": \"South Ozone Park\", \"LocationID\": 216, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.80857655199986, 40.688330889999925], [-73.80807342399996, 40.687452823999884], [-73.80787469700006, 40.68709833799987], [-73.80733621899985, 40.68612544699994], [-73.80709063399985, 40.68563031099991], [-73.80660608799995, 40.68468114599993], [-73.80606202699997, 40.68355436099992], [-73.80576840300003, 40.68293239099989], [-73.80551811199996, 40.68240820999994], [-73.80522682899998, 40.681798149999935], [-73.80492976299996, 40.68117697399989], [-73.80464517099982, 40.68057988999991], [-73.80428288199992, 40.679817355999916], [-73.80410226599996, 40.67942454999991], [-73.80323027699988, 40.677681056999894], [-73.8030752049999, 40.67734560099988], [-73.80267769599996, 40.67649796699987], [-73.80238906400001, 40.67588847899992], [-73.80202521799998, 40.675114272999934], [-73.80168194799997, 40.674383848999916], [-73.80142836799988, 40.673666833999874], [-73.80140210699984, 40.67356131999987], [-73.80130166399982, 40.67312152199985], [-73.80119461499989, 40.67242246499992], [-73.80117234299993, 40.67220503199988], [-73.80115934999986, 40.67189388999986], [-73.80115538600003, 40.67179912099987], [-73.80114238199995, 40.67148794299991], [-73.80113697199992, 40.67135852899987], [-73.8011343009999, 40.671294461999906], [-73.80113236199993, 40.67054866799994], [-73.80116000899989, 40.66980042099991], [-73.80117909099995, 40.66910026499992], [-73.80120616599989, 40.66894442399993], [-73.801250796, 40.66838594499995], [-73.80134286799985, 40.66777379699988], [-73.80141266299995, 40.66735840999995], [-73.80153052599994, 40.66683715899995], [-73.80155565300004, 40.6667533639999], [-73.8016677679998, 40.66637296299996], [-73.80168266599995, 40.66632235299992], [-73.80170297899987, 40.66626406999993], [-73.80183646299999, 40.665881093999914], [-73.80203207699995, 40.66524802499994], [-73.80214866500003, 40.66524995299992], [-73.80299527199999, 40.665216609999916], [-73.80377524499997, 40.66516018399992], [-73.8043205059999, 40.66512073299992], [-73.8057239779999, 40.664960640999894], [-73.80614396, 40.664912729999855], [-73.80764810699988, 40.6646183059999], [-73.80779777599999, 40.66458900799989], [-73.8084501619999, 40.664461300999896], [-73.80884594199996, 40.66429766899991], [-73.80945926899989, 40.664044088999944], [-73.81006836999995, 40.66379225199996], [-73.81061044899995, 40.6634808539999], [-73.81116608799995, 40.663177473999895], [-73.81176694899986, 40.66293453599996], [-73.81228672799995, 40.66282283799994], [-73.81281616300001, 40.66272443599994], [-73.81335011199994, 40.662640078999836], [-73.81388758999984, 40.662569925999925], [-73.81442762999983, 40.662514101999896], [-73.81496922499986, 40.66247266799991], [-73.81791759699993, 40.66295601599991], [-73.82306169099988, 40.66353701599996], [-73.82427005699986, 40.663576563999904], [-73.82486060999993, 40.663576789999865], [-73.82503467599989, 40.66358013399989], [-73.82571227999983, 40.66359313999991], [-73.82629450900006, 40.663638333999856], [-73.82687353900002, 40.6637041069999], [-73.82713313599999, 40.66375536799993], [-73.82789758799997, 40.66390630699991], [-73.82910033499995, 40.664250737999886], [-73.829388642, 40.664364648999886], [-73.83003156899997, 40.66458188799992], [-73.83205764999998, 40.665255315999936], [-73.83248638399985, 40.66535491099996], [-73.832705339, 40.665405772999854], [-73.83291947199984, 40.665765048999916], [-73.83295614199983, 40.665827065999956], [-73.83300277599999, 40.66592051799994], [-73.83334819299995, 40.66677318699994], [-73.83527457099994, 40.6708004029999], [-73.83600752699994, 40.67226955899984], [-73.83641423899985, 40.673115588999956], [-73.83680894799993, 40.67396111699987], [-73.83713760399998, 40.67466513399992], [-73.83749571700004, 40.67540682399994], [-73.83784372299996, 40.67607812399992], [-73.83811946699984, 40.67662988899986], [-73.83794250999986, 40.676794208999866], [-73.8395206529999, 40.679961868999946], [-73.84012856099982, 40.6811466509999], [-73.83952877099979, 40.68126341499991], [-73.83865065799998, 40.68143823699989], [-73.8377973829999, 40.681670585999875], [-73.8377888039999, 40.681673896999904], [-73.83772037399996, 40.681700306999915], [-73.83713815499993, 40.68192499999988], [-73.837115114, 40.681933891999954], [-73.83697486299991, 40.6819880179999], [-73.83807769099997, 40.68423890999988], [-73.83717651699999, 40.68449539199987], [-73.83595082799985, 40.68484354599989], [-73.8351108629999, 40.6850876819999], [-73.83426545999994, 40.685327337999894], [-73.833421907, 40.68556778199997], [-73.83257747700002, 40.685806345999914], [-73.83172298299989, 40.6860507049999], [-73.83088556299994, 40.686286547999885], [-73.83003934099995, 40.68652935899991], [-73.82918242299996, 40.68677080599985], [-73.82835620899998, 40.6870028089999], [-73.82753021399995, 40.687239482999914], [-73.82851136799994, 40.689239055999934], [-73.82768417699987, 40.68947479999991], [-73.82685467499992, 40.689709535999974], [-73.82599347399987, 40.689953661999894], [-73.82501128899997, 40.68795231799991], [-73.82398034299996, 40.68824526799993], [-73.82293216299993, 40.688543000999836], [-73.82189346699995, 40.688838225999866], [-73.82085264199988, 40.68913438299991], [-73.81993922399998, 40.689393691999925], [-73.81906075299987, 40.689642441999894], [-73.81818710999987, 40.68989207499987], [-73.8169264139999, 40.69025014699988], [-73.81607487599989, 40.690492269999886], [-73.81522826699991, 40.69073347699986], [-73.81438438, 40.69097195299984], [-73.81354014199995, 40.691211367999905], [-73.81309863099992, 40.6913181019999], [-73.81271762199997, 40.69140780299994], [-73.81186315099997, 40.69163007999991], [-73.8109881059999, 40.691855044999926], [-73.81072587499985, 40.69192242199995], [-73.81063298299996, 40.69194738299988], [-73.8102442619999, 40.69126358699989], [-73.80995601299992, 40.69075611299995], [-73.809723267, 40.69034634399991], [-73.80954787499984, 40.690037545999886], [-73.809131198, 40.68930752699989], [-73.80892417199986, 40.68894480899991], [-73.80874358599979, 40.6886258839999], [-73.80857655199986, 40.688330889999925]]]}}, {\"id\": \"216\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 217, \"Shape_Leng\": 0.0553905251285, \"Shape_Area\": 0.000114733270758, \"zone\": \"South Williamsburg\", \"LocationID\": 217, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95554165799997, 40.70759344799989], [-73.95401701399999, 40.7070802409999], [-73.95375309099992, 40.70698188099987], [-73.95291388700002, 40.70670031099991], [-73.95251503799987, 40.70655858699992], [-73.95229955499995, 40.70649029299996], [-73.95191055699979, 40.7063600789999], [-73.95108529899998, 40.7059129519999], [-73.95095606199983, 40.70584293599995], [-73.95023693799996, 40.70547324699994], [-73.94983788699977, 40.70522902199988], [-73.94947068299989, 40.7050190189999], [-73.949253357, 40.70489593099994], [-73.94866498899984, 40.704565073999945], [-73.94854962699999, 40.704500430999936], [-73.948140661, 40.704271253999885], [-73.94785367299988, 40.704111382999876], [-73.94705205299991, 40.70366394899985], [-73.94753858099985, 40.70335064999991], [-73.94937873499998, 40.70215888999993], [-73.95027424099992, 40.70157924199989], [-73.95128819399989, 40.700922364999855], [-73.95192018899999, 40.70148754899992], [-73.95255052799989, 40.70205166699994], [-73.95318085199992, 40.702616904999935], [-73.95381196900003, 40.703180979999914], [-73.95572360999994, 40.70194576999991], [-73.95745736399992, 40.70082260299993], [-73.95722517499998, 40.69999934999987], [-73.95716730100004, 40.69970786799989], [-73.95701993199991, 40.69897391399995], [-73.9579593819998, 40.69882000299991], [-73.95885874699991, 40.69867732599986], [-73.96019688899986, 40.69846272699991], [-73.96105100699992, 40.69832607899989], [-73.96189837799996, 40.69818833299994], [-73.96214289299981, 40.69814945699991], [-73.96233072899996, 40.698119593999884], [-73.96235903300003, 40.69811509199996], [-73.9623976909999, 40.69810894499991], [-73.96248295899996, 40.698095383999956], [-73.96283963699993, 40.698038667999946], [-73.9623657999999, 40.69960558599991], [-73.96217978199998, 40.70022070999989], [-73.9625725539999, 40.70051219899991], [-73.96318252099994, 40.70100009499989], [-73.96386889399994, 40.70154188899993], [-73.96451844799995, 40.702074977999864], [-73.96520438999994, 40.70262274499987], [-73.96587418600001, 40.70315769499991], [-73.96607413599993, 40.703320152999936], [-73.96653747199996, 40.70369661299984], [-73.96720961999985, 40.70425570899993], [-73.96761547799979, 40.70496163499988], [-73.96795139100001, 40.705731216999894], [-73.96838933699995, 40.70682918699989], [-73.96728890199985, 40.70700200299994], [-73.9659321969999, 40.70724387399989], [-73.96484837799986, 40.70745361099987], [-73.96442087399987, 40.707533722999884], [-73.96327873800001, 40.70772929899995], [-73.96274333099983, 40.707685081999955], [-73.96213128899981, 40.707622692999955], [-73.96186019899987, 40.70759505699993], [-73.96114422899987, 40.707523595999874], [-73.96050333599985, 40.70746152299992], [-73.96040383200001, 40.707451885999916], [-73.95993835900006, 40.70740680099993], [-73.95939542499995, 40.707352407999984], [-73.95920084799998, 40.70733158499991], [-73.958455768, 40.707251846999895], [-73.95838432400002, 40.70767669099991], [-73.95815250199996, 40.70804091799986], [-73.9579043079999, 40.70842717299994], [-73.95735981099993, 40.70823462399988], [-73.95728255399992, 40.70820730999989], [-73.95720321600002, 40.708179246999876], [-73.95672449099989, 40.70800995199991], [-73.95626945199986, 40.70784956199987], [-73.95554165799997, 40.70759344799989]]]}}, {\"id\": \"217\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 218, \"Shape_Leng\": 0.0837008281049, \"Shape_Area\": 0.000281293736407, \"zone\": \"Springfield Gardens North\", \"LocationID\": 218, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.76561573099998, 40.6747790199999], [-73.76380484799996, 40.67376993299995], [-73.76371825199992, 40.67431589199985], [-73.76367677599995, 40.67474552499992], [-73.76360070799988, 40.67474980899989], [-73.76352325899985, 40.67473995099995], [-73.76345120599991, 40.67471618299986], [-73.76339040599994, 40.674681117999924], [-73.76248600399998, 40.67419498199988], [-73.76114911299982, 40.67476361999996], [-73.76031463999993, 40.67510997099984], [-73.75880484999992, 40.67300565399993], [-73.75859630299982, 40.67274790399996], [-73.75857198899986, 40.67263733099987], [-73.75814760699994, 40.6727069139999], [-73.75643906099987, 40.672441159999906], [-73.75636239300002, 40.67242923699999], [-73.75646388399996, 40.67168400299994], [-73.7564837459999, 40.671489865999874], [-73.75646127199984, 40.67109403099993], [-73.7564247869999, 40.67063356499994], [-73.75640621299988, 40.670372914999895], [-73.75643734399979, 40.669666378999906], [-73.75668454799994, 40.66875852199992], [-73.75671602499992, 40.66856252699993], [-73.75699533399998, 40.66774126799994], [-73.75731692799982, 40.66704031599992], [-73.75763129599993, 40.66645115699992], [-73.75772789299985, 40.66646098699994], [-73.76079559400002, 40.66677326799992], [-73.761822313, 40.666877759999856], [-73.76270581499982, 40.666989484999895], [-73.76375687699996, 40.667094613999915], [-73.76493070899986, 40.6672271529999], [-73.76675402199996, 40.66743299499994], [-73.7715446069999, 40.66796326899992], [-73.77342462799993, 40.66802761899988], [-73.77431099300001, 40.667981732999884], [-73.77522378799983, 40.66791507199986], [-73.77611752599984, 40.6678457269999], [-73.77701324899986, 40.66777636199988], [-73.77791285799994, 40.667715066999925], [-73.77970512300006, 40.667590197999864], [-73.78062532299994, 40.66752330999992], [-73.78179791399985, 40.667440441999936], [-73.78265442899993, 40.66736859699988], [-73.78330043299998, 40.66731460599988], [-73.78502452299982, 40.66722991499991], [-73.78566409999999, 40.667187540999905], [-73.78697007199989, 40.66708518199989], [-73.78940999399987, 40.66684115899995], [-73.78915781999983, 40.66833809599989], [-73.78883107999984, 40.66982348699987], [-73.78847621499992, 40.67132650099996], [-73.78808647399985, 40.6730096269999], [-73.78686902099984, 40.672823923999864], [-73.78573730899993, 40.67238503499988], [-73.78532371499995, 40.67312502099992], [-73.78511037999995, 40.673478363999905], [-73.78484333899993, 40.67390332299989], [-73.78434496299991, 40.6746964399999], [-73.78416807199993, 40.67491103399994], [-73.78397384099986, 40.67511760899993], [-73.78397363999976, 40.67511780699991], [-73.78397345, 40.675117994999944], [-73.78376278399993, 40.67531538999988], [-73.78376261799986, 40.67531553299985], [-73.78376243999988, 40.675315685999905], [-73.783536133, 40.675503090999904], [-73.7832954899999, 40.675679444999915], [-73.78304214600001, 40.67584371699991], [-73.782218084, 40.676437536999906], [-73.78157185499998, 40.676901330999954], [-73.78128277899985, 40.67710972999991], [-73.7809512629998, 40.677353507999946], [-73.78025859499995, 40.6778409239999], [-73.77947817299986, 40.67836454999986], [-73.7793120879999, 40.6784368039999], [-73.77841935399988, 40.6790308799999], [-73.77820864799988, 40.67918207599993], [-73.77798227199999, 40.67932109299989], [-73.77774211299992, 40.67944657599985], [-73.77749031599994, 40.67955740299992], [-73.77722923600001, 40.679652720999904], [-73.77636248100002, 40.679935908999916], [-73.77549588399995, 40.680219045999934], [-73.77475342099996, 40.68046728099991], [-73.77469730199986, 40.680486045999906], [-73.77346815399997, 40.67971949999986], [-73.77177843899987, 40.678668299999906], [-73.77109570599985, 40.678243582999905], [-73.77030295199995, 40.67775050099993], [-73.76932752799988, 40.677143788999906], [-73.7683733069999, 40.67655024099987], [-73.76561573099998, 40.6747790199999]]]}}, {\"id\": \"218\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 219, \"Shape_Leng\": 0.155126669675, \"Shape_Area\": 0.000428176336771, \"zone\": \"Springfield Gardens South\", \"LocationID\": 219, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.761822313, 40.666877759999856], [-73.76079559299997, 40.66677326799993], [-73.75772789299985, 40.66646098699994], [-73.75763129599993, 40.66645115699992], [-73.75756437999986, 40.666448683999924], [-73.75602173399979, 40.66639171599988], [-73.75551141799991, 40.66639876399988], [-73.7552623119999, 40.666394765999954], [-73.75336722899982, 40.666400624999916], [-73.75318579199988, 40.66640002899992], [-73.75175886599992, 40.6663953199999], [-73.75073016999994, 40.666401901999954], [-73.74865865699996, 40.666383527999905], [-73.74608613399998, 40.66640039999987], [-73.74536651700001, 40.66640403199993], [-73.74440564499986, 40.66640390999991], [-73.74117817299991, 40.666402953999885], [-73.74042437599985, 40.66640252299988], [-73.74021946299993, 40.66640227699988], [-73.74010286899983, 40.666402139999896], [-73.73993110699995, 40.66640155399987], [-73.73856469500002, 40.66639689799991], [-73.73891022900001, 40.66595449199986], [-73.73920854999997, 40.66562918899998], [-73.739371342, 40.66545222799987], [-73.73954958300003, 40.66525602199989], [-73.74281884600006, 40.66530554899985], [-73.74340973899989, 40.6644041699999], [-73.74374970899983, 40.66409039299991], [-73.74405313899989, 40.66365807099993], [-73.74450922599979, 40.663821277999865], [-73.7461369799999, 40.6617167199999], [-73.74684750499985, 40.660795540999864], [-73.74679260099995, 40.65890762199992], [-73.74733230899986, 40.65912452099992], [-73.74779788899987, 40.65852210299988], [-73.74853177199994, 40.65757569699993], [-73.74807112499987, 40.65754849199991], [-73.74705195899993, 40.657488288999915], [-73.74679970800003, 40.657471280999914], [-73.74618048999982, 40.657439074999914], [-73.74506037899988, 40.657407149999884], [-73.74492278899996, 40.65647897499991], [-73.74487420299995, 40.656144557999944], [-73.74377018899996, 40.65613902399991], [-73.74328714099991, 40.65613473199991], [-73.74329846300006, 40.655423526999925], [-73.74330720899991, 40.65470855899989], [-73.74331194299997, 40.6539944199999], [-73.74331805699994, 40.653281203999946], [-73.74332546899993, 40.65253780399992], [-73.74654918000004, 40.65255348099987], [-73.74867194699979, 40.65267808799992], [-73.74920814099985, 40.652933862999895], [-73.74937142800002, 40.65303483399985], [-73.75109427999986, 40.654212440999906], [-73.75149741999985, 40.65433688299986], [-73.752088853, 40.65451770399988], [-73.75294892899993, 40.6548850399999], [-73.75371894599986, 40.65536538099989], [-73.75606608399983, 40.65648646099987], [-73.75680959499985, 40.65709328399992], [-73.75763169899994, 40.65735762999987], [-73.75819056099982, 40.65755494699994], [-73.75845384399994, 40.657647902999955], [-73.75922652199995, 40.65790223699989], [-73.76003600699983, 40.65807344199986], [-73.76165732899996, 40.658238085999855], [-73.76166370199995, 40.65789692799995], [-73.76166371699979, 40.65789649699987], [-73.76166374199983, 40.6578960729999], [-73.76168563999983, 40.657555109999876], [-73.76168565299982, 40.657554974999876], [-73.76168566599995, 40.65755484799992], [-73.76172307699989, 40.657214420999935], [-73.761723113, 40.65721414299991], [-73.76172316099992, 40.65721386399987], [-73.76177602300001, 40.65687469299997], [-73.76177603499984, 40.65687459399989], [-73.76177605899989, 40.65687448599989], [-73.76184429799997, 40.656536949999925], [-73.76196576499984, 40.656415891999856], [-73.76210493899991, 40.656303065999936], [-73.76226027499987, 40.656200664999915], [-73.76242965099998, 40.6561106019999], [-73.76261039799991, 40.65603444099988], [-73.762799522, 40.655973306999876], [-73.76299380499994, 40.65592780799994], [-73.76319000499986, 40.65589805399988], [-73.76404059399991, 40.65439329499984], [-73.76406290099993, 40.65435626599989], [-73.76417770199993, 40.65442150099993], [-73.76475455399994, 40.654746896999875], [-73.76566897099984, 40.65525346899986], [-73.766280083, 40.65557821499993], [-73.76669593900003, 40.655802847999894], [-73.76683051599994, 40.65587605299992], [-73.76729260999991, 40.65612444299994], [-73.76735274299986, 40.65616012599987], [-73.76740737399983, 40.65619021499988], [-73.7677137589999, 40.656358979999915], [-73.76786196699993, 40.65644061399993], [-73.768070047, 40.6565553269999], [-73.7687316869999, 40.65692006399991], [-73.76877799799986, 40.65694558799989], [-73.76976145199995, 40.657482235999915], [-73.7704997779999, 40.657876039999884], [-73.77291920299982, 40.65912940399993], [-73.77305501099993, 40.6591997509999], [-73.77331158599986, 40.65934671699995], [-73.7737837049999, 40.65962415999994], [-73.77382964699984, 40.65965115799987], [-73.77388075599985, 40.6596788949999], [-73.77418923599993, 40.6598605239999], [-73.77434521599992, 40.659932140999906], [-73.7745200739998, 40.660012414999905], [-73.7748492269999, 40.66014868699996], [-73.77552237699994, 40.66036546999993], [-73.7756055619998, 40.66038771199989], [-73.77644444599981, 40.660629973999946], [-73.77670850699995, 40.660706229999946], [-73.776792695, 40.66074400799986], [-73.77849144899989, 40.661506302999875], [-73.77864408999987, 40.661567012999946], [-73.77871548099994, 40.66159539999994], [-73.77876727900004, 40.66161599499987], [-73.77932933099991, 40.66183951999991], [-73.78075528499981, 40.66240660299995], [-73.78128320799988, 40.66258308299995], [-73.78140184699991, 40.66262274199985], [-73.78211253899983, 40.6628354049999], [-73.78321833499997, 40.663166283999864], [-73.784349205, 40.663402598999916], [-73.78481455499993, 40.66349858199991], [-73.78502513799997, 40.66342107199989], [-73.78618470399978, 40.66360955299993], [-73.78751177999993, 40.663825242999906], [-73.78811679799986, 40.6639058809999], [-73.78984347199989, 40.6641359919999], [-73.78990759999994, 40.66414453799993], [-73.7899673679998, 40.664153288999934], [-73.79005979999982, 40.66416690699989], [-73.79057832499984, 40.6642432999999], [-73.79118919099994, 40.66433323299989], [-73.791994646, 40.66443927699986], [-73.79256433099995, 40.66451427599985], [-73.79394317799995, 40.664702387999924], [-73.7952263039999, 40.66487742499987], [-73.79582296400001, 40.66495325599992], [-73.79589506299997, 40.66496081399986], [-73.79642246799989, 40.66501610199989], [-73.79702409499983, 40.66506588399988], [-73.79762708599996, 40.66510255299991], [-73.79785485699999, 40.665114804999895], [-73.79983020599992, 40.665221051999936], [-73.80072889200001, 40.66526937699988], [-73.80180049599981, 40.66525044099989], [-73.80188667499992, 40.665248916999886], [-73.80203207699995, 40.66524802499994], [-73.80183646299999, 40.665881093999914], [-73.80170297899987, 40.66626406999993], [-73.80168266599995, 40.66632235299992], [-73.8016677679998, 40.66637296299996], [-73.80155565300004, 40.6667533639999], [-73.80148383499996, 40.66675015199992], [-73.80139045299981, 40.66674665899994], [-73.80102008199994, 40.66672907999993], [-73.80027044399995, 40.66666752699987], [-73.79968140399984, 40.66656547099993], [-73.79942279899994, 40.666520663999925], [-73.79919289299993, 40.666518074999885], [-73.79896198599992, 40.666529485999874], [-73.79873226699989, 40.666554982999905], [-73.79850596999994, 40.66659438099988], [-73.79828522699991, 40.66664725099992], [-73.798072072, 40.66671288499993], [-73.79739484699991, 40.6669398859999], [-73.79711068999998, 40.66694966899994], [-73.7965798539999, 40.666942991999896], [-73.7960489459999, 40.666942792999855], [-73.79551810099989, 40.6669490719999], [-73.79498745799991, 40.66696182699995], [-73.79445715, 40.666981055999926], [-73.79392731599994, 40.667006752999946], [-73.79319502099987, 40.66705844699985], [-73.79306694299999, 40.667067293999935], [-73.79218446699987, 40.66704188899988], [-73.79129821899986, 40.666970784999904], [-73.79039544099997, 40.66688770399988], [-73.78940999399993, 40.66684115799993], [-73.78697007199989, 40.66708518199989], [-73.78566409999993, 40.66718753999986], [-73.78502452299999, 40.66722991399993], [-73.78330043299998, 40.66731460599988], [-73.78265442899993, 40.66736859699988], [-73.78179791400001, 40.66744044099992], [-73.78062532299994, 40.66752330999992], [-73.77970512300006, 40.667590197999864], [-73.77791285699988, 40.667715066999925], [-73.77701324899986, 40.66777636199988], [-73.77611752599984, 40.6678457269999], [-73.77522378799983, 40.66791507199986], [-73.77431099300001, 40.667981732999884], [-73.77342462799993, 40.66802761899988], [-73.7715446069999, 40.66796326899992], [-73.76675402199996, 40.66743299499994], [-73.76493070899986, 40.6672271529999], [-73.7637568769999, 40.66709461499989], [-73.76270581499982, 40.666989484999895], [-73.761822313, 40.666877759999856]]]}}, {\"id\": \"219\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 220, \"Shape_Leng\": 0.106015943267, \"Shape_Area\": 0.000313035744467, \"zone\": \"Spuyten Duyvil/Kingsbridge\", \"LocationID\": 220, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.89827216899992, 40.8894579429999], [-73.89922499099997, 40.887530802999876], [-73.89936426899996, 40.887249089999926], [-73.89965185599996, 40.88666493399989], [-73.89972588599989, 40.88651488699985], [-73.89892571799993, 40.88638350199994], [-73.89738336099998, 40.88608677199995], [-73.89715645399986, 40.88604482399988], [-73.89688178399989, 40.88599934699987], [-73.89668243100002, 40.88596458999995], [-73.89661546599986, 40.885955255999946], [-73.89711223299999, 40.885569434999866], [-73.89760048500003, 40.88517729299989], [-73.89781492099993, 40.88499918299995], [-73.89808000699986, 40.884778999999874], [-73.89835487800005, 40.8845428649999], [-73.89843428799998, 40.88447464199989], [-73.89855059899999, 40.88437472699996], [-73.89914458899997, 40.88368756599993], [-73.89988688799993, 40.882828808999925], [-73.9015065839999, 40.88095490599994], [-73.90226256899996, 40.88010809899993], [-73.90333605399985, 40.87897514599996], [-73.90369454799992, 40.878501617999895], [-73.903898338, 40.87812110799991], [-73.90417988499996, 40.877336633999974], [-73.90448826999989, 40.8764773599999], [-73.90465612299992, 40.87605891599984], [-73.90483848999986, 40.87564374299989], [-73.90503517999984, 40.87523228299991], [-73.90524600299995, 40.87482494899991], [-73.90526442899994, 40.8747911099999], [-73.90556943300005, 40.87423099199992], [-73.90605459699992, 40.87329392499993], [-73.90608130599988, 40.873239628999904], [-73.90614668899998, 40.87326020499995], [-73.90636316599984, 40.87331904699993], [-73.90662887099995, 40.87338037899989], [-73.90669984700006, 40.87339676199991], [-73.90746489699994, 40.87354735399985], [-73.90692502800006, 40.87503467099994], [-73.90666530999995, 40.8757156089999], [-73.9068187309999, 40.87661958799993], [-73.90703748899998, 40.87689516099993], [-73.90765281699993, 40.87742802299989], [-73.90835569799991, 40.87791831399994], [-73.90906235999998, 40.878486782999886], [-73.90949832099987, 40.87877673199989], [-73.91033193599998, 40.879038046999895], [-73.91148913199984, 40.87901885899989], [-73.91205829999997, 40.878119319999904], [-73.91487449199987, 40.87664214599991], [-73.91571027699979, 40.875795454999924], [-73.91574356, 40.875760999999954], [-73.91578609320084, 40.87571718248032], [-73.91726028318647, 40.87580900379031], [-73.91733338905482, 40.875802059107784], [-73.91740100893917, 40.875786904136554], [-73.91747364879211, 40.87576225007656], [-73.91768907119067, 40.87568068402884], [-73.91775669364831, 40.8756636279301], [-73.91778922586424, 40.87567695509722], [-73.91783922520156, 40.87573780495587], [-73.91791929569662, 40.87577777116814], [-73.91799187213586, 40.87580442894818], [-73.91808452282606, 40.875794993071196], [-73.91824974216237, 40.87581791600413], [-73.9182597238811, 40.875844529771996], [-73.91827974001829, 40.87585594591024], [-73.91829727517938, 40.87584835619441], [-73.91832983816866, 40.87583697616597], [-73.91908326345383, 40.876064881912875], [-73.91978381266422, 40.876421436052915], [-73.91978371651518, 40.87650125521249], [-73.91987703590954, 40.8765088872104], [-73.91992137189636, 40.876545061945706], [-73.91994332109749, 40.87654683020629], [-73.9199689385686, 40.87654889515391], [-73.92002897648118, 40.87659264706155], [-73.92006899571231, 40.87662688332882], [-73.92005645922515, 40.87665692075474], [-73.92005393019218, 40.876662981118514], [-73.9200714360083, 40.876680097883686], [-73.920080284654, 40.87668010394761], [-73.92011149588645, 40.87668012575257], [-73.92015400779998, 40.87672386589551], [-73.92025638949379, 40.87695199013827], [-73.9215771041196, 40.87813654256217], [-73.9216063027348, 40.878157102290686], [-73.92163286288483, 40.87818479384493], [-73.92169203707775, 40.8782509841015], [-73.92182211821769, 40.8783517962847], [-73.9220672367594, 40.87857241490144], [-73.92215230058137, 40.87862948601801], [-73.92221986267197, 40.87866563987871], [-73.92229996812422, 40.87868089778522], [-73.92237506622932, 40.87869615122854], [-73.92243012179054, 40.87872089477373], [-73.92245512078604, 40.87875512007153], [-73.92252221885806, 40.87876471500179], [-73.92253523103413, 40.87876657593437], [-73.92258529790826, 40.878776112092886], [-73.92267879914924, 40.87886681391701], [-73.92273788293744, 40.87890544521391], [-73.92279794475571, 40.87893209178375], [-73.92287303885503, 40.878951146023894], [-73.92295065383705, 40.878954999157195], [-73.92304082211855, 40.87893035395366], [-73.9230931982435, 40.87890555521609], [-73.92312345761908, 40.87892280715317], [-73.9231635068568, 40.878934236537866], [-73.92318603903095, 40.87893615197471], [-73.92324363504969, 40.87893048898557], [-73.92325917211696, 40.87892107114817], [-73.92326869180737, 40.87891530105498], [-73.92330375398006, 40.878907722870565], [-73.92359172576555, 40.87895215916629], [-73.92362973975199, 40.87896394312155], [-73.92374935500642, 40.87898403604847], [-73.92379441334087, 40.8789935690412], [-73.92382697888756, 40.87898028680117], [-73.9239020992123, 40.87897653587015], [-73.92394215243536, 40.87898416315756], [-73.92396967806097, 40.8789993854873], [-73.92398718777542, 40.879014600929025], [-73.92403725610207, 40.87902223524942], [-73.92405728847959, 40.879020348152864], [-73.92406481616946, 40.87900705014288], [-73.92417999610814, 40.87900522556883], [-73.92431268568636, 40.87901861501656], [-73.92449042789555, 40.87904723897836], [-73.92451795875512, 40.87905865928088], [-73.92453549398189, 40.8790510694285], [-73.92453803212285, 40.8790206638224], [-73.92458061723342, 40.87900358704249], [-73.92461565442643, 40.879018813162645], [-73.92465572010455, 40.87901503911895], [-73.92468081924542, 40.87896184311331], [-73.92472342192558, 40.87892956313769], [-73.92477853447214, 40.878904893068594], [-73.92490327486523, 40.87888836792653], [-73.92380525605498, 40.880324901636136], [-73.92182955043396, 40.884511600836916], [-73.92138842752367, 40.88545905236089], [-73.92106413087296, 40.886155561204866], [-73.92058641888602, 40.88718267830985], [-73.920398527, 40.88712930499984], [-73.91869590499982, 40.88688520299989], [-73.91866590399985, 40.8866665349999], [-73.91865219599985, 40.88644622199995], [-73.91865495699987, 40.88622532499995], [-73.91867419999998, 40.8860048999999], [-73.91870982599988, 40.88578600699989], [-73.91876151499994, 40.88556968499989], [-73.91630438600004, 40.884909510999954], [-73.91586527699988, 40.88485810699987], [-73.91489276999991, 40.8846247709999], [-73.91440838499985, 40.8849988979999], [-73.91393522100002, 40.88538139699996], [-73.91386874899996, 40.88543763699991], [-73.91347365499988, 40.885771970999926], [-73.91302404599979, 40.88617032299986], [-73.91281277499982, 40.88639518699987], [-73.91261653600003, 40.88662819299988], [-73.91243612599989, 40.88686844899993], [-73.91227221199996, 40.88711502299987], [-73.91210920699979, 40.88738250499992], [-73.91196427499982, 40.887656339999914], [-73.91183800999981, 40.887935438999904], [-73.91173089099983, 40.88821871299993], [-73.91130760799977, 40.888236029999874], [-73.91115061799995, 40.88827710599987], [-73.91075165299989, 40.88849245499989], [-73.90981767099994, 40.888996568999886], [-73.90975930299997, 40.889250027999886], [-73.90929422499988, 40.88928229699996], [-73.90900536, 40.88950014099986], [-73.90887023199998, 40.889421753999876], [-73.90881651000004, 40.889390590999895], [-73.90877717899996, 40.88937017299986], [-73.90855714199998, 40.88928966699994], [-73.90853308099986, 40.889167388999915], [-73.90848550899992, 40.88914254199987], [-73.90843131799984, 40.889124993999914], [-73.90837360299984, 40.889116015999896], [-73.90831585200002, 40.88911589699994], [-73.9079634, 40.88885975299993], [-73.90759731899983, 40.888200979999965], [-73.90738830199982, 40.88781969199987], [-73.90626252500002, 40.88757033799994], [-73.90533517099998, 40.88735459899987], [-73.90413642399992, 40.887080708999896], [-73.90381882699994, 40.88747900899994], [-73.90358612699993, 40.8878014589999], [-73.903373145, 40.88813234499985], [-73.9031806419999, 40.88847059499993], [-73.90300926099984, 40.888815111999875], [-73.9028594909999, 40.88916474399989], [-73.90273228799997, 40.88913231799993], [-73.90259906499992, 40.889112179999884], [-73.90246297399985, 40.88910514799994], [-73.902327386, 40.889111381999925], [-73.902195564, 40.889130414999926], [-73.90207033599997, 40.88915127199993], [-73.90193848899995, 40.88915604099992], [-73.90180553899997, 40.889143419999925], [-73.90167746699993, 40.88911356799987], [-73.9015597889999, 40.8890681819999], [-73.90145684499987, 40.88901019299989], [-73.90136756999983, 40.888959499999935], [-73.90126187999986, 40.88892096799992], [-73.9011444999999, 40.888898679999926], [-73.90102228799991, 40.888895126999884], [-73.90090301399997, 40.888910548999895], [-73.90079375300003, 40.88894284499989], [-73.9006996359998, 40.88898822099989], [-73.90056049099992, 40.88911965299998], [-73.90040701199986, 40.88924269799986], [-73.90024058900005, 40.88935606899991], [-73.90006286299982, 40.889458615999885], [-73.89987570700002, 40.88954945799986], [-73.89959425899988, 40.88981699899988], [-73.89853911899992, 40.89038907399991], [-73.89842801299993, 40.8904309739999], [-73.898348963, 40.89044499699987], [-73.89828904299992, 40.890442980999914], [-73.89825201999993, 40.890434969999944], [-73.89820200999998, 40.890415934999965], [-73.89801518, 40.89031454599989], [-73.89787887499983, 40.89027610099993], [-73.89827216899992, 40.8894579429999]]]}}, {\"id\": \"220\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 221, \"Shape_Leng\": 0.166218220846, \"Shape_Area\": 0.000890111567051, \"zone\": \"Stapleton\", \"LocationID\": 221, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.07530550099992, 40.637729958999955], [-74.07513243599989, 40.63772719999991], [-74.075060529, 40.637748399999936], [-74.07496303999991, 40.637773349999854], [-74.07493115799998, 40.637781509999826], [-74.07486917499988, 40.6377928099999], [-74.07469451399999, 40.637805180999884], [-74.07373948699988, 40.63786123399992], [-74.07329742299999, 40.63790350799987], [-74.07258133886755, 40.63794187429597], [-74.07257275212604, 40.637871909875905], [-74.07348353801716, 40.63778735793568], [-74.07345636159805, 40.63765394568573], [-74.07353447593354, 40.63764172263229], [-74.07348737303558, 40.6372335292362], [-74.07053959705199, 40.63747140462426], [-74.07049412213834, 40.63716188683958], [-74.07052719831867, 40.63713205870565], [-74.07341562632824, 40.63689395096554], [-74.07337342888026, 40.63661072563942], [-74.07324020722542, 40.63661891979117], [-74.07323857883866, 40.63663279738645], [-74.07251025221059, 40.636692874679404], [-74.07250421469608, 40.63667682858056], [-74.07239586630878, 40.636683774732695], [-74.07239588123399, 40.63669753216838], [-74.07210395164891, 40.63672523098916], [-74.07209483807503, 40.63664727762461], [-74.07241084247497, 40.63661727120748], [-74.07241085491569, 40.63662873618408], [-74.072507170032, 40.63662638135513], [-74.07250414244814, 40.63661033285556], [-74.07282315707188, 40.63658032340693], [-74.07282919715684, 40.63659866245107], [-74.07290142399262, 40.63658944583374], [-74.07290140645824, 40.6365733954361], [-74.07322343404338, 40.63654567542717], [-74.07323116008128, 40.636566425452884], [-74.07331604904051, 40.63655876338426], [-74.07335822045584, 40.63653609416517], [-74.07303948646671, 40.6347982312994], [-74.07283184410423, 40.63366603282566], [-74.0733282131641, 40.63205844299033], [-74.07305751299738, 40.63069065404744], [-74.0729847461233, 40.630308978981404], [-74.07278206817809, 40.63024610461542], [-74.06802173543946, 40.62878758518541], [-74.06813019247986, 40.62853904609149], [-74.07291177936548, 40.62999637940653], [-74.0729282255191, 40.62995669634175], [-74.07251259406816, 40.627768778427004], [-74.07250354307324, 40.6277481482825], [-74.07245835854137, 40.627709196353045], [-74.0724492976149, 40.627679394180596], [-74.07245527865845, 40.627644995424916], [-74.07239900616756, 40.62763110052596], [-74.0722922308001, 40.62715170770656], [-74.07271151257999, 40.62710344565109], [-74.07278382204917, 40.62704855517877], [-74.07278065950129, 40.62692796313009], [-74.07270543377275, 40.626874148562784], [-74.0725678017724, 40.626224897636625], [-74.07225500955778, 40.62460875019394], [-74.07123455549109, 40.62309329756457], [-74.07121598307037, 40.62305780569776], [-74.07114704488251, 40.62292014490076], [-74.07106179396493, 40.62279315450548], [-74.07098941522604, 40.62270880005274], [-74.07091706977349, 40.62265642963042], [-74.07082375839344, 40.622623614986125], [-74.07078879374497, 40.6226124987033], [-74.0707596091144, 40.622603221166784], [-74.07071994036913, 40.62257748182329], [-74.07058681402935, 40.6223812237282], [-74.07056693762618, 40.62232882085632], [-74.07056577813432, 40.622309548718455], [-74.07056805240435, 40.62228084561615], [-74.07055403064878, 40.622252424750364], [-74.07050617650607, 40.6222044800555], [-74.0704956626467, 40.62218494141913], [-74.07044775729958, 40.62208902217424], [-74.07039291413982, 40.62204818912425], [-74.07036956800744, 40.62202155092269], [-74.07033687509666, 40.62197803806393], [-74.07028197822274, 40.62188478924676], [-74.07028312709504, 40.621868796243334], [-74.07024340413841, 40.62179241716038], [-74.07022006282969, 40.62177022212411], [-74.07016754440262, 40.62172139139478], [-74.0700869743588, 40.621607722648356], [-74.07007995707167, 40.621586406538135], [-74.07007993001957, 40.62156064112396], [-74.07006471960437, 40.62151001179222], [-74.07002735061577, 40.62145406385068], [-74.06998300752403, 40.62141766690337], [-74.06995735676463, 40.62141768251555], [-74.0699445353289, 40.621421244273634], [-74.06992059737611, 40.62143309776266], [-74.06990374594828, 40.621439036688244], [-74.0698803921237, 40.62140529098005], [-74.06988154825585, 40.62139551832082], [-74.0698909248967, 40.62139278876934], [-74.06990602432006, 40.62138839552678], [-74.06989782466233, 40.62135108664928], [-74.06987800332105, 40.62135109869925], [-74.06985814982642, 40.62131912900937], [-74.06983970578148, 40.62132320798331], [-74.06977065208979, 40.621268541866506], [-74.06978373653142, 40.62124717450164], [-74.06975543088323, 40.6212081398724], [-74.06946265791278, 40.62108749300955], [-74.06921332498109, 40.62090256474303], [-74.06922460488548, 40.62089218540343], [-74.06875750296253, 40.620556259976595], [-74.06839268883091, 40.62029389372511], [-74.06837042857423, 40.620293994248826], [-74.06720519104827, 40.6194554055429], [-74.06669157924436, 40.61908576424745], [-74.06665653311698, 40.619016488437495], [-74.06649335296379, 40.618897601595165], [-74.06642251522541, 40.618960556505], [-74.06642027508448, 40.618962266761294], [-74.06641466743935, 40.61895799851777], [-74.06596065271778, 40.61935901559374], [-74.06591003230022, 40.61931580040389], [-74.06637761277008, 40.61889481228406], [-74.06640988953299, 40.6188644675671], [-74.06619907954864, 40.618722481288756], [-74.065572808412, 40.619274543599936], [-74.06544586069478, 40.61920262828925], [-74.0660793790844, 40.618620919774415], [-74.06668207982919, 40.61806749593851], [-74.06641558715184, 40.61787249816556], [-74.06604991535914, 40.61819541035272], [-74.06569864744148, 40.617956415586626], [-74.06546549952745, 40.61810707028078], [-74.0651374231156, 40.61789069960471], [-74.06513041282273, 40.61789638699402], [-74.0649618928053, 40.61777239957042], [-74.06435705158657, 40.618245944865606], [-74.06425721751587, 40.61817168758555], [-74.06509908407213, 40.61751095612543], [-74.06443662005032, 40.61706241460249], [-74.06401211702085, 40.6174022266361], [-74.06390892240353, 40.617328825149094], [-74.06432745858045, 40.61698850302212], [-74.06419812869672, 40.61690093459031], [-74.06391060234927, 40.61686761382144], [-74.06386816792602, 40.61685771095525], [-74.06382843924409, 40.61684263912981], [-74.06379249208877, 40.61682280646529], [-74.0637612998448, 40.61679874999733], [-74.06373570714015, 40.61677112113414], [-74.06373027901742, 40.616728369944525], [-74.06373372626567, 40.61668549952], [-74.06374596336498, 40.61664357318375], [-74.06376668678114, 40.61660363084028], [-74.06368609895254, 40.61654167122648], [-74.06355954496281, 40.61644436693834], [-74.06288724049963, 40.61575188002388], [-74.06288005786693, 40.61574448068285], [-74.06183263746597, 40.61478731064751], [-74.06130709550278, 40.61419225076559], [-74.06118151412508, 40.61386088813253], [-74.06103942014855, 40.61359054933771], [-74.06080853165754, 40.613031733806146], [-74.06073779377422, 40.61304667709509], [-74.06064940589565, 40.61280785092203], [-74.0606742614506, 40.61257799811711], [-74.06061238930377, 40.612305676737726], [-74.06050359224615, 40.61210055412032], [-74.06033680792011, 40.6119018802197], [-74.06023648814465, 40.611947735760836], [-74.06009027689035, 40.612014567895784], [-74.06007967319145, 40.61199824137404], [-74.06000457322973, 40.61203319548432], [-74.06000850450059, 40.61203838063162], [-74.05985444328618, 40.61210295860812], [-74.0598670814194, 40.61214348599649], [-74.05988716753706, 40.61220789427623], [-74.05991223842199, 40.612288287746125], [-74.05992120586593, 40.61231704193239], [-74.05995219194659, 40.6124164016141], [-74.0599613278625, 40.612445696995124], [-74.05999192119081, 40.61254379805701], [-74.06000161183916, 40.61257487347523], [-74.06001816918659, 40.612627966152125], [-74.0600417658949, 40.61270363219033], [-74.06004736821689, 40.612721594064716], [-74.06000447610441, 40.61272658503737], [-74.06001625116235, 40.61276368789537], [-74.06019772795608, 40.61278637593195], [-74.060195940971, 40.61280177462218], [-74.05999919711631, 40.612783899618606], [-74.05976930738179, 40.61205664713937], [-74.06003432329639, 40.61194273457411], [-74.06002048435357, 40.61192857421075], [-74.06027224809941, 40.61181963993919], [-74.06007754896766, 40.611830179760084], [-74.05990110428574, 40.611749173078195], [-74.05953306749007, 40.611714255069394], [-74.05950312625981, 40.61170412574045], [-74.05947565119531, 40.61169053015485], [-74.05945132766442, 40.61167380745654], [-74.05943076241836, 40.61165437479534], [-74.05941446845584, 40.61163271692135], [-74.0593389250291, 40.611565131232645], [-74.05927076216308, 40.61149313253705], [-74.05921042182443, 40.61141718769858], [-74.05915829525203, 40.61133778916807], [-74.0591147204204, 40.611255451789184], [-74.0590226904752, 40.61115235902414], [-74.05892163455023, 40.611056055400624], [-74.05890133421524, 40.611002286486254], [-74.05892744754186, 40.6109383775444], [-74.05904563473145, 40.61088963849603], [-74.05912333939817, 40.610822289089086], [-74.05912214071775, 40.610738429046535], [-74.059085843087, 40.610633677254455], [-74.05857881231054, 40.61004864489008], [-74.05848042971112, 40.61001074045373], [-74.05847736437649, 40.60992729480929], [-74.05832711414438, 40.60976822898004], [-74.05776219713343, 40.609326052209965], [-74.05737570322836, 40.60896194823916], [-74.05716383878975, 40.608742783103665], [-74.05713403543749, 40.60863567182052], [-74.05703263589402, 40.60847428919626], [-74.05699887669253, 40.6083747730916], [-74.0569659895738, 40.60826954732188], [-74.05685812390232, 40.608056357772554], [-74.0578901689999, 40.607580753999905], [-74.05778710399987, 40.607449590999906], [-74.05801067099993, 40.607368791999924], [-74.06042867199996, 40.6063527299999], [-74.06046429699992, 40.60640582499984], [-74.0651709129999, 40.604213507999894], [-74.06383152699988, 40.60258896799995], [-74.06350565699996, 40.60215005499991], [-74.06367318899999, 40.60219816599994], [-74.0642173399999, 40.60235301499994], [-74.06472638299991, 40.602497866999904], [-74.06550663199994, 40.602719880999935], [-74.06571018099996, 40.60279382299992], [-74.06898118299992, 40.60398191899988], [-74.06914030999994, 40.604039717999896], [-74.07172956199993, 40.60505809099989], [-74.07206682799995, 40.605186347999954], [-74.07242904799995, 40.60530270299992], [-74.07268510599995, 40.60538461299993], [-74.07298106899998, 40.605469929999835], [-74.07335207399997, 40.60556304399993], [-74.07358326699998, 40.605621067999905], [-74.07399965299999, 40.60572556699987], [-74.074789422, 40.60587021299993], [-74.07543239699991, 40.605987966999884], [-74.07551713899987, 40.60599795099993], [-74.07606892799996, 40.60606295199986], [-74.07656734699997, 40.606121663999915], [-74.07778089599998, 40.60617100899986], [-74.0815547009999, 40.60629029699987], [-74.08148289099995, 40.60638202899987], [-74.0813674559999, 40.6065294999999], [-74.08129735699994, 40.60661901599992], [-74.08069497999999, 40.60738826099991], [-74.08011002799992, 40.60813523899987], [-74.07985980899997, 40.60844643599988], [-74.07962112599992, 40.6087629369999], [-74.07939423999989, 40.60908436499988], [-74.07915097299991, 40.60948761699993], [-74.07841920399994, 40.61070060799994], [-74.07784324799991, 40.611655287999895], [-74.07755680899999, 40.61213759399992], [-74.07721874699989, 40.612707982999936], [-74.0768105549999, 40.61339668199991], [-74.07661374799993, 40.61372872999988], [-74.07620479299995, 40.61441868999987], [-74.0753823519999, 40.61580988199987], [-74.07527690499992, 40.61597807099987], [-74.075025699, 40.61637873999991], [-74.07454676799989, 40.61694292999989], [-74.07437079999995, 40.61712770499991], [-74.07418030999995, 40.61730493399995], [-74.07397615999992, 40.617473641999915], [-74.07375937799999, 40.61763293099988], [-74.0735311079999, 40.61778201399987], [-74.07329261599995, 40.6179201889999], [-74.07301605899991, 40.61804928399993], [-74.0727502249999, 40.6181934099999], [-74.07255048299993, 40.61831840599996], [-74.07249719099994, 40.61835175599991], [-74.072258849, 40.61852327399987], [-74.07203694399993, 40.618706741999915], [-74.0719773749999, 40.6187633899999], [-74.0718329299999, 40.61890075099991], [-74.07164799799989, 40.619103771999896], [-74.0715188819999, 40.6192627579999], [-74.0714060829999, 40.61943137899991], [-74.0713113039999, 40.61960813799994], [-74.07125950599996, 40.61973410799993], [-74.07123598299995, 40.61979131499989], [-74.07118111099999, 40.61997902699989], [-74.07115583099988, 40.62012115699988], [-74.07114727799993, 40.62016924599988], [-74.071134623, 40.62035994799992], [-74.07114283699993, 40.620549106999874], [-74.07116568799995, 40.620698785999906], [-74.07127025999996, 40.6208358429999], [-74.07174504899992, 40.621458933999925], [-74.0720638059999, 40.6218592669999], [-74.07253551699989, 40.622464021999896], [-74.07381368799996, 40.621797074999975], [-74.0744973829999, 40.62144277399996], [-74.07461777599994, 40.62138010099991], [-74.07558011999997, 40.620876468999924], [-74.0759248129999, 40.6206971299999], [-74.07626873999996, 40.62051818499985], [-74.07695604999999, 40.620160559999874], [-74.07769140399995, 40.61977633599981], [-74.07805406299991, 40.61958534199986], [-74.07938896299991, 40.61888579499989], [-74.08000094999996, 40.618552951999945], [-74.08110332499999, 40.61778874499986], [-74.08173992099994, 40.61731079299991], [-74.08274330499992, 40.616581570999905], [-74.08323479999994, 40.61621947699996], [-74.0833131969999, 40.61616342599991], [-74.08379332799989, 40.61580983899993], [-74.08404976899992, 40.61562731399987], [-74.08464853199992, 40.61520502499992], [-74.08476230899998, 40.6151046959999], [-74.08546427599995, 40.61458243099993], [-74.08617531899993, 40.614061606999904], [-74.08628169099997, 40.613975617999884], [-74.0869297369999, 40.61349961799993], [-74.08720982799996, 40.61330697299989], [-74.08742825199987, 40.61441850099989], [-74.08747218499994, 40.614560802999925], [-74.08753556499991, 40.614699614999886], [-74.087617514, 40.614832668999924], [-74.08771657499993, 40.61495785499991], [-74.08777786599995, 40.61501732399994], [-74.087778008, 40.61501746799991], [-74.08777810299993, 40.61501762099994], [-74.08782330099994, 40.61508769299994], [-74.08782337199992, 40.61508779999986], [-74.0878234189999, 40.615087934999885], [-74.08784838399993, 40.61516456799992], [-74.08784843099991, 40.61516472999992], [-74.08784843099988, 40.615164874999884], [-74.08785147599997, 40.61524292099995], [-74.08785147699994, 40.615243072999924], [-74.08785144099994, 40.61524323599987], [-74.08783375499992, 40.61531764999988], [-74.08769173799999, 40.61553515299991], [-74.087413392, 40.615929324999875], [-74.08698538899989, 40.61653298199997], [-74.0865715779999, 40.61716938599994], [-74.08626756599996, 40.61783201099995], [-74.08608006299993, 40.618515303999864], [-74.08604124499995, 40.61882017799994], [-74.08600629799997, 40.61909415099995], [-74.08589883599991, 40.619909428999925], [-74.0858126919999, 40.6203755489999], [-74.08560513399996, 40.62114273099995], [-74.0855941459999, 40.62119087199993], [-74.08548866899994, 40.62165317899995], [-74.08537203099993, 40.62220532899992], [-74.0852499159999, 40.6228008709999], [-74.08496211799991, 40.62462523299994], [-74.08415419899998, 40.626267114999905], [-74.08404820999988, 40.626441020999906], [-74.08392362399994, 40.62660935499987], [-74.08378128799998, 40.626770399999906], [-74.08362236699995, 40.62692257799987], [-74.08344837699995, 40.62706443699991], [-74.08326107499997, 40.627194742999954], [-74.08306246799998, 40.62731250399988], [-74.08298808899993, 40.62735007299989], [-74.08280689099995, 40.62740014699991], [-74.08261621699997, 40.62743470099987], [-74.08241960099991, 40.62745240299991], [-74.082220943, 40.627452544999855], [-74.08202428299997, 40.62743512499987], [-74.08183352399993, 40.627400843999965], [-74.0816522029999, 40.62735102899992], [-74.08099625499995, 40.627339813999896], [-74.08084371499999, 40.62737949099987], [-74.08076183699995, 40.62745285899991], [-74.08069514799999, 40.62753689699988], [-74.08064690599996, 40.62762867599987], [-74.08061902199996, 40.62772455599993], [-74.08061179799988, 40.6278207009999], [-74.08059556699999, 40.62791570899992], [-74.08056350699997, 40.62800951999989], [-74.0805160199999, 40.628099883999866], [-74.08045421199994, 40.628184620999946], [-74.08037988899997, 40.62826190099987], [-74.07972492199995, 40.62895068799986], [-74.08075821999996, 40.629494042999944], [-74.08173135299991, 40.63001222599993], [-74.08140615399998, 40.63030123599993], [-74.08201443099996, 40.63089152599993], [-74.08260312199998, 40.63146480799991], [-74.08344172599995, 40.632275245999914], [-74.08491321699994, 40.63363992399991], [-74.08520082199986, 40.63376239799988], [-74.08574928999992, 40.633955556999865], [-74.08443647599991, 40.63540524799989], [-74.08394671399999, 40.63565410499991], [-74.08373397299994, 40.63573714399991], [-74.08302091199991, 40.63602507099989], [-74.08177940499988, 40.63662301399993], [-74.08055168499997, 40.63731999999989], [-74.07878831099995, 40.63826855899991], [-74.078722026, 40.63828966999992], [-74.07865269699992, 40.63830402499987], [-74.07858150399997, 40.638311379999884], [-74.07850965999987, 40.63831160899986], [-74.07843838799988, 40.63830470799997], [-74.0783689039999, 40.638290794999854], [-74.07830238999998, 40.638270105999865], [-74.07823997999992, 40.63824299499991], [-74.07818273599996, 40.63820992299987], [-74.07813163399993, 40.638171453999874], [-74.0778990219999, 40.63809465199991], [-74.07698362099997, 40.63779569399988], [-74.0768399779999, 40.6377502599999], [-74.07601776799994, 40.637748104999915], [-74.07530550099992, 40.637729958999955]]]}}, {\"id\": \"221\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 222, \"Shape_Leng\": 0.0483893527976, \"Shape_Area\": 0.000130268343918, \"zone\": \"Starrett City\", \"LocationID\": 222, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.87908517399995, 40.65020211699988], [-73.87843582199996, 40.649327343999914], [-73.87798475599999, 40.64872529299991], [-73.87679877399998, 40.64714224999992], [-73.87610398000004, 40.6462147889999], [-73.87468212999995, 40.6456544679999], [-73.8745808109999, 40.64561010599992], [-73.874953578, 40.64514158399989], [-73.87530646399993, 40.64465566899992], [-73.87545861899982, 40.64442693099989], [-73.87596309099979, 40.643644483999935], [-73.87616087400002, 40.64332052799993], [-73.87617767099991, 40.643289658999905], [-73.87619932999985, 40.6432571729999], [-73.87654859999992, 40.64263892799989], [-73.87683349, 40.64211858999989], [-73.87709627099984, 40.641602954999875], [-73.87734248299998, 40.64108268099988], [-73.87752894599991, 40.64071679999995], [-73.87797341300002, 40.63980711799987], [-73.87841589406925, 40.638912189353306], [-73.87846866500722, 40.63896098961851], [-73.87851742532119, 40.63900343074181], [-73.87855952375331, 40.63904982577646], [-73.87859441254382, 40.63909957107356], [-73.878621637735, 40.639152019392206], [-73.87863962123944, 40.63920211229744], [-73.87865060839593, 40.639253365770934], [-73.87865447798258, 40.6393052144072], [-73.87864530642942, 40.63932206053328], [-73.87863986540178, 40.63933982074799], [-73.87863830068277, 40.639358019206654], [-73.87864065419832, 40.63937616832225], [-73.87864686289501, 40.63939378182933], [-73.87865676042716, 40.639410387812596], [-73.87867008161409, 40.63942554135078], [-73.87868646954585, 40.639438836437584], [-73.87870548514465, 40.6394499168594], [-73.87872661892847, 40.63945848573997], [-73.8787493046632, 40.63946431349415], [-73.87876869595337, 40.63946722348675], [-73.87878724498002, 40.63947242076227], [-73.87880448993477, 40.639479775925864], [-73.87882000147552, 40.6394891058588], [-73.87883339341573, 40.63950017827677], [-73.87884433233985, 40.63951271751358], [-73.87887394077451, 40.63956120544015], [-73.8789097272729, 40.639607212344835], [-73.87895133765872, 40.63965028288506], [-73.87899836011283, 40.639689990779296], [-73.87905032924884, 40.63972594302653], [-73.87910673071889, 40.63975778379561], [-73.87913214510957, 40.63982476352717], [-73.87916544009131, 40.639889708185976], [-73.87920634048739, 40.639952080973], [-73.87925450825593, 40.6400113663454], [-73.87930954528377, 40.64006707427804], [-73.87937099667586, 40.64011874431427], [-73.87943835451547, 40.64016594937151], [-73.879511062062, 40.64020829927226], [-73.87958851835315, 40.640245443968716], [-73.87967008317165, 40.64027707643711], [-73.87975508233784, 40.64030293521476], [-73.87979770383454, 40.64031278254277], [-73.87986381281485, 40.64033847696814], [-73.8798940439766, 40.64036327915714], [-73.87989669687771, 40.64037038815583], [-73.879900739591, 40.640381222766266], [-73.87989951993711, 40.64043667228295], [-73.87989951761271, 40.640436741972486], [-73.87988251824561, 40.640453177032086], [-73.87987480747897, 40.64046063271112], [-73.87987403655931, 40.640478638213544], [-73.87987363622284, 40.64048796394434], [-73.87987558551809, 40.64049169181637], [-73.87988928035196, 40.640517877102326], [-73.87990718901253, 40.640542804794535], [-73.87992059896217, 40.640561471263666], [-73.87994069623016, 40.64058450179664], [-73.87995416764035, 40.64059994323321], [-73.87997993971362, 40.640610221376214], [-73.88004942970176, 40.64062823049794], [-73.880058057092, 40.64063241045604], [-73.88009536277097, 40.64065048649352], [-73.88010244648875, 40.64066312653997], [-73.8801087752126, 40.64067441702609], [-73.88011878577429, 40.640719697706515], [-73.88012484055837, 40.64073094451626], [-73.88012648911366, 40.64073400664697], [-73.88012660813733, 40.64073422690356], [-73.88016325038498, 40.64074103257933], [-73.88016359994798, 40.64074109827981], [-73.88016598988065, 40.64075138116187], [-73.88016916221731, 40.640765021126654], [-73.88016574828788, 40.64079320442051], [-73.8801694841882, 40.64081768072128], [-73.88017018150138, 40.640822250957456], [-73.88017124285744, 40.64085556394908], [-73.88016780438096, 40.6408974140312], [-73.88017455805037, 40.64090208221642], [-73.88019132263119, 40.640913667867615], [-73.88019161647189, 40.640913779077614], [-73.88019356367413, 40.6409145249333], [-73.88020590080936, 40.64091282899447], [-73.88020783247619, 40.64091304058498], [-73.88021374862782, 40.64091369123027], [-73.88022196606981, 40.640920596941406], [-73.88023614067771, 40.64093250589891], [-73.8802333890754, 40.640937101692565], [-73.88023051725693, 40.64094189604992], [-73.88025625666039, 40.64097010903597], [-73.88025849683294, 40.640971820327614], [-73.88028649284936, 40.640992349780475], [-73.88033583771679, 40.64098812966351], [-73.88040314772041, 40.640971116630475], [-73.88046932031959, 40.64096179009468], [-73.88048386922625, 40.640978034547516], [-73.88048718054162, 40.64100793312853], [-73.88047923238257, 40.6410625906149], [-73.88047801022172, 40.64111896455321], [-73.88049589429755, 40.64115058723803], [-73.88052495505906, 40.641202721166366], [-73.88055289293244, 40.64125570814897], [-73.88057522492257, 40.64130868930268], [-73.8805986798831, 40.64136081738962], [-73.88062774081736, 40.64141295078826], [-73.88067700349092, 40.641454856080394], [-73.88068925431797, 40.641501847487504], [-73.88072505409015, 40.64154800859012], [-73.8807362354633, 40.64156595847575], [-73.88072836247473, 40.64157876188983], [-73.88071493065692, 40.64156593637824], [-73.8806802514142, 40.64151977475998], [-73.88062758953829, 40.64149751268648], [-73.88060969919279, 40.641469307264714], [-73.88058841799418, 40.64145561885774], [-73.88054246213981, 40.64144532038986], [-73.88052673818183, 40.64145897036521], [-73.88046058804811, 40.64145548604774], [-73.88045727891345, 40.64142473273896], [-73.88040795144309, 40.641418702188226], [-73.88039003605539, 40.64140416251383], [-73.88035864367166, 40.64140156734048], [-73.88033286817526, 40.64139385347232], [-73.88029248068578, 40.641404915733155], [-73.88024419559154, 40.64144244764598], [-73.88019060023001, 40.641479769788695], [-73.880190305821, 40.64147997369924], [-73.88018806234929, 40.641480826092675], [-73.88013640918928, 40.64152091814049], [-73.88009708225974, 40.641565293823795], [-73.8800711977313, 40.641618225292355], [-73.8800486771754, 40.641670304527764], [-73.88002616426255, 40.64171811329596], [-73.88002504480959, 40.64172900952614], [-73.88002502222493, 40.64172921640041], [-73.88001933705613, 40.64177362731715], [-73.88001926794281, 40.64181206416112], [-73.88001923722489, 40.64182914768664], [-73.8800247427096, 40.641884673904066], [-73.88003922041295, 40.64194020881062], [-73.88004145754233, 40.641943627889184], [-73.88005931552975, 40.641989772154886], [-73.88006083136442, 40.6419924180067], [-73.88006714132425, 40.64200344662675], [-73.88011977666517, 40.64204022946572], [-73.88016682754629, 40.64206504911018], [-73.88018923070796, 40.64207874043836], [-73.88018952442866, 40.642078921340236], [-73.88022395348766, 40.64210013046969], [-73.8802534515156, 40.642111173847084], [-73.880278860949, 40.6421206862727], [-73.88029385334004, 40.64212006895721], [-73.88031923202074, 40.642119020509234], [-73.88033103498226, 40.64210945618992], [-73.88033607759517, 40.64210537173018], [-73.88036531935678, 40.642056715675814], [-73.88036553332915, 40.64205672628523], [-73.88039559128313, 40.64205845495089], [-73.88043420266764, 40.64206795309516], [-73.88045835809254, 40.642073894311324], [-73.88048425397946, 40.642074959904804], [-73.88050096563104, 40.64207564638105], [-73.88052902991448, 40.642058593659], [-73.88053544673278, 40.64205948856168], [-73.88054136113625, 40.64206031425544], [-73.88055968307269, 40.642065320612], [-73.88057274225638, 40.64206888812589], [-73.88060408824188, 40.642097962253196], [-73.88063995091653, 40.64210910478272], [-73.88066592641542, 40.642107967417495], [-73.88067807913613, 40.6421074355601], [-73.8806949085863, 40.642123672968566], [-73.88069934834355, 40.642127957259106], [-73.88073289819107, 40.642177533269084], [-73.8807332072026, 40.64217765052438], [-73.880779962267, 40.64219551868346], [-73.88085171526227, 40.64220328177216], [-73.88087946326571, 40.64220396052929], [-73.88092459816858, 40.64220506504925], [-73.8809666898604, 40.642212850623096], [-73.8809667366908, 40.64221285888051], [-73.88098961395188, 40.64221709087568], [-73.88101033957028, 40.642228022284755], [-73.88103666594084, 40.64224191016461], [-73.88106111804933, 40.642256924739684], [-73.88109379117451, 40.64227698941453], [-73.88113277724881, 40.642291469005244], [-73.88113301353455, 40.64229154715029], [-73.88120138217396, 40.642309559437194], [-73.88125228305606, 40.64231870160611], [-73.88127312402558, 40.64232244560927], [-73.88132805521832, 40.642330189356606], [-73.88134663984796, 40.642335092660986], [-73.88134669678371, 40.64233510746221], [-73.88139306230616, 40.64234733997912], [-73.8814255353955, 40.642372998298896], [-73.8814379975936, 40.64238410505611], [-73.88147255731225, 40.642414899928674], [-73.88148782994817, 40.64242493236856], [-73.88148794066353, 40.64242500552515], [-73.88152855846559, 40.64245168609164], [-73.88156734043696, 40.642491186215246], [-73.88157210190602, 40.64249603788729], [-73.88157221057685, 40.642496147730476], [-73.88157720208616, 40.642508563217895], [-73.88159007500376, 40.64254058229961], [-73.88159450203496, 40.6425738992391], [-73.88159559157776, 40.64257761513889], [-73.88160453293122, 40.64260807667848], [-73.88161008072028, 40.64264053953776], [-73.88161787940886, 40.642669590147044], [-73.88162692441355, 40.64271478140323], [-73.88162899453944, 40.64272512205548], [-73.88163899855532, 40.64273914008403], [-73.88164912697144, 40.642753330123995], [-73.88168381336682, 40.64279607302993], [-73.88168742214015, 40.64280356510099], [-73.88168745106204, 40.64280362661371], [-73.88169945562682, 40.64282854744211], [-73.88171847191641, 40.64285504589389], [-73.8817512279545, 40.64290269068874], [-73.88175314434555, 40.64290547648099], [-73.88177010894674, 40.64291483648762], [-73.88179347128593, 40.64292772703098], [-73.8818032883055, 40.642938708062786], [-73.8818248082783, 40.64296277940351], [-73.88186733712442, 40.64300894705619], [-73.88188874177447, 40.64303296398694], [-73.88190314844012, 40.64304913020881], [-73.88193782457263, 40.6430978522867], [-73.88198484754011, 40.64313975470909], [-73.8820093512287, 40.643167074606794], [-73.8820262545109, 40.6431859223232], [-73.88207103387181, 40.643228675966135], [-73.88208344797692, 40.64324026625764], [-73.8821012602465, 40.64325689430421], [-73.88213256756582, 40.64330817655731], [-73.88214855443789, 40.643338739088804], [-73.88215939005505, 40.64335945303325], [-73.88219967565512, 40.64340647299237], [-73.88224108165646, 40.643453492409], [-73.88224776614891, 40.64347827152954], [-73.88224620098688, 40.64348755670327], [-73.88223870162093, 40.643532074420534], [-73.88220499245966, 40.64357047630137], [-73.88219157421167, 40.6435879439901], [-73.88217352293987, 40.64361144449094], [-73.88214875914151, 40.64366352232941], [-73.88213522252431, 40.64370963291185], [-73.88213525105553, 40.64370979159052], [-73.88213525034402, 40.64370982073981], [-73.88214832562561, 40.64374372784058], [-73.88270189929709, 40.644045875131006], [-73.88302004751286, 40.644179192376996], [-73.88315333117271, 40.64431232105569], [-73.88324554462208, 40.64443758464191], [-73.88341999203011, 40.64452381564944], [-73.88360497047857, 40.6444614180559], [-73.88369759560324, 40.64435198816349], [-73.88387223398924, 40.64432869507526], [-73.88399541004412, 40.64436793646128], [-73.88412906283656, 40.64428983874168], [-73.88432452601197, 40.64410228112628], [-73.88455063703385, 40.644008630559426], [-73.8848075041114, 40.64394630368021], [-73.88557762124442, 40.644040950146135], [-73.88574170709806, 40.64418192884243], [-73.8860058291109, 40.64450289080446], [-73.88600571220951, 40.64457125810136], [-73.88609107121216, 40.64464296511781], [-73.88624448913602, 40.6447458241452], [-73.88673134992423, 40.64513193516143], [-73.88679595217259, 40.64523947390825], [-73.88720113482746, 40.64560217210968], [-73.8872203376547, 40.64568157961348], [-73.88722035338283, 40.645681643289706], [-73.88727419528715, 40.645706222414105], [-73.88727896068909, 40.64575951052067], [-73.88732297933879, 40.64579312569286], [-73.887329971946, 40.64579846564516], [-73.88733382868713, 40.64581028135345], [-73.88735029808211, 40.64586073461325], [-73.88743444708422, 40.64593668177378], [-73.88740120112703, 40.64596388299047], [-73.88736029915225, 40.645987185682834], [-73.88738823781253, 40.64607864136912], [-73.88744693360184, 40.6461039870834], [-73.88746222111706, 40.64612539928223], [-73.88746471887154, 40.64615847201257], [-73.88753363787424, 40.646179936669576], [-73.88763668640426, 40.646325451265], [-73.8876658922947, 40.646504924553824], [-73.88763764907316, 40.646594379774484], [-73.88767074460125, 40.6466566599825], [-73.88765531016614, 40.64672278329653], [-73.88771097487337, 40.647032135551854], [-73.8877261123322, 40.64714302919987], [-73.88764430451701, 40.64719158147298], [-73.88756755982104, 40.64726737122376], [-73.88753168688038, 40.6473373655493], [-73.88751623247252, 40.647415160694216], [-73.88757232935409, 40.647467737555665], [-73.88761566189314, 40.647518357581646], [-73.88764108201272, 40.647588411819], [-73.88764613154626, 40.64762343086531], [-73.88762302100217, 40.64769732819684], [-73.88764072166725, 40.64780239000726], [-73.88771211156093, 40.647874434636535], [-73.8877708221422, 40.647891998665145], [-73.8878415352001, 40.647878513928646], [-73.88796131043709, 40.6478804575728], [-73.88796494058958, 40.647880516561145], [-73.88796538193867, 40.64788069976455], [-73.88802108784839, 40.64790391398526], [-73.88806961284439, 40.647903961222525], [-73.88812293345936, 40.64788310555651], [-73.88815394851169, 40.64787097466722], [-73.88822796911403, 40.647906952830496], [-73.88843465427944, 40.64800741559469], [-73.88863605358848, 40.648024462140285], [-73.88863638434091, 40.648024492615654], [-73.8886520498472, 40.64804500806965], [-73.88867219244943, 40.6480706523825], [-73.88868415993227, 40.64808763107775], [-73.88870351913667, 40.64811509934623], [-73.88871203763156, 40.648118877816714], [-73.88873825775943, 40.64813050704218], [-73.88874920382712, 40.64814933090574], [-73.88875167600808, 40.64815358282301], [-73.88876060719959, 40.64817750731708], [-73.88877842029953, 40.64819898761642], [-73.88877846134474, 40.64819903875239], [-73.88878187140558, 40.648203152192416], [-73.88878573742798, 40.64822995813295], [-73.88878963323985, 40.64825697217248], [-73.88879034256342, 40.64827558694118], [-73.88879178231977, 40.648313348904225], [-73.88880096709327, 40.64832516585833], [-73.8888130437334, 40.648340702560574], [-73.8888802948611, 40.6483612668378], [-73.88890716492976, 40.64838777284079], [-73.88891921009564, 40.64841496981422], [-73.88893058013272, 40.64844064782438], [-73.88893062680756, 40.6484407527422], [-73.88894519646605, 40.64844674606871], [-73.88896496571094, 40.648477878366826], [-73.88905476803649, 40.64856079392838], [-73.88911601123027, 40.64859197658822], [-73.88914998188316, 40.648641289828696], [-73.88922136766993, 40.648718256497844], [-73.88924677936008, 40.64874565360671], [-73.88925359963048, 40.64875134190052], [-73.8892625978012, 40.64875799407157], [-73.8892721038874, 40.64876422120865], [-73.88929214744313, 40.64877440663103], [-73.88931263132741, 40.64878407430837], [-73.88933353488541, 40.648793211488154], [-73.88935482843853, 40.648801808927836], [-73.88937648825117, 40.64880985487727], [-73.88939848882367, 40.64881734026525], [-73.88942080069698, 40.648824256016596], [-73.88943829765365, 40.64883061563145], [-73.88945541865267, 40.648837544478845], [-73.88947213182668, 40.64884502745089], [-73.88948840794062, 40.648853053462545], [-73.88950421689121, 40.64886160439219], [-73.88951952680837, 40.64887066647198], [-73.88953430780744, 40.64888022258564], [-73.88954853660488, 40.648890253612585], [-73.88956218375762, 40.64890074143185], [-73.8895752240009, 40.64891166775867], [-73.88958763207877, 40.64892300961778], [-73.88960918705214, 40.64894367818532], [-73.8896301004701, 40.648964726590144], [-73.88965036399765, 40.64898614058453], [-73.8896699637977, 40.64900790792629], [-73.88968889064772, 40.649030018889874], [-73.88970713291512, 40.64905245855457], [-73.88971934222157, 40.64906166087087], [-73.88973210500626, 40.64907041909603], [-73.88974538896763, 40.64907871393338], [-73.88975604953663, 40.64908475886087], [-73.88977341012759, 40.649093838787515], [-73.88978808315802, 40.64910063356155], [-73.8898031529832, 40.64910689731392], [-73.88981491121137, 40.64911515449498], [-73.88984125931684, 40.64913741577972], [-73.88984660009802, 40.64914254542143], [-73.88985587496023, 40.64915246846757], [-73.8898644592518, 40.64916274383416], [-73.88987233607851, 40.649173345705364], [-73.88987947865839, 40.64918424222571], [-73.88988587251853, 40.649195406911794], [-73.88989149505257, 40.6492068097545], [-73.88990327549786, 40.649218681875595], [-73.88991571316235, 40.649230157079586], [-73.88992878718162, 40.649241216248306], [-73.8899424709771, 40.64925183791206], [-73.88995674148609, 40.649262002448], [-73.88997157058684, 40.64927169022826], [-73.88998994466202, 40.64928298832984], [-73.89000877257922, 40.64929384224216], [-73.89002803675979, 40.649304241393914], [-73.89004771984301, 40.649314176051625], [-73.89006780073055, 40.649323635640464], [-73.89008826139944, 40.6493326121019], [-73.89010907921102, 40.64934109519473], [-73.89013023724264, 40.64934907635859], [-73.8901517102082, 40.64935655004082], [-73.89017347964773, 40.64936350633941], [-73.8901955237955, 40.649369939202415], [-73.89021782110412, 40.649375843248144], [-73.89022928523697, 40.64937628339836], [-73.89024076270594, 40.649376104375314], [-73.89025219412413, 40.64937530561958], [-73.89026352163408, 40.649373892939494], [-73.89027469001685, 40.649371872481076], [-73.89028564096586, 40.64936925558052], [-73.89029632189224, 40.649366054417385], [-73.89030667800193, 40.64936228418456], [-73.89031482009374, 40.649358760451435], [-73.89031665867228, 40.64935796510478], [-73.89032621217892, 40.64935311874011], [-73.89033529141233, 40.64934776900195], [-73.89013701016981, 40.64917041211644], [-73.8903473469377, 40.64903360577806], [-73.89053526315183, 40.649207682284306], [-73.89056792961276, 40.64920863296715], [-73.89121853499988, 40.64998769899987], [-73.89125147199995, 40.65002559699993], [-73.89004263999982, 40.65111858599992], [-73.89000589199988, 40.651148696999925], [-73.88959044299983, 40.651337545999866], [-73.88876540999999, 40.65169059099987], [-73.88794902799988, 40.652040973999924], [-73.88713099000002, 40.65239391599988], [-73.88637373700001, 40.65273440399987], [-73.88628975299989, 40.65276890099989], [-73.88621116599981, 40.65280058399992], [-73.88551222099996, 40.653092531999924], [-73.88465215899991, 40.653451653999916], [-73.88278243099995, 40.65426976799992], [-73.88221364899994, 40.65451863799996], [-73.88220052999989, 40.65444157599989], [-73.88172188099986, 40.653753826999846], [-73.88141998000002, 40.653347175999926], [-73.87992598699992, 40.651334759999905], [-73.87908517399995, 40.65020211699988]]], [[[-73.8882853135604, 40.64672241327608], [-73.88829183121516, 40.64671871903564], [-73.88846787911639, 40.64689973402523], [-73.88845814272497, 40.6469018521931], [-73.88845010497215, 40.6469049031624], [-73.88844607613953, 40.64691169919842], [-73.88844491158295, 40.64691844598866], [-73.88844964285599, 40.646932211176726], [-73.88844980935991, 40.64693269608774], [-73.88845315577242, 40.64694294898922], [-73.88845557416435, 40.646947264755944], [-73.8884598643862, 40.64695491533827], [-73.88848084323095, 40.64695711411304], [-73.8884845320914, 40.646957500218946], [-73.88853549296564, 40.646968149350386], [-73.88864137620679, 40.647079553158], [-73.88861004869375, 40.647081360931864], [-73.88858655441034, 40.647093668267225], [-73.88858247895186, 40.647110311452785], [-73.88858810217314, 40.64713621177189], [-73.8886058808228, 40.64715349145925], [-73.88862610172087, 40.64716275995369], [-73.88864283734092, 40.647185347601585], [-73.88863572588177, 40.64721640785971], [-73.8886062843818, 40.647245471330315], [-73.88859389868013, 40.64727620800545], [-73.88857810718577, 40.6473308595822], [-73.8885500525067, 40.64734193668585], [-73.88851637568528, 40.64736069152191], [-73.88846286525167, 40.647363190090786], [-73.88846254740318, 40.6473632051951], [-73.88844461141393, 40.647358063271305], [-73.88839982882779, 40.64731445673339], [-73.88839030942329, 40.64729295912507], [-73.88837636776667, 40.64726147721963], [-73.88835250194765, 40.64724890600538], [-73.88834387533157, 40.64724436213285], [-73.88834372593608, 40.647244130295974], [-73.88832486176042, 40.64721444688094], [-73.88831285460469, 40.647185051737445], [-73.88830810804127, 40.647173431208074], [-73.88830681766217, 40.64716972506336], [-73.88830364435202, 40.6471606153028], [-73.88828631655436, 40.64715299318355], [-73.88826666475885, 40.64714435056601], [-73.88820050726648, 40.647140869529046], [-73.88820047292168, 40.64714062691454], [-73.88819787718664, 40.64712194664503], [-73.8881971830517, 40.64711694993688], [-73.88819109577638, 40.64710017827498], [-73.88818605874528, 40.6470862946514], [-73.88818602021149, 40.647086189238536], [-73.88818580754698, 40.64708105679788], [-73.88818386386546, 40.64703408345029], [-73.88818956342321, 40.646979422784575], [-73.88819745070396, 40.646957221708945], [-73.88819758375213, 40.64695683920327], [-73.88820871943206, 40.64692477503195], [-73.88820645247958, 40.64690506964013], [-73.88820429805554, 40.64688633384463], [-73.88819127336532, 40.64686518361578], [-73.88818640383646, 40.64685727388419], [-73.888176393264, 40.64680857770317], [-73.88818990838848, 40.64677356952388], [-73.88822696216229, 40.646745419383286], [-73.88826623150278, 40.64673349902658], [-73.8882853135604, 40.64672241327608]]]]}}, {\"id\": \"222\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 223, \"Shape_Leng\": 0.166021925275, \"Shape_Area\": 0.00076436070058, \"zone\": \"Steinway\", \"LocationID\": 223, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90649492923363, 40.7901809903447], [-73.9025117370087, 40.78947208502822], [-73.90239936084073, 40.789551620685835], [-73.90206327886825, 40.78949440549526], [-73.90203428533837, 40.78941468642665], [-73.9016968608998, 40.78932032382403], [-73.90077375891737, 40.78914381825163], [-73.90032629743816, 40.789007662825696], [-73.90010759371266, 40.788850490099584], [-73.89961637941789, 40.78840049623635], [-73.89924748492585, 40.78814866012842], [-73.8978318016958, 40.78718217039414], [-73.89725536651031, 40.78675095694394], [-73.89702410373323, 40.786658156960044], [-73.8969399356256, 40.78662466896973], [-73.89651204913619, 40.78606865902596], [-73.89637905280317, 40.78613108158356], [-73.89640086049118, 40.78615791177362], [-73.89635251322325, 40.786179310203266], [-73.89630366096506, 40.78611920156253], [-73.89635026246756, 40.78609731961272], [-73.89636888344312, 40.78612023020103], [-73.89650254833985, 40.786057467843996], [-73.89637855982241, 40.7859089810944], [-73.89626785620453, 40.7859605719464], [-73.89628652699834, 40.78598371863644], [-73.89621803522914, 40.78601602287123], [-73.89604376668126, 40.78579998748044], [-73.89611381864462, 40.78576734216538], [-73.89613322780025, 40.78579140356489], [-73.89624259793256, 40.78574043299193], [-73.89614015089828, 40.78560716728828], [-73.89600516772018, 40.78566665023862], [-73.8960274447233, 40.785689286425686], [-73.89598001322786, 40.785711675768866], [-73.8959296297551, 40.785648944313756], [-73.89597673220975, 40.78562708775979], [-73.89599975377486, 40.78565575201834], [-73.89612632632497, 40.78559316931146], [-73.8958291437706, 40.78522152738345], [-73.89683173824393, 40.78442766370343], [-73.89677700751368, 40.784388420963595], [-73.8967585725261, 40.78440327601169], [-73.89672269281452, 40.784377549325804], [-73.8967344194984, 40.78436809934181], [-73.89672513646195, 40.78436144210911], [-73.89673005421017, 40.78435747867779], [-73.89672052173825, 40.78435064397937], [-73.89675564813932, 40.784322339375755], [-73.89677182707408, 40.7843339415984], [-73.89678844985487, 40.784320548426656], [-73.89682807945508, 40.78434896505369], [-73.89680660094372, 40.78436627012369], [-73.89685938880385, 40.78440412048266], [-73.89708339167628, 40.784251259517305], [-73.89697315561344, 40.784160497966965], [-73.89874897638398, 40.78278115576602], [-73.90114577552107, 40.78271110609054], [-73.90172373164513, 40.78212925463786], [-73.90174935055292, 40.78211723810659], [-73.90179093113794, 40.78210191951054], [-73.9018523162202, 40.78208099879839], [-73.90186925588473, 40.782074033780646], [-73.90188611408085, 40.78206581777869], [-73.90191949285618, 40.782045621351635], [-73.9019509578695, 40.782021289094516], [-73.9019834514793, 40.78199048351641], [-73.90205256434422, 40.78191807899362], [-73.90211707957926, 40.78184770612496], [-73.90217132811875, 40.781789696591616], [-73.90217815315992, 40.78178056186346], [-73.90218359459601, 40.78176976980418], [-73.90218781908894, 40.7817571387903], [-73.90230443036802, 40.78151989321846], [-73.90251248041804, 40.78110825136056], [-73.90258049398903, 40.780814898123595], [-73.90267723582103, 40.780594131742326], [-73.90267677081081, 40.78035257204668], [-73.90257929672345, 40.78028912083556], [-73.90234303882673, 40.78018724480452], [-73.90225480922946, 40.780274167814156], [-73.90218303706564, 40.78035936715213], [-73.9021496741192, 40.78078049395423], [-73.90203565790105, 40.78120310196288], [-73.90129127618921, 40.781858419270336], [-73.90121754094692, 40.78181911242194], [-73.90072847022454, 40.781979495284105], [-73.9003457266802, 40.78206605451128], [-73.90034960780264, 40.78216441052998], [-73.8994794775269, 40.78216988593734], [-73.89865437844033, 40.78199542032074], [-73.8971860476599, 40.78280720996354], [-73.89711476973946, 40.78272319367442], [-73.89705486039331, 40.782756465666786], [-73.89705045284431, 40.782765649347326], [-73.89704408072033, 40.7827741407607], [-73.89703592466749, 40.78278169917958], [-73.89702621590553, 40.782788110325896], [-73.89701522967316, 40.78279319244645], [-73.8970032774252, 40.78279680146563], [-73.89699069800244, 40.78279883506896], [-73.89697784802637, 40.78279923560485], [-73.89693756169261, 40.78282160979131], [-73.89693506111394, 40.782829076217936], [-73.89693091038077, 40.782836105159625], [-73.89692523288718, 40.78284248765727], [-73.8969181974156, 40.78284803396943], [-73.89691001311911, 40.78285257921323], [-73.89690092330346, 40.782855988265595], [-73.89689119819457, 40.78285815978087], [-73.89688112690442, 40.782859029203394], [-73.89687100883668, 40.78285857068649], [-73.89682004284245, 40.782886874384786], [-73.89681486142186, 40.782895121263614], [-73.89680797079022, 40.7829026105468], [-73.89679955755118, 40.78290913941771], [-73.89678984954256, 40.78291453106881], [-73.89677910966596, 40.7829186394892], [-73.8967676287666, 40.78292135341896], [-73.89675571775834, 40.78292259936249], [-73.89671260779281, 40.78294653981583], [-73.89670562222592, 40.78295599069708], [-73.8966966208677, 40.782964399003156], [-73.89668586681107, 40.78297151897511], [-73.89667367437603, 40.782977142509296], [-73.89666039992538, 40.78298110523994], [-73.89664643144633, 40.78298329134397], [-73.89659509004451, 40.78301180418], [-73.8965933615806, 40.783018091594485], [-73.8965903131632, 40.783024082301075], [-73.89658602567505, 40.783029617350344], [-73.89658061287393, 40.783034549882665], [-73.89657421837578, 40.783038749024776], [-73.89656701184407, 40.78304210336195], [-73.89655918448705, 40.78304452389478], [-73.89655094398557, 40.78304594639985], [-73.89654250898245, 40.78304633313439], [-73.89653410328104, 40.783045673837194], [-73.89648461205581, 40.7830731585063], [-73.8964805997714, 40.78308081936665], [-73.89647503050642, 40.783087895340195], [-73.89646804701138, 40.78309420505524], [-73.89645982828792, 40.78309958678076], [-73.89645058499843, 40.783103902572115], [-73.89644055406723, 40.783107041806566], [-73.89642999260793, 40.78310892401905], [-73.89641917133227, 40.78310950096469], [-73.89636397212593, 40.7831401567327], [-73.89636193163939, 40.78314625925535], [-73.89635854163745, 40.78315200438967], [-73.8963539030139, 40.783157221148436], [-73.89634815382301, 40.783161754270026], [-73.89634146517315, 40.783165468839286], [-73.89633403613205, 40.78316825430262], [-73.8963260878036, 40.78317002775906], [-73.89631785674719, 40.78317073642652], [-73.89630958793612, 40.78317035921375], [-73.89625550038656, 40.783200396085974], [-73.89624760541996, 40.783211000131], [-73.8962374649233, 40.78322043657913], [-73.89622537101242, 40.78322843359511], [-73.89621167207537, 40.783234760809215], [-73.8961967627367, 40.78323923595365], [-73.89618107248933, 40.7832417301131], [-73.89600297728212, 40.783336726805416], [-73.89600297461217, 40.783338436232086], [-73.89600184769625, 40.78334099600827], [-73.89600072122916, 40.78334270391824], [-73.89599959276781, 40.78334526536835], [-73.89599846784509, 40.7833469721069], [-73.89599734071753, 40.783348679513715], [-73.89599508912015, 40.78335123927995], [-73.8959939635342, 40.78335294719057], [-73.89599283640734, 40.78335465392723], [-73.8959905863623, 40.783356360823056], [-73.89598833543728, 40.783358066545404], [-73.89598721051145, 40.78335977495915], [-73.89598496024746, 40.78336148068202], [-73.89598271020125, 40.78336318808017], [-73.89598045861442, 40.78336489380175], [-73.8959793336913, 40.78336660054027], [-73.89597708342484, 40.783368307435666], [-73.89597370959535, 40.78336915742945], [-73.89597146021241, 40.783370863152776], [-73.89596921060684, 40.783372570048606], [-73.89596696123404, 40.7833734227326], [-73.89596471318599, 40.78337427374252], [-73.89596246291832, 40.78337598114008], [-73.89595909107092, 40.78337683163799], [-73.89595684169842, 40.78337768381917], [-73.89595346918802, 40.78337853548891], [-73.89595122113704, 40.783379388173785], [-73.89594784708642, 40.78338023816666], [-73.89594559903531, 40.78338109085143], [-73.89594222786035, 40.783381087807484], [-73.89593997981197, 40.783381938816945], [-73.89593660841629, 40.78338193577265], [-73.89593435904278, 40.783382788455995], [-73.89593098852892, 40.78338278541233], [-73.89592761601847, 40.783383636578726], [-73.89592536842149, 40.78338363454897], [-73.89592199724618, 40.783383631504435], [-73.8959197498697, 40.78338362947477], [-73.89574317446478, 40.78348169737461], [-73.89574204733012, 40.78348340360615], [-73.89574091951661, 40.78348596555676], [-73.89573979238119, 40.78348767229083], [-73.89573866611182, 40.78349023323773], [-73.89573641583523, 40.78349194012828], [-73.89573528958107, 40.78349364686307], [-73.89573416110616, 40.783496208813055], [-73.89573191171073, 40.78349791570443], [-73.89572966209678, 40.783499621422834], [-73.89572853496064, 40.783501328156774], [-73.89572628424244, 40.78350303504675], [-73.89572403484864, 40.7835047407653], [-73.89572178391077, 40.7835064471524], [-73.89571953451573, 40.78350815337352], [-73.89571728512053, 40.78350985959436], [-73.89571503618062, 40.78351071227383], [-73.89571278612301, 40.78351241916414], [-73.8957105360673, 40.78351412488181], [-73.89570828646838, 40.78351497588528], [-73.89570491394782, 40.78351582754778], [-73.89570266455097, 40.7835175344385], [-73.89570041495205, 40.783518385441795], [-73.89569704375366, 40.78351923710521], [-73.89569479437245, 40.78352008978386], [-73.89569142185125, 40.78352094144596], [-73.89568917225284, 40.78352179194653], [-73.89568579973087, 40.78352264411093], [-73.89568355167424, 40.783523495115276], [-73.8956801811533, 40.78352349206427], [-73.89567680775005, 40.7835243437251], [-73.89567455970759, 40.78352434169004], [-73.89567118850914, 40.78352519285007], [-73.8956678166659, 40.78352518979751], [-73.89566556994565, 40.7835251877635], [-73.8956621978817, 40.78352518471057], [-73.89548449389872, 40.78362581265404], [-73.89548336629524, 40.78362837342952], [-73.89548111532699, 40.78363093502906], [-73.8954799884035, 40.783632641258144], [-73.895478860578, 40.78363520270364], [-73.89547661117338, 40.78363690841719], [-73.89547543359768, 40.78363943195674], [-73.89547323239827, 40.78364117674741], [-73.89547098099025, 40.783643737173584], [-73.89546985516668, 40.78364544440882], [-73.89546760309798, 40.783648004331745], [-73.8954653536906, 40.78364971121788], [-73.89546310428493, 40.78365141693127], [-73.89546085333363, 40.78365312431838], [-73.89545747946356, 40.78365482901165], [-73.89545522917604, 40.78365653472401], [-73.89545297844766, 40.78365824043609], [-73.89545072903955, 40.78365994732181], [-73.89544735628792, 40.78366079897654], [-73.89544510555919, 40.78366250468835], [-73.89544173234822, 40.78366421005177], [-73.8954394842821, 40.78366506205654], [-73.89543611175125, 40.78366591320848], [-73.8954327378791, 40.78366761840366], [-73.8954293653481, 40.78366846955537], [-73.89542711573773, 40.78366932222859], [-73.89542374320577, 40.783670173882776], [-73.8954203713376, 40.78367102386218], [-73.89541699880546, 40.78367187551626], [-73.8954136276158, 40.783671872456864], [-73.89541025508595, 40.783672722435355], [-73.89540688321463, 40.78367357408966], [-73.89540463560783, 40.78367357204974], [-73.89540126441807, 40.78367356898998], [-73.89539789122516, 40.783674420140294], [-73.89539452003514, 40.783674417080334], [-73.89497050778391, 40.783909775658486], [-73.89496938282542, 40.7839114823869], [-73.89496825431539, 40.78391404332405], [-73.89496712712702, 40.78391660476501], [-73.89496599949766, 40.783919166205436], [-73.89496487233303, 40.78392087410455], [-73.89496374514425, 40.783923435545354], [-73.89496149505524, 40.78392514124823], [-73.89496036654482, 40.78392770218534], [-73.8949581162327, 40.78392940956315], [-73.8949569899512, 40.78393111629031], [-73.89495473917535, 40.78393367620431], [-73.89495248886473, 40.78393538240947], [-73.89495023877322, 40.7839370892848], [-73.89494798934481, 40.783938794988075], [-73.89494573903343, 40.783940501360526], [-73.8949434882816, 40.783942207565005], [-73.89494123885189, 40.78394391377076], [-73.89493898787828, 40.783945620645035], [-73.89493673844913, 40.78394732634802], [-73.89493336455074, 40.78394903102586], [-73.89493111492453, 40.78394988368937], [-73.8949288654932, 40.783951590564854], [-73.89492549206093, 40.783952441030955], [-73.89492324130713, 40.78395414790521], [-73.89491986941798, 40.783954997869984], [-73.89491649686505, 40.78395584950932], [-73.8949142487821, 40.783956701671336], [-73.89491087689001, 40.78395755331108], [-73.8949075036785, 40.78395840327423], [-73.89490525405182, 40.783959255937226], [-73.89490188284746, 40.78395925286285], [-73.89489851095692, 40.78396010332961], [-73.89489513840348, 40.78396095496821], [-73.8948917671992, 40.78396095189354], [-73.89488952046446, 40.783960949844385], [-73.89488614903968, 40.78396094676937], [-73.8948827773683, 40.78396179790599], [-73.89487940528232, 40.783961794830155], [-73.89471407574707, 40.784052182325034], [-73.8947129472232, 40.784054744264665], [-73.89471182090494, 40.784057305201316], [-73.89471069373161, 40.78405901309779], [-73.89470956586948, 40.78406157453553], [-73.89470843734627, 40.78406413597258], [-73.89470618790456, 40.78406584334628], [-73.89470506070603, 40.78406840310934], [-73.89470393353233, 40.78407011100578], [-73.89470168208007, 40.78407267141685], [-73.89469943264055, 40.784074377115154], [-73.89469830546662, 40.784076085011634], [-73.8946960551452, 40.78407779070914], [-73.89469380303117, 40.784080351119364], [-73.8946915535887, 40.784082058492814], [-73.89468930260671, 40.78408376368702], [-73.89468705316571, 40.78408546988765], [-73.89468480372369, 40.78408717675838], [-73.89468255475354, 40.78408802774233], [-73.89467918150143, 40.7840897330834], [-73.89467693073775, 40.78409143928271], [-73.89467468044246, 40.78409229194051], [-73.89467130785162, 40.7840939971145], [-73.89466905909967, 40.78409484927101], [-73.89466568518694, 40.784096553940884], [-73.89466343621375, 40.784097406599756], [-73.89466006453448, 40.78409825655714], [-73.89465669042808, 40.784099108187405], [-73.89465444233711, 40.78409996034433], [-73.8946510697733, 40.7841008119759], [-73.89464769787318, 40.78410166193269], [-73.8946443257803, 40.78410165884997], [-73.89464095409792, 40.78410251048206], [-73.89463870581552, 40.784102508426585], [-73.89463533325154, 40.784103360057564], [-73.89463196336244, 40.78410335697652], [-73.89462859148955, 40.784103353893556], [-73.89462522027807, 40.78410335081109], [-73.8946218490666, 40.784103347728525], [-73.89461960232684, 40.784103345674104], [-73.89461622957246, 40.78410334258998], [-73.89461285968343, 40.784103339508356], [-73.89460948850238, 40.7841024817111], [-73.89460611795216, 40.784102478628675], [-73.89460274809352, 40.78410162083229], [-73.89460050270435, 40.78410076573952], [-73.89459713215402, 40.78410076265683], [-73.89459376141407, 40.78409990485939], [-73.89459039243658, 40.78409904756605], [-73.89458814484642, 40.78409819079578], [-73.89458477699999, 40.7840964799615], [-73.89458253161125, 40.78409562486842], [-73.89457916087174, 40.78409476707058], [-73.89457691551564, 40.78409305609026], [-73.89457354631737, 40.78409219946627], [-73.89457130228129, 40.78409049016232], [-73.89456793242547, 40.784089631359805], [-73.8945656868486, 40.78408792088156], [-73.89456344281366, 40.78408621107488], [-73.89456119789922, 40.784084500094714], [-73.89455895254228, 40.784082790286654], [-73.8945567085077, 40.784081080479766], [-73.89455446447508, 40.78407936950015], [-73.89455221889907, 40.78407765918928], [-73.89444200402872, 40.78413478596389], [-73.89444424758946, 40.784137349984306], [-73.8944453679978, 40.78413991465137], [-73.89444648755892, 40.7841416234305], [-73.89444760797089, 40.78414418591964], [-73.89444872749884, 40.784146749915664], [-73.89444984658591, 40.78414931391138], [-73.89445096768834, 40.78415102386455], [-73.89445208810046, 40.78415358635364], [-73.89445320630652, 40.78415615034843], [-73.8944532022445, 40.78415871281287], [-73.89445432265519, 40.78416127630708], [-73.89445431859404, 40.784163838268995], [-73.89445431453207, 40.78416640073342], [-73.89445543406224, 40.78416896372416], [-73.8944554300003, 40.784171526188615], [-73.89445542593758, 40.784174089155535], [-73.89445429828497, 40.78417665008837], [-73.894454294223, 40.78417921255281], [-73.89445429016183, 40.78418177451462], [-73.8944531616241, 40.784184337624346], [-73.89445315891777, 40.78418604487298], [-73.8944520325863, 40.78418860630963], [-73.89445090405098, 40.78419116774406], [-73.89445089998888, 40.78419373020852], [-73.89444977211531, 40.78419629114096], [-73.89444864490147, 40.78419885307927], [-73.8944463941285, 40.78420055893892], [-73.89444626716197, 40.78420084746886], [-73.89444526559178, 40.784203120876], [-73.89444413705601, 40.784205682310414], [-73.8944430120789, 40.784207389033696], [-73.8944407603906, 40.78420995010942], [-73.89443851027798, 40.78421165630474], [-73.89443738174157, 40.784214217739034], [-73.89443513229072, 40.784215923432214], [-73.89443288195783, 40.78421762912448], [-73.89443062982703, 40.78422019070217], [-73.8944283803758, 40.784221896395245], [-73.89442612938156, 40.78422360208675], [-73.89442387992803, 40.784225308952486], [-73.89442163047623, 40.78422701464529], [-73.89441937948078, 40.78422872083926], [-73.89441600690759, 40.78422957246388], [-73.89441375657195, 40.78423127932853], [-73.89441038352743, 40.7842329839917], [-73.89440813388596, 40.78423383614247], [-73.89440475995951, 40.784235540804694], [-73.89440251185977, 40.78423639345923], [-73.89439913994963, 40.784237243408676], [-73.89439576535914, 40.784238949745394], [-73.89439351571626, 40.7842398023984], [-73.89439014314564, 40.78424065184441], [-73.89438677123266, 40.784241503468714], [-73.89438340001398, 40.78424150037917], [-73.8943800274389, 40.7842423525054], [-73.89437777934127, 40.78424320348422], [-73.89437440790242, 40.784243200394215], [-73.8943710362094, 40.78424405201838], [-73.89436766410918, 40.784244048927576], [-73.89436429222934, 40.784244045836886], [-73.89436092101074, 40.784244042746685], [-73.89435754979199, 40.78424403965639], [-73.8943541792346, 40.78424403656662], [-73.89435080801601, 40.784244033476135], [-73.8943474367972, 40.784244030385544], [-73.89434406693489, 40.78424317258161], [-73.89434069637755, 40.78424316949144], [-73.89433845098681, 40.78424231439358], [-73.89433507958181, 40.78424145658799], [-73.89433170770215, 40.78424145349633], [-73.89432833850138, 40.78424059569258], [-73.89432496929804, 40.78423973956406], [-73.8943227232489, 40.78423888279], [-73.89431935540516, 40.7842371702727], [-73.89431598465926, 40.78423631414247], [-73.89431373860967, 40.78423545787086], [-73.8943103701031, 40.7842337465255], [-73.89430812383343, 40.78423289025346], [-73.89430475620783, 40.784231179411314], [-73.89430251063374, 40.78422946842544], [-73.89429914077274, 40.784228610620225], [-73.89429689674014, 40.78422690080839], [-73.89429465182651, 40.78422519049315], [-73.89429240713474, 40.7842234795078], [-73.89429016222077, 40.78422176969502], [-73.894287916866, 40.784220059881704], [-73.89428567419266, 40.78421749418376], [-73.89428342927907, 40.78421578437084], [-73.89428118458582, 40.78421407455803], [-73.89428006418011, 40.78421151156465], [-73.89427782014994, 40.78420980074724], [-73.89427670106852, 40.78420723658245], [-73.89427445571475, 40.78420552676888], [-73.89427333530928, 40.78420296377555], [-73.89427234227921, 40.7842006880374], [-73.89427221777159, 40.78420039927694], [-73.89426997374065, 40.78419868929706], [-73.89426885333722, 40.78419612529841], [-73.89426773293206, 40.78419356230507], [-73.89426773700121, 40.78419099984061], [-73.89426661791961, 40.78418843634585], [-73.89426549616009, 40.78418672639028], [-73.89426550022667, 40.78418416560106], [-73.89426438136803, 40.78418160043122], [-73.89426438543651, 40.78417903846941], [-73.89426438950579, 40.78417647600498], [-73.89426326844011, 40.78417391301081], [-73.89426327250943, 40.784171350546465], [-73.89426327657955, 40.78416878757947], [-73.89426440578211, 40.784166226650065], [-73.89426440985139, 40.78416366418563], [-73.89426441391988, 40.78416110222373], [-73.89426554091929, 40.784158540789655], [-73.89426554498851, 40.78415597832521], [-73.89426667217508, 40.78415427043308], [-73.89426779939441, 40.78415170899916], [-73.89426892793549, 40.78414914806895], [-73.89427005647703, 40.784146586636155], [-73.89427118281638, 40.78414402419627], [-73.89427230999897, 40.78414231848181], [-73.89427343787925, 40.78413975704846], [-73.89427456620204, 40.784137194442714], [-73.89427681587465, 40.78413548875293], [-73.89427794221116, 40.7841329278207], [-73.89428019298647, 40.78413122162919], [-73.89428244401546, 40.78412866172856], [-73.8942835712019, 40.78412695333366], [-73.89428582131488, 40.78412524764413], [-73.89428807366826, 40.78412268606935], [-73.89429032311996, 40.784120980379], [-73.89429257257153, 40.784119274688614], [-73.89429482356759, 40.78411756782699], [-73.89429819660938, 40.78411586316716], [-73.89430044826526, 40.78411415697602], [-73.89430269724292, 40.78411330482687], [-73.89430607050454, 40.784111600166995], [-73.89430831995726, 40.78410989330367], [-73.89431169252575, 40.78410904168214], [-73.8943139432986, 40.784107335992644], [-73.89431731520487, 40.78410648504048], [-73.89431956418137, 40.78410563339349], [-73.89432293810341, 40.78410392873366], [-73.89432631001021, 40.78410307711113], [-73.89432968257816, 40.7841022254892], [-73.89433193064029, 40.784102227550484], [-73.89433530408981, 40.78410137592907], [-73.89433867599362, 40.78410052598142], [-73.89434204720496, 40.784100529072255], [-73.89434541911156, 40.78409967744913], [-73.89434879098383, 40.784099680540386], [-73.89435216288952, 40.78409882941966], [-73.89435553343955, 40.784098832509514], [-73.89435890531216, 40.784098835600474], [-73.89436227586219, 40.78409883869011], [-73.89436564839589, 40.78409884178148], [-73.89436789535593, 40.78409884384106], [-73.89437126521202, 40.78409970114169], [-73.89437463642331, 40.784099704231586], [-73.89437800561754, 40.78410056203403], [-73.89438137682909, 40.78410056512371], [-73.8943847466873, 40.78410142125129], [-73.89438811654294, 40.78410227905407], [-73.89439148705986, 40.7841031368572], [-73.89439373332566, 40.784103993630104], [-73.8943971023025, 40.784104849756524], [-73.894400472379, 40.78410570755909], [-73.89440271776488, 40.78410656332589], [-73.89440608560533, 40.78410827517054], [-73.89440945612567, 40.78410913129799], [-73.89441170169829, 40.784110842281756], [-73.89441394572995, 40.78411255209146], [-73.89441731558675, 40.784113409893315], [-73.8944195611603, 40.78411512037444], [-73.89442180585228, 40.784116831189756], [-73.8944251736954, 40.78411854202868], [-73.894427417726, 40.78412025301071], [-73.89442966330134, 40.78412196282154], [-73.89443190733381, 40.78412367263081], [-73.89454100171803, 40.784065692303834], [-73.89453987995448, 40.78406398184841], [-73.89453763595257, 40.78406141732678], [-73.89453651639212, 40.784059707375874], [-73.89453539597996, 40.78405714388241], [-73.89453315040394, 40.78405543407355], [-73.89453203219534, 40.7840528710847], [-73.89453091043097, 40.78405116129923], [-73.89452979002003, 40.784048597135566], [-73.8945297940783, 40.78404603517369], [-73.89452867451665, 40.784044326395374], [-73.89452755410484, 40.78404176290184], [-73.89452755816399, 40.78403920043743], [-73.89452643907497, 40.78403663694496], [-73.89452644313495, 40.784034073977935], [-73.8945253213705, 40.78403236452752], [-73.89452532542893, 40.78402980256562], [-73.89452532948816, 40.78402724010112], [-73.89452533354739, 40.78402467763672], [-73.89452533760662, 40.78402211517222], [-73.89452646478244, 40.78402040845018], [-73.89452646884082, 40.78401784648825], [-73.89452647290186, 40.784015282851065], [-73.89452760010882, 40.78401272141457], [-73.89452760416795, 40.78401015895006], [-73.8945287313435, 40.784008452228], [-73.89452985766874, 40.78400589079059], [-73.89453098619727, 40.78400332985791], [-73.89453211337262, 40.78400162313572], [-73.89453323969927, 40.7839990605256], [-73.89453436822727, 40.783996499760256], [-73.89453549474223, 40.78399479236743], [-73.89453662327091, 40.783992230931965], [-73.89453887271368, 40.783990524064116], [-73.89453999900691, 40.78398881734115], [-73.89454225068292, 40.7839862569334], [-73.89454450012349, 40.78398455123816], [-73.8945456273006, 40.783982842840544], [-73.89454787875519, 40.783980282935225], [-73.89455012907696, 40.78397857724058], [-73.89455237851892, 40.78397687037239], [-73.89455462950137, 40.783975164678274], [-73.89455687894126, 40.78397345898276], [-73.89455912703015, 40.78397260632516], [-73.89456137779206, 40.78397090063077], [-73.89456475258542, 40.78396919428935], [-73.89456700067407, 40.783968341631656], [-73.89456925165608, 40.78396663593725], [-73.89457262333171, 40.78396578480991], [-73.8945748729911, 40.78396407911418], [-73.89457824554901, 40.78396322748486], [-73.89458049495687, 40.78396237650338], [-73.89458386619197, 40.783961524872645], [-73.89458724029153, 40.783960673746975], [-73.89458948837961, 40.78395982108887], [-73.89459286093424, 40.78395897113434], [-73.89459623349147, 40.783958119504504], [-73.894599605357, 40.783958122588416], [-73.89460185432445, 40.7839572711035], [-73.89460522464705, 40.783957274185845], [-73.89460859742452, 40.78395642255582], [-73.89461196774687, 40.783956425637975], [-73.89461533983261, 40.78395642872162], [-73.8946175865672, 40.78395643077614], [-73.89462095843263, 40.78395643385942], [-73.89478403798543, 40.78386775219647], [-73.8947840420339, 40.78386519023452], [-73.8947851669973, 40.78386348300527], [-73.89478629551601, 40.78386092039464], [-73.8947874231513, 40.78385835895595], [-73.8947885503174, 40.783856652231194], [-73.89478967751104, 40.78385409129465], [-73.89479192694081, 40.78385238559444], [-73.89479305545987, 40.783849822481244], [-73.89479418196466, 40.78384811575598], [-73.89479643340498, 40.78384555584556], [-73.89479868437702, 40.78384385014663], [-73.89479981069096, 40.78384128753392], [-73.89480206144246, 40.78383958183478], [-73.8948043117545, 40.78383787513003], [-73.89480656184537, 40.78383616876004], [-73.89480881215557, 40.78383446306039], [-73.89481106158604, 40.78383275618717], [-73.89481331101454, 40.78383105048653], [-73.89481556198582, 40.78382934478726], [-73.89481781141393, 40.783827639086624], [-73.89482118528399, 40.78382678745139], [-73.89482343559621, 40.783825080076184], [-73.89482680814122, 40.78382423011483], [-73.89482905823212, 40.78382252324182], [-73.89483242945676, 40.783821671603846], [-73.89483467907723, 40.783820820617464], [-73.89483805162436, 40.78381996898053], [-73.89484029970367, 40.783819116820055], [-73.8948436715876, 40.78381826635503], [-73.89484704479567, 40.783817414718435], [-73.89485041888456, 40.783816563585155], [-73.8948526669644, 40.78381571092172], [-73.89485603750012, 40.78381571399694], [-73.89485941004429, 40.78381486403466], [-73.89486278124097, 40.78381486711027], [-73.89486615243806, 40.78381487018577], [-73.89486952432345, 40.783814018547325], [-73.89487177237567, 40.78381402059808], [-73.89487514357252, 40.78381402367333], [-73.89487851410838, 40.78381402674787], [-73.89530927247077, 40.783576114343006], [-73.89531040096466, 40.783573551727216], [-73.89531152813396, 40.783570990282776], [-73.89531265528278, 40.78356928305036], [-73.89531378156966, 40.783566722107686], [-73.89531490871939, 40.78356501420515], [-73.8953160372108, 40.78356245276197], [-73.89531716281648, 40.78356074552796], [-73.89531941310351, 40.78355904032091], [-73.89532054159481, 40.783556478877614], [-73.89532166654014, 40.78355477097291], [-73.89532391727117, 40.7835530635883], [-73.89532616689736, 40.78355135787779], [-73.89532729404469, 40.783549651147716], [-73.89532954345202, 40.783547944264264], [-73.89533179418005, 40.78354623855471], [-73.89533404446858, 40.78354453183954], [-73.895336293875, 40.783542825458525], [-73.89533854482292, 40.78354111974905], [-73.89534079288619, 40.78354026757881], [-73.89534304229129, 40.78353856186775], [-73.89534641770123, 40.783536857179215], [-73.8953486664265, 40.78353600450682], [-73.89535091515073, 40.78353515233678], [-73.89535428769524, 40.78353344764548], [-73.89535653730252, 40.78353259447109], [-73.89535878536606, 40.783531741797844], [-73.89536215788942, 40.78353089182086], [-73.8953644072755, 40.78353003914866], [-73.8953677791391, 40.78352918799816], [-73.89537115254609, 40.78352833634637], [-73.89537339926648, 40.783528338386155], [-73.89537677178932, 40.78352748840863], [-73.89538014231064, 40.78352749146851], [-73.89538239169637, 40.78352663879598], [-73.89538576376032, 40.78352664185709], [-73.89538913428123, 40.783526644916705], [-73.89539138234359, 40.78352579274534], [-73.89539475352586, 40.7835257958054], [-73.89557132620169, 40.783430292078116], [-73.8955713288801, 40.78342858432675], [-73.89557245735855, 40.783426023383505], [-73.89557358251623, 40.78342431497404], [-73.89557470965505, 40.78342260824152], [-73.89557471233522, 40.7834208993174], [-73.89557584013492, 40.783419192585534], [-73.8955769650697, 40.78341748585106], [-73.89557809154897, 40.783415777945244], [-73.89558034226594, 40.78341407223077], [-73.89558146808278, 40.78341236499439], [-73.89558259522298, 40.78341065708918], [-73.89558372015728, 40.78340895035468], [-73.89558597109594, 40.7834072434675], [-73.89558709823416, 40.78340553673493], [-73.89558934629069, 40.78340468405716], [-73.89559047254708, 40.78340297732368], [-73.89559272282256, 40.7834012716086], [-73.89559497087888, 40.783400418930725], [-73.89559722093655, 40.78339871154014], [-73.89559834673393, 40.78339785952062], [-73.8956005956717, 40.7833970068435], [-73.89560284506693, 40.78339529995468], [-73.89560509444264, 40.783394448953075], [-73.89560734271886, 40.78339359627508], [-73.89560959077187, 40.78339274527221], [-73.89561184015011, 40.7833918925952], [-73.89561408908675, 40.7833910404204], [-73.89561633714222, 40.78339018774209], [-73.89561858673788, 40.783389336740285], [-73.89562195925319, 40.78338848508044], [-73.89562420596884, 40.783388487115324], [-73.89562645556609, 40.78338763510832], [-73.89562870294274, 40.78338763714371], [-73.89563207479681, 40.78338678548284], [-73.89563432371625, 40.783386787519525], [-73.89563657043185, 40.78338678955415], [-73.89563881980776, 40.78338593788184], [-73.89564218899933, 40.78338594093286], [-73.89564443703733, 40.783385942968536], [-73.89564668397335, 40.783385945003175], [-73.89582887869106, 40.783287882831985], [-73.89583000582081, 40.78328617609704], [-73.89583113074661, 40.78328446935997], [-73.89583225921363, 40.78328190791162], [-73.89583338568458, 40.783280199500595], [-73.89583338835627, 40.783278491749186], [-73.89583563840293, 40.7832767848565], [-73.89583676553248, 40.78327507812137], [-73.89583789399813, 40.7832725171755], [-73.89583901958473, 40.783270810438935], [-73.89584014671415, 40.7832691037039], [-73.89584239698291, 40.78326739530339], [-73.89584352190911, 40.78326568789608], [-73.89584577149773, 40.783264836890005], [-73.89584689708599, 40.78326312898077], [-73.89584914801137, 40.78326142326121], [-73.89585139739397, 40.78325971754016], [-73.8958536467644, 40.783258864858475], [-73.89585477191125, 40.78325715694862], [-73.89585702195575, 40.78325545072558], [-73.89585927154383, 40.78325459971925], [-73.89586151959156, 40.783253747036106], [-73.89586376897543, 40.78325204014219], [-73.8958660192245, 40.78325118913622], [-73.89586826881491, 40.78325033645437], [-73.89587051752298, 40.78324948427431], [-73.89587388936489, 40.783248633778996], [-73.89587613741166, 40.78324778159821], [-73.8958783861204, 40.78324692891539], [-73.89588063504668, 40.78324607790803], [-73.89588400755133, 40.783245226240375], [-73.89588625558453, 40.78324522827134], [-73.89588850385209, 40.78324437558784], [-73.89589187635393, 40.78324352559529], [-73.89589412438713, 40.783243527626084], [-73.89589749423271, 40.78324353067024], [-73.89589974382244, 40.78324267798777], [-73.89590199053325, 40.78324268001724], [-73.8959053617012, 40.78324268306236], [-73.89590760973432, 40.7832426850929], [-73.89606377738421, 40.783157201805224], [-73.89605418906547, 40.78315389393809], [-73.8960455004657, 40.78314937255452], [-73.896037974756, 40.78314377460377], [-73.8960318398836, 40.78313726964354], [-73.89602728166864, 40.78313005470371], [-73.89602443817536, 40.783122348319225], [-73.89602339553039, 40.78311438391026], [-73.89602418531393, 40.78310640271234], [-73.89601982907595, 40.78310611983292], [-73.89601559929172, 40.78310527880468], [-73.89601161881058, 40.78310390405428], [-73.89600800324133, 40.78310203550976], [-73.89600485759347, 40.78309972744086], [-73.89600227322916, 40.78309704688297], [-73.8960003252084, 40.783094071689646], [-73.8959990701085, 40.783090888272085], [-73.89599854438265, 40.78308758908897], [-73.89598842603614, 40.783088950052345], [-73.89597815134013, 40.7830890501882], [-73.8959679904139, 40.783087886864166], [-73.89595821038567, 40.783085490663645], [-73.8959490683703, 40.783081924582135], [-73.89594080470867, 40.78307728237108], [-73.89593363665045, 40.78307168607322], [-73.89592775264177, 40.7830652828136], [-73.89592330737152, 40.78305824093238], [-73.89592041770368, 40.78305074555861], [-73.89591915960656, 40.78304299374372], [-73.89591956615465, 40.78303518928068], [-73.89592162665929, 40.78302753734654], [-73.8959192335144, 40.78302818730582], [-73.89591673311206, 40.783028532546744], [-73.89591419212613, 40.78302856386319], [-73.89591167831321, 40.783028280420346], [-73.89590925870489, 40.783027689775935], [-73.89590699782063, 40.78302680768015], [-73.895904955948, 40.78302565765389], [-73.89590318753406, 40.78302427036338], [-73.89590173973431, 40.78302268280106], [-73.89590065115445, 40.78302093729983], [-73.89589995082204, 40.783019080403974], [-73.89589965741153, 40.78301716162841], [-73.89589977874697, 40.78301523213773], [-73.89589074461992, 40.783017559534926], [-73.895881332576, 40.78301873811416], [-73.89587179367125, 40.78301873643823], [-73.895862382345, 40.78301755455179], [-73.89585334963449, 40.783015223980335], [-73.89584493647695, 40.78301180688939], [-73.89583736728378, 40.783007394425866], [-73.89583084395485, 40.7830021042875], [-73.89582554049217, 40.78299607758269], [-73.89582159835946, 40.78298947506698], [-73.89581912270793, 40.782982472855096], [-73.89581817957288, 40.78297525772286], [-73.89581879410997, 40.78296802212586], [-73.89582094992738, 40.7829609590649], [-73.89581810775456, 40.78296186313843], [-73.89581510553091, 40.782962389350836], [-73.89581202951504, 40.78296252258298], [-73.89580896808468, 40.782962259007235], [-73.89580600919882, 40.78296160619636], [-73.89580323787052, 40.782960582906455], [-73.89580073372352, 40.78295921853806], [-73.8957985687056, 40.78295755229136], [-73.89579680502031, 40.78295563203978], [-73.89579549334083, 40.782953512954855], [-73.8957946713532, 40.78295125592067], [-73.89579436267464, 40.78294892578488], [-73.89579457617347, 40.7829465894953], [-73.89579530571558, 40.7829443141767], [-73.89578528787293, 40.78294598086049], [-73.89577504648946, 40.782946382390584], [-73.89576485621456, 40.78294550799921], [-73.8957549903277, 40.78294338113537], [-73.89574571340852, 40.78294005883646], [-73.89573727424195, 40.78293563019876], [-73.89572989914612, 40.782930213987854], [-73.89572378590373, 40.782923955453754], [-73.89571909845655, 40.782917022435214], [-73.8957159625105, 40.78290960085943], [-73.89571446216367, 40.78290188975514], [-73.89571463765114, 40.78289409591567], [-73.89571648426619, 40.78288642835298], [-73.89571346322657, 40.78288741664731], [-73.89571026940133, 40.78288801354413], [-73.89570698988923, 40.78288820276582], [-73.8957037141256, 40.78288797915167], [-73.89570053144448, 40.7828873488002], [-73.89569752864062, 40.782886328901675], [-73.8956947876036, 40.78288494726962], [-73.89569238308417, 40.78288324158295], [-73.89569038065657, 40.78288125835706], [-73.89568883492858, 40.78287905167697], [-73.89568778805356, 40.78287668172074], [-73.89568726858153, 40.78287421311977], [-73.89568729067815, 40.782871713195384], [-73.89568785374138, 40.7828692501229], [-73.8956889424155, 40.782866891072885], [-73.89567998158762, 40.782869946102274], [-73.89567046440278, 40.78287178823511], [-73.89566067264735, 40.78287236292929], [-73.89565089623643, 40.782871653169195], [-73.89564142463114, 40.78286967996948], [-73.89563253826776, 40.78286650175287], [-73.89562450025493, 40.78286221262027], [-73.8956175485824, 40.78285693956494], [-73.89561188907587, 40.78285083871193], [-73.89560768930227, 40.78284409069575], [-73.89560507360832, 40.7828368953124], [-73.89560411943901, 40.782829465603314], [-73.89560485504498, 40.782822021547595], [-73.89560725864587, 40.782814783549526], [-73.89560367270391, 40.78281535541207], [-73.8956000110397, 40.78281545510634], [-73.895596380461, 40.78281507972422], [-73.8955928868692, 40.78281424021525], [-73.8955896321699, 40.7828129610674], [-73.89558671130004, 40.782811279592316], [-73.89558420945929, 40.782809244837644], [-73.89558219962473, 40.78280691615533], [-73.89558074042151, 40.782804361471555], [-73.8955798744133, 40.782801655304475], [-73.89557962686104, 40.78279887659094], [-73.89558000498525, 40.782796106383884], [-73.89558099775664, 40.78279342548825], [-73.89558257621648, 40.7827909121037], [-73.89557081050015, 40.78279237691986], [-73.89555888836699, 40.78279244703305], [-73.89554709424156, 40.782791120770476], [-73.89553570949546, 40.78278842977246], [-73.8955250057324, 40.78278443823796], [-73.89551523831139, 40.78277924139264], [-73.89542536848079, 40.78271997030174], [-73.89477429053532, 40.782253592494236], [-73.89542197971672, 40.7817177710163], [-73.8953575056752, 40.781657657133366], [-73.89535083030216, 40.78166394583996], [-73.89527872466454, 40.781722687462064], [-73.89528796173525, 40.781729238226504], [-73.89462439338403, 40.78226980919071], [-73.89460091774595, 40.782253158981405], [-73.8952373986715, 40.78173465477342], [-73.8952126420445, 40.7817170967075], [-73.8952434310705, 40.78169201531753], [-73.8952661225133, 40.781708110175515], [-73.89531098163849, 40.7816715663966], [-73.89531667833324, 40.78167560575161], [-73.89534092170129, 40.78165585474448], [-73.89534850467375, 40.78164926548439], [-73.89532546202713, 40.781627780695025], [-73.8953532079152, 40.78159759067088], [-73.89534178706286, 40.781588201555046], [-73.895284679113, 40.78154124925482], [-73.89527648087851, 40.78155307296019], [-73.89526579423838, 40.78156368650612], [-73.89525292084325, 40.78157279030384], [-73.89523822406923, 40.78158012738056], [-73.89523151925144, 40.781590932833076], [-73.89522278732707, 40.781600866481824], [-73.8952122200433, 40.781609710190544], [-73.8952000494492, 40.78161726975723], [-73.89518654280283, 40.78162337917888], [-73.89518621083418, 40.781644994971664], [-73.89518095259164, 40.78166623951376], [-73.8951709274017, 40.78168646907011], [-73.89515643903532, 40.78170507066042], [-73.89513792650503, 40.781721480633344], [-73.89511595076091, 40.781735201745754], [-73.895091177695, 40.781745818230114], [-73.89506435796241, 40.78175300839305], [-73.89506095403344, 40.78176120859101], [-73.8950558252207, 40.78176887274461], [-73.8950491107015, 40.78177579287629], [-73.89504099268392, 40.781781781198035], [-73.89503169146163, 40.78178667520784], [-73.89502145943706, 40.78179034209967], [-73.89501057427087, 40.7817926823669], [-73.89500426076594, 40.78181301430439], [-73.89499345089537, 40.78183222111453], [-73.89497846535502, 40.7818497329828], [-73.89495974872312, 40.78186503037893], [-73.89493785627018, 40.78187765946969], [-73.89491343748593, 40.78188724558305], [-73.89488721681121, 40.7818935043242], [-73.8948849682476, 40.78190215293248], [-73.89488094945148, 40.781910422635484], [-73.89487526147207, 40.7819181054968], [-73.8948680473298, 40.781925008335726], [-73.89485948841887, 40.781930957584585], [-73.89484979994775, 40.78193580365315], [-73.89483922552756, 40.78193942468992], [-73.89482803104511, 40.781941729646], [-73.89482005612895, 40.78196182403244], [-73.89480821823453, 40.78198078615776], [-73.8947927835019, 40.78199818970932], [-73.89477409893773, 40.78201364341398], [-73.89475258461356, 40.78202679983585], [-73.89472872422107, 40.78203736318702], [-73.89470305419827, 40.78204509597806], [-73.89470149816613, 40.78205312349085], [-73.89469817340536, 40.78206083406868], [-73.8946931762652, 40.7820680042628], [-73.89468665156055, 40.78207442628438], [-73.8946787883735, 40.78207991402622], [-73.89466981457493, 40.7820843084561], [-73.8946599902218, 40.78208748222531], [-73.89464960001914, 40.78208934335963], [-73.89464297969296, 40.78210909809309], [-73.89463214646791, 40.78212774654059], [-73.89461739957137, 40.78214477360321], [-73.89459914633294, 40.78215970896645], [-73.894577890934, 40.78217214009182], [-73.89455422048174, 40.782181723611494], [-73.89452878879166, 40.78218819481319], [-73.89452289521182, 40.78219953256316], [-73.89451476403873, 40.78221004030351], [-73.89450459124389, 40.78221946478267], [-73.89449262200559, 40.78222757885701], [-73.89447914479899, 40.78223418696569], [-73.8944644844443, 40.78223912984373], [-73.89445904521635, 40.78225694090812], [-73.89444990512632, 40.782273854184325], [-73.89443730255465, 40.78228942855654], [-73.8944215661873, 40.78230325782908], [-73.89440310644349, 40.78231498131964], [-73.89438240477216, 40.78232429326696], [-73.89436000109458, 40.78233095080531], [-73.89433647972336, 40.78233478029872], [-73.89433563703619, 40.782342605360995], [-73.89433312566332, 40.78235022099007], [-73.89432901293794, 40.78235742299937], [-73.89432340912812, 40.78236401829176], [-73.89431646448024, 40.78236983003721], [-73.89430836519104, 40.782374702413925], [-73.89429932841435, 40.7823785047857], [-73.8942895964409, 40.78238113520514], [-73.89427943019976, 40.782382523146445], [-73.89427308552676, 40.78240085243538], [-73.89426271446209, 40.78241808831561], [-73.8942486219517, 40.782433723985655], [-73.89423122236765, 40.78244729969674], [-73.89421102732304, 40.78245841627021], [-73.89418863062988, 40.782466746835475], [-73.89416469083802, 40.78247204644134], [-73.89413991187075, 40.782474159258506], [-73.89411502232694, 40.78247302316196], [-73.89409075405688, 40.78246867155745], [-73.89406782064256, 40.7824612323991], [-73.89404689641603, 40.78245092442739], [-73.89402859662968, 40.78243805073676], [-73.89401345936682, 40.78242298986389], [-73.89400177803752, 40.78242444300178], [-73.89398994178201, 40.78242452339438], [-73.89397822815634, 40.78242322915653], [-73.89396691184, 40.782420590637514], [-73.89395625819618, 40.78241666970977], [-73.89394651704815, 40.78241155831724], [-73.89393791682198, 40.78240537632005], [-73.89393065918937, 40.782398268683735], [-73.89390715107713, 40.78240092008177], [-73.89388338815877, 40.78240065450148], [-73.89385999276013, 40.78239747889801], [-73.89383757758196, 40.782391476437084], [-73.89381672965375, 40.78238280431674], [-73.89379799495987, 40.78237168965068], [-73.89378186414046, 40.78235842352004], [-73.8937687596429, 40.78234335335012], [-73.89375902465672, 40.78232687381222], [-73.89375291412807, 40.78230941648653], [-73.8937407595751, 40.78230640183597], [-73.89372943644702, 40.78230188891488], [-73.89371926175782, 40.78229600407176], [-73.89371052036864, 40.782288912064956], [-73.89370345701214, 40.782280811449766], [-73.89369826944079, 40.78227192901952], [-73.89366772689922, 40.78226410620599], [-73.89363936708129, 40.782252466576885], [-73.89361400842814, 40.7822373460434], [-73.89359238276961, 40.78221918097234], [-73.89357511420133, 40.78219849559338], [-73.89356270107572, 40.78217588686954], [-73.89355402716059, 40.782174411749175], [-73.8935457752577, 40.78217190115827], [-73.89353815779786, 40.782168419727704], [-73.89353137087878, 40.78216405708074], [-73.89352558921722, 40.78215892552584], [-73.89352096165226, 40.78215315716598], [-73.8935176073115, 40.78214690049712], [-73.89351561254607, 40.78214031658607], [-73.89348965070555, 40.782135010657626], [-73.8934652803139, 40.78212638311991], [-73.8934432306544, 40.78211469215196], [-73.89342416156191, 40.782100287606276], [-73.8934086436769, 40.782083600538535], [-73.89339714136854, 40.78206513030868], [-73.89338999883915, 40.782045429637336], [-73.8933809947239, 40.78204563512074], [-73.89337206544754, 40.782044730860974], [-73.89336344616665, 40.78204274067224], [-73.89335536387371, 40.78203971696695], [-73.89334803141958, 40.78203573937578], [-73.89334164190726, 40.782030912650356], [-73.89333636360736, 40.78202536390441], [-73.89333233552561, 40.78201923926683], [-73.89332966374293, 40.782012700032325], [-73.89332841862166, 40.78200591841439], [-73.89332863295186, 40.78199907300967], [-73.89330984873277, 40.78199986191117], [-73.89329113502652, 40.781998395589916], [-73.89327295752989, 40.781994710535415], [-73.89325576859568, 40.78198889845174], [-73.8932399959756, 40.78198110397414], [-73.89322603217543, 40.78197152107088], [-73.89321422468805, 40.781960388215126], [-73.89320486734462, 40.78194798245131], [-73.89319819300404, 40.78193461250021], [-73.89319436775718, 40.78192061107659], [-73.89316202560062, 40.78190028084161], [-73.89313414972762, 40.78187643307113], [-73.89310800167921, 40.78186982622676], [-73.8930834102953, 40.78186037704936], [-73.89306090692054, 40.78184828970713], [-73.89230750203646, 40.78246604157372], [-73.8922970209443, 40.782534350488575], [-73.89281494422083, 40.78291321498261], [-73.89274917876409, 40.78296570668481], [-73.892145935837, 40.78254084319402], [-73.89165459123714, 40.78219478153064], [-73.89171690105482, 40.782142286027295], [-73.89188955621871, 40.782258065805244], [-73.89214162714589, 40.782431726503106], [-73.89224191097428, 40.78241079906106], [-73.89300986180193, 40.78179802356213], [-73.89300067344033, 40.78177967250351], [-73.89298939715376, 40.78177656141238], [-73.89297895519823, 40.78177207268651], [-73.89296963642428, 40.7817663304951], [-73.89296169861163, 40.781759493681285], [-73.89295536133902, 40.78175175136809], [-73.89295079991076, 40.781743317726914], [-73.89292358924912, 40.78173825466916], [-73.89289787521786, 40.78172980475931], [-73.89287438169247, 40.78171820587102], [-73.89285377003881, 40.78170378452495], [-73.8928366204932, 40.78168694669624], [-73.89282341582967, 40.78166816638642], [-73.89281452776845, 40.781647972279266], [-73.89279559483698, 40.781641813716575], [-73.89277823731716, 40.781633389824336], [-73.89276292029237, 40.781622926315414], [-73.8927500541709, 40.78161070355262], [-73.8927264413444, 40.78160473077839], [-73.89270430544914, 40.78159608702238], [-73.89268416577576, 40.78158497506022], [-73.89266649478333, 40.7815716555698], [-73.89265170701735, 40.781556441015276], [-73.89264014938487, 40.78153968831779], [-73.89263209301524, 40.78152179048136], [-73.89261807616168, 40.78151765573372], [-73.89260500582098, 40.7815120068413], [-73.89259316320482, 40.78150496534142], [-73.89258280311076, 40.78149668273411], [-73.89257414843766, 40.78148733722181], [-73.89256738539288, 40.781477129875725], [-73.89254242548199, 40.78147194964012], [-73.89251880394876, 40.781463924455814], [-73.8924970782713, 40.78145324372082], [-73.89247776118444, 40.781440159505216], [-73.8924613085782, 40.781424980602154], [-73.89244810873872, 40.781408065240086], [-73.89243847318531, 40.781389812628305], [-73.89242703820902, 40.78138677167929], [-73.89241635810683, 40.78138242504229], [-73.8924066890586, 40.78137687697844], [-73.89239826299327, 40.781370260567726], [-73.8923912820232, 40.781362734515916], [-73.89238591359863, 40.78135447934798], [-73.89235450593688, 40.78134595847079], [-73.89232536996536, 40.78133362663563], [-73.89229929941422, 40.78131781979055], [-73.89227700450262, 40.78129896855047], [-73.89217832400549, 40.781370751048726], [-73.89216184201894, 40.78135819165571], [-73.89211148854682, 40.78139517920908], [-73.8920848045048, 40.78137365032211], [-73.89224627026225, 40.78125475229789], [-73.89222439440559, 40.78124229663877], [-73.89220555336304, 40.781227245336545], [-73.89219027348956, 40.78121001887567], [-73.89216923877872, 40.78120833149817], [-73.89214888304801, 40.781203962807126], [-73.89212979340492, 40.78119703880591], [-73.89211252043917, 40.781187759199106], [-73.89209756234288, 40.781176391632386], [-73.89208535054124, 40.78116326397291], [-73.89207623724933, 40.781148754852865], [-73.89207048531337, 40.78113328274863], [-73.8920682606297, 40.781117293911265], [-73.89205890137518, 40.781116286179405], [-73.89204988676438, 40.781114124131], [-73.89204146143163, 40.781110866438524], [-73.89203385401879, 40.78110660150759], [-73.89202727097202, 40.78110144507757], [-73.89202189093805, 40.78109553708098], [-73.89201785991676, 40.78108903784588], [-73.89201528729998, 40.78108212374489], [-73.89199386744068, 40.7810803387056], [-73.8919731168745, 40.781075923436], [-73.89195359069205, 40.78106899604761], [-73.8919358112311, 40.781059741852445], [-73.89192025410317, 40.781048408406164], [-73.8919073354694, 40.78103529888555], [-73.89189740090998, 40.78102076397833], [-73.8918907161779, 40.781005192501965], [-73.89188746009158, 40.780989001002474], [-73.8918877197505, 40.78097262261135], [-73.89189148820621, 40.78095649545957], [-73.89189866464768, 40.78094105095678], [-73.8919090570999, 40.78092670225121], [-73.89186486136488, 40.78089331551477], [-73.89184462538313, 40.78090252445148], [-73.89182265502505, 40.78090904425437], [-73.89179956340335, 40.78091269297924], [-73.8917759949222, 40.78091336880329], [-73.8917526072929, 40.780911052866756], [-73.89173005318031, 40.7809058097989], [-73.89170896198803, 40.78089778591513], [-73.8916899222946, 40.78088720513287], [-73.89167346542752, 40.78087436272355], [-73.8916600506369, 40.7808596170721], [-73.89165005227841, 40.7808433796756], [-73.89164374936645, 40.78082610366042], [-73.8916413177897, 40.78080827113611], [-73.89164282540101, 40.78079037974236], [-73.89164823012543, 40.780772928760925], [-73.89163850619785, 40.78076717990124], [-73.89163015626373, 40.780760295249145], [-73.8916234070446, 40.78075246174044], [-73.89161844179831, 40.780743892074796], [-73.89161539534275, 40.78073481894003], [-73.89161435039678, 40.78072548869462], [-73.89161533533172, 40.780716154677954], [-73.89161832340358, 40.78070707033189], [-73.89160188527265, 40.78069336601249], [-73.89158856861049, 40.78067782654753], [-73.89157872178795, 40.78066085845987], [-73.8915726024018, 40.78064290564628], [-73.89157037053627, 40.780624437763976], [-73.89157208457472, 40.78060593794431], [-73.8915776996741, 40.780587890153775], [-73.89158706893672, 40.780570766533316], [-73.89159994725463, 40.7805550150463], [-73.89161599772191, 40.78054104776053], [-73.89163480044746, 40.780529230067295], [-73.89165586354153, 40.78051987112379], [-73.89166112223361, 40.780506686622935], [-73.89166865651212, 40.780494156263735], [-73.8916795826261, 40.78048122234704], [-73.89169292724203, 40.780469673801136], [-73.89170839106461, 40.78045976963786], [-73.89170992453916, 40.780442185240084], [-73.89171520589215, 40.78042502561068], [-73.89172409402968, 40.780408749166], [-73.89173635150456, 40.78039379072798], [-73.89175165085925, 40.78038054990788], [-73.89176958337335, 40.780369380431154], [-73.89178966998287, 40.78036058068735], [-73.89181137407898, 40.7803543857594], [-73.89183411584222, 40.78035096114323], [-73.89183131970793, 40.780345835862846], [-73.89182971426752, 40.780340423209395], [-73.89182934654745, 40.780334881731], [-73.89183022731878, 40.78032937374906], [-73.89183233078141, 40.78032406060382], [-73.89183559532059, 40.78031909792821], [-73.89183992531079, 40.78031463108939], [-73.89184519391766, 40.78031079093021], [-73.89185124681225, 40.78030768993684], [-73.89185790669332, 40.78030541894371], [-73.89186497847899, 40.78030404447286], [-73.89187225502235, 40.78030360678522], [-73.89187952317843, 40.78030411870183], [-73.8918865700483, 40.78030556522741], [-73.89189014884877, 40.780287221920844], [-73.89189761602492, 40.78026956805684], [-73.89190877154532, 40.780253076541676], [-73.89192331657638, 40.78023818914502], [-73.89194086148919, 40.7802253046652], [-73.89196093629481, 40.780214768247106], [-73.89198300323623, 40.780206862136495], [-73.89200647119179, 40.78020179811912], [-73.89200730612095, 40.78019345396926], [-73.89200989201821, 40.780185319855185], [-73.89201416183664, 40.78017760667585], [-73.8920200048692, 40.780170514416604], [-73.89202726961882, 40.78016422696358], [-73.89203576772643, 40.780158907335846], [-73.8920452788555, 40.780154693459245], [-73.89205555640484, 40.78015169458996], [-73.89206633390033, 40.780149988481675], [-73.89207304907376, 40.78012998272488], [-73.89208379576472, 40.78011102085016], [-73.89209830973958, 40.780093569075184], [-73.8921162341394, 40.7800780564883], [-73.89213712825348, 40.78006486449857], [-73.89216047835552, 40.78005431745817], [-73.8921857103347, 40.7800466746876], [-73.89219199680349, 40.78003801576341], [-73.89219985583651, 40.78003013392446], [-73.89220912410164, 40.78002319297629], [-73.89221960897954, 40.780017337169944], [-73.89222901793929, 40.78001341150787], [-73.89223897344755, 40.78001036268577], [-73.89224933470005, 40.780008233824205], [-73.8922501476234, 40.77999868561964], [-73.89225208367779, 40.77998923129827], [-73.8922589555896, 40.77997148318573], [-73.89226976128364, 40.77995490568765], [-73.89228417521181, 40.7799399982377], [-73.8923017631205, 40.77992720995559], [-73.89232199513468, 40.77991692611601], [-73.8923442617198, 40.779909456541596], [-73.89236565941067, 40.77990530397231], [-73.89236647919967, 40.779898596629806], [-73.89236849146909, 40.77989203636961], [-73.89237165882133, 40.7798857451116], [-73.89237617181554, 40.77987954592239], [-73.89238180475073, 40.77987390108736], [-73.89238844181615, 40.779868926661415], [-73.89239594655692, 40.779864724916564], [-73.8924036436042, 40.77986156091854], [-73.89241182994762, 40.77985921141321], [-73.89242035724443, 40.779857718975336], [-73.8924290709736, 40.779857110649026], [-73.89242759248253, 40.77984708260408], [-73.89242798501732, 40.77983699618998], [-73.89243056350172, 40.77982608473777], [-73.89243532897221, 40.77981560718151], [-73.89244216696588, 40.77980581518164], [-73.89245091324001, 40.779796943932105], [-73.89246135771735, 40.77978920651141], [-73.89247324953169, 40.77978278876459], [-73.89248630305319, 40.77977784483908], [-73.89250020475025, 40.77977449348305], [-73.8924844883088, 40.77976912459995], [-73.89247016189191, 40.779761847562135], [-73.8924576228006, 40.77975286417733], [-73.89244721876898, 40.77974242357325], [-73.89243923832169, 40.779730815289774], [-73.89243390277157, 40.77971836124853], [-73.89243136008294, 40.77970540682545], [-73.89243168076788, 40.779692311273166], [-73.8924348559316, 40.77967943775762], [-73.89244079751904, 40.7796671432872], [-73.89244934075631, 40.77965576881201], [-73.89246024872139, 40.779645629768694], [-73.89253636793507, 40.77955306588064], [-73.8924774877329, 40.77951465264366], [-73.8924602412654, 40.779530694696604], [-73.8924397877444, 40.77954438717205], [-73.89241667344432, 40.779555364369116], [-73.8923915157053, 40.779563333106566], [-73.89236498644496, 40.77956808055385], [-73.89233779421183, 40.77956947991487], [-73.89231066526223, 40.77956749381555], [-73.8922843241618, 40.77956217530088], [-73.89221860284783, 40.77955925496351], [-73.89215283246001, 40.77956144577587], [-73.89208769910182, 40.779568724883845], [-73.89202388223302, 40.779581016352935], [-73.89196204758022, 40.779598191960815], [-73.89181791840944, 40.77958857362863], [-73.89166859702843, 40.77949253940986], [-73.89152956328459, 40.779256242559136], [-73.89124137199043, 40.77894869261772], [-73.89119589704391, 40.77856328788702], [-73.89218330746341, 40.777721813487354], [-73.89251064542336, 40.77739286927271], [-73.89250271923207, 40.77736469480252], [-73.89225365703376, 40.77720452353151], [-73.89213229037036, 40.77709093638001], [-73.89204077552344, 40.77696289867331], [-73.89187053975695, 40.7766558478132], [-73.89177229676267, 40.77659352107664], [-73.89165215982452, 40.77656232377481], [-73.89153058411918, 40.776476769142675], [-73.89143415104094, 40.77638052307171], [-73.89141605901193, 40.776338956117115], [-73.89140128106186, 40.7760448649402], [-73.8914801662345, 40.775896842194406], [-73.89152987178771, 40.77560563109897], [-73.89166785623843, 40.77548456204879], [-73.89176577789307, 40.7753571874525], [-73.89180883475127, 40.77521964078214], [-73.89179251014149, 40.775088728298805], [-73.89183244902246, 40.77488019994431], [-73.89034195999996, 40.77323760499998], [-73.89008921683674, 40.77362643576237], [-73.88997672719441, 40.773723610073965], [-73.88985169979834, 40.773753297181955], [-73.88975035188997, 40.77371739592509], [-73.88960719597459, 40.77362033808072], [-73.8895235330152, 40.77357518000311], [-73.88945486901865, 40.77353295106357], [-73.88942374299985, 40.773045928999906], [-73.88904000199996, 40.77091623799988], [-73.88873647199979, 40.76898593099992], [-73.8887255129998, 40.76891623899995], [-73.88862144999995, 40.76889404899988], [-73.8885192579999, 40.76885784099994], [-73.88842375699991, 40.76880764399987], [-73.88833975499979, 40.76874480599985], [-73.88827131000001, 40.76867187099991], [-73.88822109399986, 40.768592289999944], [-73.88819008199995, 40.76850987299992], [-73.88817759199992, 40.7684282569999], [-73.88781814499994, 40.767202077999876], [-73.88772988199987, 40.766914433999936], [-73.88770661799987, 40.76683859799995], [-73.88777877399983, 40.766823295999856], [-73.88835471099995, 40.76669771299988], [-73.88877310599993, 40.766511326999854], [-73.88918520499988, 40.766297690999934], [-73.88994886800005, 40.7661044099999], [-73.89042403199987, 40.76596206099988], [-73.89072002699994, 40.76588687999986], [-73.89089043699997, 40.765858197999925], [-73.89144489699993, 40.7660746539999], [-73.89165463899995, 40.76615653599992], [-73.89183535799988, 40.766227079999936], [-73.89206947799994, 40.76621963599993], [-73.89232583099981, 40.76621747199987], [-73.89277464899999, 40.766230073999914], [-73.89299530699995, 40.76624861399995], [-73.89325108799987, 40.76627144499989], [-73.89325764999997, 40.76621185499989], [-73.89349605899986, 40.765950929999924], [-73.89432371999989, 40.76614647099994], [-73.89595958499986, 40.76643103399986], [-73.90065875099992, 40.7672114339999], [-73.90202516599992, 40.767416783999884], [-73.90226637199996, 40.76745694399991], [-73.90251104199999, 40.76748593999987], [-73.90275780099984, 40.76750354699987], [-73.90300528799979, 40.76750966399989], [-73.90325211599999, 40.7675043239999], [-73.90413637799985, 40.7675203179999], [-73.90418895299992, 40.767530284999964], [-73.90427336799988, 40.76754510499992], [-73.90446527099986, 40.76758377799992], [-73.90490866399992, 40.76774556099986], [-73.905317179, 40.76789865099989], [-73.9055484799998, 40.76797657999987], [-73.90577352999988, 40.76802858899988], [-73.90611305099988, 40.76806770499989], [-73.90708728999991, 40.76823623199987], [-73.90803079099996, 40.76838473399991], [-73.90900867399995, 40.768547546999905], [-73.909980063, 40.76869723499987], [-73.91094174999986, 40.76885247299989], [-73.91194026099988, 40.769010152999954], [-73.91291678, 40.76916129699992], [-73.91388850199984, 40.769323929999906], [-73.91487376500001, 40.76947041699985], [-73.915848706, 40.76963357499988], [-73.91682128799984, 40.76979176099995], [-73.91759120499997, 40.769900870999926], [-73.91811686499992, 40.77002848599989], [-73.91770217299991, 40.7703931119999], [-73.91815346699994, 40.77070124299992], [-73.91870527799998, 40.77107888199988], [-73.91880312499995, 40.77114938899993], [-73.919421785, 40.77159517499992], [-73.920076088, 40.77206690899993], [-73.92076339599988, 40.77255254799995], [-73.92146878499983, 40.773051306999925], [-73.92216697499995, 40.77354687799987], [-73.92279264499986, 40.773962294999905], [-73.92398142900002, 40.77461511299991], [-73.92334041500001, 40.77512891199993], [-73.923277837, 40.77517762299988], [-73.92320981399997, 40.775234768999916], [-73.92302571500001, 40.77538430999989], [-73.92375761299998, 40.77654701199991], [-73.92331041499993, 40.776883294999934], [-73.92264888599983, 40.77742265099986], [-73.92200155699993, 40.77795459199994], [-73.92139486599993, 40.77845444399985], [-73.92078680499985, 40.778952266999944], [-73.92012731099999, 40.779488952999884], [-73.91939411499995, 40.78009035699992], [-73.9190647659999, 40.780359563999916], [-73.91861823099998, 40.780732868999955], [-73.91796581399993, 40.78125629199991], [-73.91975969999989, 40.782514177999886], [-73.91996458796496, 40.78263506876824], [-73.9190903467455, 40.783367646003924], [-73.91847384225471, 40.783754157697885], [-73.91818862939184, 40.78393296611367], [-73.91773807440356, 40.78407595141047], [-73.9172451642751, 40.78414200119837], [-73.91717118646426, 40.784193484371414], [-73.91718200679615, 40.78455206368194], [-73.91683277016413, 40.785191392487434], [-73.91575172894541, 40.7861075785781], [-73.91541718757415, 40.78606392941093], [-73.91534351270901, 40.786116348086814], [-73.91526013936094, 40.7861922166415], [-73.91558710139756, 40.78642963116581], [-73.91423947964563, 40.7878562095634], [-73.912599859115, 40.78937632963394], [-73.91222952426435, 40.789179279463596], [-73.91206853578718, 40.789338653812905], [-73.91195110905574, 40.78940179722099], [-73.91207975024415, 40.78952181498964], [-73.91177445569417, 40.78972251040847], [-73.91161831292102, 40.789586154430054], [-73.9112495903913, 40.78974393630263], [-73.91113056873596, 40.79011407609529], [-73.91029611590572, 40.79065675602386], [-73.90985862925774, 40.790945493781884], [-73.90677991420698, 40.79037510915573], [-73.90649492923363, 40.7901809903447]]]}}, {\"id\": \"223\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 224, \"Shape_Leng\": 0.0440204462769, \"Shape_Area\": 9.84786989135e-05, \"zone\": \"Stuy Town/Peter Cooper Village\", \"LocationID\": 224, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97511055399991, 40.735390254999864], [-73.97495992999993, 40.73536871799993], [-73.9748930509999, 40.73537519599991], [-73.9748261859999, 40.7353816739999], [-73.97474900599997, 40.735425187999944], [-73.97467193099992, 40.735398302999904], [-73.97443700979296, 40.73531635606464], [-73.97442281130168, 40.73523384837344], [-73.97440811789221, 40.735177896632045], [-73.97438778900487, 40.735091821036896], [-73.97437155854928, 40.73503034861578], [-73.97434824707203, 40.73494205848576], [-73.97431379174476, 40.73480348078473], [-73.97427029379338, 40.734630482401556], [-73.97425447779786, 40.73456678630613], [-73.97421154391887, 40.734405829321844], [-73.97418317281979, 40.73430095691658], [-73.97416917978924, 40.73423971179248], [-73.97415110840357, 40.734163966166065], [-73.97410591714231, 40.733993543547506], [-73.97406016216316, 40.73381107244231], [-73.97402174588252, 40.73366389393439], [-73.97399237089745, 40.73355457425366], [-73.97395905029705, 40.73342116553848], [-73.97394321924938, 40.73340308886323], [-73.97393531356981, 40.73336607678393], [-73.97390702695192, 40.73334971660825], [-73.9739013771136, 40.73332733730498], [-73.97389120382235, 40.73330926190455], [-73.97386631865577, 40.73331097608678], [-73.97386292958615, 40.73329118239611], [-73.97385275193625, 40.73328257140103], [-73.97384483854864, 40.73326536111373], [-73.97386520802397, 40.73324642803961], [-73.97385730036494, 40.73321630363853], [-73.97384145962663, 40.733219740365655], [-73.97384937381271, 40.733232655106264], [-73.97383126846695, 40.73324556510916], [-73.97380976846544, 40.733248999372925], [-73.97378375783396, 40.73322748070547], [-73.97380298847429, 40.733211987539875], [-73.97378376867147, 40.73319993482695], [-73.9737645264947, 40.73321284338783], [-73.97374076899366, 40.73320938893705], [-73.97372718707769, 40.73321971435091], [-73.97370795169591, 40.73321368520494], [-73.97368984982582, 40.73321885110606], [-73.97367627576054, 40.733205933888065], [-73.97369438098605, 40.733195609010984], [-73.97369212859832, 40.73317409574746], [-73.97371588914245, 40.733169796747084], [-73.97372607356364, 40.73315947055946], [-73.97373739762317, 40.73314311937235], [-73.9737577603272, 40.73313968486282], [-73.9737781392265, 40.73312419146021], [-73.97380529755296, 40.73312075748428], [-73.97381887171846, 40.7330992476546], [-73.97384036797052, 40.73310355629585], [-73.97387206174007, 40.73309495480662], [-73.97386810776207, 40.733065687195584], [-73.9737703082618, 40.732688446176816], [-73.97367702799221, 40.73231023125138], [-73.97364735766598, 40.732175493766555], [-73.9736273640125, 40.732107445607845], [-73.97360833346643, 40.73204268972965], [-73.9735396928971, 40.731764332823396], [-73.97352137914261, 40.73169016478128], [-73.97341321730852, 40.73128241642864], [-73.97338181410112, 40.73123602442203], [-73.9733659100069, 40.7312200199484], [-73.97334064688678, 40.73118304166194], [-73.97331715426388, 40.73115792717309], [-73.97328321111145, 40.73111267590847], [-73.97323625417938, 40.73105692674098], [-73.97320012915407, 40.73101663239001], [-73.9731459279161, 40.73096088099612], [-73.97309858148475, 40.7309136798907], [-73.97305304853799, 40.73086730375558], [-73.97300605975387, 40.73082313191045], [-73.97291929100172, 40.73075024150267], [-73.97283215562831, 40.730681481309645], [-73.9727461164315, 40.730605831946974], [-73.97264270492411, 40.73052353780736], [-73.97255990007288, 40.73045809361815], [-73.97247970053029, 40.73040080736292], [-73.97245936471883, 40.73038632937769], [-73.97233277439494, 40.73029657297066], [-73.97224018578797, 40.7302304971359], [-73.97223370120912, 40.730225677843165], [-73.9720891859024, 40.730118357041626], [-73.97205385214981, 40.73008665043243], [-73.97201026033457, 40.73004517148754], [-73.97196340301248, 40.73000037721745], [-73.97190238512813, 40.72993070864442], [-73.97185608949026, 40.729875550267224], [-73.97182992841701, 40.72984569147017], [-73.9717482235702, 40.729742014544556], [-73.97174494790077, 40.729734548588496], [-73.97170029038125, 40.72966570245393], [-73.97162729998553, 40.72955788414598], [-73.97157719441799, 40.72946913663781], [-73.97151882628022, 40.72935136174009], [-73.97150816031564, 40.72932396649237], [-73.97147994907196, 40.729249577698475], [-73.97145716122313, 40.729169407557954], [-73.97143981566721, 40.729095849864436], [-73.9714322181421, 40.72906361904416], [-73.97142139208222, 40.728995843972214], [-73.97141235109783, 40.728879856021706], [-73.97140659672053, 40.72879447130567], [-73.9714061790851, 40.728718764251106], [-73.97141946379264, 40.72856882754402], [-73.97142718443534, 40.72848582638753], [-73.97145269559667, 40.728341433313076], [-73.97145662182912, 40.72831923851683], [-73.97147343819977, 40.728209255928085], [-73.97151048447923, 40.72800276390659], [-73.97155553514948, 40.72770310859809], [-73.97156931647098, 40.7276937671984], [-73.97157278760149, 40.72764591976766], [-73.97158965149985, 40.72764003369322], [-73.97161872706992, 40.72765055818575], [-73.97163586152143, 40.72764797412134], [-73.97165160075566, 40.72764321253461], [-73.97168514866578, 40.72740588663651], [-73.97158604106335, 40.72739693305272], [-73.9715611436591, 40.72739468824764], [-73.97153686592743, 40.72739296510087], [-73.97152076789605, 40.727386166239775], [-73.97151282181225, 40.727374308581055], [-73.97149260058237, 40.72737312335137], [-73.9714903909284, 40.72735009810991], [-73.97162697285862, 40.72662845299842], [-73.97179602099988, 40.72671367999988], [-73.97185671300005, 40.72675868499989], [-73.97191244199995, 40.72679100899989], [-73.97197685099988, 40.726830355999944], [-73.97205015299988, 40.726880945999866], [-73.97329796799995, 40.72742800799989], [-73.97569651100005, 40.728475052999926], [-73.97580243799992, 40.72851847999986], [-73.9761329, 40.728638271999934], [-73.976152041, 40.72864520999985], [-73.976269971, 40.7286892589999], [-73.97690436400002, 40.72895782399994], [-73.97739386799988, 40.72916504799991], [-73.97802697999985, 40.729433059999984], [-73.97926849199996, 40.729981427999846], [-73.98034007599992, 40.73044246099994], [-73.98082000799998, 40.73063915899997], [-73.98150053399984, 40.730918076999934], [-73.98246876199994, 40.731314883999914], [-73.98255629299986, 40.731350133999875], [-73.98238628699993, 40.7314112439999], [-73.98232282599997, 40.731497889999915], [-73.98196782199994, 40.731982560999874], [-73.98180339299985, 40.7322047979999], [-73.98149910399985, 40.732618595999924], [-73.98125724799999, 40.73293857499988], [-73.98105323600001, 40.733208481999945], [-73.98088975699991, 40.733437070999834], [-73.980644989, 40.733779319999904], [-73.98041991299985, 40.73408982099988], [-73.98020496200002, 40.73438532199987], [-73.98009050499998, 40.73454437099989], [-73.97978904000004, 40.734961816999906], [-73.97944606000006, 40.73541451499991], [-73.9793481959999, 40.735553111999884], [-73.97890917399987, 40.736164125999906], [-73.97857449799984, 40.736677280999864], [-73.97849845599995, 40.73679095699989], [-73.97833820599989, 40.73672147099989], [-73.97792798399998, 40.73654414599988], [-73.97712441199997, 40.73619679799988], [-73.97636373299991, 40.73586797799992], [-73.97589093699993, 40.73567319899994], [-73.97522275599997, 40.73539790699988], [-73.97511055399991, 40.735390254999864]]]}}, {\"id\": \"224\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 225, \"Shape_Leng\": 0.0887992073281, \"Shape_Area\": 0.000310835395162, \"zone\": \"Stuyvesant Heights\", \"LocationID\": 225, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91741294299999, 40.68403405199993], [-73.9172663269999, 40.6833032239999], [-73.91712138499985, 40.682568471999964], [-73.9169763279999, 40.68183415899995], [-73.91683011499998, 40.68110098099987], [-73.91668638399985, 40.680371378999965], [-73.916539987, 40.67963749699986], [-73.91639300100002, 40.678905848999904], [-73.9163011989999, 40.67857711199991], [-73.91720270699984, 40.67862681499991], [-73.91796830599993, 40.67866902199989], [-73.91905083899984, 40.67872847399986], [-73.92164666399992, 40.67886992299994], [-73.92183500099998, 40.67894555899991], [-73.92197287199998, 40.67907846999987], [-73.9219999079998, 40.6791914139999], [-73.92201534599985, 40.679255899999916], [-73.92203214100006, 40.67932605899988], [-73.92205453499997, 40.6794196039999], [-73.92208787199984, 40.67955885799993], [-73.92211149999984, 40.67965755699986], [-73.92213242899994, 40.67974497799986], [-73.92215222999988, 40.67984463499988], [-73.92217715699984, 40.679970084999894], [-73.92221626999996, 40.680166934999924], [-73.92223681799994, 40.680270342999904], [-73.92225617599988, 40.68036776399987], [-73.92227774199988, 40.6804762929999], [-73.92231261899987, 40.680648472999955], [-73.92233849599998, 40.680776217999885], [-73.92235458599998, 40.68085564499991], [-73.92236962399983, 40.68092987999994], [-73.92238256000003, 40.68099373699993], [-73.92240197099996, 40.68108955299994], [-73.92242652099985, 40.68121074499992], [-73.92249266199998, 40.681616108999926], [-73.92250799299985, 40.68171006999996], [-73.9225447959999, 40.68193561299989], [-73.92267012199994, 40.68268151199991], [-73.92281374299995, 40.68341442999991], [-73.92609461699993, 40.683038709999884], [-73.92900883999985, 40.68270290499991], [-73.92916130699993, 40.68343632399992], [-73.93208669799994, 40.68310034999988], [-73.93500821699992, 40.68276376799985], [-73.93486257200001, 40.68203056899991], [-73.93471441099992, 40.6812973839999], [-73.93457098299989, 40.68056476199988], [-73.93450838999993, 40.6802216929999], [-73.93439530999997, 40.67983537999992], [-73.93435066499993, 40.67956340799991], [-73.93487210899987, 40.679593087999855], [-73.93739763699983, 40.679729899999934], [-73.93843613299988, 40.679785851999924], [-73.94032794, 40.679889974999874], [-73.94047634999994, 40.68063569499985], [-73.94062005399995, 40.68137012999991], [-73.94076893299986, 40.6821008389999], [-73.940913343, 40.682833616999865], [-73.94105783799992, 40.683566871999844], [-73.94120399299996, 40.68429983899994], [-73.94134827200003, 40.68503120299987], [-73.94149491799988, 40.685764528999904], [-73.94163933199995, 40.686497269999954], [-73.94178527599988, 40.68722837199987], [-73.94193245099987, 40.687962958999904], [-73.9420768489998, 40.6886972019999], [-73.94222203500003, 40.689427978999916], [-73.94236932699991, 40.690159944999884], [-73.94251587899991, 40.69089200099991], [-73.94266181699986, 40.691624343999855], [-73.94280765199987, 40.69235779399988], [-73.942951361, 40.693090783999864], [-73.94310040999994, 40.69382302899988], [-73.9431142779999, 40.693894719999896], [-73.94312826799985, 40.69396703799988], [-73.94324249099988, 40.69455748499995], [-73.943388021, 40.69528898999993], [-73.94352527499989, 40.6960308549999], [-73.94354024099975, 40.696108141999964], [-73.94355634200002, 40.696191282999926], [-73.94362121399985, 40.696526278999876], [-73.9436380689999, 40.696613317999905], [-73.94368427099982, 40.69685190199992], [-73.9437245359998, 40.6970598119999], [-73.9437430669999, 40.69715549899995], [-73.94378455600005, 40.69736972899993], [-73.94380383199999, 40.697469264999945], [-73.94391750199989, 40.69805620199984], [-73.94394947299996, 40.698221278999924], [-73.94299724499987, 40.69833009799986], [-73.9429206409998, 40.69833885099996], [-73.94253735900001, 40.69838264999989], [-73.94247471499993, 40.698389807999945], [-73.94185058700003, 40.69846112399994], [-73.94176673799987, 40.69847070399985], [-73.94115035299997, 40.69854113199994], [-73.94081408799997, 40.69858111999986], [-73.94072013299996, 40.698592293999916], [-73.9405942079999, 40.69860726799988], [-73.94051277999984, 40.69861695199994], [-73.94011148799991, 40.69866467199989], [-73.94002089599995, 40.69867544399993], [-73.93960973999992, 40.69872433499995], [-73.93952025999982, 40.69873497499989], [-73.93906425899979, 40.69878919499992], [-73.93896470899988, 40.698801031999935], [-73.93856854799981, 40.698848135999874], [-73.93817718599986, 40.69864022299987], [-73.93777354599992, 40.69840469299989], [-73.93739064799986, 40.69818579499985], [-73.93675978999995, 40.69783118899988], [-73.93638762599981, 40.69761530199985], [-73.93600679799988, 40.697405748999905], [-73.93522264399988, 40.69698654499993], [-73.93479699699994, 40.6967365919999], [-73.93468892700004, 40.69667495399994], [-73.93460675199995, 40.69662808499985], [-73.93457999399985, 40.696612823999956], [-73.93448784799989, 40.696560267999914], [-73.93413397299992, 40.69635843099992], [-73.93382817199998, 40.69618427699988], [-73.93311862899988, 40.69579115399997], [-73.93236605699987, 40.69535837699995], [-73.93192890099984, 40.69511465699991], [-73.93177404599997, 40.69502821499992], [-73.93145784300005, 40.6948554709999], [-73.93114516499988, 40.69467397499987], [-73.93051146699992, 40.69431652299989], [-73.9297019239999, 40.69386339699991], [-73.92882897899982, 40.69335047399987], [-73.9288283419999, 40.693350098999936], [-73.92864268999999, 40.6932410129999], [-73.92850659699988, 40.693165725999904], [-73.92804330699994, 40.692904171999906], [-73.92781120699986, 40.692774208999936], [-73.92750685299994, 40.69260381099993], [-73.92707069499987, 40.692346223999955], [-73.92644992099981, 40.692001768999944], [-73.92631612499989, 40.691928452999896], [-73.92556064099996, 40.69149083999987], [-73.92536488899991, 40.69136634399996], [-73.92439183300002, 40.690831748999926], [-73.92429213299995, 40.690773356999905], [-73.9236437979999, 40.690398837999915], [-73.9231696669999, 40.690123405999884], [-73.922807024, 40.68992392499993], [-73.92207133799985, 40.68948984099992], [-73.92130587799981, 40.68907164399991], [-73.92121369999991, 40.68901947699986], [-73.920961039, 40.6888720519999], [-73.92055013100003, 40.68864092199994], [-73.91980961899992, 40.68821124099985], [-73.91906109199992, 40.687784368999864], [-73.91878323299994, 40.687626087999924], [-73.91829345799984, 40.68735666799993], [-73.91804607, 40.68721324799989], [-73.91799580199995, 40.686965380999894], [-73.917915484, 40.68656593599996], [-73.91789200399997, 40.68644915899996], [-73.91784937499983, 40.686233166999926], [-73.91783531199984, 40.686163668999875], [-73.91778426799988, 40.68591141199988], [-73.91776547599997, 40.68581854099988], [-73.91770109499986, 40.68550035899995], [-73.91765638699995, 40.68527188299992], [-73.91764776599992, 40.68522782299993], [-73.91763742499997, 40.68517497599987], [-73.91762630599987, 40.68511814899987], [-73.91755763299994, 40.68476719199988], [-73.91741294299999, 40.68403405199993]]]}}, {\"id\": \"225\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 226, \"Shape_Leng\": 0.168957762041, \"Shape_Area\": 0.00076194414504, \"zone\": \"Sunnyside\", \"LocationID\": 226, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90989961499994, 40.749087274999944], [-73.91015860699994, 40.7485019679999], [-73.91164202799983, 40.74868457599991], [-73.91176417899987, 40.748441596999946], [-73.91182081399998, 40.74834287299993], [-73.91182610699991, 40.74833364499993], [-73.91182624999982, 40.748333392999925], [-73.91182635699991, 40.748333131999914], [-73.91187504099992, 40.74822079599992], [-73.91190979099993, 40.748105394999875], [-73.91190983999999, 40.74810522499993], [-73.91190987499982, 40.74810505399993], [-73.91193027699991, 40.747988209999924], [-73.91212028799991, 40.74729474899992], [-73.91232274099995, 40.746447085999904], [-73.9124485379999, 40.74592040199985], [-73.91213076499986, 40.74588664299989], [-73.9112472229998, 40.745783992999876], [-73.91150870799994, 40.74447070499985], [-73.91190626499984, 40.74255201199992], [-73.91193129999984, 40.74243121799987], [-73.9119412979999, 40.742382948999925], [-73.9126012949999, 40.74244224599993], [-73.91289084199997, 40.74248224999995], [-73.9137938869998, 40.742514491999934], [-73.91382781999981, 40.74241368799992], [-73.91443483199988, 40.74053105299986], [-73.91538439399999, 40.74072944299994], [-73.91567469599988, 40.740784929999904], [-73.916301085, 40.73906911799992], [-73.91651190599994, 40.738944757999874], [-73.91709074600003, 40.737144681999965], [-73.91717737999994, 40.736897870999876], [-73.91651522799987, 40.736757345999855], [-73.91687214199993, 40.7355413599999], [-73.91689741899991, 40.73544470799989], [-73.91751347899988, 40.735350984999904], [-73.91755768599998, 40.73529176299997], [-73.91759250800001, 40.735239440999834], [-73.91835109099986, 40.73315255899992], [-73.91879292099999, 40.73196944899989], [-73.91902180399987, 40.73208783799993], [-73.91903825199986, 40.73200596599987], [-73.91881334600006, 40.73191752199988], [-73.917563071, 40.731446009999885], [-73.917342506, 40.731382825999916], [-73.91736640299979, 40.731327577999906], [-73.91752107599989, 40.7310027049999], [-73.91760690999989, 40.730571980999855], [-73.9177303909999, 40.72987934399989], [-73.91796696699996, 40.7285007709999], [-73.9150723379999, 40.727970819999896], [-73.91505218099998, 40.72860118099991], [-73.91205021299994, 40.72850636299993], [-73.91200974700003, 40.72922886399989], [-73.91189388199996, 40.72956051399989], [-73.91177014899978, 40.7298527969999], [-73.91174734199988, 40.72991203299991], [-73.91148836199982, 40.72983170699988], [-73.910962087, 40.72964075299994], [-73.9105504049999, 40.72946621999993], [-73.91017627699985, 40.72927572499984], [-73.90981313899988, 40.72907325099989], [-73.90934522600006, 40.72880316199994], [-73.90712253099991, 40.727424727999924], [-73.90716705199989, 40.72738865399988], [-73.9073001379999, 40.727294369999896], [-73.90740997399989, 40.72720873899992], [-73.90804680899987, 40.72673388799993], [-73.90889173799998, 40.72606842599993], [-73.91134506899984, 40.72418216999995], [-73.91257187699985, 40.723239634999864], [-73.91270773499983, 40.723284502999874], [-73.91316969799993, 40.72308152599989], [-73.91384828299984, 40.72289841199989], [-73.91376963799999, 40.722746977999854], [-73.91367234399998, 40.72260115799988], [-73.91355755899988, 40.72246288799988], [-73.9134268889998, 40.722333936999874], [-73.91336473699985, 40.72217933499993], [-73.9133647129999, 40.72217927199988], [-73.91331372599998, 40.7220773499999], [-73.91328886299985, 40.72202764999987], [-73.913199729, 40.72188004399986], [-73.91309801799984, 40.721737595999876], [-73.91309792299982, 40.72173746899996], [-73.91309781699982, 40.72173734399993], [-73.91298420399994, 40.72160084899989], [-73.91245967199984, 40.720930533999855], [-73.9118888929999, 40.720262830999886], [-73.91132095099978, 40.71962422999992], [-73.91152490699986, 40.71956777999985], [-73.9123211429999, 40.7193483859999], [-73.91339850999998, 40.719051086999876], [-73.914265253, 40.71881117399987], [-73.91516686599985, 40.71855456599992], [-73.91610398900005, 40.71829810299989], [-73.9180503519999, 40.71776542099987], [-73.91930947299991, 40.717417101999935], [-73.92153810999987, 40.71679515599987], [-73.92232773978448, 40.71659264406108], [-73.92234972562501, 40.71664649139136], [-73.92235844722471, 40.71666784003853], [-73.92235122384676, 40.716670932907526], [-73.92241488477603, 40.7168025404293], [-73.92248379312586, 40.71695219661066], [-73.9225192815478, 40.71701815625348], [-73.92259771919795, 40.71716571262627], [-73.92267615785576, 40.71731326927821], [-73.9227941474539, 40.71753457614396], [-73.92305769386333, 40.717761573640864], [-73.92323928226051, 40.717926488066574], [-73.9234261670815, 40.7180973680073], [-73.92346602654163, 40.71813033432227], [-73.92346464691408, 40.71813625883179], [-73.92344083317268, 40.71814787807723], [-73.92344225406629, 40.71815701845247], [-73.9234476277226, 40.71816704482551], [-73.92345716590792, 40.718178767496546], [-73.92346794587377, 40.71819052550287], [-73.92347694825794, 40.71820464395342], [-73.92348314172996, 40.71821211273376], [-73.92349359141218, 40.71822312334816], [-73.92350469094387, 40.718235396038345], [-73.9235138017347, 40.71824696463085], [-73.92352244013037, 40.71825939701273], [-73.92353040240155, 40.71827217203989], [-73.92353810279464, 40.718286557496235], [-73.92354379439121, 40.718297566118395], [-73.92354915568099, 40.71830822321706], [-73.92355479997053, 40.718319591319954], [-73.9235596968506, 40.71833202121441], [-73.92356368450358, 40.71834338771438], [-73.92356761226665, 40.71835475584967], [-73.92357214222356, 40.71836823890907], [-73.92357623361883, 40.71838207298125], [-73.92357920969752, 40.71839733917026], [-73.9235820088276, 40.71841330734709], [-73.92358577561399, 40.71843104039189], [-73.92358943705864, 40.71844809002436], [-73.92359359652137, 40.71846440102603], [-73.92360061655475, 40.71848323470302], [-73.92360642980813, 40.71850204127718], [-73.92361279629088, 40.718523423944], [-73.92361923640783, 40.71854223093331], [-73.92362567165333, 40.71855645354539], [-73.92363171765355, 40.718568533397274], [-73.92363730133178, 40.71858094732593], [-73.9236427673789, 40.71859302578748], [-73.92365007474689, 40.718609194968955], [-73.92365661849291, 40.71862020415013], [-73.9236621693762, 40.71863050068331], [-73.92366925753885, 40.71864294090446], [-73.92367688864358, 40.718655022307445], [-73.92368412977082, 40.718666752147406], [-73.92368935133436, 40.71867545377241], [-73.9236956117481, 40.71868682227627], [-73.92370267524055, 40.71869870461428], [-73.92371016687846, 40.718709722796845], [-73.92371971500296, 40.71872306526936], [-73.92372611395874, 40.718736902036895], [-73.92373164800742, 40.7187505752275], [-73.92373690528115, 40.71875963739183], [-73.92374465735347, 40.7187701253544], [-73.92375491761781, 40.71878149600148], [-73.92376638207142, 40.71879536242284], [-73.9237820267752, 40.71880761933185], [-73.92379402849936, 40.71881723611648], [-73.92380674671496, 40.718831463546636], [-73.92382198001258, 40.71885133913323], [-73.9238351219118, 40.718868421833704], [-73.92384376878007, 40.71888261909163], [-73.92385334923793, 40.71889931244704], [-73.92386007813933, 40.71891424672612], [-73.92386706032462, 40.718925264561136], [-73.92387454980316, 40.71893821917454], [-73.92387958888955, 40.7189501907948], [-73.92388855796287, 40.71896318106351], [-73.92389434500978, 40.718974000927524], [-73.92390285173965, 40.718987153390536], [-73.9239090414079, 40.7189972878325], [-73.92391649802045, 40.71900741490247], [-73.92392427540345, 40.71901719037741], [-73.92393252266959, 40.71902927216988], [-73.92393867329348, 40.71904275678696], [-73.92394312815341, 40.71905944624553], [-73.92394603448032, 40.71907399988581], [-73.92394786656745, 40.71908640126851], [-73.92394943901726, 40.71909883883305], [-73.92395050174615, 40.719110887901074], [-73.92395341018386, 40.71912455984858], [-73.92395484199838, 40.71913501514193], [-73.9239551919984, 40.71914956910363], [-73.92395903593649, 40.71916233149515], [-73.92396381455407, 40.71917548149091], [-73.92396786815651, 40.719191098511196], [-73.92397016852695, 40.719207408437214], [-73.92397140459813, 40.71922442998474], [-73.92397268584806, 40.71924218834529], [-73.92397287854023, 40.71925991901506], [-73.92397336653804, 40.71927800185388], [-73.92397434741072, 40.71930043435084], [-73.923975051392, 40.71931640984393], [-73.92397555119685, 40.71932472185262], [-73.9239759929686, 40.719341057355365], [-73.92397537276364, 40.719356644986924], [-73.92397473793514, 40.71937366379086], [-73.92397464452561, 40.719392818249844], [-73.92397511878302, 40.7194123315912], [-73.92397560486825, 40.719433250825865], [-73.92397695379665, 40.71945383741931], [-73.92397661433331, 40.719471928422536], [-73.92397539769324, 40.719490370820544], [-73.92397393346643, 40.719507740882904], [-73.92397236754869, 40.71952121989886], [-73.92397159621802, 40.7195339792415], [-73.92397133400594, 40.719544956434426], [-73.92397167174758, 40.71956003139572], [-73.92397259069342, 40.719574215330276], [-73.92397250096981, 40.719590909994146], [-73.92397241310016, 40.719605786490085], [-73.92397261212277, 40.71961854764605], [-73.92397252474507, 40.719632027638184], [-73.92397146819152, 40.719646209598054], [-73.92397048345815, 40.71965861917091], [-73.92396920047416, 40.71967209837338], [-73.92396839140731, 40.719686280664114], [-73.92396737063302, 40.71969940035948], [-73.92396652595392, 40.719714303327194], [-73.9239648383132, 40.719730250938305], [-73.92396287796258, 40.7197472681959], [-73.92396065776029, 40.719763926941695], [-73.92395955005586, 40.71977990123739], [-73.92395877931119, 40.71979194909335], [-73.92395747204434, 40.719807022465055], [-73.923955063197, 40.71982208690042], [-73.92395208918511, 40.719835367471376], [-73.92394815558463, 40.719839786577204], [-73.92393107839393, 40.719848473960454], [-73.92391614720357, 40.719854398398525], [-73.92389817173341, 40.71986185218554], [-73.9238836528084, 40.71986963174933], [-73.9238708020666, 40.71987716899796], [-73.92385419630129, 40.71988756780091], [-73.92384119889218, 40.719899743768046], [-73.92383826447644, 40.71990860249048], [-73.92384565312443, 40.7199164332281], [-73.92385580808386, 40.719927444281836], [-73.92386387508843, 40.71994147915428], [-73.92387208726169, 40.71995410991191], [-73.92387763646855, 40.719966000617326], [-73.92388031880364, 40.71997947423108], [-73.923878791784, 40.71999048627015], [-73.92387549941546, 40.720001982969784], [-73.92387060543061, 40.720018279762634], [-73.92386637776903, 40.72003068551445], [-73.92386102477123, 40.72004344232816], [-73.92385634626123, 40.72005691928325], [-73.92385294982424, 40.720067183585684], [-73.92384777212683, 40.72008260569965], [-73.92384479990304, 40.720094292585465], [-73.92384064121788, 40.72010740986841], [-73.92383860518696, 40.72011874590188], [-73.92383686216742, 40.72013186528519], [-73.92383575405636, 40.72014816407813], [-73.92383927592586, 40.72016379013061], [-73.92384398143544, 40.72017833658507], [-73.9238492669484, 40.720193243605706], [-73.92385451880013, 40.720206393917245], [-73.92386045794106, 40.720218122882436], [-73.92386683865381, 40.720228419952164], [-73.92387461400548, 40.72023835725846], [-73.92388254458828, 40.720248682826565], [-73.9238918671838, 40.720263069997735], [-73.9238983156301, 40.72027479862748], [-73.9239055793793, 40.72028792396457], [-73.9239113432088, 40.7202989326132], [-73.923919623192, 40.72031296711904], [-73.9239266377356, 40.72032683058095], [-73.92393250672855, 40.72033927098248], [-73.92393822934632, 40.72035417828748], [-73.92394358934119, 40.720366942181634], [-73.92395053233457, 40.720380805594544], [-73.92395672000909, 40.72039357891363], [-73.92396330913726, 40.72040708140679], [-73.92397313388344, 40.72042680077998], [-73.92397522267473, 40.720431934846985], [-73.92398068897397, 40.72044436509592], [-73.92398590472052, 40.72045716457579], [-73.92399097924742, 40.7204710004509], [-73.92399594529478, 40.72048554656796], [-73.92400154887697, 40.72050117398884], [-73.92400734237232, 40.72051786482784], [-73.92401194196292, 40.7205317003883], [-73.92401604747515, 40.72054377793087], [-73.92402007071856, 40.72055443345097], [-73.92402572591716, 40.72056651302094], [-73.924031442296, 40.72057804314287], [-73.92403673088529, 40.720590122470405], [-73.92404162708033, 40.7206039582253], [-73.92404601516307, 40.72061675715619], [-73.92405097182021, 40.720629709078835], [-73.92405651848719, 40.720643905947426], [-73.92406161198602, 40.72066005504235], [-73.92406499736772, 40.72067070092598], [-73.92406956443814, 40.72068295048575], [-73.92407539879285, 40.7206941491476], [-73.92408331265625, 40.72070782271415], [-73.924091167956, 40.72072168671961], [-73.92410265533735, 40.72073607630514], [-73.92411410914978, 40.72074959137723], [-73.92412503339942, 40.72076060341568], [-73.92413761146997, 40.72077395672531], [-73.92415022503648, 40.720787274373606], [-73.92416013192478, 40.72079760005098], [-73.92416866729903, 40.720805954205915], [-73.92417901108873, 40.72081663147324], [-73.9241890266866, 40.72082585941625], [-73.92419991274558, 40.72083814110815], [-73.92420812627586, 40.7208491508513], [-73.9242158655834, 40.72086088047961], [-73.92422166415659, 40.720872599448455], [-73.92422716247574, 40.720887344591866], [-73.92422996407744, 40.72090136725978], [-73.92422897974355, 40.720913064173885], [-73.92422615071281, 40.72092316635863], [-73.92422252889321, 40.72093432963891], [-73.92422096542752, 40.72094566665908], [-73.92421698496739, 40.72095949656314], [-73.92421177483018, 40.720971703495074], [-73.9242057726273, 40.720983559103985], [-73.92420803239791, 40.72099385344084], [-73.92421469112381, 40.72100859934849], [-73.92421781243856, 40.721022432754026], [-73.92421220566214, 40.72103005622556], [-73.92419805836222, 40.72104311367061], [-73.92418927711957, 40.721052851752574], [-73.92418312641927, 40.72107038943791], [-73.92417833286738, 40.72109130519018], [-73.92417554976448, 40.72110282970644], [-73.92417303906538, 40.721114715087346], [-73.9241688362886, 40.721135621344146], [-73.92416658089165, 40.72115264008282], [-73.92416653051197, 40.72116576058417], [-73.92416781897656, 40.72117745028282], [-73.92416898638247, 40.72119200260039], [-73.92417073109073, 40.72120832957474], [-73.92417342397546, 40.72122288406962], [-73.92418033038982, 40.721239214443216], [-73.92418893863268, 40.721257312004845], [-73.92419728792142, 40.721273292526234], [-73.92420373531107, 40.721286083591664], [-73.92420992474553, 40.7212978132045], [-73.92421693970446, 40.72131164917253], [-73.92422205069707, 40.721323917343625], [-73.92422946953641, 40.72133635807828], [-73.92423442657227, 40.72134769938897], [-73.92423546427787, 40.72136208040768], [-73.92423468847616, 40.721378216919426], [-73.92423337602997, 40.72138760861513], [-73.92423466510306, 40.72139877680219], [-73.9242379261758, 40.72141404198266], [-73.92424364921001, 40.721429498758546], [-73.92425441686576, 40.72144298588944], [-73.92427034324635, 40.721457566677245], [-73.92428649522114, 40.72147088663668], [-73.92430045146966, 40.72147992768762], [-73.92431346186584, 40.721488311074864], [-73.92432831367466, 40.72150042465056], [-73.92434163922734, 40.72151322073087], [-73.92434971245676, 40.72152280689513], [-73.92436235781304, 40.72153873645049], [-73.92437677505144, 40.72154846180699], [-73.92439256537686, 40.72155821553774], [-73.9244088274067, 40.7215692848291], [-73.92442508959054, 40.72158041727589], [-73.92443950621615, 40.721590116322275], [-73.92445622878934, 40.72160225690817], [-73.92447486521583, 40.721617559643306], [-73.92449195162573, 40.721632168132686], [-73.9245079836073, 40.72164710947415], [-73.92452362614105, 40.72166238276377], [-73.92454009433322, 40.72167946705471], [-73.92454776466519, 40.72168924291632], [-73.92455738027354, 40.72170593572673], [-73.92456115360524, 40.72171905823001], [-73.92456327106687, 40.72173006378445], [-73.92456361197056, 40.72174247305472], [-73.92456384149197, 40.72175952039967], [-73.9245701837071, 40.72177176289196], [-73.92458022893142, 40.72178436681958], [-73.92458999565892, 40.72179381955194], [-73.92460131260097, 40.72180283253549], [-73.92461498333894, 40.72181307067512], [-73.92462496479494, 40.721820353570045], [-73.92463703551884, 40.721832824294545], [-73.92464655219553, 40.72184260186034], [-73.92466496577617, 40.721855869469906], [-73.92468728877478, 40.72186631194857], [-73.92471238383689, 40.721874955154824], [-73.9247352201419, 40.72188128249596], [-73.92475987801606, 40.72188807926037], [-73.92477989915965, 40.72189363128162], [-73.92479620262061, 40.7218989453283], [-73.92481109912397, 40.721904195463885], [-73.9248259823025, 40.721909743451526], [-73.92483760061812, 40.721913983434305], [-73.92485411517968, 40.72192131563871], [-73.92487063133576, 40.721926846090895], [-73.92488223617752, 40.72193347298252], [-73.92489722918822, 40.721945909902935], [-73.92491241131268, 40.72196013931399], [-73.92492641966595, 40.72197508715041], [-73.92493596992092, 40.72198716971546], [-73.9249436033662, 40.721998889842354], [-73.9249504772168, 40.72201133084395], [-73.92495786103832, 40.72202377100488], [-73.92496617943623, 40.72203653694395], [-73.92497619924848, 40.72205147430099], [-73.92498497838295, 40.72206533834206], [-73.92499190408849, 40.72207456470288], [-73.92500108646036, 40.72208720421688], [-73.92501055154403, 40.722101231203794], [-73.9250192017173, 40.7221136733581], [-73.9250293176932, 40.722127511294474], [-73.92504243690607, 40.72214388301797], [-73.92505017933362, 40.722153298206976], [-73.92505749462427, 40.72216307262996], [-73.9250703868396, 40.7221811997159], [-73.92508141547155, 40.72219470620336], [-73.92508955921214, 40.722205004349654], [-73.92510108236903, 40.72222011237809], [-73.92511001692077, 40.72223131132636], [-73.92512111551619, 40.72224606006672], [-73.92512758968573, 40.722255835110886], [-73.92513559141595, 40.722266133161334], [-73.92514109473282, 40.72227690720595], [-73.92514204061482, 40.72227873520337], [-73.92514667659884, 40.72229224573666], [-73.92515200046466, 40.72230536973], [-73.92515446906472, 40.72232046365768], [-73.92515207271921, 40.722335339820816], [-73.92514671628442, 40.722351995372065], [-73.92514099577905, 40.72236500346011], [-73.92514064055572, 40.72236583215185], [-73.92513492153522, 40.722377516108644], [-73.9251294655167, 40.72238813861882], [-73.92511995706556, 40.722402460165654], [-73.92511262787342, 40.722415215748285], [-73.92510477532092, 40.722430086686074], [-73.925095962869, 40.72244638918539], [-73.92508329808373, 40.72246800195519], [-73.9250779609383, 40.72247746357939], [-73.92506902906533, 40.72249480131461], [-73.92506115205212, 40.722509672233336], [-73.92505356249276, 40.72252347233964], [-73.92504484590496, 40.722538703395976], [-73.92503563051362, 40.72255535642939], [-73.92502435259331, 40.72257412481997], [-73.92501275326833, 40.72259536067002], [-73.92500707089407, 40.72260653251459], [-73.92499817485323, 40.7226227175155], [-73.92498530741834, 40.722646095036865], [-73.92497861842384, 40.72265794152464], [-73.92497349319298, 40.72266769075783], [-73.9249637677976, 40.72268610013893], [-73.9249541832126, 40.72270382442627], [-73.92494503875348, 40.722720801997696], [-73.92493801596369, 40.7227349626482], [-73.92493190430896, 40.72274915119919], [-73.92492550837535, 40.72276295207501], [-73.9249192447023, 40.72277642803456], [-73.92491025310872, 40.72279412640305], [-73.924899470711, 40.72281431807685], [-73.92489408651672, 40.72282405792342], [-73.92488162462871, 40.722843509539025], [-73.92487163418612, 40.72286513341072], [-73.92486728971545, 40.72287627043782], [-73.9248599173774, 40.72289576105525], [-73.92485077217577, 40.72291276659534], [-73.92484191126566, 40.72292975204937], [-73.9248329804374, 40.722946054952956], [-73.92482015200315, 40.72296481359908], [-73.92481162925223, 40.722975595419214], [-73.9248029190019, 40.72298569377449], [-73.92479598408008, 40.72299472111718], [-73.92478289780632, 40.72301169660296], [-73.92477010438043, 40.723030824160205], [-73.92476371392571, 40.72304088834496], [-73.92475919487978, 40.72304923274955], [-73.92474915338778, 40.72306374154622], [-73.9247372961058, 40.72308179871269], [-73.92473226484064, 40.72309329429544], [-73.92472716263893, 40.72310481730585], [-73.92472134948497, 40.723116673062044], [-73.92471429210676, 40.723129600678426], [-73.92470886951321, 40.72314112347842], [-73.92470409832374, 40.72315297991511], [-73.9246993502326, 40.72316485445947], [-73.92469465093318, 40.723175639943776], [-73.9246893707238, 40.72318645084717], [-73.92468409072853, 40.72319901893889], [-73.92467755167888, 40.723214089392116], [-73.92467172269511, 40.7232284396092], [-73.92466748617827, 40.723238891011604], [-73.92466299526907, 40.72325368403935], [-73.9246554295014, 40.72327706549346], [-73.92465148338148, 40.72329105625626], [-73.92464641233643, 40.7233057681557], [-73.92464135468852, 40.72332011937843], [-73.92463700588846, 40.72333483007487], [-73.92463355937976, 40.723346687375184], [-73.92462830986031, 40.72336227197145], [-73.92462353707093, 40.72337610019366], [-73.9246161933593, 40.72339061191695], [-73.92460918114135, 40.723406203566924], [-73.92459901536252, 40.72342461262704], [-73.92458724029376, 40.723444452818875], [-73.9245785228289, 40.72345967445597], [-73.92457248802818, 40.72346906305941], [-73.92456370306807, 40.723481267661256], [-73.92455402970798, 40.723494354378886], [-73.92454380115511, 40.72350691922104], [-73.92453545522265, 40.72351896211001], [-73.92453179550152, 40.72353045975492], [-73.92452702650351, 40.72354091197556], [-73.92451734212514, 40.723552242499544], [-73.92450004833422, 40.72356417146462], [-73.92448364585582, 40.723572058423045], [-73.9244229875485, 40.723589975986165], [-73.92433693392289, 40.72360384027381], [-73.92431405339124, 40.72360475299889], [-73.92428938620976, 40.72360446672827], [-73.92426185708126, 40.7236030432406], [-73.92423205370977, 40.72360184407611], [-73.92421456043476, 40.72360131038209], [-73.92419565801896, 40.72360124416269], [-73.92417725243759, 40.72360169977813], [-73.92415838368639, 40.72360262382558], [-73.92413821366316, 40.72360384571276], [-73.92411350850553, 40.723605196953685], [-73.92408962131452, 40.723606802699145], [-73.92406814812615, 40.723608049514034], [-73.92404589530673, 40.723609601547004], [-73.92402933335605, 40.72361167934416], [-73.92401788626974, 40.72361270710617], [-73.92399970283363, 40.723614541250235], [-73.92396961279084, 40.72361717835932], [-73.92394908612663, 40.72361851573895], [-73.92393046732963, 40.723619214422996], [-73.92390994245602, 40.72361956087592], [-73.92389038913083, 40.72361910718938], [-73.92387451846261, 40.72361802452596], [-73.92385820811734, 40.72361727595291], [-73.92383818294194, 40.72361544353513], [-73.92382000424585, 40.72361408928416], [-73.92380088889931, 40.72361421166176], [-73.92377758195006, 40.723615213959974], [-73.92374957358223, 40.72361849133683], [-73.92373442057428, 40.723620075485655], [-73.92371645160814, 40.723621711542854], [-73.92369585379056, 40.72362345340739], [-73.92367520963622, 40.723624700698366], [-73.92365677799924, 40.723626363914036], [-73.92363787416828, 40.72362829569396], [-73.92362087514346, 40.723630148656625], [-73.9236052491435, 40.72363173247472], [-73.92358727926026, 40.72363339481287], [-73.92356888400943, 40.723634815459796], [-73.92354496100383, 40.723636140628656], [-73.92352186735339, 40.723636621003294], [-73.92350394758387, 40.72363682537626], [-73.92348695034306, 40.72363654402902], [-73.92347086659312, 40.72363578566775], [-73.92345570360429, 40.72363550603773], [-73.92344009144804, 40.72363615219517], [-73.92342353042456, 40.72363682569633], [-73.9234055974963, 40.72363774974021], [-73.92338459834129, 40.72363902371492], [-73.92335988204199, 40.7236406562316], [-73.92333678844763, 40.72364223386953], [-73.9233145206555, 40.72364433473647], [-73.92328573216754, 40.723646396074024], [-73.92326263887985, 40.72364770431428], [-73.9232395559496, 40.723649004683104], [-73.92321566742926, 40.723651158729034], [-73.92319910696338, 40.7236532539981], [-73.92318045240826, 40.723653629713], [-73.92316168646867, 40.723657984614285], [-73.92313196363916, 40.723659306999245], [-73.92311225761829, 40.72365797808628], [-73.92309523824521, 40.723656264312574], [-73.92307871521786, 40.723656443250455], [-73.92306398950277, 40.723657531382145], [-73.92304208793995, 40.72366092122469], [-73.92301921560954, 40.72366480461793], [-73.9229973006625, 40.72366929241361], [-73.92296069104158, 40.72366915216279], [-73.92293413428841, 40.72366622495288], [-73.92291177917356, 40.72366308477563], [-73.92289409915618, 40.72366008392412], [-73.92287826463517, 40.723657073919505], [-73.92286333147072, 40.72365399298254], [-73.92284168885726, 40.72364881999641], [-73.92282120441699, 40.72364485213253], [-73.92279696737485, 40.72364200819898], [-73.92277276322162, 40.723640047651166], [-73.92274946035276, 40.72363788834605], [-73.92272521021324, 40.723635899780334], [-73.92270331567315, 40.7236337956897], [-73.92268282968776, 40.72363117052743], [-73.92266139852461, 40.723627292631356], [-73.92264135287964, 40.72362263280955], [-73.92261902301563, 40.72361787210574], [-73.92259107006865, 40.72361376532748], [-73.9225654605078, 40.72360982144662], [-73.92254728383853, 40.723607504219714], [-73.92252815613813, 40.723607077392465], [-73.92250766649262, 40.723607613091815], [-73.92248576885873, 40.723608371823055], [-73.92246382345051, 40.72360912281159], [-73.9224451801977, 40.72360982174165], [-73.92242698747023, 40.72361014269671], [-73.92240274462834, 40.723611548663335], [-73.92237991082241, 40.72361334373423], [-73.9223579754138, 40.7236155263934], [-73.92233231268695, 40.72361696824349], [-73.92231088437028, 40.723620492287566], [-73.92229897219161, 40.72362435713355], [-73.92228718990164, 40.72363938655336], [-73.922277656188, 40.72365354584812], [-73.9222656940553, 40.72366983729113], [-73.9222504231605, 40.72368220854989], [-73.92223359397735, 40.723690185028914], [-73.9222144702518, 40.72369689898134], [-73.9221948382105, 40.72370369266489], [-73.92218107895404, 40.72370758255221], [-73.9221624363502, 40.7237088309247], [-73.92214330961275, 40.72370755701878], [-73.92211958242353, 40.723703696734844], [-73.92209209376517, 40.72369741950495], [-73.92207557393212, 40.72369372541683], [-73.9220580844321, 40.72368995762839], [-73.922041564213, 40.72368659674591], [-73.92202457287847, 40.72368257900461], [-73.92200805343336, 40.72367855990509], [-73.92199290694263, 40.72367528119662], [-73.92197822194707, 40.72367222728487], [-73.92194937192383, 40.72366586734522], [-73.92193051002853, 40.723661460162425], [-73.9219066311652, 40.72365589682208], [-73.9218844207231, 40.72364960502941], [-73.92186508809634, 40.723643657777785], [-73.92184672311069, 40.72363876458811], [-73.92182786155625, 40.723634079126704], [-73.92181088183105, 40.723629572853554], [-73.92179667032231, 40.72362629575934], [-73.92178292005258, 40.72362252225985], [-73.92177011786582, 40.72361864067797], [-73.92174780344956, 40.72361165489237], [-73.92172496836295, 40.723603463895834], [-73.92170730455835, 40.72359703024919], [-73.92168728296133, 40.72359220777658], [-73.92166491956547, 40.72358588015437], [-73.92164260329814, 40.72357936375715], [-73.92161793603518, 40.723569990093516], [-73.92159279369614, 40.72356146311659], [-73.92156812636091, 40.72355216248377], [-73.92155162134374, 40.72354622581703], [-73.92153369522991, 40.723541360350765], [-73.92152506816119, 40.72353923827266], [-73.92151256626012, 40.72353225912887], [-73.92148694902394, 40.723524443793785], [-73.92147090534914, 40.72351905641525], [-73.92145391613036, 40.72351330887692], [-73.92143787220604, 40.723507390601114], [-73.92141367858304, 40.72349788219015], [-73.92138901309303, 40.72348631153231], [-73.92137624855995, 40.72348004535592], [-73.92135392698684, 40.72346893581692], [-73.92133302612834, 40.72345708828365], [-73.92131254152825, 40.72344368370191], [-73.921291206101, 40.72342915275493], [-73.92127212628812, 40.72341862170823], [-73.92125212089489, 40.72341059388083], [-73.92122604087237, 40.72340446181527], [-73.92120999375888, 40.72340183892499], [-73.92119296392273, 40.72339938657214], [-73.92118449107831, 40.723397961318], [-73.9211773787851, 40.72339676516617], [-73.92116223129145, 40.72339442282233], [-73.92114660949105, 40.72339226845181], [-73.92111823163248, 40.723387142629335], [-73.92110354670835, 40.72338411557765], [-73.92107559652341, 40.723377837788384], [-73.92106094777881, 40.72337475748258], [-73.92104814300374, 40.72337257872826], [-73.92102531663207, 40.723367952401205], [-73.92100388693066, 40.723362993817034], [-73.92098062255107, 40.72335784516723], [-73.92094175031816, 40.72334901109636], [-73.92092660336701, 40.723346227958984], [-73.92091050915185, 40.723342867206455], [-73.92089539860493, 40.72333984033841], [-73.92088071312541, 40.72333729841547], [-73.92085510748184, 40.72333023011707], [-73.92083556108093, 40.72332385049365], [-73.92081926712207, 40.72331977760221], [-73.92079877381357, 40.72331437886081], [-73.92077504504552, 40.72331202955763], [-73.92075174076672, 40.72331009533812], [-73.92073356094122, 40.72330981277042], [-73.92072305946152, 40.72331188555291], [-73.92071391583306, 40.72331682364268], [-73.92070138952494, 40.723329908072834], [-73.92069126659261, 40.72334176065738], [-73.920682492851, 40.72335431677778], [-73.92067277335117, 40.723366692824634], [-73.92066419891653, 40.7233803294612], [-73.92065949702021, 40.72339273476335], [-73.92065537238076, 40.723406888270155], [-73.9206513103748, 40.723419294515516], [-73.92064757931314, 40.72343134030356], [-73.9206432094334, 40.72344303552019], [-73.92063256961379, 40.72346251540544], [-73.92062640317927, 40.72347102922187], [-73.92062422272116, 40.723474035997334], [-73.92061613696757, 40.72348445829548], [-73.92059962790063, 40.723502151143094], [-73.92058373536926, 40.72351948322597], [-73.92056693765387, 40.7235304846822], [-73.92055874473739, 40.72353270395297], [-73.92054338097533, 40.72353315287746], [-73.92052520159467, 40.72353207754051], [-73.92051240133446, 40.72352665658389], [-73.9205074882824, 40.723527202679854], [-73.92048789087028, 40.72353421254118], [-73.92046642086521, 40.723543456248045], [-73.92045519421163, 40.72354794055583], [-73.92043873457798, 40.72355424209531], [-73.9204264602165, 40.72355448557341], [-73.9203969262659, 40.72360543367628], [-73.92037064235664, 40.723681113821044], [-73.92037847580916, 40.72368268477619], [-73.92039358633491, 40.72368577486949], [-73.92040966916171, 40.72368857798941], [-73.92042456800911, 40.72369191118121], [-73.92044061317388, 40.72369546127268], [-73.92045948636574, 40.72369992298612], [-73.92047253822959, 40.72370377741185], [-73.92048926848676, 40.72370820966326], [-73.9205083546329, 40.72371321229228], [-73.92052580777833, 40.723718961462836], [-73.92054419370669, 40.723724880309064], [-73.92056373915375, 40.723731396682716], [-73.92058657492441, 40.72373893086992], [-73.92061214329087, 40.72374726955754], [-73.92062681530118, 40.72375194361862], [-73.92064009136858, 40.72375541052289], [-73.92066663445847, 40.72376069719709], [-73.9206917819404, 40.7237648661704], [-73.92071601898118, 40.7237676736912], [-73.92073650541636, 40.72376933575726], [-73.9207593478999, 40.723771251401395], [-73.92078170579352, 40.72377255523485], [-73.92081641914338, 40.72377452409921], [-73.92083881199962, 40.723776241736815], [-73.9208588364714, 40.72377845295127], [-73.92087514616121, 40.72377920245132], [-73.92088913627248, 40.72377921206466], [-73.9209070691402, 40.7237791431343], [-73.92092478837887, 40.723778362065524], [-73.9209462455048, 40.72377980027506], [-73.92096392679434, 40.72378242432369], [-73.9209835116709, 40.72378619204107], [-73.92100630217578, 40.72379128658882], [-73.92103050094364, 40.72379723732215], [-73.92105009514302, 40.723802250093726], [-73.92106917060302, 40.72380722347264], [-73.92108547646293, 40.723811215085945], [-73.92110036025859, 40.72381578218221], [-73.9211210638419, 40.723824693212535], [-73.92113732965376, 40.72383257526557], [-73.9211554651799, 40.72384101629321], [-73.92117731354068, 40.72385220679656], [-73.92120104505189, 40.72386283971518], [-73.92122293166241, 40.7238720485617], [-73.92124200387715, 40.72387849145458], [-73.92126155304386, 40.723882701035876], [-73.92127832111741, 40.72388633225274], [-73.9212941527061, 40.72388994590688], [-73.92131183242266, 40.72389393842725], [-73.92133185603939, 40.72389710362381], [-73.92134862576677, 40.723899340003605], [-73.92136493282352, 40.72390179066687], [-73.92138101252571, 40.72390732131976], [-73.92139557202769, 40.723916913289614], [-73.92140450640761, 40.723927923050255], [-73.92141709680399, 40.72393086805336], [-73.92143386500696, 40.72393496748495], [-73.92146083643901, 40.723947422404024], [-73.92147176748499, 40.72395330033324], [-73.92148477864409, 40.723960396762], [-73.92149988652693, 40.723966575064324], [-73.92151374300536, 40.72396977288026], [-73.92151817990622, 40.723971558892124], [-73.92153328597037, 40.723978538636956], [-73.92154582684327, 40.72398428430482], [-73.92156328531532, 40.72399368875586], [-73.92157791535459, 40.72400363088156], [-73.92159370420008, 40.72401478920121], [-73.92160762415001, 40.724025254025634], [-73.9216214240031, 40.724035610041696], [-73.92163443618163, 40.72404261582212], [-73.9216493313895, 40.72404916402663], [-73.9216656348037, 40.72405531631629], [-73.92168565511973, 40.724061345821816], [-73.92170194744206, 40.72406588632377], [-73.92171774477397, 40.72406985002707], [-73.92173545902399, 40.724074499713296], [-73.92175479481182, 40.72407860134587], [-73.92177549283973, 40.7240818838573], [-73.92179483043961, 40.72408444524318], [-73.92181489298495, 40.72408396483194], [-73.92183379891677, 40.7240822747587], [-73.92184451115037, 40.72408157121448], [-73.92186107058602, 40.72408160875535], [-73.92187665823177, 40.724082250071184], [-73.92189462544043, 40.7240830099327], [-73.92191255372065, 40.72408617310424], [-73.92192440661712, 40.724091837013695], [-73.92193443525866, 40.72410004676225], [-73.92194647162722, 40.72411037574445], [-73.9219573905525, 40.72411556910474], [-73.92197438287592, 40.72412071397658], [-73.92199372029582, 40.7241232570685], [-73.92201165064529, 40.72412466521541], [-73.92203217195501, 40.72412750679789], [-73.92205078462618, 40.724132184350694], [-73.92206986223367, 40.724135465712024], [-73.92208803816622, 40.72413852197417], [-73.92210621542866, 40.72414232590632], [-73.92212761016063, 40.72414659036494], [-73.92214626057454, 40.724150448386894], [-73.92216396575357, 40.724151973291576], [-73.92218258271899, 40.72415355258579], [-73.92219982687453, 40.72415501401576], [-73.92221868964282, 40.72415824193368], [-73.9222340595316, 40.72416360127798], [-73.92225172329604, 40.72417099661431], [-73.92226802574567, 40.72417748253126], [-73.92228620280896, 40.724181286434835], [-73.92230392251479, 40.724180937708354], [-73.92232001360414, 40.724176067840446], [-73.9223354175614, 40.72417223282227], [-73.92235638198751, 40.72416936550097], [-73.92237410231868, 40.72416904423851], [-73.92239321762763, 40.724169660388156], [-73.92241840412618, 40.72417080330912], [-73.92244077340763, 40.72417230383941], [-73.92246219468647, 40.72417440766097], [-73.92248221887137, 40.72417777134443], [-73.92249992279847, 40.7241803857907], [-73.92251575589731, 40.724183385459526], [-73.92253299705406, 40.72418800773259], [-73.92254930440336, 40.724190873359476], [-73.92256326829788, 40.72419305322854], [-73.92258302832471, 40.72419957761191], [-73.92259126129625, 40.724203266055824], [-73.92259792197365, 40.7242062513375], [-73.92261186961284, 40.7242125731217], [-73.9226253566582, 40.724218840818544], [-73.92264165971227, 40.72422541664469], [-73.92265886271562, 40.724231244898576], [-73.92267610260356, 40.72423584135026], [-73.9226909998882, 40.72424015511669], [-73.92270938968392, 40.724243598411], [-73.92272663337585, 40.72424605939393], [-73.92274576207295, 40.7242452158333], [-73.92276253419685, 40.724244930730706], [-73.92278302197036, 40.724246177972006], [-73.92280260911701, 40.724248002070695], [-73.92284336872018, 40.724252719638415], [-73.92286129939659, 40.72425446136662], [-73.92287922843035, 40.72425648202282], [-73.92289812823829, 40.72425877170424], [-73.92291606069413, 40.7242595489735], [-73.9229312116518, 40.72425911885538], [-73.92294077605564, 40.72425893528165], [-73.92296314285906, 40.72426147219672], [-73.92314779007273, 40.72426376556558], [-73.92320876027148, 40.724263419258385], [-73.92321428988637, 40.724251815351316], [-73.92325723229807, 40.72425124021487], [-73.9234449378547, 40.72424874488516], [-73.92346284721405, 40.72424892867402], [-73.92350121459954, 40.724253843105394], [-73.92364637525688, 40.72426629472539], [-73.92368662861384, 40.72426877082092], [-73.92373820362327, 40.72428545581328], [-73.92375148655562, 40.724284311356094], [-73.92406939675712, 40.72428922258204], [-73.92412708328479, 40.72429260092605], [-73.9241468540506, 40.72428985059852], [-73.92417644379576, 40.72428070201839], [-73.92419217991092, 40.72427574219599], [-73.92428434537078, 40.72429101209338], [-73.92435683726049, 40.724307250533094], [-73.92448063404514, 40.7243393712912], [-73.9245573704283, 40.72435928708784], [-73.92465327395544, 40.72438764398434], [-73.92468803127821, 40.724403695517296], [-73.92490316165177, 40.72450657567146], [-73.92498732434241, 40.72455493335685], [-73.92506506894894, 40.7245948586997], [-73.92513666041378, 40.72463392476516], [-73.92513844593599, 40.72463498871582], [-73.9251696390123, 40.72465346910107], [-73.92528222312197, 40.72472087389907], [-73.9253014448662, 40.72473297979239], [-73.92539503255472, 40.72479984075856], [-73.92552386998959, 40.72488741719074], [-73.92553807417822, 40.7248982412345], [-73.92568785128063, 40.72501803418585], [-73.92619991273062, 40.72540408766011], [-73.92622803892246, 40.72542404253874], [-73.92655663912387, 40.725669065287285], [-73.92657286034844, 40.72568642855424], [-73.9266199471798, 40.72573931928689], [-73.92692613110464, 40.72607176474079], [-73.9270877469129, 40.72624843030603], [-73.92713796957001, 40.72630239330372], [-73.9272343323438, 40.72641205574683], [-73.92721721119939, 40.72646064526293], [-73.9277619680095, 40.72692986474524], [-73.92785278944132, 40.72699899076887], [-73.92810972310453, 40.72726138067442], [-73.92828343345148, 40.727512090834345], [-73.92855676085436, 40.72784482818558], [-73.92853805222317, 40.72788244993674], [-73.92861070615697, 40.72791977538612], [-73.92865537507299, 40.72789068006659], [-73.92877556756127, 40.72798068865881], [-73.92881964284582, 40.728008613628106], [-73.92889885311759, 40.728056596973985], [-73.92910283484903, 40.72819767868575], [-73.92923485876162, 40.72826016576104], [-73.9293925737079, 40.728308555994396], [-73.92994415137062, 40.72843310035094], [-73.93043971043073, 40.72856313728455], [-73.9307087143373, 40.72860666760421], [-73.93099163669307, 40.72861759858372], [-73.931192677967, 40.72864298727414], [-73.93126345392132, 40.728677203810534], [-73.93130248857328, 40.72870499917175], [-73.93131509360056, 40.72872944648484], [-73.9320430867126, 40.728809781103045], [-73.93413016327798, 40.729051906120176], [-73.93453176375674, 40.729104806534956], [-73.93588751333363, 40.72946511487208], [-73.9359282069927, 40.729481770738914], [-73.93613310488465, 40.72956336265126], [-73.93636315943891, 40.72964338330151], [-73.93644563967132, 40.729672145598656], [-73.9364813861402, 40.7296865198416], [-73.93725405595639, 40.730134604415746], [-73.9377791569555, 40.730400476320746], [-73.93804478507161, 40.730509967846054], [-73.9387573597808, 40.73107820170799], [-73.93901362096581, 40.73129845726269], [-73.93911054574805, 40.7313798065255], [-73.93917411650186, 40.73150623557753], [-73.93921026840314, 40.73155951002731], [-73.93936863200211, 40.731861038995305], [-73.93940973698923, 40.73193481336587], [-73.93941918776585, 40.73198156260662], [-73.93945253455881, 40.732087056714576], [-73.9394877863569, 40.7321661851416], [-73.93949645683922, 40.73219948191652], [-73.93941220925986, 40.73223586225096], [-73.93946414214876, 40.73230170873943], [-73.939471865783, 40.73233511256981], [-73.93948488081135, 40.732380901403424], [-73.93954196713415, 40.73250702929891], [-73.93966762886603, 40.73275272716036], [-73.93974902700619, 40.73290970931862], [-73.93978436990243, 40.73298199507146], [-73.93986983153752, 40.733161519792404], [-73.93991049829648, 40.733276328864335], [-73.94003200991092, 40.733245044305804], [-73.94005620356053, 40.73328637648506], [-73.94010107332389, 40.73336302898385], [-73.94016494736086, 40.73348602666648], [-73.94009696170568, 40.73351114258189], [-73.94001768032926, 40.73354043068554], [-73.94007982777362, 40.73368815610393], [-73.9401043685312, 40.73375295054129], [-73.94006709327033, 40.73376215298005], [-73.94034365972064, 40.73419527943397], [-73.94045033575216, 40.734432536274674], [-73.94066499856187, 40.73477888416132], [-73.94086498592704, 40.73507040909106], [-73.94122480467027, 40.735442890098504], [-73.94162192243846, 40.735798783650125], [-73.94163596800144, 40.73581907939849], [-73.94162290648393, 40.73582389971836], [-73.94161655224315, 40.735832848274335], [-73.94162472636452, 40.73584144233816], [-73.94180143973271, 40.73596257820129], [-73.94190459194984, 40.7360134911547], [-73.94196641273331, 40.736042284609944], [-73.94203049613847, 40.73606672046707], [-73.94205818845143, 40.73607907105276], [-73.94208360671529, 40.736091773298874], [-73.94210483382442, 40.73610473259584], [-73.9421480767946, 40.7361356684895], [-73.94216645287644, 40.736146654556165], [-73.94224043442604, 40.7361843606822], [-73.9422794992243, 40.73620169747974], [-73.94229951025788, 40.73621056815021], [-73.94240797511542, 40.73625267527337], [-73.94256066309957, 40.73631464468655], [-73.94261730140052, 40.73634000426394], [-73.9426417150164, 40.73634951601436], [-73.94264985873986, 40.73635169004053], [-73.9426591860718, 40.736353055000016], [-73.94270176880802, 40.73635420096762], [-73.94271158202731, 40.7363552686376], [-73.94272008107485, 40.73635710141665], [-73.94273679211524, 40.73636265686359], [-73.94275630720327, 40.73637084319732], [-73.94286937946475, 40.736424416010486], [-73.94291468021945, 40.736444096364934], [-73.94303934385847, 40.7364975946782], [-73.94316721664657, 40.73654898645434], [-73.94324208181114, 40.736579082146086], [-73.94328861692706, 40.73659601720271], [-73.94338584284627, 40.73663054567098], [-73.94343157326581, 40.73664748848283], [-73.9434641066377, 40.73665973288504], [-73.94357675397869, 40.736698258571536], [-73.94359480342632, 40.73671778977357], [-73.94377941508291, 40.73677548586041], [-73.94395913765192, 40.736836101268], [-73.94445052831142, 40.737001829842505], [-73.94484628751296, 40.73712779455903], [-73.94542629742466, 40.737323246383205], [-73.94560258882304, 40.737382097226046], [-73.94579981312464, 40.73745213200043], [-73.94589125600203, 40.73748258573916], [-73.94592145868833, 40.737493793036855], [-73.94592866792132, 40.737496263416126], [-73.94593191000172, 40.737497787753604], [-73.94593478459853, 40.737499455153596], [-73.94593727014934, 40.73750123930423], [-73.94593932901459, 40.73750311271366], [-73.94594095987212, 40.737505076553994], [-73.94594215413971, 40.73750712143962], [-73.94594410333028, 40.73751251570048], [-73.94594514077696, 40.737518370231044], [-73.94594669817852, 40.73752454833166], [-73.94594749665019, 40.737533130415414], [-73.9459447990511, 40.73754540314072], [-73.94593726125986, 40.7375564400967], [-73.9459182746333, 40.73756757872573], [-73.94591175927842, 40.73757093524086], [-73.94590575564354, 40.737573363732054], [-73.94590000091138, 40.73757494532625], [-73.94589726599044, 40.73757539468753], [-73.9458946374179, 40.737575609226745], [-73.94589209140617, 40.737575590440294], [-73.94588965372489, 40.737575337335066], [-73.9458872862647, 40.73757485860486], [-73.94585686433791, 40.73756888271067], [-73.9458390231052, 40.73755573572596], [-73.94582536619039, 40.73754895805148], [-73.94581929496616, 40.73754647831711], [-73.94581751868692, 40.737545946087565], [-73.94581568524721, 40.73754555823889], [-73.94581379024902, 40.73754530589001], [-73.945811813659, 40.737545178142476], [-73.94580767006968, 40.73754533869333], [-73.94580315890092, 40.737546030464884], [-73.94579426649551, 40.737548511366754], [-73.94577543679387, 40.737555427542326], [-73.94572336904317, 40.7375848673955], [-73.94568038333992, 40.73761231275128], [-73.94566737658243, 40.73762229253089], [-73.94563862186966, 40.737648214732744], [-73.9456245960258, 40.737659274404514], [-73.94558536491945, 40.737684353343674], [-73.94552863165563, 40.737715799545676], [-73.9454881134532, 40.737737004966846], [-73.94545576803195, 40.73775235199252], [-73.94542554603403, 40.73776467482329], [-73.94539673388435, 40.73777422441198], [-73.94524265235826, 40.73781652974276], [-73.9451023158267, 40.73785656205553], [-73.94505824046409, 40.73787025608997], [-73.94501457717676, 40.73788553443404], [-73.94496461225259, 40.7379058179528], [-73.9449113999564, 40.73793094325275], [-73.94489834665639, 40.73793866299152], [-73.94487440629372, 40.737956076795214], [-73.94486381627482, 40.73796292422634], [-73.94485391467622, 40.737967989169974], [-73.944842096523, 40.737972756002954], [-73.94478762138586, 40.737990525225705], [-73.94477519762279, 40.73799626458743], [-73.94476502360287, 40.73800261599569], [-73.94475495293814, 40.738011283505344], [-73.9446824542441, 40.73810797209326], [-73.94465794094907, 40.7381295730668], [-73.94463860064843, 40.73813835203952], [-73.94463507246135, 40.738138954108656], [-73.94463143837856, 40.73813924000429], [-73.94462770874196, 40.738139220620816], [-73.94462383708854, 40.7381388855492], [-73.94461565846566, 40.738137287434846], [-73.94460803640322, 40.738134897522706], [-73.94459950294983, 40.73813146247688], [-73.94456692344379, 40.738115579367374], [-73.94455295830858, 40.738109728485355], [-73.94454546685765, 40.73810733863194], [-73.94453842296237, 40.73810572464098], [-73.94453169955865, 40.73810486500772], [-73.94452525921655, 40.73810474497213], [-73.9444955775667, 40.738107314598054], [-73.94447895031381, 40.738112575456604], [-73.94446254857965, 40.7381189062452], [-73.9440212875199, 40.73828766393555], [-73.94257885812185, 40.73881602026855], [-73.9428038426758, 40.73886154746131], [-73.94280080814401, 40.73888072353937], [-73.94255612802647, 40.73902086587337], [-73.94215338158746, 40.73908005188025], [-73.94146173481568, 40.73924858381353], [-73.94140886345744, 40.73929957085316], [-73.9406430239998, 40.73898181299989], [-73.93993706299993, 40.738688177999904], [-73.93996324500002, 40.73864135599992], [-73.93982348499995, 40.73844550399986], [-73.93869664599998, 40.738140816999866], [-73.93867968899998, 40.73820830199989], [-73.93866464499993, 40.738268158999844], [-73.93864769800001, 40.7383258729999], [-73.93862452799989, 40.73840478099991], [-73.93819856599988, 40.74054323199988], [-73.93801851799995, 40.741455023999904], [-73.93765115799985, 40.74330513499994], [-73.93726110699994, 40.745176490999896], [-73.9376968159999, 40.74523857499982], [-73.9382576189999, 40.7450684369999], [-73.93925967299994, 40.744747867999884], [-73.94031559800004, 40.7444044979999], [-73.94174116799995, 40.743859150999846], [-73.94211068899993, 40.74371373399991], [-73.94297613999994, 40.743368929999875], [-73.94419561599983, 40.742882145999914], [-73.94401030399996, 40.7436394539999], [-73.94399533799991, 40.74365885099996], [-73.94383757199994, 40.74387181499993], [-73.94342742899987, 40.74423117199993], [-73.94272343099998, 40.74457686199988], [-73.94184736499984, 40.74490368899992], [-73.94032696100007, 40.74541314299995], [-73.93996463499987, 40.74574558399994], [-73.93952057199998, 40.745983778999914], [-73.93673442199999, 40.74812894999997], [-73.93663681899982, 40.748204093999895], [-73.93683480500002, 40.7483945519999], [-73.93739653799992, 40.748916439999896], [-73.93675697099987, 40.74914697199994], [-73.93645660599992, 40.74941256799986], [-73.93616086999987, 40.74967406999993], [-73.93609498999984, 40.7497315299999], [-73.93474259199988, 40.75097904399989], [-73.93383269399993, 40.7516047019999], [-73.93227623199981, 40.75195388599994], [-73.93195863799993, 40.75197425799992], [-73.93126506799997, 40.75199252199999], [-73.93015432399986, 40.75201212499993], [-73.92914224099992, 40.75203153399991], [-73.92874215599983, 40.75204162499994], [-73.92790807399994, 40.75205079499992], [-73.92646866899993, 40.752068567999856], [-73.92596072499995, 40.75209646699995], [-73.92562743799989, 40.752137243999904], [-73.9244174849999, 40.75240430199992], [-73.9229664339999, 40.75272880299985], [-73.9220402039999, 40.75293762799989], [-73.92161909499991, 40.75302765399987], [-73.92047925899985, 40.753285236999936], [-73.920276847, 40.75333811799993], [-73.91892348099987, 40.75363612399991], [-73.91763192799988, 40.753857847999896], [-73.91650534599988, 40.75389158999995], [-73.91547326299991, 40.75379181799991], [-73.91468429699998, 40.753680068999856], [-73.91446450799998, 40.75367641199993], [-73.9135418349999, 40.7534687139999], [-73.91256954199997, 40.75330421499995], [-73.91067474899997, 40.75299507599989], [-73.91035492999984, 40.75274990899987], [-73.90984318599997, 40.751638679999886], [-73.90955575999992, 40.75103731299989], [-73.90949713299983, 40.75095068399986], [-73.90949709799997, 40.75095063899988], [-73.90949707399992, 40.750950583999895], [-73.90945867199996, 40.75085521699987], [-73.9094586489999, 40.75085514499993], [-73.9094586379999, 40.75085507399991], [-73.9094433199999, 40.750755230999886], [-73.90944330899997, 40.7507551319999], [-73.90944332099993, 40.7507550329999], [-73.90945182699987, 40.75065550699988], [-73.90945183999992, 40.75065545299983], [-73.90948267299984, 40.75056086099988], [-73.90951622499988, 40.75044165999993], [-73.90971148199992, 40.749754708999944], [-73.90989961499994, 40.749087274999944]]]}}, {\"id\": \"226\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 227, \"Shape_Leng\": 0.0828301570789, \"Shape_Area\": 0.000268316529534, \"zone\": \"Sunset Park East\", \"LocationID\": 227, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.991928124, 40.6422049039999], [-73.99264456099989, 40.64172828999991], [-73.99336253699994, 40.641247798999885], [-73.99407691999981, 40.640770401999944], [-73.99433124099983, 40.64060028799991], [-73.99479539499987, 40.64028848699991], [-73.9955103369999, 40.63980966699993], [-73.99697357399992, 40.64069410499995], [-73.99917564699989, 40.642025441999934], [-73.99975690799992, 40.64146714999988], [-74.00034122099999, 40.6409053159999], [-74.00066078199991, 40.64059930899991], [-74.00092541799997, 40.64034590199991], [-74.00150771299988, 40.63978496199994], [-74.00209075399988, 40.63922525399994], [-74.00236943499988, 40.63895767799987], [-74.00267214299998, 40.63866700999992], [-74.00325273099993, 40.63810426199991], [-74.00383793099998, 40.63754593099991], [-74.00411104099993, 40.63728296099992], [-74.00442048399997, 40.63698498599989], [-74.00500346499996, 40.636425234999905], [-74.00558613, 40.63586622899986], [-74.00616926899994, 40.63530350099989], [-74.00641784699992, 40.635064849999885], [-74.00675122699992, 40.634744786999875], [-74.00735636699996, 40.6341636269999], [-74.00556490399993, 40.63308317599992], [-74.00630615499989, 40.632584087999916], [-74.00661914999998, 40.63237545799995], [-74.006806779, 40.63225039099994], [-74.00702302499997, 40.63210999499996], [-74.00773968299988, 40.63163008999991], [-74.0084560119999, 40.631150269999914], [-74.00922358699998, 40.63063393399994], [-74.01000340799995, 40.630113088999906], [-74.0109702639999, 40.63068186899988], [-74.01317645999991, 40.63201376599993], [-74.01514630699995, 40.63322193799993], [-74.01560204999993, 40.63351697799989], [-74.01591473499991, 40.633365548999905], [-74.01602010100001, 40.63331711399997], [-74.01600191100002, 40.63352419099988], [-74.016004681, 40.63373391199987], [-74.01602896399994, 40.633944430999854], [-74.01607491199997, 40.63415382299994], [-74.0161423149999, 40.634360157999915], [-74.01623055699994, 40.6345615299999], [-74.01626768999998, 40.634628405999855], [-74.01633863299998, 40.63475616099995], [-74.01646522999985, 40.63494240699986], [-74.01650516099988, 40.63499150699987], [-74.01660869499997, 40.635118833999876], [-74.01667603, 40.63518670499994], [-74.016835401, 40.63535542899995], [-74.01678445499995, 40.63539802299988], [-74.0166914329999, 40.63551966099993], [-74.0163054819999, 40.63589567399985], [-74.01606790799994, 40.636138450999894], [-74.01591339499994, 40.63628606999988], [-74.015739469, 40.636452236999894], [-74.01514575000002, 40.63701460999994], [-74.01456685499988, 40.63757090799991], [-74.0139608859999, 40.63815326699995], [-74.01616711499993, 40.63948244099991], [-74.01556266699994, 40.6400666369999], [-74.01497963599999, 40.64062546799992], [-74.01439632399996, 40.64118583699995], [-74.01381389999993, 40.64174524699988], [-74.0132320339999, 40.64230518499987], [-74.01264921199999, 40.64286578799991], [-74.01206572999999, 40.6434251979999], [-74.01148370499999, 40.64398661299994], [-74.01090222499988, 40.64454699899988], [-74.01031893899994, 40.6451066629999], [-74.00973537099993, 40.64566683499986], [-74.009152821, 40.646226734999885], [-74.00856800399991, 40.64678864999995], [-74.00798795199988, 40.64734471499993], [-74.00740505499991, 40.64790502999992], [-74.00682351799998, 40.64846488199987], [-74.00624091999993, 40.64902242499993], [-74.00566250199992, 40.64957838399988], [-74.00507350699995, 40.65014666199992], [-74.00449022999992, 40.65070538499987], [-74.00390815899992, 40.651265968999915], [-74.00332715599991, 40.65182948699987], [-74.00317064200001, 40.651980245999866], [-74.00275058799997, 40.65238483299988], [-74.00214474699993, 40.65295610699995], [-73.99779466099982, 40.650288889999935], [-73.99830602299986, 40.649740014999935], [-73.99604565399993, 40.648340653999846], [-73.99392923899993, 40.64706931099991], [-73.99180823499984, 40.64578396899985], [-73.98905872499985, 40.64411924099995], [-73.98977890299996, 40.64364488699989], [-73.99049634100004, 40.64316444099995], [-73.99121277399998, 40.64268589099995], [-73.991928124, 40.6422049039999]]]}}, {\"id\": \"227\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 228, \"Shape_Leng\": 0.177685177706, \"Shape_Area\": 0.000993414783816, \"zone\": \"Sunset Park West\", \"LocationID\": 228, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9969179149999, 40.6691534839999], [-73.99667191699996, 40.66873916799991], [-73.99659742999994, 40.66881192799984], [-73.99648142699999, 40.668925235999936], [-73.99616566099998, 40.668435497999965], [-73.99585129199981, 40.667972495999884], [-73.99507117699984, 40.66707573199994], [-73.99490350099988, 40.6667854349999], [-73.99494918099985, 40.666701642999904], [-73.99496779799996, 40.66666374099989], [-73.99498642699994, 40.66662582999985], [-73.99502872600004, 40.666515284999946], [-73.9951247889999, 40.666381354999885], [-73.99530800399985, 40.66615799899995], [-73.99575957400002, 40.66578388799989], [-73.9958739449999, 40.66573228499992], [-73.99595866299985, 40.665704548999855], [-73.995810253, 40.665530988999926], [-73.99583369399986, 40.66550726199988], [-73.99593232999982, 40.66540742499992], [-73.99605311599986, 40.66528516599988], [-73.9963724839999, 40.66496190899988], [-73.99692689899985, 40.664374914999925], [-73.99521061899985, 40.66332127999991], [-73.99510907599998, 40.66326211199989], [-73.99502082499983, 40.66320906699988], [-73.99283830599988, 40.66190848099993], [-73.99156751399993, 40.66114158699991], [-73.99063545399997, 40.660579088999896], [-73.99005216500004, 40.66113864799992], [-73.98947000099999, 40.66169904099992], [-73.98888159799985, 40.66224717499995], [-73.9887680069999, 40.66235596399985], [-73.988716511, 40.66240493799995], [-73.98804398999984, 40.661475435999925], [-73.98729619199992, 40.660441852999895], [-73.98725068899996, 40.66037895599992], [-73.98731033299997, 40.6603296239999], [-73.98744982499997, 40.660189572999926], [-73.98784059399986, 40.65981686399987], [-73.98843135299977, 40.65925121199996], [-73.9890128659999, 40.658690795999895], [-73.98959238099985, 40.658134060999934], [-73.990188415, 40.657600817999906], [-73.99238339499986, 40.65889901599996], [-73.99296401999982, 40.65833740899987], [-73.99516890199999, 40.659669521999945], [-73.99575278599988, 40.65910782999992], [-73.99633560799998, 40.65854721599988], [-73.99691824099983, 40.65798963299988], [-73.99750144599993, 40.65742847199989], [-73.9980910029999, 40.65686468799992], [-73.99867041499995, 40.65630686099992], [-73.99925656999994, 40.655745718999846], [-73.99983786599982, 40.655186157999935], [-74.00041318199995, 40.6546287019999], [-74.00099519199993, 40.65406723399986], [-74.00159439599989, 40.6534880839999], [-74.00214474699993, 40.65295610699995], [-74.00275058799997, 40.65238483299988], [-74.00317064200001, 40.651980245999866], [-74.00332715599991, 40.65182948699987], [-74.00390815899992, 40.651265968999915], [-74.00449022899996, 40.65070538499987], [-74.00507350599995, 40.6501466619999], [-74.00566250199992, 40.64957838399988], [-74.00624091999991, 40.64902242599987], [-74.00682351799998, 40.64846488199987], [-74.00740505499991, 40.64790502999992], [-74.00798795299995, 40.64734471499986], [-74.00856800399991, 40.64678864999995], [-74.009152821, 40.646226734999885], [-74.00973537099993, 40.64566683499986], [-74.01031893899994, 40.6451066629999], [-74.01090222499988, 40.64454699899988], [-74.01148370499999, 40.64398661299994], [-74.01206572999999, 40.6434251979999], [-74.01264921199999, 40.64286578799991], [-74.0132320339999, 40.64230518499987], [-74.01381389999993, 40.64174524699988], [-74.01439632399996, 40.64118583699995], [-74.01497963599999, 40.64062546799992], [-74.01556266699994, 40.6400666369999], [-74.01616711499993, 40.63948244099991], [-74.0139608859999, 40.63815326699995], [-74.01456685499988, 40.63757090799991], [-74.01514575000002, 40.63701460999994], [-74.01573946799994, 40.636452236999894], [-74.01591339499994, 40.63628606999988], [-74.01606790799994, 40.636138450999894], [-74.0163054819999, 40.63589567399985], [-74.0166914329999, 40.63551966099993], [-74.01678445499995, 40.63539802299988], [-74.016835401, 40.63535542899995], [-74.01705196399995, 40.63552755399995], [-74.01727269499997, 40.63569567599991], [-74.01750323299989, 40.635856163999904], [-74.01755212599994, 40.63588725099987], [-74.01799939199995, 40.636157921999875], [-74.01899546799997, 40.63675410399992], [-74.01955410699993, 40.637091226999914], [-74.01962150999994, 40.63713106399992], [-74.01976495999995, 40.63721743499991], [-74.02002934999997, 40.6373793659999], [-74.02032883799993, 40.63757296899991], [-74.020598444, 40.63777414199989], [-74.02084657599987, 40.637983350999924], [-74.02107776199989, 40.63820262199994], [-74.021114445, 40.638235535999904], [-74.02116029799993, 40.6382807979999], [-74.02129372099995, 40.63842824199984], [-74.02141038399992, 40.638584902999845], [-74.02150876399996, 40.63874907199992], [-74.02154646799997, 40.638830162999916], [-74.02160867799991, 40.63897677999994], [-74.02164685099997, 40.63909104799991], [-74.02167777199992, 40.63919125699987], [-74.02170098699993, 40.6392654599999], [-74.02170856799994, 40.63930090299994], [-74.02173880499991, 40.63944050699994], [-74.02174189000002, 40.63946629199994], [-74.02176040899997, 40.639824345999884], [-74.02172617299989, 40.640113319999934], [-74.0217092389999, 40.64020009799985], [-74.02166136899989, 40.64037438299987], [-74.02161540399997, 40.64050926299993], [-74.02159214600002, 40.64056920899986], [-74.021471331, 40.64082961299996], [-74.02126975099996, 40.64117350099991], [-74.02114153299986, 40.64136239499993], [-74.0211051969999, 40.641414135999895], [-74.02108520199992, 40.6414403539999], [-74.0210580239999, 40.641475982999935], [-74.02108051399998, 40.64149093699994], [-74.02111136799995, 40.64145200099989], [-74.0211362599999, 40.6414339679999], [-74.02120118199991, 40.64138693999995], [-74.02152001099994, 40.64115336799986], [-74.02172675999999, 40.641004941999924], [-74.02180903299994, 40.640950408999885], [-74.02222562199997, 40.64067782299986], [-74.02228790899991, 40.64063883599989], [-74.02246253899987, 40.640538898999836], [-74.02259016799994, 40.64043910599985], [-74.02310593199999, 40.639995325999934], [-74.02321780199998, 40.63990151499989], [-74.02331682999991, 40.63979778199991], [-74.02340085399995, 40.63968573199987], [-74.02346821799988, 40.639567334999875], [-74.02348203699995, 40.63953321299994], [-74.02351787199997, 40.63944477399987], [-74.02354939199988, 40.63932033399992], [-74.02392610699991, 40.638898857999926], [-74.02417094699996, 40.63866937099985], [-74.0245384189999, 40.63832493699993], [-74.02475437599998, 40.6381214179999], [-74.02695244699993, 40.63944606499998], [-74.02686129399991, 40.639536290999885], [-74.02664251499989, 40.63974743299988], [-74.02659984399993, 40.63978656099993], [-74.02646434999998, 40.639918292999916], [-74.02635007599996, 40.64001846499988], [-74.02628611999998, 40.64007452899986], [-74.0262626739999, 40.640094380999855], [-74.02662323799991, 40.6402011359999], [-74.02674292499995, 40.64024446799992], [-74.02691981399988, 40.640308507999926], [-74.02703439399988, 40.640349985999876], [-74.02742711399993, 40.64051433699994], [-74.02802775299989, 40.6407934899999], [-74.02839277799991, 40.64096313599989], [-74.02890829899998, 40.641173410999905], [-74.02889666799992, 40.64121402199986], [-74.02905968899994, 40.64172667299992], [-74.03260644499996, 40.643698910999944], [-74.03266558998958, 40.643720344890056], [-74.03262055640175, 40.64376540225929], [-74.03231070598248, 40.64357458661773], [-74.03223524436774, 40.64364337015534], [-74.03246968787244, 40.643786126334135], [-74.03254173795666, 40.64382548485449], [-74.0323108423411, 40.64403921448063], [-74.03226870865812, 40.6440743792925], [-74.03220689841643, 40.644029928892785], [-74.031978571573, 40.643893380255875], [-74.03189708494334, 40.6438893493698], [-74.03159762607343, 40.64420223984972], [-74.03051830863413, 40.645274425208974], [-74.02908090885776, 40.644403596930296], [-74.02899762468758, 40.64447196149296], [-74.02915418994519, 40.644581445419604], [-74.02886241127032, 40.64486602372296], [-74.02882159751947, 40.64484212565181], [-74.02909142239668, 40.64460058802812], [-74.0290865428924, 40.64458300179965], [-74.0289695821108, 40.644497461016805], [-74.02850116250595, 40.64417756470259], [-74.02842543290458, 40.644250724824936], [-74.02823134309659, 40.64443822883775], [-74.0281905304931, 40.64441433020631], [-74.02845721095777, 40.64415605924816], [-74.02820569545007, 40.644015982890345], [-74.02775298186873, 40.644506277021804], [-74.02703434223355, 40.6451978209833], [-74.02604948400315, 40.64626135032611], [-74.02954714502879, 40.648369331685906], [-74.0297407821107, 40.6484860262104], [-74.0293899048662, 40.64880405511391], [-74.02919183069376, 40.6486860167146], [-74.02571758804807, 40.64661549229093], [-74.02552081269253, 40.646755138147135], [-74.02518511713897, 40.64709632778354], [-74.02516202606716, 40.64713874849414], [-74.02483957406781, 40.64694442476538], [-74.02467396344868, 40.64709451887871], [-74.02461487103106, 40.64715112511878], [-74.02629688858224, 40.648168926521], [-74.02625506529243, 40.64821273567891], [-74.02624191290109, 40.64822653781882], [-74.02622102475574, 40.64824842780924], [-74.02488305491133, 40.64741052302421], [-74.02468390384902, 40.64760566521817], [-74.02435496936715, 40.6473992781534], [-74.02414566050564, 40.647615101259845], [-74.02512911955235, 40.6483231814572], [-74.02496792117127, 40.64860596226746], [-74.02553864578033, 40.64978215934804], [-74.02484436363444, 40.650599638057045], [-74.02520581187576, 40.650818467899384], [-74.02541553929463, 40.65061869708251], [-74.02605133196793, 40.6509943007431], [-74.02587113764648, 40.651160535797345], [-74.02592807625282, 40.65119665312026], [-74.02577633161965, 40.65133397795868], [-74.02569093920555, 40.651319546916866], [-74.02552971543649, 40.651478550692815], [-74.02491122849032, 40.65110628579329], [-74.02508999105069, 40.65092400267911], [-74.02507739284503, 40.650914412927115], [-74.0251025719023, 40.65089330557687], [-74.02477160495674, 40.650687707435374], [-74.024523280554, 40.65093755347267], [-74.02448801174862, 40.65091837716098], [-74.0244607759946, 40.65094460933543], [-74.02428919364958, 40.65111207358976], [-74.02427173047927, 40.65110251074392], [-74.02424021347534, 40.65113351631333], [-74.02550139045734, 40.65183985103419], [-74.02541604028593, 40.65193380685509], [-74.02417752973396, 40.65119177321219], [-74.02382347395066, 40.65135378491311], [-74.02365443141966, 40.65125618183632], [-74.02359400295185, 40.65131566325373], [-74.02305991047025, 40.65097430413475], [-74.02302464837881, 40.651002737893705], [-74.02302012112835, 40.651006388235146], [-74.02285141795983, 40.651169195161685], [-74.02536890150509, 40.65272143692734], [-74.02522662972656, 40.652829856959634], [-74.02272372795771, 40.651297322768784], [-74.02262161936135, 40.65131155867345], [-74.02154138319214, 40.650671622911226], [-74.02134461287908, 40.650851861184755], [-74.02128163421048, 40.65081734293942], [-74.0212465237341, 40.65083497256432], [-74.0212756487398, 40.6508615783714], [-74.02127560550258, 40.6508915349418], [-74.02139188210663, 40.650967757988425], [-74.0213595992163, 40.65099727861088], [-74.02124332438399, 40.65092597432825], [-74.02118521368588, 40.650977634170616], [-74.02112707783823, 40.650943212115436], [-74.02105230392205, 40.65109828292773], [-74.02134474360493, 40.651260470908426], [-74.0223373466683, 40.6519106053816], [-74.02225173199704, 40.65198160150678], [-74.02126416505618, 40.65132571010398], [-74.02098261225858, 40.65115682265769], [-74.02089862870834, 40.65124727521829], [-74.02380703903121, 40.65304472851146], [-74.02359434497556, 40.65321636896179], [-74.02353430165581, 40.653096426047235], [-74.0233317465082, 40.652942240183236], [-74.0230992368153, 40.652919438445565], [-74.0229417228344, 40.652879484508205], [-74.02288920245815, 40.6528223738857], [-74.02285167504505, 40.652742411591454], [-74.02195900090592, 40.65222278197071], [-74.02186900073939, 40.6522227991187], [-74.02171151888278, 40.65227423817063], [-74.02162153367613, 40.652319951290565], [-74.02123165456221, 40.652697024715714], [-74.02118670027885, 40.652839836895325], [-74.02320464727141, 40.654096116817925], [-74.02306219089226, 40.65423323629192], [-74.02210195162446, 40.65358795380762], [-74.02193694561598, 40.653576560979964], [-74.02187693558206, 40.65354801237162], [-74.02162946295488, 40.65364516521604], [-74.02128455998411, 40.65396510854651], [-74.02123961069387, 40.654130768888116], [-74.02114963484483, 40.65421075521156], [-74.02182479745494, 40.654656175785426], [-74.02164486415387, 40.65487898338249], [-74.02089469797602, 40.654433575413506], [-74.0206396915899, 40.65443933332952], [-74.02060218183952, 40.65441077951641], [-74.02060214479563, 40.654290824324356], [-74.0204746182018, 40.654216589227104], [-74.02024211371031, 40.65422234342679], [-74.02002459222682, 40.65416525925291], [-74.01961958177243, 40.65416532905225], [-74.01941702844238, 40.653999712567035], [-74.01925203374113, 40.65403401221275], [-74.01919203025426, 40.65402259849411], [-74.01924450754501, 40.653942620483654], [-74.01917698964483, 40.65388550950905], [-74.01904948304211, 40.65386839507754], [-74.01896698068877, 40.65386840871225], [-74.01892195088296, 40.65376559771667], [-74.01885441729473, 40.653651365473515], [-74.01877188306483, 40.65353713731394], [-74.01872686588689, 40.653474310524345], [-74.0186223830857, 40.653491349022964], [-74.01811897876473, 40.653885271105345], [-74.02025589052923, 40.65511826288116], [-74.02005182454292, 40.655335948874416], [-74.01787407223817, 40.65406150233079], [-74.01715319581021, 40.65477510436213], [-74.01746633453757, 40.65492680139873], [-74.01741891331396, 40.65499184067692], [-74.01980081651999, 40.656487210584636], [-74.01970596410222, 40.656581163751035], [-74.01726715936593, 40.65519418798651], [-74.01691617851066, 40.65548327495774], [-74.01665047959762, 40.65533879657211], [-74.01639437533773, 40.65562064285071], [-74.01910843803118, 40.657368884303494], [-74.01890923625238, 40.6575495644642], [-74.01860569460999, 40.65784587402284], [-74.01585370056823, 40.65621324055534], [-74.01543632269644, 40.65663239675966], [-74.017742327275, 40.658098916724896], [-74.01766644409511, 40.658185638771535], [-74.01786573109669, 40.65830844729852], [-74.01768549175459, 40.65845299276119], [-74.01748620434716, 40.65833740924325], [-74.01750516410576, 40.65827237429761], [-74.01504738221851, 40.6569503866934], [-74.01442130484548, 40.657608021951376], [-74.01745803087898, 40.6594501973867], [-74.01554165388997, 40.66075835547614], [-74.01400428392415, 40.6598119634401], [-74.01366273992599, 40.66002878063692], [-74.01246851441822, 40.65931636814533], [-74.01183862056543, 40.65891283727472], [-74.01142219567053, 40.659328694961694], [-74.011107504103, 40.65970627458062], [-74.01402247678024, 40.66152974744005], [-74.0131555493881, 40.66198159772593], [-74.012653588115, 40.66198165403757], [-74.01023484431792, 40.660461187053585], [-74.00859956968284, 40.65952092309827], [-74.00857262378977, 40.65954542776517], [-74.00778095960338, 40.660265328339584], [-74.00745858163884, 40.66055847620986], [-74.0103720129723, 40.66239888023248], [-74.01046331724483, 40.66264211979001], [-74.00955074142026, 40.6632676941304], [-74.00868371591297, 40.66337201284022], [-74.00616141136321, 40.66184506899056], [-74.00568594756898, 40.66227474036048], [-74.00553501251856, 40.6621838587469], [-74.00428358250115, 40.66144489410127], [-74.00418545737621, 40.661528306067375], [-74.00414624892208, 40.66156163568131], [-74.00424788611218, 40.66162323856307], [-74.00425415353574, 40.66162703820218], [-74.00426187385499, 40.66163171831446], [-74.00426191059573, 40.661631746290276], [-74.0042788006124, 40.66164197659999], [-74.0043958054613, 40.661712894508874], [-74.00451173232295, 40.66178315797599], [-74.00470164001472, 40.66189344110565], [-74.0047227489117, 40.66190549639521], [-74.00475221025457, 40.661922320188744], [-74.00476182862151, 40.66192817874856], [-74.00476870023624, 40.66193236482174], [-74.00495172594644, 40.66204178153432], [-74.00512364857573, 40.662144558308924], [-74.00512658939383, 40.662149228691625], [-74.0051267464898, 40.66214947846877], [-74.00512747767237, 40.66215126964575], [-74.00514101962978, 40.66218444025921], [-74.00526589544604, 40.66227016556554], [-74.00562039670646, 40.66249392634675], [-74.00578486636624, 40.662582529454504], [-74.00582714339052, 40.66259010896831], [-74.00585740199432, 40.662575170298965], [-74.00585795416234, 40.662574896865266], [-74.00585857414514, 40.66257525534364], [-74.00603815112555, 40.66267887134558], [-74.00620046233034, 40.66277252611265], [-74.00622698247925, 40.66278782786099], [-74.00622684103384, 40.66278796674929], [-74.00622501390112, 40.6627897570525], [-74.0060895834048, 40.662922401919324], [-74.00592904571474, 40.663102158135466], [-74.00592802062332, 40.663103306592866], [-74.00592360503022, 40.66310825073907], [-74.00545775556647, 40.662861366658554], [-74.0046014637945, 40.6623826685194], [-74.00422549172724, 40.66217696122081], [-74.00391334571943, 40.661977527307364], [-74.00389812942463, 40.661967805837776], [-74.00383146963537, 40.66192521575294], [-74.0038081547088, 40.66190794572821], [-74.0034880110065, 40.66224055612849], [-74.00389334189984, 40.662469174469116], [-74.00423930501728, 40.66265796345965], [-74.00436556756048, 40.662735676655466], [-74.00438799950994, 40.66271432134197], [-74.00446593631068, 40.66276195241858], [-74.00455498488702, 40.662816372912665], [-74.00455523060684, 40.66281611490984], [-74.00455892564403, 40.66281225944144], [-74.00455948505812, 40.66281167474653], [-74.00455994636002, 40.66281119107493], [-74.00456179244867, 40.66280926644005], [-74.00460158473534, 40.66276772310288], [-74.00458994364888, 40.66273373280542], [-74.00460574305602, 40.66271888517173], [-74.00454367850804, 40.66268014611805], [-74.00473298121766, 40.66249650399669], [-74.00523840365585, 40.662799535903716], [-74.00535812864251, 40.662869129731696], [-74.00515599719328, 40.663047414563835], [-74.00506160816835, 40.66299204120207], [-74.0049801050689, 40.663060997442216], [-74.00497993436086, 40.663061142361315], [-74.0049785042412, 40.66306235398569], [-74.00497796703885, 40.663062808009855], [-74.00497510679921, 40.66306522790805], [-74.00497157758612, 40.66306821324203], [-74.00519233775076, 40.663201553096535], [-74.0051928192948, 40.66320112051759], [-74.00519747590563, 40.663196929594314], [-74.00519907606538, 40.663195487943625], [-74.00520010976784, 40.66319455962422], [-74.00520105129699, 40.66319371088481], [-74.00524384971581, 40.66315519090431], [-74.00523151018425, 40.66311846966377], [-74.00524863034876, 40.663105620491386], [-74.00518629798088, 40.66306960144795], [-74.00538014413911, 40.66288392645275], [-74.00565964975691, 40.66305585356362], [-74.00599748682693, 40.66325861082859], [-74.00579501836201, 40.66343615129344], [-74.00569963747782, 40.66337678866095], [-74.00562415155503, 40.66344828896851], [-74.00562218007458, 40.66345015549427], [-74.00562053152801, 40.66345171626562], [-74.00561348681754, 40.66345839025475], [-74.0059083596534, 40.663638283630114], [-74.00591638354516, 40.663641193848974], [-74.00591778279261, 40.663641702056914], [-74.00591847405597, 40.66364195230822], [-74.00592011069159, 40.6636425462782], [-74.00630567322514, 40.66386341177632], [-74.00672835016054, 40.664118754923365], [-74.00743251429037, 40.66454244712888], [-74.00766379098464, 40.664683528366304], [-74.00772133506703, 40.66471863301977], [-74.00772545192977, 40.664742715391505], [-74.00767434594512, 40.66479110926571], [-74.00763877534806, 40.66479598033104], [-74.00730516126835, 40.66459759075257], [-74.00725742631731, 40.6646429595108], [-74.00726610280037, 40.664673048829385], [-74.00725283559042, 40.66469032364306], [-74.00731878365974, 40.66472986075765], [-74.0071253264673, 40.66491521847423], [-74.00652469600116, 40.664547227662894], [-74.00670685940631, 40.664362989856706], [-74.00679912430284, 40.66441830653351], [-74.00688215307311, 40.66434037294293], [-74.00662937859565, 40.664187930124434], [-74.00653579559389, 40.664257403636164], [-74.00664155766748, 40.66432021135603], [-74.00644979980798, 40.664506206799054], [-74.0058415763248, 40.66413802067458], [-74.00602870773491, 40.66395913653512], [-74.00611934710179, 40.664010566693165], [-74.0062030658537, 40.66393016821056], [-74.00604608518975, 40.663840005635535], [-74.00599262641798, 40.66389076321394], [-74.005921387034, 40.66384959857587], [-74.00589826436901, 40.66387569107484], [-74.00580999359873, 40.66387400202656], [-74.00575781045477, 40.66392275680161], [-74.00577974295794, 40.66398776309299], [-74.00574809232232, 40.66401194231734], [-74.00582327521114, 40.664054563853135], [-74.0057575379142, 40.664114392912], [-74.00589272459429, 40.664190318123154], [-74.00574908832141, 40.664326509889555], [-74.00574145525125, 40.66438182951614], [-74.00577046133881, 40.66439749528106], [-74.00574463033603, 40.664417639970004], [-74.00583499448777, 40.66446958031833], [-74.00580832976841, 40.66447828424341], [-74.00584924770605, 40.66450329861971], [-74.00582211847083, 40.66452876005955], [-74.00564140754479, 40.664531491895836], [-74.00565025959581, 40.66390033784382], [-74.00564925172907, 40.6638998515595], [-74.00564311080416, 40.663896890803734], [-74.00564213395874, 40.66389641993052], [-74.00556312932866, 40.663849042173354], [-74.00430053267483, 40.6630918619021], [-74.0042950800297, 40.66309558072489], [-74.00426622473461, 40.66311526298803], [-74.00426374481543, 40.66311378364021], [-74.00362862508561, 40.66273484753458], [-74.00340348595626, 40.66298572659833], [-74.00459238636759, 40.663735199832836], [-74.00432294897372, 40.66405231997527], [-74.00528132177348, 40.664657003524056], [-74.00486193792649, 40.66506482193817], [-74.00113519509667, 40.662901724055104], [-74.0009442426561, 40.66307639198391], [-74.00091723430236, 40.66305925325032], [-74.00083401108444, 40.66300644269171], [-74.00074785689004, 40.66308446140926], [-74.00049413361143, 40.66331422709332], [-74.00053780692811, 40.66333347770695], [-74.00023141373393, 40.6636279956228], [-74.000030241774, 40.66382136938716], [-74.00007728967003, 40.66384525383013], [-74.00009913667097, 40.66385634550629], [-73.99984301565564, 40.66410560044322], [-73.99971008269921, 40.66423016695416], [-73.99953152609916, 40.6643974859369], [-74.00292463516384, 40.66646231216332], [-74.00224030991373, 40.666793835145185], [-74.00203289059164, 40.6667872586681], [-74.00204667064895, 40.666769770350896], [-74.00161950021088, 40.66676627917447], [-74.00120611348305, 40.66701810720285], [-74.00146470932715, 40.6671695687076], [-74.00068774868765, 40.667545951898816], [-73.99998149339123, 40.66745027490869], [-73.99954541498883, 40.66790034997114], [-73.99902600998384, 40.66844297212693], [-73.99905372182691, 40.6688517992133], [-73.99894520195373, 40.66897377448506], [-73.99902633264175, 40.66900788840063], [-73.99905044103012, 40.668921687799546], [-73.99903627811007, 40.66891715219927], [-73.99904324792304, 40.668880220735076], [-73.99906368149783, 40.668873468189226], [-73.99909039181547, 40.66873147779713], [-73.99917543024283, 40.66831962518578], [-73.9991798832055, 40.66829663988748], [-73.99918971480948, 40.66829820450028], [-73.9992404910503, 40.66830628655996], [-73.99915332740866, 40.66866988112061], [-73.99907935327965, 40.669038271861666], [-73.99896950024663, 40.669162244242756], [-73.998655806916, 40.66951479554787], [-73.99880972556291, 40.669596180943536], [-73.99891160167903, 40.66964857863962], [-73.99890126537383, 40.66972727291531], [-73.99891684838755, 40.66980456321769], [-73.9988920731521, 40.66993843617495], [-73.99874991491649, 40.6705333732199], [-73.99871255131345, 40.67068973626811], [-73.9985582446838, 40.67133550166999], [-73.99871360475827, 40.67153537812376], [-73.99876941079444, 40.671602047052545], [-73.99868108199985, 40.67173915799989], [-73.9986188799999, 40.67183017199989], [-73.99845678099994, 40.671568042999894], [-73.99822509999998, 40.67120286799997], [-73.99798931599987, 40.670842081999915], [-73.99787130000004, 40.67066149999993], [-73.99771474299992, 40.67041544799982], [-73.99764875199986, 40.670311734999906], [-73.99757732099995, 40.67019948399994], [-73.99696465099986, 40.669234548999924], [-73.9969179149999, 40.6691534839999]]]}}, {\"id\": \"228\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 229, \"Shape_Leng\": 0.0425252900116, \"Shape_Area\": 9.47058230931e-05, \"zone\": \"Sutton Place/Turtle Bay North\", \"LocationID\": 229, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96162927534907, 40.75515238425915], [-73.96219381838414, 40.75450431515319], [-73.9622312516558, 40.75452213042181], [-73.96268703046536, 40.75400437328661], [-73.96289729202535, 40.753765517094244], [-73.9628689676921, 40.75374250915684], [-73.96316074049744, 40.753412207328125], [-73.96373572283164, 40.75278377577701], [-73.96426352137193, 40.75220690108016], [-73.96482214210387, 40.75159632579112], [-73.96485451699984, 40.751696759999916], [-73.96495708499998, 40.75174344599987], [-73.96505548699992, 40.75181258099991], [-73.96521537099983, 40.75180845999994], [-73.96518524199983, 40.75195230699995], [-73.96530480400006, 40.751931873999865], [-73.96532415299987, 40.751930897999934], [-73.96547092299994, 40.75193959699991], [-73.96688157199996, 40.75254455599991], [-73.96701854999988, 40.752600741999885], [-73.967067339, 40.75262067499993], [-73.96694827399996, 40.75278548799992], [-73.96677887099986, 40.75301878699991], [-73.96665543199994, 40.75318878499998], [-73.96661581800002, 40.75324342499992], [-73.96897803700003, 40.75425053499993], [-73.9712170449999, 40.75519332399994], [-73.97075919399998, 40.7558248509999], [-73.97031393599994, 40.75645301899989], [-73.96984859899989, 40.75707595799985], [-73.96939411299984, 40.75769952799991], [-73.96893510299992, 40.75833009199993], [-73.96847979599983, 40.75895012799986], [-73.96802436899998, 40.759578139999896], [-73.96792193499995, 40.75959308699989], [-73.96746215199992, 40.76022290599993], [-73.96699471899983, 40.76090538299993], [-73.96657146999985, 40.761481879999955], [-73.9665833839999, 40.76155093499988], [-73.96431353599988, 40.760599230999944], [-73.96355618899993, 40.76028081899991], [-73.96308073599987, 40.760083162999884], [-73.96196573699997, 40.759619460999865], [-73.95972180399991, 40.758663063999926], [-73.95903681099995, 40.75838378299988], [-73.95893448599985, 40.75833515499994], [-73.95877790811335, 40.75827092092435], [-73.95907003031202, 40.757939098020366], [-73.95917264248828, 40.757864780393994], [-73.95918103219987, 40.757856111052256], [-73.95965770876651, 40.75726014512322], [-73.95966400300107, 40.757211412186855], [-73.96003900990611, 40.75672535023881], [-73.96036455140813, 40.75626941113339], [-73.96065638180474, 40.75592314955017], [-73.96079709914974, 40.75580477274058], [-73.9607972298594, 40.755804655852046], [-73.96102790867172, 40.75561058457904], [-73.96162927534907, 40.75515238425915]]]}}, {\"id\": \"229\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 230, \"Shape_Leng\": 0.0310283096779, \"Shape_Area\": 5.60914463266e-05, \"zone\": \"Times Sq/Theatre District\", \"LocationID\": 230, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98362433299988, 40.75551634599986], [-73.98411754799999, 40.75484205299995], [-73.98640436699999, 40.755808028999866], [-73.98694873899993, 40.7560359019999], [-73.98841835499996, 40.75665498699989], [-73.98979085399996, 40.7572331319999], [-73.98929727499997, 40.7579066379999], [-73.98884253899993, 40.75853349899991], [-73.98839044599993, 40.75916418599996], [-73.98792640599991, 40.759790276999844], [-73.98747524599992, 40.76040478699989], [-73.98701514599995, 40.76103736699989], [-73.98656138499996, 40.7616601219999], [-73.98610406099989, 40.76228995299988], [-73.98564677799999, 40.76291487399988], [-73.98519152199998, 40.76354367499993], [-73.98473393799985, 40.76416910499991], [-73.98428230099987, 40.76479181499994], [-73.9824697079998, 40.764033463999944], [-73.98143351999995, 40.7635943599999], [-73.97860169399989, 40.762396071999945], [-73.97905731199987, 40.761770719999944], [-73.97951319199996, 40.7611485619999], [-73.97997188199987, 40.760524700999866], [-73.98042606399994, 40.75989908099994], [-73.980885746, 40.75927066899988], [-73.98134106999997, 40.75864529899995], [-73.98179404299992, 40.75802130299995], [-73.98225528700006, 40.75738736499995], [-73.98271205299996, 40.756771228999916], [-73.98316795999989, 40.75614022199991], [-73.98362433299988, 40.75551634599986]]]}}, {\"id\": \"230\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 231, \"Shape_Leng\": 0.0634201165587, \"Shape_Area\": 0.000166929752992, \"zone\": \"TriBeCa/Civic Center\", \"LocationID\": 231, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00931760399995, 40.72470155099989], [-74.00864380599998, 40.72422131699989], [-74.00857090999989, 40.72416935499988], [-74.00830216299993, 40.72397780899994], [-74.00807835299996, 40.72382686299987], [-74.00791205999997, 40.72369976699988], [-74.00762904199998, 40.72349770399995], [-74.00629841699995, 40.72254950399988], [-74.00591289899992, 40.7222529489999], [-74.0053098609999, 40.721824191999865], [-74.00522684899992, 40.72174422199989], [-74.00463516199997, 40.72135363299985], [-74.00392928399991, 40.72085443499993], [-74.00351186399998, 40.72055635199989], [-74.0032595509999, 40.72037709599994], [-74.00260001899987, 40.719903425999874], [-74.00188406299992, 40.71939757099991], [-74.00140055699995, 40.71905262099989], [-74.00127273399993, 40.71896142599997], [-74.00053195499999, 40.718430238999886], [-73.99995620900002, 40.71801709499987], [-74.00027443699997, 40.71761940199987], [-74.00081842499998, 40.71692970799994], [-74.00168306199991, 40.71574412299987], [-74.00211455799992, 40.715158909999936], [-74.002453932, 40.71478981999997], [-74.00272286199994, 40.71438970899991], [-74.00321983099997, 40.71403623699988], [-74.00360280399998, 40.71364542999988], [-74.004059007, 40.71313478999985], [-74.00421405399993, 40.712980742999896], [-74.00429113899997, 40.71290782599993], [-74.00470652299995, 40.712574014999866], [-74.00485768099996, 40.7124525409999], [-74.00498716699994, 40.712374856999894], [-74.00512712599992, 40.71230677399993], [-74.00527561999998, 40.71224943999993], [-74.00543047899994, 40.712203694999914], [-74.00558932999998, 40.7121700539999], [-74.00574044699991, 40.71208875699989], [-74.00589174999999, 40.71200715999994], [-74.00607752399999, 40.7119623529999], [-74.00698584099997, 40.711737498999845], [-74.00738921499999, 40.71164226999983], [-74.00746749399993, 40.71156690299989], [-74.00796787799999, 40.71144733199986], [-74.00834760999993, 40.71135854599987], [-74.00860085399992, 40.711392239999945], [-74.00883879499997, 40.711503958999906], [-74.01012167499995, 40.71208469599988], [-74.01166334799997, 40.71279805599988], [-74.01254378499996, 40.71317910399991], [-74.01359729999992, 40.7136233779999], [-74.01375388499997, 40.71369055499987], [-74.01359239999988, 40.714494316999904], [-74.01347524299992, 40.715110711999884], [-74.0133512149999, 40.71523331199992], [-74.01323384899992, 40.71565618799984], [-74.01314862399995, 40.716029349999886], [-74.0130514639999, 40.7164547709999], [-74.01286663099991, 40.717176100999886], [-74.01281566799992, 40.71735629799986], [-74.01244109199989, 40.719057672999874], [-74.0125546879999, 40.71906986299998], [-74.01270167899996, 40.71909177499986], [-74.01303068113208, 40.71913145590955], [-74.01301259080437, 40.71921511757715], [-74.01295868239187, 40.719473696443025], [-74.01296083955079, 40.71948393527159], [-74.0129144874633, 40.71970829322537], [-74.01294460912554, 40.7197131975405], [-74.01294583743953, 40.71969680842214], [-74.01302616790717, 40.71970920840729], [-74.01302432551543, 40.71972559759596], [-74.01344694115667, 40.71978202079137], [-74.01338906903435, 40.72004260727319], [-74.01396546195676, 40.72011613349159], [-74.01452415856508, 40.72018739744569], [-74.01450706139607, 40.720265906177275], [-74.01457231438083, 40.72027413774219], [-74.01451968588725, 40.72051578688329], [-74.01452586453654, 40.72051656945922], [-74.01452278990024, 40.72053205032778], [-74.01418581466784, 40.720488012020716], [-74.01414732641814, 40.720481589088564], [-74.01415861254227, 40.720431590837414], [-74.01409095949832, 40.720422747859125], [-74.01407954419398, 40.72047331152246], [-74.01400416755592, 40.720468520737754], [-74.01396301313085, 40.720464446458884], [-74.01396609722485, 40.72041363453813], [-74.01386714357216, 40.7204075723125], [-74.01385921064893, 40.72044993375783], [-74.013681739107, 40.720425901487225], [-74.0136874483683, 40.72039057527118], [-74.01364192748738, 40.72038336629997], [-74.01363580332405, 40.720419550132604], [-74.01354841723077, 40.72040685437942], [-74.01355378315908, 40.720368726324615], [-74.01350724376601, 40.72036283284043], [-74.01350069470507, 40.720401861823426], [-74.01296558526825, 40.720328675587126], [-74.01286909071982, 40.720833752211725], [-74.01323937337015, 40.72088798672807], [-74.01315895020865, 40.72125326078352], [-74.01313846153448, 40.7212513181375], [-74.01308347695151, 40.72152877453273], [-74.01308010145627, 40.721545807195234], [-74.01307805009768, 40.72154553033894], [-74.01304353513854, 40.721544723927295], [-74.01300955803927, 40.72156048651206], [-74.01253875807251, 40.721503176576356], [-74.0124917103152, 40.72177975782742], [-74.01237071715381, 40.722387580578996], [-74.01223080840143, 40.72305625707265], [-74.01209463829593, 40.72370705248105], [-74.01196432024553, 40.72432986411987], [-74.01159122599994, 40.724297603999936], [-74.01143933, 40.72428446899995], [-74.01135428299995, 40.72427718299988], [-74.011176661, 40.7251694249999], [-74.01103681999997, 40.72572537699997], [-74.01102457999994, 40.72579386199991], [-74.01092277299992, 40.72579415099991], [-74.0108123829999, 40.72578980299994], [-74.01004078699994, 40.72520616299992], [-74.00931760399995, 40.72470155099989]]]}}, {\"id\": \"231\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 232, \"Shape_Leng\": 0.0614709085331, \"Shape_Area\": 0.000216049973456, \"zone\": \"Two Bridges/Seward Park\", \"LocationID\": 232, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97635525199995, 40.71913288399983], [-73.97551012699985, 40.71895861299989], [-73.97545167499995, 40.71894655899996], [-73.97509520199993, 40.71888074599987], [-73.97497644799995, 40.71887065199991], [-73.97491298899988, 40.71886244299986], [-73.97485919899994, 40.71885547999992], [-73.97474239999998, 40.71884079399995], [-73.97468687499985, 40.718841701999914], [-73.97464720799998, 40.718842349999896], [-73.97347975524508, 40.7188614291129], [-73.9735436873773, 40.71866262945056], [-73.97359492157358, 40.71851801880885], [-73.97365255284436, 40.71835741732556], [-73.97372208271211, 40.718155790530695], [-73.97380258807485, 40.71794026346791], [-73.97390596973892, 40.717639219737386], [-73.97399195824943, 40.7174153452363], [-73.97408161583928, 40.71715532522245], [-73.97418131138618, 40.71692102577266], [-73.97422340312282, 40.71678614725209], [-73.97425084643322, 40.716698543367634], [-73.97445315383058, 40.71615168116171], [-73.97449720984376, 40.716042576306286], [-73.97463933366734, 40.71570768976437], [-73.97473312789973, 40.71548298862199], [-73.97481840216984, 40.715286372090574], [-73.97491787352749, 40.715065993886455], [-73.97500029316984, 40.71487154606126], [-73.97513489452726, 40.71460810394021], [-73.97515865241793, 40.71456160677657], [-73.97517484286146, 40.71452991316868], [-73.97518850986295, 40.714503160723865], [-73.97521319223426, 40.71444996434558], [-73.97523888264773, 40.71439457893641], [-73.97523893090127, 40.714394480105916], [-73.97580133088594, 40.713182224269396], [-73.97579346365995, 40.71317403724816], [-73.97596844519683, 40.712789573390786], [-73.97604981455497, 40.71260419377782], [-73.97612368838796, 40.7124603253984], [-73.97616531238327, 40.71242019791075], [-73.97631127867191, 40.71211864731018], [-73.97642872514517, 40.71190903250363], [-73.97650540115386, 40.711798789452466], [-73.9765500881422, 40.71173453872828], [-73.97671221539647, 40.71153576517845], [-73.97681883540065, 40.711419567021146], [-73.97699076574622, 40.71125666023054], [-73.97705659401724, 40.71119762696939], [-73.9772221356255, 40.71106873122833], [-73.97752668432635, 40.710843072733134], [-73.97772119493052, 40.710710074660696], [-73.97782091638247, 40.71064544490706], [-73.9778994961945, 40.71059986620245], [-73.9779967479038, 40.710561017391605], [-73.978079749066, 40.71053225107983], [-73.9781819069428, 40.71050686551762], [-73.97825557749202, 40.71049790955784], [-73.97837148165253, 40.710489707491575], [-73.97861560123994, 40.71047378191085], [-73.97877613342088, 40.71046331504719], [-73.97918894377, 40.71043492338827], [-73.97920726434594, 40.71043366689569], [-73.9792901523712, 40.71042814456692], [-73.98003642292828, 40.71037840126663], [-73.98028344332175, 40.710361815108435], [-73.98029934149871, 40.71034267301001], [-73.98031391320167, 40.71033259071117], [-73.98033113415963, 40.710323524412026], [-73.98033016246782, 40.71028673011878], [-73.98050948312414, 40.71027356987713], [-73.98127684210985, 40.710217251191], [-73.98123734462091, 40.709823343318455], [-73.98306019477401, 40.70964791174809], [-73.98364234472628, 40.70959187924471], [-73.98449647076768, 40.70950966393942], [-73.98655035339979, 40.709311944641996], [-73.9881649172667, 40.70915678150646], [-73.98817340967861, 40.70914751830125], [-73.98839064023261, 40.70912694916261], [-73.98844653437375, 40.709133820546924], [-73.98847967268772, 40.70915584730847], [-73.98849727173877, 40.709176305795744], [-73.9886365807184, 40.70991206151627], [-73.98864533493678, 40.709916934401114], [-73.98866750580949, 40.70992003483071], [-73.98869610271326, 40.70992844984459], [-73.98871771111234, 40.70994709051892], [-73.9887368170753, 40.709970741845105], [-73.98879717785798, 40.709966607472246], [-73.98889845980878, 40.70995473576508], [-73.98935765696673, 40.709900986707815], [-73.98946067019428, 40.709889287051254], [-73.98957449608706, 40.7098740816065], [-73.98974515155741, 40.70984876165558], [-73.99015825373382, 40.70979037266373], [-73.99051420376685, 40.709733884621315], [-73.99078913018562, 40.709689506086086], [-73.9910678346299, 40.70965723034518], [-73.9913510923706, 40.709620332703324], [-73.99170260292519, 40.70956526295599], [-73.99176779580131, 40.709555056026225], [-73.99190824271031, 40.70953292664334], [-73.99192047499996, 40.70955429399988], [-73.9919434659999, 40.70958905299992], [-73.99197509100001, 40.70964553499987], [-73.99200595899987, 40.70969647899992], [-73.99204746299989, 40.70977574499992], [-73.9919070979999, 40.70980199399986], [-73.99183817499984, 40.709810551999965], [-73.99177662699996, 40.7098190399999], [-73.99191690599989, 40.71059750599987], [-73.99197674899987, 40.71091126599985], [-73.99200166199995, 40.711015522999894], [-73.99202215999982, 40.71110129999989], [-73.99213550299994, 40.71157561499989], [-73.99215484499997, 40.71165655599985], [-73.99216806999992, 40.71171189899992], [-73.99218322399993, 40.71177531099993], [-73.99220577799984, 40.711960634999905], [-73.99221855999997, 40.71206565499991], [-73.99224319399995, 40.71226805499987], [-73.99225369299995, 40.71235432599987], [-73.99226747499985, 40.712467556999904], [-73.99239157000001, 40.71319080999998], [-73.9924556359999, 40.713762881999834], [-73.99246827699984, 40.71385682499988], [-73.99256242199989, 40.71438807699996], [-73.99213486499997, 40.7144198019999], [-73.99122551100002, 40.714486643999926], [-73.99075515299991, 40.7145540729999], [-73.99022012699989, 40.71440493999988], [-73.99022750999988, 40.714491668999926], [-73.99022155699988, 40.71457875599985], [-73.9902023599999, 40.71466442999988], [-73.98975418199988, 40.71555234799997], [-73.98915463799979, 40.716708646999905], [-73.98836626299988, 40.71644570199991], [-73.98795181500003, 40.717265627999915], [-73.98791645999997, 40.71733557399992], [-73.98787935799997, 40.7174089729999], [-73.98751208099989, 40.71807218599988], [-73.98743405799996, 40.71821307999992], [-73.98739393199992, 40.718303522999896], [-73.98736015299995, 40.71836979699993], [-73.98675136699994, 40.71959284599997], [-73.9859109239999, 40.7193389729999], [-73.985073423, 40.71908329399989], [-73.98447731799988, 40.720234238999886], [-73.98386422799992, 40.72139637799989], [-73.98382387299982, 40.72147287199987], [-73.98300308699991, 40.721222680999894], [-73.98216615299982, 40.72096885499996], [-73.98130824199984, 40.72070878599992], [-73.97966021699983, 40.72021282999992], [-73.97875234999992, 40.71993370699994], [-73.978031227, 40.71963834799987], [-73.97722379199992, 40.71930977499991], [-73.97674495699998, 40.719215233999925], [-73.97648182399995, 40.719159627999915], [-73.97635525199995, 40.71913288399983]]]}}, {\"id\": \"232\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 233, \"Shape_Leng\": 0.048035918294, \"Shape_Area\": 0.000116188094682, \"zone\": \"UN/Turtle Bay South\", \"LocationID\": 233, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.96547092299994, 40.75193959699991], [-73.96532415299987, 40.751930897999934], [-73.96530480400006, 40.751931873999865], [-73.96518524199983, 40.75195230699995], [-73.96521537099989, 40.75180846099992], [-73.96505548699992, 40.75181258099991], [-73.96495708499998, 40.75174344599987], [-73.96485451699984, 40.751696759999916], [-73.96482214210387, 40.75159632579112], [-73.96532405586481, 40.75104771827205], [-73.96791272994389, 40.747970935642165], [-73.96811893399283, 40.74772582974992], [-73.96817376101056, 40.74766675374168], [-73.96823574753452, 40.74759121839251], [-73.96836519324144, 40.74743349462013], [-73.96823171715793, 40.74736847648928], [-73.96829622113239, 40.74729565136708], [-73.96829811451514, 40.747283395002775], [-73.96830191005883, 40.74727329974719], [-73.9683161371576, 40.747274024359285], [-73.96832610433407, 40.74725203459879], [-73.96834696591314, 40.74724121830997], [-73.96836118844753, 40.747218145486315], [-73.96840008314159, 40.74716695396425], [-73.96843139158797, 40.747129456620236], [-73.96846363690707, 40.74709412848565], [-73.96847691942916, 40.74707250130821], [-73.9685006403586, 40.74704438450718], [-73.9685148595502, 40.747032843998056], [-73.9685347900076, 40.74700039777951], [-73.968544278662, 40.7469881423127], [-73.96858885224545, 40.746943443580705], [-73.96859169569538, 40.746933347552826], [-73.96860023472239, 40.74692109182235], [-73.96860971997499, 40.74691460360568], [-73.96861730876927, 40.746899455777815], [-73.96863912464634, 40.74687927464661], [-73.96866379796626, 40.746846820314914], [-73.96867043421346, 40.74683600997994], [-73.9686922519053, 40.74680860795762], [-73.96870458089755, 40.746800682796184], [-73.96871076544421, 40.746788263917146], [-73.96875271240361, 40.74674271521673], [-73.96894301109302, 40.7465183356965], [-73.96896670537485, 40.746530500799395], [-73.96897366849464, 40.7465290073365], [-73.96898899231675, 40.74650579199741], [-73.96899981595465, 40.74650193477237], [-73.96900302810072, 40.7464908088819], [-73.96901641754057, 40.746481152086254], [-73.96901961115914, 40.746472438902586], [-73.96907193333254, 40.74641298300629], [-73.96913096118837, 40.74634071530666], [-73.96921135921266, 40.74624451530387], [-73.96923432172326, 40.7462222840115], [-73.9692470758419, 40.746207781993604], [-73.96925028881681, 40.74619617144262], [-73.96932591437383, 40.7461040877104], [-73.96933228026558, 40.74610167233752], [-73.96933924459174, 40.74609265608835], [-73.96938290644789, 40.746035949024964], [-73.96940022327857, 40.74602084326546], [-73.96940086652106, 40.74601402713001], [-73.9694175339955, 40.745996966331965], [-73.96943292389848, 40.745976989231316], [-73.96946210038308, 40.74594457763359], [-73.96952046125196, 40.74586904111652], [-73.96955059658639, 40.74583882769356], [-73.96959740261153, 40.745782784111796], [-73.96962370583401, 40.745746716794365], [-73.96963204180261, 40.74574330564597], [-73.96969487777564, 40.74566679689091], [-73.96969678845916, 40.74565700029289], [-73.96987561403044, 40.74544478913759], [-73.96990575265262, 40.74545820514511], [-73.96991227975748, 40.74545175943624], [-73.96993139614943, 40.74546300191986], [-73.96992704959064, 40.745467792035086], [-73.9699737702627, 40.7454957474306], [-73.970047589049, 40.74540286129934], [-73.97009123175702, 40.745351129150954], [-73.97012962576954, 40.74530536659431], [-73.97021602288638, 40.745200579032776], [-73.97033931161239, 40.7450496320225], [-73.97032571321418, 40.745036949464414], [-73.97032571931098, 40.74502326140019], [-73.97037811940483, 40.74495230608582], [-73.97044255489001, 40.74487037600953], [-73.97046063703607, 40.744864743390664], [-73.97047240576431, 40.74486624174768], [-73.97050560032663, 40.74482376425926], [-73.9705687722909, 40.744731865306065], [-73.97062381510642, 40.74465642621006], [-73.97078834776636, 40.74440621769034], [-73.97085704434333, 40.7444326373972], [-73.97087688233859, 40.744445573430944], [-73.97088963376707, 40.74444772970183], [-73.97089742067703, 40.7444547367797], [-73.9709094613216, 40.74445635209411], [-73.97091724845431, 40.74446336034336], [-73.97092858032363, 40.744463363204204], [-73.97094062960414, 40.74447144402548], [-73.9710164805907, 40.74435508952358], [-73.9711250301855, 40.74418974894763], [-73.97111076626928, 40.7441839451093], [-73.97109154986752, 40.744182247937445], [-73.97107528455967, 40.7441760664907], [-73.97106420642824, 40.744167068050885], [-73.97092895951977, 40.74411024678126], [-73.97091343992327, 40.74410237365222], [-73.97090457107981, 40.74410349618337], [-73.97089791952085, 40.74409786612331], [-73.970879439425, 40.74409281003312], [-73.97086171010008, 40.7440759385033], [-73.97084544343367, 40.744076493254404], [-73.97083878967527, 40.74407087307438], [-73.97081588183757, 40.7440652393951], [-73.97079622300242, 40.744048880666504], [-73.97078276128667, 40.74404724738941], [-73.97077757716099, 40.74404662522193], [-73.97075833953649, 40.74403919088488], [-73.97075084955125, 40.74402980466474], [-73.97071823494029, 40.744016927848136], [-73.97070578720823, 40.74400256198671], [-73.9707031746558, 40.74399049541112], [-73.9707425727204, 40.74394832466002], [-73.97075061381996, 40.74394863411313], [-73.97076123524698, 40.74394210814419], [-73.9707920969469, 40.74395601043508], [-73.97079892989288, 40.74395279750638], [-73.97083587404146, 40.743969672238116], [-73.97083595090118, 40.74396970165843], [-73.97087357851332, 40.74398767476939], [-73.97095964992798, 40.74402569698705], [-73.97096300193458, 40.74402367277478], [-73.97117346687047, 40.74410753551064], [-73.9711819357606, 40.74410323270784], [-73.97120616502995, 40.744066029846586], [-73.97127362462288, 40.74396474842357], [-73.97135752941857, 40.74383845535952], [-73.97123423981489, 40.743784646190235], [-73.97120546301963, 40.7437715103013], [-73.97128101831673, 40.743651942051315], [-73.97124127770057, 40.74363774014748], [-73.97120787330633, 40.74369879478972], [-73.97119990329709, 40.743706321092134], [-73.97118780171984, 40.74370736326176], [-73.97109822176314, 40.743678496815434], [-73.97108533439558, 40.74366405834519], [-73.97123301520192, 40.743414663960074], [-73.97124584132808, 40.74340203417444], [-73.97125636771857, 40.74340155868206], [-73.97126710092137, 40.74338988243382], [-73.97130663606045, 40.74341340326184], [-73.97135416843827, 40.74342493319505], [-73.97135016335174, 40.743431659048724], [-73.97135725327772, 40.743436613391225], [-73.97136080173868, 40.743444277092706], [-73.97131062520039, 40.743524023190886], [-73.97135563409158, 40.743539649851975], [-73.97149421270423, 40.74333467446508], [-73.97163772240418, 40.743117200984635], [-73.97167921200406, 40.74305446402989], [-73.97191659199993, 40.74313739399983], [-73.97198880299989, 40.74316289599994], [-73.97206165299998, 40.74318802999993], [-73.97210897599994, 40.74320549099992], [-73.97223028499984, 40.743249026999884], [-73.97350835000003, 40.7437820339999], [-73.97302670299986, 40.744450040999915], [-73.97257598299979, 40.74506785499992], [-73.97212958899988, 40.745680901999954], [-73.97203331099985, 40.745804967999945], [-73.97194428900002, 40.74593187199988], [-73.97165740599984, 40.74630369399994], [-73.97121971999988, 40.74692154099996], [-73.9727379199999, 40.74755798999993], [-73.97359187599997, 40.74791866299991], [-73.97497000699997, 40.748501848999865], [-73.9758346669999, 40.74886314599996], [-73.97538295899986, 40.74948305799992], [-73.97493471299985, 40.7500996279999], [-73.9744494109998, 40.75076644499991], [-73.97395914699999, 40.751441021999895], [-73.97350255899984, 40.75206493199988], [-73.97304502599985, 40.7526916219999], [-73.97258874999983, 40.753317887999856], [-73.97213617399994, 40.7539432349999], [-73.97167699199998, 40.754569630999946], [-73.9712170449999, 40.75519332399994], [-73.96897803700003, 40.75425053499993], [-73.96661581800002, 40.75324342499992], [-73.96665543199994, 40.75318878499998], [-73.96677887099995, 40.75301878799995], [-73.96694827399996, 40.75278548799992], [-73.96706733899993, 40.75262067599991], [-73.96701854999988, 40.752600741999885], [-73.96688157199996, 40.75254455599991], [-73.96547092299994, 40.75193959699991]]], [[[-73.96421230395673, 40.74660431847665], [-73.9642736475242, 40.74654184505502], [-73.96433729322388, 40.74654572659317], [-73.96437765404302, 40.74649508803884], [-73.96444522752618, 40.74641042576318], [-73.96458318426942, 40.746454400673315], [-73.96458702399427, 40.746515898544594], [-73.96454518192593, 40.746564150472935], [-73.96447543657693, 40.74664083319782], [-73.96437443772194, 40.74670453064306], [-73.96433758671738, 40.74673484818137], [-73.96427256202216, 40.746751308084086], [-73.96421616682363, 40.74683700083098], [-73.96415980288437, 40.74686554735954], [-73.96412514486158, 40.74684576060531], [-73.96413382933486, 40.7468127961654], [-73.96421192462753, 40.746666676802946], [-73.96421230395673, 40.74660431847665]]]]}}, {\"id\": \"233\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 234, \"Shape_Leng\": 0.0360721994984, \"Shape_Area\": 7.31054382894e-05, \"zone\": \"Union Sq\", \"LocationID\": 234, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.98996936399989, 40.73490456699994], [-73.98990295999991, 40.7344347899999], [-73.9907539679999, 40.73483915499987], [-73.99119899299986, 40.73501236899993], [-73.99171498799984, 40.73521530099993], [-73.99361688399992, 40.73600449199991], [-73.99683993899998, 40.73736088899995], [-73.9963368949999, 40.7380437469999], [-73.99587409499989, 40.73868594599991], [-73.99544664199995, 40.7392721329999], [-73.99501798599992, 40.73986001099995], [-73.99459264899988, 40.740441980999925], [-73.99416492399986, 40.7410309369999], [-73.99397071699987, 40.74129843899991], [-73.99373928799994, 40.74161720799987], [-73.99329166800001, 40.74223330099994], [-73.99280340900002, 40.74291001999997], [-73.99231888499999, 40.74356832199989], [-73.991871024, 40.744185570999896], [-73.99141858599994, 40.74480268199988], [-73.99096832799982, 40.745420890999874], [-73.99051761299985, 40.746038637999895], [-73.98872905499988, 40.74528637399991], [-73.98729805499983, 40.74468149899992], [-73.98569571299986, 40.74400764199986], [-73.98413729099997, 40.74334959099992], [-73.98407485299995, 40.74332471799989], [-73.98452636699996, 40.7427070419999], [-73.98497775500003, 40.74208895799992], [-73.98542767799994, 40.741470348999876], [-73.98587913199991, 40.74085299999997], [-73.98636141699991, 40.74018480099992], [-73.9868491859999, 40.73952182199985], [-73.98703592799987, 40.739266378999886], [-73.98730154899981, 40.73890302599993], [-73.98774074499983, 40.738312187999945], [-73.98817910299996, 40.73773074899986], [-73.98859298399994, 40.73714256099993], [-73.98901559399987, 40.73656015399992], [-73.98920953899987, 40.736307120999854], [-73.98945472099987, 40.7359872469999], [-73.989874214, 40.73540825799992], [-73.98991689999981, 40.73534934199992], [-73.98994254999981, 40.73527697099995], [-73.98996152699993, 40.73520341999988], [-73.98997373899982, 40.735129042999894], [-73.9899791279999, 40.73505420099983], [-73.98997766699992, 40.73497925499991], [-73.98996936399989, 40.73490456699994]]]}}, {\"id\": \"234\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 235, \"Shape_Leng\": 0.0761668426519, \"Shape_Area\": 0.000212844547337, \"zone\": \"University Heights/Morris Heights\", \"LocationID\": 235, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.90947862999988, 40.86180812899987], [-73.90965329599997, 40.86160687999992], [-73.9098448269999, 40.8614133699999], [-73.91005219099996, 40.86122888599986], [-73.91027422499997, 40.861054534999894], [-73.91050960899994, 40.86089126999988], [-73.91075688299993, 40.860739954999865], [-73.91101447199998, 40.86060126199995], [-73.91114937399996, 40.86052367099993], [-73.9112955739999, 40.860457380999875], [-73.91145048700004, 40.86040372399991], [-73.91161125299992, 40.86036354999985], [-73.91187557299992, 40.860297808999846], [-73.91213471699986, 40.86021749399987], [-73.91238663199995, 40.86012299899993], [-73.91262932399985, 40.86001494399989], [-73.91286092999991, 40.859894146999856], [-73.91307976699996, 40.85976164299991], [-73.91328437000003, 40.859618600999944], [-73.91088488399994, 40.85877618299992], [-73.91072104299988, 40.8587062319999], [-73.91056799299989, 40.858620592999955], [-73.91042952699988, 40.8585208569999], [-73.91030880899984, 40.85840932699993], [-73.91020818099983, 40.858288788999886], [-73.90980269899988, 40.857607548999916], [-73.90976165899988, 40.857182611999896], [-73.90956720099997, 40.85759162399991], [-73.90777320599983, 40.85960753799984], [-73.90725786199998, 40.859353437999836], [-73.90707003899985, 40.85936184899988], [-73.90679734899983, 40.85937358899988], [-73.90572262499991, 40.85899298899991], [-73.90486847199993, 40.8586932249999], [-73.90526389299997, 40.85804717799987], [-73.90565953499988, 40.85739873399997], [-73.90468371399987, 40.85706030199991], [-73.90508614099994, 40.856410830999955], [-73.90522169999987, 40.856203594999926], [-73.90557007299988, 40.855787321999884], [-73.9073328969999, 40.85381641499987], [-73.9084173709999, 40.85264382199996], [-73.90921545599986, 40.851800944999894], [-73.910581699, 40.85018159499988], [-73.9106946369999, 40.85001909799993], [-73.91112232299982, 40.84941809799992], [-73.91167991199987, 40.84863951899991], [-73.91197263799982, 40.84822959499992], [-73.91236345699993, 40.84768333899996], [-73.91305693099993, 40.846708837999905], [-73.91315074799996, 40.846577003999876], [-73.91346470799996, 40.84614733699992], [-73.91360819899985, 40.84594143899991], [-73.9137516889998, 40.84573553999994], [-73.91397534899991, 40.84542943099992], [-73.91412710499995, 40.84522891399991], [-73.91416184199993, 40.845179187999925], [-73.91479384899988, 40.845164996999905], [-73.9156148529999, 40.845146555999875], [-73.91689747599989, 40.84511773399991], [-73.91729224999992, 40.84510993999991], [-73.91838402199984, 40.845055012999914], [-73.91877179199986, 40.845034466999955], [-73.91930035199984, 40.844993234999876], [-73.92043297999987, 40.84494079399986], [-73.92150714599991, 40.844873573999976], [-73.92246996799993, 40.8448133119999], [-73.92262808499991, 40.84480091099986], [-73.92273460999992, 40.844796838999876], [-73.9235400199999, 40.844758943999885], [-73.92387743499995, 40.84474050799987], [-73.92478318799985, 40.84475266399989], [-73.92511859399998, 40.8447796449999], [-73.92526006699997, 40.844793802999966], [-73.92596341899986, 40.84488974399987], [-73.92661872999987, 40.84503564699987], [-73.92668910099992, 40.845051611999914], [-73.92676979799985, 40.84507093399992], [-73.92696149299991, 40.84511449499996], [-73.92706435800001, 40.845139449999884], [-73.92740093599986, 40.84522543499989], [-73.927784594, 40.8453197489999], [-73.92796538299997, 40.845362455999904], [-73.92808209386837, 40.84539159691672], [-73.92804748400906, 40.84546459597727], [-73.927868698054, 40.845597721099985], [-73.9276707332328, 40.84603348224206], [-73.92763424713485, 40.846018784098284], [-73.92759209775825, 40.84605233675773], [-73.92754828423438, 40.84608721478045], [-73.92746013067759, 40.846194776164275], [-73.9274214151804, 40.846255083713544], [-73.92739125975535, 40.84634148384076], [-73.92736757047359, 40.84640506094841], [-73.92734184476856, 40.846463641219], [-73.92730194499312, 40.84650686718802], [-73.9272779808177, 40.84654803605297], [-73.9272247957283, 40.84658965715395], [-73.9271551981214, 40.84665082984285], [-73.92701961357912, 40.84681765789867], [-73.9269354594415, 40.84692414408519], [-73.92687934781303, 40.847039507037174], [-73.92688167685719, 40.847103401772344], [-73.92685912316018, 40.84719293598746], [-73.92685127601227, 40.84722409102787], [-73.92683023208399, 40.84728976009429], [-73.92677400524201, 40.84732556283115], [-73.92675777018512, 40.84733590131359], [-73.92669933483715, 40.84735187102644], [-73.92662687420078, 40.84739268675017], [-73.92656608810381, 40.84750982309605], [-73.92651699719853, 40.84756484067268], [-73.92645855280897, 40.84764825440047], [-73.92641413285313, 40.8477316702337], [-73.92630425882031, 40.84787010122042], [-73.92619439242105, 40.84796061201434], [-73.92611024113897, 40.848020949949884], [-73.92603310186037, 40.848072415149296], [-73.92597932978448, 40.84816292881563], [-73.9259395786478, 40.848271193179905], [-73.92586477100132, 40.84836170484416], [-73.92577127016631, 40.84842204291624], [-73.92521955514904, 40.84908224686876], [-73.92465379915153, 40.84978504325649], [-73.92445273224833, 40.85009207721127], [-73.9242240509533, 40.85056887702687], [-73.92408865809536, 40.85073735008726], [-73.92384778922904, 40.851182608857954], [-73.92394789467637, 40.851221784808004], [-73.9235176926963, 40.85183581998546], [-73.92331810320631, 40.85212978798092], [-73.92256178923255, 40.853188911763624], [-73.92174813841936, 40.85421471478316], [-73.92154374459189, 40.85410995152729], [-73.92134207683873, 40.854317435657684], [-73.92148991153954, 40.85449021828858], [-73.92151543705522, 40.8545100638913], [-73.92080070289325, 40.85538081439453], [-73.92037235451427, 40.85517296513669], [-73.92030027213424, 40.85524637134155], [-73.92056655954352, 40.855513898313646], [-73.92055014989616, 40.85561817968149], [-73.92052653132845, 40.855686668401795], [-73.92044141417742, 40.85574613630999], [-73.92035024788355, 40.85578536382873], [-73.9202939290938, 40.85584511501975], [-73.92030620609955, 40.855927119811], [-73.92002002121407, 40.8562707210165], [-73.91956620889232, 40.85684018384378], [-73.91902369112604, 40.85739653820692], [-73.91829123398816, 40.85815501495544], [-73.91811187549494, 40.858313729633515], [-73.91776041251106, 40.85870809795827], [-73.91720664647033, 40.85892892273455], [-73.91691308641663, 40.859270212209154], [-73.91580528008534, 40.86046973585608], [-73.9157621314479, 40.86064463678988], [-73.91487470201923, 40.8616916999862], [-73.91480934325483, 40.86171418964575], [-73.91472330855278, 40.86185318238911], [-73.91481817352647, 40.86189908899147], [-73.91477109687513, 40.86194630832581], [-73.91474323842307, 40.861933589990585], [-73.91470028504796, 40.861953112771886], [-73.91467162914577, 40.8619813378025], [-73.91463437515958, 40.862020419920114], [-73.91461143724548, 40.86205299399862], [-73.91461141219102, 40.8620725494817], [-73.91462116377565, 40.86209669086735], [-73.91429082559196, 40.86242801792628], [-73.91422824049913, 40.862490790359615], [-73.91343151699985, 40.86213906699985], [-73.91335994700002, 40.86211267299994], [-73.91329746900003, 40.862089627999914], [-73.91295585399997, 40.86191313599993], [-73.91278962899986, 40.86187247099988], [-73.91273784299996, 40.86186388599992], [-73.91266095199984, 40.861851634999894], [-73.91245929699984, 40.861820432999885], [-73.91208148799987, 40.8617680069999], [-73.91194992599989, 40.86175014099994], [-73.91181229099992, 40.861732386999925], [-73.91075595799987, 40.861651483999864], [-73.909866682, 40.86231006799993], [-73.90972193999986, 40.86240666799991], [-73.90956472299985, 40.862493010999884], [-73.90939703499993, 40.8625677719999], [-73.90922113399998, 40.862629947999906], [-73.9090395089999, 40.862678863999896], [-73.90905267599997, 40.86253910899997], [-73.90905268799997, 40.86253892899989], [-73.90905273599994, 40.86253873899995], [-73.90908816499991, 40.862399911999965], [-73.90908822399999, 40.86239969599992], [-73.90908830799995, 40.86239949799988], [-73.90914581699987, 40.86226429799993], [-73.90914584099987, 40.86226424499994], [-73.90914587699984, 40.86226418999998], [-73.90922418100003, 40.862135652999946], [-73.909224264, 40.862135517999874], [-73.90922437099982, 40.862135391999885], [-73.90932135399996, 40.862016161999875], [-73.90947846399986, 40.861808334999886], [-73.90947854799991, 40.86180822799987], [-73.90947862999988, 40.86180812899987]]]}}, {\"id\": \"235\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 236, \"Shape_Leng\": 0.0442519223099, \"Shape_Area\": 0.000102864345211, \"zone\": \"Upper East Side North\", \"LocationID\": 236, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95779380499984, 40.773599896999926], [-73.95825534899998, 40.772965336999896], [-73.95985681799993, 40.77364276799989], [-73.96136792699977, 40.77427876099991], [-73.9614758119999, 40.7743240109999], [-73.96158453899987, 40.774368783999904], [-73.96308760299976, 40.77500284199987], [-73.96468735799992, 40.77567708199994], [-73.96422820200003, 40.77631030899995], [-73.9637612389999, 40.77695096899986], [-73.96372602699994, 40.77700571799989], [-73.96322991899993, 40.77767704699989], [-73.9627671559999, 40.77831304099987], [-73.96230361500002, 40.77894704099985], [-73.96183836899999, 40.779584191999874], [-73.961368727, 40.78022974899988], [-73.96091849500006, 40.78085062899991], [-73.96041638899996, 40.78153163299989], [-73.959920943, 40.782212944999856], [-73.95946905999989, 40.78283332099994], [-73.95900363199988, 40.783467289999855], [-73.9585581369999, 40.78409759899993], [-73.9580956379999, 40.784717461999946], [-73.95763599400006, 40.78534345799992], [-73.957168537, 40.785982893999865], [-73.9567186309999, 40.78660079299993], [-73.95626130899994, 40.78722287599986], [-73.9557773589999, 40.78791392399995], [-73.95416383099995, 40.787226860999944], [-73.95266136699996, 40.78659501999989], [-73.95255145499999, 40.78654858099986], [-73.95244615099996, 40.78650248599994], [-73.95092733999991, 40.78586544699993], [-73.94933170599992, 40.78519312699994], [-73.94982390399991, 40.78451557099987], [-73.95028254899988, 40.78389046999996], [-73.95073913799986, 40.783261709999884], [-73.95119927299987, 40.7826333919999], [-73.95165920699998, 40.78200767399993], [-73.9521220329999, 40.78138670999992], [-73.95257421900006, 40.780753033999865], [-73.95303367999988, 40.78012545499991], [-73.95349212400005, 40.77949662199995], [-73.9539837019998, 40.77882210999991], [-73.95448224099997, 40.77813866499986], [-73.9549452009999, 40.7775039469999], [-73.9554072979999, 40.776868479999926], [-73.95586933699991, 40.776236225999845], [-73.95633492299991, 40.775600782999916], [-73.95679749099989, 40.774966310999936], [-73.95729409999986, 40.7742835549999], [-73.95779380499984, 40.773599896999926]]]}}, {\"id\": \"236\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 237, \"Shape_Leng\": 0.0422126003388, \"Shape_Area\": 9.59873206621e-05, \"zone\": \"Upper East Side South\", \"LocationID\": 237, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9661274729999, 40.762179299999914], [-73.9665833839999, 40.76155093499988], [-73.96818687799986, 40.76222682099993], [-73.96968801199992, 40.76286596199986], [-73.96980028899992, 40.76291351199995], [-73.96990489299992, 40.762958042999834], [-73.97141141699996, 40.76358747399994], [-73.97301487199995, 40.76427887899991], [-73.97255352299985, 40.76489773199991], [-73.97209852400005, 40.76551725099991], [-73.97164774299996, 40.76614246799992], [-73.97118812999996, 40.766766592999915], [-73.97072777399985, 40.76739372199987], [-73.97028165499981, 40.768025031999905], [-73.96982455299987, 40.76864600899992], [-73.96936484599995, 40.769270656999915], [-73.96890454699995, 40.769900831999905], [-73.96844155899997, 40.77051632699994], [-73.96799327799992, 40.77114722999992], [-73.96753398399983, 40.77177618399991], [-73.96703947599987, 40.772461218999844], [-73.96653616999981, 40.77314036899988], [-73.96607448999993, 40.77377298899989], [-73.96560968599992, 40.77440720099987], [-73.96514745399985, 40.77504423199984], [-73.96468735799992, 40.77567708199994], [-73.96308760299976, 40.77500284199987], [-73.96158453899987, 40.774368783999904], [-73.9614758119999, 40.7743240109999], [-73.96136792699977, 40.77427876099991], [-73.95985681799993, 40.77364276799989], [-73.95825534899998, 40.772965336999896], [-73.9587170739999, 40.77233167499991], [-73.95918179999991, 40.771696823999946], [-73.959644408, 40.771064265999925], [-73.96010955199993, 40.77043221099989], [-73.96060180599979, 40.769749186999924], [-73.9611017069999, 40.76905931899988], [-73.96155824299983, 40.76843761199996], [-73.96201413099995, 40.76781230799988], [-73.9624699899999, 40.76718493299991], [-73.96292865800001, 40.76655862799987], [-73.96340154099988, 40.76591119099987], [-73.96384291099993, 40.76530968599996], [-73.9642979039999, 40.76468198699991], [-73.96475544899988, 40.76405706999991], [-73.96521328099986, 40.7634313589999], [-73.96566959999997, 40.76280455599994], [-73.9661274729999, 40.762179299999914]]]}}, {\"id\": \"237\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 238, \"Shape_Leng\": 0.0601093114033, \"Shape_Area\": 0.000184763693765, \"zone\": \"Upper West Side North\", \"LocationID\": 238, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96884378999985, 40.78596738899994], [-73.96933573299991, 40.78529327999992], [-73.97217414999987, 40.786486460999924], [-73.97501417199996, 40.78768560599987], [-73.97639149199995, 40.78826674699993], [-73.97650149399988, 40.78831267699994], [-73.97660504199992, 40.78835448099991], [-73.97785649899998, 40.788881538999945], [-73.98035659399986, 40.789940557999934], [-73.98097150799998, 40.79016087799989], [-73.9815945909999, 40.790384123999935], [-73.98169328900003, 40.79041872799998], [-73.98176573099992, 40.79045542599991], [-73.98185980499981, 40.79050307799985], [-73.98188645923244, 40.7905165804675], [-73.98139174468552, 40.79122824600514], [-73.98128937370942, 40.79137550943299], [-73.98094708354071, 40.79186789378254], [-73.980537679464, 40.792456812624984], [-73.98043434255987, 40.792594283096705], [-73.98013222578668, 40.792993765383144], [-73.98004684398002, 40.79311352516387], [-73.9792881710042, 40.79417765745068], [-73.97828937313398, 40.795586819072305], [-73.97779475503191, 40.796284629771904], [-73.97685207845178, 40.79763134839318], [-73.97639951965265, 40.79827321084623], [-73.97635598099994, 40.79825402099994], [-73.97582955899989, 40.798021996999935], [-73.97563350899993, 40.79794620599991], [-73.97549419399992, 40.79788855199996], [-73.97539813999988, 40.79784774399989], [-73.97521014699983, 40.79777354799995], [-73.97507069999986, 40.797713349999874], [-73.97413148099996, 40.79734378799991], [-73.97365145399995, 40.79714831799993], [-73.97225661799988, 40.79656086499989], [-73.97093073199993, 40.796001047999866], [-73.97082734199992, 40.79595859999988], [-73.97073269599993, 40.79591916999989], [-73.96941543899999, 40.79536077899997], [-73.96987063299996, 40.794735204999895], [-73.96701397, 40.7935599909999], [-73.966447412, 40.79331495399989], [-73.96525997299999, 40.79280137099989], [-73.96417598599989, 40.79236204499989], [-73.96468540699995, 40.791664026999904], [-73.96517705600003, 40.79099034099996], [-73.96563269799995, 40.790365403999886], [-73.96609500599993, 40.7897343889999], [-73.96655226699977, 40.789107152999904], [-73.96700977099991, 40.78847678999989], [-73.96744908399992, 40.78786072099995], [-73.96792696399991, 40.78722157099988], [-73.96838479399985, 40.78659569699989], [-73.96884378999985, 40.78596738899994]]]}}, {\"id\": \"238\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 239, \"Shape_Leng\": 0.0636261152958, \"Shape_Area\": 0.000204715440774, \"zone\": \"Upper West Side South\", \"LocationID\": 239, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97501417199996, 40.78768560599987], [-73.97217414999987, 40.786486460999924], [-73.96933573299991, 40.78529327999992], [-73.96983225599998, 40.784610910999916], [-73.97030068199992, 40.783975413999904], [-73.97076013099995, 40.783340137999865], [-73.971222922, 40.7827062559999], [-73.97172305599986, 40.78202147599997], [-73.97357117499989, 40.77947786199996], [-73.97406668299993, 40.778805416999866], [-73.97453231499993, 40.77816778499994], [-73.97499744000007, 40.77753254599988], [-73.97783054399989, 40.77872973099989], [-73.980673654, 40.77992518199991], [-73.98140948699984, 40.780235418999894], [-73.98151911299976, 40.78028175799991], [-73.981627874, 40.780328934999936], [-73.98351422299999, 40.781125389999914], [-73.98507184299989, 40.781779680999946], [-73.98536952699989, 40.78107837199991], [-73.98567936099997, 40.780312632999944], [-73.98654378999994, 40.780657980999905], [-73.98707137499986, 40.78090638199994], [-73.98730772899984, 40.78104130299991], [-73.98736363999998, 40.78106280499984], [-73.98741432699994, 40.7810875109999], [-73.98746219899996, 40.781108608999936], [-73.98799363199988, 40.781342816999874], [-73.9881274613434, 40.78140179672366], [-73.98804128806722, 40.78158596135378], [-73.98810029382456, 40.781602878305236], [-73.98807644914505, 40.781650935001664], [-73.98801805997222, 40.78163418881039], [-73.98796079284213, 40.78177098703152], [-73.98791968459248, 40.781833477713185], [-73.98787728724994, 40.78189205133478], [-73.98647480368584, 40.783916573718756], [-73.98625187003549, 40.78423876424542], [-73.9861137272529, 40.78443891237993], [-73.98561580396364, 40.78514186259512], [-73.98546581197031, 40.78536070057545], [-73.98617270496534, 40.78606845225863], [-73.98645586240188, 40.785919219081364], [-73.98707234561537, 40.78518963882012], [-73.98711901394246, 40.7852103190041], [-73.98649778102364, 40.78595120288722], [-73.98616462880621, 40.78612188244832], [-73.98612842248579, 40.786239001331076], [-73.98607113521952, 40.786240706026625], [-73.98602727478905, 40.78622896423677], [-73.98609763784941, 40.786058225697985], [-73.98542932126932, 40.78541394218458], [-73.9850811377319, 40.78592193511045], [-73.9851988332546, 40.785966552197806], [-73.98517050238983, 40.78601333415814], [-73.9852162186736, 40.78603050181636], [-73.98525509798003, 40.785976205511524], [-73.98524273937646, 40.78597257265341], [-73.98524962933016, 40.78596313985584], [-73.98528177918678, 40.785978620950054], [-73.98524003288435, 40.78603585813679], [-73.98568388524194, 40.78622212391974], [-73.98571752900456, 40.78617599466883], [-73.98576566029756, 40.7861962748587], [-73.98568287192252, 40.7863097862131], [-73.98563627093021, 40.78629015064923], [-73.98567072256468, 40.7862429119938], [-73.98561523764442, 40.78621964571528], [-73.98520511880015, 40.78604766921281], [-73.98521103560738, 40.78603955488364], [-73.98516263994709, 40.78602099976972], [-73.98513163631195, 40.786060297020015], [-73.98501696406503, 40.78601423719574], [-73.98493597820348, 40.78613072065099], [-73.98465507883023, 40.78653474180792], [-73.98574378790082, 40.78657008285482], [-73.98589227228332, 40.78642652901959], [-73.98594285499509, 40.786452847880334], [-73.98594956155647, 40.78648711396651], [-73.98581237352651, 40.78661686535713], [-73.98513520970305, 40.78658761889557], [-73.98461942858404, 40.78658601634978], [-73.98371551265213, 40.78788609007547], [-73.98234664147557, 40.789854781275245], [-73.98188645923244, 40.7905165804675], [-73.98185980499981, 40.79050307799985], [-73.98176573099992, 40.79045542599991], [-73.98169328900003, 40.79041872799998], [-73.9815945909999, 40.790384123999935], [-73.98097150799998, 40.79016087799989], [-73.98035659399986, 40.789940557999934], [-73.97785649899998, 40.788881538999945], [-73.97660504199992, 40.78835448099991], [-73.97650149399988, 40.78831267699994], [-73.97639149199995, 40.78826674699993], [-73.97501417199996, 40.78768560599987]]]}}, {\"id\": \"239\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 240, \"Shape_Leng\": 0.146069764379, \"Shape_Area\": 0.000722130920707, \"zone\": \"Van Cortlandt Park\", \"LocationID\": 240, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.87643743099994, 40.89687059299995], [-73.8771197409999, 40.895468663999964], [-73.87572387299993, 40.89562260099996], [-73.87480618199984, 40.89572568799984], [-73.87241034499989, 40.89624128599993], [-73.87048672599987, 40.896640210999905], [-73.87006981099996, 40.89659073699993], [-73.86691694899986, 40.89608268499996], [-73.86491632499997, 40.895749494999905], [-73.8640163769999, 40.89552304699988], [-73.86355915799994, 40.89537701699987], [-73.86330852999994, 40.89529484799993], [-73.86287533299995, 40.89515402999991], [-73.86219003099983, 40.89493051899997], [-73.86214409299994, 40.89491552999989], [-73.86209147100006, 40.8948983609999], [-73.86190465899993, 40.89483740499985], [-73.86182532400001, 40.89481152099992], [-73.86176180699982, 40.894791183999914], [-73.86135927899984, 40.89466231899991], [-73.86140513999995, 40.89453904399989], [-73.86149253599987, 40.89427891699995], [-73.86192535099994, 40.89329121599993], [-73.86198229599998, 40.89316126199993], [-73.86215575100003, 40.89283406299995], [-73.86225345500002, 40.892670487999965], [-73.86274739499996, 40.891843574999946], [-73.86280050100001, 40.8917207669999], [-73.86276234599985, 40.89162005699992], [-73.86304439099993, 40.89111565099991], [-73.86381255499994, 40.889716780999954], [-73.86460021999999, 40.888299711999906], [-73.86538049299986, 40.886883474999905], [-73.86620661499991, 40.885378613999926], [-73.86624567599983, 40.8852656929999], [-73.86624572499991, 40.88526556799994], [-73.86624574899986, 40.88526544999991], [-73.86626727199993, 40.885147690999865], [-73.86627001699986, 40.88502747899987], [-73.86627001699995, 40.88502738899994], [-73.86627000499995, 40.88502729999987], [-73.86625345799993, 40.88490729999992], [-73.86621797999994, 40.88479001499995], [-73.866164848, 40.88467823899988], [-73.86616476499985, 40.884678076999954], [-73.86616465799986, 40.88467792299987], [-73.86609568299987, 40.884573620999944], [-73.8660603429999, 40.88450592699994], [-73.86606028499985, 40.884505799999914], [-73.86606024899994, 40.88450565599988], [-73.86604073399987, 40.88443347699989], [-73.86604066399985, 40.88443323499996], [-73.86604065099993, 40.88443298199994], [-73.86603810799987, 40.8843593819999], [-73.8660380959999, 40.884359192999945], [-73.86603813199996, 40.88435902199995], [-73.86605231299984, 40.88428704399993], [-73.86705258299985, 40.882255215999876], [-73.86811829499993, 40.88006228299991], [-73.8688461339999, 40.87858635699989], [-73.86893508399993, 40.87858929599985], [-73.86902622499986, 40.87857616999984], [-73.86911192999995, 40.87854672299993], [-73.86918507299987, 40.87850341099988], [-73.869221773, 40.87846483499988], [-73.8692538419999, 40.878431124999935], [-73.86943471300005, 40.87812919899994], [-73.86950315900006, 40.87814566499986], [-73.86978338299986, 40.87821297799992], [-73.86996807399987, 40.87826435199992], [-73.87005983899994, 40.878290316999916], [-73.8701497639999, 40.87831012099989], [-73.87033725199989, 40.87835140299995], [-73.87094367500002, 40.87851076699998], [-73.87024100099985, 40.88022935299995], [-73.87084076999994, 40.88043889399995], [-73.87187100700007, 40.88032811999996], [-73.87191803300003, 40.880237465999876], [-73.87307334600001, 40.88013438499995], [-73.87406751299994, 40.880038905999896], [-73.87494589799991, 40.87994766899988], [-73.87479240699987, 40.880075128999884], [-73.87467601499999, 40.88156285199993], [-73.87533699399987, 40.88167183199992], [-73.87625890099994, 40.88183203999991], [-73.87717005999983, 40.88199391299992], [-73.87812594399992, 40.88215382599989], [-73.87772817699984, 40.88345419499995], [-73.87760676499984, 40.8845230439999], [-73.87783125799984, 40.88677275999992], [-73.87799008999986, 40.8870503589999], [-73.87803562399996, 40.887118321999885], [-73.878131292, 40.886912796999866], [-73.8781494459999, 40.88687380699994], [-73.87876041599995, 40.886040198999915], [-73.87936283199987, 40.88521823999989], [-73.87995798699986, 40.88439062799987], [-73.88124422499992, 40.88270963299992], [-73.88208623599999, 40.883132220999904], [-73.88274833399998, 40.88339738699993], [-73.88319088699988, 40.88328483399989], [-73.88334213499988, 40.883226517999944], [-73.88334234799999, 40.883226436999934], [-73.88334258599987, 40.88322637399989], [-73.88350764199998, 40.883183245999874], [-73.8835078789999, 40.883183183999975], [-73.88350810399999, 40.883183146999954], [-73.88368327399996, 40.88315763299994], [-73.883683369, 40.883157614999966], [-73.88368347599999, 40.88315761499991], [-73.88386358099999, 40.88315139399987], [-73.8838637469999, 40.88315138499995], [-73.88386391399983, 40.88315139399986], [-73.88404328600001, 40.88316497399997], [-73.88404341599993, 40.88316498399992], [-73.88404353399987, 40.88316500199992], [-73.88421671500001, 40.88319766499993], [-73.88421683399996, 40.883197691999904], [-73.88421696399996, 40.8831977279999], [-73.88437887799986, 40.88324759799985], [-73.88437905799985, 40.88324765099988], [-73.88437922299993, 40.88324772299995], [-73.88452615199998, 40.88331214699985], [-73.88516495799995, 40.883631611999924], [-73.88662412999999, 40.88426730299992], [-73.88683486599984, 40.88434202599992], [-73.886928897, 40.88435571399992], [-73.88705142500004, 40.884349575999934], [-73.88758612599987, 40.88425561899988], [-73.88840157099996, 40.88441707499993], [-73.88889642199983, 40.88450391899991], [-73.88984701899983, 40.88468574699988], [-73.89080543899989, 40.884867842999945], [-73.89176617899982, 40.88504984599992], [-73.89288644500004, 40.88526430599993], [-73.89467529599987, 40.88553264999992], [-73.89508270700003, 40.885641511999914], [-73.89595674999993, 40.88582832499989], [-73.89652069099988, 40.88593698899992], [-73.89661546599986, 40.885955255999946], [-73.89668242999997, 40.885964589999965], [-73.89688178399989, 40.88599934699987], [-73.89715645399986, 40.88604482399988], [-73.89738336099998, 40.88608677199995], [-73.89892571799993, 40.88638350199994], [-73.89972588499982, 40.886514886999926], [-73.89965185599996, 40.88666493399989], [-73.89936426899996, 40.887249089999926], [-73.89922499099997, 40.887530802999876], [-73.89827216899992, 40.8894579429999], [-73.89787887499983, 40.89027610099993], [-73.89776669099993, 40.890509389999906], [-73.89769601099988, 40.890681823999934], [-73.89731936199985, 40.891615860999885], [-73.89728112499986, 40.89171068399993], [-73.89705763499997, 40.89214045799989], [-73.89675790099987, 40.89264544399993], [-73.89643170799987, 40.89365527899989], [-73.89670747099987, 40.89580502899987], [-73.896994929, 40.897878723999916], [-73.89709149099991, 40.89859331799993], [-73.89715717199994, 40.89937182699985], [-73.89713947299998, 40.8996194349999], [-73.89707329199993, 40.90008389099992], [-73.89704651399992, 40.90022470299987], [-73.8970336369998, 40.900278620999885], [-73.89702895099995, 40.90031520399991], [-73.89697054199992, 40.90061116799993], [-73.89687844599996, 40.90104057999988], [-73.89661002199993, 40.90220351299991], [-73.89640895699986, 40.9029611079999], [-73.89635785999998, 40.9032001319999], [-73.896411335, 40.90450452099991], [-73.89643503499988, 40.905092998999855], [-73.89649229399996, 40.90633965199994], [-73.8965465329998, 40.907506654999956], [-73.89659583799997, 40.909627658999916], [-73.89660613599976, 40.9102022489999], [-73.89663068599988, 40.91129887799991], [-73.8966306859999, 40.91129892699992], [-73.89663333831646, 40.91141737492314], [-73.89246906581641, 40.91016766410472], [-73.88610703995391, 40.90832927112678], [-73.88605575389856, 40.90831445280059], [-73.88599855043816, 40.90829712498603], [-73.88561523020898, 40.908181024408606], [-73.88480028050311, 40.9079307362925], [-73.88469597226361, 40.907898700147676], [-73.8840540725523, 40.90770269719356], [-73.88303257421012, 40.90733363107853], [-73.88148094380769, 40.906891856850486], [-73.87836160112978, 40.90600363600307], [-73.8781677757751, 40.905955508242734], [-73.87811197786539, 40.90594376004156], [-73.87807189372022, 40.90593244410284], [-73.87803672199264, 40.90592860716306], [-73.87786550388269, 40.90586920992736], [-73.8773681710897, 40.90571165693519], [-73.87646714999133, 40.90544411710721], [-73.87563292109846, 40.90520821290864], [-73.87483045244237, 40.90494378318406], [-73.87416054892165, 40.9047812831199], [-73.87294860376426, 40.9044410228341], [-73.87095251943089, 40.903850252857], [-73.86789043706806, 40.90298695407611], [-73.86816613899995, 40.9024482979999], [-73.86820270299982, 40.90235538299993], [-73.86899707599991, 40.90155915499991], [-73.87010369599989, 40.90074194699991], [-73.87136373199995, 40.900039151999884], [-73.87301813899998, 40.89935490499993], [-73.87388944299983, 40.89896546299988], [-73.87482041599985, 40.89844543799991], [-73.87589465400002, 40.8975594589999], [-73.87643743099994, 40.89687059299995]]]}}, {\"id\": \"240\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 241, \"Shape_Leng\": 0.0687645777649, \"Shape_Area\": 0.00025467697592, \"zone\": \"Van Cortlandt Village\", \"LocationID\": 241, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.88840157099996, 40.88441707499993], [-73.88758612599987, 40.88425561899988], [-73.88705142599984, 40.884349575999906], [-73.88704303999992, 40.88404023199992], [-73.88708705799998, 40.88379702699996], [-73.88710004499984, 40.88372994399995], [-73.88710703299991, 40.8836739859999], [-73.88712303599982, 40.883200939999874], [-73.88713206499988, 40.883150944999926], [-73.8871469759999, 40.88311793799995], [-73.88718101599989, 40.88308394199995], [-73.8872319399999, 40.88305497799989], [-73.88697007500001, 40.8827956429999], [-73.88694246799994, 40.88268276699991], [-73.88716505199984, 40.88257651099986], [-73.88706962799996, 40.882476895999915], [-73.88595071699984, 40.88050741199993], [-73.88582680399999, 40.879907896999896], [-73.88578568400006, 40.879616873999964], [-73.88575540599997, 40.87931311799986], [-73.88567517999988, 40.87887251099986], [-73.88773577599993, 40.87703388199987], [-73.88808032199985, 40.87627274599987], [-73.88824333699998, 40.875892194999935], [-73.88855699899982, 40.8751973129999], [-73.88937163799984, 40.87338229799993], [-73.88863187299991, 40.87276503099986], [-73.88857987899992, 40.87272163999993], [-73.88798774400006, 40.872227490999855], [-73.88785196299986, 40.87211417599992], [-73.8879350369998, 40.87203023199988], [-73.88833546999989, 40.871709830999905], [-73.88868449900004, 40.87149511299988], [-73.88884301599997, 40.87139806699986], [-73.88950049199994, 40.87102055599993], [-73.88991744500004, 40.870780631999864], [-73.89042763899991, 40.870487053999916], [-73.89098696999984, 40.870160714999905], [-73.89119350399992, 40.8700100329999], [-73.89138946999998, 40.869850993999904], [-73.8915740009999, 40.86968425299995], [-73.8917463349999, 40.86951054799996], [-73.8919171829999, 40.869302109999886], [-73.89221258899984, 40.86891549999992], [-73.89243509099991, 40.868620597999936], [-73.8930026999999, 40.86788106799988], [-73.89368435599988, 40.86698082099989], [-73.8937613899999, 40.866875057999955], [-73.89397201799989, 40.8666023039999], [-73.89408697099991, 40.86645343299995], [-73.89431825299994, 40.86615392699988], [-73.89436438800003, 40.86609167199985], [-73.89438130299995, 40.866068850999966], [-73.8945304879999, 40.86614380899987], [-73.89536151600007, 40.86656133099995], [-73.89544263399986, 40.866602088999855], [-73.89631315699987, 40.8670385749999], [-73.8974032329998, 40.86747418999989], [-73.89656060499982, 40.86856519999992], [-73.89891854299988, 40.86965857399992], [-73.8990732729999, 40.86952535499993], [-73.89921387699998, 40.86938138699987], [-73.89933851199986, 40.869228027999874], [-73.89944561599987, 40.869066841999924], [-73.8995340439999, 40.86889961299994], [-73.89960300399986, 40.86872820799991], [-73.89965220299995, 40.868554577999916], [-73.89942436199988, 40.86788083799993], [-73.89996501599988, 40.86801852399986], [-73.90106211299997, 40.868282647999926], [-73.90235804799984, 40.8686961209999], [-73.90318667499999, 40.86914899099988], [-73.90374975899987, 40.86959562699992], [-73.90403592599995, 40.86994916299995], [-73.90442003199995, 40.87036127599988], [-73.90448860599989, 40.870578548999895], [-73.9044342879998, 40.87224530899993], [-73.90444007299996, 40.872330327999855], [-73.90444008499996, 40.872330418999866], [-73.90444010799983, 40.87233049899992], [-73.9044689679998, 40.872415393999944], [-73.9044690149999, 40.87241552099991], [-73.90446908599996, 40.872415627999935], [-73.90452057499998, 40.87249502299987], [-73.90452070499992, 40.87249521099988], [-73.90452088299993, 40.872495391999955], [-73.90459188899993, 40.872564445999956], [-73.90459194899984, 40.8725644999999], [-73.90459203199988, 40.87256455399988], [-73.904677283, 40.872619878999934], [-73.905714254, 40.87313832199987], [-73.905866168, 40.87318265999987], [-73.90591126399978, 40.87319581699988], [-73.90601809399988, 40.8732269889999], [-73.90608130599988, 40.873239628999904], [-73.90605459699992, 40.87329392499993], [-73.90556943300005, 40.87423099199992], [-73.90526442899994, 40.8747911099999], [-73.90524600299995, 40.87482494899991], [-73.90503517999984, 40.87523228299991], [-73.90483848999986, 40.87564374299989], [-73.90465612299992, 40.87605891599984], [-73.90448826999989, 40.8764773599999], [-73.90417988499996, 40.877336633999974], [-73.903898338, 40.87812110799991], [-73.90369454799992, 40.878501617999895], [-73.90333605399985, 40.87897514599996], [-73.90226256899996, 40.88010809899993], [-73.9015065839999, 40.88095490599994], [-73.89988688799993, 40.882828808999925], [-73.89914458899997, 40.88368756599993], [-73.89855059899999, 40.88437472699996], [-73.89843428799998, 40.88447464199989], [-73.89835487899985, 40.884542864999965], [-73.89808000699986, 40.884778999999874], [-73.89781492099993, 40.88499918299995], [-73.89760048499983, 40.88517729399996], [-73.897112233, 40.88556943599986], [-73.89661546599986, 40.885955255999946], [-73.89652068999983, 40.88593698899991], [-73.89595674999993, 40.88582832499989], [-73.89508270700003, 40.885641511999914], [-73.8946752969999, 40.885532649999966], [-73.89288644599986, 40.88526430599996], [-73.89176617899982, 40.88504984599992], [-73.89080543899989, 40.884867842999945], [-73.88984701899993, 40.8846857459999], [-73.88889642199983, 40.88450391899991], [-73.88840157099996, 40.88441707499993]]]}}, {\"id\": \"241\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 242, \"Shape_Leng\": 0.138136446433, \"Shape_Area\": 0.000360072450014, \"zone\": \"Van Nest/Morris Park\", \"LocationID\": 242, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83593362199987, 40.84840635599993], [-73.83606934399998, 40.84716484899992], [-73.83615362999997, 40.84613117599993], [-73.83619907199994, 40.84582739599996], [-73.8365331269999, 40.84462069499994], [-73.83676876799983, 40.843542820999886], [-73.83688925199996, 40.84337478799992], [-73.83714579999987, 40.8427276329999], [-73.83736984299993, 40.84204138399985], [-73.83767001599998, 40.841121907999856], [-73.8376890109999, 40.84081118199989], [-73.83771374799997, 40.84040630999987], [-73.8381237629999, 40.84057018199993], [-73.83821036999994, 40.840604797999916], [-73.83830524899987, 40.840642718999916], [-73.83839056199994, 40.84067681399993], [-73.83849424699997, 40.840699219999934], [-73.83880813300001, 40.84076703399992], [-73.83936452799996, 40.84087706699995], [-73.84124785499995, 40.84062162899993], [-73.84140548300003, 40.84060250299987], [-73.84156602199995, 40.84059525799992], [-73.84172685399986, 40.84060017099996], [-73.84172710199991, 40.84060017999991], [-73.84172735099988, 40.840600206999966], [-73.84188610300001, 40.84061717399984], [-73.84188635099984, 40.840617201999876], [-73.84188659999987, 40.84061724699993], [-73.84204144699997, 40.84064589699992], [-73.84240098600006, 40.84034851399991], [-73.8426130889999, 40.840167155999886], [-73.8430625279998, 40.83979720599991], [-73.84361155699993, 40.83934832599994], [-73.84391720399984, 40.839086325999844], [-73.84458129199986, 40.838535507999914], [-73.8455308949999, 40.83917330699989], [-73.84647864699997, 40.83980920599987], [-73.84715638299993, 40.840263675999914], [-73.84779791999988, 40.84069631099985], [-73.84796242199991, 40.84082789199988], [-73.8489861779999, 40.84159995199983], [-73.85052278099987, 40.84278948899985], [-73.85088863399999, 40.84275300599993], [-73.85120896799987, 40.84271512199991], [-73.85179886599994, 40.8426476329999], [-73.85231153200002, 40.84258699399991], [-73.85333524999986, 40.84247015499987], [-73.85349746599992, 40.84245166799994], [-73.85420745799989, 40.84237073899991], [-73.85520012799999, 40.84225761399988], [-73.85553579799995, 40.84221948999986], [-73.85618402199997, 40.84214591699992], [-73.85719247999995, 40.84202764799989], [-73.85835669199989, 40.84189928599988], [-73.85919593699994, 40.841804032999924], [-73.8597245239999, 40.84174401899984], [-73.86018491899982, 40.84168908999986], [-73.86045627399983, 40.841661664999954], [-73.86162833399996, 40.84152785999986], [-73.86176166600002, 40.841512719999926], [-73.86347232399994, 40.84130593299991], [-73.86358013999998, 40.84127795699989], [-73.863639382, 40.8412652749999], [-73.86436922099999, 40.84110901799995], [-73.86448011799995, 40.84108779699994], [-73.865464502, 40.84085689199993], [-73.86636130899984, 40.8405926849999], [-73.86724819000003, 40.840343803999886], [-73.86740207099996, 40.84029027599994], [-73.86782258199997, 40.840166498999935], [-73.86830846899983, 40.83999521299996], [-73.8687407359999, 40.83982457599993], [-73.86920766899995, 40.83960004599989], [-73.86955862099985, 40.83959565299989], [-73.87021450700003, 40.839617941999954], [-73.87120605899995, 40.839674621999905], [-73.87272718499995, 40.83974894499989], [-73.87277204699981, 40.83975128199995], [-73.87283453899995, 40.839754862999925], [-73.87359713599987, 40.83979855499992], [-73.87326716999992, 40.84021994899994], [-73.87280724899995, 40.84086864499993], [-73.87244734699995, 40.84134426699992], [-73.87236365199989, 40.84147695999991], [-73.87229921099988, 40.84157999599991], [-73.87223024699995, 40.84164317099993], [-73.87153346199983, 40.84251428099987], [-73.87099518100001, 40.843111391999926], [-73.87030135699996, 40.8435486959999], [-73.86959555499995, 40.84398144199994], [-73.8688925879999, 40.844409658999936], [-73.86804200399997, 40.8448138389999], [-73.86863721699989, 40.845309842999896], [-73.86874186599977, 40.845609728999904], [-73.86869009299983, 40.84576627499991], [-73.86865877599989, 40.84592828199989], [-73.86865873999996, 40.84592845299991], [-73.8686587279999, 40.84592863299986], [-73.86864915200002, 40.846093754999906], [-73.86864914000002, 40.84609392599993], [-73.86864915199992, 40.84609409699987], [-73.86866192800001, 40.846259982999946], [-73.86866193899994, 40.846260153999914], [-73.8686619749999, 40.84626032399997], [-73.86869709499992, 40.846424344999896], [-73.86869712999989, 40.846424505999934], [-73.86869718899982, 40.8464246689999], [-73.8687539479999, 40.846584363999966], [-73.86875397099986, 40.8465844179999], [-73.86875399499986, 40.84658447099989], [-73.86883109200002, 40.84673752699992], [-73.868926528, 40.846881740999876], [-73.86843808099991, 40.84705127899987], [-73.86835465999991, 40.847080232999865], [-73.86800652899981, 40.84830196599988], [-73.86785992999985, 40.84886493099987], [-73.86837388999993, 40.84896959799991], [-73.86920944599994, 40.84923837299992], [-73.86823466099995, 40.85091373699986], [-73.86834552299989, 40.85119885199987], [-73.86844433299986, 40.85148667199995], [-73.86853091299996, 40.85177675699994], [-73.86860515599989, 40.85206864599991], [-73.86780004799985, 40.85158929399994], [-73.86680474499987, 40.85094982699991], [-73.86612130799995, 40.85052722199985], [-73.86553546999978, 40.850288634999906], [-73.86518787199995, 40.8501643859999], [-73.86473360999999, 40.8502745589999], [-73.86303938099996, 40.850611306999944], [-73.86278872899996, 40.84923798299993], [-73.86250923799993, 40.84836228099987], [-73.86160276499992, 40.84854631199996], [-73.86119240799994, 40.84860346799991], [-73.86094001199987, 40.848670934999895], [-73.86069456599995, 40.848753393999935], [-73.8604583409998, 40.84885014599992], [-73.86023344799996, 40.84896024699987], [-73.85942539999989, 40.84932393799991], [-73.8586172349999, 40.84968784799988], [-73.85780815, 40.85005115699993], [-73.85700079699997, 40.850415596999916], [-73.85619083999988, 40.85077955999987], [-73.855383516, 40.85114371899989], [-73.85457388299992, 40.85150834599995], [-73.85304526899994, 40.849547331999865], [-73.85224152599983, 40.8499109959999], [-73.85136789699986, 40.85030370399994], [-73.85049849099983, 40.85069600599994], [-73.84968869799998, 40.851058906999874], [-73.84861308099981, 40.851079957999964], [-73.8482145439999, 40.85109088899991], [-73.84729781299993, 40.85108686899989], [-73.84721027699993, 40.851036838999946], [-73.8466850059999, 40.851118077999914], [-73.84453077199993, 40.85113289899989], [-73.84430846999992, 40.85119046399989], [-73.84441301999993, 40.85104299699991], [-73.84475779899996, 40.850468460999906], [-73.84496954099991, 40.85009703099991], [-73.84502303799998, 40.84943496099992], [-73.84504985899981, 40.84919158599997], [-73.84505506699996, 40.84894655999991], [-73.84503845499987, 40.84870153999991], [-73.84500013599994, 40.84845821899989], [-73.84494053799982, 40.8482182209999], [-73.84428070699988, 40.84912269899996], [-73.84361735799992, 40.85008033999988], [-73.84194933699985, 40.852512679999904], [-73.84149163399988, 40.85319428599991], [-73.84134190199984, 40.853368113999906], [-73.84118561999999, 40.85353858499997], [-73.84102291699999, 40.85370555799991], [-73.84085393099993, 40.85386889199994], [-73.84067880099991, 40.85402845199995], [-73.84049767399992, 40.8541841049999], [-73.84031070099996, 40.85433571999993], [-73.84011803799997, 40.854483171999924], [-73.83991984599993, 40.85462633599991], [-73.83971628999994, 40.85476509399998], [-73.83935709000001, 40.854986753999924], [-73.83898125999988, 40.85519446599994], [-73.83859024199985, 40.85538719199999], [-73.83818561199983, 40.8555640099999], [-73.83776909799984, 40.855724111999905], [-73.83734249299998, 40.85586686399994], [-73.83690771399995, 40.85599174899995], [-73.83646667999994, 40.856098438999915], [-73.83562457399984, 40.85636963099991], [-73.8353782199999, 40.85644896499984], [-73.83499552599994, 40.85657120099989], [-73.83667557299988, 40.85665266299989], [-73.83691212799982, 40.85700810699997], [-73.83690582299991, 40.85718069599995], [-73.8368191849999, 40.85727680899989], [-73.83628873399996, 40.85747692099994], [-73.83533372599987, 40.8578313009999], [-73.83437708299991, 40.858190848999854], [-73.83348699499999, 40.85852949499987], [-73.83308051999994, 40.85867901899998], [-73.83190625000005, 40.859070198999895], [-73.83103340899994, 40.85934303399997], [-73.83017269399993, 40.85963375399996], [-73.83002075599987, 40.85968150899988], [-73.829874887, 40.859740419999895], [-73.82973719699993, 40.85980973499988], [-73.82960951799994, 40.85988842099988], [-73.82945155499976, 40.860000568999894], [-73.82881978099991, 40.8604491129999], [-73.82847346699995, 40.860731352999906], [-73.82872856899989, 40.860413327999915], [-73.82959409199992, 40.85935976399989], [-73.83055471299986, 40.85819039599993], [-73.83072266700006, 40.857994586999915], [-73.831114492, 40.8575243299999], [-73.83128338399985, 40.857349961999894], [-73.83193393799985, 40.856693516999876], [-73.8321371229998, 40.85654369099993], [-73.83238468599995, 40.856352210999916], [-73.83268903299997, 40.8561323829999], [-73.8327621999999, 40.856079539999925], [-73.83283502299986, 40.8560358449999], [-73.83281265300002, 40.85597971299992], [-73.83242972799992, 40.85582103999992], [-73.83229711899999, 40.85574641199985], [-73.83216420699983, 40.85567161299986], [-73.83200598799996, 40.855594876999916], [-73.83183634999988, 40.85553171199996], [-73.83165843199998, 40.855483447999916], [-73.83147565799987, 40.855450852999844], [-73.83129157699989, 40.85543410499991], [-73.83157449000001, 40.855379596999946], [-73.83193102199998, 40.85528511599992], [-73.83228148899984, 40.85517744999994], [-73.832624647, 40.855056966999925], [-73.83295928399993, 40.854924105999956], [-73.83350017799994, 40.85467968399997], [-73.83390573999993, 40.85446777999996], [-73.83408232099997, 40.8543626399999], [-73.83424656499982, 40.854243551999915], [-73.83439574299983, 40.85411187099993], [-73.83452741999996, 40.853969322999866], [-73.83463970199993, 40.853817993999904], [-73.83473123599998, 40.853660186999896], [-73.83480132399997, 40.85349832599997], [-73.83504028899982, 40.853126736999876], [-73.83519691899983, 40.85284353099991], [-73.83533589099999, 40.85255469799993], [-73.83545671599992, 40.85226119199988], [-73.83555900199993, 40.85196401299994], [-73.83580031399995, 40.8496768159999], [-73.83593362199987, 40.84840635599993]]]}}, {\"id\": \"242\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 243, \"Shape_Leng\": 0.0943314999364, \"Shape_Area\": 0.000438377341503, \"zone\": \"Washington Heights North\", \"LocationID\": 243, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.93156536999994, 40.86958215799991], [-73.93141437399996, 40.868961927999905], [-73.93127671599983, 40.8688435939999], [-73.93117877500005, 40.8687688669999], [-73.93091942499993, 40.868571000999935], [-73.92997974799981, 40.86779614899992], [-73.92934104099999, 40.86725723999993], [-73.92872203199994, 40.86674266299988], [-73.92805770299987, 40.866199363999954], [-73.92726915099985, 40.86553455999994], [-73.92661948999994, 40.86452429399989], [-73.92601092899994, 40.8635736649999], [-73.92537318199986, 40.86258081399991], [-73.92474153399992, 40.86159363799992], [-73.92499818399988, 40.86146373799991], [-73.92496864099992, 40.86128326999995], [-73.9249677899999, 40.861253467999894], [-73.92496310699987, 40.861089282999934], [-73.92498050399995, 40.8608925189999], [-73.92502162200003, 40.860695310999894], [-73.92508669599982, 40.86050012699992], [-73.92517535599984, 40.86030947899993], [-73.9252866269999, 40.860125796999945], [-73.92541898499994, 40.85995133099993], [-73.9255703789999, 40.859788079999895], [-73.92573839899995, 40.85963767999988], [-73.9259203249999, 40.85950140899994], [-73.92611328899989, 40.85938011099994], [-73.92629594899988, 40.859287413999915], [-73.92646803499991, 40.859178123999854], [-73.9266257519999, 40.85905332599989], [-73.92676563199994, 40.85891478399991], [-73.9268847099999, 40.85876482899988], [-73.92698078699978, 40.85860624099995], [-73.92705253299991, 40.85844209099991], [-73.92709951599986, 40.85827552099992], [-73.92712222099993, 40.858109610999854], [-73.92691117499983, 40.85750876099994], [-73.92687811100001, 40.85761400599994], [-73.92682273599989, 40.85772223799992], [-73.92674232299986, 40.85782976799987], [-73.92663553999984, 40.85793192399994], [-73.92650299499998, 40.858023418999906], [-73.92634775499995, 40.858098970999926], [-73.9261752689999, 40.858154095999936], [-73.92599285799992, 40.8581858659999], [-73.92580872999993, 40.85819341999995], [-73.92563075899986, 40.8581779519999], [-73.925465479, 40.858142410999974], [-73.92531748399992, 40.858090806999954], [-73.92518924999993, 40.85802753699993], [-73.92508143699999, 40.85795678799993], [-73.92496717399985, 40.8578110039999], [-73.92486929999991, 40.85765619099987], [-73.92478952099987, 40.857493996999885], [-73.9247291889999, 40.857326286999886], [-73.92468915299989, 40.85715512199993], [-73.92466977999999, 40.856982610999886], [-73.92467094699998, 40.8568108889999], [-73.92469199999985, 40.8566419969999], [-73.92679937899987, 40.85371714499991], [-73.92723935000001, 40.85311537199989], [-73.92769565799996, 40.85248778199988], [-73.92703315699981, 40.852200774999886], [-73.92707982300001, 40.851432342999885], [-73.92747680599985, 40.85069436999992], [-73.92824918899986, 40.84983985499985], [-73.92842288, 40.849745680999895], [-73.92858520000006, 40.849638801999966], [-73.92873382199987, 40.849520488999886], [-73.92886675399986, 40.849392312999925], [-73.92898242699985, 40.8492561219999], [-73.92970731599995, 40.84812261199988], [-73.9306210789998, 40.848480976999895], [-73.93088430099981, 40.84811893799988], [-73.93116151799987, 40.84774060099992], [-73.93139355399987, 40.84742823099985], [-73.93284984899991, 40.84803810999989], [-73.93423946400003, 40.84862621199994], [-73.93547108199992, 40.849144382999924], [-73.93639683100004, 40.84954119499985], [-73.93575394299994, 40.8501682029999], [-73.93666142799988, 40.85044732599994], [-73.93736531699994, 40.850644529999954], [-73.93828416699976, 40.85090854599985], [-73.93923603299991, 40.851178486999906], [-73.93939953299987, 40.851232664999955], [-73.93939969799992, 40.851232718999896], [-73.93939985299984, 40.85123275599991], [-73.93957073799986, 40.85127555599993], [-73.93974700799991, 40.85130623699991], [-73.9397471139998, 40.8513062549999], [-73.93974720899999, 40.851306264999906], [-73.939926313, 40.85132427799988], [-73.93992646699995, 40.851324295999895], [-73.93992663199994, 40.85132430499989], [-73.94010632899983, 40.85132953199994], [-73.94010645799983, 40.851329531999916], [-73.94010658899992, 40.851329521999936], [-73.94028476699981, 40.8513221499999], [-73.94048563299987, 40.85131271699995], [-73.94048575099985, 40.851312708999934], [-73.9404858699999, 40.851312708999906], [-73.94068785099988, 40.85131542399989], [-73.94068804199996, 40.85131542399991], [-73.94068822000003, 40.85131543299985], [-73.94088954899996, 40.85133036799992], [-73.94088972799987, 40.85133037599988], [-73.94088990599982, 40.85133040399996], [-73.94108881800004, 40.851357429999894], [-73.94108888899997, 40.851357438999905], [-73.9410889599999, 40.85135745699992], [-73.94126220299982, 40.8513919619999], [-73.94185191299992, 40.851821335999944], [-73.94203871799984, 40.851931028999914], [-73.94221850999999, 40.85179850399986], [-73.94238599199987, 40.85165518499987], [-73.94247684499997, 40.85156459099987], [-73.9425395099999, 40.851502104999895], [-73.94267766599995, 40.85134048199992], [-73.94279927299996, 40.85117167299994], [-73.94290346199983, 40.85099716399988], [-73.94298963999995, 40.8508185039999], [-73.94309647899989, 40.850387302999884], [-73.94311799599984, 40.85023724299993], [-73.94315143499992, 40.85000403399993], [-73.94316093899997, 40.84993545899992], [-73.94316977700004, 40.849847511999904], [-73.94317806099995, 40.84979994299991], [-73.94381229899996, 40.84991964099994], [-73.94396593999994, 40.849950809999946], [-73.94409966200001, 40.849977402999855], [-73.94472103299991, 40.85010096199993], [-73.944768533, 40.85011040299986], [-73.944914825, 40.85013703799994], [-73.9458970349999, 40.85032798199988], [-73.94692626452422, 40.850528055834154], [-73.94686899614919, 40.85058959685871], [-73.94681367391628, 40.85064903917808], [-73.94680219627986, 40.85066137249107], [-73.94651595882908, 40.850968926946535], [-73.94650234446286, 40.85097627305156], [-73.94648747390856, 40.85098204659227], [-73.94647165993504, 40.850986126135254], [-73.94645689400723, 40.85098827764361], [-73.94644188478783, 40.850988950102874], [-73.94642688811145, 40.85098813205078], [-73.94641211347599, 40.85098582788765], [-73.94639786213489, 40.85098207729574], [-73.9463843785491, 40.85097694461102], [-73.94637189400876, 40.85097051787691], [-73.94633856812649, 40.85094852447611], [-73.94632053353631, 40.850923937715315], [-73.94632862612754, 40.85090502942344], [-73.94635149986354, 40.85089369453736], [-73.94635191475699, 40.85087055856909], [-73.94633765223675, 40.85085347928549], [-73.94632696500435, 40.85084661540101], [-73.94631554216869, 40.850839278924475], [-73.94629678670805, 40.85083372583879], [-73.94627461183848, 40.8508415934715], [-73.9462433633095, 40.85088523552586], [-73.94619735733023, 40.850906420201966], [-73.94617267171917, 40.85095878975112], [-73.94619565359775, 40.850975009538296], [-73.94620054046909, 40.85103239211626], [-73.9461857520811, 40.85103238521867], [-73.94617185524098, 40.851053071063646], [-73.94615791969895, 40.85104824156037], [-73.94614668962957, 40.85103223464638], [-73.94614508385327, 40.85100838241098], [-73.94613353603556, 40.850984828080854], [-73.94611934283114, 40.85096964315818], [-73.94609624718471, 40.85096724398696], [-73.94608145021917, 40.85098164676894], [-73.94608704722194, 40.851009023386204], [-73.9460970573703, 40.85103358549356], [-73.94609776719916, 40.85103455913642], [-73.94614136091342, 40.851094723094654], [-73.94613637747574, 40.85115582829458], [-73.94611728194423, 40.85117372143474], [-73.9460476559076, 40.8511896630908], [-73.9459646207679, 40.85117296552844], [-73.94593014421993, 40.85119079648432], [-73.94589567174965, 40.85118840282922], [-73.94584084991659, 40.85117052999496], [-73.94576720433773, 40.851170495396815], [-73.94572488131935, 40.851197841284424], [-73.94572798690905, 40.851228774441324], [-73.9456825554423, 40.851227564647836], [-73.94565279045506, 40.851212080342066], [-73.945612031476, 40.85124061483366], [-73.94555556310249, 40.85132148895218], [-73.94552736155029, 40.85132147563915], [-73.94551948943962, 40.85136311070993], [-73.94548187577574, 40.85137380003347], [-73.94544893316966, 40.851428516042574], [-73.94542229851784, 40.85142731553617], [-73.94539567234408, 40.851416594838255], [-73.9453784200023, 40.851427294768904], [-73.94538936847742, 40.85145346795048], [-73.94532198248487, 40.85147009472928], [-73.9453125651084, 40.851485561054524], [-73.94527966082822, 40.851495062960375], [-73.94525302780075, 40.85149266412057], [-73.9452436006128, 40.85152002543679], [-73.94527256188402, 40.851546811601956], [-73.94526627919835, 40.8515717887104], [-73.94519574791492, 40.85159317711649], [-73.94515345120476, 40.851587204237546], [-73.94506567411703, 40.851639507848795], [-73.94504370869103, 40.851671618664454], [-73.94500607672323, 40.85170254180599], [-73.944977851949, 40.85173227213227], [-73.94493240846673, 40.85173105265653], [-73.94487719641315, 40.85175440837052], [-73.94477691248362, 40.851796831953266], [-73.94476566471432, 40.851801590264635], [-73.94471435250203, 40.851815911218964], [-73.9446937046878, 40.851824601986216], [-73.94468755000356, 40.851836237725294], [-73.94469257824674, 40.85186300317582], [-73.94468965406844, 40.851877645331385], [-73.94468309631044, 40.8519104913526], [-73.94466706665953, 40.85192169852068], [-73.94463221068919, 40.85190996135586], [-73.94458832934104, 40.851927789069904], [-73.94457734596685, 40.8519432540793], [-73.94450369998015, 40.85195749066511], [-73.94448648497414, 40.85192417370039], [-73.94450307651685, 40.85189982511769], [-73.94450507247345, 40.85189033001256], [-73.94448754202686, 40.85188922312717], [-73.94446190429296, 40.851899805303866], [-73.94445360950604, 40.85189798049818], [-73.94442225918847, 40.85190510658742], [-73.94439719907635, 40.85189319951097], [-73.94440663561446, 40.85185513117773], [-73.94436748956578, 40.851825368522476], [-73.94432519023107, 40.85182297062387], [-73.94431732295752, 40.851856276514525], [-73.94429066589659, 40.85188244102266], [-73.94424520021663, 40.85192049193891], [-73.94421071900311, 40.85192998456854], [-73.94422164925084, 40.851976391831485], [-73.94419654622922, 40.85201564098287], [-73.94413386022264, 40.85202750564373], [-73.94407120589058, 40.85201558028507], [-73.94401323929921, 40.852000090761386], [-73.9439912652317, 40.852041718878716], [-73.94395986779188, 40.85211903729921], [-73.94388148068221, 40.852172533479965], [-73.94379606084516, 40.85219807473172], [-73.94360072252731, 40.85223107230981], [-73.94347978545615, 40.85228188294709], [-73.94305902944303, 40.85260143014694], [-73.94302644371187, 40.852626177870995], [-73.94279838505354, 40.852851233889844], [-73.94260996683576, 40.85306791474675], [-73.94247365925723, 40.85317127831177], [-73.94205514258485, 40.853654107455704], [-73.94186996426673, 40.853867739442755], [-73.94182774425003, 40.853989960632596], [-73.94173105661464, 40.854108388549314], [-73.9415815880044, 40.85434714380842], [-73.94153736742429, 40.85443564359609], [-73.94150213189877, 40.85450012674853], [-73.94149762132163, 40.85450838436768], [-73.9414098064236, 40.85468463098871], [-73.94131311131314, 40.85479553975522], [-73.94126592051121, 40.85494407878855], [-73.94123366359767, 40.855009887976884], [-73.94103277003084, 40.85530503914508], [-73.94099928999188, 40.85535861893858], [-73.94100914313304, 40.855415039634785], [-73.94095704472404, 40.85552033439455], [-73.94085539855477, 40.855621830778425], [-73.94076854842876, 40.855807971467506], [-73.9407462171037, 40.85584744692031], [-73.94069419380095, 40.855853066060135], [-73.94061975134593, 40.85601100074242], [-73.9404883126927, 40.85620462882266], [-73.9404051777662, 40.856372905782024], [-73.94035056461426, 40.85651580307319], [-73.94029347497097, 40.85664177065756], [-73.94014717618717, 40.85683351705708], [-73.9400852345242, 40.85685605129765], [-73.94001827015856, 40.85695004614774], [-73.93994880677141, 40.8570666142561], [-73.93978146211117, 40.85723483699592], [-73.93968482410261, 40.85728932103607], [-73.93961041073217, 40.85739647639676], [-73.93940211514372, 40.85766529797215], [-73.93936488888228, 40.857740505738334], [-73.93928303850637, 40.857860822791295], [-73.9392223197617, 40.85789369423105], [-73.93911094164396, 40.85804091122572], [-73.93904122641769, 40.85820201636388], [-73.93891471443493, 40.85840693742316], [-73.93865680565516, 40.858734030419306], [-73.93865306164695, 40.85876881362194], [-73.93850174725024, 40.859011335007715], [-73.9384372879185, 40.859073362121116], [-73.93841643396333, 40.85916624546453], [-73.93830957434292, 40.85938052360819], [-73.93785336586474, 40.85983027123292], [-73.93773556377965, 40.85993351223933], [-73.9375573431849, 40.86008135632522], [-73.93752944172567, 40.86009664128206], [-73.93753066305457, 40.860111878847555], [-73.9375521865698, 40.860123674582944], [-73.93758716318406, 40.86014954032659], [-73.93758054206488, 40.86020320665115], [-73.9373543613443, 40.86051512129023], [-73.93713577511515, 40.860783636699566], [-73.93709390407562, 40.86092524299121], [-73.93704223142178, 40.861057532978926], [-73.93696851748112, 40.86114321902511], [-73.93681096972892, 40.86125802374979], [-73.93680279963627, 40.86127908212909], [-73.93681851547501, 40.861315818680424], [-73.9368104544925, 40.86136346605991], [-73.93647561664903, 40.86192659117076], [-73.93596813006776, 40.86254409289871], [-73.93559460797813, 40.86296505185206], [-73.93485089173119, 40.86396557693492], [-73.93475540717563, 40.86409633974318], [-73.93461870076229, 40.86426987104493], [-73.93443446851097, 40.86450373173374], [-73.93435568513519, 40.86473290706842], [-73.9342316407922, 40.864813897822636], [-73.9341923030187, 40.864879107531024], [-73.93408173881329, 40.8649778101482], [-73.93401664426698, 40.86503554829155], [-73.93401648524471, 40.86519581766667], [-73.93395001711579, 40.865346746209795], [-73.93386283123469, 40.865410046111776], [-73.93378916436362, 40.86542863426855], [-73.93363055568597, 40.865709019423264], [-73.93357158007582, 40.865766750944246], [-73.9335739612204, 40.86584129553878], [-73.93349285424156, 40.8659456148249], [-73.93343635069695, 40.86598658199527], [-73.93343628914424, 40.86604807580756], [-73.93340182700808, 40.86613564683986], [-73.93332317557575, 40.86623809501412], [-73.93325688879918, 40.86624178458094], [-73.93313150185679, 40.86642621378959], [-73.93307500307841, 40.86655741872507], [-73.93283349428312, 40.86698204820547], [-73.93266332721451, 40.867237066436985], [-73.9327564862724, 40.86726436016792], [-73.93276260783126, 40.867252297557144], [-73.93290171891765, 40.86729304389972], [-73.93283958205069, 40.86741527691135], [-73.93264569574575, 40.86735847764239], [-73.93266035056361, 40.86732966135371], [-73.93259823617254, 40.867311462916476], [-73.93254851408642, 40.86741391805599], [-73.93251910022944, 40.86741057820754], [-73.93242547900691, 40.86756827092844], [-73.93238662235719, 40.86769389324469], [-73.93236431431154, 40.86786330100779], [-73.93235078136804, 40.86797912029534], [-73.93235431562957, 40.868007195697864], [-73.93236746809532, 40.86803721447595], [-73.93237341288611, 40.86806348807323], [-73.93227397128045, 40.86847715197786], [-73.9322812400666, 40.86853253815755], [-73.93226769826255, 40.86856231897239], [-73.9323228438442, 40.86857811188703], [-73.93231591011308, 40.868604142896054], [-73.93244643748234, 40.8686597654689], [-73.93249214298584, 40.86867751966221], [-73.93249377280186, 40.86867480942451], [-73.93253077796827, 40.86861327191389], [-73.93253690662307, 40.868603078375486], [-73.9325689181631, 40.86854984723809], [-73.93257970501192, 40.86853190982324], [-73.932614318916, 40.86847434737947], [-73.93262211514612, 40.86846138498572], [-73.93262525075036, 40.86845616797416], [-73.93262844350281, 40.8684572743171], [-73.93265083512807, 40.86846503348183], [-73.93254546521851, 40.86864025923291], [-73.93255050661638, 40.86864200559115], [-73.9325680426414, 40.86864808335343], [-73.93252354560028, 40.86872207835888], [-73.93250125163004, 40.86871435273154], [-73.93250619965718, 40.86870612539119], [-73.93249532884039, 40.868701950391426], [-73.9323522987469, 40.86864702816238], [-73.93230597278854, 40.86862923954242], [-73.93229830514184, 40.86864739503351], [-73.93225031754949, 40.86863950937264], [-73.93225427749094, 40.86868339935806], [-73.93227751053237, 40.868711157475076], [-73.93228732044285, 40.86887514239039], [-73.9323138544822, 40.86892372034371], [-73.9323603509378, 40.86894960979842], [-73.93245670800944, 40.86896227317821], [-73.93253729354952, 40.86896800236226], [-73.93256895482746, 40.86897940371653], [-73.93256795169869, 40.869044391473814], [-73.93250469803131, 40.869129558665186], [-73.93239016804613, 40.869187303109975], [-73.93229327689872, 40.8691995019422], [-73.93218903826057, 40.86936697638912], [-73.93246883069914, 40.86947609938703], [-73.93235370807281, 40.869755255296546], [-73.93226581636979, 40.86973477164008], [-73.93235934466009, 40.86950736321936], [-73.9321333660305, 40.86941597544102], [-73.93210997106698, 40.86947724910169], [-73.9320771261944, 40.869542684159946], [-73.93205908286353, 40.8695828168867], [-73.9320534384771, 40.86960075074436], [-73.9320500216973, 40.869641746782705], [-73.93206712211828, 40.8696994277864], [-73.93209566102911, 40.869759651145166], [-73.93217976024559, 40.86982236640027], [-73.9321985510756, 40.8698448708717], [-73.93156536999994, 40.86958215799991]]]}}, {\"id\": \"243\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 244, \"Shape_Leng\": 0.0805686044599, \"Shape_Area\": 0.000359702747951, \"zone\": \"Washington Heights South\", \"LocationID\": 244, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.94068822000003, 40.85131543299985], [-73.94068804199996, 40.85131542399991], [-73.94068785199995, 40.85131542399988], [-73.9404858699999, 40.851312708999906], [-73.94048575099985, 40.851312708999934], [-73.94048563299987, 40.85131271699995], [-73.94028476699981, 40.8513221499999], [-73.94010659000006, 40.85132952199992], [-73.94010645899996, 40.85132953199995], [-73.94010632899983, 40.85132953199994], [-73.93992663199994, 40.85132430499989], [-73.93992646699995, 40.851324295999895], [-73.939926313, 40.85132427799988], [-73.93974720899999, 40.851306264999906], [-73.9397471139998, 40.8513062549999], [-73.93974700799991, 40.85130623699991], [-73.93957073799986, 40.85127555599993], [-73.93939985299984, 40.85123275599991], [-73.93939969799992, 40.851232718999896], [-73.93939953299987, 40.851232664999955], [-73.93923603299991, 40.851178486999906], [-73.93828416699976, 40.85090854599985], [-73.93736531799983, 40.85064452999992], [-73.93666142799988, 40.85044732599994], [-73.93575394299994, 40.8501682029999], [-73.93639683100004, 40.84954119499985], [-73.93547108199992, 40.849144382999924], [-73.93423946399999, 40.84862621099995], [-73.93284984899991, 40.84803810999989], [-73.93139355399987, 40.84742823099985], [-73.93185010299997, 40.846802632999925], [-73.93191507399995, 40.84671052399989], [-73.93198004399996, 40.846618414999895], [-73.93201002999993, 40.84657596599987], [-73.93204002799989, 40.84653350699988], [-73.93207258099989, 40.84649567899991], [-73.93210093699993, 40.84645772999987], [-73.932186635, 40.84634306699992], [-73.93224030199994, 40.84626460299991], [-73.932325153, 40.84615390099991], [-73.9327702049999, 40.845541105999864], [-73.93315044200003, 40.84502194299991], [-73.93323248499983, 40.84490992199988], [-73.93371553899995, 40.84424637499989], [-73.93421198500002, 40.843568902999934], [-73.93467106399996, 40.84295724299987], [-73.93514017499996, 40.84231167399989], [-73.93556697199989, 40.841712527999924], [-73.93561105199986, 40.84160899499988], [-73.93561108599988, 40.84160889599986], [-73.93561111099991, 40.84160880699991], [-73.93563664999986, 40.84150019499987], [-73.93563666199992, 40.84150013999988], [-73.93564243699991, 40.84138944599988], [-73.93564244899994, 40.84138928299992], [-73.9356424249998, 40.8413891299999], [-73.93562830199987, 40.841279613999916], [-73.93562826699996, 40.84127936199993], [-73.93562818399985, 40.84127909999994], [-73.93559506299997, 40.841173399999924], [-73.93543637099998, 40.84075133099985], [-73.93529543300004, 40.84047481799991], [-73.93519180599998, 40.84033568799992], [-73.93519169999996, 40.840335543999885], [-73.93519161699993, 40.840335399999915], [-73.93510666600004, 40.84018372699994], [-73.93504376899986, 40.84002162099989], [-73.93504374499992, 40.840021566999916], [-73.93504373300003, 40.84002151299995], [-73.93500577799999, 40.83985200999991], [-73.93500575400004, 40.83985191099992], [-73.93500574300005, 40.83985180199987], [-73.93499462399996, 40.839678216999864], [-73.93499462399987, 40.83967813599992], [-73.93499463699986, 40.83967805499986], [-73.93501119299988, 40.83950397199993], [-73.93505508799984, 40.8393332359999], [-73.93512471799994, 40.839169636999905], [-73.93512477699991, 40.839169484999886], [-73.93512487199997, 40.83916933099994], [-73.93521769499985, 40.83901592099991], [-73.93521772999993, 40.83901586699992], [-73.93521776599987, 40.83901582199993], [-73.93533051999992, 40.838875039999934], [-73.93533061499988, 40.83887491299987], [-73.93533073299984, 40.83887479599988], [-73.93545962199988, 40.83874840399992], [-73.93563711299996, 40.838631419999956], [-73.93580416699986, 40.83850497699989], [-73.93595950299988, 40.838369897999904], [-73.93610200500004, 40.83822716899989], [-73.936230758, 40.838077855999884], [-73.93638316899992, 40.837850124999875], [-73.93669078700005, 40.83719324999992], [-73.93702353999993, 40.836468946999894], [-73.93738281099989, 40.83569072899993], [-73.937751162, 40.834890575999964], [-73.93786306199998, 40.83463774999996], [-73.93789804799981, 40.83454769599993], [-73.93793968599998, 40.834459289999856], [-73.93798783999989, 40.834372813999934], [-73.93804235799993, 40.83428854499991], [-73.93810306499991, 40.83420675399989], [-73.93816976599996, 40.83412770199994], [-73.93824224799982, 40.83405164199993], [-73.93832027799995, 40.833978817999906], [-73.93840360699998, 40.83390946299992], [-73.93849196799988, 40.8338437999999], [-73.9391548789999, 40.83327799299992], [-73.93927348599993, 40.833199648999916], [-73.93927371099988, 40.83319949599997], [-73.93927390099999, 40.83319934299988], [-73.93938209499981, 40.83311096299988], [-73.93938216599997, 40.83311089899993], [-73.93938222499987, 40.833110835999946], [-73.93947800199997, 40.83301364299994], [-73.93947814499981, 40.833013488999924], [-73.9394782749999, 40.833013327999886], [-73.93955977199981, 40.83290896699987], [-73.93955986799995, 40.83290884099993], [-73.93955995000002, 40.83290869699989], [-73.93962603899988, 40.832798852999865], [-73.93962606299988, 40.83279880799989], [-73.93962608699985, 40.83279875399988], [-73.93967601899988, 40.83268537099987], [-73.94007535699996, 40.83206722799985], [-73.94016848499993, 40.83195905599989], [-73.94024600800003, 40.83184212099991], [-73.94030591799994, 40.83171845099985], [-73.94034688799985, 40.831590404999865], [-73.94036830799996, 40.831460492999945], [-73.94037033300005, 40.8313312539999], [-73.94034643299983, 40.8304578419999], [-73.94136799599993, 40.83089083799994], [-73.942979662, 40.8315679799999], [-73.94571933699994, 40.832721412999874], [-73.94582395199988, 40.8327654609999], [-73.94592558199986, 40.83280648999989], [-73.94829108199995, 40.83386118099992], [-73.94890543299992, 40.83406529899992], [-73.94919448899992, 40.834210658999886], [-73.94941570599993, 40.83435858999992], [-73.94957431799986, 40.834419361999934], [-73.94964119799997, 40.83443432999993], [-73.94970807899995, 40.83444928999993], [-73.94977616299995, 40.834455964999904], [-73.94985590499991, 40.834459286999945], [-73.95015521048293, 40.83439675940791], [-73.94997968030412, 40.834875878605715], [-73.94988915620443, 40.83528472075901], [-73.94979826505212, 40.83548984121035], [-73.94978471338267, 40.83552043388431], [-73.9497425788605, 40.83562808782486], [-73.94973891528046, 40.83570565447781], [-73.94971243579302, 40.83577499028423], [-73.94969313831761, 40.835862580943484], [-73.94968349007459, 40.835921883370666], [-73.94968950571388, 40.836041139876336], [-73.94969117640214, 40.836074277497474], [-73.94968634744743, 40.836109863324964], [-73.94965628913988, 40.83619093997075], [-73.94964541816809, 40.836220254750174], [-73.94962137643174, 40.83623210360099], [-73.9496014970846, 40.83630509930472], [-73.94958334065868, 40.83648393783685], [-73.94954600729368, 40.83660437150179], [-73.9495099176216, 40.836665489687476], [-73.9494570191996, 40.83670561947379], [-73.94938852906095, 40.83671562194061], [-73.94932846081275, 40.836719250936476], [-73.94929034657056, 40.83672929248489], [-73.94928698598419, 40.836732775778955], [-73.94928446234258, 40.83673664541798], [-73.94928285052792, 40.83674078658172], [-73.94928219836629, 40.836745076392866], [-73.94928252520906, 40.83674938756381], [-73.94928382135787, 40.83675359217271], [-73.94928604835373, 40.83675756546006], [-73.94928914011717, 40.83676118952971], [-73.94929300490867, 40.83676435684791], [-73.94929752805186, 40.83676697343328], [-73.9493025753354, 40.836768961646385], [-73.94937713355948, 40.83678033441992], [-73.94943994696526, 40.83679169931687], [-73.94944230013112, 40.83679215701846], [-73.94946292070985, 40.836797746086475], [-73.94948204647808, 40.83680583985768], [-73.94949739893782, 40.836814981451575], [-73.94951081592383, 40.836825737294554], [-73.94952200610777, 40.83683787384183], [-73.94953069616135, 40.83685106989402], [-73.94953674996701, 40.83686508974796], [-73.94954003717058, 40.8368796315235], [-73.94918058897858, 40.837864010121635], [-73.9491523412929, 40.83793355255762], [-73.94900459767892, 40.83842558853353], [-73.9489289116868, 40.8386434262861], [-73.94883961981111, 40.83887563584406], [-73.94875763601367, 40.83902023748219], [-73.94865775026906, 40.8393013447156], [-73.94856054520493, 40.839564885664224], [-73.94849640289038, 40.83966759317245], [-73.94842860791495, 40.83986085465317], [-73.94833417923732, 40.83999733588596], [-73.948274498937, 40.84006354116475], [-73.94823352622083, 40.84011759626999], [-73.94807484255449, 40.8404662594779], [-73.94796701014049, 40.84065410819866], [-73.94786003452222, 40.84089601309009], [-73.94776196087692, 40.841125756833804], [-73.94770934809952, 40.84126360686614], [-73.94760600401781, 40.841409547682304], [-73.94755609818566, 40.8415286673706], [-73.94748896929306, 40.8416974638817], [-73.94738322154217, 40.84194082630449], [-73.94727430439123, 40.84205178297], [-73.94721047555673, 40.84210112669678], [-73.94716788222361, 40.842159099043336], [-73.94713367972246, 40.84225862411276], [-73.94712053311035, 40.84228559744328], [-73.9471071517441, 40.842306787578885], [-73.94696012382835, 40.84253957012679], [-73.94693502862746, 40.84261919850936], [-73.94686918645269, 40.842704823396836], [-73.94678768212576, 40.84281512747888], [-73.94675510074843, 40.84288846101302], [-73.94670984756608, 40.8429441246515], [-73.94661642596199, 40.84302009833804], [-73.94654808278268, 40.84311998115941], [-73.94645858066579, 40.843254384265926], [-73.94636919991518, 40.84345477995481], [-73.94631349566346, 40.84355977043898], [-73.9462489191819, 40.84367120457853], [-73.94619788899398, 40.843754098967956], [-73.94612625578611, 40.84388856876928], [-73.94612416421772, 40.84389249655712], [-73.94611933577549, 40.84400473749403], [-73.94609745833881, 40.844055052047274], [-73.94609415172181, 40.84409735524896], [-73.94611365510724, 40.844196277526095], [-73.94611699818816, 40.8442249656692], [-73.94609316413097, 40.84429114865286], [-73.94610537772549, 40.8443497221328], [-73.94608425805812, 40.84446818196512], [-73.94609124077483, 40.84449880134603], [-73.94607880557066, 40.844597670328326], [-73.94610929980433, 40.84485857499071], [-73.94612272152777, 40.844879554715824], [-73.94612836835266, 40.84491180766122], [-73.94614692398876, 40.84493975742343], [-73.94616018722793, 40.84502984491951], [-73.94616086907139, 40.84512582705777], [-73.94619138728324, 40.8451979727793], [-73.9462057455364, 40.84523179311284], [-73.94622271166769, 40.84525916634131], [-73.9462106518648, 40.84533362513183], [-73.94617704845383, 40.84536591521315], [-73.94617174983877, 40.845392891580666], [-73.94620682406678, 40.845461745738916], [-73.94623062385415, 40.84557698074459], [-73.94627664957967, 40.84569065991839], [-73.94630774814597, 40.845740934326024], [-73.94643878255472, 40.8458859229234], [-73.94646620992474, 40.845881260909486], [-73.94647950512845, 40.845919778073025], [-73.9464454448152, 40.84592148809303], [-73.94648387687232, 40.845964687313604], [-73.94651730200567, 40.845995184403726], [-73.9465038958519, 40.84602819976756], [-73.94645597912805, 40.84605816509614], [-73.9464258195365, 40.84606333862832], [-73.94634839459351, 40.84615644860679], [-73.9463507804993, 40.84624033128926], [-73.94640288265272, 40.846361291622344], [-73.94643643817498, 40.846439919994204], [-73.94647353057901, 40.846528618163475], [-73.94651683205107, 40.84659313985066], [-73.94664058210437, 40.84668489453904], [-73.94671767879166, 40.84682475824712], [-73.94678956849305, 40.84699674934532], [-73.94682445342639, 40.84713426206979], [-73.94680935266598, 40.84728844663744], [-73.94678220543965, 40.84735338919667], [-73.9467493384277, 40.84740595933717], [-73.94669858704907, 40.847497393872366], [-73.9466857685378, 40.847728395044264], [-73.94665606434612, 40.847883320483525], [-73.9466405820577, 40.847939292150464], [-73.94662285734358, 40.84814551949148], [-73.94662027748582, 40.848214017831104], [-73.94661949998884, 40.8482784259119], [-73.94664422572272, 40.84839933388536], [-73.94665040763155, 40.848667604216814], [-73.9466785376539, 40.848837459092984], [-73.9466355001267, 40.848912425059694], [-73.94665064310523, 40.84900501041168], [-73.9466649962483, 40.849045694100845], [-73.94670601762992, 40.84909437729706], [-73.94672051176013, 40.84923178391935], [-73.94673313406388, 40.8492674972145], [-73.94675656427756, 40.84930266769501], [-73.94681164673382, 40.84935193378878], [-73.94683872036457, 40.84938282135098], [-73.94685525004344, 40.84942768019967], [-73.9468547984756, 40.849539471265004], [-73.94688244417446, 40.84959284156913], [-73.94689505793289, 40.84963945308073], [-73.94690063210635, 40.84967414925802], [-73.94696652237195, 40.84981517403269], [-73.9469592294381, 40.849848958490455], [-73.94695108849987, 40.84985916446554], [-73.94695222118273, 40.8498900983644], [-73.94699578658114, 40.84995592429224], [-73.94695814474174, 40.84998525944927], [-73.94696706524539, 40.84999566592066], [-73.94698516576905, 40.84999453307053], [-73.94702782494728, 40.85004183181243], [-73.94702536171071, 40.85006211459108], [-73.94700527357732, 40.85008186727115], [-73.94697447146315, 40.850089895081226], [-73.94695342915061, 40.850097735222064], [-73.94696404495703, 40.8501116557755], [-73.94698645041183, 40.850118830885414], [-73.94701781699507, 40.85012699598264], [-73.94702897203565, 40.85014476043616], [-73.9470192100042, 40.850195092735596], [-73.94704980259199, 40.850223349228315], [-73.9470439927834, 40.85025474598785], [-73.94700406576864, 40.8503293287002], [-73.94700459608349, 40.85034406799342], [-73.9470111027366, 40.850356292656755], [-73.94700167436984, 40.850386520112636], [-73.9469980830223, 40.850398030842655], [-73.9469644175959, 40.85046552581819], [-73.94692626452422, 40.850528055834154], [-73.9458970349999, 40.85032798199988], [-73.944914825, 40.85013703799994], [-73.944768533, 40.85011040299986], [-73.94472103299982, 40.850100962999875], [-73.94409966200001, 40.849977402999855], [-73.94396593999994, 40.849950809999946], [-73.94381229899996, 40.84991964099994], [-73.94317806099984, 40.84979994399989], [-73.94316977700004, 40.849847511999904], [-73.94316093899997, 40.84993545899992], [-73.94315143499992, 40.85000403399993], [-73.94311799599984, 40.85023724299993], [-73.94309647899989, 40.850387302999884], [-73.94298963999995, 40.8508185039999], [-73.94290346199983, 40.85099716399988], [-73.94279927299996, 40.85117167299994], [-73.94267766599995, 40.85134048199992], [-73.9425395099999, 40.851502104999895], [-73.94247684499997, 40.85156459099987], [-73.94238599199987, 40.85165518499987], [-73.94221851099991, 40.85179850399986], [-73.94203871799984, 40.851931028999914], [-73.94185191299992, 40.851821335999944], [-73.94126220299982, 40.8513919619999], [-73.9410889599999, 40.85135745699992], [-73.94108888899997, 40.851357438999905], [-73.94108881800004, 40.851357429999894], [-73.94088990599982, 40.85133040399996], [-73.94088972799987, 40.85133037599988], [-73.94088954899996, 40.85133036799992], [-73.94068822000003, 40.85131543299985]]]}}, {\"id\": \"244\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 245, \"Shape_Leng\": 0.0959830596604, \"Shape_Area\": 0.000466175414294, \"zone\": \"West Brighton\", \"LocationID\": 245, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.09787969199995, 40.64035805499988], [-74.09767290799995, 40.63896067899989], [-74.09686242099997, 40.63900747099991], [-74.09637906299997, 40.63903536999989], [-74.09570924799992, 40.63909864099989], [-74.09409312599998, 40.639266913999975], [-74.09386276399991, 40.63802491699991], [-74.09362634099998, 40.63728369499989], [-74.09338942299995, 40.63659955499988], [-74.09242513499994, 40.63558181299992], [-74.09182025899999, 40.63555529799987], [-74.09090698799996, 40.635485728999875], [-74.09027738599993, 40.63549902999994], [-74.08970378299989, 40.63568250099992], [-74.08961180999991, 40.63574147799996], [-74.08952415699996, 40.635804156999875], [-74.08944108199995, 40.63587035599987], [-74.08936282699987, 40.63593987999993], [-74.089289621, 40.63601252499995], [-74.08922167799987, 40.63608807999989], [-74.08915919799995, 40.63616632399986], [-74.08910236299987, 40.63624702699983], [-74.08891735099989, 40.636528383999874], [-74.08870582299987, 40.63684520599986], [-74.08759775199998, 40.636262023999926], [-74.0869059279999, 40.635857242999904], [-74.08587882899995, 40.63669158999991], [-74.08568983299989, 40.6366072799999], [-74.08513463099995, 40.63635959899995], [-74.08373397299994, 40.63573714399991], [-74.08394671399999, 40.63565410499991], [-74.08443647599991, 40.63540524799989], [-74.08574928999992, 40.633955556999865], [-74.08714129599997, 40.63223951599987], [-74.08761781699995, 40.63164751199986], [-74.08807764899994, 40.63108071699992], [-74.08820728099995, 40.63090392499991], [-74.08835621399997, 40.63073496399992], [-74.08852313599994, 40.63057561499994], [-74.08870635899996, 40.63042749999993], [-74.08890387699988, 40.63029198999987], [-74.08902975899994, 40.6302131689999], [-74.08914567599997, 40.630124621999904], [-74.08924974899998, 40.63002750999987], [-74.08934045699996, 40.62992328499993], [-74.08941665299993, 40.629813551999895], [-74.09158095999993, 40.626545922999924], [-74.09223971399994, 40.62574291599987], [-74.09395327099995, 40.624333906999894], [-74.09533507499991, 40.62320160699987], [-74.09639991999997, 40.62232899699995], [-74.09751734399991, 40.621418887999866], [-74.09814034099995, 40.62091145599987], [-74.09967734799997, 40.61966605399991], [-74.10004426999996, 40.61935795499992], [-74.10039572599996, 40.619038962999866], [-74.10073089199992, 40.61870975499989], [-74.10104903199996, 40.61837105999995], [-74.10134946699999, 40.61802367099989], [-74.10189835099996, 40.61745221099992], [-74.10254940599991, 40.61685368499989], [-74.10350925299994, 40.61641607499991], [-74.10458409200001, 40.61761290099989], [-74.1049532439999, 40.618022562999876], [-74.10511622199992, 40.618203003999945], [-74.10518709699997, 40.618281473999915], [-74.10552598899993, 40.61859232899992], [-74.10608939599997, 40.618984194999925], [-74.10734089099991, 40.61954307699993], [-74.10770635299994, 40.61969949699991], [-74.1091080619999, 40.62029623899989], [-74.11001550699994, 40.62068834399986], [-74.11094392899996, 40.621194488999876], [-74.11153393899995, 40.621603645999855], [-74.11235088099993, 40.62226529199995], [-74.1132624959999, 40.62305442499995], [-74.11358478199992, 40.62369716899985], [-74.11393395699994, 40.62433020399988], [-74.11407428699995, 40.62458176999998], [-74.11471448499998, 40.625811685999906], [-74.11473772799998, 40.6258563459999], [-74.11475499199994, 40.625889287999875], [-74.11503453399992, 40.62642257599991], [-74.11524316499992, 40.62682656499988], [-74.11526026599996, 40.62685967799992], [-74.115275528, 40.626889055999875], [-74.11544777599987, 40.62722061799991], [-74.11560933499997, 40.62754898299993], [-74.11588524899992, 40.628088936999895], [-74.11595162399986, 40.628230133999864], [-74.11614321199995, 40.62870226399988], [-74.11618948699994, 40.62894119699992], [-74.11634635499993, 40.629835003999844], [-74.11650496399994, 40.63075571399985], [-74.11666642599987, 40.6317101019999], [-74.11683619599992, 40.63267481799986], [-74.11695838499996, 40.633351407999896], [-74.11704470999996, 40.633816609999904], [-74.11707785799997, 40.634007107999906], [-74.1171994709999, 40.634568097999924], [-74.1173836129999, 40.63539539899992], [-74.1174598659999, 40.63576940499993], [-74.11746820599997, 40.63588019599996], [-74.11747417299995, 40.635943517999884], [-74.11747983199996, 40.63602690599986], [-74.11748576400001, 40.6361134059999], [-74.11749409699998, 40.63622409099987], [-74.1175003539999, 40.63630719399988], [-74.11751136899993, 40.636441801999894], [-74.11751731899994, 40.6365325329999], [-74.11752704599999, 40.63666172499988], [-74.11753352399988, 40.63674776299993], [-74.11754902299995, 40.63695363199987], [-74.11765722499997, 40.63806246599996], [-74.11709261399987, 40.63812694699992], [-74.11641377299995, 40.63820248399986], [-74.11588274199994, 40.63826439099994], [-74.11519147399999, 40.63834768099987], [-74.1149955969999, 40.63729222599993], [-74.1137364029999, 40.63742912299987], [-74.11279390199998, 40.637530809999916], [-74.11174806399994, 40.63764295499989], [-74.11080857499996, 40.637746199999874], [-74.10974833299994, 40.63786583299993], [-74.10954135499992, 40.637987429999896], [-74.10938968899991, 40.638066945999896], [-74.10938961799994, 40.63806698199992], [-74.10938954799997, 40.63806700899991], [-74.109225354, 40.63813328099991], [-74.10922521199993, 40.638133334999935], [-74.10922508199995, 40.638133370999896], [-74.10905134699988, 40.63818482099989], [-74.10905120499997, 40.638184865999904], [-74.10905105199996, 40.638184892999895], [-74.10887129699995, 40.638220507999904], [-74.1088711309999, 40.6382205439999], [-74.10887095399988, 40.638220561999894], [-74.10868871399997, 40.63823996099989], [-74.10822759399989, 40.638314039999905], [-74.10746323199996, 40.638438487999956], [-74.1073365579999, 40.63854326299991], [-74.10729545999992, 40.63858007999987], [-74.10724963099997, 40.63861349899986], [-74.10719955199997, 40.63864316999993], [-74.10714575, 40.63866877999992], [-74.10708878899997, 40.638690059999874], [-74.10702926799993, 40.63870678799987], [-74.10696781199987, 40.63871878699992], [-74.10690506699991, 40.63872593199991], [-74.10542286599998, 40.63889896699986], [-74.10480990899997, 40.638976700999876], [-74.10393828599992, 40.63919020399988], [-74.10305261099988, 40.639401245999856], [-74.10216582799997, 40.63961813399992], [-74.10129653399997, 40.639827157999925], [-74.10042458199996, 40.6400410299999], [-74.09990409699999, 40.64016295499988], [-74.0992883379999, 40.64020472299996], [-74.09787969199995, 40.64035805499988]]]}}, {\"id\": \"245\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 246, \"Shape_Leng\": 0.0694671937437, \"Shape_Area\": 0.000281298327076, \"zone\": \"West Chelsea/Hudson Yards\", \"LocationID\": 246, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00439976203513, 40.76267135909888], [-74.00220082073264, 40.761728432226434], [-74.00178824022305, 40.76229452064428], [-74.00153309083484, 40.762644598063744], [-74.00133748499992, 40.76220852199988], [-74.00127796299991, 40.76207582499987], [-74.0011505409999, 40.76202581199986], [-74.00103587499999, 40.76196835199993], [-73.99831474500002, 40.76082320899991], [-73.99623223099987, 40.759946031999924], [-73.99547450199992, 40.759626856999894], [-73.9959423659999, 40.758980978999965], [-73.99640941599995, 40.75833915899993], [-73.99663529299985, 40.75801292199992], [-73.996800897, 40.757789159999945], [-73.99685244899985, 40.75773270899988], [-73.99692177899989, 40.75763758199988], [-73.997233738, 40.7572095429999], [-73.99730671399993, 40.75711050699996], [-73.99775836599989, 40.75648752899987], [-73.99820870599997, 40.75586865599991], [-73.9986606719999, 40.75525176299991], [-73.99914292099987, 40.754584933999865], [-73.99963228999997, 40.75391808499994], [-73.99992459199981, 40.75351627999993], [-74.00001529900001, 40.75339158699995], [-74.00052667499992, 40.75268862399987], [-74.00086139300001, 40.75221483899992], [-74.000974548, 40.752054664999896], [-74.0014297059999, 40.7514457209999], [-74.00188266699993, 40.75082793099988], [-74.00233168699992, 40.7502082579999], [-74.002431475, 40.75007275199989], [-74.002458683, 40.75003580599991], [-74.00278526999996, 40.74959231999985], [-74.00312806199997, 40.749121859999946], [-74.00316116799988, 40.749076422999906], [-74.003235611, 40.74897425499992], [-74.00368344599993, 40.74835499999993], [-74.00416830599991, 40.7476892919999], [-74.0046527429999, 40.74702295299989], [-74.0051071349999, 40.746402772999865], [-74.00553403899988, 40.74581556899987], [-74.00595977599997, 40.74523098499991], [-74.00638477199996, 40.74464590299987], [-74.00681149999996, 40.74406132399991], [-74.00696604399992, 40.74384701399987], [-74.00723774599994, 40.743470231999865], [-74.00770461499994, 40.74283381299992], [-74.00789568799996, 40.74257005099984], [-74.00820401899988, 40.74214751799986], [-74.00869902299993, 40.742361722999924], [-74.00879080599995, 40.742375143999915], [-74.00889339799991, 40.742384166999926], [-74.00902439199994, 40.742399742999964], [-74.00916608208487, 40.74241659123162], [-74.0090915684619, 40.74288614957498], [-74.0092401357503, 40.742899518409985], [-74.00924074160105, 40.74291316086339], [-74.00962015348915, 40.74295124889236], [-74.00956843411761, 40.74325778472634], [-74.00958464345709, 40.743263520483076], [-74.01211940028666, 40.74351661218259], [-74.01213726936172, 40.74353192705926], [-74.01214813183589, 40.743559301547556], [-74.0120900828902, 40.74389763764887], [-74.01206903486668, 40.74392249354837], [-74.01202085006405, 40.74393977020892], [-74.01197763342863, 40.74393556042096], [-74.00951201767357, 40.743676093760264], [-74.0094886836422, 40.74369568117951], [-74.00943800870867, 40.7439712559757], [-74.0093942693211, 40.743968716838154], [-74.00904207504226, 40.7439338712551], [-74.00890032962846, 40.74391792709608], [-74.00880283523892, 40.74443934762423], [-74.0087278005888, 40.74485823290996], [-74.00872986359411, 40.74488153782966], [-74.00893975844279, 40.74490921733238], [-74.00957440612915, 40.74499290866285], [-74.00961227924408, 40.74499790315053], [-74.00955832929739, 40.745288825298694], [-74.00991660813445, 40.745327255007005], [-74.0099343667769, 40.74531707810186], [-74.00993796184929, 40.74529670191295], [-74.00994341816717, 40.74526576973634], [-74.00995048769146, 40.7452256903209], [-74.00995852601345, 40.745180119646584], [-74.00996759959413, 40.74512868110846], [-74.0099735068339, 40.7450951941157], [-74.00998135739337, 40.74505068238846], [-74.0099891727315, 40.745006384762036], [-74.0099982260172, 40.744955059302036], [-74.00998685784812, 40.74494416421516], [-74.00986948474724, 40.74493391770714], [-74.00987211085476, 40.74491650212583], [-74.01016790535392, 40.74494232970965], [-74.01016465197365, 40.74496392462974], [-74.01005375656213, 40.744954234929914], [-74.01003848384612, 40.744963458078104], [-74.01002361252539, 40.745046345317995], [-74.01001404323704, 40.74509967661553], [-74.01000814033448, 40.74513257676554], [-74.00999930170602, 40.74518183979107], [-74.00999250524056, 40.7452197190643], [-74.00998320132649, 40.74527157308871], [-74.00997743638763, 40.745303703608926], [-74.0099739502191, 40.74532313510926], [-74.0099909538001, 40.745336686673895], [-74.01046920011858, 40.74538676602893], [-74.0104883559496, 40.74537571509468], [-74.0104918741746, 40.745355167013756], [-74.0104969344119, 40.74532561543082], [-74.01050332781186, 40.7452882709089], [-74.01051171655337, 40.74523927525828], [-74.01052083436335, 40.745186021358805], [-74.01052681665274, 40.74515108070604], [-74.01052968455852, 40.745134332512635], [-74.01055044755888, 40.745013057710004], [-74.01053188168753, 40.74500298186922], [-74.01041700660251, 40.744993582703295], [-74.01041937209395, 40.74497682467356], [-74.01076419780681, 40.74500503202081], [-74.01076156046132, 40.745023726183454], [-74.0106062154018, 40.74501101647424], [-74.01058579200937, 40.74502044656155], [-74.01056801091896, 40.74513793282638], [-74.01056370849734, 40.74516636225321], [-74.01055056928816, 40.7452531820778], [-74.01053978205333, 40.745324455419386], [-74.01053373534734, 40.745364407457025], [-74.010531065377, 40.745382050389416], [-74.01054520392145, 40.74539213501163], [-74.01110359804879, 40.745447670243976], [-74.01111624163278, 40.74543612462538], [-74.01112466229964, 40.74537776220242], [-74.01113010029036, 40.745343228325524], [-74.01113887006922, 40.7452875486318], [-74.01114545795524, 40.74524572071857], [-74.01115627315946, 40.745177047156155], [-74.0111611341844, 40.745146176461084], [-74.01117183119814, 40.74507825828334], [-74.01115328709845, 40.745068643572814], [-74.0110428174822, 40.74506096342358], [-74.0110445915082, 40.74504621240204], [-74.01138769974187, 40.745073112682675], [-74.01138567845328, 40.74508800864402], [-74.01123038057233, 40.74507583116526], [-74.01121085294035, 40.74517080368261], [-74.01120592447445, 40.74519477063079], [-74.01118816871931, 40.745281126091655], [-74.0111810296498, 40.7453158494414], [-74.01116675082783, 40.745385297981905], [-74.01115419176905, 40.74544637808387], [-74.01116709976122, 40.745460352539965], [-74.01173958552816, 40.74551991819018], [-74.01175140766142, 40.74544238769607], [-74.01175495761667, 40.7454190976719], [-74.01176281151672, 40.74536759428581], [-74.01177255159179, 40.745303706497886], [-74.01177404681451, 40.745293903213344], [-74.01178757177937, 40.74520520356581], [-74.01179527645724, 40.74515466698494], [-74.0117827350081, 40.745134956631], [-74.01166900853472, 40.74512733119143], [-74.01167067583802, 40.74511397784238], [-74.01183642872407, 40.74512596367872], [-74.01177025765051, 40.74555278454016], [-74.0117663544952, 40.74557798004347], [-74.0117827582808, 40.745601572255396], [-74.01171897530907, 40.74593984499781], [-74.0117112349654, 40.74596125915166], [-74.01167885505592, 40.745979461815935], [-74.01163451251718, 40.74597839430353], [-74.01055524902252, 40.74587110492542], [-74.01055133475721, 40.74590288399968], [-74.01012821388646, 40.745857238928934], [-74.01012821162075, 40.745842335463074], [-74.00994668462272, 40.74581951444566], [-74.00994807913845, 40.74580017091193], [-74.00946389855962, 40.74575014365948], [-74.00945258331889, 40.745817656644206], [-74.00944206850693, 40.745880389871516], [-74.00943615127545, 40.745915693320605], [-74.00942517623243, 40.7459811773634], [-74.00941984903601, 40.74601296304771], [-74.00940851345764, 40.746080594635856], [-74.00940272604319, 40.746115130634095], [-74.0093920495925, 40.74617882395999], [-74.00938671069996, 40.746210681009], [-74.00938198694908, 40.74623886784371], [-74.00937801611612, 40.74626255972148], [-74.00936550495776, 40.746337200862094], [-74.0115282185556, 40.74657522476392], [-74.01156206111087, 40.74658699142647], [-74.0115852836195, 40.74660905095146], [-74.01158238743089, 40.74663553579254], [-74.01152538638327, 40.74697314077381], [-74.01150411235528, 40.74699080250515], [-74.01147606280952, 40.74699668563957], [-74.01143737944645, 40.74699227651282], [-74.00924314592588, 40.74676279075376], [-74.0091373625992, 40.74735976280298], [-74.00956047962376, 40.74740279960449], [-74.00956103476226, 40.74738773359524], [-74.00956765300351, 40.747387877453264], [-74.00956826626553, 40.74737108340655], [-74.0095503986723, 40.74737070611278], [-74.0095515541536, 40.74733934176787], [-74.00996384996868, 40.74737780301255], [-74.0099583844545, 40.7474116720828], [-74.00990669970031, 40.74740684960846], [-74.00990158912954, 40.7474384843489], [-74.0113567533772, 40.747592418346755], [-74.01139477623487, 40.74761090210864], [-74.01140673935517, 40.74763055057068], [-74.01134693158455, 40.747976205492634], [-74.01133174441699, 40.748000575341926], [-74.01130352993741, 40.748014571908875], [-74.0112661125396, 40.74801371918114], [-74.0090584790135, 40.74778564533286], [-74.00894606502312, 40.74841189316866], [-74.00897918405973, 40.74841270962695], [-74.00897918237624, 40.74840021938044], [-74.00907100927546, 40.74840976720642], [-74.00907390770347, 40.748395061700776], [-74.00941511636573, 40.74842958691727], [-74.00941607672524, 40.748436935037866], [-74.00952336741506, 40.748447956808185], [-74.00951882360114, 40.74847354043597], [-74.01116687561546, 40.74864504676614], [-74.01119472751145, 40.74865681411392], [-74.0112156181341, 40.74867799224354], [-74.01121562130406, 40.74869682117583], [-74.01114914281891, 40.74902574726484], [-74.01113289942975, 40.749041633830956], [-74.01110891182984, 40.74904987642213], [-74.01107797059221, 40.74905164311724], [-74.00994817938025, 40.74893198800576], [-74.00956004978684, 40.749935041791524], [-74.00972257401406, 40.74997202900862], [-74.00972771074939, 40.74996378830078], [-74.01006544547259, 40.75006803825169], [-74.01050182205222, 40.75020981164461], [-74.01075118603501, 40.750293977204855], [-74.010808158009, 40.75031931136799], [-74.01075658033923, 40.75041085295745], [-74.01069531261547, 40.75039451481687], [-74.01048679093364, 40.75032668876917], [-74.00968282368922, 40.75006275779379], [-74.00969350260826, 40.75004393617553], [-74.0095424804557, 40.75000094902341], [-74.00951808686504, 40.750107004781405], [-74.00947001922506, 40.7500811641566], [-74.00947559600827, 40.75007515771969], [-74.00925340917918, 40.74998520622449], [-74.00911896644935, 40.75033244243161], [-74.0090009825946, 40.75064818378257], [-74.00870618650575, 40.75142680500339], [-74.00873137829987, 40.75146589496972], [-74.00866729071824, 40.751628197220754], [-74.00862089839437, 40.751652577922414], [-74.00853256925012, 40.75188360930431], [-74.00876004993509, 40.75194801470975], [-74.0091408211999, 40.752097594098984], [-74.00912661123239, 40.752116895734865], [-74.00874567715042, 40.75196439532701], [-74.00865115936809, 40.751939512113076], [-74.00864318237626, 40.75195650203188], [-74.00896912527908, 40.752076791478736], [-74.008957422948, 40.75209022513094], [-74.00903895969972, 40.75212515615248], [-74.00904909796255, 40.75211834944301], [-74.00910757975154, 40.752139712111855], [-74.00910596820003, 40.75214212494943], [-74.00905774009662, 40.75221435124321], [-74.00899734911651, 40.75218910594712], [-74.00900762808337, 40.75217684273936], [-74.00892887650465, 40.75214400187013], [-74.00891472422607, 40.75216362847581], [-74.00867223768384, 40.752062250000726], [-74.00859053101694, 40.752028105434356], [-74.00858095760466, 40.75204388241159], [-74.0086645656199, 40.75207286737424], [-74.00934089245268, 40.75235560954557], [-74.00932795206235, 40.752373503046165], [-74.0084789557279, 40.75201857191992], [-74.00842636447226, 40.75215785023921], [-74.00881779819426, 40.75232434585419], [-74.0088027370307, 40.7523448540372], [-74.0088394166214, 40.75235881653463], [-74.00885173565295, 40.75234228177297], [-74.00986566961525, 40.75276341313934], [-74.00984627922618, 40.7527854419594], [-74.00988922619945, 40.75280368052555], [-74.00990631519595, 40.75278041119838], [-74.0099905581974, 40.75281618714907], [-74.00992010631788, 40.75291212601695], [-74.00997058797012, 40.75293356415314], [-74.00991916275483, 40.753003591459176], [-74.00978913116431, 40.75294837008777], [-74.00988774868313, 40.75281407765329], [-74.00983854674402, 40.752794967983014], [-74.00983056913974, 40.752803464244515], [-74.00882994505388, 40.75238001600713], [-74.00883632714992, 40.752371521109524], [-74.00879210090841, 40.7523521383795], [-74.00868792219562, 40.75249399862838], [-74.00835352838844, 40.752350739342255], [-74.00801152240543, 40.753256452612064], [-74.0079038694304, 40.753541538745814], [-74.00773968687764, 40.7539714956911], [-74.00762013296321, 40.754281475520166], [-74.00774478605632, 40.754336542719415], [-74.00759664325838, 40.75453044968608], [-74.00747193016366, 40.75447681451166], [-74.00729394650305, 40.754723184450846], [-74.00728499944329, 40.75473556820319], [-74.00741362036896, 40.7547893026694], [-74.00727181174757, 40.754985594535206], [-74.00714468360962, 40.754932481173356], [-74.00695917416672, 40.75518079484938], [-74.00680635763678, 40.75539177564034], [-74.00670291840642, 40.7555323674909], [-74.00656620007939, 40.75572286294469], [-74.00620548925966, 40.75622084397099], [-74.00609900474534, 40.756367850497476], [-74.00595462291497, 40.756575842184745], [-74.00591251611402, 40.756636499969474], [-74.00580062242977, 40.756801694277364], [-74.00541885841004, 40.757323140851746], [-74.00521187705773, 40.757231998539886], [-74.00480392560178, 40.75780984316466], [-74.00496777078648, 40.757884164447944], [-74.00527894797887, 40.75802855492821], [-74.00503105136634, 40.75835701647307], [-74.00696428749305, 40.7591600575616], [-74.00699375268461, 40.75917614764261], [-74.00701038304956, 40.75920372038661], [-74.00700358904015, 40.759231870341075], [-74.0065295998783, 40.75987475124118], [-74.00650687640179, 40.759892032317126], [-74.0064758837195, 40.75989360126805], [-74.00644352901304, 40.75989046096713], [-74.00631515935251, 40.759842822044654], [-74.00450483093914, 40.75908499655263], [-74.0042062912538, 40.759501707688415], [-74.00390742032765, 40.75937569218354], [-74.00380657835008, 40.75951073146594], [-74.0040798895684, 40.75962604474266], [-74.00424697128052, 40.75969921417133], [-74.00420247432244, 40.75976107188177], [-74.00414863814689, 40.75983591417084], [-74.00424487588268, 40.75988657316653], [-74.00425967856711, 40.75987530767853], [-74.00431889425072, 40.759910685763444], [-74.00426869650676, 40.75995927901334], [-74.00421059345649, 40.75992456661], [-74.00422266066995, 40.75991310304368], [-74.00413171622804, 40.75986164239597], [-74.00411162007795, 40.7598930073072], [-74.00366891453531, 40.75970596957665], [-74.00357251040072, 40.759846558973116], [-74.00392854933564, 40.760003019720415], [-74.00382559595194, 40.760133136666866], [-74.00358800911194, 40.760451706951315], [-74.0033848822989, 40.760728870706316], [-74.0031333899139, 40.76107834509637], [-74.00278691552613, 40.760928969703315], [-74.00233528130057, 40.76154372700156], [-74.00454520983433, 40.76249280194477], [-74.00455814361395, 40.76250787607542], [-74.00455641560922, 40.762524265025135], [-74.00444951527267, 40.76266456720002], [-74.00442709659245, 40.76267570735634], [-74.00439976203513, 40.76267135909888]]]}}, {\"id\": \"246\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 247, \"Shape_Leng\": 0.0929678582271, \"Shape_Area\": 0.000205617157843, \"zone\": \"West Concourse\", \"LocationID\": 247, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.91222180499983, 40.84235659099988], [-73.91240381699996, 40.84181649699988], [-73.91246930299995, 40.841620256999924], [-73.9126225619999, 40.84115715899991], [-73.91269769199985, 40.840930128999894], [-73.9127782319999, 40.840691064999945], [-73.912960009, 40.84019622899994], [-73.91313156299994, 40.83981053999988], [-73.91371372699993, 40.83880893499991], [-73.91377104999991, 40.838721647999925], [-73.91383125899985, 40.83863904499986], [-73.91423754099989, 40.83810879599992], [-73.91477300600005, 40.8375141009999], [-73.9153159669999, 40.83694742399996], [-73.91587067499995, 40.836393593999915], [-73.91647433399987, 40.835779030999895], [-73.91667316899982, 40.83556740499994], [-73.91720735700002, 40.83502151399988], [-73.9176438849999, 40.83457540699993], [-73.91767974800003, 40.83453875599988], [-73.91770299099986, 40.83451500899989], [-73.91773115000002, 40.83448623099987], [-73.9177866579999, 40.83442951199987], [-73.91839346699996, 40.833809411999866], [-73.91903136599996, 40.83308734899991], [-73.91949947100001, 40.8324794919999], [-73.91986089399987, 40.83194722699992], [-73.9204611219999, 40.830914820999936], [-73.92117904700004, 40.82962895599993], [-73.9213893, 40.82923701399989], [-73.92174660200004, 40.82860240899991], [-73.92183738200004, 40.82843621099994], [-73.92211960699997, 40.82792587599993], [-73.922233014, 40.82772080999991], [-73.92264951599996, 40.826972409999904], [-73.92267670999986, 40.82684999599992], [-73.92273725099989, 40.826718249999935], [-73.9228330809999, 40.826507101999944], [-73.9229413879998, 40.82629934999992], [-73.92306184099985, 40.82609561099991], [-73.92347379199992, 40.82541321999991], [-73.924313102, 40.82399409699991], [-73.92437413499995, 40.8238206829999], [-73.92442266399998, 40.823644865999924], [-73.92445847599993, 40.823467428999955], [-73.92456530199985, 40.823145454999896], [-73.92469079500003, 40.82282671599988], [-73.9248346779998, 40.82251216799988], [-73.92499654899987, 40.82220270099992], [-73.92517593199985, 40.821899196999944], [-73.92537221999993, 40.82160251999987], [-73.92573567299983, 40.8211742359999], [-73.92565452999989, 40.82112523299984], [-73.92610404099997, 40.820745970999944], [-73.92628127099995, 40.820537159999944], [-73.92634345699999, 40.82057855199989], [-73.92662466400002, 40.820026393999946], [-73.92687781699988, 40.81947887199991], [-73.92731653099999, 40.81855963399991], [-73.92775576099996, 40.81757238099987], [-73.92821699000002, 40.81653564699991], [-73.92885642099992, 40.81487986599991], [-73.9289656749999, 40.81482825499994], [-73.92919204499995, 40.814496010999946], [-73.92921297999986, 40.81438195799993], [-73.93027812699984, 40.813248024999965], [-73.93039605099992, 40.81330374699994], [-73.93056964999984, 40.8133732979999], [-73.93104478900003, 40.813588441999876], [-73.93115852299998, 40.813635488999914], [-73.9312360049999, 40.813666728999884], [-73.93130722899988, 40.81369873899996], [-73.93139319399977, 40.81374459799993], [-73.93222960499986, 40.81409433299993], [-73.93236752499996, 40.81411173099992], [-73.93242632000944, 40.81411751897961], [-73.93242159978016, 40.81416728688591], [-73.93241760768083, 40.81420939451084], [-73.93237568818476, 40.81465157530502], [-73.9323656418262, 40.81503296487465], [-73.93236209199594, 40.81534211832778], [-73.9323406284524, 40.817211819802026], [-73.93233582202046, 40.81763040488161], [-73.93238745042831, 40.81942438714826], [-73.9323884755583, 40.819460027887075], [-73.93238971325029, 40.81950284733681], [-73.93239300371323, 40.819553493219956], [-73.93239451747947, 40.8195913361847], [-73.93242874737304, 40.820447140295244], [-73.93245923880322, 40.82112061214107], [-73.93246603518885, 40.82157743497579], [-73.93251180889193, 40.822138815917775], [-73.93254883578302, 40.82259921675289], [-73.93257862720847, 40.823040711045486], [-73.93261139002465, 40.82348761957525], [-73.93266867562046, 40.824078351887685], [-73.93270505627338, 40.824559724000416], [-73.93272841961661, 40.824912542957264], [-73.932730540894, 40.824984348755905], [-73.93273843975052, 40.82512551193915], [-73.93274693841828, 40.82521678307599], [-73.93275761662524, 40.82532227041264], [-73.93276594581108, 40.82545822768752], [-73.93281954048418, 40.82560989294258], [-73.93295940307416, 40.826923447769275], [-73.93298275135784, 40.82722440674656], [-73.93300760566254, 40.827535713706915], [-73.93305881683789, 40.82808624381175], [-73.9330725108407, 40.82815615201811], [-73.93303721899991, 40.828159845999906], [-73.93294190900001, 40.8281677059999], [-73.93250790099984, 40.82820039299993], [-73.93208873599988, 40.828232744999916], [-73.93146631099984, 40.82828175999986], [-73.93126199199995, 40.82829117399994], [-73.93119978199987, 40.82829838699997], [-73.9311413199999, 40.82830380799988], [-73.93095939299991, 40.828337088999916], [-73.93064968200004, 40.82841702999988], [-73.93033933099994, 40.82849555499994], [-73.93003491599993, 40.828587589999906], [-73.929737931, 40.828692722999904], [-73.9294498339999, 40.82881040399989], [-73.9291719679999, 40.828939958999854], [-73.92889437100001, 40.82910412899992], [-73.92875620599996, 40.829185834999855], [-73.92861034599994, 40.82928090899994], [-73.92850073899989, 40.829352349999965], [-73.92825730199993, 40.829530513999906], [-73.92802724899998, 40.82971953899987], [-73.92781189699987, 40.829918431999936], [-73.92781180199998, 40.829918520999854], [-73.92781170699992, 40.829918619999894], [-73.927612063, 40.830126482999916], [-73.92755002799989, 40.83019959099992], [-73.92742893599986, 40.83034231499989], [-73.92726316799988, 40.83056486699991], [-73.9268058709999, 40.8310355669999], [-73.92642918799987, 40.83142328399986], [-73.92537540099983, 40.83262809599993], [-73.92502907499991, 40.83327258399991], [-73.92439355099988, 40.834484026999874], [-73.92414918399993, 40.83482856699992], [-73.92399049499993, 40.83505229799987], [-73.92340016399986, 40.83543276299991], [-73.92237223199994, 40.83610874299993], [-73.921882894, 40.8364257009999], [-73.92036502999983, 40.83742570699993], [-73.92001799499995, 40.83765612799989], [-73.91969564899983, 40.83787180699988], [-73.91938436399987, 40.838076523999845], [-73.919308634, 40.83812632299984], [-73.919162653, 40.838253711999926], [-73.91901668400004, 40.83838109299989], [-73.91859196099995, 40.83898208999995], [-73.91813727499989, 40.83960685399992], [-73.91768361999985, 40.84022549599997], [-73.91729851599982, 40.84076611799985], [-73.91689176399984, 40.84134725699992], [-73.91618344399991, 40.84233489999991], [-73.91768745899992, 40.84223933299993], [-73.91794965199988, 40.842229570999976], [-73.91795571399985, 40.84310189999992], [-73.91783013899986, 40.84320987999994], [-73.91771856499989, 40.84332957399993], [-73.91762366200004, 40.843459320999905], [-73.91754763700003, 40.843597024999895], [-73.91749203599986, 40.84374026199993], [-73.91745769299985, 40.84388636899992], [-73.917444646, 40.8440326819999], [-73.91745219900001, 40.844176629999865], [-73.91749072699996, 40.844299016999884], [-73.9175114879999, 40.8444254339999], [-73.91751358199987, 40.844553629999936], [-73.9174967979999, 40.8446811909999], [-73.91746161499991, 40.84480576699992], [-73.91740919699994, 40.84492513499992], [-73.91732494799987, 40.845057482999955], [-73.91729224999992, 40.84510993999991], [-73.91689747599989, 40.84511773399991], [-73.91684309499995, 40.84507077899988], [-73.91611927099989, 40.84485579299997], [-73.915866478, 40.84479392499991], [-73.91474419399994, 40.8443425039999], [-73.91385799399991, 40.84397879399993], [-73.91302876899995, 40.8436416209999], [-73.91293803199989, 40.8435975549999], [-73.91211222199998, 40.843260341999915], [-73.91193751499992, 40.843202387999916], [-73.91197045599988, 40.84310439399992], [-73.91222180499983, 40.84235659099988]]]}}, {\"id\": \"247\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 248, \"Shape_Leng\": 0.0569191167671, \"Shape_Area\": 0.000149592769799, \"zone\": \"West Farms/Bronx River\", \"LocationID\": 248, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.86393748099981, 40.84004456599994], [-73.86355551599988, 40.83786561999994], [-73.863156384, 40.835530821999924], [-73.86295952099994, 40.834420322999925], [-73.862942603, 40.83429960099993], [-73.86289761399996, 40.83400338399992], [-73.86289361599997, 40.83396426099987], [-73.86288113299992, 40.833889072999916], [-73.86284851199984, 40.833690230999885], [-73.86272700700006, 40.833072220999945], [-73.86271114399993, 40.83299150899997], [-73.86366561299998, 40.832695324999904], [-73.86456547299998, 40.832389234999916], [-73.86546044899993, 40.83213164199989], [-73.86635554699981, 40.83187448399987], [-73.86725770399998, 40.83162391799994], [-73.86814994399988, 40.83136038499988], [-73.86907935400002, 40.83108592799988], [-73.87000670000006, 40.83082034099988], [-73.87090345699994, 40.83056365399988], [-73.87179422399991, 40.830305438999915], [-73.8727554649999, 40.83002788199985], [-73.87322660099987, 40.82990616699988], [-73.8732728849999, 40.8298937199999], [-73.87332596899981, 40.82987665899985], [-73.87372870299997, 40.829752885999945], [-73.87460867899986, 40.8295029049999], [-73.8755070249999, 40.82923413499987], [-73.87640134499988, 40.82897729499997], [-73.87729930499995, 40.8287427739999], [-73.8782350109998, 40.82866001899993], [-73.87870283599995, 40.82862110599992], [-73.87917067099991, 40.828582182999874], [-73.88011211299997, 40.82851166599995], [-73.88104510399991, 40.82843181299996], [-73.88198096699992, 40.8283559799999], [-73.88295104599979, 40.82830386599987], [-73.88365759199986, 40.82827878599987], [-73.88378414099992, 40.828274291999946], [-73.8835201879999, 40.82926018099994], [-73.88357411800004, 40.829478645999934], [-73.88390094800003, 40.82980281599986], [-73.88384720899988, 40.829872518999906], [-73.88251168699992, 40.83155430699988], [-73.88057338999994, 40.83403109299997], [-73.88007189299988, 40.834671876999934], [-73.88005188499987, 40.83470205899989], [-73.87940077099982, 40.83447236499995], [-73.87925239499992, 40.83442478999991], [-73.87916928999984, 40.83452626099988], [-73.87906181199985, 40.83463927099988], [-73.87892705599994, 40.83478096399987], [-73.87866481499992, 40.83502565899989], [-73.87838370399996, 40.83525903199987], [-73.87808502000001, 40.83547984299993], [-73.87801301899998, 40.835530071999905], [-73.8779460599999, 40.83557678399992], [-73.87783304799999, 40.835655621999926], [-73.87756524599996, 40.8358187519999], [-73.87747843299982, 40.83586472899987], [-73.87746425099982, 40.835872232999876], [-73.87734872299994, 40.83593341499993], [-73.87728333599982, 40.83596804299988], [-73.87698921499998, 40.83610249599995], [-73.87673323499986, 40.836205617999944], [-73.87655099199999, 40.83634482899989], [-73.8764548759999, 40.83638732799986], [-73.87628029499993, 40.83646153199987], [-73.87615142899986, 40.83649857399996], [-73.87609408699988, 40.83652621199991], [-73.87584064899983, 40.836612485999915], [-73.87565111799996, 40.83665954899988], [-73.87545635000001, 40.836691981999934], [-73.87525928600002, 40.83670942699994], [-73.87498772399994, 40.836772977999864], [-73.87490147299994, 40.83680809399989], [-73.87482228199987, 40.836859479999866], [-73.87475779999978, 40.83692560499988], [-73.87471452199992, 40.837002253999934], [-73.87469588499982, 40.83708331499986], [-73.87470145099985, 40.83716231099987], [-73.87472739999993, 40.83723397399996], [-73.874908511, 40.83735960999999], [-73.87471801599999, 40.83731259599995], [-73.87454332199997, 40.83726133799991], [-73.87437386799988, 40.8372425229999], [-73.87430028599981, 40.83724244299989], [-73.87419768699993, 40.83725079499988], [-73.87407495699992, 40.83728961699988], [-73.87401051400002, 40.837310004999914], [-73.87379590199993, 40.837445132999896], [-73.87367306200004, 40.837541539999926], [-73.87353901799992, 40.83766841799995], [-73.87347510499988, 40.837746339999896], [-73.87322928299983, 40.83815590199992], [-73.87314714699997, 40.838263898999905], [-73.87312728200004, 40.83833509699991], [-73.87298043000007, 40.83895248499995], [-73.87277204699981, 40.83975128199995], [-73.87272718499995, 40.83974894499989], [-73.87120605899995, 40.839674621999905], [-73.87021450700003, 40.839617941999954], [-73.86955862099985, 40.83959565299989], [-73.86920766899992, 40.839600046999905], [-73.8687407359999, 40.83982457599993], [-73.86830846899983, 40.83999521299996], [-73.86782258199997, 40.840166498999935], [-73.86740207099996, 40.84029027599994], [-73.86724819000003, 40.840343803999886], [-73.86636130899984, 40.8405926849999], [-73.865464502, 40.84085689199993], [-73.86448011799995, 40.84108779699994], [-73.86393748099981, 40.84004456599994]]]}}, {\"id\": \"248\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 249, \"Shape_Leng\": 0.03638434365, \"Shape_Area\": 7.22155957729e-05, \"zone\": \"West Village\", \"LocationID\": 249, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.00250642399995, 40.729016384999966], [-74.00281726599992, 40.72836312899993], [-74.00312767499993, 40.72839380099989], [-74.00531369199997, 40.72860449399992], [-74.0070902079999, 40.72877225299993], [-74.00697298799992, 40.729494989999864], [-74.00685682099994, 40.73022823599989], [-74.00673713399992, 40.73094706599989], [-74.00662407199995, 40.73166937899992], [-74.00650135299995, 40.73242587499988], [-74.00640707699989, 40.733053939999884], [-74.0062942419999, 40.73374204199993], [-74.00618179699998, 40.734439131999885], [-74.00606869899997, 40.735133213999916], [-74.00596018000002, 40.73584500699992], [-74.00585387099987, 40.73653322599989], [-74.00579620299999, 40.7369216979999], [-74.0056870449999, 40.737551878999916], [-74.00559149599997, 40.73813833999991], [-74.00549494099988, 40.73875392299988], [-74.00539035999991, 40.73941197299994], [-74.00529178799992, 40.74002322299992], [-74.00515808199998, 40.74085808099991], [-74.00252381499989, 40.73975264399995], [-73.99968392199986, 40.73855224899986], [-73.99683993899998, 40.73736088899995], [-73.99734132799996, 40.736667888999925], [-73.99780443099984, 40.73603215799991], [-73.99827418999986, 40.73539113999986], [-73.99853021599989, 40.73503789199989], [-73.99871443800002, 40.734783860999855], [-73.999187803, 40.73413483399989], [-73.99955589199998, 40.73361059599991], [-73.99975771599985, 40.73334132199987], [-74.000066577, 40.7329292359999], [-74.00051066399996, 40.73231522299996], [-74.00095442299988, 40.73170609799994], [-74.00141242499994, 40.7310652559999], [-74.00168465499992, 40.73058156199995], [-74.00213899299997, 40.72979574399987], [-74.00220337299997, 40.72965325599993], [-74.00250642399995, 40.729016384999966]]]}}, {\"id\": \"249\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 250, \"Shape_Leng\": 0.0796263465454, \"Shape_Area\": 0.000240975845956, \"zone\": \"Westchester Village/Unionport\", \"LocationID\": 250, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8455308949999, 40.83917330699989], [-73.84458129199986, 40.838535507999914], [-73.84391720399984, 40.839086325999844], [-73.84361155600004, 40.83934832599991], [-73.8430625279998, 40.83979720599991], [-73.8426130889999, 40.840167155999886], [-73.84240098600006, 40.84034851399991], [-73.84204144699997, 40.84064589699992], [-73.84188659999987, 40.84061724699993], [-73.84188635200002, 40.84061720199987], [-73.84188610300001, 40.84061717399984], [-73.84172735199984, 40.840600206999945], [-73.84172710199991, 40.84060017999991], [-73.84172685399986, 40.84060017099996], [-73.84156602199994, 40.84059525899988], [-73.84140548300003, 40.84060250299987], [-73.84124785499995, 40.84062162899993], [-73.83936452799996, 40.84087706699995], [-73.83956700299987, 40.84054722799986], [-73.83972014457761, 40.83956540599305], [-73.84030282537191, 40.839524313944956], [-73.84027376887582, 40.839206273397785], [-73.84022375735451, 40.83913683912731], [-73.84016248431055, 40.83906606578865], [-73.84016698511662, 40.83900415346525], [-73.84014540732095, 40.83894399734026], [-73.84010873870932, 40.83885211820888], [-73.84003063598396, 40.83885759214778], [-73.84004441282266, 40.83894435352963], [-73.84002808869018, 40.83894787100791], [-73.84000970504759, 40.838838090455866], [-73.84009889636496, 40.838827455411305], [-73.8400687997399, 40.838752040090846], [-73.84010742962782, 40.83862842940528], [-73.84013531533367, 40.83861417742005], [-73.83995595134354, 40.8382865171441], [-73.84000264348624, 40.83825197547385], [-73.83993341062431, 40.83790249607463], [-73.83992026416509, 40.83782066325925], [-73.83985132007375, 40.83782174943225], [-73.83985578100564, 40.83790495779814], [-73.83984179647017, 40.83790493838009], [-73.83982833571835, 40.8376871787612], [-73.83984697847083, 40.83768897554632], [-73.83984907065555, 40.83778811149494], [-73.83991581404842, 40.83778279647398], [-73.83995029818121, 40.83758389658748], [-73.83988714189624, 40.836991133824895], [-73.83984192033122, 40.83688527655309], [-73.83983441084683, 40.83676600822796], [-73.83985909371548, 40.8366782868412], [-73.83983837999635, 40.83641969717585], [-73.83984235556295, 40.836116241463884], [-73.83975458111978, 40.83613197084447], [-73.83977909556177, 40.836296305692365], [-73.83975942901569, 40.836296278370426], [-73.83972573144956, 40.83613392228422], [-73.8396995054642, 40.836135876860794], [-73.83969302493071, 40.8361049996907], [-73.83968253637161, 40.8361049851119], [-73.83968247158569, 40.8361318706336], [-73.83966936372369, 40.83613085614669], [-73.83965902852728, 40.836067113758915], [-73.8396721387774, 40.8360671319843], [-73.83967473692059, 40.83607709289496], [-73.83982556569191, 40.83605440057276], [-73.83982172292757, 40.836016554886164], [-73.83971549108533, 40.836031344087495], [-73.83971159080296, 40.83601739881694], [-73.83982175649022, 40.836002614580096], [-73.83981792943698, 40.83595879533936], [-73.83972482818312, 40.83596464124923], [-73.83972485456955, 40.83595368822329], [-73.83981403197603, 40.83594385431263], [-73.8398102282455, 40.83589007730176], [-73.83975907248129, 40.83589996404146], [-73.83975779019478, 40.83588801343359], [-73.83981026660724, 40.835874144805004], [-73.83979992825567, 40.83581139835366], [-73.83965173481587, 40.83582911599167], [-73.83969574806115, 40.83606318124508], [-73.83968001567382, 40.83606315937752], [-73.83960850400152, 40.83570805084221], [-73.83962599488439, 40.83569971218477], [-73.83965573346725, 40.835802235937734], [-73.83979605087632, 40.83578848940347], [-73.83978960150327, 40.8357446676903], [-73.83966108593472, 40.83575743368833], [-73.83965849740537, 40.83574348973624], [-73.83979226941015, 40.83572575105285], [-73.8397792782457, 40.83567594483305], [-73.83968615907075, 40.83568975696178], [-73.83968226014206, 40.835675810016035], [-73.83977669622621, 40.83565901410466], [-73.83977542353503, 40.83564307978601], [-73.83979488581072, 40.835619207246324], [-73.83974862799982, 40.83555882699993], [-73.83972396100002, 40.835478541999876], [-73.83970551900002, 40.835393542999896], [-73.83967775499997, 40.83530853299991], [-73.83964701399998, 40.83516923099997], [-73.83963165699996, 40.83509367899995], [-73.8396350259999, 40.834985108999916], [-73.83961961899993, 40.834930799999924], [-73.83962906799987, 40.834878885999935], [-73.83964694899996, 40.83484286999988], [-73.83964780499996, 40.83484114599992], [-73.83959266810578, 40.83456047582325], [-73.8395802779145, 40.834491723790286], [-73.8395308518402, 40.834368918117875], [-73.8395688621081, 40.83423049122159], [-73.83959108476137, 40.834041688045254], [-73.83975991139047, 40.8337178972479], [-73.83978358795973, 40.83366724279681], [-73.83992048489266, 40.83332079515101], [-73.84018151749495, 40.83304342523689], [-73.84074229145547, 40.83260077900852], [-73.84112675314104, 40.832286983121335], [-73.84181725906056, 40.83172619382725], [-73.84221334682346, 40.83155050713854], [-73.84234432773579, 40.83130200977934], [-73.84252890878093, 40.83112315091929], [-73.8426274568155, 40.83100012051874], [-73.84275884293768, 40.8308184841428], [-73.84270324854322, 40.83074502666989], [-73.84293731237476, 40.83012140218999], [-73.84305256297348, 40.83010156708151], [-73.84309408652338, 40.8300209700227], [-73.84309950959586, 40.82996001179511], [-73.84302116297862, 40.82996158712057], [-73.84305345327975, 40.82976209003935], [-73.8431069159989, 40.82979413931984], [-73.84303612708139, 40.8293069823434], [-73.84303394542376, 40.82893498494016], [-73.84299484136855, 40.828827466701455], [-73.84294598405269, 40.828702916201635], [-73.84291548074185, 40.82862246867612], [-73.84289527503904, 40.828569176560826], [-73.843104377, 40.82853286499992], [-73.84370821, 40.8284387989999], [-73.84390624999997, 40.82840782999993], [-73.84389248199977, 40.82834904499988], [-73.84385743299994, 40.828199343999934], [-73.843851037, 40.82817200499993], [-73.8438380529999, 40.82810246899986], [-73.84381208799992, 40.827962324999895], [-73.84379428399991, 40.827890342999865], [-73.84375757299995, 40.82778278299986], [-73.84374214699994, 40.827733152999876], [-73.843728169, 40.82766817299993], [-73.84358503199985, 40.82700294399991], [-73.84341053499996, 40.82628047299992], [-73.84324615099987, 40.82555394499987], [-73.84306938499985, 40.82481661099991], [-73.84289858800003, 40.82408305899986], [-73.84595550900002, 40.823670982999886], [-73.849017675, 40.823259957999895], [-73.85207828999987, 40.82284723799996], [-73.85513639799981, 40.822436188999866], [-73.855312208, 40.82319243099992], [-73.85548601399985, 40.82391214599993], [-73.85564695999979, 40.82463286199989], [-73.85588881599983, 40.82567371799994], [-73.855971624, 40.82606387799991], [-73.85619656499995, 40.82604099499984], [-73.85621834399984, 40.82616062899996], [-73.856260876, 40.82632936699985], [-73.85626925499986, 40.826393776999936], [-73.85629023899993, 40.82648152999993], [-73.85632684699985, 40.82662383499993], [-73.85635677499994, 40.82675208499993], [-73.85615112900003, 40.82677004599987], [-73.85632851899997, 40.82750768399989], [-73.85638435699991, 40.82775374199988], [-73.85641767299995, 40.82790051999994], [-73.85649133899996, 40.8282250779999], [-73.85665560399987, 40.82894491299988], [-73.85682187599986, 40.82966476899987], [-73.85698995199976, 40.83038523999988], [-73.85715675899989, 40.83110226799994], [-73.85732320899999, 40.83182148399993], [-73.85737529499985, 40.83204654599997], [-73.85738092499992, 40.832135234999924], [-73.85738951499987, 40.83225419999993], [-73.857391213, 40.83229737199986], [-73.85766180999998, 40.832379254999864], [-73.85847699499993, 40.83262592199991], [-73.859119143, 40.83282022499991], [-73.85934693599994, 40.83288902199993], [-73.86008270300002, 40.833111456999866], [-73.86032508999983, 40.83319863999985], [-73.86033845899995, 40.833253165999935], [-73.85983937599993, 40.83334089599988], [-73.8590567219999, 40.833413224999894], [-73.85773226199993, 40.83355730699993], [-73.85465135700002, 40.83388802699987], [-73.85347099099992, 40.83401491699993], [-73.85281379799993, 40.83407976199991], [-73.85188599699991, 40.83417942099992], [-73.85168459699997, 40.8342010519999], [-73.85185868799985, 40.834777746999876], [-73.85207005999993, 40.83562530299994], [-73.85220720999986, 40.836347494999934], [-73.85233148699994, 40.83701052699997], [-73.85247938999986, 40.83778428699986], [-73.85266480499992, 40.8387800839999], [-73.85286997899986, 40.839851303999936], [-73.85303510299988, 40.840735457999884], [-73.85309460199986, 40.8410469599999], [-73.85333524999986, 40.84247015499987], [-73.85231153200002, 40.84258699399991], [-73.85179886599994, 40.8426476329999], [-73.85120896799987, 40.84271512199991], [-73.85088863399999, 40.84275300599993], [-73.85052278099987, 40.84278948899985], [-73.8489861779999, 40.84159995199983], [-73.84796242199991, 40.84082789199988], [-73.84779791999988, 40.84069631099985], [-73.84715638299993, 40.840263675999914], [-73.84647864699997, 40.83980920599987], [-73.8455308949999, 40.83917330699989]]]}}, {\"id\": \"250\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 251, \"Shape_Leng\": 0.1377111611, \"Shape_Area\": 0.000625754983157, \"zone\": \"Westerleigh\", \"LocationID\": 251, \"borough\": \"Staten Island\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.13107460299996, 40.631147728999956], [-74.13059499799988, 40.6308802829999], [-74.13047428999985, 40.6308855089999], [-74.13014141499997, 40.62996520899991], [-74.13024789099997, 40.6296091299999], [-74.13033008299993, 40.62938433899989], [-74.13056374599996, 40.628707729999896], [-74.13069219599991, 40.6283369019999], [-74.13099290799993, 40.62737410999992], [-74.1313098529999, 40.62632834299998], [-74.13069767299993, 40.62634095599988], [-74.13007946899988, 40.62641859599995], [-74.12920250099991, 40.62654060599987], [-74.12826227099993, 40.6266735039999], [-74.12666303799995, 40.626889617999915], [-74.12513287199992, 40.62710260099988], [-74.12424800199987, 40.62721164199989], [-74.12365042899994, 40.627286876999904], [-74.12270654899999, 40.627407395999896], [-74.12216225499999, 40.627476891999876], [-74.12125677899994, 40.62782274999989], [-74.12056862499992, 40.6269886999999], [-74.12012887999998, 40.62653615299989], [-74.11967435799993, 40.62619142199989], [-74.11916467799988, 40.62582382799994], [-74.11845939699991, 40.62526898999987], [-74.11828960499999, 40.62512868299993], [-74.11810485699996, 40.624997729999926], [-74.11790654899988, 40.624877460999876], [-74.11769631299995, 40.62476906499987], [-74.11747606599991, 40.62467350199995], [-74.11724781899987, 40.62459150899986], [-74.11701375799994, 40.62452357699987], [-74.11627412799992, 40.62423942099988], [-74.11541921799997, 40.62391096299995], [-74.1132624959999, 40.62305442499995], [-74.11235088099993, 40.62226529199995], [-74.11153393800001, 40.62160364599985], [-74.11094392899996, 40.621194488999876], [-74.11001550699994, 40.62068834399986], [-74.1091080619999, 40.62029623899989], [-74.10770635299994, 40.61969949699991], [-74.10734089099991, 40.61954307699993], [-74.10608939599997, 40.618984194999925], [-74.10552598799994, 40.618592329999906], [-74.10518709699997, 40.618281473999915], [-74.10511622199992, 40.618203003999945], [-74.1049532429999, 40.618022562999904], [-74.10458409200001, 40.61761290099989], [-74.10350925299994, 40.61641607499991], [-74.10302593099988, 40.615781107999936], [-74.10256802199996, 40.61518744299984], [-74.10209434299992, 40.6146004909999], [-74.10127614399993, 40.6139854019999], [-74.10068375899988, 40.61355314999987], [-74.10055168499994, 40.61346054699993], [-74.09987733699994, 40.612918652999866], [-74.09935042999992, 40.61241907699984], [-74.0986215139999, 40.61151891199989], [-74.0984706099999, 40.61135390299989], [-74.09810312099995, 40.61096151899992], [-74.09787758299997, 40.61072932199992], [-74.0977696879999, 40.61062359299988], [-74.09938452999991, 40.61056138999991], [-74.09946989799994, 40.61055810099989], [-74.10003616599987, 40.61053628199992], [-74.10082806899992, 40.6105066719999], [-74.10138596799993, 40.610485807999915], [-74.101654781, 40.61047575399986], [-74.10181225499998, 40.61046986299992], [-74.10239596999999, 40.61044802799987], [-74.10290182399991, 40.610429102999895], [-74.10337596999989, 40.6104134719999], [-74.10429401399988, 40.610383201999944], [-74.10549388999988, 40.61034361799989], [-74.10667509699992, 40.610286802999916], [-74.10722963899997, 40.610252444999915], [-74.10778645199997, 40.61023505699993], [-74.10834417899991, 40.6102347679999], [-74.10885268299994, 40.610250109999896], [-74.10890146199993, 40.61025158499995], [-74.10945696499992, 40.61028543899993], [-74.11000934099998, 40.61033620599996], [-74.11047956499992, 40.61041865599992], [-74.11082654399989, 40.6105069149999], [-74.11113638099991, 40.610586059999925], [-74.11263747399992, 40.61096736699994], [-74.11295306899993, 40.61103897299988], [-74.11327407999993, 40.6110979379999], [-74.11359920399994, 40.61114391299989], [-74.11392707099994, 40.6111766829999], [-74.11425629899995, 40.611196149999955], [-74.11458547799997, 40.6112022959999], [-74.11508745399988, 40.61120943099993], [-74.1156683379999, 40.611175384999896], [-74.11651105799997, 40.611050927999905], [-74.11677996799996, 40.61099204899987], [-74.11687458199992, 40.610971825999904], [-74.11738923899999, 40.61083321499991], [-74.118326509, 40.61058077499993], [-74.11993967899996, 40.61014626799986], [-74.1204407989999, 40.610018066999906], [-74.12135179599994, 40.609765267999926], [-74.12247214699988, 40.609481234999905], [-74.12252324799988, 40.609468999999905], [-74.12326521999991, 40.60929134799988], [-74.1235063919999, 40.60923360199991], [-74.12398126100001, 40.60913521899988], [-74.12452618799996, 40.60902231899992], [-74.1251456319999, 40.608908470999914], [-74.12520040399997, 40.60889840499993], [-74.12630497799988, 40.608699642999966], [-74.12650911499996, 40.60866290799986], [-74.12674361599996, 40.60863191499992], [-74.12742960799994, 40.60854975999994], [-74.12784267099993, 40.60850028899994], [-74.12874897499987, 40.608391740999934], [-74.12983323799993, 40.608303942999925], [-74.13094002299995, 40.60821431199985], [-74.131568734, 40.60818117199994], [-74.13210459699991, 40.608152923999924], [-74.13211741399986, 40.60826434899993], [-74.13215051199998, 40.608551963999936], [-74.13311710799991, 40.608505831999864], [-74.13404435499996, 40.60845319899988], [-74.13497712799997, 40.60841117899987], [-74.1359058139999, 40.6083589069999], [-74.1371740599999, 40.60829269799993], [-74.13810528799995, 40.60824232499986], [-74.13903904999992, 40.608195812999924], [-74.13997406999988, 40.60814639199986], [-74.14092528299996, 40.60808963199984], [-74.14188196399996, 40.60804294099991], [-74.142832906, 40.60799259499987], [-74.14391816399994, 40.60798983699987], [-74.14418304099992, 40.60799797199993], [-74.14444741899989, 40.608012796999915], [-74.14471100599997, 40.60803429699994], [-74.14497351299995, 40.60806244799991], [-74.14523464999989, 40.608097218999895], [-74.14549412999996, 40.608138571999866], [-74.14575166799996, 40.60818646199988], [-74.14600697999994, 40.60824083499989], [-74.14631056999997, 40.60830708799991], [-74.14661053099996, 40.608382978999906], [-74.14690609299997, 40.60846833999988], [-74.14719646699989, 40.60856292999989], [-74.147469717, 40.60865584999992], [-74.14773969799992, 40.608754164999915], [-74.14800622599992, 40.60885780899986], [-74.14826911999995, 40.60896670999988], [-74.14852820199994, 40.60908079499994], [-74.148783295, 40.609199986999904], [-74.14954343599996, 40.60962717299988], [-74.14893620799994, 40.60975083699993], [-74.14774036799996, 40.609999249999866], [-74.14696864199988, 40.610151355999896], [-74.14689344899988, 40.610631709999964], [-74.14680105499993, 40.6113349189999], [-74.14670113499992, 40.61207152099992], [-74.14654340799991, 40.61326141199991], [-74.14645944899998, 40.6138914249999], [-74.1464070389999, 40.61427315099995], [-74.14603562099998, 40.61701277699996], [-74.14512419099998, 40.6169659789999], [-74.14419491899997, 40.61687026199994], [-74.14326214899995, 40.616740238999924], [-74.14233611399993, 40.616608643999925], [-74.1417342679999, 40.61653721899988], [-74.14161689099993, 40.61652246999996], [-74.14107167799996, 40.61645643799993], [-74.14072587999992, 40.61641455499993], [-74.13982184799993, 40.61629982499988], [-74.13964602399992, 40.61701454899985], [-74.13944209899994, 40.617765813999895], [-74.13923849299987, 40.61851915699989], [-74.13900214099995, 40.619394642999886], [-74.13873783699994, 40.62037370299989], [-74.13853418199996, 40.6211282339999], [-74.13836666899986, 40.621737116999896], [-74.13861274099999, 40.62177287799988], [-74.13850455799997, 40.62263379699992], [-74.13826358199992, 40.624243769999936], [-74.13805654699992, 40.62422387299995], [-74.13784655299993, 40.62421900999986], [-74.13763642599997, 40.62422942999995], [-74.13742903499994, 40.624255017999886], [-74.137227136, 40.624295196999846], [-74.13708396099996, 40.624347681999915], [-74.13694386099996, 40.624404781999914], [-74.13680709299996, 40.62446639299985], [-74.13667390599991, 40.6245324029999], [-74.13654454499994, 40.6246026899999], [-74.13641924499991, 40.62467712699986], [-74.13629823599996, 40.62475557699985], [-74.13618173899991, 40.62483789599987], [-74.13581008299997, 40.62506433999991], [-74.13793305099995, 40.626956696999834], [-74.13805849499992, 40.62706851799986], [-74.13789126699994, 40.62745287999997], [-74.13806393799992, 40.627568470999904], [-74.13785800099993, 40.62769048799992], [-74.13730784599991, 40.62819029599987], [-74.13674237799992, 40.62866253699986], [-74.13604609099994, 40.62818801299994], [-74.13586108399996, 40.62806152799989], [-74.13408698299992, 40.62954168099984], [-74.13343731299992, 40.62907434599989], [-74.13228531899989, 40.630018989999925], [-74.1316183449999, 40.630583688999884], [-74.13107460299996, 40.631147728999956]]]}}, {\"id\": \"251\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 252, \"Shape_Leng\": 0.158004373564, \"Shape_Area\": 0.00102526406057, \"zone\": \"Whitestone\", \"LocationID\": 252, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.82049919995306, 40.80101146781907], [-73.82040454927233, 40.80063950029248], [-73.8203923130523, 40.8006453615289], [-73.82037687199814, 40.80065395479625], [-73.82035900219094, 40.800662878657036], [-73.82034016050163, 40.80067160114268], [-73.82032573273936, 40.800678197407294], [-73.82031007436558, 40.800684071896114], [-73.82029661962055, 40.800690363772794], [-73.82028142890316, 40.80069882188981], [-73.82026229996207, 40.80070876148789], [-73.82024566641648, 40.80071597615652], [-73.8202234279822, 40.800723459838366], [-73.82020827859925, 40.80072966730964], [-73.82019261876997, 40.80073615391711], [-73.82017602269929, 40.80074254792832], [-73.8201618111261, 40.800748181574924], [-73.82014789303919, 40.80075460723408], [-73.82012926148793, 40.80076492468523], [-73.82011500217105, 40.80077492513015], [-73.82010066719741, 40.80078697847634], [-73.82008087032678, 40.80080242790971], [-73.82006933325167, 40.800809622530736], [-73.82005782160546, 40.80081564618917], [-73.82004118635471, 40.80082347346513], [-73.8200276980661, 40.80082926904845], [-73.82000745456675, 40.800834919559875], [-73.81999040217745, 40.80083917035242], [-73.81997185610044, 40.80084403144923], [-73.8199533214342, 40.80084874815422], [-73.81994256448044, 40.80085644839917], [-73.81993109004819, 40.80086614794038], [-73.81991954856151, 40.80087488176069], [-73.81990965833285, 40.80088644631944], [-73.81990706461919, 40.80089453808854], [-73.81990569452488, 40.80090617911473], [-73.81990613592077, 40.800917977261655], [-73.81990574325437, 40.800932006043396], [-73.81990606543906, 40.80094402848774], [-73.81990996957263, 40.80095531707219], [-73.81991364984894, 40.80097014544845], [-73.8199121196076, 40.80097530316798], [-73.81990803413413, 40.800978646450965], [-73.81990182068907, 40.80098008650629], [-73.81988947103542, 40.8009765012704], [-73.81988336287415, 40.80097846467033], [-73.81987037642868, 40.80098681795599], [-73.81985865416124, 40.800991959748046], [-73.81985721886358, 40.80099217327732], [-73.81983965044095, 40.80099483895563], [-73.81982823806005, 40.80099487489432], [-73.81981538958014, 40.80099199984768], [-73.81980890338927, 40.80098474072734], [-73.81980355412048, 40.80097323360556], [-73.8198000063264, 40.800961594274455], [-73.81979545394888, 40.800949537742696], [-73.81978656757732, 40.800935800188824], [-73.81977576989408, 40.80092351879012], [-73.81976160908972, 40.80091045699296], [-73.8197428603402, 40.800897910530814], [-73.81972211480347, 40.800887531566396], [-73.81970375559746, 40.800880199090386], [-73.81968729409633, 40.80087643669323], [-73.8196683781757, 40.800873363170744], [-73.81965043002965, 40.80087111136241], [-73.81963489972662, 40.80086938534287], [-73.81961619208606, 40.80086812140951], [-73.81959897832714, 40.80086625772545], [-73.81957962669078, 40.800862409540244], [-73.81956245126247, 40.80085494554434], [-73.8195499324419, 40.80084847689893], [-73.81953449948413, 40.80084101562644], [-73.81951808458177, 40.80083308290654], [-73.81950121227939, 40.800823835232876], [-73.81948584786856, 40.80081296894467], [-73.81946040604019, 40.80079113759381], [-73.81944528658519, 40.80077327532988], [-73.81943678901052, 40.800764886766096], [-73.81942096509312, 40.800752921286644], [-73.81940696695627, 40.8007455967674], [-73.81939199504005, 40.80073893596323], [-73.81937413652587, 40.80073031695543], [-73.81935632707153, 40.80072092455794], [-73.81933948214659, 40.8007098934303], [-73.81932120799769, 40.80069765404645], [-73.8192990644269, 40.800682905935254], [-73.81928774396401, 40.800675169968876], [-73.81927646012072, 40.800667750848895], [-73.81926468213902, 40.80065858366555], [-73.8192538597349, 40.80065074963688], [-73.8192444936212, 40.80064373809397], [-73.81922719307305, 40.80063075255677], [-73.81921327193635, 40.80062090621045], [-73.81920130450621, 40.800611774406185], [-73.81919002444997, 40.80060280617828], [-73.81917514052282, 40.80058974367653], [-73.81915817148864, 40.80057629912723], [-73.8191424947756, 40.80056284839426], [-73.81912616697637, 40.80054943232031], [-73.81911106847645, 40.800536729652556], [-73.81908501071571, 40.80051489607839], [-73.81906605879296, 40.80049883769076], [-73.81905502678212, 40.80048970784191], [-73.81904317956817, 40.80048016175238], [-73.81903113213716, 40.80046990386784], [-73.8190183387, 40.80045964481082], [-73.81900550896377, 40.80044935939381], [-73.81899157094638, 40.80043723431635], [-73.8189779170938, 40.80042511085601], [-73.81896248963965, 40.800411146187194], [-73.81894721538023, 40.80039824487187], [-73.81893319547234, 40.800385760149396], [-73.8189203355856, 40.80037327775018], [-73.81890761748033, 40.8003615152604], [-73.81889555120752, 40.80034977138347], [-73.81888434357268, 40.80033990178583], [-73.81887360958905, 40.80033039478576], [-73.81886344217126, 40.80032161707462], [-73.81885277800716, 40.80031245980726], [-73.81883908295644, 40.80030245127109], [-73.81882440672894, 40.80029193727515], [-73.81880973978633, 40.800282063405746], [-73.8187955277619, 40.80027457480313], [-73.8187793618741, 40.80026684222585], [-73.8187622247354, 40.80025868327533], [-73.81874415771607, 40.8002486683563], [-73.81870899877953, 40.80022701035811], [-73.81869195492317, 40.80021504285851], [-73.81867630461416, 40.80020475179527], [-73.81866162931662, 40.800193670874606], [-73.81865013772534, 40.80018415261021], [-73.81863982515077, 40.80017640124491], [-73.81862984578224, 40.80016837030087], [-73.8186176007066, 40.800156986794754], [-73.81860502724125, 40.80014411782715], [-73.81858913957856, 40.80012976640528], [-73.81857381686241, 40.800116899641054], [-73.8185614263993, 40.8001066148653], [-73.81855162505244, 40.80009822468543], [-73.81854740999043, 40.80009265286358], [-73.8185323465083, 40.80008060720767], [-73.81851635016453, 40.80006715588488], [-73.81849601989705, 40.800052436773754], [-73.81847391511279, 40.800036869029796], [-73.81845419847089, 40.800023086840746], [-73.81843350228678, 40.800008042310765], [-73.81842269328845, 40.79999965153788], [-73.81841227645265, 40.79999151183172], [-73.81840229615311, 40.79998376147094], [-73.81838066494184, 40.79996835578168], [-73.8183684116438, 40.799959691672186], [-73.81835568671484, 40.79995033292882], [-73.81832923041566, 40.79993100288715], [-73.81831552481894, 40.79992095808581], [-73.81830229964709, 40.799912184383565], [-73.81828715680925, 40.79990307478434], [-73.81827294836097, 40.79989465032639], [-73.81826018390831, 40.799886696711425], [-73.81824838872056, 40.7998794117086], [-73.81823609959068, 40.799871207050074], [-73.818212512374, 40.799855969455216], [-73.81819928744804, 40.79984743831797], [-73.81818652301168, 40.79983948536489], [-73.81817328581903, 40.79983116997769], [-73.81815907806109, 40.799822503767864], [-73.81814635048075, 40.799814055161264], [-73.81813311642097, 40.799804677497185], [-73.8181189593013, 40.79979458689537], [-73.81810234193345, 40.79978297128074], [-73.81808816961643, 40.7997742503373], [-73.81807345282117, 40.79976519532511], [-73.81805975418524, 40.79975634060291], [-73.81804605705439, 40.79974717847282], [-73.81803185167449, 40.799737654856806], [-73.81801717178567, 40.79972833034761], [-73.81800152304712, 40.79971770598647], [-73.817985921389, 40.79970711855301], [-73.817971206244, 40.799697792977035], [-73.81794329860905, 40.799680676577225], [-73.81793053512148, 40.79967272192178], [-73.81791925106877, 40.7996647712769], [-73.81790694964546, 40.79965689926844], [-73.81789422378189, 40.79964760463414], [-73.81788149792246, 40.79963855156978], [-73.81787151795258, 40.79963104491212], [-73.81785360240538, 40.799617346211974], [-73.81783493339428, 40.79960202634947], [-73.81781377322392, 40.79958794424355], [-73.81780199223385, 40.79957993902603], [-73.81779069898937, 40.7995714113971], [-73.81777654228932, 40.7995612850706], [-73.8177643143817, 40.79955217967476], [-73.81775278355012, 40.79954379540315], [-73.81774099217424, 40.79953540200703], [-73.81772873900863, 40.79952707103666], [-73.81771745657635, 40.799518868416406], [-73.81770570287786, 40.79950935852076], [-73.81769489335721, 40.79950170445365], [-73.8176799667018, 40.79949180255387], [-73.81766600984703, 40.799482289336815], [-73.817648705424, 40.7994706993733], [-73.81762661459345, 40.79945436522656], [-73.81761240936785, 40.7994442667795], [-73.81759921230812, 40.79943431265747], [-73.81758502009886, 40.799424213725096], [-73.81756194646383, 40.799407986064274], [-73.81754175802995, 40.79939362768893], [-73.81752322782053, 40.79937919068526], [-73.8175040189219, 40.79936641813983], [-73.81748523333897, 40.799354915432346], [-73.81747076259346, 40.79934687844762], [-73.81745613456414, 40.79933606682684], [-73.8174398004204, 40.799325036729634], [-73.8174247401818, 40.79931170366827], [-73.81740429876693, 40.799299019785], [-73.81738598276688, 40.79928888682118], [-73.81736282283454, 40.79927787337336], [-73.81733915474054, 40.799266390380005], [-73.81732588575856, 40.799260822257686], [-73.81731311724012, 40.79925451597327], [-73.81730031209244, 40.79924862475675], [-73.81728706976284, 40.79924226394281], [-73.81727523443656, 40.79923675186273], [-73.81725301720094, 40.79922757780624], [-73.81723566878601, 40.799219454979735], [-73.81722069543777, 40.799213198462205], [-73.81720813882664, 40.79920728015542], [-73.81718328428886, 40.7991965340347], [-73.81717578003556, 40.79919322624152], [-73.81715983426098, 40.79918746505354], [-73.8171424303444, 40.79918197980955], [-73.81712207277631, 40.799177166176314], [-73.81709882474345, 40.79917249235996], [-73.81707604931336, 40.799168475820224], [-73.81705085723912, 40.79916360893547], [-73.8170222676227, 40.79915801646115], [-73.81700508481417, 40.7991540625605], [-73.8169878664468, 40.7991498127512], [-73.81696877861198, 40.79914486490873], [-73.81695205938497, 40.799140119676906], [-73.81693586346815, 40.79913482679612], [-73.81692110001387, 40.79912994025912], [-73.81690540209898, 40.799124565910155], [-73.81688869448094, 40.79912009174333], [-73.81687248379598, 40.79911592359461], [-73.81685722059562, 40.799111837696096], [-73.81684151847931, 40.7991079135976], [-73.81682578072974, 40.79910380114201], [-73.8168109899126, 40.799100291772184], [-73.81679622205118, 40.79909733309248], [-73.81678243562176, 40.79909445671784], [-73.81675529326716, 40.7990882362476], [-73.81674001687544, 40.79908502311905], [-73.81672427696331, 40.79908146012882], [-73.81670853904163, 40.79907789546443], [-73.81670002505695, 40.799075910154045], [-73.81669475441785, 40.79907468637709], [-73.816675370543, 40.7990696661469], [-73.81666133734032, 40.79906618660961], [-73.8166448773395, 40.7990625846025], [-73.81661874132057, 40.799056617658664], [-73.81659693828932, 40.79905158535793], [-73.81658361651031, 40.79904865542062], [-73.81656673346953, 40.79904313624059], [-73.8165485476371, 40.799037677136575], [-73.81653529932524, 40.799033288165866], [-73.81651639859038, 40.79902494749008], [-73.81650243611864, 40.7990175677235], [-73.8164913280699, 40.79901032752483], [-73.81647735191558, 40.79899951681666], [-73.81646965727438, 40.79898828523332], [-73.81646163371576, 40.798975538530144], [-73.81645246797443, 40.79896072927969], [-73.81644430357927, 40.798947974142216], [-73.81643386411537, 40.798934837109186], [-73.8164235301729, 40.79892244740574], [-73.81640659548852, 40.798905535901305], [-73.81638772645348, 40.798885641709354], [-73.81637620047697, 40.798875681899], [-73.81636080309183, 40.79886449013784], [-73.81634242337212, 40.798851664942525], [-73.81632652026309, 40.798843679306515], [-73.81631349165232, 40.79883710361823], [-73.81630051488024, 40.79882859946052], [-73.81628854195914, 40.79882171854548], [-73.81628288763298, 40.79881838734075], [-73.81627594436371, 40.79880506706438], [-73.81626787833002, 40.79879934461275], [-73.81625472374584, 40.79878713919878], [-73.81624420954162, 40.79877570492302], [-73.8162345633379, 40.798763118116256], [-73.81622293670387, 40.79875090692568], [-73.8162133520338, 40.7987413465586], [-73.81620262970858, 40.79872784837124], [-73.8161904875062, 40.798713944015326], [-73.81617933119495, 40.79870305651709], [-73.81616589978309, 40.798688050466914], [-73.81615459893241, 40.7986737867671], [-73.81614299803788, 40.79866064785451], [-73.81613079419463, 40.79864712099566], [-73.81611754243514, 40.79863172824301], [-73.8161085807101, 40.79861971947416], [-73.81609927442656, 40.798604360487616], [-73.81609413376435, 40.79859480720013], [-73.8160858687864, 40.79857980067169], [-73.81607935134153, 40.79857155875241], [-73.81606735188909, 40.798561220553154], [-73.81605394465207, 40.798550329450684], [-73.81603876983723, 40.798539931735505], [-73.816026038666, 40.79853296833298], [-73.81601082087283, 40.79852536201865], [-73.81599613439198, 40.7985186211483], [-73.8159833646266, 40.79851278311398], [-73.81596793344923, 40.79850525703296], [-73.81595372067311, 40.798498372506586], [-73.81593629589116, 40.79849184403399], [-73.81592059964356, 40.79848610937336], [-73.81590367015849, 40.79848023771561], [-73.81588574534095, 40.79847485146154], [-73.8158673357198, 40.798468923491214], [-73.81584770601887, 40.79846224657363], [-73.81583248197222, 40.79845715276936], [-73.81581775579993, 40.798451941653134], [-73.81579985674536, 40.79844537621664], [-73.8157841257758, 40.798439371431094], [-73.81577041609565, 40.79843495466293], [-73.81575616360166, 40.79842960977321], [-73.81574000979177, 40.79842170507211], [-73.81572039127308, 40.79841179474572], [-73.81570199213954, 40.79840185135289], [-73.81569232334162, 40.798397720468145], [-73.81567586906978, 40.79839157060707], [-73.81565772099717, 40.79838490540769], [-73.81563982694729, 40.798376942961056], [-73.81562608702995, 40.79837071668704], [-73.8156133537509, 40.79836430271799], [-73.81560033524546, 40.79835835602178], [-73.81558709335448, 40.79835195932906], [-73.81557190932354, 40.798345487658864], [-73.8155557046396, 40.79833893310394], [-73.81554002070406, 40.79833275765204], [-73.81552626607565, 40.79832763047564], [-73.81551425337895, 40.7983221179295], [-73.81549810069912, 40.798314302487], [-73.81548164791278, 40.79830763059177], [-73.81546447400227, 40.798300578765115], [-73.81544387674795, 40.798292747488226], [-73.81542644145914, 40.79828591151252], [-73.8154095276851, 40.79827878051965], [-73.81539459540231, 40.79827112901381], [-73.81537962061495, 40.79826553046], [-73.81536316791401, 40.798259074823115], [-73.815346974238, 40.79825309671058], [-73.81533055467733, 40.79824725258916], [-73.8153121481771, 40.79824000175538], [-73.81530060856933, 40.79823529138568], [-73.81528491788299, 40.79822711714415], [-73.81526948886314, 40.798219104144145], [-73.81524985299129, 40.798210806642615], [-73.81523268635723, 40.798205042727055], [-73.81521528508323, 40.798198675509454], [-73.81519884414837, 40.79819220177504], [-73.81517823493321, 40.79818481119019], [-73.81515662049448, 40.798176546183115], [-73.81513309967957, 40.79816747617102], [-73.81510954599655, 40.79815801945321], [-73.81508404248532, 40.798146101001876], [-73.8150700798322, 40.79813890986088], [-73.81505659024279, 40.79813227951413], [-73.81504263884548, 40.798124927061096], [-73.81502914989186, 40.79811806988289], [-73.81501468082487, 40.79810987009055], [-73.8149992514518, 40.79810176759513], [-73.81498331023157, 40.798094529376336], [-73.81496689411837, 40.79808747048275], [-73.81495387530495, 40.79808199194578], [-73.81494038313099, 40.798075982429836], [-73.81492346275004, 40.79806690471118], [-73.81490731008795, 40.79805889217561], [-73.81489753789192, 40.798053769643346], [-73.81488208804964, 40.798048360737866], [-73.81486660735591, 40.79804207679633], [-73.81484625902685, 40.798034363247105], [-73.81482613576495, 40.7980268125552], [-73.81480358514975, 40.7980185623952], [-73.81478640546105, 40.79801340215292], [-73.81476748640418, 40.79800739210531], [-73.8147512901144, 40.79800231519045], [-73.8147323804279, 40.79799737614111], [-73.81471715688336, 40.79799161627673], [-73.81470446768195, 40.79798288791607], [-73.81470052927918, 40.79797985524089], [-73.8146927142876, 40.79797383723192], [-73.81467977553595, 40.79796495534921], [-73.81467140259664, 40.79795909776851], [-73.8146642355569, 40.79795408762621], [-73.8146518005806, 40.79794281194396], [-73.8146378953514, 40.797931893070455], [-73.81462488214582, 40.79792465884025], [-73.81460774504265, 40.797916806676135], [-73.8145915414399, 40.79791025198665], [-73.81457115166883, 40.797904779642465], [-73.81455397100835, 40.79790022262229], [-73.81453531095558, 40.79789487115985], [-73.81450613017519, 40.797884224545896], [-73.81449656725863, 40.797880201951976], [-73.81447838702681, 40.79787298686371], [-73.81446172180917, 40.79786612567932], [-73.81444602800991, 40.79785937360732], [-73.81443083161342, 40.797852514782825], [-73.81441367476789, 40.79784353544798], [-73.814397515615, 40.79783401811819], [-73.81438377926207, 40.797826278947646], [-73.81437772033159, 40.79782336028193], [-73.81436736561623, 40.79781838167568], [-73.81435311656463, 40.797812127122455], [-73.81433914923454, 40.79780666625331], [-73.81432320486326, 40.79780068772771], [-73.81430822758301, 40.79779605297043], [-73.81429228199, 40.79779075140813], [-73.81427163737345, 40.79778704347901], [-73.81425245956895, 40.79778500394859], [-73.81423574212853, 40.79778399665023], [-73.81421462435387, 40.797780557999005], [-73.81419791453479, 40.79777682071561], [-73.81417853323323, 40.79777108104925], [-73.81416377118336, 40.79776595091281], [-73.81414981480735, 40.797760444471386], [-73.81412503053268, 40.79775024678071], [-73.81410248867043, 40.79773892676371], [-73.81407795712317, 40.7977271996296], [-73.81406450648764, 40.797719631589935], [-73.81405050911255, 40.797711999509694], [-73.81403900802484, 40.79770619061195], [-73.81401498287853, 40.79769556157188], [-73.81399291191686, 40.797685584505636], [-73.8139747714611, 40.79767655912354], [-73.81396402744772, 40.79766662780539], [-73.81394981695435, 40.79765930110248], [-73.81393193039827, 40.79765274360329], [-73.81391648627458, 40.797645586785784], [-73.8138993159446, 40.79763737428281], [-73.81388164701266, 40.797629384953574], [-73.81386498155204, 40.79762249671034], [-73.81385026362024, 40.79761453844295], [-73.8138382947005, 40.79760631623802], [-73.8138213744634, 40.79759728342091], [-73.81380814896725, 40.79758955371777], [-73.81379589098282, 40.7975832490384], [-73.81377701242235, 40.797575446539895], [-73.81376474742939, 40.79757117627287], [-73.8137463118747, 40.79756640082924], [-73.81372396579235, 40.79756134030113], [-73.81370898608684, 40.797557606245086], [-73.813694767553, 40.79755317081637], [-73.813679076332, 40.79754578791093], [-73.81366682246971, 40.797537781502506], [-73.81365672554256, 40.79752949797495], [-73.81364448203017, 40.79751803268644], [-73.81363735676354, 40.79750641367473], [-73.81363223727841, 40.79749386194196], [-73.81362930956557, 40.79748150205109], [-73.81362880095362, 40.79746860647262], [-73.81362807557187, 40.797456997305225], [-73.81362796930762, 40.79744393049625], [-73.81362905346168, 40.797429515860784], [-73.81363097237593, 40.797417011190554], [-73.81363549060625, 40.79740282659117], [-73.81364167178127, 40.79738732994166], [-73.81364730879639, 40.79737163422971], [-73.81365275799477, 40.797359323564784], [-73.8136585037273, 40.797347392153576], [-73.81366765918304, 40.79732761467891], [-73.81367253519188, 40.797316969125426], [-73.81367688799823, 40.797307458547394], [-73.81368666413815, 40.79728581617772], [-73.81369100029453, 40.79727366788661], [-73.81369616816892, 40.797260420298365], [-73.81370172690912, 40.7972471644627], [-73.81370591633107, 40.797236167167554], [-73.81370919053616, 40.79722664479341], [-73.81371743898944, 40.79720909845813], [-73.81372970392076, 40.79718784055657], [-73.8137349711503, 40.79717681685776], [-73.81374391110171, 40.7971581731745], [-73.8137491075235, 40.79714763685915], [-73.81374596947316, 40.797143245138784], [-73.81371657753401, 40.79710210837836], [-73.81369963725258, 40.79707941566509], [-73.81342050415856, 40.79769302041066], [-73.813350137201, 40.7976723226576], [-73.81348648309174, 40.79734593975082], [-73.81337822710192, 40.79731420176152], [-73.8134935036856, 40.79705838037773], [-73.81336974991873, 40.79707455952279], [-73.81327739074668, 40.79710575644098], [-73.8132509965705, 40.79719635910535], [-73.81324649695341, 40.79721182110076], [-73.813159963843, 40.797197536205594], [-73.81305279469925, 40.79745501885081], [-73.81293369674125, 40.79742600809672], [-73.81304287555434, 40.797178206756236], [-73.81294054438418, 40.797161311658634], [-73.81293578008048, 40.79716166393872], [-73.81293494789331, 40.79716245481534], [-73.81293284475183, 40.797164459862834], [-73.8129291648328, 40.797166605752984], [-73.81292336146136, 40.79716998285826], [-73.8129202516569, 40.797171796465584], [-73.8128838645248, 40.79716656522699], [-73.8128748386123, 40.79720339089459], [-73.81286187927007, 40.79727905644295], [-73.8128564385509, 40.79728728181292], [-73.81281285857212, 40.7972819609744], [-73.81280889302123, 40.79729317504365], [-73.81280166284506, 40.79730661575021], [-73.81279234462082, 40.797320766220274], [-73.81278370953858, 40.79733233221378], [-73.81277515293642, 40.79734538545603], [-73.81277230577534, 40.797354763420444], [-73.81276881553268, 40.79736549126415], [-73.81276578266439, 40.79737819396863], [-73.81276405096054, 40.79738732999527], [-73.81276257472183, 40.797398278557964], [-73.81275869851312, 40.79741526435161], [-73.81275582015948, 40.79742366073419], [-73.81275271305223, 40.797437127400336], [-73.8127491690451, 40.797454113733224], [-73.81274572828688, 40.797468327522594], [-73.81274192974571, 40.797483639193885], [-73.81273820425044, 40.797497825046264], [-73.81273325798236, 40.79751238718732], [-73.81273007863845, 40.797525826261285], [-73.81272693129098, 40.79754152597348], [-73.81272522534644, 40.797554039686005], [-73.81272411198792, 40.79756638432308], [-73.81272266986312, 40.797577583725804], [-73.81272035923969, 40.79759009762841], [-73.81271721148526, 40.79760562663117], [-73.81271511993852, 40.79761627499302], [-73.81271176323354, 40.797629911960534], [-73.81270576732189, 40.79765547713051], [-73.81270208188423, 40.797667798825074], [-73.81269912076951, 40.79768013978927], [-73.81269573423666, 40.79769207498587], [-73.8126923703309, 40.797704037693315], [-73.81268833315595, 40.79771562002926], [-73.81268298891642, 40.797728317132126], [-73.81267734672261, 40.797741374599944], [-73.81267265880527, 40.79775256051823], [-73.81266528833807, 40.79776506519115], [-73.81265900839935, 40.79777757197113], [-73.81265367026252, 40.79778801854782], [-73.81264741968197, 40.79780293807388], [-73.81264333650182, 40.797813394898505], [-73.81263965611126, 40.79782054827667], [-73.81263727248698, 40.797825155033024], [-73.81263062884526, 40.79784062340207], [-73.81262644389636, 40.79784995479337], [-73.81261968125939, 40.79786564041625], [-73.81261238475768, 40.797881306407746], [-73.81260707611486, 40.79789364305031], [-73.8126028416929, 40.79790801703865], [-73.81259847522662, 40.79791847440609], [-73.81259374717058, 40.79793114519963], [-73.81258835898772, 40.79794683188242], [-73.81258390689374, 40.79796233245858], [-73.81258098164079, 40.797974294702854], [-73.81257581468293, 40.797987352937916], [-73.81256990139292, 40.798000220488106], [-73.81256509432681, 40.798011432006646], [-73.81255851915725, 40.79802355198735], [-73.81255231385722, 40.79803474481425], [-73.81254798974666, 40.79804706445951], [-73.81253971935718, 40.79805918101207], [-73.81251976853277, 40.79807487148138], [-73.81250447289142, 40.79808248240446], [-73.8124871944264, 40.79808701115291], [-73.81247216146885, 40.798090111881145], [-73.81245763606678, 40.79809302345942], [-73.81244209193719, 40.79809673481911], [-73.8124216396584, 40.79810073571461], [-73.81240168387899, 40.79810534134321], [-73.81238097565138, 40.798111925724726], [-73.81236614994538, 40.79811680974788], [-73.81234840845782, 40.798121832256356], [-73.81233166889062, 40.79812482196662], [-73.81231000662297, 40.79812489542009], [-73.81229327292131, 40.798125804135395], [-73.81227088457793, 40.79812741562136], [-73.81225365567188, 40.79812694680175], [-73.8122418447208, 40.79812582792779], [-73.81222614069394, 40.79812275958522], [-73.81221163774894, 40.79811848550762], [-73.81219425065645, 40.79811134288449], [-73.81218075829688, 40.79810571935825], [-73.81216552179895, 40.79810086930032], [-73.81214635499957, 40.798098748187854], [-73.81212492730536, 40.79809967670287], [-73.81210600578792, 40.79809898815703], [-73.81208560683429, 40.79809705449707], [-73.81206889293433, 40.79809477654628], [-73.81205366097402, 40.79809261743932], [-73.8120369507112, 40.798089052225535], [-73.81201828860804, 40.798084401278004], [-73.81199958946941, 40.798080454040736], [-73.81198045124523, 40.798076414950366], [-73.81195736238278, 40.79807036195667], [-73.81193328615508, 40.79806524381367], [-73.8119111878113, 40.79806073332424], [-73.81189104345013, 40.79805639505244], [-73.81187091028694, 40.79805246354665], [-73.81185173546726, 40.79804905460535], [-73.81183405253485, 40.79804614430591], [-73.81181683312202, 40.79804232576843], [-73.8117996525634, 40.79803802029594], [-73.81178616019908, 40.798032181620776], [-73.81177838321304, 40.79802505345333], [-73.81177454305899, 40.798012891883666], [-73.81177278320803, 40.797998092674874], [-73.8117721645394, 40.797986511647224], [-73.81177201960828, 40.797974930220626], [-73.81177191147192, 40.79796257605991], [-73.81177101400081, 40.797949102881496], [-73.8117696414228, 40.797935647019976], [-73.81176826706177, 40.797923289618794], [-73.81176638468267, 40.79791019310403], [-73.81176461603962, 40.79789858272427], [-73.81176303610033, 40.79788307350154], [-73.81176160152847, 40.797866439751466], [-73.81176123787368, 40.79785276926218], [-73.81176151918584, 40.797841378504984], [-73.81176055294284, 40.79782696874768], [-73.81175995900935, 40.79781911503005], [-73.8117592958143, 40.79781049661405], [-73.81175701095391, 40.79778880891265], [-73.81175556681939, 40.79776300715793], [-73.81175563627974, 40.79775122856859], [-73.81175668823023, 40.79773946029513], [-73.81175703225773, 40.79773068152684], [-73.81175823993705, 40.797714591690344], [-73.81175853817825, 40.79770976527131], [-73.811710363145, 40.79757777248557], [-73.81164514802275, 40.79743379186729], [-73.81162840215558, 40.797392783459706], [-73.81160938202306, 40.79737240944349], [-73.81149731828836, 40.797344175630805], [-73.81142258808796, 40.7973341750994], [-73.81141359783149, 40.79733314267482], [-73.81139493127712, 40.79733023069814], [-73.81137601742242, 40.797326606331275], [-73.81136027759761, 40.79732372661244], [-73.81134724983879, 40.797321561637474], [-73.81132710244584, 40.79731815151888], [-73.81130991860967, 40.79731507945581], [-73.81129417812912, 40.79731219972686], [-73.81127847342105, 40.797309454577196], [-73.81125930899428, 40.79730682002288], [-73.81123815426196, 40.797303850170955], [-73.8112180055982, 40.797301125209614], [-73.81119587931884, 40.79729809881049], [-73.81117128896017, 40.79729514090888], [-73.81114817783076, 40.797292473569144], [-73.81112261529013, 40.79728921704197], [-73.81109705208627, 40.79728642823876], [-73.81105405315587, 40.79728155833045], [-73.81103806106402, 40.79727980291903], [-73.81102256650476, 40.79727804999617], [-73.81100906400286, 40.7972759379802], [-73.8109844753117, 40.79727208796918], [-73.81095993501873, 40.79726804102266], [-73.81093779801479, 40.79726476259825], [-73.81091914145856, 40.79726214758326], [-73.81090487891294, 40.7972605662012], [-73.81088570261545, 40.79725808535347], [-73.8108694624798, 40.79725619448614], [-73.81084834187374, 40.797253639076104], [-73.81083136757358, 40.797251548313746], [-73.81081587345116, 40.797249956188175], [-73.8108001282021, 40.797248534356505], [-73.81077875780143, 40.79724669050631], [-73.81076326452711, 40.79724480085002], [-73.81074752214538, 40.797242603876285], [-73.81072835591264, 40.797240618726356], [-73.8107045214169, 40.797237842558474], [-73.81068632861088, 40.797235254568804], [-73.81067802496406, 40.797234332440965], [-73.81066347858139, 40.797232722586045], [-73.81065535249127, 40.79723161027188], [-73.81063275189202, 40.797228962932785], [-73.8106126016818, 40.79722687563175], [-73.81059809021173, 40.79722539281021], [-73.81058024354648, 40.79722558026155], [-73.81056081979685, 40.79722245063018], [-73.81054262321187, 40.79722096157759], [-73.81052518125497, 40.797220770408444], [-73.81050672905847, 40.79721769731316], [-73.81048756667118, 40.79721390168356], [-73.81047334442174, 40.79721110441564], [-73.81045391994175, 40.797208002239074], [-73.8104354777112, 40.79720522499854], [-73.81041629934923, 40.797203248488], [-73.81039814806662, 40.797198472473106], [-73.81037948239039, 40.79719528090105], [-73.81035735861853, 40.79719139979826], [-73.81033966453865, 40.79718793976742], [-73.81032099957967, 40.79718473059691], [-73.810302809205, 40.79718099031156], [-73.81028414762471, 40.79717613269167], [-73.8102723533939, 40.79717378132689], [-73.81025271421358, 40.79717076001287], [-73.81023523139702, 40.7971686678228], [-73.8102190260215, 40.79716704747431], [-73.81019887882917, 40.79716383534107], [-73.8101797102143, 40.797158447154274], [-73.81015958378141, 40.7971522087035], [-73.81014464461192, 40.79714680770858], [-73.81012747528214, 40.79713868410242], [-73.81010933372707, 40.79713027945967], [-73.81009215859119, 40.79712388503484], [-73.81007743107779, 40.79711924963545], [-73.81006343385967, 40.79711609317823], [-73.81004625437062, 40.79711099538527], [-73.81003051943543, 40.79710644007199], [-73.81001236523934, 40.797102781050086], [-73.80999759495545, 40.79710062243163], [-73.80998041307762, 40.79709644182592], [-73.80996298971401, 40.797089749447565], [-73.80994974455959, 40.79708454169634], [-73.80993600151355, 40.79707984474496], [-73.80992174394906, 40.79707630906658], [-73.80990138968068, 40.79707128726627], [-73.80988984361326, 40.797069043320185], [-73.80987113560074, 40.797067806448744], [-73.80985762672962, 40.79706841489821], [-73.80983795394502, 40.797065059077426], [-73.80982174290129, 40.79706126820573], [-73.80980455944795, 40.79705764643555], [-73.80978340759057, 40.79705387487999], [-73.80976459797382, 40.79705132207973], [-73.80974398905943, 40.79704793940154], [-73.80972973014313, 40.797045114004405], [-73.8097130241487, 40.79704042201939], [-73.80969905185482, 40.79703688678439], [-73.80968479427642, 40.7970335948258], [-73.80966905877176, 40.797029562141155], [-73.80965361506765, 40.79702670897934], [-73.80963736095772, 40.79702572808021], [-73.80962038294969, 40.79702473609907], [-73.80960244412229, 40.79702402246318], [-73.80958622616353, 40.797022734207665], [-73.80956558817455, 40.79701742306558], [-73.80954753413428, 40.79701583541887], [-73.80952268958546, 40.797014641456755], [-73.80951409062794, 40.797013475007546], [-73.8094944186066, 40.797009787094176], [-73.80947254416617, 40.79700594124622], [-73.80945387858829, 40.797002749524154], [-73.80943669452826, 40.79699950597026], [-73.80941753243617, 40.796995881373554], [-73.80939391855175, 40.79699101659363], [-73.8093693804077, 40.796986057973754], [-73.80934627316799, 40.79698209480544], [-73.80932760703666, 40.79697910074194], [-73.8093123774852, 40.796976220414415], [-73.8092900334429, 40.796970556775186], [-73.80927281774164, 40.7969660521814], [-73.80925513734745, 40.79696193379955], [-73.80924387840999, 40.79695881745693], [-73.80922767374307, 40.796952668258186], [-73.8092102534081, 40.796944958556956], [-73.80919456028478, 40.79693823156787], [-73.80917912794055, 40.79693150668354], [-73.80915996584244, 40.79692382039277], [-73.80914110119411, 40.79691580205357], [-73.8091322623125, 40.7969113655869], [-73.80911535048463, 40.796904142536754], [-73.80909745606357, 40.79689656655615], [-73.80908616701892, 40.796891118195774], [-73.80907143110211, 40.79688538150973], [-73.80907072097281, 40.796885104754175], [-73.80905354429485, 40.79687915093073], [-73.80903341023478, 40.796875714131126], [-73.80901965126861, 40.79687207044294], [-73.80900391810268, 40.79686701928463], [-73.80898577195752, 40.79686028937491], [-73.80897401932314, 40.796855307965956], [-73.80895904858392, 40.79684882756643], [-73.80894310831127, 40.796841668587966], [-73.80892546162437, 40.796834580489886], [-73.80890997785619, 40.79682912348329], [-73.80889110948904, 40.796822420337996], [-73.80887516404648, 40.796816990035474], [-73.80886090939916, 40.796812707737544], [-73.80884103043104, 40.796807270907344], [-73.80882409683717, 40.79680342542361], [-73.80880688040418, 40.796798894457346], [-73.80879338042196, 40.79679596315426], [-73.80878378171906, 40.79679605478256], [-73.80876898946094, 40.796801659763446], [-73.80875743667507, 40.796801783830546], [-73.80874548028396, 40.79680159328424], [-73.80872936734276, 40.79680098959068], [-73.8087136659147, 40.79679718131568], [-73.80869968005003, 40.79679035004337], [-73.80868351883343, 40.796781616426905], [-73.80866756808085, 40.796777889484176], [-73.80864789968977, 40.79677305572622], [-73.80863317201425, 40.796768691867534], [-73.80861440179193, 40.79676451780218], [-73.80860184495185, 40.79676339848871], [-73.80858291373147, 40.796761935380665], [-73.80856891135323, 40.796760623857], [-73.80855019494895, 40.79676303328068], [-73.80853392846265, 40.79676638340484], [-73.80851716669386, 40.79676833671538], [-73.80850091691468, 40.79677009269176], [-73.80847733110448, 40.79676755034129], [-73.80845913952308, 40.79676461120349], [-73.80844759279134, 40.79676240346452], [-73.8084279398142, 40.796760723358936], [-73.80841169801579, 40.79675939971804], [-73.80839646373336, 40.79675773465751], [-73.8083802093966, 40.79675701492001], [-73.80836180477306, 40.796753805361874], [-73.80834682797352, 40.79674899912062], [-73.80833282483468, 40.79674420270511], [-73.80831687898839, 40.79673893482401], [-73.80829626784501, 40.79673217490735], [-73.80827762062165, 40.79672717207623], [-73.80825500891919, 40.79672395600585], [-73.80823779769253, 40.796722000162326], [-73.80822205447603, 40.796720164528985], [-73.80820630829253, 40.79671989458029], [-73.8081917791541, 40.796720392580426], [-73.80817557123811, 40.796719402672906], [-73.80816130936734, 40.79671764974122], [-73.8081455680239, 40.796715624294364], [-73.80812163219092, 40.796711550777054], [-73.80810995368086, 40.796730441427044], [-73.80778748187672, 40.79668633750116], [-73.8076775124608, 40.796672214788266], [-73.80767740565278, 40.79667217825706], [-73.80747196889736, 40.79660817064827], [-73.8074654442394, 40.79660659255057], [-73.80744754051358, 40.796602249536356], [-73.80742760562904, 40.79659966750282], [-73.8074116102241, 40.7965994513074], [-73.80739395120524, 40.796608437176744], [-73.80704316723262, 40.79654748776339], [-73.80642626151342, 40.79640785415845], [-73.8064194096874, 40.79640630342859], [-73.8063633145256, 40.79639360543771], [-73.80632825394339, 40.79656033627029], [-73.80632013155257, 40.79655929334695], [-73.80630503940789, 40.79662616654991], [-73.80631451940103, 40.796626182479905], [-73.80627331882216, 40.79682269333237], [-73.80623135012367, 40.79681747758278], [-73.80627525037363, 40.796624059611865], [-73.8062833766485, 40.79662407326925], [-73.80629846516301, 40.79655822766332], [-73.80629034788133, 40.79655512619506], [-73.80633076714675, 40.796392740435934], [-73.8063152984311, 40.796394416166784], [-73.80629654242199, 40.79639809500013], [-73.80628026898333, 40.796403749776005], [-73.8062674235055, 40.79641201231489], [-73.80625797464138, 40.796421685931584], [-73.8062487657038, 40.79643473659161], [-73.8062423354427, 40.796445729852366], [-73.80623626670253, 40.796458237311825], [-73.80622875229942, 40.79647112882171], [-73.80622009174196, 40.79648321629291], [-73.80621099652869, 40.79649346745746], [-73.80619840214537, 40.796501262177415], [-73.80617916661214, 40.79650638994973], [-73.8061661323947, 40.79650678248943], [-73.80614472276086, 40.79650581872913], [-73.806127995381, 40.796504439671104], [-73.80610831639821, 40.79650356006465], [-73.80608423063069, 40.79650236730741], [-73.80606112922906, 40.79650076174371], [-73.80602544709858, 40.79649756730248], [-73.80600089255304, 40.79649437549669], [-73.80597825948341, 40.79649068903468], [-73.80595714687374, 40.79648576410021], [-73.80593700219839, 40.79648205602015], [-73.805915882523, 40.79647910283949], [-73.80589130613835, 40.79647569369314], [-73.80586819986425, 40.79647153047321], [-73.80585101869465, 40.796467251082035], [-73.80582991150861, 40.79646024563617], [-73.80581764463896, 40.79645731656382], [-73.80580191197902, 40.79645266183609], [-73.80578794170157, 40.79644779279518], [-73.80577123952722, 40.79644216427995], [-73.80575309750408, 40.79643417290387], [-73.80573520356036, 40.79642638062802], [-73.80572099337196, 40.796419576089455], [-73.80570700476177, 40.79641359095395], [-73.805691559233, 40.7964073157198], [-73.80567684032862, 40.796400040912076], [-73.80565946003267, 40.79639073656896], [-73.80564280467912, 40.79638123593327], [-73.80563127343525, 40.796373778196134], [-73.80561731327747, 40.7963662083079], [-73.80560188750287, 40.796357402456984], [-73.80558573126645, 40.796346911742305], [-73.80557448693745, 40.79633866125079], [-73.80556324373815, 40.796330024949285], [-73.80555059305485, 40.796320476609566], [-73.80553718556862, 40.79631041319148], [-73.80552400234379, 40.79630047646424], [-73.80551279570177, 40.796291479537345], [-73.80550072914782, 40.79628058996452], [-73.80548783217695, 40.7962700866429], [-73.80547629446686, 40.79626062963154], [-73.80546459835206, 40.796248804213405], [-73.80545166035743, 40.7962360496127], [-73.80544014211031, 40.79622438699183], [-73.80542724845094, 40.79621275773087], [-73.80541756419926, 40.796201871668075], [-73.8054090101802, 40.79618931326416], [-73.80540153443536, 40.79617656838161], [-73.80539573274972, 40.7961708215133], [-73.80537703025449, 40.796168565701926], [-73.80536638630139, 40.79616813427669], [-73.805358191995, 40.79622729474982], [-73.80535141713001, 40.79622831258501], [-73.80534456769023, 40.7962550605126], [-73.80535269325873, 40.79625507423445], [-73.80534980160489, 40.79631785131887], [-73.80530104939162, 40.796316740876854], [-73.80531071962977, 40.7962519171942], [-73.80532426114101, 40.79625194006576], [-73.80532975387253, 40.796226217951194], [-73.80532433939833, 40.79622517953089], [-73.8053394448758, 40.796163928288756], [-73.80533377312027, 40.796162675336944], [-73.80531364094746, 40.796158958457376], [-73.80529962989374, 40.79615668358573], [-73.80528046127395, 40.79615552526747], [-73.80525954163315, 40.79615703032125], [-73.80523689628167, 40.796157820975026], [-73.8052225881337, 40.796159821844114], [-73.80520139929341, 40.796164504890974], [-73.80518217470588, 40.796170569485014], [-73.80516468832593, 40.79617352891591], [-73.80514545931521, 40.796176433298086], [-73.80513069275192, 40.79617734429853], [-73.80511571013656, 40.79617517531318], [-73.80509556517517, 40.796171387008116], [-73.80507987209377, 40.7961648494309], [-73.80506295214927, 40.79615669801512], [-73.80505068945472, 40.79615214789262], [-73.80503937067661, 40.79614466365112], [-73.80501360845619, 40.79614267075878], [-73.80498771399782, 40.79614283333299], [-73.80495432127336, 40.79614626778609], [-73.80492191157646, 40.79615327880182], [-73.80489114466053, 40.796163723659575], [-73.80483686471369, 40.79617182108918], [-73.8047817239482, 40.79617516310486], [-73.8047264413334, 40.79617370613075], [-73.8046717376879, 40.79616746916408], [-73.80464949613757, 40.796362981805984], [-73.80462782077647, 40.79636500362028], [-73.80466149435813, 40.79642784265904], [-73.8046343775328, 40.79643808895626], [-73.80459395457137, 40.79636803404015], [-73.80462921073267, 40.7963526563424], [-73.80464873976852, 40.79615919598311], [-73.80462441010651, 40.796142687004036], [-73.80460953451075, 40.796135457863315], [-73.8045932738987, 40.796138518125574], [-73.80457614516594, 40.796141213550975], [-73.80455768026067, 40.796146342862215], [-73.80453769015642, 40.79615086464995], [-73.80451994706745, 40.796156408631695], [-73.80449870742996, 40.7961662870946], [-73.80447943665723, 40.796175745724426], [-73.80446363544613, 40.79618195052848], [-73.80444884031962, 40.79618845405957], [-73.80443231494576, 40.79619496403454], [-73.80441601209601, 40.79620256396969], [-73.80440169427989, 40.79620761854028], [-73.80438593705851, 40.7962110759925], [-73.80436129307107, 40.79621430308925], [-73.804348259354, 40.79621409099277], [-73.80434380665973, 40.796232782279766], [-73.80433026988077, 40.79623070074944], [-73.80429711363743, 40.79645295479134], [-73.80432012674352, 40.79645608118631], [-73.80431189289475, 40.79649311945956], [-73.80429293553247, 40.79649205799797], [-73.80427353614904, 40.79664126132841], [-73.8042356338882, 40.79663502181065], [-73.80425907810624, 40.79649200050714], [-73.80424147814796, 40.796489912066626], [-73.8042429505587, 40.7964497749908], [-73.8042673271141, 40.796449816387295], [-73.80431138699923, 40.79621287616002], [-73.8042899654997, 40.79621261530619], [-73.80426782617938, 40.796214253134515], [-73.80425351934029, 40.79621556987958], [-73.80423480566085, 40.796217051019426], [-73.80421143335055, 40.796218713044354], [-73.80419839289532, 40.79622094159628], [-73.80417716911634, 40.796225705477816], [-73.80416535642863, 40.79622900877676], [-73.80412218500379, 40.796528819706644], [-73.8040923878819, 40.796529798342384], [-73.8041364476697, 40.79623423117777], [-73.80412813146489, 40.7962354022881], [-73.80411471611377, 40.79623992696619], [-73.80410066074184, 40.79624462997462], [-73.80407969967965, 40.79624843101396], [-73.80406711935909, 40.796251678049764], [-73.80404987656715, 40.796256259873836], [-73.80403683589735, 40.79625840715736], [-73.80402201336017, 40.7962621460838], [-73.80400524331003, 40.796267358935346], [-73.80395823064364, 40.79655735901795], [-73.8039311446323, 40.79655731294737], [-73.80397418925072, 40.79627825591965], [-73.80395838852573, 40.796284271351574], [-73.80394308744964, 40.79628934228563], [-73.80392534169847, 40.79629598513761], [-73.8039070863923, 40.79630229324206], [-73.80388881964235, 40.79630904275336], [-73.8038740040376, 40.79631431268667], [-73.80386141775243, 40.79631912755978], [-73.80384612412581, 40.796325854649226], [-73.80381886202728, 40.79651492494286], [-73.80382563258884, 40.7965149364661], [-73.80381716794693, 40.79663019323661], [-73.80377383371204, 40.79662909019768], [-73.80379312974867, 40.796514881143835], [-73.80380261038667, 40.79651489728153], [-73.80382294195988, 40.79633501284091], [-73.80381452007485, 40.796338227893145], [-73.80379875661363, 40.796343856383096], [-73.80378346523015, 40.79634981821225], [-73.80377084078584, 40.79635553379425], [-73.80375800294107, 40.79636132858606], [-73.80373406114016, 40.79637130114762], [-73.803713847762, 40.79637898495606], [-73.80369410956779, 40.79638594820592], [-73.80367585440008, 40.79639241776835], [-73.80365958021285, 40.79639832580685], [-73.8036430150915, 40.796406085507584], [-73.80360867810487, 40.796605137254886], [-73.80361815268297, 40.796607211948924], [-73.80361250744673, 40.79668439415801], [-73.80357594508133, 40.796683302582316], [-73.80358565890843, 40.796604068756515], [-73.80359378518052, 40.7966040826035], [-73.80361288546553, 40.79642085682052], [-73.80359559638315, 40.79642914834672], [-73.80358525110182, 40.796433795127015], [-73.80357165469471, 40.79643958910802], [-73.80355708604628, 40.79644519262939], [-73.80354081032112, 40.79645183792744], [-73.80352920378782, 40.7964579405262], [-73.80351687211322, 40.796464529386455], [-73.80350403526204, 40.79646974886958], [-73.80348822878426, 40.79647790721151], [-73.80347562877238, 40.79648705321944], [-73.80346385689631, 40.79649726274884], [-73.80345417357385, 40.796510123408844], [-73.8034504973114, 40.79652318378254], [-73.80344894059499, 40.79653646287301], [-73.80344742272665, 40.796548977946415], [-73.80344553835042, 40.79656094173905], [-73.80344369578071, 40.79657532048094], [-73.80344123052014, 40.79659139820807], [-73.80344024312643, 40.79660485065268], [-73.80343933812479, 40.79661832082783], [-73.80343746756938, 40.79662970885924], [-73.80343461451517, 40.796641095214916], [-73.80343133110553, 40.79665324659522], [-73.80342210240818, 40.796668908213775], [-73.80341320056758, 40.79667764539373], [-73.8034012144535, 40.79668805390621], [-73.80338930348425, 40.79669690238192], [-73.80337425529169, 40.796704675017985], [-73.80335968645421, 40.796710765007575], [-73.80334996362961, 40.79671302642995], [-73.80333421283683, 40.79671380099854], [-73.80331622699072, 40.79671338332618], [-73.8032975542383, 40.79671252254703], [-73.80327956562543, 40.796713041331024], [-73.80325865101189, 40.79671306879196], [-73.80323948126862, 40.79671239712883], [-73.80321980170481, 40.796711705829374], [-73.80319964931766, 40.796710410627966], [-73.80318267007644, 40.7967101926684], [-73.8031629872915, 40.796710591106226], [-73.8031452609785, 40.79671037253551], [-73.80313149084127, 40.7967109242997], [-73.80311476899286, 40.79671155394541], [-73.80309631998239, 40.79671144118146], [-73.8030776116352, 40.796710616658906], [-73.80305684072003, 40.79671035601754], [-73.8029997067683, 40.79670883609421], [-73.8029716822265, 40.79668512306075], [-73.8029649323685, 40.79659566387474], [-73.80295830316544, 40.79640959134531], [-73.80294925160216, 40.796404929555855], [-73.80293438234197, 40.79640029299987], [-73.80291966493796, 40.79639678314213], [-73.8029054106603, 40.79639253511996], [-73.80288996006088, 40.796388286223], [-73.80286932091553, 40.79638338783343], [-73.80285483860484, 40.7963802916545], [-73.80284305054556, 40.79637524757408], [-73.80282883359777, 40.79637082839547], [-73.80280674002948, 40.79636502772807], [-73.80279125662157, 40.796363821524814], [-73.80277867985394, 40.79636539417764], [-73.80276267486211, 40.79636813950395], [-73.80274520035988, 40.79636681295466], [-73.80272775636631, 40.79636352289022], [-73.80271057416486, 40.796359594329495], [-73.80269289795285, 40.796354701482315], [-73.80267352024784, 40.79634822092956], [-73.802660748789, 40.796343561628035], [-73.80264036022909, 40.796338097055646], [-73.80263479739018, 40.7963398979772], [-73.8026308623715, 40.79634828409239], [-73.80262670946287, 40.796362278415785], [-73.80262272892377, 40.79636609117787], [-73.80261322240626, 40.79635503447497], [-73.80260474280809, 40.79634171073391], [-73.80259968008895, 40.796330669848665], [-73.80259458984929, 40.79631679304674], [-73.80259127882734, 40.79630237919008], [-73.80257934475026, 40.7962954431361], [-73.80256042483781, 40.796293897807814], [-73.80254565454585, 40.79629218016549], [-73.80252994857635, 40.79628983889648], [-73.80251298962955, 40.79628712658777], [-73.8024952957972, 40.79628380037706], [-73.80247562580497, 40.79627970552148], [-73.80245599232745, 40.796275637696525], [-73.80243877245934, 40.79627275254617], [-73.80242602952, 40.79627045252708], [-73.80240609420619, 40.79626811337311], [-73.80239061099638, 40.796266627683885], [-73.80236872599959, 40.796266482611124], [-73.8023535941851, 40.79626640288978], [-73.80233538569176, 40.79626513817682], [-73.80231448761299, 40.796263950097966], [-73.80229602831757, 40.7962632602307], [-73.80227659618754, 40.796262731191355], [-73.80226038840061, 40.796262209355376], [-73.80224143004784, 40.796261959382555], [-73.80222445003956, 40.79626203897286], [-73.80220302420825, 40.796262713687966], [-73.80218656535925, 40.79626263116485], [-73.80216835248213, 40.79626284298653], [-73.8021481721449, 40.79626343908189], [-73.802130466399, 40.796264425902756], [-73.80211027052047, 40.7962656521952], [-73.80209452071918, 40.796266642871146], [-73.80207924242717, 40.796268155860986], [-73.80206005228753, 40.796270239088535], [-73.80204134721818, 40.796272286960516], [-73.80202829757927, 40.79627349853986], [-73.80201128804933, 40.79627542417773], [-73.80198985243008, 40.79627960983292], [-73.80196868129457, 40.796282401122504], [-73.80194750952698, 40.79628555241896], [-73.8019302375867, 40.79628809844296], [-73.80191202601752, 40.796291498066516], [-73.80189745097366, 40.796295732852734], [-73.80188195549098, 40.79629866890624], [-73.80186594983456, 40.79630163038208], [-73.80184892695604, 40.79630418553339], [-73.80182161320671, 40.79630855102588], [-73.80181347984326, 40.79630996871342], [-73.80179620718019, 40.79631290169978], [-73.80177947698411, 40.79631614185258], [-73.80176331229939, 40.79641980689219], [-73.80177549860801, 40.79642085712563], [-73.80176584941816, 40.79647744802096], [-73.80175501879769, 40.79647640011811], [-73.80174543088519, 40.796512406605586], [-73.80172106831401, 40.79650721832071], [-73.80175550251785, 40.79632129591802], [-73.80174842418782, 40.796323005234456], [-73.80173361368553, 40.79632635573125], [-73.80171489307945, 40.79633013272033], [-73.8017037946004, 40.79633228358364], [-73.80168461858926, 40.796333979789445], [-73.80166959318859, 40.796334062661046], [-73.8016533732407, 40.7963335489276], [-73.80163169501581, 40.79633532039337], [-73.80161742295034, 40.79633729507837], [-73.80159820789405, 40.79633973886827], [-73.80157800485001, 40.796343432033694], [-73.80156322101915, 40.796346261047994], [-73.80154742752093, 40.79634977350082], [-73.80153413694384, 40.796352307054654], [-73.8015146919568, 40.79635608326664], [-73.80149792583289, 40.796359683496355], [-73.80147573299986, 40.796363480768385], [-73.80145799689467, 40.79636679988939], [-73.80143779242482, 40.7963711874204], [-73.80142770205491, 40.796373393769514], [-73.80141216948559, 40.796376851704906], [-73.80139637317968, 40.79638130009641], [-73.80138133427633, 40.79638583104069], [-73.80136578377727, 40.796390937388566], [-73.8013497738193, 40.79639554673248], [-73.80133376247049, 40.796400399821025], [-73.80131994336858, 40.796405482342784], [-73.80130486581015, 40.796410922203144], [-73.80128784249827, 40.79641788921872], [-73.80127525576586, 40.79642278438851], [-73.8012635401182, 40.79642542114168], [-73.80125286385423, 40.796433444286116], [-73.80123853711241, 40.796441686121256], [-73.80122150671556, 40.796450356512814], [-73.8012057027607, 40.796457577577264], [-73.80119112074898, 40.79646367531168], [-73.8011758275835, 40.796470367537864], [-73.80116053429656, 40.796476877996845], [-73.80113832991798, 40.79648452056114], [-73.80111021893406, 40.79648955092577], [-73.80109174499097, 40.7964939676837], [-73.8010742560219, 40.79649799865226], [-73.80105697876785, 40.79650221928735], [-73.80105270487208, 40.796504581221846], [-73.80105234940854, 40.7965048407752], [-73.8010390964323, 40.79651454358488], [-73.8010260435534, 40.796520725857825], [-73.80101126230589, 40.79652237458959], [-73.80099724908189, 40.79651295842261], [-73.80098709475455, 40.796500783910474], [-73.80097937436199, 40.79648726435071], [-73.80097496351827, 40.79647622451938], [-73.80097218192778, 40.796463126771684], [-73.80097149820028, 40.796450419127275], [-73.8009717940961, 40.796438452631314], [-73.8009700623576, 40.796423104317654], [-73.80096720441102, 40.796411878868895], [-73.80096377690244, 40.796401004407294], [-73.80095706562146, 40.796386773432765], [-73.80094885668566, 40.796378142422355], [-73.80093513210224, 40.79636738403013], [-73.80092312792937, 40.79635976317486], [-73.80090645704925, 40.79635133331902], [-73.8008905198351, 40.79634348001005], [-73.80087826048806, 40.79633789414244], [-73.80086307346713, 40.79633253590837], [-73.80084564154882, 40.796329074871416], [-73.80082718668105, 40.79632712498583], [-73.80080944610103, 40.79632780631236], [-73.80079149908737, 40.79633021597372], [-73.80077548151479, 40.7963330152806], [-73.80076143013369, 40.79633659379395], [-73.80074440101508, 40.79634092185982], [-73.80072839626222, 40.79634356102807], [-73.80071238154792, 40.79634591906943], [-73.8006971004959, 40.79634811638005], [-73.80067173268918, 40.79635174668059], [-73.80065920162392, 40.796353868162015], [-73.80064461930634, 40.79636035331308], [-73.80064068730506, 40.79637154893389], [-73.80063995811872, 40.79637360069604], [-73.8006369837156, 40.79638965118283], [-73.80063576415175, 40.796400870090075], [-73.8006333403199, 40.79641525599174], [-73.80062752602298, 40.79642570069056], [-73.80062184740085, 40.7964267623605], [-73.80061622405263, 40.79642482525408], [-73.80060782494134, 40.796416554070035], [-73.80060754119205, 40.79641627414679], [-73.80060744918362, 40.796415345729656], [-73.80060596182751, 40.79640167325428], [-73.80060127994771, 40.79638613170049], [-73.80060009976283, 40.796384643541316], [-73.80059411459065, 40.79637714229044], [-73.80057920611054, 40.796365859798634], [-73.80056648026334, 40.79635779622812], [-73.80055487781824, 40.79634655563871], [-73.80054635361815, 40.79633641062984], [-73.80053630920082, 40.79632327232663], [-73.80052713996201, 40.79631071249456], [-73.80051806622043, 40.79630188998289], [-73.80050483037087, 40.79629365431186], [-73.80049160101078, 40.79628781476605], [-73.80047518251, 40.79628221177242], [-73.80045652197957, 40.796277767018616], [-73.80044053586028, 40.7962741914906], [-73.80042458719116, 40.79627018347457], [-73.80040541403451, 40.79626614357133], [-73.80039069456676, 40.796263155395856], [-73.80036547554788, 40.79633692038319], [-73.80028591188065, 40.796323549055714], [-73.80032163038219, 40.79625357193045], [-73.80029381644957, 40.79625497350856], [-73.80027683020073, 40.796257059252625], [-73.80027196375512, 40.79625780384678], [-73.80025662935843, 40.79626014926658], [-73.80023494299958, 40.79626492049996], [-73.80021669617476, 40.79626850859577], [-73.80020438000442, 40.79627005394539], [-73.80012080488287, 40.79663141271139], [-73.80008350203387, 40.79662756747462], [-73.8001061966088, 40.79652552022694], [-73.80012111139985, 40.79652932663622], [-73.80012619978814, 40.79649152565467], [-73.80006403229244, 40.79648385607758], [-73.80007912196696, 40.796429058401266], [-73.80013384459131, 40.796431043825166], [-73.80014389123812, 40.796398922005814], [-73.80013145889008, 40.796397009925165], [-73.80014405015416, 40.7963459884526], [-73.80015898192978, 40.79634412432019], [-73.80018235388114, 40.796266913696826], [-73.80014788579977, 40.7962610329355], [-73.80012820994482, 40.796259160898025], [-73.80011007447156, 40.79625715817823], [-73.80010420807422, 40.79625343715025], [-73.80009214191587, 40.79625467769023], [-73.80006953790154, 40.79625326896516], [-73.80005184517341, 40.7962496183871], [-73.80003711898871, 40.79624498338018], [-73.80002190146452, 40.79623806766627], [-73.80000892416695, 40.796231038401885], [-73.79999006404911, 40.79622164938486], [-73.79997044383867, 40.79621329552516], [-73.79995275855609, 40.79620739273353], [-73.79993656052181, 40.79620319591864], [-73.79991028638486, 40.79619689122896], [-73.79989209648481, 40.7961933491572], [-73.79987296005098, 40.796189137348], [-73.79985722418023, 40.796185489132334], [-73.79984300442064, 40.79618179094423], [-73.79982777405681, 40.79617893683496], [-73.79980958312706, 40.79617596014662], [-73.79979040656494, 40.79617351363638], [-73.79976975753038, 40.79617200052935], [-73.7997495701958, 40.79617089396722], [-73.79973334755276, 40.79617100080789], [-73.79971142246335, 40.79617225064719], [-73.79969419075046, 40.79617313019885], [-73.79968067900413, 40.79617450270642], [-73.79966586398123, 40.796179258300214], [-73.79965401659112, 40.796190431251624], [-73.79964743054035, 40.79620573765431], [-73.7996471880429, 40.796215264233474], [-73.79964913389605, 40.79623041358494], [-73.79965288305965, 40.7962442788058], [-73.79965740033595, 40.79625532708224], [-73.79966259905612, 40.796268258355326], [-73.79966916852729, 40.79628248915947], [-73.79967547696066, 40.796296367036305], [-73.79967980703289, 40.79631058473401], [-73.79967688098031, 40.796322358860266], [-73.79967362498456, 40.7963253240965], [-73.79966872949248, 40.79632978348992], [-73.7996566146603, 40.79633547035819], [-73.79963988478306, 40.79633900722375], [-73.79962511332712, 40.796337478006066], [-73.79960548292512, 40.79633222060566], [-73.7995868313636, 40.79632437177867], [-73.79957360324309, 40.79631814514583], [-73.79956913507226, 40.79631421946039], [-73.79956502217931, 40.79631061017801], [-73.79955567949882, 40.79629692570379], [-73.7995518868553, 40.79628549990874], [-73.79954855941892, 40.79627294965463], [-73.79953894969762, 40.7962611277643], [-73.79952616090942, 40.79625051574062], [-73.79951229230872, 40.79623986565193], [-73.79949978243134, 40.796231360574424], [-73.79948485160043, 40.796223545509406], [-73.79946988100619, 40.79621736960891], [-73.79945221348257, 40.79620968379879], [-73.79943604799755, 40.796202595649554], [-73.79941862884057, 40.796195154518315], [-73.79940489053642, 40.79618904391099], [-73.79939191142736, 40.79618234778178], [-73.79938044493386, 40.7961733240058], [-73.7993739088909, 40.79616411716154], [-73.79936903390198, 40.7961497021637], [-73.79936493097021, 40.79613511495324], [-73.79936036945237, 40.796123112531], [-73.7993530051208, 40.796113356435676], [-73.7993392679995, 40.79610298498183], [-73.79932384219168, 40.79609448502781], [-73.79930839755895, 40.79608813589731], [-73.79929245187914, 40.796083173672514], [-73.79927671742351, 40.796078563114044], [-73.79926052236367, 40.7960738430273], [-73.79924236198163, 40.79606821052252], [-73.79922519281368, 40.79606414643891], [-73.79920945494361, 40.79606118263981], [-73.79919028118897, 40.7960573851098], [-73.79917089630341, 40.79605342588177], [-73.79914412235162, 40.7960479220904], [-73.79913650747572, 40.796046476646225], [-73.79912224589431, 40.7960445342971], [-73.7991023460217, 40.796042490985556], [-73.79908216240186, 40.79604009704793], [-73.79906743699222, 40.796038675390086], [-73.7990480121733, 40.796036192138025], [-73.79903175979597, 40.79603476781349], [-73.79901605331288, 40.79603287620509], [-73.79899540637172, 40.79603061697241], [-73.79897526045968, 40.796027564204444], [-73.7989617239453, 40.796025171954206], [-73.79894797183704, 40.79602336549755], [-73.79893103859114, 40.796019877566174], [-73.79891628202127, 40.79601702483288], [-73.7988936889727, 40.79601223975712], [-73.79888412419602, 40.79600923290808], [-73.79886250737025, 40.79600265834815], [-73.79884236755629, 40.79599730058574], [-73.7988261696431, 40.79599310361464], [-73.7988094980293, 40.79598933786237], [-73.79879228361186, 40.79598450670394], [-73.79877535333848, 40.79597974958247], [-73.79875792623176, 40.795975054748226], [-73.79874404894636, 40.79597142017925], [-73.79873706372604, 40.79596958815439], [-73.79872429345869, 40.79596464948793], [-73.79871080239967, 40.79595911451137], [-73.79869265794949, 40.79595224845813], [-73.7986767201006, 40.79594450406742], [-73.79866788360786, 40.79593976877286], [-73.7986603114518, 40.79593572187007], [-73.79865039393185, 40.79593164289306], [-73.79864166023147, 40.79592805230953], [-73.7986259659863, 40.79592212699287], [-73.79860930325653, 40.79591543422357], [-73.79859780932355, 40.79590758835969], [-73.79858655954654, 40.795901230044194], [-73.79857655365221, 40.79589102735077], [-73.79856759450402, 40.79588006189988], [-73.79855920869174, 40.79587085345149], [-73.79854479080151, 40.79585844561401], [-73.79853413510928, 40.79584771977205], [-73.79852481785419, 40.79583694249297], [-73.79851657310793, 40.795828616476584], [-73.79850627113808, 40.795818575759], [-73.79849554338615, 40.795808210469595], [-73.7984799407683, 40.79579547840143], [-73.79846556486665, 40.79578450229931], [-73.79845270192942, 40.795775096208416], [-73.79843860996212, 40.79576433787985], [-73.79842203844572, 40.795750909716574], [-73.79840984754314, 40.79574218913753], [-73.79839399157177, 40.79573123791188], [-73.79837918620188, 40.795721163004366], [-73.79836274355313, 40.79571213812791], [-73.79834543315964, 40.79570394935792], [-73.79833298318368, 40.79569896564129], [-73.79831880512701, 40.79569306052549], [-73.79829932203045, 40.79568628152612], [-73.79828667622118, 40.79567914559811], [-73.79826932036622, 40.79567047041085], [-73.79825133255324, 40.79566273058178], [-73.7982303796071, 40.79565231168665], [-73.79821446551013, 40.795644845030154], [-73.79819816917238, 40.79563818031647], [-73.79818525669026, 40.795633213866864], [-73.79817107928987, 40.79562710116998], [-73.79815527967298, 40.7956209231435], [-73.79814210642174, 40.79561595623244], [-73.79812897914934, 40.79561160204061], [-73.79811108045207, 40.79560588856535], [-73.7980927750326, 40.795601489616715], [-73.79808137211997, 40.79559892091023], [-73.79806025837736, 40.79559443578703], [-73.79804268637255, 40.79559061339755], [-73.79802157304503, 40.795585993242504], [-73.79800615379436, 40.7955832730837], [-73.79798771425926, 40.79558014305576], [-73.79797301677294, 40.795577624015415], [-73.79795974255607, 40.79557477258462], [-73.79793782652314, 40.795569025069], [-73.79792332365892, 40.795565208041516], [-73.79790740890826, 40.7955615329438], [-73.79789832686915, 40.795559221921835], [-73.79767259943343, 40.795937111998605], [-73.79769744193771, 40.795948499081994], [-73.79766994008878, 40.79599382238175], [-73.79758295471345, 40.79596720215479], [-73.79761542612654, 40.79592377762284], [-73.79763779690308, 40.79592948863961], [-73.79787014914405, 40.795552013146484], [-73.79784464434063, 40.79554585424318], [-73.79782444503947, 40.79554063122325], [-73.79780704134033, 40.79553590998416], [-73.797791128113, 40.795531955440616], [-73.79777449092573, 40.79552808908322], [-73.79775789033545, 40.79552451210403], [-73.79774121880918, 40.7955203406178], [-73.7977279329275, 40.79551749014499], [-73.7977131820566, 40.79551310558236], [-73.79769315057041, 40.79550736082476], [-73.79767124818858, 40.79550141576968], [-73.79764980808868, 40.795495173326856], [-73.7976331375614, 40.79549067783258], [-73.7976140897327, 40.795485204839196], [-73.79759932647495, 40.79548083900295], [-73.79758077321341, 40.79547619578823], [-73.7975646779006, 40.79547348426707], [-73.79754779349089, 40.795469446734245], [-73.79753118113057, 40.79546576215032], [-73.79751815428708, 40.79546327228798], [-73.79749920579077, 40.7954598618552], [-73.79747981658083, 40.795457414083636], [-73.79746137620008, 40.7954545814949], [-73.7974443322619, 40.79545219161408], [-73.79742732477537, 40.79544963108649], [-73.79741173665438, 40.795447876167735], [-73.79739382518174, 40.79544639492012], [-73.79737732260054, 40.79544510444496], [-73.79735897134293, 40.79544435249451], [-73.79733705524903, 40.795442638540955], [-73.7973219980645, 40.79544135059937], [-73.79730490010195, 40.795437448853654], [-73.79728820652683, 40.79543698744806], [-73.79727119608756, 40.79543923621153], [-73.79725679210726, 40.795449223702555], [-73.79724128273867, 40.795452879802454], [-73.79723241146328, 40.795447532868266], [-73.79723144562945, 40.795438085416585], [-73.79723508078402, 40.79542703329173], [-73.79723956239108, 40.79541461347051], [-73.79724211223676, 40.79541010733433], [-73.79724685379013, 40.79540172281963], [-73.79725209088424, 40.79538979132317], [-73.79725643301914, 40.79537662476282], [-73.79725795937395, 40.79536169657784], [-73.79725814691703, 40.7953546913302], [-73.79725820028814, 40.7953525839544], [-73.79725115660972, 40.79534247698615], [-73.79723849400851, 40.79533751086955], [-73.79722012032018, 40.795332264653396], [-73.79720454504239, 40.79532608739329], [-73.79718859775392, 40.79532169194696], [-73.79717408258826, 40.795317964932025], [-73.7971595455066, 40.79531354549417], [-73.79714308909722, 40.795308689623354], [-73.79712783133192, 40.79530371842457], [-73.79711357425079, 40.79530037063109], [-73.79709866318268, 40.79529823741664], [-73.79708053766431, 40.79529713401588], [-73.7970638232024, 40.795295726692636], [-73.79705038771282, 40.79529483022955], [-73.79702796654013, 40.79529207013656], [-73.79700998564532, 40.79528994976508], [-73.79698789062624, 40.795288955621615], [-73.79696684355919, 40.79528586590168], [-73.79693984258415, 40.79528080312872], [-73.79691508645405, 40.79527439337988], [-73.796890840059, 40.79526781214025], [-73.79687221836762, 40.7952619352008], [-73.79685256100464, 40.795254129723254], [-73.79683487523477, 40.79524856016119], [-73.79681863521287, 40.79524287688226], [-73.79679807499622, 40.79523578061414], [-73.79678926844697, 40.79523293692441], [-73.79676409743335, 40.79522257398446], [-73.79675010102949, 40.79521565998949], [-73.79673567660085, 40.795208943420675], [-73.79651939930964, 40.7955947934965], [-73.79654175232895, 40.79560617576532], [-73.79651175643305, 40.79565338558407], [-73.79648690334503, 40.7956457789387], [-73.7964343873751, 40.79573643035909], [-73.79640953426332, 40.795728824199614], [-73.7964670270416, 40.795638182583744], [-73.79644218574505, 40.79562679523604], [-73.79646720043996, 40.7955814678336], [-73.79649206440247, 40.79558529228156], [-73.79670587562217, 40.79519762595181], [-73.79669238103237, 40.79519291964716], [-73.79667705374753, 40.79518734550945], [-73.79666194262768, 40.79518100532228], [-73.79664707776274, 40.795175161945394], [-73.7966357899755, 40.79516972995422], [-73.79661779069339, 40.79516231484811], [-73.79659841538498, 40.79515506751681], [-73.79658446392915, 40.795148892032984], [-73.79657173309991, 40.79514269273537], [-73.79655875114958, 40.79513737467931], [-73.79654395497091, 40.79513221591909], [-73.79652623685229, 40.795125926060585], [-73.79649931607187, 40.795118025277645], [-73.79648271545763, 40.79511429549851], [-73.79646413823691, 40.79510954450978], [-73.79644420978643, 40.79510492565462], [-73.7964247547517, 40.79510091038862], [-73.79640621291949, 40.79509612310004], [-73.79638928364724, 40.79509145625978], [-73.79637599774739, 40.795088658401184], [-73.79635845980678, 40.79508508894538], [-73.79634586000752, 40.79508240152203], [-73.7963311434106, 40.79507861959891], [-73.79631349062726, 40.795073897693456], [-73.79629299992664, 40.795067458492454], [-73.796278677112, 40.795062714489944], [-73.79626532398866, 40.795058548487205], [-73.79624648828745, 40.79505349009978], [-73.79622483477702, 40.79504727566059], [-73.79620599982903, 40.79504175724301], [-73.79618698373636, 40.79503730245606], [-73.79616770775884, 40.79503241398564], [-73.79615443364219, 40.79502959032339], [-73.79613453860351, 40.79502609674189], [-73.79611590612106, 40.795024218827606], [-73.79589584128126, 40.7950061483473], [-73.79567528495986, 40.794991954928264], [-73.7954543553932, 40.794981646179366], [-73.79542480058724, 40.794980563948776], [-73.79534159459615, 40.794977887778714], [-73.79528852796409, 40.79498209047893], [-73.79523689150072, 40.79499229583411], [-73.79518785394659, 40.795008272855874], [-73.7951514290729, 40.795037425474455], [-73.79511057192093, 40.795062960548094], [-73.79510579688188, 40.79506478322655], [-73.79510069762432, 40.79506599695271], [-73.79509540758376, 40.795066569966465], [-73.79509006518835, 40.79506648727329], [-73.79508481023595, 40.79506575103706], [-73.7950797802359, 40.79506438052331], [-73.79507510681178, 40.795062411595254], [-73.79507091225572, 40.79505989577509], [-73.79506730632944, 40.795056898895616], [-73.7950643833914, 40.79505349937831], [-73.7950622199276, 40.795049786180414], [-73.79506087255106, 40.795045856467496], [-73.79507450159595, 40.79500518994561], [-73.79507043234777, 40.79497635861885], [-73.79506236850439, 40.79494801690581], [-73.7950085765647, 40.79492438234954], [-73.79495135663082, 40.79490597583461], [-73.79489158425001, 40.79489307900665], [-73.7948301740244, 40.79488588920489], [-73.79476806561526, 40.79488451644345], [-73.79470620936787, 40.79488898172742], [-73.79469490531206, 40.794900576391846], [-73.79468120795204, 40.7949105702425], [-73.79466550345413, 40.7949186815238], [-73.79464823457407, 40.794924681555166], [-73.79462988817102, 40.794928401178296], [-73.79461098148356, 40.79492973552613], [-73.79448191996578, 40.794834982265705], [-73.79447401015962, 40.79483527762806], [-73.79446614286849, 40.79483458867915], [-73.79445852875008, 40.794832933866566], [-73.79445137168297, 40.79483035750003], [-73.79444486330684, 40.79482692856544], [-73.7944391778926, 40.79482273887715], [-73.79443446767492, 40.794817900619734], [-73.79443085877568, 40.79481254334405], [-73.79442844782817, 40.794806810498535], [-73.79442729938788, 40.79480085558764], [-73.79442749979548, 40.79479820148108], [-73.79442829014225, 40.7947956117509], [-73.79442964748594, 40.79479316157155], [-73.79443153242586, 40.794790922066674], [-73.7944338902459, 40.794788958244276], [-73.79443665250339, 40.79478732711004], [-73.79443973901621, 40.79478607601239], [-73.79444306018883, 40.79478524126793], [-73.79444651961522, 40.79478484710754], [-73.7944500168751, 40.79478490497299], [-73.79445345045039, 40.79478541318445], [-73.79445672067227, 40.79478635698973], [-73.79445973261227, 40.79478770899212], [-73.79446239884044, 40.794789429945666], [-73.79446464196207, 40.79479146989504], [-73.7944882591118, 40.79479302537538], [-73.79451135093834, 40.79479709864146], [-73.79453345509013, 40.79480360813727], [-73.79455412899082, 40.794812423527695], [-73.79456285223858, 40.79480590492039], [-73.79457019411687, 40.794798467882195], [-73.79457598937672, 40.794790279803735], [-73.79458010758086, 40.79478152498025], [-73.79458245603871, 40.7947724004627], [-73.7945829818924, 40.79476311162303], [-73.79458167330685, 40.794753867531384], [-73.79455425618643, 40.79469705303749], [-73.79451948101098, 40.79464261424753], [-73.79447769833916, 40.79459109991582], [-73.7944418434089, 40.794541923583644], [-73.79441344570641, 40.79449000968993], [-73.79439286110842, 40.79443600884564], [-73.79438034757158, 40.79438059781493], [-73.79437606190031, 40.7943244710331], [-73.79438005778432, 40.79426833190424], [-73.79437546739086, 40.794222838290764], [-73.79438022516155, 40.79417735461067], [-73.79439421572687, 40.79413298360019], [-73.794417099873, 40.794090801017624], [-73.79444832276786, 40.794051829561404], [-73.7944871274128, 40.7940170140758], [-73.79453257299878, 40.79398719864455], [-73.79454360972017, 40.79397743767958], [-73.79455656132109, 40.79396953614117], [-73.79457064406509, 40.79396016993204], [-73.79458306861208, 40.793950123795454], [-73.79459407145329, 40.793940336966294], [-73.7946043709085, 40.79392790198053], [-73.79460907123847, 40.79391330448173], [-73.79460973137644, 40.793899032488916], [-73.79461057042656, 40.79388457134357], [-73.7946127132746, 40.793869698733545], [-73.79461459323522, 40.79385572643953], [-73.79461567270313, 40.79384393205767], [-73.79461643102601, 40.79383248723184], [-73.7946162863885, 40.79382178092444], [-73.79461611932695, 40.79380662526836], [-73.79461619706902, 40.79379307242944], [-73.7946156265293, 40.79377830303815], [-73.7946152702715, 40.79376688437034], [-73.79461462119819, 40.793754420994496], [-73.7946137126846, 40.79374195548111], [-73.79461196720068, 40.793723843705735], [-73.79461013968987, 40.793709099045955], [-73.79460739239804, 40.79369704471379], [-73.79460494180181, 40.79368499208291], [-73.79460179039852, 40.79367331580554], [-73.79459934002439, 40.79366126317492], [-73.794596452371, 40.7936483357845], [-73.79459215926552, 40.7936341899895], [-73.79458888336598, 40.793620928196354], [-73.79458564641244, 40.79360604650112], [-73.79458448640393, 40.79359429252216], [-73.7945843488073, 40.79358109177268], [-73.79458621884706, 40.793570334106974], [-73.79459035040817, 40.793556320258496], [-73.79459468221486, 40.79354263226862], [-73.7945995750269, 40.79352751142776], [-73.79460379310005, 40.793515605705764], [-73.79460747207493, 40.79350639048727], [-73.79461362299521, 40.793490516515334], [-73.79461841616039, 40.79347325351537], [-73.79462121486351, 40.793464424884256], [-73.79462883150332, 40.79344244872752], [-73.7946349442311, 40.79342744749395], [-73.79464018582509, 40.79341444462706], [-73.7946479798946, 40.79339604294102], [-73.7946555077745, 40.79337939745216], [-73.79466374411744, 40.793363464705564], [-73.79467244034359, 40.79334465016687], [-73.79467640385886, 40.79333472514483], [-73.79468109422703, 40.79332405375105], [-73.79468586834619, 40.79331267102567], [-73.79469042215402, 40.79329989141527], [-73.79469712351771, 40.79327863381512], [-73.79470050095907, 40.79326318694609], [-73.79470190954046, 40.793252095247915], [-73.79470464656971, 40.79324060016918], [-73.79470847677955, 40.79322800924298], [-73.79471264255977, 40.793214320449735], [-73.7947162688371, 40.79319852322497], [-73.7947201089581, 40.793182716496936], [-73.79472704424772, 40.79315843296352], [-73.79473214776363, 40.79314044210998], [-73.79473722550095, 40.793123161518295], [-73.7947417941934, 40.793105546811645], [-73.79474581791324, 40.79308759742345], [-73.7947488060191, 40.79306787293383], [-73.79475254579602, 40.793049932253616], [-73.79475661908518, 40.79303163215481], [-73.79476125302656, 40.79301580053716], [-73.79476574764517, 40.79299925719028], [-73.79477056371599, 40.79298197613085], [-73.79477479087163, 40.79296760343762], [-73.79477802260726, 40.79295321908797], [-73.79478009224506, 40.792935330381134], [-73.79478137864442, 40.79291781737958], [-73.79478041565106, 40.79290356879514], [-73.79477686526542, 40.79288281561495], [-73.79477366900427, 40.79286667303141], [-73.7947699898107, 40.79285323981521], [-73.7947660398668, 40.792842831291956], [-73.79476020271436, 40.79282624324601], [-73.79475609852271, 40.79281245847398], [-73.79475046940583, 40.792797103115255], [-73.7947449812176, 40.79278658597585], [-73.79473679247478, 40.792771667487834], [-73.79472946306416, 40.792758145849476], [-73.79472270985227, 40.792746769563045], [-73.79471718788803, 40.79273534219599], [-73.79471310199348, 40.792723348637395], [-73.79471053660359, 40.79271343007723], [-73.79470801660072, 40.79270101664061], [-73.79470613603486, 40.79268825354459], [-73.79470737521757, 40.79267447711513], [-73.79471124038173, 40.792662048749776], [-73.79471408530152, 40.79265787522851], [-73.79471954908433, 40.79264985228008], [-73.79473316431998, 40.79263766559541], [-73.794747854367, 40.792626904454785], [-73.79476883844725, 40.79261544136373], [-73.7947817100901, 40.79261035860794], [-73.7948014846438, 40.79260283908525], [-73.7948164560528, 40.79259683230773], [-73.79483110959207, 40.79259018535003], [-73.79484506634651, 40.79258297007646], [-73.79485899868224, 40.79257588107197], [-73.79487339339393, 40.792568387812054], [-73.79488652534009, 40.792559514405944], [-73.79489808029669, 40.79255094392606], [-73.79490752948263, 40.792541738964594], [-73.794914609934, 40.79253154004693], [-73.79492147125467, 40.79251991794492], [-73.7949274201559, 40.792507960843324], [-73.79493227027359, 40.79249534475149], [-73.79493585234243, 40.792482563401016], [-73.79493729075463, 40.792469139796836], [-73.79493816409332, 40.792455039389495], [-73.79493767315564, 40.79244131482739], [-73.7949383644732, 40.79242862884885], [-73.79493883421397, 40.79241452722071], [-73.79493821817327, 40.7924027742111], [-73.79493526379345, 40.79238896401616], [-73.7949318062081, 40.79237657639002], [-73.79492892588645, 40.792361695336005], [-73.79492773912166, 40.792346736045346], [-73.79492786300479, 40.7923412248444], [-73.79493082375255, 40.79231847462109], [-73.79493267385875, 40.79230664474915], [-73.79493520498374, 40.79229248480394], [-73.79493610870844, 40.79227996873893], [-73.7949366538639, 40.792268362201916], [-73.79493673317057, 40.79225427964681], [-73.79493778411938, 40.79223998086979], [-73.79493930906199, 40.79222565663438], [-73.79494075819505, 40.79221311574067], [-73.79494203241866, 40.79219953755274], [-73.79494587131747, 40.79218409149509], [-73.79494974481752, 40.792168646671534], [-73.79495314924843, 40.79215212885005], [-73.79495576121096, 40.79213920136549], [-73.79495775348728, 40.79212293248739], [-73.79495885389295, 40.7921082473165], [-73.7949589962517, 40.79209630685428], [-73.7949589280257, 40.79208417721621], [-73.79495950882445, 40.7920687253618], [-73.79495909296607, 40.79205359673375], [-73.79495990501024, 40.792024754809226], [-73.7949601902037, 40.79201281460101], [-73.79496019091042, 40.79200084758516], [-73.79496109819607, 40.791987458717806], [-73.79496327179088, 40.791974008946035], [-73.79496613596092, 40.79195984841915], [-73.79496993708129, 40.79194492497257], [-73.79497312335535, 40.79193002723616], [-73.79497472147854, 40.79191491106919], [-73.79497562812634, 40.79190151332162], [-73.7949762125005, 40.791888835360425], [-73.79497523156435, 40.79187618141886], [-73.79497299639056, 40.79186360599336], [-73.79497015548864, 40.7918510311655], [-73.79496498146193, 40.79183048969969], [-73.79495984336572, 40.79180994879995], [-73.794954878743, 40.79179084038406], [-73.79495077102447, 40.791774200640674], [-73.79494377022742, 40.79175069424006], [-73.79491173908899, 40.791717505844716], [-73.79488771963608, 40.791680616067744], [-73.79487243177032, 40.791641130574945], [-73.79486633368519, 40.79160023282998], [-73.79486960813617, 40.79155914862218], [-73.79485406236526, 40.7914653294894], [-73.79482960325622, 40.79137261454391], [-73.79479635971158, 40.7912814922681], [-73.7947816666935, 40.79124783703308], [-73.79476202145314, 40.79121761495717], [-73.79474673672595, 40.79119946970773], [-73.79472945263014, 40.79118239234308], [-73.79471981170987, 40.79116948039709], [-73.79471222949653, 40.79115794013066], [-73.79470454502517, 40.791144607828514], [-73.79469717838538, 40.791131996954334], [-73.79469592947264, 40.79112992411411], [-73.79468995269066, 40.79112007083999], [-73.79468144277753, 40.79110572805948], [-73.79467220456496, 40.79108925037568], [-73.79466435072615, 40.79107323451245], [-73.79465925836791, 40.791060925212804], [-73.79465265566238, 40.791046477773854], [-73.79464998480046, 40.79104062067829], [-73.79464703032944, 40.79103416802658], [-73.7946391638494, 40.79101853862036], [-73.79463460293387, 40.791010732180034], [-73.79462835064085, 40.79099016180311], [-73.79462505620809, 40.790979017829955], [-73.79461594545715, 40.790959316339496], [-73.7946079146054, 40.79093959918247], [-73.79460316376613, 40.7909279832066], [-73.7945964489682, 40.79091158188168], [-73.79458772975563, 40.79089186349624], [-73.79458361633066, 40.79088112146363], [-73.79457479024903, 40.790861394175344], [-73.79457078150446, 40.79085137419643], [-73.7945619296379, 40.79082879071814], [-73.79455741237943, 40.79081805835119], [-73.79454801070054, 40.79079693287163], [-73.79454128187366, 40.790777515053094], [-73.79453505363188, 40.790760952594184], [-73.7945301039987, 40.79074826526983], [-73.79452400304338, 40.79073631317264], [-73.79451840553797, 40.79072293080191], [-73.79451292468137, 40.79070989122861], [-73.79450545895085, 40.79069477632772], [-73.79449922169232, 40.79068106898835], [-73.79449149878917, 40.79066508031729], [-73.79448238428611, 40.790646621010275], [-73.79447563763632, 40.79063327344379], [-73.79446716657733, 40.79061766103934], [-73.7944618882688, 40.79060798908812], [-73.79445599717465, 40.79059710935714], [-73.79444910197826, 40.790585544498555], [-73.79444215932591, 40.79057364467115], [-73.79443444762006, 40.790558178528045], [-73.79442526177719, 40.79053991844435], [-73.79441852002448, 40.79052477802459], [-73.7944129564774, 40.790511403917534], [-73.7944083356295, 40.79050011266166], [-73.79440224434303, 40.790484972730994], [-73.79440178712217, 40.79048356771623], [-73.79439784323337, 40.790471548093734], [-73.79439337271718, 40.79045740196747], [-73.7943889673392, 40.790445075117205], [-73.79438420233413, 40.790434144285506], [-73.79437655855733, 40.79041938856768], [-73.79436821926048, 40.79040323746078], [-73.79436196045286, 40.79039259981313], [-73.79436018129554, 40.79038957950344], [-73.79434792535335, 40.79036800982879], [-73.79434137053015, 40.790353589929964], [-73.7943353520762, 40.7903380720202], [-73.79432846068629, 40.79032150886808], [-73.79432154783133, 40.79030426066523], [-73.79431395015501, 40.79028557956637], [-73.79430570458467, 40.790266212801164], [-73.79429666637722, 40.79024649382216], [-73.79428754610858, 40.79022644098504], [-73.79428311094897, 40.79021605955127], [-73.79427418291424, 40.790195286863614], [-73.79426578020373, 40.79017699130829], [-73.79425926772764, 40.790160078196955], [-73.7942554627349, 40.79014934658578], [-73.79425313714486, 40.790142768228726], [-73.79424756861381, 40.79012332498791], [-73.79424428641408, 40.79011217985136], [-73.7942336926241, 40.790078095453424], [-73.7942275357841, 40.790061155500716], [-73.79422148169029, 40.79004598932417], [-73.79421505675202, 40.79003156964916], [-73.7942081993475, 40.790019291686036], [-73.79420166179669, 40.79000700608485], [-73.79419610627916, 40.78999505544519], [-73.79419083518341, 40.7899830790116], [-73.7941853185527, 40.7899700299757], [-73.79417832287687, 40.78995243081665], [-73.79417158807009, 40.789934795937555], [-73.79416446201712, 40.78991751986916], [-73.79415995022116, 40.789905346607235], [-73.79415434690665, 40.7898936753126], [-73.79414575519934, 40.789882349478745], [-73.79413668563573, 40.78986923914602], [-73.79412817672485, 40.789854519893105], [-73.79411902284379, 40.789837672247884], [-73.79411023037532, 40.78981885397413], [-73.7941050313679, 40.78980652686651], [-73.79409987294365, 40.789792766482265], [-73.79409421254188, 40.78978010366624], [-73.79408659735557, 40.78976391697505], [-73.79407939586397, 40.78974774090651], [-73.79407276315243, 40.789731889177986], [-73.79406534369772, 40.78971356788911], [-73.79405742236868, 40.789692392407375], [-73.79405029959906, 40.789674072820226], [-73.79404284597742, 40.78965540016756], [-73.79403473841359, 40.789637429618004], [-73.79402867744963, 40.78962408324827], [-73.79402261261383, 40.78961213168931], [-73.79401445433065, 40.78959915917305], [-73.79400519274277, 40.78957867401022], [-73.79399525153924, 40.789559709266534], [-73.7939862116625, 40.78954037673929], [-73.79397943437094, 40.789525317980015], [-73.79397375798852, 40.78951391659637], [-73.79396773256312, 40.78950070363707], [-73.79396070185861, 40.789487193029956], [-73.79395254962347, 40.78947227504536], [-73.79394377663391, 40.78945844620684], [-73.79393468415904, 40.78944465130666], [-73.79393368216904, 40.78944329038013], [-73.7939245140734, 40.78943085565157], [-73.7939159522165, 40.78941810705817], [-73.7939074378363, 40.78940533140986], [-73.79390031635003, 40.78939395440985], [-73.79389479238411, 40.78938323780425], [-73.79388945027503, 40.78937144952723], [-73.79388490062964, 40.78935999637827], [-73.79387650299029, 40.7893399178261], [-73.79386941356547, 40.78932231847865], [-73.7938618873089, 40.78930399765446], [-73.7938579485176, 40.78929430109732], [-73.79385277898099, 40.7892839453034], [-73.79384121752716, 40.789260016879766], [-73.79383529787947, 40.78924718287743], [-73.79382453606576, 40.78922575904965], [-73.79381717578386, 40.789211003966116], [-73.7938116590685, 40.789197955413016], [-73.79380595365934, 40.78918477099362], [-73.79380123117537, 40.78917205606402], [-73.79379751466355, 40.78915934343663], [-73.7937928927188, 40.789144485029084], [-73.79379175293722, 40.78914128643407], [-73.79378864029793, 40.78913251039784], [-73.79378268703296, 40.789118937377], [-73.79377783352228, 40.789109815635705], [-73.79377152013572, 40.78909822430016], [-73.79376412048383, 40.78908473111283], [-73.79375837416896, 40.78907330211837], [-73.79375126484103, 40.78906193333896], [-73.79374011158927, 40.78904799200269], [-73.79372761531054, 40.78903534440716], [-73.79372207428247, 40.78903833453836], [-73.79370362194146, 40.78904368561593], [-73.79368586744265, 40.78904931787508], [-73.79366598465121, 40.789057639462705], [-73.7936603775312, 40.7890584127739], [-73.7936446964654, 40.78906056286633], [-73.79338158574151, 40.78949236880248], [-73.7934087886445, 40.78954346194088], [-73.79338628565984, 40.78958123095257], [-73.79322229668836, 40.78952422173967], [-73.79324728647568, 40.7894864572206], [-73.79331449772687, 40.78946956370559], [-73.79362007276016, 40.788996637749804], [-73.79361070599788, 40.78899514373587], [-73.79359204027723, 40.788992752050866], [-73.79357580273829, 40.78898734657069], [-73.79355631645792, 40.7889820977689], [-73.79353856440639, 40.788979265779176], [-73.7935227863599, 40.788978434898496], [-73.79350346628591, 40.788976950518006], [-73.7934855569334, 40.78897565863618], [-73.79346997126022, 40.78897335202481], [-73.79345335941076, 40.788970026356914], [-73.79343369196052, 40.78896619950193], [-73.7934185672628, 40.78896436311668], [-73.7934008839739, 40.78896260222009], [-73.79338830648537, 40.78896094981673], [-73.7933843509896, 40.7889604289247], [-73.7933596983348, 40.788955279551644], [-73.79334490202703, 40.78895111061649], [-73.79332492097404, 40.788945283930204], [-73.79330846591378, 40.788940590005936], [-73.79329508130505, 40.78893587411213], [-73.79328112097696, 40.788929311722214], [-73.7932628102149, 40.78891958936511], [-73.79324618916168, 40.78891135665863], [-73.79323111189646, 40.78890603446109], [-73.79321728564682, 40.78890193861573], [-73.7932005858231, 40.78889629017865], [-73.79318268826287, 40.788890927231314], [-73.79316692371502, 40.788885333740765], [-73.79315207264622, 40.78887978628183], [-73.79313432289509, 40.78887239753758], [-73.79311876623373, 40.78886849776241], [-73.79310747838457, 40.7888639476076], [-73.79308764837894, 40.78885485322936], [-73.79307470450007, 40.788849399815604], [-73.79306290763347, 40.78884438101108], [-73.79306030443946, 40.788843277369125], [-73.793045915972, 40.788837326152695], [-73.79302867353495, 40.78883082149809], [-73.79301554754561, 40.78882659186026], [-73.79299837220297, 40.788821265864115], [-73.79292049233945, 40.78880336858098], [-73.79286156729569, 40.78878964771914], [-73.79280669580459, 40.788783046246145], [-73.79275907929781, 40.78877919650147], [-73.79268790112779, 40.78877761872353], [-73.79263216028738, 40.78878412883775], [-73.79258763290983, 40.78879612395846], [-73.79253973216706, 40.7888228011319], [-73.79251505089977, 40.78883450862251], [-73.79249623443704, 40.788842957070024], [-73.79246425194015, 40.78885974786806], [-73.7923529115416, 40.78890684180988], [-73.79223970763987, 40.788996795887314], [-73.79220477113718, 40.7890147429105], [-73.79215684849902, 40.78903675479969], [-73.79210253101192, 40.78906127723173], [-73.79205591912397, 40.78909217901132], [-73.79200172966996, 40.789159438746445], [-73.79196317856, 40.78920767232353], [-73.79194212953739, 40.78923291109136], [-73.79191510216272, 40.78926012139201], [-73.79186140055127, 40.78929991645493], [-73.79179280401884, 40.789351616549254], [-73.79156371699993, 40.789103095999884], [-73.79133754899989, 40.78885774099988], [-73.79084786600005, 40.78840689799993], [-73.790653771, 40.788249452999864], [-73.79058298999988, 40.78819290999989], [-73.7906355999999, 40.78816951899994], [-73.79153624300002, 40.78775191299995], [-73.7933258969999, 40.78693204899985], [-73.79392031799995, 40.78667573999992], [-73.79454742299993, 40.78646433199993], [-73.79491977600001, 40.78633880799987], [-73.79582463099997, 40.7861305879999], [-73.79685935799988, 40.78599452199992], [-73.7978420549998, 40.78589425899987], [-73.79880021900001, 40.7859175479999], [-73.7992594699999, 40.785934836999935], [-73.79973844499978, 40.78595286999994], [-73.80067959399986, 40.786001101999894], [-73.80157320499991, 40.78603473299989], [-73.80215169199994, 40.78605649899994], [-73.80242791599977, 40.786068273999945], [-73.80242243499985, 40.78601897999991], [-73.80241070199989, 40.78581721099992], [-73.80254724699995, 40.78467334699992], [-73.80274225299986, 40.783253667999894], [-73.8028401069999, 40.78255003699992], [-73.80293830099997, 40.78183503399991], [-73.80299781999996, 40.78141047599991], [-73.80314563299987, 40.78132556799988], [-73.80324974699994, 40.781275479999884], [-73.8030331849998, 40.78094086999989], [-73.803066233, 40.78077013599987], [-73.80313481699993, 40.780415805999894], [-73.80320064599984, 40.77988363099988], [-73.8032260339999, 40.779702924999846], [-73.80332690999992, 40.778995479999935], [-73.80342253199996, 40.77833770799983], [-73.80346278599977, 40.778008173999865], [-73.8035964729998, 40.77703047199987], [-73.80369287499998, 40.77631968799989], [-73.8037902289999, 40.77561011199992], [-73.80659228499994, 40.77583186199997], [-73.8094525209999, 40.77605807899988], [-73.812686263, 40.77631279399991], [-73.81485850699985, 40.776485922999875], [-73.81478821799982, 40.77699573899987], [-73.81475882999987, 40.77720350099991], [-73.81466305899991, 40.77790468699987], [-73.81452997799975, 40.77888817499992], [-73.81728146999997, 40.7791055129999], [-73.81930985099999, 40.77926005999988], [-73.8198104219999, 40.77929698899986], [-73.82032141599986, 40.77933468099991], [-73.82132278499986, 40.779409258999856], [-73.82223999299987, 40.77949023699989], [-73.82331286899986, 40.77957865999986], [-73.82291816299993, 40.78053566699987], [-73.82277596999982, 40.780833776999906], [-73.8226154139999, 40.781126823999934], [-73.82243697299995, 40.781413836999896], [-73.82224127699999, 40.781693878999924], [-73.82384132799996, 40.781716885999835], [-73.82462418799997, 40.78170582499991], [-73.82498993100002, 40.7817014639999], [-73.82513740299993, 40.78169970599994], [-73.82528484199993, 40.78169729199987], [-73.82569679299998, 40.78169053099986], [-73.82678475899996, 40.78166766899991], [-73.82770726099993, 40.781667916999865], [-73.83218249399988, 40.78165887099986], [-73.83669385900005, 40.78164957499991], [-73.83777640299981, 40.78165237399991], [-73.83869921299997, 40.781676956999874], [-73.83862028899992, 40.7832271429999], [-73.83853062500003, 40.78470030299992], [-73.83845605499975, 40.78582446799988], [-73.83834963799995, 40.78585030599993], [-73.83824429099994, 40.78778591599995], [-73.83916328899986, 40.78782010599988], [-73.84008157499997, 40.78785153099992], [-73.83999984499997, 40.789251996999916], [-73.83992306199995, 40.790649901999906], [-73.83900524099994, 40.79061732199994], [-73.83895676099984, 40.79143843099996], [-73.83987372999988, 40.7914705419999], [-73.84081532599994, 40.7915048849999], [-73.84076080099992, 40.79250035099991], [-73.84067828300006, 40.79390789199993], [-73.84050805399978, 40.79385179899986], [-73.84050794799997, 40.793851761999875], [-73.84034364899992, 40.79378386199987], [-73.84018738599987, 40.793704751999925], [-73.84018731600003, 40.7937047159999], [-73.84018725699998, 40.79370467999992], [-73.84004094899993, 40.793615254999885], [-73.83990622699986, 40.793516588999935], [-73.83978455899992, 40.79341008099993], [-73.83966987799987, 40.793300689999946], [-73.83966981899991, 40.793300636999966], [-73.83953626099989, 40.79320132299989], [-73.83953619099991, 40.79320127799991], [-73.83938616999993, 40.79311469299992], [-73.83922303099997, 40.79304304599991], [-73.83905088399997, 40.79298788499988], [-73.83905076599997, 40.79298784899991], [-73.83905064799991, 40.79298782099989], [-73.83887362400004, 40.792949805999854], [-73.83818069199988, 40.792748501999945], [-73.83663288911686, 40.79251637576927], [-73.8366080165945, 40.792319381924464], [-73.83654148605072, 40.7920734960935], [-73.83653932567061, 40.79186763839274], [-73.83653187324715, 40.79174343240566], [-73.83660316329728, 40.791661144192254], [-73.83671513353327, 40.791582950598716], [-73.83682042117202, 40.79153361707885], [-73.83690586384843, 40.791493503452216], [-73.83694476868976, 40.79147148753523], [-73.83706020618943, 40.791438386284135], [-73.83722288087088, 40.79143034893579], [-73.83730267089945, 40.791410155512565], [-73.83742329381235, 40.79139283791763], [-73.8374525315691, 40.79137045689218], [-73.83750910549357, 40.79133235462143], [-73.83751359396337, 40.79132874122885], [-73.83751751246105, 40.79132483989425], [-73.83752089932669, 40.79132065619988], [-73.83752373147357, 40.79131616649231], [-73.8375278430181, 40.791306230831026], [-73.83752990905938, 40.79129484201922], [-73.83752906416629, 40.79127199589806], [-73.83752091447616, 40.79122738245857], [-73.8375188131358, 40.79120979285177], [-73.8375191987064, 40.7911926113245], [-73.83752056797638, 40.79118502184319], [-73.83752274044276, 40.791177983144856], [-73.8375336338759, 40.79115226208252], [-73.83754577477379, 40.79112938869815], [-73.83755939058666, 40.79110892171534], [-73.83757458989415, 40.79109071654427], [-73.83758076461456, 40.79108509790157], [-73.8375887630135, 40.79107951650588], [-73.83761861408517, 40.791063159843596], [-73.83762879578263, 40.79105675428099], [-73.83763726122933, 40.79104955356977], [-73.83764043314315, 40.79104582959373], [-73.83764293158194, 40.79104196026183], [-73.83764565993727, 40.79103555307709], [-73.83764705075262, 40.79102861194759], [-73.83764711720666, 40.79102115615742], [-73.8376458451299, 40.79101321165314], [-73.837639582358, 40.7909966432838], [-73.83763931193997, 40.79099590411601], [-73.83762759160071, 40.79097713933916], [-73.83762000783256, 40.790967898677394], [-73.83761057596885, 40.79095833091476], [-73.8375624755526, 40.790917875448685], [-73.8375519436678, 40.79090773034616], [-73.83754353102601, 40.79089808226201], [-73.83752623224662, 40.79087392450832], [-73.83751540542212, 40.790853404978215], [-73.83751003589187, 40.7908345591807], [-73.83750940657339, 40.7908257152994], [-73.83751015056319, 40.79081719785083], [-73.83751053377519, 40.79081552296997], [-73.83751113004466, 40.79081388474235], [-73.83751291473975, 40.790810655526556], [-73.83751555297322, 40.790807488493236], [-73.83751906737257, 40.790804324370335], [-73.83752688826179, 40.7907989409111], [-73.83754690552153, 40.79078730785384], [-73.83755477557729, 40.790781375481366], [-73.83755749560598, 40.790778470404646], [-73.83755948101535, 40.79077554737223], [-73.83756074492716, 40.79077255849073], [-73.83756110417634, 40.790771037866655], [-73.83756129773764, 40.79076948903237], [-73.8375614571039, 40.790733171432095], [-73.83755720396665, 40.79069907251719], [-73.83754750568994, 40.79065299891959], [-73.83753790568913, 40.790615524704336], [-73.83753119804766, 40.7905964788383], [-73.83751302747666, 40.790555839878934], [-73.83750614052263, 40.79053774781545], [-73.83750435831055, 40.790529667564236], [-73.83750394246857, 40.79052041890715], [-73.83750739724938, 40.79048893453551], [-73.83750762598892, 40.79047748818398], [-73.83750559139764, 40.79046651793164], [-73.83750358883579, 40.79046166072542], [-73.83738034396794, 40.790458135050606], [-73.83737820331633, 40.790487977178934], [-73.83702682792106, 40.79049690516986], [-73.83702688559563, 40.7904733444821], [-73.83681814957055, 40.790469906359775], [-73.83681410438504, 40.79043377227646], [-73.83701872622908, 40.79042934965848], [-73.83701257693829, 40.79040892012138], [-73.8373722220789, 40.79039843212046], [-73.83737212998825, 40.79043613206933], [-73.83748191403235, 40.790434497921844], [-73.83745089190637, 40.7904016491869], [-73.83741435117264, 40.79035859907438], [-73.83739429181088, 40.790329213627935], [-73.83739029705, 40.79032059115644], [-73.83738781967514, 40.79031141966715], [-73.83738681297147, 40.79030160310206], [-73.83738726435584, 40.790290968390174], [-73.83739143934415, 40.79027011050179], [-73.83740848436261, 40.79021360997666], [-73.8374107295372, 40.79019691806673], [-73.83741000655037, 40.790182121204836], [-73.83739734462176, 40.79010277772621], [-73.8373822323697, 40.79000383668329], [-73.83737790012118, 40.789967801555065], [-73.8373734632574, 40.78991215290862], [-73.83737352065216, 40.78990291362907], [-73.83737506283465, 40.78989247981079], [-73.83738634556121, 40.78985242315374], [-73.83738906354782, 40.78983617317225], [-73.83738894139299, 40.789827617645514], [-73.83738742040757, 40.78981989894494], [-73.83738447564578, 40.78981285235747], [-73.83738008413603, 40.789806407657814], [-73.83737555701477, 40.789801880597295], [-73.8373695962358, 40.789797469116465], [-73.83734725697254, 40.78978490233697], [-73.8373396629284, 40.789779939066264], [-73.83728739529073, 40.78973836974983], [-73.83727913342925, 40.78973059546133], [-73.83727167793585, 40.78972188684896], [-73.8372589211474, 40.78970129152464], [-73.83724908465298, 40.789677639465964], [-73.83723554145408, 40.78963476566826], [-73.83723208215262, 40.789619920545384], [-73.83723098502493, 40.78960337753787], [-73.83723210980034, 40.78958462985101], [-73.83723892974659, 40.78952835865341], [-73.83723862933164, 40.789510681618744], [-73.83723591790589, 40.789495026728865], [-73.83723436552562, 40.78949069333481], [-73.83723199401557, 40.78948638525277], [-73.83722878188219, 40.789482058728495], [-73.83722467003669, 40.789477630585644], [-73.83719323890342, 40.78945273262391], [-73.83718405411784, 40.7894440924258], [-73.8371806869702, 40.78943988328401], [-73.83717815048219, 40.78943571099845], [-73.83717640705441, 40.789431538158034], [-73.837175445348, 40.789427314321536], [-73.83717302340241, 40.789395504358794], [-73.83717427271856, 40.78936396192733], [-73.8371791828275, 40.78933318003952], [-73.83718766733921, 40.78930360218018], [-73.83722674841863, 40.7892038911416], [-73.83723934588721, 40.78916879711397], [-73.83724215608919, 40.78916027102371], [-73.83726072517644, 40.78910392862914], [-73.83727225675233, 40.789063664765806], [-73.8372804584517, 40.7890189130522], [-73.83728085145214, 40.78901296075893], [-73.83728076133985, 40.789011051512034], [-73.83728065697098, 40.78901037606971], [-73.83728049134774, 40.78900987242201], [-73.83728039786988, 40.78900970107914], [-73.83728035021717, 40.78900962847329], [-73.83728029129763, 40.7890095664057], [-73.83728024446076, 40.78900952077265], [-73.83728017293761, 40.789009475104834], [-73.83728010158885, 40.78900944819991], [-73.8372800429803, 40.789009439238356], [-73.83727997092765, 40.78900942975516], [-73.8372799001519, 40.78900943903671], [-73.83727981701145, 40.78900945701214], [-73.8372797461712, 40.789009492594936], [-73.83727966298792, 40.78900952816041], [-73.83727958041864, 40.7890095829922], [-73.83727940241451, 40.78900972597511], [-73.8372790108047, 40.78901014926153], [-73.83727858238785, 40.78901075191562], [-73.83727758270638, 40.78901250600303], [-73.83726600112918, 40.789039234915805], [-73.83724431967146, 40.78910221194871], [-73.83718350813176, 40.78924554070898], [-73.83716311195374, 40.78930123503309], [-73.83714210587829, 40.78937385793882], [-73.83714035500098, 40.78938246359029], [-73.83713962153668, 40.78939144107801], [-73.8371412213885, 40.789410875448205], [-73.83714658267745, 40.78943218991013], [-73.83716208040494, 40.789476066105365], [-73.83716755539365, 40.78949487638945], [-73.83717067065936, 40.78951465548935], [-73.8371700220552, 40.7895323762478], [-73.83716883843061, 40.78953705707807], [-73.83716668296843, 40.78954156582692], [-73.83716356735178, 40.78954590200822], [-73.83715950535763, 40.78955001990688], [-73.83715450863986, 40.789553931266134], [-73.83714862250145, 40.789557587735], [-73.83713432989514, 40.789564033014294], [-73.83711662821919, 40.78956925757452], [-73.83709699937467, 40.78957270531407], [-73.83707698362737, 40.78957410969868], [-73.83706743412114, 40.7895739968566], [-73.83705851436486, 40.789573335923095], [-73.83702441777633, 40.78956769554796], [-73.8369875229482, 40.78955784581486], [-73.83695332278464, 40.789546172347386], [-73.83687511481787, 40.78951671340881], [-73.83683725484383, 40.78950452193139], [-73.83680302130759, 40.78949641381511], [-73.83677086474431, 40.789492090348666], [-73.83670469134006, 40.789487836117225], [-73.83655947459667, 40.7894809666191], [-73.83650023157024, 40.78947697522877], [-73.83647469417723, 40.78947389506173], [-73.83640120618942, 40.78946187636406], [-73.83637809572132, 40.78946017868962], [-73.8363573014982, 40.7894606793846], [-73.83632428085821, 40.789464909921705], [-73.83628837475382, 40.78947263093072], [-73.8362546207189, 40.78948199355209], [-73.83617393310658, 40.7895067775186], [-73.83613203529262, 40.78951739770043], [-73.83601155152638, 40.78954192678144], [-73.83589707105418, 40.78956265576187], [-73.83584206544015, 40.78957040254113], [-73.83578722039788, 40.78957506186206], [-73.83573311865497, 40.78957656955579], [-73.83568022069021, 40.789574917568295], [-73.83565808161073, 40.789572670981116], [-73.8356361852034, 40.78956873138879], [-73.83561551147777, 40.78956326335915], [-73.83559665355094, 40.789556455869906], [-73.83558026327755, 40.78954854354983], [-73.8355663876015, 40.789539617768334], [-73.83555521539449, 40.78952977780324], [-73.83555067957171, 40.78952454805663], [-73.83554685397552, 40.78951912231342], [-73.83553285102528, 40.789494445774736], [-73.83552040718325, 40.78946732324285], [-73.83550928485678, 40.78943730189322], [-73.83549889589764, 40.78940280715278], [-73.83549690623326, 40.78939306991383], [-73.83549597285105, 40.78938206484244], [-73.83549634275938, 40.789333159676424], [-73.83549451291069, 40.789321098417766], [-73.83549110346662, 40.78931041363859], [-73.83548346034148, 40.78929640078706], [-73.8354724855882, 40.78928303032971], [-73.83545814552562, 40.789270285800015], [-73.83544041606656, 40.789258103502654], [-73.83541948454024, 40.78924658304619], [-73.83539490470189, 40.7892354180575], [-73.83536640185693, 40.78922445385129], [-73.83533150242093, 40.78921274271692], [-73.83528735229477, 40.7892001718458], [-73.83523905080068, 40.78918932422048], [-73.83518580449777, 40.789179998843466], [-73.83515382566965, 40.78917565758925], [-73.83512500940951, 40.789171744689746], [-73.83497802698217, 40.78915567587135], [-73.83483050344489, 40.789142740993135], [-73.83478381869124, 40.78913931502711], [-73.83477833485269, 40.789138920182104], [-73.83457184016791, 40.789127476498], [-73.83440941982468, 40.789115851524414], [-73.83432302797362, 40.78910966696789], [-73.83428255364467, 40.78910581717561], [-73.83424292462092, 40.78910010503191], [-73.83420808568339, 40.78909261712729], [-73.83416564274562, 40.789079957147834], [-73.83414953337565, 40.78907336093968], [-73.83413566521263, 40.78906613655134], [-73.83410403110443, 40.78904725278699], [-73.83409441601134, 40.78904023387084], [-73.83408663721427, 40.789033297171855], [-73.83407004851237, 40.78901474078978], [-73.83405560943376, 40.78899463018405], [-73.83403866408011, 40.788966716486065], [-73.83402388778488, 40.78893935656515], [-73.83401535853562, 40.788919407080456], [-73.83401305866653, 40.7889107676647], [-73.8340119530433, 40.788902643434355], [-73.83401331908667, 40.78883504520462], [-73.83401007226306, 40.788773472753945], [-73.83400506304766, 40.78873885059656], [-73.83400091286155, 40.78872585871744], [-73.83399530269573, 40.78871444131709], [-73.8339602331381, 40.788661909770376], [-73.83391732792282, 40.788601604797655], [-73.83390129515621, 40.78858379635434], [-73.83387417482537, 40.788560289799754], [-73.83385767920349, 40.78854725432054], [-73.83384261055957, 40.78853703917674], [-73.83382786803368, 40.78852889461295], [-73.83381316764698, 40.78852267799082], [-73.83379548109895, 40.78851716904366], [-73.83377597877146, 40.78851263800127], [-73.83375444807191, 40.78850904200486], [-73.83373021417556, 40.78850626013133], [-73.83365829867995, 40.788501103791305], [-73.83362430603371, 40.78849680477201], [-73.83359974634502, 40.78849165426107], [-73.83358017726817, 40.788485529922234], [-73.83353782460624, 40.78846958332585], [-73.83351863585291, 40.78846324325045], [-73.83350311185124, 40.788459484169934], [-73.83348846083359, 40.78845751753059], [-73.83347264458874, 40.78845709831348], [-73.83345664820672, 40.78845820113765], [-73.8334409313363, 40.78846079031423], [-73.83342600487724, 40.78846478482934], [-73.83340396992567, 40.788473072796116], [-73.83335417722044, 40.78849500980026], [-73.83330930201302, 40.788512414123275], [-73.83327559007826, 40.788523738997405], [-73.83326688772705, 40.78852602251193], [-73.83325759535903, 40.7885277657402], [-73.83323709224194, 40.788529716728426], [-73.83321371725937, 40.788529682903736], [-73.83318342623784, 40.78852767600099], [-73.83316390709939, 40.78852520706892], [-73.83311955649597, 40.78851742579979], [-73.8330984850652, 40.788515279105695], [-73.8330617942644, 40.78851436136726], [-73.83301120153718, 40.78851485514788], [-73.83296901193475, 40.78851487576383], [-73.83294877370874, 40.78851570114621], [-73.8329293717141, 40.78851811420314], [-73.83292072972804, 40.78851996539578], [-73.8329126686747, 40.78852228616637], [-73.83290227987415, 40.7885262597178], [-73.83289140098701, 40.78853159286456], [-73.83285624257194, 40.78855285698122], [-73.83284200543767, 40.78856039206553], [-73.83280758193703, 40.78857714543798], [-73.83279684900029, 40.788581316663404], [-73.83278675905991, 40.788584238913394], [-73.83277829537045, 40.78858582901446], [-73.83276866011296, 40.78858683224849], [-73.83272615785675, 40.788588544161385], [-73.8326597673408, 40.78859588658422], [-73.8326010642634, 40.78860389666712], [-73.83256885862595, 40.788609549603336], [-73.83253872405467, 40.788616403849254], [-73.83251063648018, 40.788624476626765], [-73.832484593941, 40.78863375872054], [-73.83240767132301, 40.78866474145495], [-73.83232539607748, 40.78869879597107], [-73.83228544136185, 40.788714388900296], [-73.83224825358656, 40.78872723004952], [-73.83223198697084, 40.788731537574776], [-73.83219614055196, 40.78873880725586], [-73.8321831471977, 40.788742281242556], [-73.83216100632328, 40.7887500831612], [-73.83214084292744, 40.7887590504147], [-73.83211813285888, 40.78877203892129], [-73.8320967198753, 40.78878766700121], [-73.83208628075752, 40.788797485033406], [-73.83206704304985, 40.78881948358381], [-73.83205805125756, 40.78882856543776], [-73.83198201145302, 40.78889385838392], [-73.83195418739795, 40.78891953598802], [-73.83193642535558, 40.78893888842594], [-73.8319189243068, 40.788962942507844], [-73.83190176756774, 40.78899169718287], [-73.83186571165793, 40.789058372985224], [-73.83183792154615, 40.78910774205707], [-73.83181680664136, 40.789145695125974], [-73.83180112232989, 40.78917762151521], [-73.83179240131156, 40.78919212520118], [-73.83178006235028, 40.78920838781622], [-73.83177319645581, 40.7892155190687], [-73.8317646012586, 40.78922226952416], [-73.83175433638331, 40.78922859437237], [-73.83174255613113, 40.78923440253686], [-73.8317185418299, 40.78924347181292], [-73.83168931605502, 40.789251651284594], [-73.83166578492866, 40.78925643444713], [-73.83164202240388, 40.789259370633935], [-73.83162642848136, 40.7892601312027], [-73.83161005472476, 40.789259847114046], [-73.83157493485916, 40.78925614860075], [-73.83156328224034, 40.78925375405066], [-73.83155137266962, 40.78925020638137], [-73.83152669521067, 40.78923963547459], [-73.8314980541266, 40.78922301428251], [-73.8314733739143, 40.78920448255158], [-73.83146520678658, 40.78919667163011], [-73.83145804844237, 40.789188188228366], [-73.83145205188639, 40.78917919205464], [-73.83144734754092, 40.78916988215248], [-73.83144590068468, 40.789170510769765], [-73.8314445011178, 40.78917126577036], [-73.83144187749328, 40.78917318814078], [-73.83143957313953, 40.78917558976566], [-73.83143767101389, 40.78917835148847], [-73.83143628889388, 40.78918135654858], [-73.83143548738316, 40.7891844349964], [-73.83143531293646, 40.78918743361403], [-73.83143576810112, 40.78919021687709], [-73.83143628604451, 40.78919162300547], [-73.8314369940182, 40.78919292939909], [-73.83144863809258, 40.78920818303407], [-73.83146277309501, 40.789221693522826], [-73.83147868152108, 40.789232784687734], [-73.8314871408735, 40.789237307176215], [-73.83149585179967, 40.78924110213385], [-73.83157593804363, 40.78927162924895], [-73.83159563330902, 40.78927964580881], [-73.83166863318911, 40.78930602010325], [-73.83167776142388, 40.789308752372754], [-73.83168820514707, 40.78931111817424], [-73.83171973215707, 40.78931608057961], [-73.83180053881553, 40.78932457330035], [-73.83183317302232, 40.78933214097065], [-73.83186317859902, 40.78934326906807], [-73.8318836035023, 40.789353465981264], [-73.8318915404827, 40.78935834886287], [-73.83189837711858, 40.78936333015152], [-73.83191627640754, 40.789378629047505], [-73.83193340194333, 40.78939527723496], [-73.83195118463726, 40.78941477213221], [-73.83196548578324, 40.78943290164429], [-73.83197646026684, 40.789450476821436], [-73.83198460174005, 40.78946875164193], [-73.83199053527478, 40.789488435649915], [-73.83199270968751, 40.78950485564092], [-73.83199234604422, 40.78951243714399], [-73.83198967695232, 40.789532901844204], [-73.8319896071385, 40.789536809272875], [-73.83198921420797, 40.78956114947362], [-73.83199196951847, 40.78958181088227], [-73.83199969267065, 40.789606108802936], [-73.83201458258128, 40.78964058344902], [-73.83203429590176, 40.789683269177246], [-73.83204000858483, 40.789696848066406], [-73.8320698642152, 40.789767851819555], [-73.83209381087113, 40.789819134240446], [-73.83210275748144, 40.78984209155442], [-73.83211841977344, 40.789894163342545], [-73.83213625756267, 40.78996181748948], [-73.8321411593998, 40.78998194191761], [-73.8321487200091, 40.79003324485319], [-73.83214914505982, 40.79004344056603], [-73.83214820430493, 40.79005445985478], [-73.83214207124183, 40.790080916754], [-73.83213206576016, 40.79011125895777], [-73.83212057206087, 40.79014000431848], [-73.83209261415041, 40.79019969343439], [-73.83207110810395, 40.79024250739354], [-73.83205476749029, 40.790271191909326], [-73.83205224652362, 40.79027496174553], [-73.83201384400635, 40.79033225810057], [-73.8319663976777, 40.79040000589913], [-73.83195529683569, 40.79041409216989], [-73.83191507342448, 40.79046023785868], [-73.83189574731281, 40.79048918804797], [-73.83187554905086, 40.79052553019267], [-73.83185670648172, 40.79056470079969], [-73.8318419080428, 40.790602086122554], [-73.83183057998171, 40.79063978291101], [-73.83182483351099, 40.790672984592376], [-73.83181918328192, 40.790752113164835], [-73.83181353814663, 40.790819534901765], [-73.83180901533764, 40.79085608870737], [-73.83180861574978, 40.79085930276894], [-73.83180369612917, 40.790879124062], [-73.83179395262198, 40.790903990880544], [-73.8317704456496, 40.79095586129706], [-73.83175692999276, 40.79098266740849], [-73.83175371702879, 40.79098820161012], [-73.83174331571016, 40.791006106281586], [-73.83172443576608, 40.79103198568103], [-73.83169117649335, 40.791069847943795], [-73.83166080927866, 40.79110259115967], [-73.83163531132125, 40.79112603075498], [-73.83155495695918, 40.791186380962436], [-73.8315088724828, 40.791221532964684], [-73.83148597421784, 40.79124309283912], [-73.83143773019351, 40.79129791700838], [-73.83138529261699, 40.791357238606835], [-73.8313735338278, 40.79136894466194], [-73.83135904834073, 40.79138067537314], [-73.83134155443767, 40.79139269803261], [-73.83131810478132, 40.791407037392815], [-73.8312710216949, 40.79143428901628], [-73.8312331137428, 40.79145464832485], [-73.83115486096933, 40.791494245102456], [-73.8310761015808, 40.791532968779734], [-73.83103001627167, 40.791554098379464], [-73.83099216040134, 40.79156786522635], [-73.8309626970729, 40.79157601668993], [-73.83088186308555, 40.79159581721124], [-73.83078035632461, 40.79162440401886], [-73.83070547729544, 40.79164199713384], [-73.83065972132633, 40.791650214548554], [-73.83061230109162, 40.79165585446681], [-73.83056779313932, 40.79165792332028], [-73.83052501206322, 40.791655978107556], [-73.83049016506872, 40.791651811937335], [-73.8304624528629, 40.791646305327006], [-73.83038005411207, 40.79162499531639], [-73.8302865399306, 40.79159977843827], [-73.83028200474374, 40.79159855636367], [-73.83019817362353, 40.79157169701324], [-73.83015912679089, 40.79156042717525], [-73.83012625490682, 40.79155278849131], [-73.8301107627126, 40.79155052316211], [-73.83009369185004, 40.791549552150094], [-73.83007470930407, 40.7915498573733], [-73.83005211074472, 40.79155144452654], [-73.83000804438622, 40.79155652156569], [-73.82997422391135, 40.791563432860954], [-73.82988506125005, 40.79158759716495], [-73.8297779457615, 40.79161724580165], [-73.82974037698273, 40.791630426993144], [-73.82968966519786, 40.79165343204063], [-73.82956542120624, 40.7917140964476], [-73.82948296063849, 40.79175392023617], [-73.82943290570364, 40.79177505253849], [-73.82929535710204, 40.7918254837798], [-73.8291706896936, 40.79187075641019], [-73.82914479588074, 40.791881704007466], [-73.8291183969597, 40.791895784914686], [-73.82909089929302, 40.7919132681345], [-73.82905874229748, 40.7919364079677], [-73.8289611930875, 40.792010285505796], [-73.82889011027119, 40.79206686589296], [-73.8288276159301, 40.79211952532807], [-73.82872574289628, 40.79219729372179], [-73.82863315730178, 40.79226665718168], [-73.82855854927205, 40.792320945232795], [-73.8285362694082, 40.792336382554964], [-73.8285183028931, 40.79234743193241], [-73.82850117052847, 40.79235623184518], [-73.82848447116561, 40.79236294369336], [-73.82846554640045, 40.79236853399551], [-73.82840984635601, 40.79238217476346], [-73.82836657648583, 40.79239541955735], [-73.828605761492, 40.79252530005947], [-73.82927850999312, 40.79289060783458], [-73.8293032551809, 40.792912635368644], [-73.82930732843701, 40.79293620434294], [-73.82922857533939, 40.79301934041085], [-73.82921203169039, 40.79302245750089], [-73.82919758430083, 40.793014582336745], [-73.82846130340072, 40.79260798825903], [-73.82829155299557, 40.79251424596058], [-73.82828845797178, 40.792524335422684], [-73.82828392815601, 40.7925348651814], [-73.82827013504226, 40.79255815896752], [-73.82826111144131, 40.79257020265734], [-73.82825526169445, 40.79257800095748], [-73.82823426229459, 40.792602346781884], [-73.82821200657723, 40.79262689762364], [-73.82820348065087, 40.79263416942086], [-73.82819476870971, 40.7926398930058], [-73.82817584627136, 40.79264923918226], [-73.828146465749, 40.7926611985705], [-73.82808189662184, 40.79268648840044], [-73.82805258813137, 40.79270263533602], [-73.82803976792373, 40.79271165621281], [-73.82802835772806, 40.792721211755], [-73.8280040387698, 40.79274607459317], [-73.82798633399265, 40.79277006460614], [-73.82795357705758, 40.792827926883305], [-73.82792932227444, 40.792878507455086], [-73.82789820942337, 40.79295580544549], [-73.82787507470991, 40.79301766246831], [-73.8278487557192, 40.79308767321416], [-73.82782132489064, 40.79316141073646], [-73.82780895581442, 40.79319734939527], [-73.82780227405307, 40.79322492179025], [-73.82779277842641, 40.79329538919932], [-73.82778571374463, 40.79336037663222], [-73.82778378411446, 40.79339207155309], [-73.82778457368786, 40.793420789925996], [-73.82779720735266, 40.79349792808428], [-73.82780452240922, 40.793573996183966], [-73.82781379322529, 40.79365915109741], [-73.82782028396574, 40.79370553699783], [-73.82782926370653, 40.79373849979562], [-73.82786390498843, 40.79383196097708], [-73.82788750813612, 40.79389588703248], [-73.82789752025388, 40.793928238222854], [-73.82790968658152, 40.79397972016701], [-73.82791705183374, 40.794013329538956], [-73.82792022633377, 40.79403709757745], [-73.82792031712073, 40.7940564678279], [-73.82791783454614, 40.79408216418048], [-73.82790638647182, 40.79415181014585], [-73.82789394577956, 40.79424771287994], [-73.82788720393613, 40.79432058051203], [-73.82788650586653, 40.79433889618451], [-73.82788722287647, 40.79434539840992], [-73.8278887132347, 40.79435119148287], [-73.82789413592968, 40.79436643500175], [-73.82789894642588, 40.794375988446866], [-73.82791556374134, 40.794401235700484], [-73.8279338422003, 40.7944259913994], [-73.82795977829309, 40.79446789385134], [-73.82797321529853, 40.79449508198826], [-73.8279776450364, 40.79450970367481], [-73.82797996235428, 40.794525096845376], [-73.82798037396266, 40.794540514646], [-73.82797916037191, 40.794555910755754], [-73.82797293455938, 40.79459007562093], [-73.82796885705426, 40.794632015693416], [-73.82796950868526, 40.79467337648526], [-73.82796724490338, 40.79471028378869], [-73.82796673849063, 40.794749978487914], [-73.82796997734857, 40.7947852009929], [-73.82797171977279, 40.79479437041037], [-73.82797409504168, 40.79480124517305], [-73.82797743419616, 40.79480686912257], [-73.82797948961695, 40.79480922223503], [-73.82798181867787, 40.794811288114936], [-73.82798375778684, 40.79481258765455], [-73.82798601853261, 40.79481370758455], [-73.82798858771062, 40.79481463967634], [-73.82799146422416, 40.79481538175048], [-73.82799813388105, 40.79481632934268], [-73.82800604847579, 40.79481653816271], [-73.82804679082173, 40.794814266496225], [-73.82805741824836, 40.79481492246426], [-73.82806165747047, 40.79481575653053], [-73.82806533892197, 40.79481695982811], [-73.82806630987092, 40.79481741141684], [-73.82806720933907, 40.79481797162295], [-73.82806802525225, 40.7948186211629], [-73.82806875780553, 40.794819369921065], [-73.82806904143246, 40.79481973923481], [-73.82806940658371, 40.794820208012766], [-73.82806997268547, 40.79482113661224], [-73.82807084415114, 40.794823235998926], [-73.82807133624974, 40.79482567673892], [-73.82807142397816, 40.79482832443962], [-73.82807108549217, 40.794831016568786], [-73.82807035586674, 40.79483357241907], [-73.82806792658884, 40.79483838749305], [-73.82806416917738, 40.7948433795029], [-73.82804996174586, 40.794857117664186], [-73.82804185283631, 40.79486360672115], [-73.82802255294415, 40.79487631085413], [-73.82801595078091, 40.79488165913233], [-73.82801298038208, 40.79488483315758], [-73.82801043644797, 40.79488837771489], [-73.82800837665492, 40.79489223760718], [-73.82800683694197, 40.794896324434575], [-73.8280054641173, 40.794904840377235], [-73.82800565470579, 40.79490905492801], [-73.82800645038532, 40.79491309967332], [-73.82801372355921, 40.794932408770045], [-73.82802487159839, 40.79495152429235], [-73.8280522646091, 40.79498880955108], [-73.82808388029115, 40.79502473375978], [-73.82809360846584, 40.79503363565327], [-73.8281070896511, 40.79504412039351], [-73.8281159857097, 40.79504977021587], [-73.82813785302835, 40.79506132085697], [-73.8281427741178, 40.795064442992185], [-73.82814690119959, 40.795067565451], [-73.82816011502915, 40.795080605552194], [-73.82817505476636, 40.7950996648955], [-73.82819916200815, 40.795134171838825], [-73.82820799674265, 40.79514958409877], [-73.82821393552645, 40.79516247011635], [-73.8282301195962, 40.795204619306816], [-73.82824392764014, 40.79523957979457], [-73.82826262833551, 40.79529818471794], [-73.82828572162478, 40.795362884050505], [-73.82831145744987, 40.795441041269136], [-73.82832651007615, 40.795481081458185], [-73.82834276079831, 40.79551530661652], [-73.82835875024011, 40.79554135497714], [-73.82838528126483, 40.79557781095901], [-73.82841279822466, 40.79561336828901], [-73.82843540215093, 40.79563804791929], [-73.82844701207814, 40.79564816094262], [-73.82845983360038, 40.795657049315366], [-73.82847351197225, 40.79566448048679], [-73.8284877636875, 40.79567029254025], [-73.82851832007442, 40.79568019820653], [-73.82852965393732, 40.79568281757275], [-73.82854017146603, 40.79568434613912], [-73.82855782430934, 40.7956854532701], [-73.82857242430367, 40.79568485413637], [-73.82857644262674, 40.79568426506282], [-73.8285808521636, 40.79568327199739], [-73.82859974191038, 40.79567786054278], [-73.82860453183888, 40.79567698497474], [-73.82860895260286, 40.795676596858755], [-73.82861011408166, 40.795676598585786], [-73.82861129852257, 40.79567667171268], [-73.82861369144146, 40.79567707113293], [-73.82861613028093, 40.795677786071565], [-73.82861855689164, 40.795678798684406], [-73.82862098271494, 40.7956801170295], [-73.828623300956, 40.795681695883964], [-73.82862545174703, 40.79568349964343], [-73.82862697694698, 40.79568504280856], [-73.82862735510265, 40.795685429851545], [-73.82863180743121, 40.79569136234261], [-73.82863534447927, 40.79569790443742], [-73.82863787466573, 40.79570492331978], [-73.82863931372879, 40.795712183491666], [-73.8286395783204, 40.79572894227562], [-73.82863657631914, 40.79574909998143], [-73.8286343147076, 40.79575729128531], [-73.82862557729028, 40.79578201490872], [-73.82862144922055, 40.79580725047355], [-73.82862189637106, 40.7958175713651], [-73.82862402869226, 40.7958268939663], [-73.82862725470012, 40.7958347240355], [-73.82863190221398, 40.79584310570086], [-73.82864862033784, 40.79586677808698], [-73.82866925976047, 40.79589502186539], [-73.82868433460277, 40.795912504592756], [-73.82869161684445, 40.79591907151723], [-73.8286951052269, 40.79592148873051], [-73.82869843051249, 40.79592329590934], [-73.82870155594443, 40.79592447088572], [-73.82870304740108, 40.79592482390004], [-73.82870449295862, 40.79592502423077], [-73.82870587897408, 40.79592506247586], [-73.82870721909322, 40.79592494703224], [-73.82870849922915, 40.79592466950245], [-73.82870973258431, 40.79592423928765], [-73.82871204621736, 40.79592307239428], [-73.82871437187114, 40.79592151736277], [-73.82872286299764, 40.79591446175447], [-73.82872444183317, 40.79591341070165], [-73.82872600720093, 40.79591253904838], [-73.82874087242854, 40.79590594254771], [-73.82875497212024, 40.79590146979826], [-73.82875611037336, 40.795901247005546], [-73.8287572831818, 40.79590113064255], [-73.82875850462355, 40.795901132457175], [-73.82875975973262, 40.79590124304575], [-73.82876105137812, 40.79590146123973], [-73.82876235352467, 40.79590179638194], [-73.82876504033172, 40.79590278190396], [-73.82876778660132, 40.79590421765475], [-73.82877045961781, 40.795906022689735], [-73.82877290726593, 40.79590811620353], [-73.82877495075456, 40.79591033308548], [-73.82878010953803, 40.795917923947414], [-73.82878406885135, 40.795926312793014], [-73.82878674776182, 40.795935340353225], [-73.82878808676787, 40.79594474302249], [-73.82878811739303, 40.795951091772906], [-73.82878736459838, 40.79595786236098], [-73.82878342625989, 40.79597337253346], [-73.82876332470153, 40.796021888143386], [-73.82875467221558, 40.796050105976256], [-73.82875327145067, 40.796060513579825], [-73.82875353193232, 40.79606996959524], [-73.82875464160588, 40.7960760591119], [-73.82875659178978, 40.79608250000494], [-73.82876324874735, 40.796097179411774], [-73.82878634962549, 40.796140968645645], [-73.82880615987096, 40.7961785674521], [-73.82880772835877, 40.7961811992581], [-73.82880962942951, 40.796183768401015], [-73.82881432169924, 40.79618856541965], [-73.82881996270321, 40.79619269039536], [-73.82882303973867, 40.79619443236953], [-73.82882623513585, 40.79619592256134], [-73.82883505366314, 40.79619940225701], [-73.82884211049873, 40.79620157448198], [-73.82884862349995, 40.796202808260055], [-73.82885169200794, 40.79620306594696], [-73.82885466653137, 40.79620309666498], [-73.82887840472594, 40.79620137641177], [-73.82892258328465, 40.796200018528324], [-73.828930900435, 40.79620019337192], [-73.82893972601295, 40.79620101729227], [-73.82895951707334, 40.79620468397004], [-73.82897991724741, 40.79621090965885], [-73.82900266573404, 40.796220750337355], [-73.82902261611012, 40.796231568391306], [-73.82904143606983, 40.79624419353537], [-73.82904883248631, 40.796247960421574], [-73.82906100849343, 40.79625429083096], [-73.82908611870918, 40.796267340256044], [-73.82913126948527, 40.796293620543096], [-73.82913875856414, 40.79630260027655], [-73.8291476483809, 40.796315248041054], [-73.82915588121533, 40.796330072994465], [-73.82916737925088, 40.79635630338244], [-73.82917114555754, 40.79636614336686], [-73.82917971956068, 40.79638615523828], [-73.82918736837664, 40.79640247582675], [-73.82919891573279, 40.79642323976793], [-73.82920448083085, 40.796434198469754], [-73.82920965518458, 40.796444777147656], [-73.82921580381061, 40.796454610776216], [-73.82922393581754, 40.7964672480303], [-73.82923352421639, 40.79647543880834], [-73.82924703839497, 40.7964869949526], [-73.82925200048693, 40.796497483352866], [-73.82925727015667, 40.79651267447484], [-73.8292606322496, 40.79652266647282], [-73.8292619633283, 40.79652608009616], [-73.82926604809887, 40.79653655898752], [-73.8292712558008, 40.79654823785014], [-73.82927444778716, 40.79656009381648], [-73.82927759304934, 40.79657211874665], [-73.82927990172637, 40.796586728524716], [-73.82928113630847, 40.79659966246471], [-73.82928333380296, 40.79661593761409], [-73.82928647195324, 40.7966307374214], [-73.82928856731736, 40.796645148198316], [-73.8292909774409, 40.79666189189533], [-73.82929449211723, 40.79667299129231], [-73.82930334030262, 40.79668359416922], [-73.82931538346615, 40.79669105079403], [-73.82933274065235, 40.796700883422915], [-73.82934864305284, 40.796710154528434], [-73.82937273079625, 40.796724867911976], [-73.82938545954309, 40.796733099511066], [-73.82939727945084, 40.79674011336269], [-73.82940234219689, 40.7967432547505], [-73.82940857757869, 40.79674712694351], [-73.82941556721865, 40.79675232487442], [-73.8294265150935, 40.796761652296354], [-73.82943556043782, 40.7967694815716], [-73.82944761505512, 40.79677729770874], [-73.82945737123772, 40.7967895498628], [-73.82946361428056, 40.796804030432405], [-73.82946637122622, 40.796814571011275], [-73.82945991534359, 40.79682336759143], [-73.82945583429438, 40.7968252708385], [-73.82945546638351, 40.79682544100242], [-73.82944790883194, 40.796828950865624], [-73.82943548347617, 40.79683186869486], [-73.82941825785157, 40.796830807898004], [-73.82939934401158, 40.79682685328194], [-73.82938338468514, 40.7968212013146], [-73.82936696838196, 40.796814261404684], [-73.82934623333284, 40.79680426178348], [-73.82932152634481, 40.79679075349179], [-73.8293003607649, 40.79677792623624], [-73.82929094280402, 40.796772537576366], [-73.8292707527818, 40.79675812579862], [-73.82926066599057, 40.79674973610358], [-73.82925144264567, 40.79674264433191], [-73.829246256631, 40.796736514272055], [-73.82923964136586, 40.796723905568705], [-73.82923739382846, 40.796712790457214], [-73.8292402495708, 40.796699151767086], [-73.82924388162897, 40.79668726084508], [-73.8292457107191, 40.79668129379261], [-73.82924388345312, 40.796672971279484], [-73.82923324157332, 40.796655142722315], [-73.82922415029546, 40.79664667324891], [-73.82921483560507, 40.79663798834138], [-73.82920390880294, 40.79662948867257], [-73.82918321758689, 40.79661652757519], [-73.82916539769322, 40.796606541622374], [-73.82914801503854, 40.796597375345904], [-73.82913307919159, 40.7965901306904], [-73.82912052352859, 40.79658388214869], [-73.82910266443135, 40.796575435349894], [-73.8290932565701, 40.79657042328531], [-73.82907780805219, 40.79656452878604], [-73.82905864191761, 40.796562114134225], [-73.82904158374792, 40.79656471881432], [-73.82902309341942, 40.7965672215223], [-73.8290026733165, 40.79656845269594], [-73.8289846857334, 40.79656866808387], [-73.82896378516148, 40.79656806179086], [-73.82892358442233, 40.7965662454626], [-73.82891120174484, 40.79656611953234], [-73.82888595450925, 40.79656500939099], [-73.8288582530758, 40.796564086076444], [-73.82883446085646, 40.79656344748984], [-73.82881553981146, 40.79656268060484], [-73.82879508820055, 40.79656270501046], [-73.82877370654465, 40.79656478942916], [-73.82874447314435, 40.79656974361264], [-73.82872716532219, 40.79657281560578], [-73.82870939560682, 40.79657540058397], [-73.82869454285871, 40.79657760224124], [-73.82868016371482, 40.79658002204834], [-73.82866675586334, 40.79658217157149], [-73.82863803621638, 40.796585614241266], [-73.82862365934415, 40.79658732139213], [-73.82860833140212, 40.79658957728008], [-73.82859349157125, 40.796591553957185], [-73.82857618436267, 40.79659438268179], [-73.82853317480047, 40.796602225231375], [-73.82851684884285, 40.79660579436069], [-73.8285005120372, 40.79660903847251], [-73.82848417580745, 40.79661205860174], [-73.82846831113262, 40.79661566409342], [-73.82845443980872, 40.7966188761848], [-73.82844200326554, 40.79662168533399], [-73.82842248760095, 40.79662599569234], [-73.8284024732032, 40.79663068474958], [-73.82838736715539, 40.79663450763465], [-73.82836830991194, 40.796640818415256], [-73.82835215055864, 40.79664867390282], [-73.8283310592047, 40.796662032953336], [-73.82832049419267, 40.7966696267174], [-73.82829961184392, 40.796684850294454], [-73.8282897295235, 40.796693813586366], [-73.8282809776311, 40.79670131958413], [-73.8282625192494, 40.796714142275626], [-73.82824143571027, 40.796724457387555], [-73.82821848353318, 40.796733644946386], [-73.82819115157852, 40.79674123129953], [-73.82816968869709, 40.796746827651916], [-73.8281445656651, 40.796752660617265], [-73.82812314716782, 40.796755321817294], [-73.82810125995778, 40.79675578438317], [-73.82808178967328, 40.79675641237909], [-73.82805654056614, 40.79675549202698], [-73.82803834286437, 40.796754592073334], [-73.82801769208149, 40.79675310161574], [-73.82799281466637, 40.79675114700088], [-73.82796201029873, 40.79674926428172], [-73.82793030431735, 40.79674798346688], [-73.82790028871491, 40.79674864896262], [-73.82787023505595, 40.796750306478145], [-73.82783938997113, 40.79675070167381], [-73.82781020270696, 40.796751396359866], [-73.82775695778432, 40.79675331555154], [-73.82772607638644, 40.79675436836667], [-73.82769859391973, 40.7967559489332], [-73.82767536606408, 40.79675714769786], [-73.82765520698014, 40.7967579992433], [-73.82763697115004, 40.7967578642571], [-73.82761877332662, 40.79675692805159], [-73.82759009008097, 40.796755542939636], [-73.82757380520768, 40.79675683331848], [-73.82756429760532, 40.79675827773812], [-73.82754735503714, 40.796762422094474], [-73.82752596323047, 40.79676370482573], [-73.82750500819391, 40.796765978448434], [-73.82748697197574, 40.79677073279408], [-73.82747038295471, 40.79677539984455], [-73.8274508283503, 40.796780998919445], [-73.82743330638438, 40.796784403605386], [-73.82741768831188, 40.796789018829], [-73.82739957276964, 40.796797581058534], [-73.82737218442077, 40.79681743232667], [-73.8273535426513, 40.79683203756921], [-73.82734790341695, 40.796836423979954], [-73.82733950746093, 40.79684374096838], [-73.82732794744197, 40.796855079685535], [-73.82731470177303, 40.79686843221022], [-73.8273053898948, 40.796876846789566], [-73.82729508449476, 40.79688432341333], [-73.82728527175662, 40.796894187506574], [-73.82727942289542, 40.796905956939895], [-73.82727725896208, 40.79691813814351], [-73.82728240098972, 40.79692739340957], [-73.8272889449065, 40.796939838446335], [-73.8272887365554, 40.79695166189837], [-73.82728748729234, 40.7969537944657], [-73.82728467788968, 40.79695859018938], [-73.82728314189592, 40.796961207982676], [-73.8272805093019, 40.79696242931756], [-73.8272790382898, 40.796963111621444], [-73.82727261992385, 40.79696608278045], [-73.82725849390543, 40.79696630368565], [-73.82724276836349, 40.79696219266931], [-73.82723696869755, 40.79695551044121], [-73.82722952070569, 40.796944360021755], [-73.82721950476653, 40.79693152212379], [-73.82721191019695, 40.79692223637446], [-73.8272074346915, 40.79692083367755], [-73.82720180343523, 40.79692213997383], [-73.82718839026421, 40.79692637852444], [-73.82717201225188, 40.79693151408134], [-73.82715779620057, 40.7969396693373], [-73.82714408833661, 40.79694805101017], [-73.82712741604782, 40.7969569133887], [-73.82711175491708, 40.79696501269606], [-73.82709508032535, 40.79697475842694], [-73.82707642035966, 40.79698741762403], [-73.82706244374643, 40.796999112868306], [-73.82704565175071, 40.79701314454931], [-73.82703059174072, 40.79702667373822], [-73.8270144299609, 40.79704427347519], [-73.82700062491989, 40.7970629740375], [-73.82698898205707, 40.79708392904759], [-73.82698435541114, 40.797094790805836], [-73.82697942175973, 40.7971056249638], [-73.82697613317659, 40.79711276129334], [-73.82697509656563, 40.79711500223019], [-73.82697091510649, 40.79712827768959], [-73.82697001638891, 40.79714082029546], [-73.82696958332555, 40.79715265329329], [-73.82697037969984, 40.79716557621308], [-73.82697204859538, 40.797175187463445], [-73.82697520517927, 40.797186863350305], [-73.82697978921613, 40.79720130420573], [-73.82698279311242, 40.79721278117811], [-73.82698554335774, 40.79722609451623], [-73.82699079751283, 40.79724699363738], [-73.8269934694796, 40.79725811110455], [-73.82699897185924, 40.797270527287274], [-73.8270121432208, 40.79728633413084], [-73.82703330269895, 40.79730140261423], [-73.82704654320895, 40.797308672471516], [-73.82705837493968, 40.79731540765079], [-73.82707441225828, 40.797327156635156], [-73.82708843152089, 40.79734056812576], [-73.82710156847041, 40.797355463737546], [-73.82711225531149, 40.79736997857354], [-73.82711969691147, 40.79738368592774], [-73.82712728725102, 40.79739928469705], [-73.82713648523875, 40.79741634435143], [-73.82714904596592, 40.797434554752705], [-73.827161276906, 40.797452005433705], [-73.82717235863593, 40.797469474158376], [-73.82718039545175, 40.79748615496871], [-73.8271866948234, 40.79749711369461], [-73.82719240016688, 40.79750898169425], [-73.8271961600784, 40.797521198077405], [-73.82719785571354, 40.79753912213346], [-73.82719705421918, 40.79755575233404], [-73.82719495057225, 40.797571615494704], [-73.82719101670102, 40.79758522504254], [-73.82718643262824, 40.797598113424364], [-73.82718213808052, 40.797609525672776], [-73.82717365601191, 40.79762220131633], [-73.82716326243056, 40.79763209149887], [-73.82714947039155, 40.797645244498916], [-73.82712912435537, 40.79766373758642], [-73.82711439259286, 40.79767381877065], [-73.82709623419143, 40.79768494017031], [-73.82707221507523, 40.79769834837304], [-73.82706744562513, 40.79770078122262], [-73.82706071893233, 40.79770420204715], [-73.82703818958812, 40.79771446099343], [-73.82703055912111, 40.79771845691439], [-73.82697931778351, 40.79773814647965], [-73.82691928311844, 40.79774952881238], [-73.82689978813907, 40.79775487475237], [-73.82688418235938, 40.797759408169355], [-73.8268684709334, 40.79776370705788], [-73.8268666088378, 40.797764217390764], [-73.82685100446659, 40.79776786861763], [-73.8268473410131, 40.797768844644736], [-73.82683076310852, 40.7977732777543], [-73.82681001426134, 40.797778018034585], [-73.82679270318214, 40.79778244065561], [-73.82677636279003, 40.797786919681116], [-73.82675807820914, 40.79779177187656], [-73.82673856166687, 40.79779624545296], [-73.82671545112177, 40.797798046956686], [-73.826694572381, 40.7977974665872], [-73.82667174584928, 40.79779474901365], [-73.82665286668373, 40.79779106522315], [-73.82663395331255, 40.79778722708737], [-73.82661605440353, 40.79778068963265], [-73.82657894654045, 40.797760490241515], [-73.82654264563084, 40.797735427790144], [-73.82650832844257, 40.79770845081927], [-73.82647621310204, 40.79768205528219], [-73.826457818813, 40.79766083765889], [-73.8264164207575, 40.79760489041546], [-73.82640779152851, 40.797592450901156], [-73.82638690818652, 40.79756678138774], [-73.82636659926142, 40.797543337805756], [-73.82635936342012, 40.797533115405315], [-73.82634395738273, 40.7975152529369], [-73.82633117140915, 40.79750183500532], [-73.8263144770076, 40.7974875630029], [-73.82630125088053, 40.79747952900073], [-73.82628606681082, 40.79747190600121], [-73.82626841794904, 40.797464882543224], [-73.82625030331073, 40.79745911984531], [-73.82623312581936, 40.79745283487339], [-73.82621357170845, 40.7974448713986], [-73.82619545967715, 40.797438198531324], [-73.82618023649592, 40.79743222340459], [-73.8261713276451, 40.797427023556196], [-73.82616700235472, 40.79742254142913], [-73.82614856928551, 40.79741598500308], [-73.82612773868075, 40.79741111746487], [-73.82610304625223, 40.797405785077935], [-73.82608027172392, 40.797400942407705], [-73.82605846825301, 40.79739660461154], [-73.82603619215142, 40.79739201531282], [-73.82601537230752, 40.79738775974029], [-73.82598606248501, 40.7973804941534], [-73.82596861990483, 40.79737637919871], [-73.82595263113721, 40.797373195364095], [-73.82593519870257, 40.7973690799172], [-73.82591335337773, 40.79736718270567], [-73.82589392212995, 40.79736663134858], [-73.82587402789434, 40.79736643930057], [-73.82585311222957, 40.79736646097577], [-73.82583024288262, 40.797366696473], [-73.82580789360196, 40.79736691583142], [-73.82578553239327, 40.79736756553998], [-73.82576219954422, 40.79736818780886], [-73.82573789275413, 40.797369493276406], [-73.82573406442111, 40.7973697932222], [-73.8257125643848, 40.79737148991091], [-73.82568727081356, 40.797373919142736], [-73.82566682367386, 40.79737676965396], [-73.82564879294486, 40.79737965129051], [-73.82563123724267, 40.797382290396335], [-73.82560666434136, 40.79738571043985], [-73.82558961698389, 40.79738843105546], [-73.82557158799422, 40.79739089872794], [-73.82555525141953, 40.79739392614762], [-73.82553768829824, 40.79739914092848], [-73.8255352506272, 40.79740234249946], [-73.82554212608748, 40.79741468175813], [-73.82553800097944, 40.79742921821309], [-73.82553181122282, 40.7974349985167], [-73.82552009620126, 40.79743803409917], [-73.82550843644336, 40.79743815147462], [-73.8254937244918, 40.79742731394092], [-73.82548346181532, 40.797418383380375], [-73.82547591149512, 40.79740582799391], [-73.82547457149775, 40.79739234604665], [-73.82546689282017, 40.79738385094201], [-73.82545531877895, 40.79737801627008], [-73.8254396246873, 40.797370743686855], [-73.82542459167145, 40.797360374393556], [-73.82540928500612, 40.79734583481299], [-73.82539243858434, 40.79733531836187], [-73.82536830573423, 40.79732477277685], [-73.8253554600605, 40.79732050336674], [-73.82533369628057, 40.797314508836465], [-73.82531554049231, 40.797310780858155], [-73.82529854222267, 40.79730882721592], [-73.82527960816476, 40.797308249034664], [-73.82526060781885, 40.79731041766126], [-73.8252267520703, 40.797315607694834], [-73.82520825573873, 40.79731975823043], [-73.82519068280165, 40.79732434271366], [-73.82516577770203, 40.797332039581306], [-73.82514036020439, 40.79734144676907], [-73.82511588636221, 40.797352468148645], [-73.82510243068933, 40.797359039515946], [-73.82507990262411, 40.79736924430661], [-73.82506496699443, 40.79737577656827], [-73.82504612538247, 40.79738480723419], [-73.825025037248, 40.797397102904384], [-73.82501009483249, 40.79740621202173], [-73.82499364106665, 40.79741749750835], [-73.82497670188856, 40.79742911496073], [-73.82495973803958, 40.7974407229916], [-73.82494233868215, 40.7974514575522], [-73.82492540431832, 40.797460986299434], [-73.82490897841186, 40.79747105591801], [-73.82489251068328, 40.79748327900771], [-73.82487655546714, 40.79749450107279], [-73.82483827649422, 40.797519008810845], [-73.82481787170418, 40.79753245772011], [-73.82479945953686, 40.797545136190216], [-73.82478400216343, 40.79755631561001], [-73.82476949624709, 40.797566144377896], [-73.82475599888767, 40.79757529083935], [-73.82474250297047, 40.79758388815373], [-73.82473096752513, 40.797590867473154], [-73.82471992705914, 40.797598567735136], [-73.82470444905408, 40.797608531214706], [-73.82468899079853, 40.79762003394401], [-73.82467474034358, 40.797631538508426], [-73.82466162794213, 40.79764316173478], [-73.82464801455137, 40.79765634318738], [-73.8246355677732, 40.79766675202779], [-73.82462408362619, 40.79767642404762], [-73.82461427369883, 40.797685026452335], [-73.8246055570433, 40.79769250575524], [-73.82458991288311, 40.79770678207114], [-73.82457080784845, 40.797721772638674], [-73.82455039686182, 40.79773719393289], [-73.82453884555974, 40.79774548794707], [-73.82452679765181, 40.797753592905536], [-73.82451478244087, 40.797762255101546], [-73.82450227059607, 40.797770989581146], [-73.82449075041026, 40.7977806878343], [-73.8244789594347, 40.797789998690654], [-73.82445416824199, 40.79780865641873], [-73.82444160729729, 40.79781769655053], [-73.8244290932539, 40.797827006300416], [-73.82441508643042, 40.79783620505123], [-73.82441070721514, 40.79783908031688], [-73.82440210139622, 40.79784472252399], [-73.82439009015607, 40.797852034292056], [-73.82437142036838, 40.797863532085884], [-73.8243532973943, 40.79787440047978], [-73.82433169850215, 40.79788641698487], [-73.82430916152028, 40.797899971277864], [-73.82429809724165, 40.797907635277596], [-73.82428655665724, 40.79791593799392], [-73.8242758723422, 40.7979238820027], [-73.8242653021729, 40.79793284390028], [-73.82425220489337, 40.79794286472391], [-73.82423891686308, 40.7979536603945], [-73.82422639884372, 40.79796445556239], [-73.82421314703053, 40.79797522381104], [-73.82420077313753, 40.79798528141499], [-73.82418861460366, 40.79799459218334], [-73.82417660072626, 40.79800296670512], [-73.82416407743435, 40.79801082664873], [-73.82415594961215, 40.798015173423764], [-73.82414299162568, 40.79802210494718], [-73.82412631945525, 40.798031002741624], [-73.82411261469552, 40.79803764598401], [-73.82409895036022, 40.798043172228176], [-73.82408766834955, 40.79804820639922], [-73.82406589617567, 40.79805851969601], [-73.82404823572678, 40.7980692537225], [-73.82403593250702, 40.798079662716795], [-73.82402524666442, 40.79809174909628], [-73.8240172265261, 40.798104803987094], [-73.82401315587505, 40.79811602698823], [-73.82401199915849, 40.798127632710006], [-73.8240137819841, 40.79813910789447], [-73.82401763990725, 40.79815448539658], [-73.82402219214279, 40.798167062601756], [-73.82402851212223, 40.79817930996775], [-73.82403494232726, 40.79819414241587], [-73.8240333795598, 40.79819782290469], [-73.82401551273695, 40.798206053449015], [-73.82399574244394, 40.798212668821435], [-73.82398402759729, 40.798215559841395], [-73.82396863489204, 40.79821993001969], [-73.82395047568913, 40.798226430912024], [-73.82394787857629, 40.79822736408152], [-73.82402784484827, 40.79836118320873], [-73.82400662435127, 40.79836690597245], [-73.82393176109085, 40.79823238214063], [-73.8239173778405, 40.798236069848386], [-73.8239076208587, 40.798237758339866], [-73.82390027314939, 40.79822927201366], [-73.82389948153417, 40.79822835343624], [-73.8238899102278, 40.798213687392284], [-73.8238842393205, 40.79820291774517], [-73.82387813415245, 40.798190309515164], [-73.82387313549054, 40.798180667334215], [-73.8238676785514, 40.79816934853054], [-73.82386122910178, 40.79815738859853], [-73.82385948384699, 40.79815413510464], [-73.82384776617477, 40.798157845639864], [-73.82382993975929, 40.79816413074683], [-73.82381501957484, 40.798169294356164], [-73.82379816566444, 40.79817503196445], [-73.8237776714884, 40.798181953099125], [-73.82375221660752, 40.798191711223616], [-73.82373778979141, 40.79819794824134], [-73.82372433799061, 40.79820369053847], [-73.82371042208517, 40.798209873385844], [-73.82368203516228, 40.798221248654784], [-73.82366956653428, 40.79822699295204], [-73.82366001431268, 40.79823202101608], [-73.82364333992571, 40.79824171129888], [-73.82362922407796, 40.79825163165327], [-73.82361040829849, 40.79826439006234], [-73.82358959241631, 40.79828063881076], [-73.82357362384315, 40.79829699498726], [-73.8235635697473, 40.79830373270763], [-73.82354913262645, 40.79831384152882], [-73.82353841606223, 40.798324829185475], [-73.82352815484006, 40.79833801513394], [-73.82351837317447, 40.798349004890106], [-73.82350720893336, 40.79836296342067], [-73.8234985809342, 40.79837691645229], [-73.8234944639465, 40.798387950060246], [-73.82349460019589, 40.798389940301014], [-73.8234954114755, 40.798401648875725], [-73.82350293523476, 40.79841073775033], [-73.82351315246025, 40.79842321609788], [-73.82351365730578, 40.79843817402499], [-73.82350600015731, 40.798438818832395], [-73.82350148440605, 40.79843920007392], [-73.82349304805373, 40.798434828484815], [-73.82348385821915, 40.79842363897253], [-73.82347251264153, 40.79841228416122], [-73.823460731983, 40.798404225746246], [-73.82345033382505, 40.798402038693744], [-73.82343323450802, 40.798406514407844], [-73.82367983693867, 40.79876994649952], [-73.82364368780398, 40.79878224199503], [-73.82340872309875, 40.79841781429594], [-73.82341152585504, 40.798433353203414], [-73.8234097025365, 40.798441832211964], [-73.8234030406431, 40.798446820120795], [-73.82338938032765, 40.798450860338164], [-73.82337529959138, 40.798447516570725], [-73.82336058757198, 40.79844115420241], [-73.82334751413694, 40.79843794700047], [-73.82333338435068, 40.79843946456705], [-73.8233143755997, 40.798444820027406], [-73.82328823838274, 40.79845254207463], [-73.8232772556516, 40.7984560648775], [-73.82325371060094, 40.79846953616837], [-73.82324241802799, 40.798478496864064], [-73.82323365195495, 40.79848597598691], [-73.8232197511897, 40.798500281090796], [-73.82321205420408, 40.79851127401777], [-73.82321139334279, 40.798519394626844], [-73.82320630261005, 40.7985311661872], [-73.82319351549555, 40.79854473571745], [-73.82317382658194, 40.79856089615796], [-73.82316181173549, 40.79856954883151], [-73.82315027235262, 40.798577302622746], [-73.82314020765315, 40.79858351878377], [-73.8231240065516, 40.79859331844132], [-73.82311396976115, 40.798602469349674], [-73.82310769668375, 40.79861310377281], [-73.82310839122141, 40.798623839519415], [-73.82311590493838, 40.7986364482758], [-73.82312337819698, 40.79865150803102], [-73.82313573115682, 40.79866727619846], [-73.82313766008336, 40.7986726639212], [-73.82313673921938, 40.79868023565306], [-73.82312841093474, 40.79868845439324], [-73.82311489512173, 40.798691018065], [-73.8231085737902, 40.79868886471187], [-73.8230960275046, 40.79868329836289], [-73.82330552497581, 40.79896286612052], [-73.82327118223391, 40.79897516359795], [-73.82305360689695, 40.79866039654989], [-73.8230372416952, 40.79865180787303], [-73.82302103408746, 40.798646298381755], [-73.8230026297057, 40.79864228972395], [-73.82298307296713, 40.79864892334321], [-73.82295909405569, 40.79866011640211], [-73.82294809615982, 40.79866526012424], [-73.82292700919174, 40.79867632142047], [-73.82289590533946, 40.79869358954433], [-73.82287477601989, 40.798707470717694], [-73.82285414291074, 40.79872201738985], [-73.82283416766423, 40.7987384478854], [-73.82282341708411, 40.798748148326524], [-73.82281323701933, 40.79875784847091], [-73.82280299536882, 40.798767909201544], [-73.82279111529684, 40.79877906637786], [-73.822779308267, 40.79878986298349], [-73.82276780065699, 40.798799562255475], [-73.82275520220757, 40.79880895344544], [-73.82274269160244, 40.79881681325352], [-73.8227306453943, 40.7988241247944], [-73.82271764914078, 40.79883156939552], [-73.82270466647762, 40.79883904199288], [-73.82269168154079, 40.7988470360911], [-73.82268060684444, 40.798854178412874], [-73.82265858258556, 40.79886584264889], [-73.82263553621992, 40.79887862119331], [-73.82261101579184, 40.798893189314455], [-73.82259935617279, 40.79890184247571], [-73.82258745707641, 40.79891151365305], [-73.82257800073869, 40.79892011760028], [-73.8225691285687, 40.798927983492305], [-73.82255326774539, 40.798943709462336], [-73.822540564193, 40.7989572773729], [-73.8227772788283, 40.79932022137899], [-73.822742935804, 40.799332518865874], [-73.82251916455868, 40.79898773568924], [-73.82250330031033, 40.79900440581912], [-73.82249659895142, 40.79901116787401], [-73.82247961996698, 40.799028385766775], [-73.82246038700627, 40.79904630395867], [-73.822450611425, 40.79905490740364], [-73.82244095680252, 40.799063121873026], [-73.82243106099395, 40.79907208514291], [-73.82241227004738, 40.79908891375366], [-73.82262174556764, 40.79940780807213], [-73.82259282686805, 40.799417369630845], [-73.82238754736628, 40.79911289157356], [-73.82237077314443, 40.79912861493816], [-73.82235317818854, 40.79914582365406], [-73.82234452634404, 40.79915478833084], [-73.82232800460899, 40.799173862917414], [-73.82231764347806, 40.7991842841021], [-73.82230779182088, 40.799195110644554], [-73.82229692766207, 40.79920774540007], [-73.82228583694548, 40.79922076745852], [-73.82227340628576, 40.79923413875037], [-73.82226257672107, 40.79924677472853], [-73.82225214226386, 40.799258312349714], [-73.82224290607851, 40.79926913916538], [-73.82223391759052, 40.799280327714285], [-73.82222579350075, 40.799291508716735], [-73.8222182547426, 40.79930160103739], [-73.82220488665996, 40.79931994101361], [-73.82219005802659, 40.799339378873235], [-73.82218186146437, 40.799350946241255], [-73.82217323847071, 40.799362873632695], [-73.82216442676987, 40.79937480073235], [-73.82215564923526, 40.799386727884006], [-73.82214658872353, 40.799398293916575], [-73.82212930449958, 40.79942324759961], [-73.82212114273379, 40.799435562681886], [-73.8221138515337, 40.79944564549362], [-73.82210659374235, 40.79945683720566], [-73.82209909838669, 40.79946766786897], [-73.82209190587866, 40.79948107352625], [-73.82208117471227, 40.79949695175417], [-73.82207037323403, 40.79951257623915], [-73.82206010615744, 40.799527445004365], [-73.82204920945453, 40.79954355683873], [-73.82204868514184, 40.79954432999572], [-73.82203928318566, 40.79955957953919], [-73.8220298933749, 40.79957481971902], [-73.82202061198767, 40.79958933049333], [-73.82201226958716, 40.799602734370126], [-73.82200193371685, 40.79961685602886], [-73.82199062698199, 40.79963097668998], [-73.82197884761408, 40.79964434895511], [-73.82196943174213, 40.799655888123105], [-73.8219595790757, 40.79966671513629], [-73.82194910988227, 40.79967751372211], [-73.8219360760507, 40.799690505270625], [-73.8219249157312, 40.79970278940515], [-73.82191596602848, 40.79971287014451], [-73.82190473794974, 40.79972402823555], [-73.82189354604726, 40.79973517649737], [-73.82188235110192, 40.79974746057427], [-73.82187484687593, 40.799757543543805], [-73.8218714089457, 40.79976215757087], [-73.82185814664778, 40.799776320261124], [-73.82184193888526, 40.799788676663994], [-73.82182869117914, 40.79979730099057], [-73.82181248826221, 40.79980759616688], [-73.82180068691181, 40.79981597897685], [-73.82179207914507, 40.799822170467614], [-73.82177979890135, 40.799832390454], [-73.82176213888194, 40.79984701389647], [-73.82175519183347, 40.799852685943875], [-73.82174443958425, 40.799863295272296], [-73.82173446113313, 40.79987669659847], [-73.82172024104517, 40.79989023744567], [-73.82170694796247, 40.799902291598734], [-73.8216956852832, 40.79991292693704], [-73.82168293714544, 40.79992572239155], [-73.82166696739415, 40.799941455272865], [-73.82165135315044, 40.7999579559675], [-73.82163578469607, 40.799974807025926], [-73.82161789179172, 40.79999198769278], [-73.82160695410658, 40.80000094877601], [-73.82159516299573, 40.80001025933509], [-73.82158353845654, 40.80001869734453], [-73.82157102359831, 40.80002800728345], [-73.8215597305343, 40.80003696781219], [-73.82154817904052, 40.80004504524934], [-73.82153660313013, 40.800053348304054], [-73.82152607223834, 40.80006063458997], [-73.82150642539645, 40.80007394084927], [-73.82149078825876, 40.80008547034405], [-73.82147972283319, 40.80009330528056], [-73.8214635046579, 40.8001049683945], [-73.82144779783906, 40.80011575949209], [-73.82143064647927, 40.80012602567126], [-73.82133769042089, 40.800188919002146], [-73.8212493571626, 40.80025554887279], [-73.82116590644789, 40.80032571936929], [-73.82108758366641, 40.80039922416617], [-73.82103688606428, 40.800440806257264], [-73.82098071917677, 40.80047810984697], [-73.82091970490475, 40.8005107218901], [-73.82085451882595, 40.800538281288105], [-73.82078588271298, 40.80056048288757], [-73.82071455654156, 40.80057708085979], [-73.82064133007556, 40.800587891421806], [-73.82062521884053, 40.80059077576215], [-73.82060479533888, 40.8005928326833], [-73.82059406654892, 40.80059470717764], [-73.82057433750883, 40.80059846857027], [-73.8205557928929, 40.80060233784681], [-73.82053823274919, 40.80060656045581], [-73.82051871637849, 40.80061075354743], [-73.82049338047337, 40.80061494645224], [-73.82046903023078, 40.80061905092555], [-73.82044951258345, 40.80062346915729], [-73.82043245966628, 40.80062786442104], [-73.82052810788423, 40.80100602386803], [-73.82049919995306, 40.80101146781907]]]}}, {\"id\": \"252\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 253, \"Shape_Leng\": 0.0360514998192, \"Shape_Area\": 7.83395761191e-05, \"zone\": \"Willets Point\", \"LocationID\": 253, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.83908354399988, 40.76525691299991], [-73.83790429599993, 40.764587796999876], [-73.8369514729998, 40.76283036999996], [-73.83691694399995, 40.762766679999885], [-73.83689970199995, 40.7626981179999], [-73.83689455800001, 40.76260268399992], [-73.83679107299986, 40.76116667099992], [-73.83861811099983, 40.75709081599992], [-73.83867321899987, 40.75701997899995], [-73.83875129699992, 40.756994990999935], [-73.8391232999999, 40.75682806199994], [-73.83978335899995, 40.75654003599987], [-73.84045537199984, 40.75626729599988], [-73.84113839099994, 40.756010244999864], [-73.84183146799998, 40.75576926799991], [-73.84253360699991, 40.75554469899986], [-73.84324381399976, 40.75533680399995], [-73.84363442799987, 40.75615699399987], [-73.84404431200002, 40.756944772999915], [-73.84444970399977, 40.75774348399989], [-73.84479027099985, 40.75840315899994], [-73.84513404100002, 40.75906847499993], [-73.8454898629999, 40.75972867199994], [-73.84585235799983, 40.76045539799988], [-73.8459114399999, 40.76058469999987], [-73.84597463499995, 40.76071730199993], [-73.84601517799996, 40.76080248899988], [-73.84606731299985, 40.76090886399986], [-73.84663046699997, 40.76129165899989], [-73.84670933714987, 40.76134526931977], [-73.8456278805877, 40.76214274135956], [-73.84523206915873, 40.76243460761822], [-73.8447450720674, 40.76275563070319], [-73.84467648051529, 40.76275218085763], [-73.84428576361255, 40.76299855215024], [-73.84411566722947, 40.76313619957922], [-73.84377426607954, 40.763299322966375], [-73.8435936131629, 40.76337054278836], [-73.84366832633641, 40.76346990739617], [-73.84361523822149, 40.76349671406157], [-73.84368787664133, 40.76361272608367], [-73.84365028079019, 40.76362779479571], [-73.8435710063455, 40.763515132804585], [-73.84352013946436, 40.76353690309135], [-73.84343628780456, 40.763432566193686], [-73.8431432418535, 40.763548094765], [-73.84306961269657, 40.76359256915376], [-73.84300450700954, 40.76370422400581], [-73.84298349190715, 40.7638318513791], [-73.84300671126373, 40.76399462233396], [-73.84305145175345, 40.76409735815317], [-73.84304240810822, 40.764321319920285], [-73.84298479705527, 40.7644235941218], [-73.84272819396486, 40.76462854893166], [-73.84261312263459, 40.76468298969289], [-73.8423114797326, 40.764831665329005], [-73.84211652153546, 40.764901034197], [-73.84183322138846, 40.76497505396951], [-73.84107085269765, 40.765208406720454], [-73.84052635479259, 40.76538799020267], [-73.84046519597908, 40.76530287000054], [-73.84016724084617, 40.765308545429356], [-73.83992813236753, 40.76535021190038], [-73.83970053428521, 40.76528958843764], [-73.83957826853546, 40.76525232297297], [-73.8394581342306, 40.76544576822929], [-73.83935912299995, 40.765409871999886], [-73.83922344399986, 40.765336291999844], [-73.83908354399988, 40.76525691299991]]]}}, {\"id\": \"253\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 254, \"Shape_Leng\": 0.0858863754861, \"Shape_Area\": 0.000360040216032, \"zone\": \"Williamsbridge/Olinville\", \"LocationID\": 254, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.85186563799999, 40.87905886499989], [-73.85138374099995, 40.878476222999886], [-73.85091896099982, 40.877893440999856], [-73.8521121739999, 40.87729787999995], [-73.85254854099988, 40.877091586999924], [-73.85172236099986, 40.87684337099995], [-73.85093186799989, 40.875830013999874], [-73.8506895569999, 40.87548667799993], [-73.85143247999996, 40.87498727099993], [-73.85206424599991, 40.874503646999905], [-73.85215435899995, 40.87443419899993], [-73.85278034299986, 40.87395322899988], [-73.85362131899994, 40.87331208699986], [-73.85363638799987, 40.87330059899996], [-73.854508051, 40.87358094199986], [-73.85551758400005, 40.87390395599987], [-73.85577052399987, 40.87398371299992], [-73.85651292399994, 40.874217792999936], [-73.85655118199992, 40.87303042899989], [-73.85661382499993, 40.87125016699991], [-73.85754854399994, 40.87126631799993], [-73.85848731699984, 40.87128405999989], [-73.85942680499996, 40.8713006969999], [-73.86036598199989, 40.87131862199991], [-73.86137924099984, 40.8713365119999], [-73.86239212899996, 40.8713567339999], [-73.86333198300004, 40.8713751569999], [-73.86426604499991, 40.87138964999991], [-73.865208901, 40.87140764999989], [-73.86614489499986, 40.871422984999924], [-73.8671586419999, 40.871441912999934], [-73.86825887799992, 40.87146232799988], [-73.86935122399996, 40.871483353999906], [-73.87056192499989, 40.87151077799995], [-73.87037667199995, 40.8732843719999], [-73.8703131089999, 40.87534062099997], [-73.87031508399987, 40.87561752299991], [-73.8703168469999, 40.87573572299987], [-73.87031256599984, 40.87583307799987], [-73.87030212499997, 40.875930164999914], [-73.87028554699997, 40.876026758999956], [-73.87026287199994, 40.87612263599991], [-73.87023414999992, 40.876217573999924], [-73.87019944899986, 40.87631135499987], [-73.8701809419998, 40.87635452999992], [-73.8701541849999, 40.876416957999886], [-73.87004672699982, 40.876670530999945], [-73.86948896199988, 40.87798946799993], [-73.86943471300005, 40.87812919899994], [-73.8692538419999, 40.878431124999935], [-73.869221773, 40.87846483499988], [-73.86918507299987, 40.87850341099988], [-73.86911192999995, 40.87854672299993], [-73.86902622499986, 40.87857616999984], [-73.86893508399993, 40.87858929599985], [-73.86884613499979, 40.878586356999946], [-73.86811829499993, 40.88006228299991], [-73.86705258299985, 40.882255215999876], [-73.86605231299984, 40.88428704399993], [-73.86603813199996, 40.88435902199995], [-73.8660380959999, 40.884359192999945], [-73.86603810799987, 40.8843593819999], [-73.86604065199995, 40.88443298199991], [-73.86604066399985, 40.88443323499996], [-73.86604073399987, 40.88443347699989], [-73.86606024899994, 40.88450565599988], [-73.86606028499985, 40.884505799999914], [-73.86606034399993, 40.884505926999914], [-73.8660956839999, 40.884573620999916], [-73.86616465799986, 40.88467792299987], [-73.86616476499985, 40.884678076999954], [-73.866164848, 40.88467823899988], [-73.86621797999994, 40.88479001499995], [-73.86625345799993, 40.88490729999992], [-73.866270006, 40.88502729999992], [-73.86627001699988, 40.8850273899999], [-73.86627001699986, 40.88502747899987], [-73.86626727199993, 40.885147690999865], [-73.86624574899986, 40.88526544999991], [-73.86624572499991, 40.88526556799994], [-73.86624567699987, 40.88526569299995], [-73.86620661499991, 40.885378613999926], [-73.86538049299986, 40.886883474999905], [-73.86460021999999, 40.888299711999906], [-73.8638125559999, 40.88971678099993], [-73.86304439099993, 40.89111565099991], [-73.86276234599985, 40.89162005699992], [-73.86280050199993, 40.8917207669999], [-73.86274739599986, 40.891843574999946], [-73.86225345500002, 40.892670487999965], [-73.86215575199998, 40.89283406299989], [-73.86198229599998, 40.89316126199993], [-73.86192535099994, 40.89329121599993], [-73.86149253599987, 40.89427891699995], [-73.86140513999995, 40.89453904399989], [-73.86135927899984, 40.89466231899991], [-73.860710982, 40.89537054399996], [-73.859833381, 40.895085226999875], [-73.85695619399986, 40.89416793799992], [-73.85736281700004, 40.89341143899987], [-73.85566327299986, 40.8928523939999], [-73.85525378499982, 40.89272062599986], [-73.85449169199995, 40.89247851799996], [-73.85340274699983, 40.89213229999996], [-73.85182948399994, 40.891629781999924], [-73.85149873299996, 40.891534796999935], [-73.85069084099977, 40.89130277099987], [-73.85040125099985, 40.891205458999885], [-73.84874153399994, 40.890647714999936], [-73.84741355299995, 40.89022219799991], [-73.84632533099996, 40.889877226999886], [-73.84590568599998, 40.889741368999864], [-73.84626182299986, 40.88909833799986], [-73.84645600899995, 40.888748434999854], [-73.84665457799996, 40.88839061999992], [-73.84704957999995, 40.8876718159999], [-73.84744396900003, 40.88697853099991], [-73.84748708800002, 40.886914537999864], [-73.84785777299996, 40.88636441199987], [-73.84790388199997, 40.88629598399993], [-73.84798702099991, 40.886172117999884], [-73.84802165699985, 40.886120514999924], [-73.84804918100001, 40.88607950799991], [-73.84809415299982, 40.886012505999865], [-73.84813585799989, 40.88595036999992], [-73.84817178599984, 40.88589684199993], [-73.84820835300003, 40.88584236099989], [-73.84824941299993, 40.88578118699991], [-73.84828534100001, 40.885727657999915], [-73.84832190799986, 40.8856731779999], [-73.84836417199985, 40.885610210999886], [-73.84846274099996, 40.88546103199993], [-73.84876385000003, 40.88500531699991], [-73.84882044099987, 40.88491966699994], [-73.84927742099983, 40.88424150399993], [-73.84973488199986, 40.8835558029999], [-73.85020149399993, 40.88286986999993], [-73.8506512309999, 40.882191789999865], [-73.851106036, 40.88150897099993], [-73.85156600599989, 40.88082536399993], [-73.85203280500001, 40.88013997199988], [-73.85248305800003, 40.87946370499989], [-73.85207011699983, 40.87929765699988], [-73.85186563799999, 40.87905886499989]]]}}, {\"id\": \"254\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 255, \"Shape_Leng\": 0.0623841997664, \"Shape_Area\": 0.000172309184842, \"zone\": \"Williamsburg (North Side)\", \"LocationID\": 255, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.96176070375392, 40.72522879205536], [-73.96153679379327, 40.7251089186773], [-73.9614126353891, 40.725177219246326], [-73.96116366334205, 40.72526460591537], [-73.9604323022029, 40.725114108830184], [-73.96036651068175, 40.72504730919287], [-73.95875928184918, 40.724541354184765], [-73.95833078213447, 40.72441170088298], [-73.95814292503528, 40.724404284201356], [-73.95771989199996, 40.72428495699989], [-73.95627311499997, 40.723389875999914], [-73.95477662900004, 40.72245950499994], [-73.95418330299992, 40.72263642799991], [-73.95382461699991, 40.722759734999904], [-73.95332036799981, 40.72293269999991], [-73.95249594800003, 40.72321773399991], [-73.95164632299998, 40.723485856999936], [-73.95064594299997, 40.72170569299989], [-73.94986599500001, 40.72217948399988], [-73.94907816799986, 40.72254028699993], [-73.94887459699996, 40.721292406999886], [-73.94853260499984, 40.71916717599985], [-73.94841638599995, 40.71845553299995], [-73.94721341799986, 40.718560985999865], [-73.94832167599995, 40.71786451199997], [-73.94866397500004, 40.7176827709999], [-73.94904281699998, 40.717428518999924], [-73.94954034299991, 40.717121995999946], [-73.9498379229999, 40.716938665999855], [-73.95004619199985, 40.71680522799992], [-73.95114433699997, 40.71603452099991], [-73.95182429600005, 40.715475968999925], [-73.95210878499995, 40.715258893999874], [-73.95275381300002, 40.71473599099994], [-73.95335294599992, 40.71418365499992], [-73.95348472199983, 40.71407020799988], [-73.95433904200002, 40.714071701999906], [-73.95503638699986, 40.71291373099997], [-73.95547596099992, 40.7122654259999], [-73.95680951699988, 40.712756387999924], [-73.95834207500002, 40.71330630099992], [-73.95979763199988, 40.713887133999926], [-73.96128719699986, 40.71450229699994], [-73.96277758699983, 40.71510663499988], [-73.96447229899992, 40.71573310599994], [-73.96609959099992, 40.71620611299992], [-73.96663975699981, 40.71636484999994], [-73.96736391666703, 40.71648367952745], [-73.96729235333521, 40.71651557312143], [-73.96724404348966, 40.716585349916386], [-73.9672633131556, 40.71666616768813], [-73.96711354289924, 40.71689753910394], [-73.9671515999894, 40.716919273179705], [-73.96726798356039, 40.7169857409556], [-73.96676841729844, 40.717705403518835], [-73.9665969033269, 40.717967980887636], [-73.96717386981618, 40.71827675878373], [-73.96718435668602, 40.71826224255602], [-73.96729652521333, 40.71831181417356], [-73.96721948180506, 40.718415969504754], [-73.96717135968257, 40.71848342806873], [-73.96705912061108, 40.718438980286464], [-73.96708162217523, 40.71840615665343], [-73.96714225214744, 40.718317715272924], [-73.96656241897492, 40.7180158668062], [-73.96655263001192, 40.71803865314392], [-73.96653720436414, 40.71803482687111], [-73.96652792581678, 40.71803252419761], [-73.96651367937793, 40.71805251673066], [-73.966322244646, 40.71832115687809], [-73.9662235827445, 40.71827065347471], [-73.96620531446817, 40.71826130162309], [-73.96615035842788, 40.71831123846767], [-73.96604482029281, 40.71840837581112], [-73.96607655887611, 40.71843191816668], [-73.96610196657393, 40.71845076521724], [-73.96611826905765, 40.71850526164153], [-73.9658789353266, 40.71875507768634], [-73.96634259840096, 40.7190187424301], [-73.96637067848123, 40.71898970835092], [-73.96662993360903, 40.71913242786092], [-73.96654344630538, 40.719222942469415], [-73.96631897740717, 40.719108420008254], [-73.96636503305272, 40.71905633078414], [-73.96585847175166, 40.718773228346414], [-73.96570636269995, 40.718914902636634], [-73.96570332142306, 40.71891811422538], [-73.96563350220046, 40.71893371662328], [-73.96552577399207, 40.71890089271253], [-73.96515385976045, 40.71926968304744], [-73.96528401324463, 40.71936086098301], [-73.9652884928012, 40.71936399960859], [-73.9653961343865, 40.719564879254236], [-73.9655469308472, 40.71965919869085], [-73.96538515805561, 40.719954272782395], [-73.9653743106172, 40.72009363306692], [-73.96513713562288, 40.72034359475416], [-73.96501325438103, 40.72029027209133], [-73.96490548671237, 40.72032712936547], [-73.96456213748016, 40.720123070522234], [-73.96446386180361, 40.72006466299988], [-73.96438190823226, 40.72013942466022], [-73.96435606131355, 40.72016300359871], [-73.96446581250262, 40.72022477634749], [-73.96447993393507, 40.72023272348612], [-73.96439765915947, 40.72029673502918], [-73.96468283641285, 40.72047045735994], [-73.96459313964766, 40.72054484421151], [-73.96435090640152, 40.72040403237185], [-73.96432509603717, 40.72047236934719], [-73.96429659818648, 40.720540085337504], [-73.96425246396787, 40.720598255889065], [-73.96420274484129, 40.720653757188906], [-73.96414771830368, 40.72070627945257], [-73.96408769147868, 40.72075552952228], [-73.96504914549352, 40.72135734425181], [-73.9648942203191, 40.721475120882026], [-73.96399793724473, 40.720938438072736], [-73.96360201823921, 40.72153866694272], [-73.96357950967109, 40.721572788715775], [-73.96347810162712, 40.72155651302858], [-73.96347715752879, 40.72155636128484], [-73.96345365315541, 40.72157889003159], [-73.96298124491233, 40.72203167639525], [-73.9627566541133, 40.72239901506301], [-73.96225346277055, 40.723532297772984], [-73.96223504982149, 40.72357025913259], [-73.9620727192193, 40.72388030070138], [-73.96200744799489, 40.723991901303876], [-73.96190256156959, 40.72409829024043], [-73.96189426706076, 40.7241067020505], [-73.96165984300669, 40.724860454231504], [-73.96211461583417, 40.72514806319948], [-73.96190239621568, 40.72532050090551], [-73.96176070375392, 40.72522879205536]]]}}, {\"id\": \"255\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 256, \"Shape_Leng\": 0.0679149669603, \"Shape_Area\": 0.000168611097013, \"zone\": \"Williamsburg (South Side)\", \"LocationID\": 256, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95834207500002, 40.71330630099992], [-73.95680951699988, 40.712756387999924], [-73.95547596099992, 40.7122654259999], [-73.95503638699986, 40.71291373099997], [-73.95433904200002, 40.714071701999906], [-73.95348472199983, 40.71407020799988], [-73.95325272999993, 40.71406562999987], [-73.95316497599985, 40.714065097999914], [-73.9530314939999, 40.714062846999894], [-73.9529104979999, 40.714060986999876], [-73.95162366299994, 40.7140663609999], [-73.94937231899985, 40.71406687799986], [-73.94924526199999, 40.7133203089999], [-73.94913082999982, 40.71261242099995], [-73.94901358399997, 40.71190375599989], [-73.94889233899988, 40.71116753499993], [-73.94877149699997, 40.71043225899994], [-73.94865533499987, 40.70972498299988], [-73.95087492699996, 40.70951311499991], [-73.95082963899989, 40.70914359799985], [-73.95076027199997, 40.708802859999906], [-73.95065919499987, 40.70816998299994], [-73.95053010000007, 40.70739098499994], [-73.95040588999979, 40.706652745999875], [-73.95028918399986, 40.70591343899985], [-73.95023693799996, 40.70547324699994], [-73.95095606199983, 40.70584293599995], [-73.95108529899998, 40.7059129519999], [-73.95191055699979, 40.7063600789999], [-73.95229955499995, 40.70649029299996], [-73.95251503799987, 40.70655858699992], [-73.95291388700002, 40.70670031099991], [-73.95375309099992, 40.70698188099987], [-73.95401701399999, 40.7070802409999], [-73.95554165799997, 40.70759344799989], [-73.95626945199986, 40.70784956199987], [-73.95672449099989, 40.70800995199991], [-73.95720321600002, 40.708179246999876], [-73.95728255399992, 40.70820730999989], [-73.95735981099993, 40.70823462399988], [-73.9579043079999, 40.70842717299994], [-73.95815250199996, 40.70804091799986], [-73.95838432400002, 40.70767669099991], [-73.958455768, 40.707251846999895], [-73.95920084799998, 40.70733158499991], [-73.95939542499995, 40.707352407999984], [-73.95993835900006, 40.70740680099993], [-73.96040383200001, 40.707451885999916], [-73.96050333599985, 40.70746152299992], [-73.96114422899987, 40.707523595999874], [-73.96186019899987, 40.70759505699993], [-73.96213128899981, 40.707622692999955], [-73.96274332999992, 40.70768508199991], [-73.96327873800001, 40.70772929899995], [-73.96442087299985, 40.70753372399989], [-73.96484837799986, 40.70745361099987], [-73.9659321969999, 40.70724387399989], [-73.96728890199985, 40.70700200299994], [-73.96838933699995, 40.70682918699989], [-73.96929296374243, 40.70709333104775], [-73.96962252142447, 40.707627588777555], [-73.96984864823531, 40.708003041687675], [-73.96994088125628, 40.70931023140589], [-73.96960817478119, 40.710164917598135], [-73.96955679589692, 40.71029607131979], [-73.96964728659579, 40.71037491098549], [-73.96983077701677, 40.71039021573928], [-73.9699448936496, 40.71039973365078], [-73.9700095322252, 40.7104539376276], [-73.96997064587534, 40.71050807121897], [-73.96988006133613, 40.71050800918952], [-73.96981825475457, 40.71051001036166], [-73.96973123629752, 40.71051282944216], [-73.96963399936386, 40.710665396155825], [-73.96971161435603, 40.71069006736659], [-73.96979416000536, 40.710740506479446], [-73.96975026678481, 40.71083287772713], [-73.96972883126466, 40.7108397014988], [-73.9693489494688, 40.71096061478666], [-73.96936184310228, 40.711000012698555], [-73.96955914381311, 40.71109735316971], [-73.96958170746989, 40.71110848477031], [-73.96969388597714, 40.71113727505959], [-73.96976280357137, 40.711172617759935], [-73.96970449314463, 40.71123658451135], [-73.96956219103225, 40.71119709704367], [-73.96950659655987, 40.711177142260965], [-73.9694522407415, 40.711157632229146], [-73.96933582617378, 40.71111323878581], [-73.96925812194723, 40.71116242174906], [-73.96923861019852, 40.71124610934427], [-73.9693914266158, 40.71135201441385], [-73.96945195784745, 40.711393965517324], [-73.96942598248582, 40.711472725127365], [-73.96930299630868, 40.71151202924544], [-73.96928944419821, 40.71150686060597], [-73.96918658210186, 40.711467636156115], [-73.96911535361018, 40.7115118989782], [-73.96893565827622, 40.71201705475273], [-73.96886701832432, 40.71223244408627], [-73.96880934254597, 40.71225605597726], [-73.96886922090012, 40.712266197516286], [-73.96883145652384, 40.71239276314765], [-73.96892343767315, 40.71242023948873], [-73.96894455591715, 40.71242654772024], [-73.96891951556479, 40.71247675466192], [-73.9688977507757, 40.71252500074198], [-73.96887248523522, 40.71257874444156], [-73.96885323825141, 40.71260793198413], [-73.96883197798022, 40.71264017567564], [-73.96881607240603, 40.71268527097035], [-73.96879580997484, 40.71273703885378], [-73.96873958212066, 40.712722760687406], [-73.9687292790881, 40.71272014379823], [-73.96872432113675, 40.71273024097742], [-73.96866711839206, 40.712846703241325], [-73.96854773954306, 40.7128167857182], [-73.96849857488866, 40.712804464738696], [-73.96846973347022, 40.712824709346144], [-73.96847298005669, 40.71285342770885], [-73.96845231624204, 40.71292318664481], [-73.96846827481582, 40.7129267938521], [-73.96856767224719, 40.712949263785035], [-73.96849927445857, 40.71310284084997], [-73.9688422905027, 40.713223925021595], [-73.96839125147874, 40.71426191817128], [-73.96800830687327, 40.71512072380612], [-73.96765448249123, 40.715869109725304], [-73.96736391666705, 40.71648367902752], [-73.96663975699981, 40.71636484999994], [-73.96609959099992, 40.71620611299992], [-73.96447229899992, 40.71573310599994], [-73.96277758699983, 40.71510663499988], [-73.96128719699986, 40.71450229699994], [-73.95979763199988, 40.713887133999926], [-73.95834207500002, 40.71330630099992]]]}}, {\"id\": \"256\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 257, \"Shape_Leng\": 0.0586690259793, \"Shape_Area\": 0.00013890947321, \"zone\": \"Windsor Terrace\", \"LocationID\": 257, \"borough\": \"Brooklyn\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.97984261899994, 40.660727440999956], [-73.97984227599983, 40.66072741399994], [-73.9798419799999, 40.660727449999854], [-73.97975708399997, 40.660738466999874], [-73.97975683499992, 40.66073850199989], [-73.9797565759999, 40.66073860199992], [-73.97968049599983, 40.660766593999895], [-73.97968033000002, 40.6607666569999], [-73.97968019899987, 40.66076673799995], [-73.97962025699994, 40.66080686399987], [-73.9796200439999, 40.66080700799993], [-73.97961986699994, 40.66080721499986], [-73.97957870099991, 40.66085324199988], [-73.97751171399987, 40.65986275599987], [-73.97515738099997, 40.65872279399991], [-73.97448233999991, 40.65833944999988], [-73.97436701299992, 40.65818790399992], [-73.97426735599991, 40.658028786999886], [-73.97418457500004, 40.6578635769999], [-73.97418455199994, 40.6578635049999], [-73.97411953499982, 40.657693751999915], [-73.97407283499992, 40.657521192999916], [-73.97407281199997, 40.657521085999896], [-73.97407278699997, 40.65752097799993], [-73.97404458299992, 40.65734725299994], [-73.97379189999988, 40.65639232299987], [-73.97355493999991, 40.6556439389999], [-73.97326628099982, 40.65464926099985], [-73.97305103699992, 40.65391250599987], [-73.97268288599984, 40.65263927799989], [-73.97232846299997, 40.65139792899986], [-73.9725172119999, 40.65131415399985], [-73.97265581899993, 40.65117357099994], [-73.97267189899989, 40.65112201399995], [-73.97268018299987, 40.6510693989999], [-73.97268056299997, 40.651016407999855], [-73.97267303599992, 40.65096372799994], [-73.9726576979999, 40.650912039999874], [-73.9726347479999, 40.650862013999934], [-73.97252930399988, 40.65075307999995], [-73.97234114899992, 40.65065904799994], [-73.9721097859998, 40.65066414299987], [-73.9719089039999, 40.65005140999992], [-73.97155930399988, 40.648822827999915], [-73.97139631700004, 40.64825778599986], [-73.97121038999984, 40.64762015399986], [-73.97084113699994, 40.64637857099993], [-73.97174398199986, 40.64600110999986], [-73.972085748, 40.64586232499991], [-73.97310135399987, 40.645450975999914], [-73.9741399689999, 40.64502975399993], [-73.97442369399982, 40.644913064999876], [-73.974642245, 40.644813218999914], [-73.97467086799998, 40.64493508299988], [-73.97474803899986, 40.645263590999875], [-73.97482741699987, 40.64566342099995], [-73.97591918799998, 40.64554427299992], [-73.97684760999987, 40.645442020999944], [-73.97777380600002, 40.645341876999964], [-73.97819074399992, 40.64752599199992], [-73.97911821899996, 40.647428060999914], [-73.98029115499993, 40.647297160999855], [-73.9803633469999, 40.64769374399988], [-73.98071174699994, 40.64952408699987], [-73.98105378299987, 40.6513038189999], [-73.98110911899991, 40.65159173199992], [-73.98129697399993, 40.65263750699989], [-73.9814529379999, 40.65341633599998], [-73.98179568699996, 40.65524609699991], [-73.98402397199989, 40.65659116999997], [-73.98622761299997, 40.65792048299986], [-73.98843135299977, 40.65925121199996], [-73.98784059399986, 40.65981686399987], [-73.98744982499997, 40.660189572999926], [-73.98731033299997, 40.6603296239999], [-73.98725068899996, 40.66037895599992], [-73.98713178899986, 40.660374367999935], [-73.9868846449999, 40.6603663889999], [-73.9868065409999, 40.66024190199991], [-73.98671251899984, 40.66012320299989], [-73.98660379699987, 40.660012021999876], [-73.98648200899994, 40.659909879999915], [-73.985086794, 40.65901328799989], [-73.98448024099987, 40.659598336999935], [-73.98391792699988, 40.66013748799985], [-73.98323712099993, 40.660936220999886], [-73.98273252399984, 40.66153948399988], [-73.982230342, 40.66214149199994], [-73.98017161099993, 40.66115465799992], [-73.98020628599997, 40.66110595499992], [-73.98020645099992, 40.66110571199987], [-73.98020654499996, 40.661105459999945], [-73.98022710399982, 40.6610481279999], [-73.98022716299988, 40.66104797499988], [-73.98022716299992, 40.66104784799991], [-73.980229014, 40.66098557799995], [-73.98022902599985, 40.660985325999896], [-73.980228943, 40.660985055999895], [-73.98021012299994, 40.660923555999936], [-73.98021005299988, 40.66092332199989], [-73.98020992299999, 40.660923132999876], [-73.98017209799997, 40.660867879999905], [-73.98017201599991, 40.660867753999845], [-73.98017186199999, 40.660867618999895], [-73.98012012799995, 40.66082312499993], [-73.98011991500002, 40.66082293599988], [-73.98011964299997, 40.66082279099989], [-73.98006025399978, 40.66079075799991], [-73.98000138699987, 40.660757698999866], [-73.98000113899982, 40.660757563999894], [-73.98000088999997, 40.66075749199991], [-73.97992682399993, 40.66073473199991], [-73.9799267049999, 40.66073469699993], [-73.97992656299992, 40.66073468699993], [-73.97984261899994, 40.660727440999956]]]}}, {\"id\": \"257\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 258, \"Shape_Leng\": 0.0890133787693, \"Shape_Area\": 0.000366209617143, \"zone\": \"Woodhaven\", \"LocationID\": 258, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.8450419489999, 40.68931894699996], [-73.84500533199989, 40.689243910999885], [-73.84430714400003, 40.68944166799988], [-73.84419150899994, 40.689474407999924], [-73.84415010299988, 40.6893932859999], [-73.84386364499983, 40.688832111999936], [-73.84356447799996, 40.68821396099985], [-73.84300297399979, 40.68703791599988], [-73.84217464599998, 40.6853535589999], [-73.84119605699982, 40.68335136599989], [-73.84134091999981, 40.68331050299987], [-73.84151391999985, 40.683261680999884], [-73.84237029099977, 40.683018421999904], [-73.84323090399981, 40.682775196999884], [-73.84408914999997, 40.68252929899992], [-73.84451063800005, 40.68241508799991], [-73.84496190700004, 40.68230745499992], [-73.84505837899985, 40.682284450999866], [-73.84513349899997, 40.68226653299996], [-73.84520200299993, 40.68225066799988], [-73.84529042399988, 40.68223044499989], [-73.84583237399983, 40.68210644999986], [-73.84671464299996, 40.68185849299988], [-73.84757845099998, 40.68157986799987], [-73.84786817599985, 40.681486831999905], [-73.84863383999999, 40.68168402799994], [-73.84921620599982, 40.68184421099987], [-73.8497640749999, 40.681986613999925], [-73.85060478199985, 40.68219378299992], [-73.85087162799985, 40.68225954199993], [-73.85197864599989, 40.682564210999864], [-73.851639288, 40.68188627199993], [-73.85131773799996, 40.681240015999904], [-73.85099089299987, 40.68056836699987], [-73.85168205499998, 40.68036797699993], [-73.85223417099996, 40.68020789099992], [-73.85267726099994, 40.68007941499991], [-73.85392389199987, 40.679686155999875], [-73.85439330799991, 40.67966011399988], [-73.85653410299985, 40.67952772499995], [-73.85683137000001, 40.67950933799989], [-73.85721078599984, 40.67948586399992], [-73.85779268399989, 40.67945571699988], [-73.85819836099995, 40.67943084899987], [-73.85878576899987, 40.68063046299989], [-73.85929512400006, 40.680486365999855], [-73.85966271799995, 40.68038237199995], [-73.85977462599983, 40.6803507119999], [-73.86053300099987, 40.680134447999905], [-73.86066198699996, 40.680097537999856], [-73.861423295, 40.67987881299996], [-73.86156047099999, 40.679839967999875], [-73.86229626199986, 40.679631579999906], [-73.86334943599995, 40.67935163999989], [-73.86378674500001, 40.68111374699991], [-73.86399008800005, 40.68192857599995], [-73.86410096700003, 40.68237284999988], [-73.8649940449999, 40.68212896999991], [-73.86602666899996, 40.681918051999915], [-73.86629002999987, 40.68311500699992], [-73.86630263900003, 40.68318669399984], [-73.86631496899999, 40.683248942999924], [-73.86650407899994, 40.684474051999885], [-73.86659848299999, 40.68526955899992], [-73.86687058599995, 40.686330590999944], [-73.86701802999987, 40.686830348999884], [-73.86728057399996, 40.68774928599988], [-73.867460889, 40.688415017999915], [-73.86762340399984, 40.6891453419999], [-73.86776832199985, 40.68980328899986], [-73.867909485, 40.69047626099993], [-73.8680588529999, 40.69118395799991], [-73.86859478099983, 40.693884374999854], [-73.86868454399996, 40.694034691999924], [-73.86842489800001, 40.69471811999993], [-73.86681433899996, 40.69506535099989], [-73.8645570889999, 40.695520456999915], [-73.86368656399996, 40.695691417999925], [-73.86236834199985, 40.69595014299991], [-73.86016364299988, 40.6963846579999], [-73.85928498399986, 40.69655371099988], [-73.8583307899999, 40.69674371099995], [-73.85744447700002, 40.6969129849999], [-73.85657185199995, 40.69706469199983], [-73.85567421900002, 40.69717935799988], [-73.85478796099983, 40.69729662299988], [-73.85283544399991, 40.69754896199987], [-73.85277238699999, 40.69755711199987], [-73.85262948599983, 40.697652077999926], [-73.8526096609999, 40.69780503999987], [-73.85260789899989, 40.697959664999885], [-73.85262446500002, 40.698114106999896], [-73.85265915099988, 40.698266471999915], [-73.8527113089999, 40.69841496199995], [-73.85328620599985, 40.69928735699989], [-73.85377512599996, 40.6999095679999], [-73.85429389599985, 40.700445760999926], [-73.85443455699989, 40.70060295199994], [-73.85410116499996, 40.70051374799992], [-73.85070842300001, 40.70020312499988], [-73.84913837299978, 40.69828274099995], [-73.84947763499991, 40.69819580699987], [-73.84894128999987, 40.697556847999905], [-73.84773325500002, 40.69477527999988], [-73.84661793899996, 40.692495182999885], [-73.84596733000002, 40.69116596499991], [-73.8450419489999, 40.68931894699996]]]}}, {\"id\": \"258\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 259, \"Shape_Leng\": 0.126750305191, \"Shape_Area\": 0.000394552487366, \"zone\": \"Woodlawn/Wakefield\", \"LocationID\": 259, \"borough\": \"Bronx\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.851071161919, 40.91037152011096], [-73.85207287919152, 40.90919708288196], [-73.85234744980255, 40.90886284229923], [-73.85304970791424, 40.908035926950646], [-73.85347522366716, 40.90753029351422], [-73.85257767308545, 40.9072001911051], [-73.85228059672555, 40.90708126681254], [-73.85142507791112, 40.90678017523082], [-73.85092059381417, 40.9066164942448], [-73.85040148913299, 40.9072181700019], [-73.8501731754292, 40.90750786582528], [-73.84933641428515, 40.90712531413303], [-73.84863913199048, 40.906806522083095], [-73.84708032907056, 40.906156733267686], [-73.84623270413469, 40.90589867238338], [-73.84538061108482, 40.90565580829525], [-73.84481886992495, 40.90552994983734], [-73.84492921909838, 40.90466300388434], [-73.84489264904629, 40.904190866898055], [-73.84441319516424, 40.90417133002449], [-73.84339559917244, 40.90417112477921], [-73.84237543789729, 40.90416640488811], [-73.84138689269206, 40.90417274176649], [-73.84106272893486, 40.90321411616408], [-73.84102047404488, 40.902814135238714], [-73.84099491992694, 40.90255182499083], [-73.84094803906034, 40.9022894581423], [-73.84087973380167, 40.90202853878415], [-73.84079014185733, 40.90177056122788], [-73.84067966362085, 40.90151703123142], [-73.84054892587938, 40.90126937918311], [-73.84039877912073, 40.901028994013096], [-73.84023030117935, 40.90079713029749], [-73.84004474667755, 40.900574957094115], [-73.83924604720933, 40.89950115402459], [-73.83921744879804, 40.89942651221205], [-73.83919486690272, 40.8993506922155], [-73.83917838505819, 40.899273973025196], [-73.83916806407633, 40.899196636286135], [-73.8391639419338, 40.89911896727798], [-73.83916603380511, 40.89904125127209], [-73.83917433174886, 40.898963774370074], [-73.83918880498874, 40.898886822108956], [-73.83920940088679, 40.898810677061086], [-73.83923604304178, 40.89873561981261], [-73.8396643320591, 40.897885515052764], [-73.83964812806491, 40.89733053517026], [-73.83896625889744, 40.89559320648411], [-73.83839199161751, 40.894067008023995], [-73.83804277292896, 40.89414233144516], [-73.83737956161106, 40.893924741304346], [-73.83645878199995, 40.89151612599989], [-73.83554853999995, 40.88913350399993], [-73.83643359099989, 40.888935663999916], [-73.83732165099995, 40.8887419639999], [-73.83822700999995, 40.8886445249999], [-73.83920549899996, 40.88870221699993], [-73.83939762699993, 40.88872801299994], [-73.839593563, 40.8887395329999], [-73.83979050699996, 40.8887364029999], [-73.83998563399987, 40.88871867199988], [-73.84017616899983, 40.88868679899991], [-73.84105278699985, 40.88855346699988], [-73.84196149599987, 40.88845596699989], [-73.84285880200002, 40.88830687399991], [-73.84313196799998, 40.88826507899989], [-73.8431922509999, 40.88825585499997], [-73.84362775199982, 40.88818922099994], [-73.84367707000004, 40.888181673999924], [-73.84376573899986, 40.88816810699991], [-73.844021975, 40.88812915899988], [-73.84407903899991, 40.88812048499992], [-73.84432835199986, 40.88808258799989], [-73.84437965699982, 40.888074788999845], [-73.84465398699979, 40.888033088999855], [-73.84525554699991, 40.887943469999875], [-73.84572953199992, 40.88787170599993], [-73.84704957999995, 40.8876718159999], [-73.84665457799996, 40.88839061999992], [-73.84645600899995, 40.888748434999854], [-73.84626182299986, 40.88909833799986], [-73.84590568599998, 40.889741368999864], [-73.84632533099996, 40.889877226999886], [-73.84741355299995, 40.89022219799991], [-73.84874153399994, 40.890647714999936], [-73.85040125099985, 40.891205458999885], [-73.85069084099977, 40.89130277099987], [-73.85149873299996, 40.891534796999935], [-73.85182948399994, 40.891629781999924], [-73.85340274699983, 40.89213229999996], [-73.85449169199995, 40.89247851799996], [-73.85525378499982, 40.89272062599986], [-73.85566327299986, 40.8928523939999], [-73.85736281700004, 40.89341143899987], [-73.85695619399986, 40.89416793799992], [-73.859833381, 40.895085226999875], [-73.860710982, 40.89537054399996], [-73.86135927899984, 40.89466231899991], [-73.86176180699982, 40.894791183999914], [-73.86182532400001, 40.89481152099992], [-73.86190465899993, 40.89483740499985], [-73.86209147100006, 40.8948983609999], [-73.86214409299994, 40.89491552999989], [-73.86219003099983, 40.89493051899997], [-73.86287533299995, 40.89515402999991], [-73.86330852999983, 40.89529484899985], [-73.86355915799994, 40.89537701699987], [-73.8640163769999, 40.89552304699988], [-73.86491632499997, 40.895749494999905], [-73.86691694899986, 40.89608268499996], [-73.87006981099996, 40.89659073699993], [-73.87048672599987, 40.896640210999905], [-73.87241034499989, 40.89624128599993], [-73.87480618199984, 40.89572568799984], [-73.87572387299993, 40.89562260099996], [-73.8771197409999, 40.895468663999964], [-73.87643743099987, 40.89687059399993], [-73.87589465499988, 40.89755945899988], [-73.87482041599985, 40.89844543799991], [-73.87388944299983, 40.89896546299988], [-73.87301813899998, 40.89935490499993], [-73.87136373199995, 40.900039151999884], [-73.87010369599989, 40.90074194699991], [-73.86899707599991, 40.90155915499991], [-73.86820270299982, 40.90235538299993], [-73.86816613899995, 40.9024482979999], [-73.86789043706806, 40.90298695407611], [-73.86652124506384, 40.902515930065604], [-73.86565794312916, 40.90224320777016], [-73.86477258258628, 40.902012441936456], [-73.86452327049332, 40.90192428981169], [-73.86452318784423, 40.901924262026625], [-73.86305636216943, 40.901507381475604], [-73.8627985062273, 40.90153552211173], [-73.86266208683459, 40.90145918769716], [-73.86245772613617, 40.90135839675722], [-73.86224439564484, 40.9012677067675], [-73.86202342509037, 40.901187776596416], [-73.8617962597029, 40.90111912264372], [-73.8615643835939, 40.90106212291692], [-73.86134100817148, 40.90111157015065], [-73.86109329963253, 40.9010320751406], [-73.86088138470656, 40.90096407592493], [-73.8607838009884, 40.90093304493069], [-73.86068622789415, 40.900902013901295], [-73.86050354630507, 40.90084609257245], [-73.86027030983603, 40.90077469545771], [-73.86019317316998, 40.900750257856956], [-73.8601070690549, 40.900721982278384], [-73.85946778750369, 40.90051720913338], [-73.85939698623689, 40.90060610923642], [-73.85932418498184, 40.90069752719921], [-73.85907692320508, 40.901007993050335], [-73.85923576184192, 40.9010986128155], [-73.85941984095663, 40.90139312500627], [-73.85938315290944, 40.901540390859196], [-73.85937696512505, 40.90156521808216], [-73.85928099691559, 40.901555501188646], [-73.85920552011949, 40.9015478638293], [-73.8590270417611, 40.90144490987672], [-73.8588640130816, 40.90171169502483], [-73.85928903113775, 40.901869112257906], [-73.85938576811449, 40.901904940946395], [-73.85946389979054, 40.90193387781683], [-73.8594985118269, 40.90210160475055], [-73.85953201388178, 40.90224311106177], [-73.85957882974115, 40.9024408428857], [-73.85926432520931, 40.90266947944823], [-73.85917899583926, 40.9026560784762], [-73.85910498785931, 40.90264445521745], [-73.85903604846267, 40.902583495972216], [-73.85896189921328, 40.90251792785657], [-73.85896181324718, 40.902517844842365], [-73.85872824816724, 40.9022889707851], [-73.85840075376072, 40.90221325589944], [-73.8581600000242, 40.90230571010129], [-73.85816460299762, 40.90259586077945], [-73.85796932314177, 40.902850907843764], [-73.85774747446301, 40.902787546959594], [-73.85755967678821, 40.90291779547928], [-73.85711090489973, 40.90364953284796], [-73.85717176301259, 40.90408206521806], [-73.85699157373413, 40.904584209031626], [-73.85704144142174, 40.9046405672139], [-73.85710151389124, 40.90470845794119], [-73.85718621345484, 40.90469201710322], [-73.85735935217048, 40.90473096099366], [-73.85743457609159, 40.904841543098904], [-73.85717766087977, 40.90528654183071], [-73.85705022716802, 40.905291535878256], [-73.85680038324311, 40.90514224808929], [-73.85673146602299, 40.905101069184795], [-73.85665436861377, 40.90505500013992], [-73.85636081350269, 40.90510215402868], [-73.85601023344621, 40.9053059838135], [-73.85601160852788, 40.9054820832207], [-73.85635545169798, 40.90552811796204], [-73.85642386921508, 40.905544740897305], [-73.85650251730299, 40.905563848898225], [-73.85692398190014, 40.90597417005546], [-73.85681013619275, 40.90615582816297], [-73.85655732522444, 40.90626203242296], [-73.85595622622013, 40.90621850362047], [-73.85588044197486, 40.90624143018391], [-73.85580463797965, 40.90626436406324], [-73.85552697719521, 40.90638025122957], [-73.85513792091523, 40.906866889070514], [-73.85423776693442, 40.907032868127104], [-73.85374103615364, 40.90778791636777], [-73.85383360999785, 40.90797434990978], [-73.85422905350244, 40.90801858927425], [-73.85471078319821, 40.908006309471446], [-73.85475581976394, 40.90808399311854], [-73.85479372768647, 40.908158331842266], [-73.85491109686753, 40.908354869668344], [-73.85458940544117, 40.9089394991491], [-73.853916460476, 40.909465603221335], [-73.8537750789652, 40.909531555120545], [-73.85360061710124, 40.90960674122533], [-73.8533535191553, 40.91004314037642], [-73.85295618460705, 40.910353044089575], [-73.8527699159019, 40.91040600803299], [-73.85258474775196, 40.910310617032046], [-73.85266035896096, 40.910015193079055], [-73.85245653391823, 40.9098605090357], [-73.85227451608235, 40.90985409932151], [-73.85178316975063, 40.91016063082594], [-73.85149242468167, 40.91027164024944], [-73.85144753984513, 40.9104496711014], [-73.851071161919, 40.91037152011096]]]}}, {\"id\": \"259\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 260, \"Shape_Leng\": 0.133514154636, \"Shape_Area\": 0.000422345326907, \"zone\": \"Woodside\", \"LocationID\": 260, \"borough\": \"Queens\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.9017537339999, 40.760775474999946], [-73.90147053099994, 40.759633049999884], [-73.90137640199998, 40.75964203799992], [-73.90127448199993, 40.7596514789999], [-73.89901804799997, 40.7598912129999], [-73.89886718099987, 40.759904206999835], [-73.89877250699988, 40.75991557699992], [-73.89866137799999, 40.759921016999904], [-73.89906072899988, 40.75932801599991], [-73.89941035199989, 40.758808838999904], [-73.89948479499999, 40.75868286799992], [-73.89956743900002, 40.758545262999846], [-73.89960731899997, 40.75845463499991], [-73.89964864399998, 40.75837966699987], [-73.89967747499992, 40.758296863999945], [-73.89971668599982, 40.75816257899985], [-73.89976111400001, 40.757985758999915], [-73.89978921699989, 40.7578984789999], [-73.89980933199992, 40.75778375299994], [-73.89984592699983, 40.75756215299985], [-73.89986047499984, 40.75733746299991], [-73.89985245499989, 40.757111608999885], [-73.89982174199989, 40.75688653699989], [-73.89979518199993, 40.75677539099996], [-73.89976862099998, 40.75666423699991], [-73.89972650399997, 40.756541866999875], [-73.89969372599982, 40.75644660899991], [-73.89959808699994, 40.756235437999926], [-73.89923583599995, 40.75553001699992], [-73.89912259300003, 40.7551692929999], [-73.89905512900002, 40.754949824999905], [-73.89897117099991, 40.754715304999905], [-73.89855812999998, 40.7541370959999], [-73.89826519799995, 40.75366541399989], [-73.89790235299994, 40.75308771499985], [-73.89770744899984, 40.75273485899988], [-73.89762297800004, 40.752523183999905], [-73.89757880899995, 40.75238393499996], [-73.89748654099999, 40.752014347999896], [-73.89733942799997, 40.751419567999875], [-73.89728972099992, 40.750872780999906], [-73.8972114379999, 40.75046637499993], [-73.8970765, 40.75011887599991], [-73.89648281799981, 40.74907123399993], [-73.89624245599988, 40.74871954499987], [-73.89610755799994, 40.7485008479999], [-73.89583033399997, 40.74801363999995], [-73.89566359199999, 40.747695496999896], [-73.89535589999988, 40.74710841299986], [-73.89502251499994, 40.7464722929999], [-73.89496338799982, 40.74637214699989], [-73.89490427299994, 40.746271992999944], [-73.89480782, 40.74606246499995], [-73.89473484699982, 40.74584579899992], [-73.8946865809999, 40.74562431999986], [-73.89466369299987, 40.74540049699994], [-73.89466630799988, 40.74517683999988], [-73.894693961, 40.74495583499987], [-73.89474567599989, 40.74473983299986], [-73.89478949299988, 40.74458478799991], [-73.89484503999992, 40.744431661999876], [-73.8949120899999, 40.74428123099996], [-73.89499026399977, 40.744134212999896], [-73.89524819599988, 40.74381972999992], [-73.89540662199998, 40.74365743999987], [-73.89557809299986, 40.74350275399994], [-73.89576162799989, 40.74335656899989], [-73.89638129299982, 40.74284491999988], [-73.89706891399995, 40.74228451999994], [-73.897207419, 40.74217393599989], [-73.89732904499989, 40.74207636599991], [-73.89762150399993, 40.741841761999865], [-73.89818354199996, 40.74139090199992], [-73.89868042099991, 40.74097933999994], [-73.8988332499998, 40.74085897699991], [-73.89889421999989, 40.740814239999935], [-73.89896997799984, 40.74075331499988], [-73.89908954799988, 40.74065715499993], [-73.89916350300004, 40.74059856999987], [-73.89920950799993, 40.7405454809999], [-73.89974871399977, 40.74013220399991], [-73.90057825399985, 40.73949289599988], [-73.90102306499992, 40.73926666799989], [-73.90174321599993, 40.73890039799996], [-73.90268254099986, 40.73856750299995], [-73.90316328499986, 40.738390687999924], [-73.903575259, 40.73824323399991], [-73.90364752599992, 40.73821736899992], [-73.90369365299995, 40.73819895599996], [-73.90410900599981, 40.738033214999895], [-73.90463288899997, 40.73782416299991], [-73.90486204399993, 40.73772361199986], [-73.90556671799989, 40.737414398999896], [-73.90598452499982, 40.7372186479999], [-73.90646283399997, 40.73702598499988], [-73.90672950499994, 40.73691348499992], [-73.90739546899985, 40.736709318999935], [-73.907619066, 40.73665029899992], [-73.90789283699988, 40.736597387999936], [-73.90890220399997, 40.73644071099989], [-73.90926002999988, 40.736403179999854], [-73.90928542399993, 40.73633405899988], [-73.90930551799993, 40.73628728499989], [-73.90932941199996, 40.73623244499992], [-73.90959716499998, 40.7356564569999], [-73.90991066500006, 40.73499734099988], [-73.91023874099996, 40.73433921699993], [-73.91068876399997, 40.73347108099995], [-73.90628903499987, 40.732400751999876], [-73.9059835329999, 40.73326568399991], [-73.90568645899998, 40.73391582499989], [-73.9060183039999, 40.73401073899992], [-73.90573589799986, 40.73463343599988], [-73.90516948899983, 40.73585964299986], [-73.90470297599984, 40.73567990299986], [-73.90455512499977, 40.73560527299987], [-73.90378660899988, 40.735217315999876], [-73.903367991, 40.73501264999993], [-73.90333476000002, 40.734996402999876], [-73.90286649499984, 40.734767459999915], [-73.90244611599991, 40.73457283799989], [-73.90194324799982, 40.73434002499993], [-73.90107970900003, 40.73391463399987], [-73.9025965079999, 40.73210442799992], [-73.904218336, 40.730125830999945], [-73.905180579, 40.72895657699995], [-73.90573360899982, 40.7285107579999], [-73.90673028099992, 40.72773268299985], [-73.90693049899986, 40.72757602999992], [-73.90706867299988, 40.7274665229999], [-73.90712253099991, 40.727424727999924], [-73.90934522699985, 40.72880316199991], [-73.90981313899988, 40.72907325099989], [-73.91017627699985, 40.72927572499984], [-73.9105504049999, 40.72946621999993], [-73.910962087, 40.72964075299994], [-73.91148836199982, 40.72983170699988], [-73.91174734199988, 40.72991203299991], [-73.91177014899998, 40.72985279599989], [-73.91189388099994, 40.72956051399995], [-73.91200974700003, 40.72922886399989], [-73.91205021299994, 40.72850636299993], [-73.91505218099987, 40.728601179999906], [-73.9150723379999, 40.727970819999896], [-73.91796696699996, 40.7285007709999], [-73.9177303909999, 40.72987934399989], [-73.91760690999989, 40.730571980999855], [-73.91752107599989, 40.7310027049999], [-73.91736640299979, 40.731327577999906], [-73.917342506, 40.731382825999916], [-73.917563071, 40.731446009999885], [-73.91881334600006, 40.73191752199988], [-73.91903825199986, 40.73200596599987], [-73.91902180399987, 40.73208783799993], [-73.91879292099999, 40.73196944899989], [-73.9183510909999, 40.7331525579999], [-73.91759250800001, 40.735239440999834], [-73.91755768599998, 40.73529176299997], [-73.91751347899988, 40.735350984999904], [-73.91689741899991, 40.73544470799989], [-73.91687214200005, 40.73554135899994], [-73.91651522799987, 40.736757345999855], [-73.91717737999979, 40.73689786999995], [-73.91709074600003, 40.737144681999965], [-73.91651190599994, 40.738944757999874], [-73.916301085, 40.73906911799992], [-73.91567469599995, 40.74078492899987], [-73.91538439399999, 40.74072944299994], [-73.91443483199988, 40.74053105299986], [-73.91382781999981, 40.74241368799992], [-73.9137938869998, 40.742514491999934], [-73.91289084199997, 40.74248224999995], [-73.9126012949999, 40.74244224599993], [-73.9119412979999, 40.742382948999925], [-73.91193129999984, 40.74243121799987], [-73.91190626499984, 40.74255201199992], [-73.91150870799994, 40.74447070499985], [-73.9112472229998, 40.745783992999876], [-73.91213076499986, 40.74588664299989], [-73.9124485379999, 40.74592040199985], [-73.91235352599999, 40.74631819599992], [-73.91232274099995, 40.746447085999904], [-73.91212028799991, 40.74729474899992], [-73.91193027699991, 40.747988209999924], [-73.91190987499982, 40.74810505399993], [-73.91190983999999, 40.74810522499993], [-73.91190979099993, 40.748105394999875], [-73.91187504099992, 40.74822079599992], [-73.91182635699991, 40.748333131999914], [-73.91182624999982, 40.748333392999925], [-73.91182610699991, 40.74833364499993], [-73.91182081399998, 40.74834287299993], [-73.91176417899987, 40.748441596999946], [-73.91164202799983, 40.74868457599991], [-73.91015860699994, 40.7485019679999], [-73.90989961499994, 40.749087274999944], [-73.90971148199992, 40.749754708999944], [-73.90951622599984, 40.75044165999988], [-73.90948267299984, 40.75056086099988], [-73.90945183999995, 40.75065545399988], [-73.90945182799983, 40.75065550699994], [-73.90944332099993, 40.7507550329999], [-73.90944330899997, 40.7507551319999], [-73.90944331999988, 40.750755231999925], [-73.9094586379999, 40.75085507399991], [-73.9094586489999, 40.75085514499993], [-73.90945867299993, 40.75085521699992], [-73.90949707399992, 40.750950583999895], [-73.90949709799997, 40.75095063899988], [-73.90949713399976, 40.75095068299986], [-73.90955575999992, 40.75103731299989], [-73.90984318599997, 40.751638679999886], [-73.91035493000007, 40.75274990799994], [-73.91067474899997, 40.752995076999895], [-73.91023124699991, 40.75297652299985], [-73.90995439699995, 40.753217822999936], [-73.90969314899992, 40.75346973299989], [-73.90944851199998, 40.75373135399991], [-73.90922902699994, 40.75399264199997], [-73.90922139799983, 40.754001722999895], [-73.90917451199998, 40.75406416699992], [-73.90901259099992, 40.75427981499996], [-73.90879316499999, 40.75468561599994], [-73.90876818500003, 40.754731814999936], [-73.908736451, 40.75479049999993], [-73.90871371100005, 40.7548325549999], [-73.90868067299994, 40.75489365399992], [-73.90865671500002, 40.75493796099989], [-73.90862474299999, 40.75499709099988], [-73.90860265599999, 40.7550379379999], [-73.90854932099997, 40.75513657099991], [-73.90853125399997, 40.75516998399991], [-73.908509019, 40.75521110499989], [-73.90846544599992, 40.75529168599991], [-73.90840546799983, 40.755452399999896], [-73.90835663299995, 40.7555832579999], [-73.90830736999996, 40.7557152619999], [-73.90827537899996, 40.7558009829999], [-73.90824210899991, 40.75589012999991], [-73.90820623799995, 40.75598624799986], [-73.90817188199993, 40.75607830499995], [-73.90813601099995, 40.75617442399993], [-73.90810013899984, 40.75627054199987], [-73.90806081699996, 40.75637590799995], [-73.90802408399996, 40.75647433199986], [-73.90798755699986, 40.756572205999916], [-73.9079488889999, 40.75667581499994], [-73.90782817499984, 40.75699926499987], [-73.9074261499998, 40.75810005299993], [-73.90705781899989, 40.75867814499987], [-73.90649338799987, 40.75956398999992], [-73.90631505099988, 40.75984387599992], [-73.90699741899994, 40.76014299099993], [-73.90779405499987, 40.760517799999896], [-73.90753042700004, 40.7608446779999], [-73.907419729, 40.76098192599993], [-73.90703976299993, 40.76145304399993], [-73.90677808699982, 40.7617774819999], [-73.90651032099997, 40.76210946999992], [-73.90625691699984, 40.76242520699992], [-73.90606811099998, 40.7626604449999], [-73.90566075299986, 40.7631679879999], [-73.90519286699994, 40.763750940999934], [-73.90451229800004, 40.764589063999914], [-73.90370771199999, 40.765597148999916], [-73.90361058199981, 40.76572657799992], [-73.90356915599997, 40.765781788999924], [-73.90331045299985, 40.76516814899989], [-73.90299205599985, 40.76441183899989], [-73.90276793600005, 40.76387945699989], [-73.90225241299993, 40.762643825999895], [-73.9017537339999, 40.760775474999946]]]}}, {\"id\": \"260\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 261, \"Shape_Leng\": 0.0271204563616, \"Shape_Area\": 3.43423231652e-05, \"zone\": \"World Trade Center\", \"LocationID\": 261, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-74.01332610899989, 40.7050307879999], [-74.01326979499997, 40.704553495999974], [-74.01419226999991, 40.70457040399989], [-74.01428922899998, 40.70454907499989], [-74.01437031200001, 40.70455803799994], [-74.01451712299998, 40.704605674999854], [-74.01466709799998, 40.70464721299988], [-74.01478059499996, 40.70467411299991], [-74.01489542099996, 40.7046975179999], [-74.01496125199999, 40.704759797999884], [-74.01516257, 40.704805094999884], [-74.01527219799995, 40.70483003699996], [-74.01539758699995, 40.70480014899991], [-74.01565756599994, 40.70483308799993], [-74.01565582200001, 40.70488263499986], [-74.0158739609999, 40.70488211299989], [-74.01621736899986, 40.7048530269999], [-74.01629888499991, 40.70484641499997], [-74.01637519799986, 40.704838484999925], [-74.01652040399995, 40.70479971799988], [-74.01662761199991, 40.704799430999856], [-74.01668613099999, 40.70479933799992], [-74.01665417299996, 40.7049638209999], [-74.016622178, 40.70509448299991], [-74.01640305499988, 40.70566620199989], [-74.0162515959999, 40.70607261399987], [-74.0160596679999, 40.70670512999991], [-74.01593869399987, 40.70700757399988], [-74.01590498299998, 40.70709269499987], [-74.01588574099995, 40.70714217599992], [-74.01586395999995, 40.707200182999905], [-74.01584985499994, 40.70724726799987], [-74.01571815199996, 40.70758531099992], [-74.01568829399997, 40.70766382499986], [-74.015643777, 40.707756098999894], [-74.01559220199994, 40.70788324299986], [-74.01534330799996, 40.70849681699988], [-74.01495415399995, 40.70943269199994], [-74.01487257099997, 40.70968487399991], [-74.01482432799993, 40.70982336899991], [-74.01466507099994, 40.71042693799992], [-74.01445559399995, 40.71090715299986], [-74.01426893599995, 40.711467815999924], [-74.01417753399997, 40.71188266599996], [-74.01404744599986, 40.71243781499994], [-74.01375388499997, 40.71369055499987], [-74.01359729999992, 40.7136233779999], [-74.01254378499996, 40.71317910399991], [-74.01166334799997, 40.71279805599988], [-74.01012167499995, 40.71208469599988], [-74.00883879499997, 40.711503958999906], [-74.00860085399992, 40.711392239999945], [-74.00906264699988, 40.71088670399993], [-74.00947357899996, 40.710412542999954], [-74.00954277299995, 40.71032749299992], [-74.00998739099994, 40.70980567299989], [-74.01054246399991, 40.70915414899993], [-74.01084638199993, 40.70879754499991], [-74.01101198499991, 40.70859402199989], [-74.01118228399991, 40.70838557399989], [-74.01124834999997, 40.708304700999854], [-74.01170249899995, 40.707755712999834], [-74.01194691299999, 40.70745153199993], [-74.01250773999996, 40.706767066999916], [-74.01325396399993, 40.705870513999905], [-74.01330942899993, 40.70570683099988], [-74.01334484799996, 40.7055388269999], [-74.01335946199991, 40.705368655999926], [-74.01335309799998, 40.70519857699984], [-74.01332610899989, 40.7050307879999]]]}}, {\"id\": \"261\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 262, \"Shape_Leng\": 0.0490636231541, \"Shape_Area\": 0.000122330270966, \"zone\": \"Yorkville East\", \"LocationID\": 262, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"MultiPolygon\", \"coordinates\": [[[[-73.94383256699986, 40.78285908899991], [-73.9437600529999, 40.78282893199988], [-73.94360760400002, 40.782865369999904], [-73.94354420706223, 40.78288052416259], [-73.94364823539011, 40.782656161333264], [-73.94388068102414, 40.7815320438669], [-73.94387075988702, 40.781273026571576], [-73.94371468850554, 40.780629815528805], [-73.94360047475412, 40.78015909946612], [-73.94321386265217, 40.779317588660014], [-73.94300423950462, 40.779639495474214], [-73.94278066641961, 40.77950615406811], [-73.94271600545085, 40.77954416947613], [-73.94262799584376, 40.77932311896862], [-73.94271237476224, 40.77921485693976], [-73.94253556320854, 40.77909095606247], [-73.94289340818789, 40.7786140932462], [-73.94265530676857, 40.778124965573824], [-73.94245740990823, 40.777407473347616], [-73.94243848174499, 40.777315235765926], [-73.94232881653723, 40.77724265564125], [-73.9423613254944, 40.777164737886814], [-73.942302455358, 40.777144704412], [-73.94224790662773, 40.77711854006399], [-73.94219877702626, 40.77708677136877], [-73.94215605521731, 40.77705003763204], [-73.9421206009128, 40.777009078074094], [-73.94209312757077, 40.77696471695306], [-73.94207418803879, 40.77691784697707], [-73.94200266722274, 40.776185317382726], [-73.94208160457381, 40.775957016317044], [-73.94240100949044, 40.77572692752608], [-73.9425876964068, 40.7752189565281], [-73.94293043781381, 40.774676268035776], [-73.94333894750484, 40.77421119259063], [-73.94366431814754, 40.77384076467938], [-73.94396084370824, 40.773530420594504], [-73.94442433595846, 40.773045322515856], [-73.94487889596702, 40.772569562007305], [-73.94561514012511, 40.771943026587664], [-73.94640418919063, 40.77126904561923], [-73.94690764133769, 40.7707339117235], [-73.9474897547817, 40.77011515400953], [-73.94752257899997, 40.770129694999895], [-73.94761452999984, 40.77017042899995], [-73.94766538399982, 40.77019325299993], [-73.94771625099996, 40.77021606799992], [-73.94779385500001, 40.77024915199992], [-73.94807387899996, 40.77039439499987], [-73.95043560599999, 40.77139110299989], [-73.95268514999995, 40.772339404999926], [-73.95218621999996, 40.7730198449999], [-73.95181761599994, 40.77352690399989], [-73.95172338299997, 40.77365653199994], [-73.9512617339999, 40.774291940999944], [-73.950798347, 40.774927767999884], [-73.95033856500001, 40.77556320899991], [-73.94987249699994, 40.77619615099996], [-73.9493736669999, 40.776875747999945], [-73.9488804199999, 40.777553284999904], [-73.94842272599978, 40.77818264099987], [-73.94796768199993, 40.778806465999914], [-73.94750818099986, 40.77943712299992], [-73.9470492189998, 40.78006434699995], [-73.94659059199999, 40.78069197399986], [-73.94647764199985, 40.78084530299988], [-73.94642443299978, 40.78091753399989], [-73.94630631700001, 40.781077875999856], [-73.946197187, 40.78122601799986], [-73.94613091699998, 40.78131597799994], [-73.94588094499986, 40.78165816599989], [-73.94573785299986, 40.78185404199995], [-73.94566981599996, 40.781947175999925], [-73.94536565199984, 40.78236245099988], [-73.94525035499998, 40.78251986399991], [-73.94521308599991, 40.782570746999944], [-73.94472264499986, 40.78324757199987], [-73.94464666000002, 40.78321787299997], [-73.94405439399999, 40.782964357999894], [-73.94395460199989, 40.78291551999991], [-73.94389324599985, 40.78288772699988], [-73.94383256699986, 40.78285908899991]]], [[[-73.93804640603422, 40.78082954427547], [-73.93790133623952, 40.780749996172545], [-73.93779448726416, 40.78078125875544], [-73.93779944179558, 40.781031339796634], [-73.93764631821479, 40.78100121589284], [-73.93765365963804, 40.78079000755279], [-73.93759894622617, 40.78046784086141], [-73.93779269036344, 40.78035790421227], [-73.93823773509392, 40.7802590794654], [-73.9383962259169, 40.78026243389376], [-73.93861200821945, 40.78033895586891], [-73.93923746178228, 40.77973339405837], [-73.939318219876, 40.779718001826865], [-73.93953015793255, 40.779527620260424], [-73.93958378972465, 40.779576474007136], [-73.93943238100762, 40.779812200274584], [-73.93949329752046, 40.779873691315366], [-73.93937515379653, 40.77997877184918], [-73.9393052849678, 40.7799512245521], [-73.93921789625274, 40.78001352022147], [-73.9387280292659, 40.780510743704596], [-73.93883951845037, 40.78059924979984], [-73.93882863410155, 40.78076345072766], [-73.9387442308927, 40.78104387604228], [-73.93830097951646, 40.781362084364964], [-73.9380821326871, 40.781418437777184], [-73.93796757721641, 40.781289334060304], [-73.93804948921758, 40.78105580604241], [-73.93806723089875, 40.78094470320065], [-73.93804640603422, 40.78082954427547]]]]}}, {\"id\": \"262\", \"type\": \"Feature\", \"properties\": {\"OBJECTID\": 263, \"Shape_Leng\": 0.0370166252994, \"Shape_Area\": 6.57697664169e-05, \"zone\": \"Yorkville West\", \"LocationID\": 263, \"borough\": \"Manhattan\"}, \"geometry\": {\"type\": \"Polygon\", \"coordinates\": [[[-73.95218621999996, 40.7730198449999], [-73.95268514999995, 40.772339404999926], [-73.95505481599999, 40.773336850999954], [-73.95729409999986, 40.7742835549999], [-73.95679749099989, 40.774966310999936], [-73.95633492299991, 40.775600782999916], [-73.95586933699991, 40.776236225999845], [-73.9554072979999, 40.776868479999926], [-73.9549452009999, 40.7775039469999], [-73.95448224099997, 40.77813866499986], [-73.9539837019998, 40.77882210999991], [-73.95349212400005, 40.77949662199995], [-73.95303367999988, 40.78012545499991], [-73.95257421900006, 40.780753033999865], [-73.9521220329999, 40.78138670999992], [-73.95165920699998, 40.78200767399993], [-73.95119927299987, 40.7826333919999], [-73.95073913799986, 40.783261709999884], [-73.95028254899988, 40.78389046999996], [-73.94982390399991, 40.78451557099987], [-73.94933170599992, 40.78519312699994], [-73.94706938199998, 40.78423622699993], [-73.94472264499986, 40.78324757199987], [-73.94521308599991, 40.782570746999944], [-73.94525035499998, 40.78251986399991], [-73.94536565199984, 40.78236245099988], [-73.94566981599996, 40.781947175999925], [-73.94573785299986, 40.78185404199995], [-73.94588094499986, 40.78165816599989], [-73.94613091699998, 40.78131597799994], [-73.946197187, 40.78122601799986], [-73.94630631700001, 40.781077875999856], [-73.94642443299978, 40.78091753399989], [-73.94647764199985, 40.78084530299988], [-73.94659059199999, 40.78069197399986], [-73.9470492189998, 40.78006434699995], [-73.94750818099986, 40.77943712299992], [-73.94796768199993, 40.778806465999914], [-73.94842272599978, 40.77818264099987], [-73.9488804199999, 40.777553284999904], [-73.9493736669999, 40.776875747999945], [-73.94987249699994, 40.77619615099996], [-73.95033856500001, 40.77556320899991], [-73.950798347, 40.774927767999884], [-73.9512617339999, 40.774291940999944], [-73.95172338299997, 40.77365653199994], [-73.95181761599994, 40.77352690399989], [-73.95218621999996, 40.7730198449999]]]}}]}"},"id":"18193dc2-af18-430d-8541-849ff6cf74a5","type":"GeoJSONDataSource"},{"attributes":{"plot":{"id":"3058396c-651f-4f74-b4de-a7e737c8665a","subtype":"Figure","type":"Plot"},"ticker":{"id":"7297bf13-8681-427a-88cb-a60cd7da3cc9","type":"BasicTicker"},"visible":false},"id":"5ab58e62-0c50-4e83-a7f6-40cf7b7ce86c","type":"Grid"},{"attributes":{},"id":"cdd73a27-a419-4f2d-be6c-98f427087f04","type":"BasicTicker"},{"attributes":{},"id":"502eda31-9e62-48de-b5af-2bd75bfc8082","type":"BasicTickFormatter"}],"root_ids":["62ba6d47-3ae9-4aa5-be3a-40808ec4ef1a","ae53cc6f-7f97-4812-a208-ab913d90eb10","3058396c-651f-4f74-b4de-a7e737c8665a","9a6525f3-7059-45ab-90ee-2d7e83c0a07a"]},"title":"Bokeh Application","version":"0.12.9"}};
          var render_items = [{"docid":"3eecbd1c-0b14-4777-8f07-6537dfa40e4c","elementid":"deeab81b-7229-4831-8db6-ac0dd99e8ed5","modelid":"ae53cc6f-7f97-4812-a208-ab913d90eb10"},{"docid":"3eecbd1c-0b14-4777-8f07-6537dfa40e4c","elementid":"95089bfe-ef2e-4c17-89f1-9681a14d8107","modelid":"3058396c-651f-4f74-b4de-a7e737c8665a"},{"docid":"3eecbd1c-0b14-4777-8f07-6537dfa40e4c","elementid":"ee11cb3f-5262-44a5-8438-9ecbad37cf86","modelid":"9a6525f3-7059-45ab-90ee-2d7e83c0a07a"}];

          root.Bokeh.embed.embed_items(docs_json, render_items);
        }

        if (root.Bokeh !== undefined) {
          embed_document(root);
        } else {
          var attempts = 0;
          var timer = setInterval(function(root) {
            if (root.Bokeh !== undefined) {
              embed_document(root);
              clearInterval(timer);
            }
            attempts++;
            if (attempts &gt; 100) {
              console.log("Bokeh: ERROR: Unable to embed document because BokehJS library is missing")
              clearInterval(timer);
            }
          }, 10, root)
        }
      })(window);
    });
  };
  if (document.readyState != "loading") fn();
  else document.addEventListener("DOMContentLoaded", fn);
})();

&lt;/script&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/09/21/accelerating-geopandas-1/"/>
    <summary>This work is supported by Anaconda Inc., the Data
Driven Discovery Initiative from the Moore
Foundation, and NASA SBIR
NNX16CG43P</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2017-09-21T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/09/18/pangeo-1/</id>
    <title>Dask on HPC - Initial Work</title>
    <updated>2017-09-18T00: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://anaconda.com"&gt;Anaconda Inc.&lt;/a&gt; and the &lt;a class="reference external" href="https://www.nsf.gov/funding/pgm_summ.jsp?pims_id=504780"&gt;NSF
EarthCube&lt;/a&gt; program.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;We &lt;a class="reference external" href="http://blogs.ei.columbia.edu/2017/09/13/pangeo-project-will-improve-access-to-climate-data/"&gt;recently
announced&lt;/a&gt;
a collaboration between the &lt;a class="reference external" href="https://ncar.ucar.edu/"&gt;National Center for Atmospheric Research
(NCAR)&lt;/a&gt;, &lt;a class="reference external" href="http://www.ldeo.columbia.edu/"&gt;Columbia
University&lt;/a&gt;, and Anaconda Inc to accelerate the
analysis of atmospheric and oceanographic data on high performance computers
(HPC) with XArray and Dask. The &lt;a class="reference external" href="https://figshare.com/articles/Pangeo_NSF_Earthcube_Proposal/5361094"&gt;full
text&lt;/a&gt; of
the proposed work is &lt;a class="reference external" href="https://figshare.com/articles/Pangeo_NSF_Earthcube_Proposal/5361094"&gt;available
here&lt;/a&gt;. We
are very grateful to the NSF EarthCube program for funding this work, which
feels particularly relevant today in the wake (and continued threat) of the
major storms Harvey, Irma, and Jose.&lt;/p&gt;
&lt;p&gt;This is a collaboration of academic scientists (Columbia), infrastructure
stewards (NCAR), and software developers (Anaconda and Columbia and NCAR) to
scale current workflows with XArray and Jupyter onto big-iron HPC systems and
peta-scale datasets. In the first week after the grant closed a few of us
focused on the quickest path to get science groups up and running with XArray,
Dask, and Jupyter on these HPC systems. This blogpost details what we achieved
and some of the new challenges that we’ve found in that first week. We hope to
follow this blogpost with many more to come in the future.
Today we cover the following topics:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Deploying Dask with MPI&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Interactive deployments on a batch job scheduler, in this case PBS&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The virtues of JupyterLab in a remote system&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Network performance and 3GB/s infiniband&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Modernizing XArray’s interactions with Dask’s distributed scheduler&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;A video walkthrough deploying Dask on XArray on an HPC system is available &lt;a class="reference external" href="https://www.youtube.com/watch?v=7i5m78DSr34"&gt;on
YouTube&lt;/a&gt; and instructions for
atmospheric scientists with access to the &lt;a class="reference external" href="https://www2.cisl.ucar.edu/resources/computational-systems/cheyenne"&gt;Cheyenne
Supercomputer&lt;/a&gt;
is available
&lt;a class="reference external" href="https://github.com/pangeo-data/pangeo-discussion/wiki/Getting-Started-with-Dask-on-Cheyenne"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Now lets start with technical issues:&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/2017/09/18/pangeo-1.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 id="deploying-dask-with-mpi"&gt;

&lt;p&gt;HPC systems use job schedulers like SGE, SLURM, PBS, LSF, and others. Dask
has been deployed on all of these systems before either by academic groups or
financial companies. However every time we do this it’s a little different and
generally tailored to a particular cluster.&lt;/p&gt;
&lt;p&gt;We wanted to make something more general. This started out as a &lt;a class="reference external" href="https://github.com/dask/distributed/issues/1260"&gt;GitHub issue
on PBS scripts&lt;/a&gt; that tried to
make a simple common template that people could copy-and-modify.
Unfortunately, there were significant challenges with this. HPC systems and
their job schedulers seem to focus and easily support only two common use
cases:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Embarrassingly parallel “run this script 1000 times” jobs. This is too
simple for what we have to do.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://en.wikipedia.org/wiki/Message_Passing_Interface"&gt;MPI&lt;/a&gt; jobs. This
seemed like overkill, but is the approach that we ended up taking.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Deploying dask is somewhere between these two. It falls into the master-slave
pattern (or perhaps more appropriately coordinator-workers). We ended up
building an &lt;a class="reference external" href="http://mpi4py.readthedocs.io/en/stable/"&gt;MPI4Py&lt;/a&gt; program that
launches Dask. MPI is well supported, and more importantly consistently
supported, by all HPC job schedulers so depending on MPI provides a level of
stability across machines. Now dask.distributed ships with a new &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-mpi&lt;/span&gt;&lt;/code&gt;
executable:&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;mpirun --np 4 dask-mpi
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;To be clear, Dask isn’t using MPI for inter-process communication. It’s still
using TCP. We’re just using MPI to launch a scheduler and several workers and
hook them all together. In pseudocode the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-mpi&lt;/span&gt;&lt;/code&gt; executable looks
something like this:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;mpi4py&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;MPI&lt;/span&gt;
&lt;span class="n"&gt;comm&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;MPI&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;COMM_WORLD&lt;/span&gt;
&lt;span class="n"&gt;rank&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;comm&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Get_rank&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;rank&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;start_dask_scheduler&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="n"&gt;start_dask_worker&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Socially this is useful because &lt;em&gt;every&lt;/em&gt; cluster management team knows how to
support MPI, so anyone with access to such a cluster has someone they can ask
for help. We’ve successfully translated the question “How do I start Dask?” to
the question “How do I run this MPI program?” which is a question that the
technical staff at supercomputer facilities are generally much better equipped
to handle.&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/2017/09/18/pangeo-1.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="working-interactively-on-a-batch-scheduler"&gt;
&lt;h1&gt;Working Interactively on a Batch Scheduler&lt;/h1&gt;
&lt;p&gt;Our collaboration is focused on interactive analysis of big datasets. This
means that people expect to open up Jupyter notebooks, connect to clusters
of many machines, and compute on those machines while they sit at their
computer.&lt;/p&gt;
&lt;img src="/images/pangeo-dask-client.png" width="60%"&gt;
&lt;p&gt;Unfortunately most job schedulers were designed for batch scheduling. They
will try to run your job quickly, but don’t mind waiting for a few hours for a
nice set of machines on the super computer to open up. As you ask for more
time and more machines, waiting times can increase drastically. For most MPI
jobs this is fine because people aren’t expecting to get a result right away
and they’re certainly not interacting with the program, but in our case we
really do want some results right away, even if they’re only part of what we
asked for.&lt;/p&gt;
&lt;p&gt;Handling this problem long term will require both technical work and policy
decisions. In the short term we take advantage of two facts:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Many small jobs can start more quickly than a few large ones. These take
advantage of holes in the schedule that are too small to be used by larger
jobs.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask doesn’t need to be started all at once. Workers can come and go.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;And so I find that if I ask for several single machine jobs I can easily cobble
together a sizable cluster that starts very quickly. In practice this looks
like the following:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;qsub&lt;/span&gt; &lt;span class="n"&gt;start&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sh&lt;/span&gt;      &lt;span class="c1"&gt;# only ask for one machine&lt;/span&gt;
&lt;span class="n"&gt;qsub&lt;/span&gt; &lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;one&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sh&lt;/span&gt;  &lt;span class="c1"&gt;# ask for one more machine&lt;/span&gt;
&lt;span class="n"&gt;qsub&lt;/span&gt; &lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;one&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sh&lt;/span&gt;  &lt;span class="c1"&gt;# ask for one more machine&lt;/span&gt;
&lt;span class="n"&gt;qsub&lt;/span&gt; &lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;one&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sh&lt;/span&gt;  &lt;span class="c1"&gt;# ask for one more machine&lt;/span&gt;
&lt;span class="n"&gt;qsub&lt;/span&gt; &lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;one&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sh&lt;/span&gt;  &lt;span class="c1"&gt;# ask for one more machine&lt;/span&gt;
&lt;span class="n"&gt;qsub&lt;/span&gt; &lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;one&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sh&lt;/span&gt;  &lt;span class="c1"&gt;# ask for one more machine&lt;/span&gt;
&lt;span class="n"&gt;qsub&lt;/span&gt; &lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;one&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sh&lt;/span&gt;  &lt;span class="c1"&gt;# ask for one more machine&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Our main job has a wall time of about an hour. The workers have shorter wall
times. They can come and go as needed throughout the computation as our
computational needs change.&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/2017/09/18/pangeo-1.md&lt;/span&gt;, line 146)&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="jupyter-lab-and-web-frontends"&gt;
&lt;h1&gt;Jupyter Lab and Web Frontends&lt;/h1&gt;
&lt;p&gt;Our scientific collaborators enjoy building Jupyter notebooks of their work.
This allows them to manage their code, scientific thoughts, and visual outputs
all at once and for them serves as an artifact that they can share with their
scientific teams and collaborators. To help them with this we start a Jupyter
server on the same machine in their allocation that is running the Dask
scheduler. We then provide them with SSH-tunneling lines that they can
copy-and-paste to get access to the Jupyter server from their personal
computer.&lt;/p&gt;
&lt;p&gt;We’ve been using the new Jupyter Lab rather than the classic notebook. This is
especially convenient for us because it provides much of the interactive
experience that they lost by not working on their local machine. They get a
file browser, terminals, easy visualization of textfiles and so on without
having to repeatedly SSH into the HPC system. We get all of this functionality
on a single connection and with an intuitive Jupyter interface.&lt;/p&gt;
&lt;p&gt;For now we give them a script to set all of this up. It starts Jupyter Lab
using Dask and then prints out the SSH-tunneling line.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;scheduler_file&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scheduler.json&amp;#39;&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;socket&lt;/span&gt;
&lt;span class="n"&gt;host&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;run_on_scheduler&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;socket&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;gethostname&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;start_jlab&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dask_scheduler&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;subprocess&lt;/span&gt;
    &lt;span class="n"&gt;proc&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;subprocess&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Popen&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;jupyter&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;lab&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;--ip&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;host&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;--no-browser&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
    &lt;span class="n"&gt;dask_scheduler&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;jlab_proc&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;proc&lt;/span&gt;

&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;run_on_scheduler&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;start_jlab&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;ssh -N -L 8787:&lt;/span&gt;&lt;span class="si"&gt;%s&lt;/span&gt;&lt;span class="s2"&gt;:8787 -L 8888:&lt;/span&gt;&lt;span class="si"&gt;%s&lt;/span&gt;&lt;span class="s2"&gt;:8888 -L 8789:&lt;/span&gt;&lt;span class="si"&gt;%s&lt;/span&gt;&lt;span class="s2"&gt;:8789 cheyenne.ucar.edu&amp;quot;&lt;/span&gt; &lt;span class="o"&gt;%&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;host&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;host&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;host&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Long term we would like to switch to an entirely point-and-click interface
(perhaps something like JupyterHub) but this will requires additional thinking
about deploying distributed resources along with the Jupyter server instance.&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/2017/09/18/pangeo-1.md&lt;/span&gt;, line 188)&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="network-performance-on-infiniband"&gt;
&lt;h1&gt;Network Performance on Infiniband&lt;/h1&gt;
&lt;p&gt;The intended computations move several terabytes across the cluster.
On this cluster Dask gets about 1GB/s simultaneous read/write network bandwidth
per machine using the high-speed Infiniband network. For any commodity or
cloud-based system this is &lt;em&gt;very fast&lt;/em&gt; (about 10x faster than what I observe on
Amazon). However for a super-computer this is only about 30% of what’s
possible (see &lt;a class="reference external" href="https://www2.cisl.ucar.edu/resources/computational-systems/cheyenne"&gt;hardware specs&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;I suspect that this is due to byte-handling in Tornado, the networking library
that Dask uses under the hood. The following image shows the diagnostic
dashboard for one worker after a communication-heavy workload. We see 1GB/s
for both read and write. We also see 100% CPU usage.&lt;/p&gt;
&lt;p&gt;&lt;a href="/images/pangeo-network.png"&gt;&lt;img src="/images/pangeo-network.png" width="70%"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Network performance is a big question for HPC users looking at Dask. If we can
get near MPI bandwidth then that may help to reduce concerns for this
performance-oriented community.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/pangeo-data/pangeo-discussion/issues/6"&gt;Github issue for this project&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/tornadoweb/tornado/issues/2147"&gt;Github issue for Tornado&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;a class="reference external" href="https://stackoverflow.com/questions/43881157/how-do-i-use-an-infiniband-network-with-dask"&gt;&lt;em&gt;How do I use Infiniband network with Dask?&lt;/em&gt;&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/2017/09/18/pangeo-1.md&lt;/span&gt;, line 213)&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="xarray-and-dask-distributed"&gt;
&lt;h1&gt;XArray and Dask.distributed&lt;/h1&gt;
&lt;p&gt;XArray was the first major project to use Dask internally. This early
integration was critical to prove out Dask’s internals with user feedback.
However it also means that some parts of XArray were designed well before some
of the newer parts of Dask, notably the asynchronous distributed scheduling
features.&lt;/p&gt;
&lt;p&gt;XArray can still use Dask on a distributed cluster, but only with the subset of
features that are also available with the single machine scheduler. This means
that persisting data in distributed RAM, parallel debugging, publishing shared
datasets, and so on all require significantly more work today with XArray than
they should.&lt;/p&gt;
&lt;p&gt;To address this we plan to update XArray to follow a newly proposed &lt;a class="reference external" href="https://github.com/dask/dask/pull/1068#issuecomment-326591640"&gt;Dask
interface&lt;/a&gt;.
This is complex enough to handle all Dask scheduling features, but light weight
enough not to actually require any dependence on the Dask library itself.
(Work by &lt;a class="reference external" href="http://jcrist.github.io/"&gt;Jim Crist&lt;/a&gt;.)&lt;/p&gt;
&lt;p&gt;We will also eventually need to look at reducing overhead for inspecting
several NetCDF files, but we haven’t yet run into this, so I plan to wait.&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/2017/09/18/pangeo-1.md&lt;/span&gt;, line 236)&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;We think we’re at a decent point for scientific users to start playing with the
system. We have a &lt;a class="reference external" href="https://github.com/pangeo-data/pangeo-discussion/wiki/Getting-Started-with-Dask-on-Cheyenne"&gt;Getting Started with Dask on Cheyenne&lt;/a&gt;
wiki page that our first set of guinea pig users have successfully run through
without much trouble. We’ve also identified a number of issues that the
software developers can work on while the scientific teams spin up.&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/tornadoweb/tornado/issues/2147"&gt;Zero copy Tornado writes&lt;/a&gt; to &lt;a class="reference external" href="https://github.com/pangeo-data/pangeo-discussion/issues/6"&gt;improve network bandwidth&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/pangeo-data/pangeo-discussion/issues/5"&gt;Enable Dask.distributed features in XArray&lt;/a&gt; by &lt;a class="reference external" href="https://github.com/dask/dask/pull/1068"&gt;formalizing dask’s expected interface&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/pangeo-data/pangeo-discussion/issues/8"&gt;Dynamic deployments on batch job schedulers&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We would love to engage other collaborators throughout this process. If you or
your group work on related problems we would love to hear from you. This grant
isn’t just about serving the scientific needs of researchers at Columbia and
NCAR, but about building long-term systems that can benefit the entire
atmospheric and oceanographic community. Please engage on the
&lt;a class="reference external" href="https://github.com/pangeo-data/pangeo-discussion/issues"&gt;Pangeo GitHub issue tracker&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/09/18/pangeo-1/"/>
    <summary>This work is supported by Anaconda Inc. and the NSF
EarthCube program.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="pangeo" label="pangeo"/>
    <category term="scipy" label="scipy"/>
    <published>2017-09-18T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/08/30/dask-0.15.2/</id>
    <title>Dask Release 0.15.2</title>
    <updated>2017-08-30T00: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://anaconda.com"&gt;Anaconda Inc.&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;I’m pleased to announce the release of Dask version 0.15.2. This release
contains stability enhancements and bug fixes. This blogpost outlines
notable changes since the 0.15.0 release on June 11th.&lt;/p&gt;
&lt;p&gt;You can conda install Dask:&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 pip install from PyPI&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[complete] --upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Conda packages are available both on the defaults and conda-forge channels.&lt;/p&gt;
&lt;p&gt;Full changelogs are available here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/blob/master/docs/source/changelog.rst"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/docs/source/changelog.rst"&gt;dask/distributed&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Some notable changes follow.&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/2017/08/30/dask-0.15.2.md&lt;/span&gt;, line 33)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="new-dask-core-and-dask-conda-packages"&gt;

&lt;p&gt;On conda there are now three relevant Dask packages:&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;dask-core&lt;/span&gt;&lt;/code&gt;: Package that includes only the core Dask package. This has no
dependencies other than the standard library. This is primarily intended
for down-stream libraries that depend on certain parts of Dask.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed&lt;/span&gt;&lt;/code&gt;: Dask’s distributed scheduler, depends on Tornado,
cloudpickle, and other libraries.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt;: Metapackage that includes dask-core, distributed, and all relevant
libraries like NumPy, Pandas, Bokeh, etc.. This is intended for users to
install&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This organization is designed to both allow downstream libraries to only
depend on the parts of Dask that they need while also making the default
behavior for users all-inclusive.&lt;/p&gt;
&lt;p&gt;Downstream libraries may want to change conda dependencies from dask to
dask-core. They will then need to be careful to include the necessary
libraries (like numpy or cloudpickle) based on their user community.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2017/08/30/dask-0.15.2.md&lt;/span&gt;, line 54)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="improved-deployment"&gt;
&lt;h1&gt;Improved Deployment&lt;/h1&gt;
&lt;p&gt;Due to increased deployment on Docker or other systems with complex networking
rules &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-worker&lt;/span&gt;&lt;/code&gt; processes now include separate &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;--contact-address&lt;/span&gt;&lt;/code&gt; and
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;--listen-address&lt;/span&gt;&lt;/code&gt; keywords that can be used to specify addresses that they
advertise and addresses on which they listen. This is especially helpful when
the perspective of ones network can shift dramatically.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;worker&lt;/span&gt; &lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;address&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;8786&lt;/span&gt; \
            &lt;span class="o"&gt;--&lt;/span&gt;&lt;span class="n"&gt;contact&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;address&lt;/span&gt; &lt;span class="mf"&gt;192.168.0.100&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;9000&lt;/span&gt;  &lt;span class="c1"&gt;# contact me at 192.168.0.100:9000&lt;/span&gt;
            &lt;span class="o"&gt;--&lt;/span&gt;&lt;span class="n"&gt;listen&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;address&lt;/span&gt; &lt;span class="mf"&gt;172.142.0.100&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;9000&lt;/span&gt;  &lt;span class="c1"&gt;# I listen on this host&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Additionally other services like the HTTP and Bokeh servers now respect the
hosts provided by &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;--listen-address&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;--host&lt;/span&gt;&lt;/code&gt; keywords and will not be
visible outside of the specified network.&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/2017/08/30/dask-0.15.2.md&lt;/span&gt;, line 72)&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-memory-file-descriptor-and-process-leaks"&gt;
&lt;h1&gt;Avoid memory, file descriptor, and process leaks&lt;/h1&gt;
&lt;p&gt;There were a few occasions where Dask would leak resources in complex
situations. Many of these have now been cleaned up. We’re grateful to all
those who were able to provide very detailed case studies that demonstrated
these issues and even more grateful to those who participated in resolving
them.&lt;/p&gt;
&lt;p&gt;There is undoubtedly more work to do here and we look forward to future
collaboration.&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/2017/08/30/dask-0.15.2.md&lt;/span&gt;, line 83)&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="array-and-dataframe-apis"&gt;
&lt;h1&gt;Array and DataFrame APIs&lt;/h1&gt;
&lt;p&gt;As usual, Dask array and dataframe have a new set of functions that fill out
their API relative to NumPy and Pandas.&lt;/p&gt;
&lt;p&gt;See the full APIs for further reference:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask.pydata.org/en/latest/array-api.html"&gt;http://dask.pydata.org/en/latest/array-api.html&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask.pydata.org/en/latest/dataframe-api.html"&gt;http://dask.pydata.org/en/latest/dataframe-api.html&lt;/a&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/2017/08/30/dask-0.15.2.md&lt;/span&gt;, line 93)&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="deprecations"&gt;
&lt;h1&gt;Deprecations&lt;/h1&gt;
&lt;p&gt;Officially deprecated dask.distributed.Executor, users should use dask.distributed.Client
instead. Previously this was set to an alias.&lt;/p&gt;
&lt;p&gt;Removed &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Bag.concat&lt;/span&gt;&lt;/code&gt;, users should use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Bag.flatten&lt;/span&gt;&lt;/code&gt; instead.&lt;/p&gt;
&lt;p&gt;Removed magic tuple unpacking in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Bag.map&lt;/span&gt;&lt;/code&gt; like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;bag.map(lambda&lt;/span&gt; &lt;span class="pre"&gt;x,&lt;/span&gt; &lt;span class="pre"&gt;y:&lt;/span&gt; &lt;span class="pre"&gt;x&lt;/span&gt; &lt;span class="pre"&gt;+&lt;/span&gt; &lt;span class="pre"&gt;y)&lt;/span&gt;&lt;/code&gt;.
Users should unpack manually instead.&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/2017/08/30/dask-0.15.2.md&lt;/span&gt;, line 103)&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="julia"&gt;
&lt;h1&gt;Julia&lt;/h1&gt;
&lt;p&gt;Developers from the Invenia have been building Julia workers and clients that
operate with the Dask.distributed scheduler. They have been helpful in raising
issues necessary to ensure cross-language support.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Link: &lt;a class="reference external" href="https://github.com/invenia/DaskDistributedDispatcher.jl"&gt;github.com/invenia/DaskDistributedDispatcher.jl&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Relevant issue: &lt;a class="reference external" href="https://github.com/dask/distributed/issues/586"&gt;dask/distribugted #586&lt;/a&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/2017/08/30/dask-0.15.2.md&lt;/span&gt;, line 112)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;The following people contributed to the dask/dask repository since the 0.15.0
release on June 11th&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Bogdan&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Elliott Sales de Andrade&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bruce Merry&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Erik Welch&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Fabian Keller&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Bourbeau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jeff Reback&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John A Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Luke Canavan&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Mark Dunne&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Olivier Grisel&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Søren Fuglede Jørgensen&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stephan Hoyer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Yu Feng&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The following people contributed to the dask/distributed repository since the
1.17.1 release on June 14th:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dan Brown&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Elliott Sales de Andrade&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Eric Davies&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Erik Welch&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Evan Welch&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John A Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;James Bourbeau&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jeremiah Lowin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Julius Neuffer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Paul Anton Letnes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Peter Waller&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Sohaib Iftikhar&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Additionally we’re happy to announce that John Kirkham
(&lt;a class="reference external" href="http://github.com/jakirkham"&gt;&amp;#64;jakirkham&lt;/a&gt;) has accepted commit rights to the
Dask organization and become a core contributor. John has been active through
the Dask project, and particularly active in Dask.array.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/08/30/dask-0.15.2/"/>
    <summary>This work is supported by Anaconda Inc.
and the Data Driven Discovery Initiative from the Moore
Foundation.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2017-08-30T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/07/03/scaling/</id>
    <title>Dask Benchmarks</title>
    <updated>2017-07-03T00:00:00+00:00</updated>
    <content type="html">&lt;!-- markdownlint-disable-file --&gt;
&lt;link href="https://cdn.pydata.org/bokeh/release/bokeh-0.12.6.min.css"
      rel="stylesheet" type="text/css"&gt;
&lt;link href="https://cdn.pydata.org/bokeh/release/bokeh-widgets-0.12.6.min.css"
      rel="stylesheet" type="text/css"&gt;
&lt;script src="https://cdn.pydata.org/bokeh/release/bokeh-0.12.6.min.js"&gt;&lt;/script&gt;
&lt;script src="https://cdn.pydata.org/bokeh/release/bokeh-widgets-0.12.6.min.js"&gt;&lt;/script&gt;
&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="https://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&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/2017/07/03/scaling.md&lt;/span&gt;, line 26)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;We measure the performance of Dask’s distributed scheduler for a variety of
different workloads under increasing scales of both problem and cluster size.
This helps to answer questions about dask’s scalability and also helps to
educate readers on the sorts of computations that scale well.&lt;/p&gt;
&lt;p&gt;We will vary our computations in a few ways to see how they stress performance.
We consider the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Computational and communication patterns&lt;/strong&gt; like embarrassingly parallel, fully
sequential, bulk communication, many-small communication, nearest neighbor,
tree reductions, and dynamic graphs.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Varying &lt;strong&gt;task duration&lt;/strong&gt; ranging from very fast (microsecond) tasks, to
100ms and 1s long tasks. Faster tasks make it harder for the central
scheduler to keep up with the workers.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Varying &lt;strong&gt;cluster size&lt;/strong&gt; from one two-core worker to 256 two-core workers
and varying &lt;strong&gt;dataset size&lt;/strong&gt; which we scale linearly with the number of
workers. This means that we’re measuring &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Scalability#Weak_versus_strong_scaling"&gt;weak scaling&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Varying APIs between &lt;strong&gt;tasks&lt;/strong&gt;, multidimensional &lt;strong&gt;arrays&lt;/strong&gt; and
&lt;strong&gt;dataframes&lt;/strong&gt; all of which have cases in the above categories but depend
on different in-memory computational systems like NumPy or Pandas.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We will start with benchmarks for straight tasks, which are the most flexible
system and also the easiest to understand. This will help us to understand
scaling limits on arrays and dataframes.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: we did not tune our benchmarks or configuration at all for these
experiments. They are well below what is possible, but perhaps representative
of what a beginning user might experience upon setting up a cluster without
expertise or thinking about configuration.&lt;/em&gt;&lt;/p&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;
A Note on Benchmarks and Bias
-----------------------------
&lt;p&gt;&lt;em&gt;you can safely skip this section if you’re in a rush&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;This is a technical document, not a marketing piece. These benchmarks adhere
to the principles laid out in &lt;a class="reference internal" href="#/2017/03/09/biased-benchmarks"&gt;&lt;span class="xref myst"&gt;this
blogpost&lt;/span&gt;&lt;/a&gt; and
attempt to avoid those pitfalls around developer bias. In particular the
following are true:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;We decided on a set of benchmarks before we ran them on a cluster&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We did not improve the software or tweak the benchmarks after seeing the
results.
These were run on the current release of Dask in the wild that was put out
weeks ago, not on a development branch.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The computations were constructed naively, as a novice would write them.
They were not tweaked for extra performance.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The cluster was configured naively, without attention to scale or special
parameters&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We estimate that expert use would result in about a 5-10x scaling improvement
over what we’ll see. We’ll detail how to improve scaling with expert methods
at the bottom of the post.&lt;/p&gt;
&lt;p&gt;All that being said the author of this blogpost is paid to write this software
and so you probably shouldn’t trust him. We invite readers to explore things
independently. All configuration, notebooks, plotting code, and data are
available below:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/martindurant/dask-kubernetes"&gt;dask-kubernetes&lt;/a&gt; for
cluster deployment&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/4c198b13e92f881161ef175810c7f6bc#file-scaling-gcs-ipynb"&gt;Jupyter notebook for benchmarks&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://nbviewer.jupyter.org/urls/gist.githubusercontent.com/mrocklin/4c198b13e92f881161ef175810c7f6bc/raw/2fc5a40805fb306eec2af6573e2e93a9d72883cb/scaling-plots.ipynb"&gt;Jupyter notebook for plots&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://storage.googleapis.com/dask-data/scaling-data.csv"&gt;Benchmark results data on GCS&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;hr&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/2017/07/03/scaling.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="tasks"&gt;
&lt;h1&gt;Tasks&lt;/h1&gt;
&lt;p&gt;We start by benchmarking the task scheduling API. Dask’s task scheduling APIs
are at the heart of the other “big data” APIs (like dataframes). We start with
tasks because they’re the simplest and most raw representation of Dask. Mostly
we’ll run the following functions on integers, but you could fill in any
function here, like a pandas dataframe method or sklearn routine.&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;time&lt;/span&gt;

&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;inc&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="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&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;x&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&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;y&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;slowinc&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;delay&lt;/span&gt;&lt;span class="o"&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;time&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sleep&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;delay&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;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&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;slowadd&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;delay&lt;/span&gt;&lt;span class="o"&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;time&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sleep&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;delay&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;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;y&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;slowsum&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;L&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;delay&lt;/span&gt;&lt;span class="o"&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;time&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sleep&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;delay&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;sum&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;L&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;section id="embarrassingly-parallel-tasks"&gt;
&lt;h2&gt;Embarrassingly Parallel Tasks&lt;/h2&gt;
&lt;p&gt;We run the following code on our cluster and measure how long they take to
complete:&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;futures&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;slowinc&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;4&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;n&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;delay&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="c1"&gt;# 1s delay&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&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="n"&gt;futures&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;slowinc&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;100&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;n_cores&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="c1"&gt;# 100ms delay&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&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="n"&gt;futures&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inc&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="n"&gt;n_cores&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="mi"&gt;200&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;     &lt;span class="c1"&gt;# fast&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img src="/images/embarrassingly-parallel-5-black-on-white.svg"
     width="50%"&gt;&lt;/p&gt;
&lt;p&gt;We see that for fast tasks the system can process around 2000-3000 tasks per
second. This is mostly bound by scheduler and client overhead. Adding more
workers into the system doesn’t give us any more tasks per second. However if
our tasks take any amount of time (like 100ms or 1s) then we see decent
speedups.&lt;/p&gt;
&lt;p&gt;If you switch to &lt;em&gt;linear&lt;/em&gt; scales on the plots, you’ll see that as we get out to
512 cores we start to slow down by about a factor of two. I’m surprised to see
this behavior (hooray benchmarks) because all of Dask’s scheduling decisions
are independent of cluster size. My first guess is that the scheduler may be
being swamped with administrative messages, but we’ll have to dig in a bit
deeper here.&lt;/p&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="36dc768f-23a6-4b1c-8cbe-a0346af1d182"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;/section&gt;
&lt;section id="tree-reduction"&gt;
&lt;h2&gt;Tree Reduction&lt;/h2&gt;
&lt;p&gt;Not all computations are embarrassingly parallel. Many computations have
dependencies between them. Consider a tree reduction, where we combine
neighboring elements until there is only one left. This stresses task
dependencies and small data movement.&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;delayed&lt;/span&gt;

&lt;span class="n"&gt;L&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="mi"&gt;7&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;n&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="k"&gt;while&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;L&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;  &lt;span class="c1"&gt;# while there is more than one element left&lt;/span&gt;
    &lt;span class="c1"&gt;# add neighbors together&lt;/span&gt;
    &lt;span class="n"&gt;L&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;slowadd&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="k"&gt;for&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="ow"&gt;in&lt;/span&gt; &lt;span class="nb"&gt;zip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;L&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;L&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="n"&gt;L&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img src="/images/tree-reduction-black-on-white.svg"
     width="50%"&gt;&lt;/p&gt;
&lt;p&gt;We see similar scaling to the embarrassingly parallel case. Things proceed
linearly until they get to around 3000 tasks per second, at which point they
fall behind linear scaling. Dask doesn’t seem to mind dependencies, even
custom situations like this one.&lt;/p&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="90a82d6b-b090-49df-8900-62ca8188d39a"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;/section&gt;
&lt;section id="nearest-neighbor"&gt;
&lt;h2&gt;Nearest Neighbor&lt;/h2&gt;
&lt;p&gt;Nearest neighbor computations are common in data analysis when you need to share a bit of data between neighboring elements, such as frequently occurs in timeseries computations in dataframes or overlapping image processing in arrays or PDE computations.&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;L&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;n&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;L&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;slowadd&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;L&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="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;L&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;L&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;slowadd&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;L&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="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;L&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;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;L&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img src="/images/nearest-neighbor-black-on-white.svg"
     width="50%"&gt;&lt;/p&gt;
&lt;p&gt;Scaling is similar to the tree reduction case. Interesting dependency
structures don’t incur significant overhead or scaling costs.&lt;/p&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="11288c56-ea8f-45d3-9629-62c82b24eb54"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;/section&gt;
&lt;section id="sequential"&gt;
&lt;h2&gt;Sequential&lt;/h2&gt;
&lt;p&gt;&lt;img src="/images/sequential-black-on-white.svg"
     width="10%" align="right"&gt;&lt;/p&gt;
&lt;p&gt;We consider a computation that isn’t parallel at all, but is instead highly sequential. Increasing the number of workers shouldn’t help here (there is only one thing to do at a time) but this does demonstrate the extra stresses that arise from a large number of workers. Note that we have turned off task fusion for this, so here we’re measuring how many roundtrips can occur between the scheduler and worker every second.&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;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;1&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;100&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;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inc&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So we get something like 100 roundtrips per second, or around 10ms roundtrip
latencies. It turns out that a decent chunk of this cost was due to an
optimization; workers prefer to batch small messages for higher throughput. In
this case that optimization hurts us. Still though, we’re about 2-4x faster
than video frame-rate here (video runs at around 24Hz or 40ms between frames).&lt;/p&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="733be4a9-6378-4112-bcaf-bfb1a4217c9c"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;/section&gt;
&lt;section id="client-in-the-loop"&gt;
&lt;h2&gt;Client in the loop&lt;/h2&gt;
&lt;p&gt;Finally we consider a reduction that consumes whichever futures finish first
and adds them together. This is an example of using client-side logic within
the computation, which is often helpful in complex algorithms. This also
scales a little bit better because there are fewer dependencies to track within
the scheduler. The client takes on a bit of the load.&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.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;as_completed&lt;/span&gt;
&lt;span class="n"&gt;futures&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;slowinc&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="n"&gt;n&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&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;as_completed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;batches&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;batches&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="k"&gt;while&lt;/span&gt; &lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="k"&gt;try&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;batch&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;next&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;batches&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;batch&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="p"&gt;:&lt;/span&gt;
            &lt;span class="n"&gt;batch&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="nb"&gt;next&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;batches&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="k"&gt;except&lt;/span&gt; &lt;span class="ne"&gt;StopIteration&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="k"&gt;break&lt;/span&gt;
    &lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;slowsum&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;batch&lt;/span&gt;&lt;span class="p"&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;add&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="3c6ec7c3-ffd7-4ec9-8ef7-1e8b21a044f5"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;/section&gt;
&lt;section id="tasks-complete"&gt;
&lt;h2&gt;Tasks: Complete&lt;/h2&gt;
&lt;p&gt;We show most of the plots from above for comparison.&lt;/p&gt;
&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="30baf2d4-57bd-4393-8a2c-c9deb1a1edc9"&gt;&lt;/div&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/2017/07/03/scaling.md&lt;/span&gt;, line 320)&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="arrays"&gt;
&lt;h1&gt;Arrays&lt;/h1&gt;
&lt;p&gt;When we combine NumPy arrays with the task scheduling system above we get
dask.array, a distributed multi-dimensional array. This section shows
computations like the last section (maps, reductions, nearest-neighbor), but
now these computations are motivated by actual data-oriented computations and
involve real data movement.&lt;/p&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
### Create Dataset
&lt;p&gt;We make a square array with somewhat random data. This array scales with the
number of cores. We cut it into uniform chunks of size 2000 by 2000.&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;N&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;5000&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;math&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sqrt&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_cores&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;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;randint&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;10000&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="n"&gt;N&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;N&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;2000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2000&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Creating this array is embarrassingly parallel. There is an odd corner in the
graph here that I’m not able to explain.&lt;/p&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="bdce2a6b-8576-4724-8736-24f4b86d6487"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;section id="elementwise-computation"&gt;
&lt;h2&gt;Elementwise Computation&lt;/h2&gt;
&lt;p&gt;We perform some numerical computation element-by-element on this 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;y&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;sin&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt; &lt;span class="mi"&gt;2&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;cos&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is also embarrassingly parallel. Each task here takes around 300ms
(the time it takes to call this on a single 2000 by 2000 numpy array chunk).&lt;/p&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="d919e07f-2795-4bfd-9f97-d0aaf24c0a2a"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;/section&gt;
&lt;section id="reductions"&gt;
&lt;h2&gt;Reductions&lt;/h2&gt;
&lt;p&gt;We sum the array. This is implemented as a tree reduction.&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;std&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="8f38e6fe-fab7-480b-9eb8-0f72781e24d7"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;/section&gt;
&lt;section id="random-access"&gt;
&lt;h2&gt;Random Access&lt;/h2&gt;
&lt;p&gt;We get a single element from the array. This shouldn’t get any faster with
more workers, but it may get slower depending on how much base-line load a
worker adds to the 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="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1234&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;4567&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We get around 400-800 bytes per second, which translates to response times of
10-20ms, about twice the speed of video framerate. We see that performance
does degrade once we have a hundred or so active connections.&lt;/p&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="88e05fb2-abac-4b1e-bcf9-a04d9148a6bf"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;/section&gt;
&lt;section id="communication"&gt;
&lt;h2&gt;Communication&lt;/h2&gt;
&lt;p&gt;We add the array to its transpose. This forces different chunks to move around
the network so that they can add to each other. Roughly half of the array
moves on the network.&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;y&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;T&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The task structure of this computation is something like nearest-neighbors. It
has a regular pattern with a small number of connections per task. It’s really
more a test of the network hardware, which we see does not impose any
additional scaling limitations (this looks like normal slightly-sub-linear
scaling).&lt;/p&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="7b3301e3-8beb-4da4-bbef-9eb52793f8c1"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;/section&gt;
&lt;section id="rechunking"&gt;
&lt;h2&gt;Rechunking&lt;/h2&gt;
&lt;p&gt;Sometimes communication is composed of many small transfers. For example if
you have a time series of images so that each image is a chunk, you might want
to rechunk the data so that all of the time values for each pixel are in a
chunk instead. Doing this can be very challenging because every output chunk
requires a little bit of data from every input chunk, resulting in potentially
n-squared transfers.&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;y&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;rechunk&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;200&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&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;20000&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This computation can be &lt;em&gt;very hard&lt;/em&gt;. We see that dask does it more slowly than
fast computations like reductions, but it still scales decently well up to
hundreds of workers.&lt;/p&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="a1a28d56-81f7-4580-8400-f7aec3cd4f59"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;/section&gt;
&lt;section id="id1"&gt;
&lt;h2&gt;Nearest Neighbor&lt;/h2&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/2017/07/03/scaling.md&lt;/span&gt;, line 479); &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: “nearest neighbor”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Dask.array includes the ability to overlap small bits of neighboring blocks to
enable functions that require a bit of continuity like derivatives or spatial
smoothing 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="n"&gt;y&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;map_overlap&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;slowinc&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;depth&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;delay&lt;/span&gt;&lt;span class="o"&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="f04e6ad2-5317-4dc8-babd-f3b8d85ae09c"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;/section&gt;
&lt;section id="array-complete"&gt;
&lt;h2&gt;Array Complete&lt;/h2&gt;
&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="ea30e0d3-0a1b-40bb-be2f-e0b54af96aab"&gt;&lt;/div&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/2017/07/03/scaling.md&lt;/span&gt;, line 506)&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="dataframes"&gt;
&lt;h1&gt;DataFrames&lt;/h1&gt;
&lt;p&gt;We can combine Pandas Dataframes with Dask to obtain Dask dataframes,
distributed tables. This section will be much like the last section on arrays
but will instead focus on pandas-style computations.&lt;/p&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
### Create Dataset
&lt;p&gt;We make an array of random integers with ten columns and two million rows per
core, but into chunks of size one million. We turn this into a dataframe of
integers:&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;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;randint&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;10000&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="n"&gt;N&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;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="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_dask_array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="c6417fff-a706-4e33-9337-548f59f230f9"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;section id="elementwise"&gt;
&lt;h2&gt;Elementwise&lt;/h2&gt;
&lt;p&gt;We can perform 100ms tasks or try out a bunch of arithmetic.&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;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_partitions&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;slowinc&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&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;3&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;4&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;5&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;6&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;7&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;8&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;9&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="3d2dcdc7-c91f-4e16-9a6f-88b4837e6375"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;/section&gt;
&lt;section id="id2"&gt;
&lt;h2&gt;Random access&lt;/h2&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/2017/07/03/scaling.md&lt;/span&gt;, line 560); &lt;em&gt;&lt;a href="#id2"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “random access”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Similarly we can try random access with loc.&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;loc&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;123456&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="e2d30245-cca5-41bc-8691-59037c33c9f2"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;/section&gt;
&lt;section id="id3"&gt;
&lt;h2&gt;Reductions&lt;/h2&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/2017/07/03/scaling.md&lt;/span&gt;, line 578); &lt;em&gt;&lt;a href="#id3"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “reductions”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;We can try reductions along the full dataset or a single series:&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;std&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;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;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;std&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Groupby aggregations like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.groupby(...).column.mean()&lt;/span&gt;&lt;/code&gt; operate very
similarly to reductions, just with slightly more complexity.&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;)[&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="055fa730-cc11-4b5d-8237-c067f0fcb5d5"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;/section&gt;
&lt;section id="shuffles"&gt;
&lt;h2&gt;Shuffles&lt;/h2&gt;
&lt;p&gt;However operations like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.groupby(...).apply(...)&lt;/span&gt;&lt;/code&gt; are much harder to
accomplish because we actually need to construct the groups. This requires a
full shuffle of all of the data, which can be quite expensive.&lt;/p&gt;
&lt;p&gt;This is also the same operation that occurs when we sort or call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;set_index&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;apply&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;len&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="c1"&gt;# this would be faster as df.groupby(0).size()&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="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_index&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="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;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This still performs decently and scales well out to a hundred or so workers.&lt;/p&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="a46691fb-a1b1-47fb-9d8f-9a6933ba78b7"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&gt;
&lt;/section&gt;
&lt;section id="timeseries-operations"&gt;
&lt;h2&gt;Timeseries operations&lt;/h2&gt;
&lt;p&gt;Timeseries operations often require nearest neighbor computations. Here we
look at rolling aggregations, but cumulative operations, resampling, and so
on are all much the same.&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;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rolling&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;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;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&gt;&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="ea096ed4-c553-404e-b18f-a1336a4d4156"&gt;&lt;/div&gt;
&lt;/div&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;/section&gt;
&lt;section id="dataframes-complete"&gt;
&lt;h2&gt;Dataframes: Complete&lt;/h2&gt;
&lt;div class="bk-root"&gt;
&lt;div class="bk-plotdiv" id="3bad34c6-9772-45c7-88d7-62c3eaa3fa97"&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&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/2017/07/03/scaling.md&lt;/span&gt;, line 666)&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="analysis"&gt;
&lt;h1&gt;Analysis&lt;/h1&gt;
&lt;p&gt;Let’s start with a few main observations:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;The longer your individual tasks take, the better Dask (or any distributed
system) will scale. As you increase the number of workers you should also
endeavor to increase average task size, for example by increasing the
in-memory size of your array chunks or dataframe partitions.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The Dask scheduler + Client currently maxes out at around 3000 tasks per
second. Another way to put this is that if our computations take 100ms
then we can saturate about 300 cores, which is more-or-less what we observe
here.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Adding dependencies is generally free in modest cases such as in a reduction or
nearest-neighbor computation. It doesn’t matter what structure your
dependencies take, as long as parallelism is still abundant.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Adding more substantial dependencies, such as in array rechunking or
dataframe shuffling, can be more costly, but dask collection algorithms
(array, dataframe) are built to maintain scalability even at scale.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The scheduler seems to slow down at 256 workers, even for long task&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;lengths. This suggests that we may have an overhead issue that needs to be
resolved.&lt;/p&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&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/2017/07/03/scaling.md&lt;/span&gt;, line 693)&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="expert-approach"&gt;
&lt;h1&gt;Expert Approach&lt;/h1&gt;
&lt;p&gt;So given our experience here, let’s now tweak settings to make Dask run well.
We want to avoid two things:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Lots of independent worker processes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Lots of small tasks&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;So lets change some things:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Bigger workers:&lt;/strong&gt; Rather than have 256 two-core workers lets deploy 32
sixteen-core workers.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Bigger chunks:&lt;/strong&gt; Rather than have 2000 by 2000 numpy array chunks lets
bump this up to 10,000 by 10,000.&lt;/p&gt;
&lt;p&gt;Rather than 1,000,000 row Pandas dataframe partitions let’s bump this up to 10,000,000.&lt;/p&gt;
&lt;p&gt;These sizes are still well within comfortable memory limits. Each is about
a Gigabyte in our case.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;When we make these changes we find that all metrics improve at larger scales.
Some notable improvements are included in a table below (sorry for not having
pretty plots in this case).&lt;/p&gt;
&lt;table&gt;
&lt;thead&gt;
&lt;td&gt;&lt;b&gt;Benchmark&lt;/b&gt;&lt;/td&gt;&lt;td&gt;&lt;b&gt;Small&lt;/b&gt;&lt;/td&gt;&lt;td&gt;&lt;b&gt;Big&lt;/b&gt;&lt;/td&gt;&lt;td&gt;&lt;b&gt;Unit&lt;/b&gt;&lt;/td&gt;
&lt;/thead&gt;
&lt;tr&gt;
&lt;td&gt;Tasks: Embarrassingly parallel&lt;/td&gt;
&lt;td&gt;3500&lt;/td&gt;
&lt;td&gt;3800&lt;/td&gt;
&lt;td&gt;tasks/s&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;Array: Elementwise sin(x)**2 + cos(x)**2&lt;/td&gt;
&lt;td&gt;2400&lt;/td&gt;
&lt;td&gt;6500&lt;/td&gt;
&lt;td&gt;MB/s&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;DataFrames: Elementwise arithmetic &lt;/td&gt;
&lt;td&gt;9600&lt;/td&gt;
&lt;td&gt;66000&lt;/td&gt;
&lt;td&gt;MB/s&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;Arrays: Rechunk&lt;/td&gt;
&lt;td&gt;4700&lt;/td&gt;
&lt;td&gt;4800&lt;/td&gt;
&lt;td&gt;MB/s&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;DataFrames: Set index&lt;/td&gt;
&lt;td&gt;1400&lt;/td&gt;
&lt;td&gt;1000&lt;/td&gt;
&lt;td&gt;MB/s&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;We see that for some operations we can get significant improvements
(dask.dataframe is now churning through data at 60/s) and for other operations
that are largely scheduler or network bound this doesn’t strongly improve the
situation (and sometimes hurts).&lt;/p&gt;
&lt;p&gt;Still though, even with naive settings we’re routinely pushing through 10s of
gigabytes a second on a modest cluster. These speeds are available for a very
wide range of computations.&lt;/p&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;div class="row"&gt;
&lt;div class="col-md-6"&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/2017/07/03/scaling.md&lt;/span&gt;, line 768)&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;Hopefully these notes help people to understand Dask’s scalability. Like all
tools it has limits, but even under normal settings Dask should scale well out
to a hundred workers or so. Once you reach this limit you might want to start
taking other factors into consideration, especially threads-per-worker and
block size, both of which can help push well into the thousands-of-cores range.&lt;/p&gt;
&lt;p&gt;The included notebooks are self contained, with code to both run and time the
computations as well as produce the Bokeh figures. I would love to see other
people reproduce these benchmarks (or others!) on different hardware or with
different settings.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/4c198b13e92f881161ef175810c7f6bc#file-scaling-gcs-ipynb"&gt;Jupyter notebook for benchmarks&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://nbviewer.jupyter.org/urls/gist.githubusercontent.com/mrocklin/4c198b13e92f881161ef175810c7f6bc/raw/2fc5a40805fb306eec2af6573e2e93a9d72883cb/scaling-plots.ipynb"&gt;Jupyter notebook for plots&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/div&gt;
&lt;div class="col-md-6"&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/2017/07/03/scaling.md&lt;/span&gt;, line 788)&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="tooling"&gt;
&lt;h1&gt;Tooling&lt;/h1&gt;
&lt;p&gt;This blogpost made use of the following tools:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/martindurant/dask-kubernetes"&gt;Dask-kubernetes&lt;/a&gt;: for
deploying clusters of varying sizes on Google compute engine&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://bokeh.pydata.org/en/latest/"&gt;Bokeh&lt;/a&gt;: for plotting
(&lt;a class="reference external" href="https://bokeh.pydata.org/en/latest/docs/gallery.html"&gt;gallery&lt;/a&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://gcsfs.readthedocs.io/en/latest/"&gt;gcsfs&lt;/a&gt;: for storage on Google
cloud storage&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;script type="text/javascript"&gt;
    (function() {
  var fn = function() {
    Bokeh.safely(function() {
      var docs_json = {"c37a7732-0c94-4514-bc5b-189fa40a390f":{"roots":{"references":[{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"e007db40-8804-4fc3-8fd0-f0d696c2697c","type":"Circle"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,506.8406659004797]}},"id":"aa15e897-6c86-46d5-9c8a-1976053e42e0","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"8b74d8e5-a13e-4955-bfca-bcab0e2df8a7","type":"ColumnDataSource"},"glyph":{"id":"fe27d651-f139-49a9-8553-62e6d681be99","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"e007db40-8804-4fc3-8fd0-f0d696c2697c","type":"Circle"},"selection_glyph":null},"id":"72ff53d1-eae1-44b0-9e17-1576c8ed8435","type":"GlyphRenderer"},{"attributes":{"below":[{"id":"ec90fd4d-97e4-4514-b1b9-a342144ecfd4","type":"LogAxis"}],"left":[{"id":"4d0b5ee2-e522-422c-ac00-fb6e8bd64d19","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"ec90fd4d-97e4-4514-b1b9-a342144ecfd4","type":"LogAxis"},{"id":"900cd3cf-7e4a-4349-ac71-a261a5822bff","type":"Grid"},{"id":"4d0b5ee2-e522-422c-ac00-fb6e8bd64d19","type":"LogAxis"},{"id":"4802e78d-1853-4c20-8f57-d2ef5d94f259","type":"Grid"},{"id":"add7c6e9-6c2c-44db-a848-5316f1f5604e","type":"BoxAnnotation"},{"id":"9c0133b2-694e-4b46-9a57-b640227dede1","type":"Legend"},{"id":"574611f2-05d6-4ba6-8e30-00a164f87763","type":"GlyphRenderer"},{"id":"9bf05800-53d0-4c83-9dc9-6d36bf273717","type":"GlyphRenderer"},{"id":"8761dbdd-001d-4bdf-adee-a45a6d304b7e","type":"GlyphRenderer"},{"id":"74e9596d-b53b-4669-87c7-c36ea13c283b","type":"GlyphRenderer"},{"id":"b5da0192-a4d1-4258-993d-08035157554c","type":"GlyphRenderer"},{"id":"020543b4-4bdc-4f9d-813e-9e578b5b9566","type":"GlyphRenderer"}],"title":{"id":"e9b463ca-461c-448b-996c-36d409061c1a","type":"Title"},"tool_events":{"id":"7354dc19-fb54-4295-ae5e-42975743213a","type":"ToolEvents"},"toolbar":{"id":"c7f571ae-9d27-4b2f-b54e-dc53e40e99f8","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"ec984df4-d8d4-49d2-bdd8-09e7ed64bd84","type":"DataRange1d"},"x_scale":{"id":"9f289e7e-4d9e-4659-a45e-0d47e06c494f","type":"LogScale"},"y_range":{"id":"bc249175-2057-4b6a-8901-302cdcd70cec","type":"DataRange1d"},"y_scale":{"id":"8b140c20-6d76-44f3-91e6-e42463f894b7","type":"LogScale"}},"id":"235a3d83-e4c1-4855-b3f3-96247ff0b146","subtype":"Figure","type":"Plot"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"9a6593d5-0250-4e46-af7e-b2a7748c0733","type":"FixedTicker"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"7c99fea2-3216-4d84-b912-8d9584fad011","type":"Panel"},{"id":"e3565d29-459e-453a-82a3-5fb3335eab1b","type":"Panel"}]},"id":"948f0ba1-2615-4ca3-9634-5eade5d4eab8","type":"Tabs"},{"attributes":{"plot":null,"text":"Tasks: Nearest Neighbor"},"id":"e9b463ca-461c-448b-996c-36d409061c1a","type":"Title"},{"attributes":{"callback":null,"start":0},"id":"ec984df4-d8d4-49d2-bdd8-09e7ed64bd84","type":"DataRange1d"},{"attributes":{},"id":"7354dc19-fb54-4295-ae5e-42975743213a","type":"ToolEvents"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"e0ba9b60-db70-4b11-90ea-2868bc56108f","type":"PanTool"},{"id":"119970ac-6463-4f32-a7e2-f3bb89fcae56","type":"WheelZoomTool"},{"id":"f50914d8-200a-4f31-abfd-26c17a885c0c","type":"BoxZoomTool"},{"id":"25aa7dd9-490d-4122-9f8a-fa60ccf8b1ac","type":"SaveTool"},{"id":"f26e3fe2-102a-4632-99cc-17710686c86a","type":"ResetTool"},{"id":"3b7b47f2-f735-4ce9-94a7-152fe76da795","type":"HelpTool"}]},"id":"c7f571ae-9d27-4b2f-b54e-dc53e40e99f8","type":"Toolbar"},{"attributes":{"num_minor_ticks":10},"id":"b835afc1-9f31-47a8-8415-37c4fc09a68f","type":"LogTicker"},{"attributes":{"plot":{"id":"235a3d83-e4c1-4855-b3f3-96247ff0b146","subtype":"Figure","type":"Plot"}},"id":"3b7b47f2-f735-4ce9-94a7-152fe76da795","type":"HelpTool"},{"attributes":{"callback":null,"end":3889.605662888761,"start":0},"id":"bc249175-2057-4b6a-8901-302cdcd70cec","type":"DataRange1d"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"add7c6e9-6c2c-44db-a848-5316f1f5604e","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"235a3d83-e4c1-4855-b3f3-96247ff0b146","subtype":"Figure","type":"Plot"},"ticker":{"id":"b835afc1-9f31-47a8-8415-37c4fc09a68f","type":"LogTicker"}},"id":"900cd3cf-7e4a-4349-ac71-a261a5822bff","type":"Grid"},{"attributes":{},"id":"9f289e7e-4d9e-4659-a45e-0d47e06c494f","type":"LogScale"},{"attributes":{"axis_label":"cores","formatter":{"id":"43614ea2-6c98-4c4b-b244-cd6997748460","type":"LogTickFormatter"},"plot":{"id":"235a3d83-e4c1-4855-b3f3-96247ff0b146","subtype":"Figure","type":"Plot"},"ticker":{"id":"9a6593d5-0250-4e46-af7e-b2a7748c0733","type":"FixedTicker"}},"id":"ec90fd4d-97e4-4514-b1b9-a342144ecfd4","type":"LogAxis"},{"attributes":{"plot":{"id":"235a3d83-e4c1-4855-b3f3-96247ff0b146","subtype":"Figure","type":"Plot"}},"id":"f26e3fe2-102a-4632-99cc-17710686c86a","type":"ResetTool"},{"attributes":{"ticker":null},"id":"43614ea2-6c98-4c4b-b244-cd6997748460","type":"LogTickFormatter"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,429249.4818984265]}},"id":"8d20c76c-a0e7-490a-990d-8778b22525db","type":"ColumnDataSource"},{"attributes":{"ticker":null},"id":"dbe6206b-2096-45ac-9348-a3ceb73aca41","type":"LogTickFormatter"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"YiGd3kr1M0C4DswHLGBDQAJlPBqPJVNAF1KOo6LJYkAH1cQ/wblxQALKM0XtrH9ADDdmn9fbi0DY4pMc/3KVQBZyRig+NJpA","dtype":"float64","shape":[9]}}},"id":"a044186a-fb4c-40df-99b5-041e0e8bae53","type":"ColumnDataSource"},{"attributes":{"items":[{"id":"e3759520-c0b2-49f6-b20d-9837dd3812ff","type":"LegendItem"},{"id":"d078ef74-743b-4d72-930b-cdccaa71686b","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"235a3d83-e4c1-4855-b3f3-96247ff0b146","subtype":"Figure","type":"Plot"}},"id":"9c0133b2-694e-4b46-9a57-b640227dede1","type":"Legend"},{"attributes":{"callback":null,"start":0},"id":"b001ffc1-f73c-4a86-8852-4cc84bccf134","type":"DataRange1d"},{"attributes":{"label":{"value":"100ms"},"renderers":[{"id":"574611f2-05d6-4ba6-8e30-00a164f87763","type":"GlyphRenderer"}]},"id":"e3759520-c0b2-49f6-b20d-9837dd3812ff","type":"LegendItem"},{"attributes":{"data_source":{"id":"c258ceef-ff77-4c99-bede-d114e0e8661e","type":"ColumnDataSource"},"glyph":{"id":"9921a856-db94-45ec-965b-a4c82007d7c9","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"916990fa-9a5f-41cf-86af-c752ab75187b","type":"Line"},"selection_glyph":null},"id":"8761dbdd-001d-4bdf-adee-a45a6d304b7e","type":"GlyphRenderer"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"d0630987-de58-4730-9b74-9d48879ccebb","type":"Circle"},{"attributes":{"data_source":{"id":"a044186a-fb4c-40df-99b5-041e0e8bae53","type":"ColumnDataSource"},"glyph":{"id":"d0630987-de58-4730-9b74-9d48879ccebb","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"92c4857d-a271-4cd8-ae95-b9d97f4b5500","type":"Circle"},"selection_glyph":null},"id":"9bf05800-53d0-4c83-9dc9-6d36bf273717","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"92c4857d-a271-4cd8-ae95-b9d97f4b5500","type":"Circle"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"f3de3fe5-0544-441d-a3fe-a189685e7c38","type":"Line"},{"attributes":{"data_source":{"id":"8d20c76c-a0e7-490a-990d-8778b22525db","type":"ColumnDataSource"},"glyph":{"id":"a151b77f-3293-4751-b50f-342b95f0d038","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"f3de3fe5-0544-441d-a3fe-a189685e7c38","type":"Line"},"selection_glyph":null},"id":"020543b4-4bdc-4f9d-813e-9e578b5b9566","type":"GlyphRenderer"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"9921a856-db94-45ec-965b-a4c82007d7c9","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"916990fa-9a5f-41cf-86af-c752ab75187b","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"98d27QUzmkDK8eq3P0GeQDr+4mLrCqlAVTMqiTA6rEBCN3IZNmOuQOX8onhe0a1A7S7Drxhtq0DDEdAa+eaqQHPoMPdNM6lA","dtype":"float64","shape":[9]}}},"id":"f4cb967a-e3e7-4c2a-85d7-94edafe74ddb","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,5109.292459316874]}},"id":"99f31978-e596-4b53-9442-bfdc9bf67fd9","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"98d27QUzmkDK8eq3P0GeQDr+4mLrCqlAVTMqiTA6rEBCN3IZNmOuQOX8onhe0a1A7S7Drxhtq0DDEdAa+eaqQHPoMPdNM6lA","dtype":"float64","shape":[9]}}},"id":"c258ceef-ff77-4c99-bede-d114e0e8661e","type":"ColumnDataSource"},{"attributes":{"label":{"value":"1us"},"renderers":[{"id":"8761dbdd-001d-4bdf-adee-a45a6d304b7e","type":"GlyphRenderer"}]},"id":"d078ef74-743b-4d72-930b-cdccaa71686b","type":"LegendItem"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"3b37257b-29ce-4b6b-ab05-d55f53cb5318","type":"Line"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"5612f988-0273-4d0b-82df-e2a036b566dd","type":"Circle"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"3b9715d7-d3cf-43b1-83b5-5d46099190a6","type":"Circle"},{"attributes":{"data_source":{"id":"99f31978-e596-4b53-9442-bfdc9bf67fd9","type":"ColumnDataSource"},"glyph":{"id":"3b37257b-29ce-4b6b-ab05-d55f53cb5318","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"4f639ad0-1df4-4065-b96f-87db75286b8d","type":"Line"},"selection_glyph":null},"id":"b5da0192-a4d1-4258-993d-08035157554c","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"1a533ad6-a74d-45af-8afd-5fadb94dc063","type":"Circle"},{"attributes":{"data_source":{"id":"f4cb967a-e3e7-4c2a-85d7-94edafe74ddb","type":"ColumnDataSource"},"glyph":{"id":"3b9715d7-d3cf-43b1-83b5-5d46099190a6","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"1a533ad6-a74d-45af-8afd-5fadb94dc063","type":"Circle"},"selection_glyph":null},"id":"74e9596d-b53b-4669-87c7-c36ea13c283b","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"24450ec2-23b0-43b9-a45e-645b0d14b4e0","type":"ColumnDataSource"},"glyph":{"id":"802d8d3b-1602-4f78-8b1a-ccbbd26324ae","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"5612f988-0273-4d0b-82df-e2a036b566dd","type":"Circle"},"selection_glyph":null},"id":"2f5b7913-d5eb-4f8d-8015-4df1e3921af0","type":"GlyphRenderer"},{"attributes":{"plot":null,"text":"Tasks: Nearest Neighbor"},"id":"b34839f9-43a6-4db1-b259-9ae621bab554","type":"Title"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"3a7b15e0-3597-4a99-8089-6ecce1d9ca49","type":"PanTool"},{"id":"cd0d9904-16bc-4916-b074-b04f42c3c0c8","type":"WheelZoomTool"},{"id":"a0edbc21-c056-4cdc-a171-0c99928b8ab7","type":"BoxZoomTool"},{"id":"8b476975-4b0a-41e2-b278-f7b49bc29967","type":"SaveTool"},{"id":"fbef2fb6-1564-4b5c-b1ab-205dd4802aea","type":"ResetTool"},{"id":"b34e007a-6220-4ded-a494-4c692ef5e34e","type":"HelpTool"}]},"id":"35bf0528-69b8-462b-b22b-f20921cbf075","type":"Toolbar"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"4f639ad0-1df4-4065-b96f-87db75286b8d","type":"Line"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"a151b77f-3293-4751-b50f-342b95f0d038","type":"Line"},{"attributes":{},"id":"96fee7b2-af76-440c-980d-23642c8291a7","type":"ToolEvents"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"YiGd3kr1M0C4DswHLGBDQAJlPBqPJVNAF1KOo6LJYkAH1cQ/wblxQALKM0XtrH9ADDdmn9fbi0DY4pMc/3KVQBZyRig+NJpA","dtype":"float64","shape":[9]}}},"id":"5393ef18-b0dd-4e69-aa17-c8a894b3e553","type":"ColumnDataSource"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"63046e3d-cbb0-40b2-ba3e-840605226eac","type":"Line"},{"attributes":{"below":[{"id":"8712c194-d4cf-4fc1-8238-75a3367208c5","type":"LinearAxis"}],"left":[{"id":"ec103342-fbb5-4e1e-9218-efecfbb465dc","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"8712c194-d4cf-4fc1-8238-75a3367208c5","type":"LinearAxis"},{"id":"097ec216-0566-4402-a20f-c9f8c6bf53a6","type":"Grid"},{"id":"ec103342-fbb5-4e1e-9218-efecfbb465dc","type":"LinearAxis"},{"id":"c815bc0b-8a5a-44e6-953d-8085a5106242","type":"Grid"},{"id":"7125017b-656e-407f-9006-42d58445db17","type":"BoxAnnotation"},{"id":"8b367327-372a-448e-99a6-fd80f7367292","type":"Legend"},{"id":"0e811120-b5fb-4b6a-acaf-28b0e4a6c1d1","type":"GlyphRenderer"},{"id":"2f5b7913-d5eb-4f8d-8015-4df1e3921af0","type":"GlyphRenderer"},{"id":"34a85c78-f3f8-4f21-91e8-1fe0f4fae357","type":"GlyphRenderer"},{"id":"c3588a2b-2bad-4eac-a8e9-cdba1cb268e2","type":"GlyphRenderer"},{"id":"8fca08fa-ae6e-4a96-b377-c51a5fac6c86","type":"GlyphRenderer"},{"id":"68b04869-9d0b-402b-8294-fb51737caed5","type":"GlyphRenderer"}],"title":{"id":"b34839f9-43a6-4db1-b259-9ae621bab554","type":"Title"},"tool_events":{"id":"96fee7b2-af76-440c-980d-23642c8291a7","type":"ToolEvents"},"toolbar":{"id":"35bf0528-69b8-462b-b22b-f20921cbf075","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"b001ffc1-f73c-4a86-8852-4cc84bccf134","type":"DataRange1d"},"x_scale":{"id":"e1be6568-4bdb-4bb3-8e27-b00adda77f05","type":"LinearScale"},"y_range":{"id":"e7f927e6-d0b1-43e7-a944-2fafdd81d46d","type":"DataRange1d"},"y_scale":{"id":"0258df5a-ee7f-4f26-bade-9deb6f13bd65","type":"LinearScale"}},"id":"d596527a-b70a-42bb-897a-44a65ed6ae5d","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"e1be6568-4bdb-4bb3-8e27-b00adda77f05","type":"LinearScale"},{"attributes":{"callback":null,"end":3889.605662888761,"start":0},"id":"e7f927e6-d0b1-43e7-a944-2fafdd81d46d","type":"DataRange1d"},{"attributes":{},"id":"0258df5a-ee7f-4f26-bade-9deb6f13bd65","type":"LinearScale"},{"attributes":{"plot":{"id":"d596527a-b70a-42bb-897a-44a65ed6ae5d","subtype":"Figure","type":"Plot"},"ticker":{"id":"bdbd2aa8-f4d8-4d12-ac1b-a1da26502d0e","type":"BasicTicker"}},"id":"097ec216-0566-4402-a20f-c9f8c6bf53a6","type":"Grid"},{"attributes":{"axis_label":"cores","formatter":{"id":"43f1f63c-8840-41d0-bd91-73753f416566","type":"BasicTickFormatter"},"plot":{"id":"d596527a-b70a-42bb-897a-44a65ed6ae5d","subtype":"Figure","type":"Plot"},"ticker":{"id":"0e46acf9-09da-417d-a968-159dea7e0873","type":"FixedTicker"}},"id":"8712c194-d4cf-4fc1-8238-75a3367208c5","type":"LinearAxis"},{"attributes":{},"id":"bdbd2aa8-f4d8-4d12-ac1b-a1da26502d0e","type":"BasicTicker"},{"attributes":{"axis_label":"tasks/s","formatter":{"id":"aa405d77-23c5-4840-adfb-fcbfde7f3700","type":"BasicTickFormatter"},"plot":{"id":"d596527a-b70a-42bb-897a-44a65ed6ae5d","subtype":"Figure","type":"Plot"},"ticker":{"id":"2d761743-2cf4-43a2-8c8e-d0cc887a92af","type":"BasicTicker"}},"id":"ec103342-fbb5-4e1e-9218-efecfbb465dc","type":"LinearAxis"},{"attributes":{},"id":"2d761743-2cf4-43a2-8c8e-d0cc887a92af","type":"BasicTicker"},{"attributes":{"dimension":1,"plot":{"id":"d596527a-b70a-42bb-897a-44a65ed6ae5d","subtype":"Figure","type":"Plot"},"ticker":{"id":"2d761743-2cf4-43a2-8c8e-d0cc887a92af","type":"BasicTicker"}},"id":"c815bc0b-8a5a-44e6-953d-8085a5106242","type":"Grid"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"70dff99f-4db9-4aff-9bb8-0fb5923eb273","type":"Line"},{"attributes":{"data_source":{"id":"5393ef18-b0dd-4e69-aa17-c8a894b3e553","type":"ColumnDataSource"},"glyph":{"id":"70dff99f-4db9-4aff-9bb8-0fb5923eb273","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"63046e3d-cbb0-40b2-ba3e-840605226eac","type":"Line"},"selection_glyph":null},"id":"0e811120-b5fb-4b6a-acaf-28b0e4a6c1d1","type":"GlyphRenderer"},{"attributes":{"child":{"id":"d596527a-b70a-42bb-897a-44a65ed6ae5d","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"e3565d29-459e-453a-82a3-5fb3335eab1b","type":"Panel"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"7125017b-656e-407f-9006-42d58445db17","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"d596527a-b70a-42bb-897a-44a65ed6ae5d","subtype":"Figure","type":"Plot"}},"id":"3a7b15e0-3597-4a99-8089-6ecce1d9ca49","type":"PanTool"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"802d8d3b-1602-4f78-8b1a-ccbbd26324ae","type":"Circle"},{"attributes":{"plot":{"id":"d596527a-b70a-42bb-897a-44a65ed6ae5d","subtype":"Figure","type":"Plot"}},"id":"cd0d9904-16bc-4916-b074-b04f42c3c0c8","type":"WheelZoomTool"},{"attributes":{"overlay":{"id":"7125017b-656e-407f-9006-42d58445db17","type":"BoxAnnotation"},"plot":{"id":"d596527a-b70a-42bb-897a-44a65ed6ae5d","subtype":"Figure","type":"Plot"}},"id":"a0edbc21-c056-4cdc-a171-0c99928b8ab7","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"d596527a-b70a-42bb-897a-44a65ed6ae5d","subtype":"Figure","type":"Plot"}},"id":"8b476975-4b0a-41e2-b278-f7b49bc29967","type":"SaveTool"},{"attributes":{"plot":{"id":"d596527a-b70a-42bb-897a-44a65ed6ae5d","subtype":"Figure","type":"Plot"}},"id":"fbef2fb6-1564-4b5c-b1ab-205dd4802aea","type":"ResetTool"},{"attributes":{"plot":{"id":"d596527a-b70a-42bb-897a-44a65ed6ae5d","subtype":"Figure","type":"Plot"}},"id":"b34e007a-6220-4ded-a494-4c692ef5e34e","type":"HelpTool"},{"attributes":{},"id":"aa405d77-23c5-4840-adfb-fcbfde7f3700","type":"BasicTickFormatter"},{"attributes":{"below":[{"id":"5782e52f-d045-4066-8e67-4ff6b658a5b9","type":"LinearAxis"}],"left":[{"id":"b6d97613-9b92-4c51-93c9-494f3435f0be","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"5782e52f-d045-4066-8e67-4ff6b658a5b9","type":"LinearAxis"},{"id":"eb20f2d2-177b-4983-8352-8bfbceddd49c","type":"Grid"},{"id":"b6d97613-9b92-4c51-93c9-494f3435f0be","type":"LinearAxis"},{"id":"46f80ab7-5004-455a-a548-55ec92fffb80","type":"Grid"},{"id":"656cf7cc-3748-4c60-ad17-48948d4f1067","type":"BoxAnnotation"},{"id":"e428b65c-bedd-4afe-b852-3e0e275615e3","type":"Legend"},{"id":"4bf1be6f-0524-4691-a469-c271ed97e0b9","type":"GlyphRenderer"},{"id":"fad64154-6abb-410f-a32f-9691b20da3f9","type":"GlyphRenderer"},{"id":"c100652d-4b2f-441d-a19b-1d5d96238306","type":"GlyphRenderer"},{"id":"6216cd53-1ee7-4d64-bed7-99aa32b36d65","type":"GlyphRenderer"},{"id":"610e6598-84c3-4377-bce3-4258e91e8129","type":"GlyphRenderer"},{"id":"eae59b8c-4aae-42b2-af00-907169e4c2d0","type":"GlyphRenderer"}],"title":{"id":"5ed47176-1165-43e2-acb6-dcbc4ffee512","type":"Title"},"tool_events":{"id":"19779bf4-dc96-4d5f-a96c-650727df217c","type":"ToolEvents"},"toolbar":{"id":"344426e7-d635-4e04-a5de-4ffd275f71c6","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"8e2a6dae-41dd-4920-8ff6-8780ed089f0a","type":"DataRange1d"},"x_scale":{"id":"1e10a22c-12aa-4a41-915c-f85a8ed32090","type":"LinearScale"},"y_range":{"id":"e4513607-4e88-4611-b52e-319f23abdd30","type":"DataRange1d"},"y_scale":{"id":"1bb1ad9c-effe-4731-9798-9d4a4aa13fa5","type":"LinearScale"}},"id":"bd7a4267-cf17-4206-a88f-6685cd6fdc61","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"43f1f63c-8840-41d0-bd91-73753f416566","type":"BasicTickFormatter"},{"attributes":{"data_source":{"id":"798b04c0-7810-433c-9d10-e52806569ced","type":"ColumnDataSource"},"glyph":{"id":"11d6ffa4-ab28-4bc2-b71f-bdb2f16a1e20","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"1d27dfc8-f33e-4a00-859c-228c9443d3a6","type":"Line"},"selection_glyph":null},"id":"a5327d44-b21a-412c-8c94-f9363f357d96","type":"GlyphRenderer"},{"attributes":{"overlay":{"id":"16d3717d-7fcf-4009-b087-43fb39f20e1e","type":"BoxAnnotation"},"plot":{"id":"4fab0e63-76cb-46a6-843e-af38f25fd676","subtype":"Figure","type":"Plot"}},"id":"50075953-0cab-4112-abce-afe021f330ea","type":"BoxZoomTool"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"16d3717d-7fcf-4009-b087-43fb39f20e1e","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"4fab0e63-76cb-46a6-843e-af38f25fd676","subtype":"Figure","type":"Plot"}},"id":"216cc837-f8a2-4718-a2b9-ab6aae1188f5","type":"SaveTool"},{"attributes":{"plot":{"id":"4fab0e63-76cb-46a6-843e-af38f25fd676","subtype":"Figure","type":"Plot"}},"id":"497251b6-85e2-450e-a3df-dc513d4623d8","type":"ResetTool"},{"attributes":{"plot":{"id":"4fab0e63-76cb-46a6-843e-af38f25fd676","subtype":"Figure","type":"Plot"}},"id":"da51f522-93e8-4265-a989-c54b2d742198","type":"HelpTool"},{"attributes":{"data_source":{"id":"540ea8dd-e79f-4d23-8d1d-2260fd9c9cc4","type":"ColumnDataSource"},"glyph":{"id":"a07c6be1-0e1c-4254-b77a-11bcee7c1e3c","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"e44f06cb-5e14-4173-829a-78e5edb37dd6","type":"Line"},"selection_glyph":null},"id":"0c8fbb3b-c3b4-497a-95d9-2952c6e6eeaf","type":"GlyphRenderer"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"5cf39d3a-1212-4d68-abe5-26f177c25f18","type":"Line"},{"attributes":{},"id":"94f6bf21-e27e-4108-87fb-96b57789a1b2","type":"BasicTickFormatter"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"BSBNWCE2dEC9BpWUVImFQDzSCcCUkJBAVLgx0fyxnkCFGyLiy4+jQE7fanACZ6xAEeby5mMgsEDryZN7k+GwQFqGlput7rBA","dtype":"float64","shape":[9]}}},"id":"8b605df9-daae-46b1-8b66-17e858c53271","type":"ColumnDataSource"},{"attributes":{"items":[{"id":"9c0152fe-a0bb-465d-b3aa-5de9592b1e82","type":"LegendItem"},{"id":"e3124601-7f30-4128-812e-4afdebf41a1e","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"4fab0e63-76cb-46a6-843e-af38f25fd676","subtype":"Figure","type":"Plot"}},"id":"c5106ceb-af0f-41eb-a0a3-00686ab2f704","type":"Legend"},{"attributes":{"label":{"value":"1us"},"renderers":[{"id":"0c8fbb3b-c3b4-497a-95d9-2952c6e6eeaf","type":"GlyphRenderer"}]},"id":"e3124601-7f30-4128-812e-4afdebf41a1e","type":"LegendItem"},{"attributes":{"label":{"value":"100ms"},"renderers":[{"id":"47190973-cc0e-4e9f-9e83-1c6099856fa6","type":"GlyphRenderer"}]},"id":"9c0152fe-a0bb-465d-b3aa-5de9592b1e82","type":"LegendItem"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"a07c6be1-0e1c-4254-b77a-11bcee7c1e3c","type":"Line"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"513004ff-e78b-4e56-9c41-4646f797f42b","type":"Circle"},{"attributes":{"data_source":{"id":"8b605df9-daae-46b1-8b66-17e858c53271","type":"ColumnDataSource"},"glyph":{"id":"513004ff-e78b-4e56-9c41-4646f797f42b","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"6f97b8de-fdc5-4e2d-870c-d99b2113bd24","type":"Circle"},"selection_glyph":null},"id":"75535fe1-cdc3-44dd-af24-2d0e3adb77e1","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"6f97b8de-fdc5-4e2d-870c-d99b2113bd24","type":"Circle"},{"attributes":{"children":[{"id":"4adbb7d8-6616-44e4-be42-6d199ba330ec","type":"WidgetBox"},{"id":"5392f54b-cdeb-4555-b378-01c7d6dede56","type":"WidgetBox"}],"sizing_mode":"scale_width"},"id":"2562d80e-1e0a-4237-9f75-df3916b837df","type":"Row"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"e44f06cb-5e14-4173-829a-78e5edb37dd6","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"Gmr7TkyPlEBRf7wkoNCfQH9VJgZniqdALjvyvyi0rECvSL87J26wQO0MDDGQELJANhKvi9UxskBz0zc+QhSyQD61nBkBcrFA","dtype":"float64","shape":[9]}}},"id":"e5963e2e-6c71-420e-952f-d9e7cd5d5cce","type":"ColumnDataSource"},{"attributes":{"children":[{"id":"d5eef40f-39b6-49cb-9230-302039c21841","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"4adbb7d8-6616-44e4-be42-6d199ba330ec","type":"WidgetBox"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"22e1c530-a0b1-4c90-8339-87536ab99c31","type":"Circle"},{"attributes":{"data_source":{"id":"2b1426ac-bef1-47be-bd6f-2c91cfcd041e","type":"ColumnDataSource"},"glyph":{"id":"5cf39d3a-1212-4d68-abe5-26f177c25f18","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"fb651baa-3b08-4a63-94ae-dead5e491646","type":"Line"},"selection_glyph":null},"id":"ddc24851-3ca0-4a00-b6cf-df22004ede8c","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"8fab1eeb-2015-497e-b3ad-0a658f76ebe1","type":"Circle"},{"attributes":{"data_source":{"id":"e5963e2e-6c71-420e-952f-d9e7cd5d5cce","type":"ColumnDataSource"},"glyph":{"id":"22e1c530-a0b1-4c90-8339-87536ab99c31","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"8fab1eeb-2015-497e-b3ad-0a658f76ebe1","type":"Circle"},"selection_glyph":null},"id":"0310db36-3f75-438c-be17-cdf7fd57fe79","type":"GlyphRenderer"},{"attributes":{"children":[{"id":"4311ab7e-c88e-42cf-b077-bfd6acbda2f0","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"b2d08de9-8122-4c7e-8580-06583132cad3","type":"WidgetBox"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,82786.0840579272]}},"id":"2b1426ac-bef1-47be-bd6f-2c91cfcd041e","type":"ColumnDataSource"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"fb651baa-3b08-4a63-94ae-dead5e491646","type":"Line"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"ea66368f-c907-4b57-a0fa-fc3448cd572a","type":"Line"},{"attributes":{"data_source":{"id":"3bba641a-0ba0-4372-9468-28fcab8e2930","type":"ColumnDataSource"},"glyph":{"id":"ea66368f-c907-4b57-a0fa-fc3448cd572a","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"db5662e3-c646-431a-a02b-a341a39ac170","type":"Line"},"selection_glyph":null},"id":"6551e6ea-0811-4028-ad05-fef84dbb3abf","type":"GlyphRenderer"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"db5662e3-c646-431a-a02b-a341a39ac170","type":"Line"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"e252e15d-c6d5-4642-8016-3528d9a053b7","type":"FixedTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,336851.07713094505]}},"id":"3bba641a-0ba0-4372-9468-28fcab8e2930","type":"ColumnDataSource"},{"attributes":{"children":[{"id":"50a686fd-e2c5-440a-92af-bbfc9d851b96","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"97bd2a78-88d3-40cc-a011-e9307cede1c8","type":"WidgetBox"},{"attributes":{"children":[{"id":"f9c919ee-8bdd-4d0f-b48c-bc38b6926c14","type":"ToolbarBox"},{"id":"344a1d6e-41e7-4fa4-86a6-6537dbb6c9da","type":"Column"}],"sizing_mode":"scale_width"},"id":"ed39ed3e-c52c-4159-85ae-c61b8a19409c","type":"Column"},{"attributes":{"children":[{"id":"97bd2a78-88d3-40cc-a011-e9307cede1c8","type":"WidgetBox"},{"id":"6beb2b9d-7ce1-443c-a599-6f7ff7d88cc9","type":"WidgetBox"}],"sizing_mode":"scale_width"},"id":"fb713db1-aa51-4406-b604-cd896be47fdf","type":"Row"},{"attributes":{"children":[{"id":"716ab0cd-3d4a-4000-afeb-483507e81870","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"e4f2702d-389c-44d0-b7f1-a01a1d855234","type":"WidgetBox"},{"attributes":{"children":[{"id":"d5449472-9151-45a2-82ef-636c1e6e7fad","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"6beb2b9d-7ce1-443c-a599-6f7ff7d88cc9","type":"WidgetBox"},{"attributes":{"sizing_mode":"scale_width","toolbar_location":"above","tools":[{"id":"3940e366-9f09-44b8-97f8-2bd1a03e885c","type":"PanTool"},{"id":"285fd6ac-775c-4e1a-936a-68a5442242b7","type":"WheelZoomTool"},{"id":"8546812f-01ca-4877-98fd-c31364acbe3d","type":"BoxZoomTool"},{"id":"00dffc1d-3080-4d12-b51b-e86572d80407","type":"SaveTool"},{"id":"44cf5793-22b4-47bf-a7a1-da3521d2ceea","type":"ResetTool"},{"id":"1e78d025-bd71-4310-9185-a2857b169623","type":"HelpTool"},{"id":"84c5ebf4-804c-45e8-86a3-23655f9c10cf","type":"PanTool"},{"id":"52678c06-d482-4439-b4f7-fdd872eb00e1","type":"WheelZoomTool"},{"id":"1636d618-670c-4438-b4f8-ee09061d2c5e","type":"BoxZoomTool"},{"id":"de891a7c-65b8-4d1c-9b61-242f54a10f4b","type":"SaveTool"},{"id":"a72122d2-7c05-4909-b806-34da1879e5a5","type":"ResetTool"},{"id":"35ae2e1d-7679-4f32-9cb4-5e5c38fc97c4","type":"HelpTool"},{"id":"f240d2c0-ff6c-491a-ab4e-61eae5d526ed","type":"PanTool"},{"id":"e2ce7b1b-cf07-411c-b287-d1b33ecf17f9","type":"WheelZoomTool"},{"id":"d9cf7207-525f-49a5-9224-035cd6dfc2dc","type":"BoxZoomTool"},{"id":"e4a0baa0-5ecc-4689-a4f4-6ee954c27052","type":"SaveTool"},{"id":"58423d76-834c-42ce-9dc5-b8729ce97321","type":"ResetTool"},{"id":"9b541db3-5f39-403a-ac85-bd080ebaa38a","type":"HelpTool"},{"id":"ee8f3c74-9606-467b-9649-496ff3790d92","type":"PanTool"},{"id":"20be3e8f-d733-4775-88b3-846d52aa9ab7","type":"WheelZoomTool"},{"id":"157c4d22-7e1e-4cb9-b5de-51771a77c65b","type":"BoxZoomTool"},{"id":"1d85faba-8a6b-4095-961d-e7b90412bc93","type":"SaveTool"},{"id":"c39730d2-c985-4f53-97f8-e14e1accf06e","type":"ResetTool"},{"id":"b5fe7351-f4ac-4095-818c-83276790a403","type":"HelpTool"},{"id":"69b885bb-30f4-426f-9d42-de58458e2ff9","type":"PanTool"},{"id":"dc2065d3-d923-4d56-8f0a-098f825d4480","type":"WheelZoomTool"},{"id":"7fb2a7a7-6e7c-4f7e-abe6-bb0b4d4547c0","type":"BoxZoomTool"},{"id":"6e62c218-37ff-42bd-806b-b4c4eaaf4155","type":"SaveTool"},{"id":"5862d233-d9e4-402f-a9b0-823b4b28838d","type":"ResetTool"},{"id":"1abec02d-e81f-4d20-8096-046d0dca57f1","type":"HelpTool"},{"id":"9cdbfea0-78a1-4816-95b3-20711d60ba64","type":"PanTool"},{"id":"d917a161-3ebd-4ff0-9e66-b1890604b6c7","type":"WheelZoomTool"},{"id":"c2e5b325-c321-4c08-b3da-cdf89da70574","type":"BoxZoomTool"},{"id":"60a7d2f5-41bc-4215-ac31-864b2bf34829","type":"SaveTool"},{"id":"66346441-b239-489b-900f-c4d4660d7478","type":"ResetTool"},{"id":"9e9eec61-23ba-4987-af83-bc09e49775bd","type":"HelpTool"},{"id":"d7e00d52-b6a0-43d3-b2ce-aba245b42337","type":"PanTool"},{"id":"b288bf15-ef2e-4945-9eb4-ccaf8cd6c334","type":"WheelZoomTool"},{"id":"6007b073-348b-41bc-a701-31198057ecfe","type":"BoxZoomTool"},{"id":"2a2d4a72-6109-438e-aa25-4a5692f74faf","type":"SaveTool"},{"id":"d377ce77-c83a-4022-83a8-021b1f1aa6b0","type":"ResetTool"},{"id":"16a5b5ca-891a-4048-9e4c-749d27fa4e99","type":"HelpTool"},{"id":"6c0f4cd7-3b7f-450a-8ec5-5ad54dee9602","type":"PanTool"},{"id":"268bb071-8229-41d7-81ce-b1a2b2b810d9","type":"WheelZoomTool"},{"id":"8af95738-37f1-4d91-bff1-a0ad5a3357a7","type":"BoxZoomTool"},{"id":"45ddb6b8-b536-4147-a7e6-f7302b12e46a","type":"SaveTool"},{"id":"19085b66-4288-4213-ba84-7d22eb1a99ac","type":"ResetTool"},{"id":"8c843b78-9689-42c8-a19b-383f83ed3cae","type":"HelpTool"},{"id":"8361b22b-3ff7-4956-88ad-047a13352874","type":"PanTool"},{"id":"775682b6-14e9-4776-8728-d6245765d7d7","type":"WheelZoomTool"},{"id":"ed4b4a35-3f55-4984-b0c8-a056878f6a04","type":"BoxZoomTool"},{"id":"470cd535-e7e1-4478-b54d-b49a97059204","type":"SaveTool"},{"id":"3ce9e121-6fd7-401f-a547-da593e4dc77b","type":"ResetTool"},{"id":"16d88d36-65a2-4f3e-b237-11a79023b49a","type":"HelpTool"},{"id":"322e809f-122d-44ab-adfb-fff13baa036d","type":"PanTool"},{"id":"e612c372-7409-4d8a-944c-1206bca538ab","type":"WheelZoomTool"},{"id":"58a240d7-30be-4420-b93a-ca1289863d13","type":"BoxZoomTool"},{"id":"86c94690-79f6-4288-ba82-e17a4d185cb9","type":"SaveTool"},{"id":"edc59f29-54a7-45a3-84bb-03fcb65ef6df","type":"ResetTool"},{"id":"b4032102-911b-4c4c-8f60-4ecff36ecb9b","type":"HelpTool"},{"id":"6b4da656-890a-4f15-8308-bba17bf46855","type":"PanTool"},{"id":"a81c27f4-60d4-49ea-a1a6-2ac54ae5f301","type":"WheelZoomTool"},{"id":"62446d6b-d3d2-4d9b-91a7-1846e468a714","type":"BoxZoomTool"},{"id":"64b8c2ea-0b92-4c14-9235-028576947076","type":"SaveTool"},{"id":"9b3a52e3-e27a-4f6b-b6ba-e0db55d8bed9","type":"ResetTool"},{"id":"92a69e8a-7390-4350-9ad6-284193c2055f","type":"HelpTool"},{"id":"dc08ebd3-6a70-4b66-9ed5-ab2033b02617","type":"PanTool"},{"id":"261f447f-6e57-4caa-9612-990ac53104cc","type":"WheelZoomTool"},{"id":"ef6a0b65-2aec-4574-8101-1c476620c1ed","type":"BoxZoomTool"},{"id":"4c7d084f-250c-4af4-beb5-bd802fa783d4","type":"SaveTool"},{"id":"6df2cb4a-cb98-4ae4-bb00-2a88c169f1d5","type":"ResetTool"},{"id":"217745ac-c1b5-4fac-a108-b550990c8efa","type":"HelpTool"}]},"id":"f9c919ee-8bdd-4d0f-b48c-bc38b6926c14","type":"ToolbarBox"},{"attributes":{"children":[{"id":"e4f2702d-389c-44d0-b7f1-a01a1d855234","type":"WidgetBox"},{"id":"b2d08de9-8122-4c7e-8580-06583132cad3","type":"WidgetBox"}],"sizing_mode":"scale_width"},"id":"5329528e-3df4-406e-b886-28d673e4447c","type":"Row"},{"attributes":{"children":[{"id":"7fc598d6-b20a-457d-80b4-34bf7c5ca49e","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"5392f54b-cdeb-4555-b378-01c7d6dede56","type":"WidgetBox"},{"attributes":{"label":{"value":"100ms"},"renderers":[{"id":"a6a7460d-9127-40a4-a3b5-041c6162c91d","type":"GlyphRenderer"}]},"id":"f9533df3-a6f2-407a-a408-6a557236dd27","type":"LegendItem"},{"attributes":{"callback":null,"start":0},"id":"ae5f6fd6-f43f-466a-ba47-9ec627c63007","type":"DataRange1d"},{"attributes":{"label":{"value":"df.groupby(0)[1].mean()"},"renderers":[{"id":"05ced829-496d-462f-ae2d-1f5499a1f090","type":"GlyphRenderer"}]},"id":"0dee6c19-e80f-4b2a-8572-c5cd37b26727","type":"LegendItem"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"ce6d9538-d93f-4540-9638-af71fd1def1d","type":"Line"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"07d91c02-e062-456a-9384-315be72b3083","type":"Line"},{"attributes":{"data_source":{"id":"c661d9d5-4983-489a-9fcf-603afdd0f6d9","type":"ColumnDataSource"},"glyph":{"id":"2870733a-83c0-42fa-b0c5-46055057169f","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"21f20435-a99a-4de2-bb0e-d4e943c3e436","type":"Line"},"selection_glyph":null},"id":"acfdfd9a-88eb-4304-8a3e-84611f33f47b","type":"GlyphRenderer"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"001b88c9-3f27-41f4-926b-0ac15ed5689f","type":"Line"},{"attributes":{},"id":"68486794-1408-4221-b928-fa766fa764ad","type":"LinearScale"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"kTbDifc1jEA5B/1PBoKaQOuHU50ntahAZ9QJI5VLtkAx9mM3CDnEQLuRZacBKdBAqmH6cOlv00AR3Nhh5G/eQEoAPNDmhd1A","dtype":"float64","shape":[9]}}},"id":"a923277a-6ebf-49d3-931e-6010777b924e","type":"ColumnDataSource"},{"attributes":{"callback":null,"end":26586.15297054953,"start":0},"id":"749e5bb6-c494-4e2e-9e01-e5861c6f02f1","type":"DataRange1d"},{"attributes":{"line_color":{"value":"#2ca02c"},"x":{"field":"x"},"y":{"field":"y"}},"id":"af8ec2ab-0eb9-4319-8850-85c543cf609e","type":"Line"},{"attributes":{},"id":"d67d8d4b-835a-41e8-8bc5-d4a6faf5158f","type":"LinearScale"},{"attributes":{"data_source":{"id":"b0869052-3aa1-458f-84c4-0ad2af85c8a4","type":"ColumnDataSource"},"glyph":{"id":"001b88c9-3f27-41f4-926b-0ac15ed5689f","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"ce6d9538-d93f-4540-9638-af71fd1def1d","type":"Line"},"selection_glyph":null},"id":"a6a7460d-9127-40a4-a3b5-041c6162c91d","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"764739d9-0946-4e3c-92d5-e83dab730b60","subtype":"Figure","type":"Plot"},"ticker":{"id":"f9d318ce-04f0-482f-a599-56fd4fa97d31","type":"BasicTicker"}},"id":"940bc7b4-5373-490f-881f-eef70bc28979","type":"Grid"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"4ee4b42b-979d-4f58-9941-87e79c4e041c","type":"Circle"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"e83adf33-b642-416f-b7d2-66d599f504ca","type":"Line"},{"attributes":{"axis_label":"cores","formatter":{"id":"eb928f75-ac1a-440d-a878-1e362607141a","type":"BasicTickFormatter"},"plot":{"id":"764739d9-0946-4e3c-92d5-e83dab730b60","subtype":"Figure","type":"Plot"},"ticker":{"id":"89edb50a-f462-4468-9f7d-c3780147d18c","type":"FixedTicker"}},"id":"9ec87ca6-9a84-4edf-84fa-bec88e938f62","type":"LinearAxis"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"5bc49d92-fc97-4b58-a172-03a07173681b","type":"Circle"},{"attributes":{},"id":"f9d318ce-04f0-482f-a599-56fd4fa97d31","type":"BasicTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"CPDOA7wEdkDPvmrzVKmGQPPjMBfneZZAL4RSELzboUAewsNdADiwQGQLuqvxDLlAiDO3aHUuwUBjrjECPpnGQMOz+hDDrsRA","dtype":"float64","shape":[9]}}},"id":"15c59cd6-40c0-409b-b871-6db811ffa5b0","type":"ColumnDataSource"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"89e861d7-44d0-4fd6-b419-f305fd33df7c","type":"BasicTickFormatter"},"plot":{"id":"764739d9-0946-4e3c-92d5-e83dab730b60","subtype":"Figure","type":"Plot"},"ticker":{"id":"40bb2014-3905-4a83-bf53-5035bd8bbd4e","type":"BasicTicker"}},"id":"8b50a8f3-cb96-4f46-b0eb-421532888322","type":"LinearAxis"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"9d53358f-a4b9-4b60-ac35-5f0e5f26d9fe","type":"PanTool"},{"id":"71b8f90f-69b5-4a11-b22e-9c8037bd5bef","type":"WheelZoomTool"},{"id":"09ada8fa-169d-49df-abc7-0703b7616c81","type":"BoxZoomTool"},{"id":"168f662e-50de-4018-b25f-82741503df6e","type":"SaveTool"},{"id":"ff50f331-ff6c-4fc5-9f0d-272907ba7131","type":"ResetTool"},{"id":"0a9d28ae-0c46-4fa8-93b9-7586884f78b2","type":"HelpTool"}]},"id":"310269a5-cd66-453b-91ab-8c1a0a8effa2","type":"Toolbar"},{"attributes":{"line_color":{"value":"#2ca02c"},"x":{"field":"x"},"y":{"field":"y"}},"id":"2cbdd32b-6f36-4da1-819d-9f23498a0cf0","type":"Line"},{"attributes":{},"id":"40bb2014-3905-4a83-bf53-5035bd8bbd4e","type":"BasicTicker"},{"attributes":{"below":[{"id":"b9355d3b-be49-47fd-8d7c-aa58437774a1","type":"LinearAxis"}],"left":[{"id":"7f7381b1-b3c1-442b-9895-0807a0393276","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"b9355d3b-be49-47fd-8d7c-aa58437774a1","type":"LinearAxis"},{"id":"7d32d4ca-4bcb-4b99-80f8-4754c9adb392","type":"Grid"},{"id":"7f7381b1-b3c1-442b-9895-0807a0393276","type":"LinearAxis"},{"id":"af6933eb-7eb8-4827-8ca1-0222eae27f31","type":"Grid"},{"id":"31e613bb-109b-4dd8-a7b8-dedb6ac90927","type":"BoxAnnotation"},{"id":"ce4c2571-807d-49e2-86ee-7b9ef2271d9c","type":"Legend"},{"id":"3543525d-36fc-463f-b01a-5ace3e3a5e70","type":"GlyphRenderer"},{"id":"d385e74e-efcb-4bd7-b948-857f407fe29f","type":"GlyphRenderer"},{"id":"a48aa2d3-5a7c-4537-b210-38f11a1832f4","type":"GlyphRenderer"},{"id":"53e417c9-c14d-4f62-b71e-b64f00b18b47","type":"GlyphRenderer"},{"id":"212c18d2-a6f0-4ad8-9c48-a97150375720","type":"GlyphRenderer"},{"id":"297420ff-5aff-4487-b0f0-78eba07a1421","type":"GlyphRenderer"},{"id":"cc17ce54-c27b-4773-ace9-fd26405a4b4d","type":"GlyphRenderer"},{"id":"0e99105b-1eec-4047-8453-f246267675b9","type":"GlyphRenderer"},{"id":"4f423387-fdc4-4b44-b254-02e16e1ed7d5","type":"GlyphRenderer"}],"title":{"id":"b855bdcf-49d0-4831-b74d-e3ae5412f989","type":"Title"},"tool_events":{"id":"721dbc85-e610-47c7-8f41-dda7878cd574","type":"ToolEvents"},"toolbar":{"id":"310269a5-cd66-453b-91ab-8c1a0a8effa2","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"48ea96b9-707e-4d47-8ccb-4694ea2ba5ba","type":"DataRange1d"},"x_scale":{"id":"40f1492c-7169-4025-83b2-48425797ef38","type":"LinearScale"},"y_range":{"id":"c3a76bce-924f-4592-8dca-1e241e089d42","type":"DataRange1d"},"y_scale":{"id":"52e94f31-a4bd-4799-a606-d59624f1d7da","type":"LinearScale"}},"id":"dbde8c5e-a16f-4c03-9c99-8f0f50a1bbd4","subtype":"Figure","type":"Plot"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"PhAgSQGEM0BmSs6U6HdDQAwo6ea6PVNAoNqNGAnLYkDbj4LPiEVyQN1kKI9W54BAZTy4mhPIjED7LEuHcqKWQMQ4SLgPgZ5A","dtype":"float64","shape":[9]}}},"id":"01406b6d-6a17-4881-978b-e7d727a550cb","type":"ColumnDataSource"},{"attributes":{"dimension":1,"plot":{"id":"764739d9-0946-4e3c-92d5-e83dab730b60","subtype":"Figure","type":"Plot"},"ticker":{"id":"40bb2014-3905-4a83-bf53-5035bd8bbd4e","type":"BasicTicker"}},"id":"badd4c30-b02a-4b7c-b266-8b9a6136e084","type":"Grid"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"58fe4fa5-dcd4-44c0-8470-313aa0bb0b1d","type":"Line"},{"attributes":{"callback":null,"start":0},"id":"48ea96b9-707e-4d47-8ccb-4694ea2ba5ba","type":"DataRange1d"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"57a5cbde-0274-4268-87f8-613a696ecb16","type":"Circle"},{"attributes":{},"id":"eb928f75-ac1a-440d-a878-1e362607141a","type":"BasicTickFormatter"},{"attributes":{"child":{"id":"dbde8c5e-a16f-4c03-9c99-8f0f50a1bbd4","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"26ff1b0e-0ff1-4a7e-be62-eb15afe2b542","type":"Panel"},{"attributes":{},"id":"e8aa3b5e-d639-45de-869e-0d9a1b28a5f9","type":"ToolEvents"},{"attributes":{"data_source":{"id":"1d4b06d5-b304-40e9-8cfd-5b687a0981b5","type":"ColumnDataSource"},"glyph":{"id":"07d91c02-e062-456a-9384-315be72b3083","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"58fe4fa5-dcd4-44c0-8470-313aa0bb0b1d","type":"Line"},"selection_glyph":null},"id":"a881979e-2750-4ccf-b89d-e358d963f201","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"kTbDifc1jEA5B/1PBoKaQOuHU50ntahAZ9QJI5VLtkAx9mM3CDnEQLuRZacBKdBAqmH6cOlv00AR3Nhh5G/eQEoAPNDmhd1A","dtype":"float64","shape":[9]}}},"id":"acde6a51-8f85-4dc7-a6ba-f8d7ad9f2648","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,463070.74804554624]}},"id":"e1013d68-16ec-4e99-89b3-b9cc61b6aaa4","type":"ColumnDataSource"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"6e2d097b-519f-4d25-9d8f-71339b740fee","type":"BoxAnnotation"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"77a721b5-c3cf-4306-aec5-1a831b2890e6","type":"Line"},{"attributes":{"data_source":{"id":"a28d6861-99d3-4706-be92-9be70b1e5cb8","type":"ColumnDataSource"},"glyph":{"id":"2cbdd32b-6f36-4da1-819d-9f23498a0cf0","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"43b4164d-0a5c-489b-ad75-c5850978a99d","type":"Line"},"selection_glyph":null},"id":"494cd700-056c-434d-bb7e-5fd639a6400e","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"764739d9-0946-4e3c-92d5-e83dab730b60","subtype":"Figure","type":"Plot"}},"id":"9cdbfea0-78a1-4816-95b3-20711d60ba64","type":"PanTool"},{"attributes":{},"id":"2271dbe4-edd0-41c8-b581-78f6706f05cc","type":"BasicTicker"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"703f3b02-e1f8-40f8-8191-a6fd37198ded","type":"Circle"},{"attributes":{"plot":{"id":"44d846b0-5ac0-4386-b977-fefe682eb1fa","subtype":"Figure","type":"Plot"},"ticker":{"id":"c7cd4249-3e45-47ad-b403-7086b88c1c1d","type":"LogTicker"}},"id":"e7ef135f-2aa1-4d9b-9c96-514f9a5702e4","type":"Grid"},{"attributes":{"data_source":{"id":"0b519c29-087a-4f44-8156-70db42cbcd01","type":"ColumnDataSource"},"glyph":{"id":"58a16226-cc0a-49f9-8e51-27fd1224a9e5","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"119e9dc5-2f9b-4e29-9aeb-5d23a8de4ccf","type":"Line"},"selection_glyph":null},"id":"81e42002-a8be-4345-b4e1-df093044c5f1","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"01406b6d-6a17-4881-978b-e7d727a550cb","type":"ColumnDataSource"},"glyph":{"id":"57a5cbde-0274-4268-87f8-613a696ecb16","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"703f3b02-e1f8-40f8-8191-a6fd37198ded","type":"Circle"},"selection_glyph":null},"id":"a6539ac4-c610-470f-a3a3-c918c3b7e923","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"764739d9-0946-4e3c-92d5-e83dab730b60","subtype":"Figure","type":"Plot"}},"id":"d917a161-3ebd-4ff0-9e66-b1890604b6c7","type":"WheelZoomTool"},{"attributes":{"plot":null,"text":"DataFrames: Reductions"},"id":"b855bdcf-49d0-4831-b74d-e3ae5412f989","type":"Title"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"78faa5a9-3f86-4363-9f9e-f3ec7d2e2979","type":"FixedTicker"},{"attributes":{"overlay":{"id":"6e2d097b-519f-4d25-9d8f-71339b740fee","type":"BoxAnnotation"},"plot":{"id":"764739d9-0946-4e3c-92d5-e83dab730b60","subtype":"Figure","type":"Plot"}},"id":"c2e5b325-c321-4c08-b3da-cdf89da70574","type":"BoxZoomTool"},{"attributes":{"axis_label":"cores","formatter":{"id":"bf3acc1e-c75c-4182-8881-12c577ee7678","type":"LogTickFormatter"},"plot":{"id":"44d846b0-5ac0-4386-b977-fefe682eb1fa","subtype":"Figure","type":"Plot"},"ticker":{"id":"7a39f017-1a6a-48be-b9dc-2cfcffd5a9d0","type":"FixedTicker"}},"id":"1a1d9b6b-53b1-43ed-90a4-9884182fa0f3","type":"LogAxis"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"1Kb//XpDnED+4P/fNNapQHf4V3ait6lAprCFQjc6r0DGsrWLmNarQOaB18HP4q1ArjxETcD/rEC2aNo+EvGrQGxqBt5Qg6tA","dtype":"float64","shape":[9]}}},"id":"ba10e327-d886-4fb0-83d6-95a0d28b2077","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"764739d9-0946-4e3c-92d5-e83dab730b60","subtype":"Figure","type":"Plot"}},"id":"60a7d2f5-41bc-4215-ac31-864b2bf34829","type":"SaveTool"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"0bc873c0-748f-492a-a171-c53189241066","type":"Circle"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"1Kb//XpDnED+4P/fNNapQHf4V3ait6lAprCFQjc6r0DGsrWLmNarQOaB18HP4q1ArjxETcD/rEC2aNo+EvGrQGxqBt5Qg6tA","dtype":"float64","shape":[9]}}},"id":"a28d6861-99d3-4706-be92-9be70b1e5cb8","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"764739d9-0946-4e3c-92d5-e83dab730b60","subtype":"Figure","type":"Plot"}},"id":"66346441-b239-489b-900f-c4d4660d7478","type":"ResetTool"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"43b4164d-0a5c-489b-ad75-c5850978a99d","type":"Line"},{"attributes":{"plot":{"id":"764739d9-0946-4e3c-92d5-e83dab730b60","subtype":"Figure","type":"Plot"}},"id":"9e9eec61-23ba-4987-af83-bc09e49775bd","type":"HelpTool"},{"attributes":{"callback":null,"end":54450.69515151354,"start":0},"id":"30b0e041-8696-46b0-afac-b993237dd5e2","type":"DataRange1d"},{"attributes":{"data_source":{"id":"e1013d68-16ec-4e99-89b3-b9cc61b6aaa4","type":"ColumnDataSource"},"glyph":{"id":"92b1ebc2-f5c3-4338-aa83-522144f6edf8","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"e83adf33-b642-416f-b7d2-66d599f504ca","type":"Line"},"selection_glyph":null},"id":"abc7b321-64df-4b9f-af6b-1c535dfa2c81","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"0/MveXGpk0CwnWLotLafQFzSKK/0oatAGQzzlhiFvEBuV2Og1WzIQFHk16Cvv8pAs1osroIq1EBv606nwBHYQN/8RMqJ9tlA","dtype":"float64","shape":[9]}}},"id":"1d4b06d5-b304-40e9-8cfd-5b687a0981b5","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"acde6a51-8f85-4dc7-a6ba-f8d7ad9f2648","type":"ColumnDataSource"},"glyph":{"id":"5bc49d92-fc97-4b58-a172-03a07173681b","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"0bc873c0-748f-492a-a171-c53189241066","type":"Circle"},"selection_glyph":null},"id":"d7a8e78a-acbf-40cf-8053-6ee0e66f2b88","type":"GlyphRenderer"},{"attributes":{"label":{"value":"1us"},"renderers":[{"id":"494cd700-056c-434d-bb7e-5fd639a6400e","type":"GlyphRenderer"}]},"id":"0cb38f80-7295-488c-94f0-6da0bc31d650","type":"LegendItem"},{"attributes":{"label":{"value":"x[12345, 23456]"},"renderers":[{"id":"bbf95936-5035-451e-a57a-3569ca7497d0","type":"GlyphRenderer"}]},"id":"aea13c1c-9ff9-4089-b0a7-e253c556541e","type":"LegendItem"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"51bde773-d544-40ca-95ac-f509ded2ce8f","type":"FixedTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,4996.005022052732]}},"id":"4b8c6c40-6a39-4ed8-bea3-cd03b5cdcf20","type":"ColumnDataSource"},{"attributes":{},"id":"89e861d7-44d0-4fd6-b419-f305fd33df7c","type":"BasicTickFormatter"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,90187.75092977297]}},"id":"c661d9d5-4983-489a-9fcf-603afdd0f6d9","type":"ColumnDataSource"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"4f57502d-f735-4ccc-82ca-e963c02312e8","type":"Circle"},{"attributes":{},"id":"20705683-411c-4b3d-8b0f-bae6aad074a7","type":"LogScale"},{"attributes":{"data_source":{"id":"ce4017e6-9e2d-42c6-90aa-b10b01a952d1","type":"ColumnDataSource"},"glyph":{"id":"7a2b565a-76a5-47cc-b363-6c71c9e408be","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"79773228-b238-46de-a46e-225b8a19bdec","type":"Circle"},"selection_glyph":null},"id":"1220894e-d5c5-4272-bf99-4f2213703bda","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"ba10e327-d886-4fb0-83d6-95a0d28b2077","type":"ColumnDataSource"},"glyph":{"id":"4f57502d-f735-4ccc-82ca-e963c02312e8","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"1e078d9d-e2c4-45ca-8d6f-79d782b698b9","type":"Circle"},"selection_glyph":null},"id":"ace2b496-86ba-4ad5-a5b7-25ca170cf0ff","type":"GlyphRenderer"},{"attributes":{"items":[{"id":"0f3ae3ee-88b3-4931-a341-7e5f4501bca1","type":"LegendItem"},{"id":"4f1c1d69-f1c9-4f18-a4ff-7b464185c69a","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"764739d9-0946-4e3c-92d5-e83dab730b60","subtype":"Figure","type":"Plot"}},"id":"12862a29-a060-4fff-b654-18b2c2b74aa7","type":"Legend"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"21f20435-a99a-4de2-bb0e-d4e943c3e436","type":"Line"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"1e078d9d-e2c4-45ca-8d6f-79d782b698b9","type":"Circle"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"c4dddac3-6ae5-4394-81ee-7b43d84e191c","type":"Line"},{"attributes":{"callback":null,"end":31167.568472113522,"start":0},"id":"c3a76bce-924f-4592-8dca-1e241e089d42","type":"DataRange1d"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"92b1ebc2-f5c3-4338-aa83-522144f6edf8","type":"Line"},{"attributes":{"label":{"value":"x.std()"},"renderers":[{"id":"a881979e-2750-4ccf-b89d-e358d963f201","type":"GlyphRenderer"}]},"id":"0f3ae3ee-88b3-4931-a341-7e5f4501bca1","type":"LegendItem"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"2870733a-83c0-42fa-b0c5-46055057169f","type":"Line"},{"attributes":{"data_source":{"id":"2366c833-409a-45a4-a48d-3dafbf3758b3","type":"ColumnDataSource"},"glyph":{"id":"3b61a9c5-5f13-478d-a23a-c2aec972ca77","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"2bc79701-541f-4812-aed0-f3294aef0fcd","type":"Circle"},"selection_glyph":null},"id":"60298c59-be4e-4074-a111-cf48a70a3975","type":"GlyphRenderer"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"58a16226-cc0a-49f9-8e51-27fd1224a9e5","type":"Line"},{"attributes":{},"id":"40f1492c-7169-4025-83b2-48425797ef38","type":"LinearScale"},{"attributes":{"data_source":{"id":"aa15e897-6c86-46d5-9c8a-1976053e42e0","type":"ColumnDataSource"},"glyph":{"id":"bf1999e5-e42b-464b-9db0-ea3083c3c1d1","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"7abab916-ce13-45b2-b8d4-a4898c863064","type":"Line"},"selection_glyph":null},"id":"7ab82ebf-e918-475c-bc9e-b885ae95fd38","type":"GlyphRenderer"},{"attributes":{"callback":null,"start":0},"id":"99879e3e-cb86-4ed8-a9ec-b60d8ff48f74","type":"DataRange1d"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"txeRHSzYhUCjQkyym/ODQDXnP5WM5IpAL2chJm3jhEB0ZkdqW+WBQI2SGi6KzoBAUViGGKc/hEDneAuUg4eCQAnnNrBvOH1A","dtype":"float64","shape":[9]}}},"id":"5649e48b-92ca-4927-a7df-20ef2fd756e0","type":"ColumnDataSource"},{"attributes":{"num_minor_ticks":10},"id":"c7cd4249-3e45-47ad-b403-7086b88c1c1d","type":"LogTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"gT1T0rcmkkBfZLyai4CeQPrpFfFJcqpAtJAQZsY2tkD936fnDJnCQEN9+GXL79BAP/LDy9Ijz0A7bbDIfRrUQJv5zSYbI9NA","dtype":"float64","shape":[9]}}},"id":"b75993a1-dab9-4167-a4ff-744b0f50fd93","type":"ColumnDataSource"},{"attributes":{"items":[{"id":"d82896f9-20c8-4dde-9f78-84e02a60992d","type":"LegendItem"},{"id":"975dd06b-ac52-4693-963f-4c915b65d961","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"0a3bfea8-d1bb-4d64-a752-ba72a5231006","subtype":"Figure","type":"Plot"}},"id":"b142de0f-ddfe-4ba6-9b4e-21e324ef3bb7","type":"Legend"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"623ace2e-2755-4bd6-acb7-31ce376f74e0","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"119e9dc5-2f9b-4e29-9aeb-5d23a8de4ccf","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"1f09cd72-46d5-44f3-bc9c-3e031e52d214","type":"Line"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"2bc79701-541f-4812-aed0-f3294aef0fcd","type":"Circle"},{"attributes":{"items":[{"id":"aea13c1c-9ff9-4089-b0a7-e253c556541e","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"32468298-7c8b-4961-a1e3-2247d13004a8","subtype":"Figure","type":"Plot"}},"id":"2597e3d6-7fdb-4bf5-b6f7-59aa315a357d","type":"Legend"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"4251447a-159a-41b2-933e-dc4d435891ab","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"29f934af-0a2f-475c-b30f-41c636927b76","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"gT1T0rcmkkBfZLyai4CeQPrpFfFJcqpAtJAQZsY2tkD936fnDJnCQEN9+GXL79BAP/LDy9Ijz0A7bbDIfRrUQJv5zSYbI9NA","dtype":"float64","shape":[9]}}},"id":"0b519c29-087a-4f44-8156-70db42cbcd01","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,231102.94226687078]}},"id":"32c9197d-5315-41b6-9e6c-c2737380b947","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"4b8c6c40-6a39-4ed8-bea3-cd03b5cdcf20","type":"ColumnDataSource"},"glyph":{"id":"623ace2e-2755-4bd6-acb7-31ce376f74e0","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"29f934af-0a2f-475c-b30f-41c636927b76","type":"Line"},"selection_glyph":null},"id":"60b83405-6ab4-4cc9-a41c-02803e9efe91","type":"GlyphRenderer"},{"attributes":{"label":{"value":"x.std(axis=0)"},"renderers":[{"id":"81e42002-a8be-4345-b4e1-df093044c5f1","type":"GlyphRenderer"}]},"id":"4f1c1d69-f1c9-4f18-a4ff-7b464185c69a","type":"LegendItem"},{"attributes":{},"id":"52e94f31-a4bd-4799-a606-d59624f1d7da","type":"LinearScale"},{"attributes":{"callback":null,"start":0},"id":"59143419-5a59-4079-b1b6-0baec3f3748d","type":"DataRange1d"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"90385b33-d105-4e0d-a88c-a8eab6abc79b","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"663560ce-17e4-47cd-9a05-eb90e0101f4c","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"txeRHSzYhUCjQkyym/ODQDXnP5WM5IpAL2chJm3jhEB0ZkdqW+WBQI2SGi6KzoBAUViGGKc/hEDneAuUg4eCQAnnNrBvOH1A","dtype":"float64","shape":[9]}}},"id":"26b4c843-d146-42c6-9f7c-a5c786a8ed80","type":"ColumnDataSource"},{"attributes":{"below":[{"id":"f9ceb0bb-d6df-4bd8-99d2-19e4daff9739","type":"LinearAxis"}],"left":[{"id":"d69ad060-24dc-4df7-b601-5ad79928397a","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"f9ceb0bb-d6df-4bd8-99d2-19e4daff9739","type":"LinearAxis"},{"id":"c35e9a65-e8cf-47d5-ae34-2bcc54919712","type":"Grid"},{"id":"d69ad060-24dc-4df7-b601-5ad79928397a","type":"LinearAxis"},{"id":"c0978681-84c6-4bb7-bdd3-57f79db94c3c","type":"Grid"},{"id":"d8b19c6c-70f9-41cb-aba5-a65bbc695962","type":"BoxAnnotation"},{"id":"17dd8ac4-5015-44f3-b808-be58f596fc84","type":"Legend"},{"id":"587410b9-0902-470c-bf93-59d3ebbd3ed6","type":"GlyphRenderer"},{"id":"60298c59-be4e-4074-a111-cf48a70a3975","type":"GlyphRenderer"},{"id":"3e3a4880-9622-4757-8216-1579e675e963","type":"GlyphRenderer"},{"id":"6cdd9453-036b-46a1-8702-56743bbbbb70","type":"GlyphRenderer"},{"id":"d1ebdf62-fdde-4a98-b4cb-020bb3f8a1a8","type":"GlyphRenderer"},{"id":"456a4a3c-1e23-4e6a-bedd-81c07d21ef13","type":"GlyphRenderer"},{"id":"709df9d8-d49b-4549-8089-964dc4ed02d1","type":"GlyphRenderer"},{"id":"0d73ec87-f247-42dd-aa8e-6a30c4849c0e","type":"GlyphRenderer"},{"id":"36248ec8-5457-4eb1-adb0-908e0f6b885e","type":"GlyphRenderer"}],"title":{"id":"967319c6-0c31-492f-b322-f8f817477f2d","type":"Title"},"tool_events":{"id":"e8aa3b5e-d639-45de-869e-0d9a1b28a5f9","type":"ToolEvents"},"toolbar":{"id":"f5b16683-200e-4e29-81cb-326fec0bed88","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"59143419-5a59-4079-b1b6-0baec3f3748d","type":"DataRange1d"},"x_scale":{"id":"0f59b806-374d-4a2d-ab91-87a07c54fe51","type":"LinearScale"},"y_range":{"id":"07e1eb75-b4b6-44ce-a243-c0bd7102bfa3","type":"DataRange1d"},"y_scale":{"id":"9b40eff8-fd36-462b-ba68-b6b1ff7caee3","type":"LinearScale"}},"id":"6340551f-85a0-416d-aac2-8c0234cd2408","subtype":"Figure","type":"Plot"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"fdf3243b-03c6-4e71-8bb1-8a5bc2723d4e","type":"Circle"},{"attributes":{},"id":"721dbc85-e610-47c7-8f41-dda7878cd574","type":"ToolEvents"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"8946c13e-3fd9-4cd2-ad6b-048b3f3d3184","type":"PanTool"},{"id":"18467990-49d6-4857-8698-0ecf850b8dae","type":"WheelZoomTool"},{"id":"f12f9873-2e5a-4d0e-b72a-022169603cba","type":"BoxZoomTool"},{"id":"70ca44b0-73ae-499b-bf6e-a24ee883237a","type":"SaveTool"},{"id":"0888f93d-b6d7-486b-b5f6-52007244ea56","type":"ResetTool"},{"id":"12b05cc5-f225-412a-a571-bb01258df41c","type":"HelpTool"}]},"id":"f5b16683-200e-4e29-81cb-326fec0bed88","type":"Toolbar"},{"attributes":{"data_source":{"id":"8f276392-afce-4e80-935d-28392f36571b","type":"ColumnDataSource"},"glyph":{"id":"90385b33-d105-4e0d-a88c-a8eab6abc79b","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"f0ebf59d-e2e3-4945-9f9e-9f549d51541a","type":"Line"},"selection_glyph":null},"id":"7a8d5d6f-8fb1-4d50-88a8-5a53336ee7fa","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"32c9197d-5315-41b6-9e6c-c2737380b947","type":"ColumnDataSource"},"glyph":{"id":"4251447a-159a-41b2-933e-dc4d435891ab","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"1f09cd72-46d5-44f3-bc9c-3e031e52d214","type":"Line"},"selection_glyph":null},"id":"0ff3ae63-4217-4531-910b-ebe12ee6c8bd","type":"GlyphRenderer"},{"attributes":{},"id":"f7835f9e-6dfd-43fc-a5c2-b8ac698081c6","type":"BasicTickFormatter"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"331fcea2-8946-4bfe-9335-c9556c0aac3b","type":"Circle"},{"attributes":{"plot":null,"text":"Tasks: Embarrassingly Parallel"},"id":"967319c6-0c31-492f-b322-f8f817477f2d","type":"Title"},{"attributes":{"data_source":{"id":"b75993a1-dab9-4167-a4ff-744b0f50fd93","type":"ColumnDataSource"},"glyph":{"id":"fdf3243b-03c6-4e71-8bb1-8a5bc2723d4e","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"331fcea2-8946-4bfe-9335-c9556c0aac3b","type":"Circle"},"selection_glyph":null},"id":"72786059-6377-45c0-9914-95820228350f","type":"GlyphRenderer"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"3ba055f8-c19e-4f01-80ac-432301ad1555","type":"Line"},{"attributes":{},"id":"0f59b806-374d-4a2d-ab91-87a07c54fe51","type":"LinearScale"},{"attributes":{"ticker":null},"id":"f9fcb2a3-381c-4e09-99c0-5f86896d31d0","type":"LogTickFormatter"},{"attributes":{"callback":null,"end":3997.1079293993125,"start":0},"id":"07e1eb75-b4b6-44ce-a243-c0bd7102bfa3","type":"DataRange1d"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"txeRHSzYhUCjQkyym/ODQDXnP5WM5IpAL2chJm3jhEB0ZkdqW+WBQI2SGi6KzoBAUViGGKc/hEDneAuUg4eCQAnnNrBvOH1A","dtype":"float64","shape":[9]}}},"id":"9dce6d6a-a1f8-4fe1-a323-dee8ffeb69ab","type":"ColumnDataSource"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"9dde85bf-6688-4bae-b751-cbf4ecd3c2bd","type":"Line"},{"attributes":{},"id":"9b40eff8-fd36-462b-ba68-b6b1ff7caee3","type":"LinearScale"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,322140.36834698654]}},"id":"8f276392-afce-4e80-935d-28392f36571b","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"6340551f-85a0-416d-aac2-8c0234cd2408","subtype":"Figure","type":"Plot"},"ticker":{"id":"6d59dcbf-1771-4e95-bb38-55a2b41e58dd","type":"BasicTicker"}},"id":"c35e9a65-e8cf-47d5-ae34-2bcc54919712","type":"Grid"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"f0ebf59d-e2e3-4945-9f9e-9f549d51541a","type":"Line"},{"attributes":{"axis_label":"cores","formatter":{"id":"f7835f9e-6dfd-43fc-a5c2-b8ac698081c6","type":"BasicTickFormatter"},"plot":{"id":"6340551f-85a0-416d-aac2-8c0234cd2408","subtype":"Figure","type":"Plot"},"ticker":{"id":"20d3ccb9-5f8a-499b-baf5-3dcdf25ae187","type":"FixedTicker"}},"id":"f9ceb0bb-d6df-4bd8-99d2-19e4daff9739","type":"LinearAxis"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"5306f2a4-4a35-4dd1-9d4b-6a683dbca024","type":"Line"},{"attributes":{"data_source":{"id":"a923277a-6ebf-49d3-931e-6010777b924e","type":"ColumnDataSource"},"glyph":{"id":"af8ec2ab-0eb9-4319-8850-85c543cf609e","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"77a721b5-c3cf-4306-aec5-1a831b2890e6","type":"Line"},"selection_glyph":null},"id":"05ced829-496d-462f-ae2d-1f5499a1f090","type":"GlyphRenderer"},{"attributes":{},"id":"6d59dcbf-1771-4e95-bb38-55a2b41e58dd","type":"BasicTicker"},{"attributes":{"data_source":{"id":"6010cab9-9325-4735-a3f3-8239177e6e8b","type":"ColumnDataSource"},"glyph":{"id":"5306f2a4-4a35-4dd1-9d4b-6a683dbca024","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"880fff54-54b2-491c-b7e2-bf9d6ca7ad35","type":"Line"},"selection_glyph":null},"id":"123e0dbf-a6dd-488a-88e8-eadf20a2fe99","type":"GlyphRenderer"},{"attributes":{"axis_label":"tasks/s","formatter":{"id":"4e1f28d6-af74-478c-899b-a98e1cc3e250","type":"BasicTickFormatter"},"plot":{"id":"6340551f-85a0-416d-aac2-8c0234cd2408","subtype":"Figure","type":"Plot"},"ticker":{"id":"e3c6b138-1bae-4b8f-ad55-97d925753006","type":"BasicTicker"}},"id":"d69ad060-24dc-4df7-b601-5ad79928397a","type":"LinearAxis"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"880fff54-54b2-491c-b7e2-bf9d6ca7ad35","type":"Line"},{"attributes":{},"id":"e3c6b138-1bae-4b8f-ad55-97d925753006","type":"BasicTicker"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"a2174dc6-19a1-44b0-811a-4d939b79bdec","type":"Line"},{"attributes":{"dimension":1,"plot":{"id":"6340551f-85a0-416d-aac2-8c0234cd2408","subtype":"Figure","type":"Plot"},"ticker":{"id":"e3c6b138-1bae-4b8f-ad55-97d925753006","type":"BasicTicker"}},"id":"c0978681-84c6-4bb7-bdd3-57f79db94c3c","type":"Grid"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"89edb50a-f462-4468-9f7d-c3780147d18c","type":"FixedTicker"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"828a515a-f371-424b-b63a-aa1088e19bd7","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,297389.95539566135]}},"id":"6010cab9-9325-4735-a3f3-8239177e6e8b","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"33c8a65f-1b5c-4354-8f44-cc9f8b4db9f0","type":"ColumnDataSource"},"glyph":{"id":"828a515a-f371-424b-b63a-aa1088e19bd7","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"663560ce-17e4-47cd-9a05-eb90e0101f4c","type":"Line"},"selection_glyph":null},"id":"587410b9-0902-470c-bf93-59d3ebbd3ed6","type":"GlyphRenderer"},{"attributes":{"ticker":null},"id":"7823bb98-4116-4089-9937-625fcb9679f0","type":"LogTickFormatter"},{"attributes":{"child":{"id":"6340551f-85a0-416d-aac2-8c0234cd2408","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"e36df08d-6b52-45fd-90e8-47b2576ad722","type":"Panel"},{"attributes":{"below":[{"id":"5242f714-cfa3-4375-8d8b-c774d1daa5cc","type":"LogAxis"}],"left":[{"id":"ffe59c70-5de8-4542-8ed3-f20a2c9d112a","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"5242f714-cfa3-4375-8d8b-c774d1daa5cc","type":"LogAxis"},{"id":"be11599d-6a02-4b3f-94bc-d8073d089be2","type":"Grid"},{"id":"ffe59c70-5de8-4542-8ed3-f20a2c9d112a","type":"LogAxis"},{"id":"bb28c07a-adf0-4c5f-8192-0278a75f4a80","type":"Grid"},{"id":"0cc6521b-a666-4a7b-a5d3-503c5254fe2c","type":"BoxAnnotation"},{"id":"2597e3d6-7fdb-4bf5-b6f7-59aa315a357d","type":"Legend"},{"id":"bbf95936-5035-451e-a57a-3569ca7497d0","type":"GlyphRenderer"},{"id":"48a027b0-fa86-43fd-86fc-e465a24274e1","type":"GlyphRenderer"},{"id":"c97583c4-51e9-4cff-9769-b4b540deaac8","type":"GlyphRenderer"}],"title":{"id":"b4046279-007b-408b-8cf8-fd84395c0440","type":"Title"},"tool_events":{"id":"bd08573e-dca8-4504-9fde-ba11b52f83e2","type":"ToolEvents"},"toolbar":{"id":"a38a7d79-9cee-46a7-b6ec-37c882d77164","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"601a8953-512e-4f09-b0c0-d1b38f8ea0d0","type":"DataRange1d"},"x_scale":{"id":"0b49eb6f-8950-4bb3-a051-c1546acb1148","type":"LogScale"},"y_range":{"id":"28959523-898e-4099-a510-a779775606a4","type":"DataRange1d"},"y_scale":{"id":"5aa2abcf-9ad7-4f75-8dff-2d63f2be6008","type":"LogScale"}},"id":"32468298-7c8b-4961-a1e3-2247d13004a8","subtype":"Figure","type":"Plot"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"d8b19c6c-70f9-41cb-aba5-a65bbc695962","type":"BoxAnnotation"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"37525836-a385-4ab9-81df-19e6291ffdc9","type":"Panel"},{"id":"51a14984-ff50-4ffd-9f85-404e8b4bfba6","type":"Panel"}]},"id":"4311ab7e-c88e-42cf-b077-bfd6acbda2f0","type":"Tabs"},{"attributes":{"plot":{"id":"6340551f-85a0-416d-aac2-8c0234cd2408","subtype":"Figure","type":"Plot"}},"id":"8946c13e-3fd9-4cd2-ad6b-048b3f3d3184","type":"PanTool"},{"attributes":{"child":{"id":"32468298-7c8b-4961-a1e3-2247d13004a8","subtype":"Figure","type":"Plot"},"title":"log"},"id":"37525836-a385-4ab9-81df-19e6291ffdc9","type":"Panel"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"w1YWXnOt/z84PQvNaIsPQIhEPP1NRB9ArYqH3f3CLkDSt1Z9g8U9QH5TAs78FU1AkemwSLPlVkAELKWLNq1mQC1uH4c19HBA","dtype":"float64","shape":[9]}}},"id":"2366c833-409a-45a4-a48d-3dafbf3758b3","type":"ColumnDataSource"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"41b4c7b8-3e47-4ce9-ba5c-75ab1567adb8","type":"Line"},{"attributes":{"plot":{"id":"6340551f-85a0-416d-aac2-8c0234cd2408","subtype":"Figure","type":"Plot"}},"id":"18467990-49d6-4857-8698-0ecf850b8dae","type":"WheelZoomTool"},{"attributes":{"data_source":{"id":"26b4c843-d146-42c6-9f7c-a5c786a8ed80","type":"ColumnDataSource"},"glyph":{"id":"c0faf656-4336-42f4-a308-ab59b7805475","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"41b4c7b8-3e47-4ce9-ba5c-75ab1567adb8","type":"Line"},"selection_glyph":null},"id":"bbf95936-5035-451e-a57a-3569ca7497d0","type":"GlyphRenderer"},{"attributes":{"overlay":{"id":"d8b19c6c-70f9-41cb-aba5-a65bbc695962","type":"BoxAnnotation"},"plot":{"id":"6340551f-85a0-416d-aac2-8c0234cd2408","subtype":"Figure","type":"Plot"}},"id":"f12f9873-2e5a-4d0e-b72a-022169603cba","type":"BoxZoomTool"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"c0faf656-4336-42f4-a308-ab59b7805475","type":"Line"},{"attributes":{"plot":{"id":"6340551f-85a0-416d-aac2-8c0234cd2408","subtype":"Figure","type":"Plot"}},"id":"70ca44b0-73ae-499b-bf6e-a24ee883237a","type":"SaveTool"},{"attributes":{"plot":null,"text":"Arrays: Random Access"},"id":"b4046279-007b-408b-8cf8-fd84395c0440","type":"Title"},{"attributes":{"plot":{"id":"6340551f-85a0-416d-aac2-8c0234cd2408","subtype":"Figure","type":"Plot"}},"id":"0888f93d-b6d7-486b-b5f6-52007244ea56","type":"ResetTool"},{"attributes":{"callback":null,"start":0},"id":"601a8953-512e-4f09-b0c0-d1b38f8ea0d0","type":"DataRange1d"},{"attributes":{"plot":{"id":"6340551f-85a0-416d-aac2-8c0234cd2408","subtype":"Figure","type":"Plot"}},"id":"12b05cc5-f225-412a-a571-bb01258df41c","type":"HelpTool"},{"attributes":{},"id":"bd08573e-dca8-4504-9fde-ba11b52f83e2","type":"ToolEvents"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"w1YWXnOt/z84PQvNaIsPQIhEPP1NRB9ArYqH3f3CLkDSt1Z9g8U9QH5TAs78FU1AkemwSLPlVkAELKWLNq1mQC1uH4c19HBA","dtype":"float64","shape":[9]}}},"id":"33c8a65f-1b5c-4354-8f44-cc9f8b4db9f0","type":"ColumnDataSource"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"d7e00d52-b6a0-43d3-b2ce-aba245b42337","type":"PanTool"},{"id":"b288bf15-ef2e-4945-9eb4-ccaf8cd6c334","type":"WheelZoomTool"},{"id":"6007b073-348b-41bc-a701-31198057ecfe","type":"BoxZoomTool"},{"id":"2a2d4a72-6109-438e-aa25-4a5692f74faf","type":"SaveTool"},{"id":"d377ce77-c83a-4022-83a8-021b1f1aa6b0","type":"ResetTool"},{"id":"16a5b5ca-891a-4048-9e4c-749d27fa4e99","type":"HelpTool"}]},"id":"a38a7d79-9cee-46a7-b6ec-37c882d77164","type":"Toolbar"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"PhAgSQGEM0BmSs6U6HdDQAwo6ea6PVNAoNqNGAnLYkDbj4LPiEVyQN1kKI9W54BAZTy4mhPIjED7LEuHcqKWQMQ4SLgPgZ5A","dtype":"float64","shape":[9]}}},"id":"43cd3931-9b82-4894-90ca-dc205db37be1","type":"ColumnDataSource"},{"attributes":{"num_minor_ticks":10},"id":"25639941-7512-4fdc-a73b-680bf1c2b0c4","type":"LogTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,123285.74943667026]}},"id":"c6b1dde5-997a-42d4-a3be-0099f9fafcca","type":"ColumnDataSource"},{"attributes":{},"id":"4e1f28d6-af74-478c-899b-a98e1cc3e250","type":"BasicTickFormatter"},{"attributes":{"num_minor_ticks":10},"id":"180b269d-1945-4fe3-a9a4-96715b0344ae","type":"LogTicker"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"3b61a9c5-5f13-478d-a23a-c2aec972ca77","type":"Circle"},{"attributes":{"callback":null,"end":860.5686440460619,"start":0},"id":"28959523-898e-4099-a510-a779775606a4","type":"DataRange1d"},{"attributes":{"items":[{"id":"89a6234b-fee7-4b95-ae76-2a28b32ccfb9","type":"LegendItem"},{"id":"3dc9f913-a083-4fee-9bdb-73613184eac0","type":"LegendItem"},{"id":"a8dc3374-6ebe-49de-b7b0-c9600eb77fb9","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"6340551f-85a0-416d-aac2-8c0234cd2408","subtype":"Figure","type":"Plot"}},"id":"17dd8ac4-5015-44f3-b808-be58f596fc84","type":"Legend"},{"attributes":{"dimension":1,"plot":{"id":"32468298-7c8b-4961-a1e3-2247d13004a8","subtype":"Figure","type":"Plot"},"ticker":{"id":"180b269d-1945-4fe3-a9a4-96715b0344ae","type":"LogTicker"}},"id":"bb28c07a-adf0-4c5f-8192-0278a75f4a80","type":"Grid"},{"attributes":{"ticker":null},"id":"0ec83177-9fe3-476b-aef4-91a0528ef45d","type":"LogTickFormatter"},{"attributes":{"plot":{"id":"32468298-7c8b-4961-a1e3-2247d13004a8","subtype":"Figure","type":"Plot"},"ticker":{"id":"25639941-7512-4fdc-a73b-680bf1c2b0c4","type":"LogTicker"}},"id":"be11599d-6a02-4b3f-94bc-d8073d089be2","type":"Grid"},{"attributes":{"label":{"value":"1s"},"renderers":[{"id":"587410b9-0902-470c-bf93-59d3ebbd3ed6","type":"GlyphRenderer"}]},"id":"89a6234b-fee7-4b95-ae76-2a28b32ccfb9","type":"LegendItem"},{"attributes":{},"id":"0b49eb6f-8950-4bb3-a051-c1546acb1148","type":"LogScale"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"0ca8d799-8349-44ac-a2a0-a2d717114aca","type":"Line"},{"attributes":{"axis_label":"cores","formatter":{"id":"7823bb98-4116-4089-9937-625fcb9679f0","type":"LogTickFormatter"},"plot":{"id":"32468298-7c8b-4961-a1e3-2247d13004a8","subtype":"Figure","type":"Plot"},"ticker":{"id":"481c2585-6dc3-4cc1-bd72-f61b21d604e8","type":"FixedTicker"}},"id":"5242f714-cfa3-4375-8d8b-c774d1daa5cc","type":"LogAxis"},{"attributes":{"data_source":{"id":"43cd3931-9b82-4894-90ca-dc205db37be1","type":"ColumnDataSource"},"glyph":{"id":"0ca8d799-8349-44ac-a2a0-a2d717114aca","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"b8402b64-0090-41e8-b7dd-10e6ce8ed190","type":"Line"},"selection_glyph":null},"id":"3e3a4880-9622-4757-8216-1579e675e963","type":"GlyphRenderer"},{"attributes":{},"id":"5aa2abcf-9ad7-4f75-8dff-2d63f2be6008","type":"LogScale"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"b8402b64-0090-41e8-b7dd-10e6ce8ed190","type":"Line"},{"attributes":{"axis_label":"bytes/s","formatter":{"id":"f9fcb2a3-381c-4e09-99c0-5f86896d31d0","type":"LogTickFormatter"},"plot":{"id":"32468298-7c8b-4961-a1e3-2247d13004a8","subtype":"Figure","type":"Plot"},"ticker":{"id":"180b269d-1945-4fe3-a9a4-96715b0344ae","type":"LogTicker"}},"id":"ffe59c70-5de8-4542-8ed3-f20a2c9d112a","type":"LogAxis"},{"attributes":{"plot":{"id":"bf138360-c752-43f5-8fb6-8e184f623a29","subtype":"Figure","type":"Plot"},"ticker":{"id":"2babd4e6-afe8-471e-bc2a-78e938844e28","type":"LogTicker"}},"id":"faf2ba21-d806-4401-b08c-f410365985c0","type":"Grid"},{"attributes":{"num_minor_ticks":10},"id":"2babd4e6-afe8-471e-bc2a-78e938844e28","type":"LogTicker"},{"attributes":{"num_minor_ticks":10},"id":"417d361c-4a40-4733-9159-5f1811af26b7","type":"LogTicker"},{"attributes":{},"id":"ac0e2ba7-9ee1-4bea-91e9-941fa39bd039","type":"LogScale"},{"attributes":{"data_source":{"id":"a68fcac2-6ff7-4160-ab3f-ef566bea5cbb","type":"ColumnDataSource"},"glyph":{"id":"18874344-f743-48f0-b079-2842f76113a4","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"ebe800c9-1421-4745-b177-d089dafce886","type":"Line"},"selection_glyph":null},"id":"a34be81d-c82c-49d1-a9f4-8a1badfe7633","type":"GlyphRenderer"},{"attributes":{"dimension":1,"plot":{"id":"bf138360-c752-43f5-8fb6-8e184f623a29","subtype":"Figure","type":"Plot"},"ticker":{"id":"417d361c-4a40-4733-9159-5f1811af26b7","type":"LogTicker"}},"id":"8960a848-f444-45e4-b72b-b24fd0383d3e","type":"Grid"},{"attributes":{"label":{"value":"x + x.T"},"renderers":[{"id":"d5fcae1d-47da-48ad-b0be-3810b2422c05","type":"GlyphRenderer"}]},"id":"3b9f743b-42cd-47ea-b43e-100516cbdeda","type":"LegendItem"},{"attributes":{"items":[{"id":"3b9f743b-42cd-47ea-b43e-100516cbdeda","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"bf138360-c752-43f5-8fb6-8e184f623a29","subtype":"Figure","type":"Plot"}},"id":"9d47119f-4b80-4765-8852-50252e2f4aca","type":"Legend"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"5mBoSrF9l0Bna2xc6/uUQD2eEyk3/Z1A41+WwWA8rkCg3qY5Aga3QAVA74snN8NAncj380uyxEDOuHXJpNfQQN8hVioyK9NA","dtype":"float64","shape":[9]}}},"id":"4a4de69b-8e72-4fd9-8190-3e4c0b557405","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"eb0e208c-28e1-4479-bc5b-348fb9762087","type":"ColumnDataSource"},"glyph":{"id":"368751b3-85a8-4719-9b67-3f8e54c728eb","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"3d141fb6-7f10-43b8-8df2-de69f45d185d","type":"Line"},"selection_glyph":null},"id":"0845f557-d6e7-420f-9522-e6c54163a1cf","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"bf138360-c752-43f5-8fb6-8e184f623a29","subtype":"Figure","type":"Plot"}},"id":"8361b22b-3ff7-4956-88ad-047a13352874","type":"PanTool"},{"attributes":{"data_source":{"id":"4a4de69b-8e72-4fd9-8190-3e4c0b557405","type":"ColumnDataSource"},"glyph":{"id":"aaad284f-67f6-4517-a94f-cadc721226eb","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"186312de-983f-4edf-ae08-fb5ffefa454d","type":"Circle"},"selection_glyph":null},"id":"52752298-39d9-4847-892a-1eba3a73b48b","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"bf138360-c752-43f5-8fb6-8e184f623a29","subtype":"Figure","type":"Plot"}},"id":"775682b6-14e9-4776-8728-d6245765d7d7","type":"WheelZoomTool"},{"attributes":{"overlay":{"id":"70ff432f-e2c0-4f3d-8740-3a64d1b9851b","type":"BoxAnnotation"},"plot":{"id":"bf138360-c752-43f5-8fb6-8e184f623a29","subtype":"Figure","type":"Plot"}},"id":"ed4b4a35-3f55-4984-b0c8-a056878f6a04","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"bf138360-c752-43f5-8fb6-8e184f623a29","subtype":"Figure","type":"Plot"}},"id":"470cd535-e7e1-4478-b54d-b49a97059204","type":"SaveTool"},{"attributes":{"plot":{"id":"bf138360-c752-43f5-8fb6-8e184f623a29","subtype":"Figure","type":"Plot"}},"id":"3ce9e121-6fd7-401f-a547-da593e4dc77b","type":"ResetTool"},{"attributes":{"plot":{"id":"bf138360-c752-43f5-8fb6-8e184f623a29","subtype":"Figure","type":"Plot"}},"id":"16d88d36-65a2-4f3e-b237-11a79023b49a","type":"HelpTool"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"aaad284f-67f6-4517-a94f-cadc721226eb","type":"Circle"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"186312de-983f-4edf-ae08-fb5ffefa454d","type":"Circle"},{"attributes":{"label":{"value":"x + x.T"},"renderers":[{"id":"578be135-9e3c-48f2-b070-14bea88b02d1","type":"GlyphRenderer"}]},"id":"df99970f-4041-44f8-b3a0-0bf1e59c404e","type":"LegendItem"},{"attributes":{"ticker":null},"id":"fb147d32-4323-41ae-9418-c219938a98fc","type":"LogTickFormatter"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"368751b3-85a8-4719-9b67-3f8e54c728eb","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"3d141fb6-7f10-43b8-8df2-de69f45d185d","type":"Line"},{"attributes":{"plot":null,"text":"Arrays: Bulk Communication"},"id":"1fdc5123-d219-45c3-b368-ae3d861946d2","type":"Title"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"baf0ecde-67cc-4c97-afc6-2208995fddb4","type":"FixedTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,384876.3226637974]}},"id":"eb0e208c-28e1-4479-bc5b-348fb9762087","type":"ColumnDataSource"},{"attributes":{"below":[{"id":"c7cad613-5c33-42bc-82d2-9efb32eaa493","type":"LinearAxis"}],"left":[{"id":"bcd530b5-5ba1-432f-ad2e-5b552403d553","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"c7cad613-5c33-42bc-82d2-9efb32eaa493","type":"LinearAxis"},{"id":"71d72d02-5d18-4b43-8947-6c8922496a2d","type":"Grid"},{"id":"bcd530b5-5ba1-432f-ad2e-5b552403d553","type":"LinearAxis"},{"id":"42f34310-41c5-4e5b-8ab3-445c347cc2e7","type":"Grid"},{"id":"8dfd84f0-a5b6-4f38-8e13-8b69195e9d75","type":"BoxAnnotation"},{"id":"79e12eb2-6783-4266-9832-ac658476172b","type":"Legend"},{"id":"578be135-9e3c-48f2-b070-14bea88b02d1","type":"GlyphRenderer"},{"id":"fd8659b9-5774-4ade-9016-a647c66e4fda","type":"GlyphRenderer"},{"id":"a34be81d-c82c-49d1-a9f4-8a1badfe7633","type":"GlyphRenderer"}],"title":{"id":"1fdc5123-d219-45c3-b368-ae3d861946d2","type":"Title"},"tool_events":{"id":"c84634d1-74c1-40d3-a358-ac12699478c5","type":"ToolEvents"},"toolbar":{"id":"d3acbb56-ea5e-42fb-ad0f-1081293957ba","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"dff6e4ba-fb61-418f-ad54-a9e4654930b8","type":"DataRange1d"},"x_scale":{"id":"55f6f30e-c5c2-4a42-bcae-8f7abfbae0aa","type":"LinearScale"},"y_range":{"id":"4ea7ba70-545d-4315-8170-2bc5f7a661eb","type":"DataRange1d"},"y_scale":{"id":"1bb3fef9-ec3c-45de-b36b-1b91a4324738","type":"LinearScale"}},"id":"354d44e6-7d5b-46b5-add8-5e0812ed0ef4","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"c84634d1-74c1-40d3-a358-ac12699478c5","type":"ToolEvents"},{"attributes":{"data_source":{"id":"1b752d80-be80-4b1d-a56a-695e21cd9d48","type":"ColumnDataSource"},"glyph":{"id":"c4dddac3-6ae5-4394-81ee-7b43d84e191c","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"753873da-d320-4b8e-80cc-437382ba91c0","type":"Line"},"selection_glyph":null},"id":"578be135-9e3c-48f2-b070-14bea88b02d1","type":"GlyphRenderer"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"322e809f-122d-44ab-adfb-fff13baa036d","type":"PanTool"},{"id":"e612c372-7409-4d8a-944c-1206bca538ab","type":"WheelZoomTool"},{"id":"58a240d7-30be-4420-b93a-ca1289863d13","type":"BoxZoomTool"},{"id":"86c94690-79f6-4288-ba82-e17a4d185cb9","type":"SaveTool"},{"id":"edc59f29-54a7-45a3-84bb-03fcb65ef6df","type":"ResetTool"},{"id":"b4032102-911b-4c4c-8f60-4ecff36ecb9b","type":"HelpTool"}]},"id":"d3acbb56-ea5e-42fb-ad0f-1081293957ba","type":"Toolbar"},{"attributes":{"callback":null,"start":0},"id":"dff6e4ba-fb61-418f-ad54-a9e4654930b8","type":"DataRange1d"},{"attributes":{},"id":"b02b3b99-c4b6-4243-b106-38dc86d0b2b7","type":"BasicTickFormatter"},{"attributes":{},"id":"55f6f30e-c5c2-4a42-bcae-8f7abfbae0aa","type":"LinearScale"},{"attributes":{"callback":null,"end":19628.783834012105,"start":0},"id":"4ea7ba70-545d-4315-8170-2bc5f7a661eb","type":"DataRange1d"},{"attributes":{},"id":"1bb3fef9-ec3c-45de-b36b-1b91a4324738","type":"LinearScale"},{"attributes":{"plot":{"id":"354d44e6-7d5b-46b5-add8-5e0812ed0ef4","subtype":"Figure","type":"Plot"},"ticker":{"id":"aa201ab7-4771-437b-9124-88d4bc1b61d0","type":"BasicTicker"}},"id":"71d72d02-5d18-4b43-8947-6c8922496a2d","type":"Grid"},{"attributes":{"axis_label":"cores","formatter":{"id":"b02b3b99-c4b6-4243-b106-38dc86d0b2b7","type":"BasicTickFormatter"},"plot":{"id":"354d44e6-7d5b-46b5-add8-5e0812ed0ef4","subtype":"Figure","type":"Plot"},"ticker":{"id":"eaf76343-9b43-45f1-bf84-e7ea82ce7024","type":"FixedTicker"}},"id":"c7cad613-5c33-42bc-82d2-9efb32eaa493","type":"LinearAxis"},{"attributes":{},"id":"aa201ab7-4771-437b-9124-88d4bc1b61d0","type":"BasicTicker"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"14d3d91b-e2c9-428c-af94-8bd41548a98e","type":"BasicTickFormatter"},"plot":{"id":"354d44e6-7d5b-46b5-add8-5e0812ed0ef4","subtype":"Figure","type":"Plot"},"ticker":{"id":"1ca03440-5163-40b6-bc8e-7868360ed4bd","type":"BasicTicker"}},"id":"bcd530b5-5ba1-432f-ad2e-5b552403d553","type":"LinearAxis"},{"attributes":{},"id":"1ca03440-5163-40b6-bc8e-7868360ed4bd","type":"BasicTicker"},{"attributes":{"dimension":1,"plot":{"id":"354d44e6-7d5b-46b5-add8-5e0812ed0ef4","subtype":"Figure","type":"Plot"},"ticker":{"id":"1ca03440-5163-40b6-bc8e-7868360ed4bd","type":"BasicTicker"}},"id":"42f34310-41c5-4e5b-8ab3-445c347cc2e7","type":"Grid"},{"attributes":{},"id":"14d3d91b-e2c9-428c-af94-8bd41548a98e","type":"BasicTickFormatter"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"753873da-d320-4b8e-80cc-437382ba91c0","type":"Line"},{"attributes":{"ticker":null},"id":"cb7c8cb6-9623-4908-96f9-761c89176a89","type":"LogTickFormatter"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"8dfd84f0-a5b6-4f38-8e13-8b69195e9d75","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"354d44e6-7d5b-46b5-add8-5e0812ed0ef4","subtype":"Figure","type":"Plot"}},"id":"322e809f-122d-44ab-adfb-fff13baa036d","type":"PanTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"5mBoSrF9l0Bna2xc6/uUQD2eEyk3/Z1A41+WwWA8rkCg3qY5Aga3QAVA74snN8NAncj380uyxEDOuHXJpNfQQN8hVioyK9NA","dtype":"float64","shape":[9]}}},"id":"94a916f8-c48b-40f3-ac0e-46d95d98c641","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"354d44e6-7d5b-46b5-add8-5e0812ed0ef4","subtype":"Figure","type":"Plot"}},"id":"e612c372-7409-4d8a-944c-1206bca538ab","type":"WheelZoomTool"},{"attributes":{"overlay":{"id":"8dfd84f0-a5b6-4f38-8e13-8b69195e9d75","type":"BoxAnnotation"},"plot":{"id":"354d44e6-7d5b-46b5-add8-5e0812ed0ef4","subtype":"Figure","type":"Plot"}},"id":"58a240d7-30be-4420-b93a-ca1289863d13","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"354d44e6-7d5b-46b5-add8-5e0812ed0ef4","subtype":"Figure","type":"Plot"}},"id":"86c94690-79f6-4288-ba82-e17a4d185cb9","type":"SaveTool"},{"attributes":{"plot":{"id":"354d44e6-7d5b-46b5-add8-5e0812ed0ef4","subtype":"Figure","type":"Plot"}},"id":"edc59f29-54a7-45a3-84bb-03fcb65ef6df","type":"ResetTool"},{"attributes":{"plot":{"id":"354d44e6-7d5b-46b5-add8-5e0812ed0ef4","subtype":"Figure","type":"Plot"}},"id":"b4032102-911b-4c4c-8f60-4ecff36ecb9b","type":"HelpTool"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"526fabff-3c81-46d6-9de0-1f353adfca9d","type":"Circle"},{"attributes":{"axis_label":"cores","formatter":{"id":"f7bbd14e-534b-45b2-aa72-80a33d3d048e","type":"BasicTickFormatter"},"plot":{"id":"dbde8c5e-a16f-4c03-9c99-8f0f50a1bbd4","subtype":"Figure","type":"Plot"},"ticker":{"id":"426bd2ab-0882-4386-b2c7-83a24a666182","type":"FixedTicker"}},"id":"b9355d3b-be49-47fd-8d7c-aa58437774a1","type":"LinearAxis"},{"attributes":{"items":[{"id":"df99970f-4041-44f8-b3a0-0bf1e59c404e","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"354d44e6-7d5b-46b5-add8-5e0812ed0ef4","subtype":"Figure","type":"Plot"}},"id":"79e12eb2-6783-4266-9832-ac658476172b","type":"Legend"},{"attributes":{"data_source":{"id":"94a916f8-c48b-40f3-ac0e-46d95d98c641","type":"ColumnDataSource"},"glyph":{"id":"526fabff-3c81-46d6-9de0-1f353adfca9d","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"a340b282-bd7f-45ec-9882-8e1c1a6213c3","type":"Circle"},"selection_glyph":null},"id":"fd8659b9-5774-4ade-9016-a647c66e4fda","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"a340b282-bd7f-45ec-9882-8e1c1a6213c3","type":"Circle"},{"attributes":{"child":{"id":"26cfe895-d550-439a-9fd8-f8b33f7650c4","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"3fead079-f081-4604-957d-6844b5640f7e","type":"Panel"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"753a306f-dd3d-44ba-9c81-3ebbc7a35a31","type":"Line"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"18874344-f743-48f0-b079-2842f76113a4","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"ebe800c9-1421-4745-b177-d089dafce886","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"s8bqjTzBiUBTem6Z0ZyGQFtOQa+MMZNAS+oBg8Twl0Arqf2CF8SjQIPxxSmeUatANdBkm/kbrkBhrYhhFXK0QGeZb4e3brJA","dtype":"float64","shape":[9]}}},"id":"a1335509-0bcd-498a-bb3c-5af46436f736","type":"ColumnDataSource"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"eaf76343-9b43-45f1-bf84-e7ea82ce7024","type":"FixedTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,384876.3226637974]}},"id":"a68fcac2-6ff7-4160-ab3f-ef566bea5cbb","type":"ColumnDataSource"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"365a4993-c8d9-48d5-a165-7a4e9206a3d1","type":"Line"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"8181542c-7323-46eb-9d9d-21a39b2f337b","type":"Panel"},{"id":"3fead079-f081-4604-957d-6844b5640f7e","type":"Panel"}]},"id":"7fc598d6-b20a-457d-80b4-34bf7c5ca49e","type":"Tabs"},{"attributes":{"child":{"id":"6283fd05-bec6-41be-819f-32dcc5714915","subtype":"Figure","type":"Plot"},"title":"log"},"id":"8181542c-7323-46eb-9d9d-21a39b2f337b","type":"Panel"},{"attributes":{"below":[{"id":"69e7f477-33dd-49bb-8742-8a4acc5e5cba","type":"LogAxis"}],"left":[{"id":"6630e827-9e7c-490e-a37e-85408a6a4ae2","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"69e7f477-33dd-49bb-8742-8a4acc5e5cba","type":"LogAxis"},{"id":"a80b5d2e-a365-49b7-815c-151a0ac219f5","type":"Grid"},{"id":"6630e827-9e7c-490e-a37e-85408a6a4ae2","type":"LogAxis"},{"id":"d0f0b675-2b8e-4fd7-9f32-6cbcb749cf23","type":"Grid"},{"id":"dde7e77b-1917-4489-9e8d-103c92197c86","type":"BoxAnnotation"},{"id":"ae2f1fe7-e7ad-4a3c-9994-cd087b537257","type":"Legend"},{"id":"0d08fe1d-5742-4c92-84b6-7ecbbbf5f300","type":"GlyphRenderer"},{"id":"6af73c8b-fc8e-4c8b-a659-a7220854b1e0","type":"GlyphRenderer"},{"id":"ec6a1d09-8449-46cf-b1be-3e11e093564e","type":"GlyphRenderer"}],"title":{"id":"706a83b4-c08b-4b8b-8dd9-08e67528b683","type":"Title"},"tool_events":{"id":"9f244216-2c49-46aa-86e7-c7a71dc00710","type":"ToolEvents"},"toolbar":{"id":"0084c66b-ce81-4d6d-bee6-69dffd00663c","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"87b21d20-ff5f-42fd-8677-3f4c6a3905e5","type":"DataRange1d"},"x_scale":{"id":"e188ceed-f72f-4184-827b-8cf9dcd2016b","type":"LogScale"},"y_range":{"id":"b3a359d4-53fe-4c66-9a9d-198781a27d5d","type":"DataRange1d"},"y_scale":{"id":"098ce152-b87a-43ed-962e-e78371796667","type":"LogScale"}},"id":"6283fd05-bec6-41be-819f-32dcc5714915","subtype":"Figure","type":"Plot"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"s8bqjTzBiUBTem6Z0ZyGQFtOQa+MMZNAS+oBg8Twl0Arqf2CF8SjQIPxxSmeUatANdBkm/kbrkBhrYhhFXK0QGeZb4e3brJA","dtype":"float64","shape":[9]}}},"id":"102a1503-0b67-42f0-bf39-b7f25a94fe39","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"102a1503-0b67-42f0-bf39-b7f25a94fe39","type":"ColumnDataSource"},"glyph":{"id":"365a4993-c8d9-48d5-a165-7a4e9206a3d1","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"753a306f-dd3d-44ba-9c81-3ebbc7a35a31","type":"Line"},"selection_glyph":null},"id":"0d08fe1d-5742-4c92-84b6-7ecbbbf5f300","type":"GlyphRenderer"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"dde7e77b-1917-4489-9e8d-103c92197c86","type":"BoxAnnotation"},{"attributes":{"plot":null,"text":"Arrays: Rechunking"},"id":"706a83b4-c08b-4b8b-8dd9-08e67528b683","type":"Title"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"ebad1b33-c8f9-4a5e-909d-89826fa6727b","type":"Line"},{"attributes":{"plot":{"id":"235a3d83-e4c1-4855-b3f3-96247ff0b146","subtype":"Figure","type":"Plot"}},"id":"119970ac-6463-4f32-a7e2-f3bb89fcae56","type":"WheelZoomTool"},{"attributes":{"data_source":{"id":"6a885f9a-7765-41fa-af8a-dc7292f329df","type":"ColumnDataSource"},"glyph":{"id":"b6b35bed-a3cf-4ff2-94af-733dab6ed569","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"ebad1b33-c8f9-4a5e-909d-89826fa6727b","type":"Line"},"selection_glyph":null},"id":"4a334960-fb43-4949-8e13-5cc44f095565","type":"GlyphRenderer"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"ba1f4919-aa9e-4f0c-85be-efee379561cb","type":"Line"},{"attributes":{"data_source":{"id":"3d533bc9-6340-4496-bb65-6e45171a28e4","type":"ColumnDataSource"},"glyph":{"id":"173a4f64-1c36-478b-8939-3df3b90611ec","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"3d65a81f-7ebd-459e-b72f-8393275a710d","type":"Line"},"selection_glyph":null},"id":"39e06dbc-9a1e-441a-b66e-387a27314ce3","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"37dbf4b3-9234-42a7-b459-ca9e5ff375fb","type":"ColumnDataSource"},"glyph":{"id":"66b2fd67-9922-4aeb-8161-2fc03aeb60cb","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"f8360fdc-5a2e-4f5d-bb69-da02af36803d","type":"Line"},"selection_glyph":null},"id":"1d3782ea-eb9f-4992-bf64-5839a46961ee","type":"GlyphRenderer"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"a7ddf49e-16f8-4cb1-ab9f-d9074592a233","type":"Line"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"173a4f64-1c36-478b-8939-3df3b90611ec","type":"Line"},{"attributes":{"data_source":{"id":"2f0e9839-fadf-4d77-8731-a9c5631134a1","type":"ColumnDataSource"},"glyph":{"id":"ba1f4919-aa9e-4f0c-85be-efee379561cb","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"a7ddf49e-16f8-4cb1-ab9f-d9074592a233","type":"Line"},"selection_glyph":null},"id":"d580f476-985c-4210-a9c9-b6ebbdbe8ef8","type":"GlyphRenderer"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"3d65a81f-7ebd-459e-b72f-8393275a710d","type":"Line"},{"attributes":{"callback":null,"start":0},"id":"3e2b5630-e9ec-4bdd-8ce7-388ee58a26bd","type":"DataRange1d"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"78e8d05c-b35c-4703-ae94-e6433b0aa44e","type":"FixedTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,324077.3921569089]}},"id":"3d533bc9-6340-4496-bb65-6e45171a28e4","type":"ColumnDataSource"},{"attributes":{"plot":null,"text":"Tasks: Tree Reduction"},"id":"1c723287-ce27-47d7-b9bc-3997ea760765","type":"Title"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"c2bbde99-cab6-4b0f-9f59-5606c544b5da","type":"PanTool"},{"id":"04ab7ca2-17b0-4ef9-ab95-d3e14701499a","type":"WheelZoomTool"},{"id":"9f4a3ad1-0ccf-45d0-a913-8a7ae8430a44","type":"BoxZoomTool"},{"id":"916517f6-446b-48e0-b5fa-d83ea96d5d34","type":"SaveTool"},{"id":"c6ec5b39-7ffa-4884-8e8a-269660118fda","type":"ResetTool"},{"id":"a5ff4436-25c4-4322-89c0-97a8dbc1e017","type":"HelpTool"}]},"id":"b7046cc2-9a48-4f2f-ae29-090d2ca5f4f9","type":"Toolbar"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"1d27dfc8-f33e-4a00-859c-228c9443d3a6","type":"Line"},{"attributes":{},"id":"2c6240ac-1e30-44cb-ab71-b7d0757e35e6","type":"ToolEvents"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"cd58c1ae-ad48-4da0-9d98-64682209015a","type":"Line"},{"attributes":{},"id":"9a4ab36b-33e0-4d16-8c20-e55f98f2610a","type":"BasicTickFormatter"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"f61e049a-3d57-4ba9-ab9d-017dbc17cca7","type":"Line"},{"attributes":{"below":[{"id":"9fa573a4-6877-4f0e-bc99-05420273670b","type":"LinearAxis"}],"left":[{"id":"64566afc-a366-4138-9295-90f1146057ba","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"9fa573a4-6877-4f0e-bc99-05420273670b","type":"LinearAxis"},{"id":"16de9ec2-8e18-4421-9eaa-5af8e7ad7095","type":"Grid"},{"id":"64566afc-a366-4138-9295-90f1146057ba","type":"LinearAxis"},{"id":"6f050933-0f43-41ef-9ea2-2a30766b9370","type":"Grid"},{"id":"c0d7f6ca-e987-4799-8a2e-aabbcfd2f677","type":"BoxAnnotation"},{"id":"b142de0f-ddfe-4ba6-9b4e-21e324ef3bb7","type":"Legend"},{"id":"0ea10786-847d-49d5-b396-c435c2c06fdb","type":"GlyphRenderer"},{"id":"a946cd7b-d171-475c-a913-65ac28152b4a","type":"GlyphRenderer"},{"id":"1d3782ea-eb9f-4992-bf64-5839a46961ee","type":"GlyphRenderer"},{"id":"5c6f5d7b-68f9-4646-9849-50cfd2e8bc5d","type":"GlyphRenderer"},{"id":"460b4115-9f55-49b9-aa33-6f2a4c60ff3c","type":"GlyphRenderer"},{"id":"d0523baa-6be1-4235-af10-98771e7cc0da","type":"GlyphRenderer"}],"title":{"id":"1c723287-ce27-47d7-b9bc-3997ea760765","type":"Title"},"tool_events":{"id":"2c6240ac-1e30-44cb-ab71-b7d0757e35e6","type":"ToolEvents"},"toolbar":{"id":"b7046cc2-9a48-4f2f-ae29-090d2ca5f4f9","type":"Toolbar"},"x_range":{"id":"3e2b5630-e9ec-4bdd-8ce7-388ee58a26bd","type":"DataRange1d"},"x_scale":{"id":"26530279-850c-463b-ba0a-d6f4df44cfc1","type":"LinearScale"},"y_range":{"id":"4ee7d578-0ba1-4170-89ab-35e3a5aa8c2c","type":"DataRange1d"},"y_scale":{"id":"48d8bdaf-97c2-41ff-ba56-13589552217e","type":"LinearScale"}},"id":"0a3bfea8-d1bb-4d64-a752-ba72a5231006","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"26530279-850c-463b-ba0a-d6f4df44cfc1","type":"LinearScale"},{"attributes":{"callback":null,"end":3596.14811695509,"start":0},"id":"4ee7d578-0ba1-4170-89ab-35e3a5aa8c2c","type":"DataRange1d"},{"attributes":{},"id":"48d8bdaf-97c2-41ff-ba56-13589552217e","type":"LinearScale"},{"attributes":{"plot":{"id":"0a3bfea8-d1bb-4d64-a752-ba72a5231006","subtype":"Figure","type":"Plot"},"ticker":{"id":"a6bcb22e-c9a8-4ac8-a2b9-285ba96e9c01","type":"BasicTicker"}},"id":"16de9ec2-8e18-4421-9eaa-5af8e7ad7095","type":"Grid"},{"attributes":{"axis_label":"cores","formatter":{"id":"9a4ab36b-33e0-4d16-8c20-e55f98f2610a","type":"BasicTickFormatter"},"plot":{"id":"0a3bfea8-d1bb-4d64-a752-ba72a5231006","subtype":"Figure","type":"Plot"},"ticker":{"id":"f08e67a9-d608-4d46-a0c4-347b7cea6f5d","type":"FixedTicker"}},"id":"9fa573a4-6877-4f0e-bc99-05420273670b","type":"LinearAxis"},{"attributes":{},"id":"a6bcb22e-c9a8-4ac8-a2b9-285ba96e9c01","type":"BasicTicker"},{"attributes":{"axis_label":"tasks/s","formatter":{"id":"231b4f77-befe-49ea-9d6b-c885949da9ac","type":"BasicTickFormatter"},"plot":{"id":"0a3bfea8-d1bb-4d64-a752-ba72a5231006","subtype":"Figure","type":"Plot"},"ticker":{"id":"469a3306-2662-4292-b7fb-d3bac28461b3","type":"BasicTicker"}},"id":"64566afc-a366-4138-9295-90f1146057ba","type":"LinearAxis"},{"attributes":{},"id":"469a3306-2662-4292-b7fb-d3bac28461b3","type":"BasicTicker"},{"attributes":{"dimension":1,"plot":{"id":"0a3bfea8-d1bb-4d64-a752-ba72a5231006","subtype":"Figure","type":"Plot"},"ticker":{"id":"469a3306-2662-4292-b7fb-d3bac28461b3","type":"BasicTicker"}},"id":"6f050933-0f43-41ef-9ea2-2a30766b9370","type":"Grid"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"4c9ac1d7-5c2c-4f08-8a68-70c2ab6471e8","type":"Line"},{"attributes":{"data_source":{"id":"b8364272-88ad-444d-ad2a-5e8d9b4df272","type":"ColumnDataSource"},"glyph":{"id":"4c9ac1d7-5c2c-4f08-8a68-70c2ab6471e8","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"f61e049a-3d57-4ba9-ab9d-017dbc17cca7","type":"Line"},"selection_glyph":null},"id":"0ea10786-847d-49d5-b396-c435c2c06fdb","type":"GlyphRenderer"},{"attributes":{"child":{"id":"0a3bfea8-d1bb-4d64-a752-ba72a5231006","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"890493da-ef86-4d46-938a-2dd5a182a384","type":"Panel"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"c0d7f6ca-e987-4799-8a2e-aabbcfd2f677","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"0a3bfea8-d1bb-4d64-a752-ba72a5231006","subtype":"Figure","type":"Plot"}},"id":"c2bbde99-cab6-4b0f-9f59-5606c544b5da","type":"PanTool"},{"attributes":{"data_source":{"id":"538ce727-535f-48dd-a93f-9b165a35f219","type":"ColumnDataSource"},"glyph":{"id":"59098c5b-cfc9-49b6-909c-86c75aada0e9","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"cc390953-da7a-464d-a85f-de7c32bfbec8","type":"Circle"},"selection_glyph":null},"id":"a946cd7b-d171-475c-a913-65ac28152b4a","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"0a3bfea8-d1bb-4d64-a752-ba72a5231006","subtype":"Figure","type":"Plot"}},"id":"04ab7ca2-17b0-4ef9-ab95-d3e14701499a","type":"WheelZoomTool"},{"attributes":{"overlay":{"id":"c0d7f6ca-e987-4799-8a2e-aabbcfd2f677","type":"BoxAnnotation"},"plot":{"id":"0a3bfea8-d1bb-4d64-a752-ba72a5231006","subtype":"Figure","type":"Plot"}},"id":"9f4a3ad1-0ccf-45d0-a913-8a7ae8430a44","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"0a3bfea8-d1bb-4d64-a752-ba72a5231006","subtype":"Figure","type":"Plot"}},"id":"916517f6-446b-48e0-b5fa-d83ea96d5d34","type":"SaveTool"},{"attributes":{"plot":{"id":"0a3bfea8-d1bb-4d64-a752-ba72a5231006","subtype":"Figure","type":"Plot"}},"id":"c6ec5b39-7ffa-4884-8e8a-269660118fda","type":"ResetTool"},{"attributes":{"plot":{"id":"0a3bfea8-d1bb-4d64-a752-ba72a5231006","subtype":"Figure","type":"Plot"}},"id":"a5ff4436-25c4-4322-89c0-97a8dbc1e017","type":"HelpTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"Qs2j4G2rQkAGPm5LixNSQAEIVOXlumFAHhOcgak6cUC7DoHNWWyAQGIjzN8v5o5Auzk4xMOJnECcfp9Q3SOlQDBM/NVLGKxA","dtype":"float64","shape":[9]}}},"id":"b8364272-88ad-444d-ad2a-5e8d9b4df272","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"Qs2j4G2rQkAGPm5LixNSQAEIVOXlumFAHhOcgak6cUC7DoHNWWyAQGIjzN8v5o5Auzk4xMOJnECcfp9Q3SOlQDBM/NVLGKxA","dtype":"float64","shape":[9]}}},"id":"538ce727-535f-48dd-a93f-9b165a35f219","type":"ColumnDataSource"},{"attributes":{},"id":"231b4f77-befe-49ea-9d6b-c885949da9ac","type":"BasicTickFormatter"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"59098c5b-cfc9-49b6-909c-86c75aada0e9","type":"Circle"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"cc390953-da7a-464d-a85f-de7c32bfbec8","type":"Circle"},{"attributes":{"plot":{"id":"235a3d83-e4c1-4855-b3f3-96247ff0b146","subtype":"Figure","type":"Plot"}},"id":"e0ba9b60-db70-4b11-90ea-2868bc56108f","type":"PanTool"},{"attributes":{"plot":{"id":"235a3d83-e4c1-4855-b3f3-96247ff0b146","subtype":"Figure","type":"Plot"}},"id":"25aa7dd9-490d-4122-9f8a-fa60ccf8b1ac","type":"SaveTool"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"66b2fd67-9922-4aeb-8161-2fc03aeb60cb","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"f8360fdc-5a2e-4f5d-bb69-da02af36803d","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"q5SRkbXHk0C5OgbIBTCbQNOi7PG6oZ9AYTvcNx2BokB7bWhsEu6mQOBg60Yg8aVA9irH/+SCpUCs8pL575ilQPIu2GiALKRA","dtype":"float64","shape":[9]}}},"id":"bec4eb5f-65ec-4ad6-a8cd-38bcc0872add","type":"ColumnDataSource"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"f8595518-de48-4037-b343-1ed562d926b5","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"q5SRkbXHk0C5OgbIBTCbQNOi7PG6oZ9AYTvcNx2BokB7bWhsEu6mQOBg60Yg8aVA9irH/+SCpUCs8pL575ilQPIu2GiALKRA","dtype":"float64","shape":[9]}}},"id":"37dbf4b3-9234-42a7-b459-ca9e5ff375fb","type":"ColumnDataSource"},{"attributes":{"label":{"value":"1us"},"renderers":[{"id":"1d3782ea-eb9f-4992-bf64-5839a46961ee","type":"GlyphRenderer"}]},"id":"975dd06b-ac52-4693-963f-4c915b65d961","type":"LegendItem"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"95198c65-f703-41cd-974a-dc37f0b71e31","type":"Line"},{"attributes":{"child":{"id":"235a3d83-e4c1-4855-b3f3-96247ff0b146","subtype":"Figure","type":"Plot"},"title":"log"},"id":"7c99fea2-3216-4d84-b912-8d9584fad011","type":"Panel"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"2dbcb092-c667-447b-b5cc-03eb073fbf4b","type":"Circle"},{"attributes":{"data_source":{"id":"dc2c511a-5f42-4f7b-82cc-61a1134db0cb","type":"ColumnDataSource"},"glyph":{"id":"95198c65-f703-41cd-974a-dc37f0b71e31","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"ae54716f-8e3b-44bf-9c64-958fc094c6f1","type":"Line"},"selection_glyph":null},"id":"460b4115-9f55-49b9-aa33-6f2a4c60ff3c","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"c57200fb-e539-4a6c-bb2e-811c3a804e74","type":"Circle"},{"attributes":{"data_source":{"id":"bec4eb5f-65ec-4ad6-a8cd-38bcc0872add","type":"ColumnDataSource"},"glyph":{"id":"2dbcb092-c667-447b-b5cc-03eb073fbf4b","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"c57200fb-e539-4a6c-bb2e-811c3a804e74","type":"Circle"},"selection_glyph":null},"id":"5c6f5d7b-68f9-4646-9849-50cfd2e8bc5d","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"YiGd3kr1M0C4DswHLGBDQAJlPBqPJVNAF1KOo6LJYkAH1cQ/wblxQALKM0XtrH9ADDdmn9fbi0DY4pMc/3KVQBZyRig+NJpA","dtype":"float64","shape":[9]}}},"id":"eac1398b-7784-45d0-b9b5-b28ae1fb5381","type":"ColumnDataSource"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"ae317913-c907-4b2b-a755-528e99973858","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,9558.858417964195]}},"id":"dc2c511a-5f42-4f7b-82cc-61a1134db0cb","type":"ColumnDataSource"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"ae54716f-8e3b-44bf-9c64-958fc094c6f1","type":"Line"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"55a957b5-c201-4d84-8a24-080397bd507a","type":"Line"},{"attributes":{"data_source":{"id":"a66076e3-3f19-4847-bdaf-a37ee50a4b9b","type":"ColumnDataSource"},"glyph":{"id":"55a957b5-c201-4d84-8a24-080397bd507a","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"a2d7f58b-d5fb-4c2d-90f5-693d1cd03750","type":"Line"},"selection_glyph":null},"id":"d0523baa-6be1-4235-af10-98771e7cc0da","type":"GlyphRenderer"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"a2d7f58b-d5fb-4c2d-90f5-693d1cd03750","type":"Line"},{"attributes":{"items":[{"id":"8750c30b-8f3b-4482-917f-12aade87bf36","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"b1f9c2ed-12a3-425c-a852-37fafe3ee9c8","subtype":"Figure","type":"Plot"}},"id":"88daa403-982e-4d7e-a63e-e5f4401d6e08","type":"Legend"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"f08e67a9-d608-4d46-a0c4-347b7cea6f5d","type":"FixedTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,324077.3921569089]}},"id":"a66076e3-3f19-4847-bdaf-a37ee50a4b9b","type":"ColumnDataSource"},{"attributes":{"num_minor_ticks":10},"id":"9f143fc4-5cc2-4a94-9b7b-a04f077573e1","type":"LogTicker"},{"attributes":{"axis_label":"tasks/s","formatter":{"id":"dbe6206b-2096-45ac-9348-a3ceb73aca41","type":"LogTickFormatter"},"plot":{"id":"235a3d83-e4c1-4855-b3f3-96247ff0b146","subtype":"Figure","type":"Plot"},"ticker":{"id":"9f143fc4-5cc2-4a94-9b7b-a04f077573e1","type":"LogTicker"}},"id":"4d0b5ee2-e522-422c-ac00-fb6e8bd64d19","type":"LogAxis"},{"attributes":{"data_source":{"id":"eac1398b-7784-45d0-b9b5-b28ae1fb5381","type":"ColumnDataSource"},"glyph":{"id":"ae317913-c907-4b2b-a755-528e99973858","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"f8595518-de48-4037-b343-1ed562d926b5","type":"Line"},"selection_glyph":null},"id":"574611f2-05d6-4ba6-8e30-00a164f87763","type":"GlyphRenderer"},{"attributes":{"dimension":1,"plot":{"id":"235a3d83-e4c1-4855-b3f3-96247ff0b146","subtype":"Figure","type":"Plot"},"ticker":{"id":"9f143fc4-5cc2-4a94-9b7b-a04f077573e1","type":"LogTicker"}},"id":"4802e78d-1853-4c20-8f57-d2ef5d94f259","type":"Grid"},{"attributes":{"plot":{"id":"2e73804a-3798-4c2c-8afb-a230f37ff380","subtype":"Figure","type":"Plot"}},"id":"6c2a2f99-b315-4295-938e-4182eefbf3a8","type":"WheelZoomTool"},{"attributes":{"plot":{"id":"2e73804a-3798-4c2c-8afb-a230f37ff380","subtype":"Figure","type":"Plot"}},"id":"0a12f286-72b9-40f7-9747-cce4fca208b7","type":"PanTool"},{"attributes":{"plot":{"id":"2e73804a-3798-4c2c-8afb-a230f37ff380","subtype":"Figure","type":"Plot"}},"id":"541700d3-9318-4921-895f-bc6a03b02ecb","type":"ResetTool"},{"attributes":{"below":[{"id":"1cc6a791-0f6a-43c7-8e1a-d0f4b145011b","type":"LinearAxis"}],"left":[{"id":"0e06e81c-7091-4673-987b-28bdfdd387d4","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"1cc6a791-0f6a-43c7-8e1a-d0f4b145011b","type":"LinearAxis"},{"id":"7b9bfd50-057e-41f4-a84f-83020213d396","type":"Grid"},{"id":"0e06e81c-7091-4673-987b-28bdfdd387d4","type":"LinearAxis"},{"id":"7efa5fcd-1384-4e51-8ab3-c56083f3d139","type":"Grid"},{"id":"8f746b66-eb35-40f8-b041-d9f5e016b234","type":"BoxAnnotation"},{"id":"115c68a3-af39-4a59-8199-23ad4a0f78ba","type":"Legend"},{"id":"1da83794-de84-4f89-b7a4-931fd0be822e","type":"GlyphRenderer"},{"id":"a86e104b-497a-42c1-8cb6-aa5f1efd5a97","type":"GlyphRenderer"},{"id":"a209df9a-abf2-4be6-83bc-504ea82e6cdc","type":"GlyphRenderer"},{"id":"399be48b-b05d-48ac-ad82-8bb5171ea982","type":"GlyphRenderer"},{"id":"dae703f0-1c58-4dea-971e-c35560dcd430","type":"GlyphRenderer"},{"id":"07b8d079-8049-4102-a197-ff15e44f8107","type":"GlyphRenderer"}],"title":{"id":"267ba9af-9cac-4fd7-b84f-8a9c428335de","type":"Title"},"tool_events":{"id":"c5e95cab-1de3-47a7-b377-8bf0ec0314d4","type":"ToolEvents"},"toolbar":{"id":"9f5af416-f08d-4983-86a7-a899b6204e52","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"31156497-7771-498e-ab35-a21d97ac706a","type":"DataRange1d"},"x_scale":{"id":"dfdfa1fc-2b37-45fd-a322-6c60455c18fa","type":"LinearScale"},"y_range":{"id":"b3122fae-1c0d-4ad5-9c93-30c05c13efe3","type":"DataRange1d"},"y_scale":{"id":"fa32776a-b5d5-432e-8518-e7c10cb39af5","type":"LinearScale"}},"id":"d83086bf-c820-4079-b0fb-49267a76c0bb","subtype":"Figure","type":"Plot"},{"attributes":{"axis_label":"cores","formatter":{"id":"265ba6b0-f36c-412e-ba03-f9440d16790e","type":"BasicTickFormatter"},"plot":{"id":"d83086bf-c820-4079-b0fb-49267a76c0bb","subtype":"Figure","type":"Plot"},"ticker":{"id":"b4587e42-522f-44d1-bdae-1770fdb06934","type":"FixedTicker"}},"id":"1cc6a791-0f6a-43c7-8e1a-d0f4b145011b","type":"LinearAxis"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"7eb0f20a-143f-4774-b4a4-74fc590cd505","type":"Line"},{"attributes":{"plot":{"id":"d83086bf-c820-4079-b0fb-49267a76c0bb","subtype":"Figure","type":"Plot"},"ticker":{"id":"9f2d71d4-52cb-45c8-93f2-ef9646ec39d2","type":"BasicTicker"}},"id":"7b9bfd50-057e-41f4-a84f-83020213d396","type":"Grid"},{"attributes":{},"id":"dfdfa1fc-2b37-45fd-a322-6c60455c18fa","type":"LinearScale"},{"attributes":{"callback":null,"end":1846.5368297047141,"start":0},"id":"b3122fae-1c0d-4ad5-9c93-30c05c13efe3","type":"DataRange1d"},{"attributes":{"plot":{"id":"2e73804a-3798-4c2c-8afb-a230f37ff380","subtype":"Figure","type":"Plot"}},"id":"aaeac710-c162-4542-bc3a-0a5dabae8022","type":"SaveTool"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"efc5e5bd-a54b-4f82-91ac-81f1452b4983","type":"BoxAnnotation"},{"attributes":{"data_source":{"id":"f4ba03ae-1f10-4452-8b9f-402abae81f36","type":"ColumnDataSource"},"glyph":{"id":"d43e85b9-4414-4ae4-8294-35cc8428cfef","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"510ba703-2928-48e3-809c-40633b21ad4c","type":"Line"},"selection_glyph":null},"id":"ac6c1b73-a582-457c-9f3d-112602cdb911","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"909c8428-c38b-43cc-bfe0-dc197b86d528","type":"ColumnDataSource"},"glyph":{"id":"254c5a68-0980-4842-8685-481c9eda7e49","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"8cdf174b-749d-462a-bb44-f0f89e1670f3","type":"Circle"},"selection_glyph":null},"id":"a86e104b-497a-42c1-8cb6-aa5f1efd5a97","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"Uji9ucBYZUAdsMJb8tRkQL1Vo71Kh3FAm3a94PqXe0BPlY81iK2BQAwxqXWHd45APa02fXXbmEATpa+2JdqcQKoSrs0vSZpA","dtype":"float64","shape":[9]}}},"id":"f4ba03ae-1f10-4452-8b9f-402abae81f36","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"2e73804a-3798-4c2c-8afb-a230f37ff380","subtype":"Figure","type":"Plot"}},"id":"21b20b53-1ceb-423e-a2ec-b801c3464c1e","type":"HelpTool"},{"attributes":{"plot":{"id":"e761952d-458d-4815-b822-4649d8dc2644","subtype":"Figure","type":"Plot"}},"id":"de4229e6-b7b3-4b99-a9cf-99e8e2addfe1","type":"HelpTool"},{"attributes":{"ticker":null},"id":"b5b3cacd-17da-4b7a-8255-21f22b67b039","type":"LogTickFormatter"},{"attributes":{"items":[{"id":"e7a616d5-1ff2-476f-94a9-483ba5a296ec","type":"LegendItem"},{"id":"b0557011-7268-4e67-bce4-b6b09d62330d","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"2e73804a-3798-4c2c-8afb-a230f37ff380","subtype":"Figure","type":"Plot"}},"id":"65412682-021f-4cbc-8a36-490fb1d821b7","type":"Legend"},{"attributes":{"data_source":{"id":"6f91da52-632c-4841-85ce-3bd6defdeda0","type":"ColumnDataSource"},"glyph":{"id":"7df733ee-85ac-443a-a6f5-d4cf488ea15c","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"15228f53-57af-4a54-b48e-e18360e7c5bd","type":"Circle"},"selection_glyph":null},"id":"f3e364a8-c1f0-40ba-aed0-daf9ceae878e","type":"GlyphRenderer"},{"attributes":{"label":{"value":"df.groupby(...).apply(...)"},"renderers":[{"id":"ac6c1b73-a582-457c-9f3d-112602cdb911","type":"GlyphRenderer"}]},"id":"e7a616d5-1ff2-476f-94a9-483ba5a296ec","type":"LegendItem"},{"attributes":{"overlay":{"id":"efc5e5bd-a54b-4f82-91ac-81f1452b4983","type":"BoxAnnotation"},"plot":{"id":"2e73804a-3798-4c2c-8afb-a230f37ff380","subtype":"Figure","type":"Plot"}},"id":"2d6b0dc4-acfe-49bb-9dda-551a36dcf187","type":"BoxZoomTool"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"8cdf174b-749d-462a-bb44-f0f89e1670f3","type":"Circle"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"Uji9ucBYZUAdsMJb8tRkQL1Vo71Kh3FAm3a94PqXe0BPlY81iK2BQAwxqXWHd45APa02fXXbmEATpa+2JdqcQKoSrs0vSZpA","dtype":"float64","shape":[9]}}},"id":"b9cfce9f-3078-42a6-945f-13a8c1e3d479","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"d83086bf-c820-4079-b0fb-49267a76c0bb","subtype":"Figure","type":"Plot"}},"id":"c6324a86-b7a0-4476-8559-7e4c969cbc1d","type":"HelpTool"},{"attributes":{},"id":"fa32776a-b5d5-432e-8518-e7c10cb39af5","type":"LinearScale"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"458fd79c-5c42-4892-a276-8baa6a0bdb43","type":"Line"},{"attributes":{"ticker":null},"id":"e5dad457-ce21-46f9-a4bf-b881885257b3","type":"LogTickFormatter"},{"attributes":{"plot":{"id":"d83086bf-c820-4079-b0fb-49267a76c0bb","subtype":"Figure","type":"Plot"}},"id":"1867925c-2af4-4db7-8e7f-f5ebe7c04b2c","type":"SaveTool"},{"attributes":{"dimension":1,"plot":{"id":"2e73804a-3798-4c2c-8afb-a230f37ff380","subtype":"Figure","type":"Plot"},"ticker":{"id":"bf227976-4636-4ae8-b651-79ca2bb13121","type":"LogTicker"}},"id":"c27ae204-4506-4af7-a62e-eb46a04d85b5","type":"Grid"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"6f7d4cfd-27ef-4af1-96e2-a3739b094421","type":"Circle"},{"attributes":{"plot":{"id":"d83086bf-c820-4079-b0fb-49267a76c0bb","subtype":"Figure","type":"Plot"}},"id":"139b60e4-50a4-4e7d-859b-078ddbdbdcc7","type":"ResetTool"},{"attributes":{"overlay":{"id":"8f746b66-eb35-40f8-b041-d9f5e016b234","type":"BoxAnnotation"},"plot":{"id":"d83086bf-c820-4079-b0fb-49267a76c0bb","subtype":"Figure","type":"Plot"}},"id":"cd115f44-a4b3-4792-b271-ff55420fa101","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"d83086bf-c820-4079-b0fb-49267a76c0bb","subtype":"Figure","type":"Plot"}},"id":"04994c9c-5fd6-43a7-8350-ba05388e5dda","type":"WheelZoomTool"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"015a7a30-f537-48db-b3c9-b228a55542cc","type":"Line"},{"attributes":{"plot":{"id":"d83086bf-c820-4079-b0fb-49267a76c0bb","subtype":"Figure","type":"Plot"}},"id":"9bfe2281-8649-489d-b24b-7d071f5e0448","type":"PanTool"},{"attributes":{},"id":"2e4711a0-181a-485b-bc93-efe06f4dc412","type":"BasicTickFormatter"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"8f746b66-eb35-40f8-b041-d9f5e016b234","type":"BoxAnnotation"},{"attributes":{},"id":"49ee7698-c72f-46d2-a94e-80d1221828d2","type":"BasicTicker"},{"attributes":{},"id":"9f2d71d4-52cb-45c8-93f2-ef9646ec39d2","type":"BasicTicker"},{"attributes":{"data_source":{"id":"e23c1abe-7f18-425a-858d-19ad4714f55a","type":"ColumnDataSource"},"glyph":{"id":"19b84dcd-5f69-4967-a022-fc9ba521eb31","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"015a7a30-f537-48db-b3c9-b228a55542cc","type":"Line"},"selection_glyph":null},"id":"d3f0205b-45dc-444f-9485-3a821bd6f1e5","type":"GlyphRenderer"},{"attributes":{"dimension":1,"plot":{"id":"d83086bf-c820-4079-b0fb-49267a76c0bb","subtype":"Figure","type":"Plot"},"ticker":{"id":"49ee7698-c72f-46d2-a94e-80d1221828d2","type":"BasicTicker"}},"id":"7efa5fcd-1384-4e51-8ab3-c56083f3d139","type":"Grid"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"79603191-d085-4ef7-94a5-4fedb238bd7b","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"adfe0308-4aa4-4654-b9b5-fccc26b80a2e","type":"Line"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"254c5a68-0980-4842-8685-481c9eda7e49","type":"Circle"},{"attributes":{"data_source":{"id":"b9cfce9f-3078-42a6-945f-13a8c1e3d479","type":"ColumnDataSource"},"glyph":{"id":"9e49cfe9-b237-4ef0-834d-3bc6e46582ae","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"6f7d4cfd-27ef-4af1-96e2-a3739b094421","type":"Circle"},"selection_glyph":null},"id":"cf7e8366-7581-4e12-93ef-b6b229126bc9","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,31832.26009933853]}},"id":"033f584b-d1ee-4b4b-b6a7-e7ed41f66129","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"033f584b-d1ee-4b4b-b6a7-e7ed41f66129","type":"ColumnDataSource"},"glyph":{"id":"b1f089ea-54b7-4d44-8d9d-afe7c04eee25","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"458fd79c-5c42-4892-a276-8baa6a0bdb43","type":"Line"},"selection_glyph":null},"id":"b614a223-bbd2-40b1-9145-a3094667aad7","type":"GlyphRenderer"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"d43e85b9-4414-4ae4-8294-35cc8428cfef","type":"Line"},{"attributes":{"callback":null,"start":0},"id":"31156497-7771-498e-ab35-a21d97ac706a","type":"DataRange1d"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"07998121-7f06-437a-b1cb-c2c6736f6931","type":"FixedTicker"},{"attributes":{"plot":null,"text":"DataFrames: Full Shuffle"},"id":"267ba9af-9cac-4fd7-b84f-8a9c428335de","type":"Title"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"9bfe2281-8649-489d-b24b-7d071f5e0448","type":"PanTool"},{"id":"04994c9c-5fd6-43a7-8350-ba05388e5dda","type":"WheelZoomTool"},{"id":"cd115f44-a4b3-4792-b271-ff55420fa101","type":"BoxZoomTool"},{"id":"1867925c-2af4-4db7-8e7f-f5ebe7c04b2c","type":"SaveTool"},{"id":"139b60e4-50a4-4e7d-859b-078ddbdbdcc7","type":"ResetTool"},{"id":"c6324a86-b7a0-4476-8559-7e4c969cbc1d","type":"HelpTool"}]},"id":"9f5af416-f08d-4983-86a7-a899b6204e52","type":"Toolbar"},{"attributes":{},"id":"c5e95cab-1de3-47a7-b377-8bf0ec0314d4","type":"ToolEvents"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"2845b27f-001c-4d22-be1f-a9fe59104f77","type":"Line"},{"attributes":{"data_source":{"id":"f2711f11-dc8a-44ce-bde7-5ba41b2e3f1a","type":"ColumnDataSource"},"glyph":{"id":"7eb0f20a-143f-4774-b4a4-74fc590cd505","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"adfe0308-4aa4-4654-b9b5-fccc26b80a2e","type":"Line"},"selection_glyph":null},"id":"1da83794-de84-4f89-b7a4-931fd0be822e","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"LbJ3pRAWX0BbUHF3/95iQIiYAtArN3BAAzvcglmDd0A7RNmNS3J/QPRV3VjCgItAGs1hBVAyk0A6pgUWMmqYQM/xIEy8hpZA","dtype":"float64","shape":[9]}}},"id":"82241e1f-3146-4835-9d0f-898d74080655","type":"ColumnDataSource"},{"attributes":{},"id":"265ba6b0-f36c-412e-ba03-f9440d16790e","type":"BasicTickFormatter"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"Uji9ucBYZUAdsMJb8tRkQL1Vo71Kh3FAm3a94PqXe0BPlY81iK2BQAwxqXWHd45APa02fXXbmEATpa+2JdqcQKoSrs0vSZpA","dtype":"float64","shape":[9]}}},"id":"909c8428-c38b-43cc-bfe0-dc197b86d528","type":"ColumnDataSource"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"9e49cfe9-b237-4ef0-834d-3bc6e46582ae","type":"Circle"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"b1f089ea-54b7-4d44-8d9d-afe7c04eee25","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,43718.02267323494]}},"id":"e23c1abe-7f18-425a-858d-19ad4714f55a","type":"ColumnDataSource"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"2e4711a0-181a-485b-bc93-efe06f4dc412","type":"BasicTickFormatter"},"plot":{"id":"d83086bf-c820-4079-b0fb-49267a76c0bb","subtype":"Figure","type":"Plot"},"ticker":{"id":"49ee7698-c72f-46d2-a94e-80d1221828d2","type":"BasicTicker"}},"id":"0e06e81c-7091-4673-987b-28bdfdd387d4","type":"LinearAxis"},{"attributes":{"data_source":{"id":"82241e1f-3146-4835-9d0f-898d74080655","type":"ColumnDataSource"},"glyph":{"id":"79603191-d085-4ef7-94a5-4fedb238bd7b","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"2845b27f-001c-4d22-be1f-a9fe59104f77","type":"Line"},"selection_glyph":null},"id":"fb4f1318-40bf-4b51-8d8c-b441c1eed3c0","type":"GlyphRenderer"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"19b84dcd-5f69-4967-a022-fc9ba521eb31","type":"Line"},{"attributes":{},"id":"cbb9bc6c-1f65-4274-bf49-6c454c02a800","type":"LogScale"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"Uji9ucBYZUAdsMJb8tRkQL1Vo71Kh3FAm3a94PqXe0BPlY81iK2BQAwxqXWHd45APa02fXXbmEATpa+2JdqcQKoSrs0vSZpA","dtype":"float64","shape":[9]}}},"id":"f2711f11-dc8a-44ce-bde7-5ba41b2e3f1a","type":"ColumnDataSource"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"510ba703-2928-48e3-809c-40633b21ad4c","type":"Line"},{"attributes":{"label":{"value":"df.groupby(...).apply(...)"},"renderers":[{"id":"1da83794-de84-4f89-b7a4-931fd0be822e","type":"GlyphRenderer"}]},"id":"b8084d43-5b05-43f2-845a-d9bcf1c4ada1","type":"LegendItem"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"b08b8553-f625-4e5e-a796-321b0764389e","type":"LogTickFormatter"},"plot":{"id":"44d846b0-5ac0-4386-b977-fefe682eb1fa","subtype":"Figure","type":"Plot"},"ticker":{"id":"f2dee069-b0ad-4698-b4df-e5a2390d9bfe","type":"LogTicker"}},"id":"3b6698c3-66db-4eea-8093-a3b4f2d0c841","type":"LogAxis"},{"attributes":{"num_minor_ticks":10},"id":"bf227976-4636-4ae8-b651-79ca2bb13121","type":"LogTicker"},{"attributes":{},"id":"0abf6ccc-6de3-47eb-bbb9-5d656a49d2f8","type":"BasicTickFormatter"},{"attributes":{"dimension":1,"plot":{"id":"097ea389-c4b4-49e1-9023-05ce1d47706a","subtype":"Figure","type":"Plot"},"ticker":{"id":"c4c181b0-9765-469e-a9ca-9fdc9a816d58","type":"LogTicker"}},"id":"35c32da5-a86c-4f8e-a851-c129b4c80bcd","type":"Grid"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"epLhQ0J1YkAd5AFIZzFxQJg0vPmU7ntAyXEzRLI4iUCSgRntGWCSQEF51ldnFptA8ymMPoR+nkBOxBAU3zWiQFbn5NVayKJA","dtype":"float64","shape":[9]}}},"id":"989aa921-a6b0-437c-b8f5-0e724a7aa8df","type":"ColumnDataSource"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"bfbd3b84-ea65-468d-80a4-0bc507dbc1f7","type":"LogTickFormatter"},"plot":{"id":"097ea389-c4b4-49e1-9023-05ce1d47706a","subtype":"Figure","type":"Plot"},"ticker":{"id":"c4c181b0-9765-469e-a9ca-9fdc9a816d58","type":"LogTicker"}},"id":"3f82da9d-8419-4425-a965-030097068d53","type":"LogAxis"},{"attributes":{"items":[{"id":"9c4ecca6-3fb9-4b99-a76f-29b0cd62cf8b","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"3f1de3ec-80fb-4cbf-bbd3-870be4431819","subtype":"Figure","type":"Plot"}},"id":"e7cd8d6c-fd1b-47c5-9046-bb0dd0d37454","type":"Legend"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,31832.26009933853]}},"id":"7b5f2355-7a70-4f3d-8db3-05c17d0c3f53","type":"ColumnDataSource"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"b420ab84-de1a-46fb-9d5e-6e58aa4f92d7","type":"BoxAnnotation"},{"attributes":{"label":{"value":"sin(x)**2 + cos(x)**2"},"renderers":[{"id":"f7346b9b-f65b-4d3c-9113-394fac1afe19","type":"GlyphRenderer"}]},"id":"9c4ecca6-3fb9-4b99-a76f-29b0cd62cf8b","type":"LegendItem"},{"attributes":{"data_source":{"id":"6d3d16e4-ca7a-4e07-9444-55fa9920aacc","type":"ColumnDataSource"},"glyph":{"id":"b7d8221d-be30-4e16-8ec6-b34a9423c39f","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"81838cf4-afce-46e4-baf9-1c01298db7f9","type":"Line"},"selection_glyph":null},"id":"8eaa3582-91c7-47cd-ab32-3ebb97258b0b","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"d7c6a4f2-50e5-4522-bb85-49619ead6b2e","type":"ColumnDataSource"},"glyph":{"id":"19f187e5-9ed9-49d6-9323-f06acb06abd8","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"ad7a29b5-2c7d-4ffe-999b-249b07a66a8e","type":"Line"},"selection_glyph":null},"id":"9318cc6d-b0c6-40cb-b3dc-d1660534ca98","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"44d846b0-5ac0-4386-b977-fefe682eb1fa","subtype":"Figure","type":"Plot"}},"id":"84c5ebf4-804c-45e8-86a3-23655f9c10cf","type":"PanTool"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"df4b4c32-b151-4eaa-afa8-f2f1c99944af","type":"Circle"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"uUoxEbZlhUBIyqgVRxWTQHa8HsxSVKFAbE0WQbrQskAfV+6MsWjBQB8gXbcI8NBAaYcHrUvT1EBXicp+gH7TQA1jrj5WlupA","dtype":"float64","shape":[9]}}},"id":"d6b42dd1-cc84-41b8-9d72-cfc98b3531e0","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"989aa921-a6b0-437c-b8f5-0e724a7aa8df","type":"ColumnDataSource"},"glyph":{"id":"df4b4c32-b151-4eaa-afa8-f2f1c99944af","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"1f8c1dcf-cc0e-4738-b1d7-8aad57c363b9","type":"Circle"},"selection_glyph":null},"id":"17c349ae-33d2-40cd-9f5b-e7687a90507d","type":"GlyphRenderer"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"774d9556-37a0-4ca2-8dfb-37d9450954dd","type":"Panel"},{"id":"da378e56-f9e8-445e-b465-2cde4901f11a","type":"Panel"}]},"id":"50a686fd-e2c5-440a-92af-bbfc9d851b96","type":"Tabs"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"1f8c1dcf-cc0e-4738-b1d7-8aad57c363b9","type":"Circle"},{"attributes":{"below":[{"id":"1a1d9b6b-53b1-43ed-90a4-9884182fa0f3","type":"LogAxis"}],"left":[{"id":"3b6698c3-66db-4eea-8093-a3b4f2d0c841","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"1a1d9b6b-53b1-43ed-90a4-9884182fa0f3","type":"LogAxis"},{"id":"e7ef135f-2aa1-4d9b-9c96-514f9a5702e4","type":"Grid"},{"id":"3b6698c3-66db-4eea-8093-a3b4f2d0c841","type":"LogAxis"},{"id":"71be614d-aa83-4b60-b3de-4f3789b14c84","type":"Grid"},{"id":"b420ab84-de1a-46fb-9d5e-6e58aa4f92d7","type":"BoxAnnotation"},{"id":"8096a52d-2eb0-43ae-8c02-d0ccfd8764fe","type":"Legend"},{"id":"8eaa3582-91c7-47cd-ab32-3ebb97258b0b","type":"GlyphRenderer"},{"id":"a54bc275-3a80-4e52-9c0c-ed5ea82ddd80","type":"GlyphRenderer"},{"id":"f054c46b-cd16-493d-a94b-9eafe0508f67","type":"GlyphRenderer"}],"title":{"id":"a9518ca9-f7cc-4199-aa14-9ceaf2c866a7","type":"Title"},"tool_events":{"id":"0981117e-0db9-4157-b86f-b41efd853da2","type":"ToolEvents"},"toolbar":{"id":"f9f5e6ae-d027-438f-8503-8b771446cba4","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"99879e3e-cb86-4ed8-a9ec-b60d8ff48f74","type":"DataRange1d"},"x_scale":{"id":"13eb8f2b-7872-4adc-a796-69ac87b9efe0","type":"LogScale"},"y_range":{"id":"30b0e041-8696-46b0-afac-b993237dd5e2","type":"DataRange1d"},"y_scale":{"id":"20705683-411c-4b3d-8b0f-bae6aad074a7","type":"LogScale"}},"id":"44d846b0-5ac0-4386-b977-fefe682eb1fa","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"837eab4e-6440-4689-a9ca-37aaa40b9633","type":"LogScale"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"81838cf4-afce-46e4-baf9-1c01298db7f9","type":"Line"},{"attributes":{"axis_label":"cores","formatter":{"id":"6f9bfbc1-54df-45f3-9318-b72b981535ea","type":"LogTickFormatter"},"plot":{"id":"097ea389-c4b4-49e1-9023-05ce1d47706a","subtype":"Figure","type":"Plot"},"ticker":{"id":"ec591052-6ee6-4799-aba5-cec91ccbd76c","type":"FixedTicker"}},"id":"3856bd1f-7162-45f9-9de9-b2cddeb604f6","type":"LogAxis"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"19f187e5-9ed9-49d6-9323-f06acb06abd8","type":"Line"},{"attributes":{"child":{"id":"44d846b0-5ac0-4386-b977-fefe682eb1fa","subtype":"Figure","type":"Plot"},"title":"log"},"id":"774d9556-37a0-4ca2-8dfb-37d9450954dd","type":"Panel"},{"attributes":{"plot":null,"text":"Arrays: Create"},"id":"a9518ca9-f7cc-4199-aa14-9ceaf2c866a7","type":"Title"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"ad7a29b5-2c7d-4ffe-999b-249b07a66a8e","type":"Line"},{"attributes":{"num_minor_ticks":10},"id":"43f23bff-cadb-4d03-bbc9-af672ade5478","type":"LogTicker"},{"attributes":{"plot":{"id":"097ea389-c4b4-49e1-9023-05ce1d47706a","subtype":"Figure","type":"Plot"},"ticker":{"id":"7bed437b-39bf-4be0-8e06-06cfcfe0d424","type":"LogTicker"}},"id":"1ec5a4b7-1e26-4570-8001-17fcdb0f684c","type":"Grid"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"b7d8221d-be30-4e16-8ec6-b34a9423c39f","type":"Line"},{"attributes":{"data_source":{"id":"efa8cf3b-3bda-4291-808a-01b3c438e313","type":"ColumnDataSource"},"glyph":{"id":"6f728c8d-c3bd-4ac6-b3d5-e7049adc2931","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"dfeb258a-6b2a-4e5f-998b-e0fada92b7f4","type":"Line"},"selection_glyph":null},"id":"dae703f0-1c58-4dea-971e-c35560dcd430","type":"GlyphRenderer"},{"attributes":{"num_minor_ticks":10},"id":"f2dee069-b0ad-4698-b4df-e5a2390d9bfe","type":"LogTicker"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"5fac156b-7f91-4578-b83b-8858f80abfd0","type":"FixedTicker"},{"attributes":{"axis_label":"cores","formatter":{"id":"728fd7c7-985f-4258-b738-77558af59308","type":"LogTickFormatter"},"plot":{"id":"e761952d-458d-4815-b822-4649d8dc2644","subtype":"Figure","type":"Plot"},"ticker":{"id":"460ef359-dfb1-403c-837f-2f484f0985b2","type":"FixedTicker"}},"id":"89620215-3410-4fc6-88c0-a36a5b8d0eff","type":"LogAxis"},{"attributes":{"ticker":null},"id":"bf3acc1e-c75c-4182-8881-12c577ee7678","type":"LogTickFormatter"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,37802.07078627213]}},"id":"d7c6a4f2-50e5-4522-bb85-49619ead6b2e","type":"ColumnDataSource"},{"attributes":{},"id":"e698c7cc-b5b7-4fa3-a87f-4a477779ff5b","type":"LogScale"},{"attributes":{},"id":"0981117e-0db9-4157-b86f-b41efd853da2","type":"ToolEvents"},{"attributes":{"num_minor_ticks":10},"id":"c4c181b0-9765-469e-a9ca-9fdc9a816d58","type":"LogTicker"},{"attributes":{"callback":null,"start":0},"id":"00b584b3-d9a5-4d49-bd82-f00cbda59075","type":"DataRange1d"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"84c5ebf4-804c-45e8-86a3-23655f9c10cf","type":"PanTool"},{"id":"52678c06-d482-4439-b4f7-fdd872eb00e1","type":"WheelZoomTool"},{"id":"1636d618-670c-4438-b4f8-ee09061d2c5e","type":"BoxZoomTool"},{"id":"de891a7c-65b8-4d1c-9b61-242f54a10f4b","type":"SaveTool"},{"id":"a72122d2-7c05-4909-b806-34da1879e5a5","type":"ResetTool"},{"id":"35ae2e1d-7679-4f32-9cb4-5e5c38fc97c4","type":"HelpTool"}]},"id":"f9f5e6ae-d027-438f-8503-8b771446cba4","type":"Toolbar"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"e230fb50-90d8-4680-914f-3e15f5488fb7","type":"Panel"},{"id":"a5b8f31d-2417-4153-b180-1655c30646ec","type":"Panel"}]},"id":"716ab0cd-3d4a-4000-afeb-483507e81870","type":"Tabs"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"b9db48f5-2638-4c94-a364-8e39eba47393","type":"PanTool"},{"id":"51b20535-8160-496c-986d-8baaecbb2697","type":"WheelZoomTool"},{"id":"e97e20d3-03e2-428d-aa0f-d00fd9bde72e","type":"BoxZoomTool"},{"id":"86256809-cc92-4e15-ae4a-6278598dfde4","type":"SaveTool"},{"id":"ee40caee-98db-40f4-b0db-7656427ffd03","type":"ResetTool"},{"id":"de4229e6-b7b3-4b99-a9cf-99e8e2addfe1","type":"HelpTool"}]},"id":"ce98c481-bf2e-4620-8761-d1a2a476326f","type":"Toolbar"},{"attributes":{"dimension":1,"plot":{"id":"44d846b0-5ac0-4386-b977-fefe682eb1fa","subtype":"Figure","type":"Plot"},"ticker":{"id":"f2dee069-b0ad-4698-b4df-e5a2390d9bfe","type":"LogTicker"}},"id":"71be614d-aa83-4b60-b3de-4f3789b14c84","type":"Grid"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"95dcdf2b-c4d6-45a7-ad97-b9c34e92a11f","type":"Line"},{"attributes":{"callback":null,"end":16490.85092430819,"start":0},"id":"b60405f4-e776-44b3-9ee4-a2c80585564e","type":"DataRange1d"},{"attributes":{"plot":{"id":"44d846b0-5ac0-4386-b977-fefe682eb1fa","subtype":"Figure","type":"Plot"}},"id":"52678c06-d482-4439-b4f7-fdd872eb00e1","type":"WheelZoomTool"},{"attributes":{"below":[{"id":"3856bd1f-7162-45f9-9de9-b2cddeb604f6","type":"LogAxis"}],"left":[{"id":"3f82da9d-8419-4425-a965-030097068d53","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"3856bd1f-7162-45f9-9de9-b2cddeb604f6","type":"LogAxis"},{"id":"1ec5a4b7-1e26-4570-8001-17fcdb0f684c","type":"Grid"},{"id":"3f82da9d-8419-4425-a965-030097068d53","type":"LogAxis"},{"id":"35c32da5-a86c-4f8e-a851-c129b4c80bcd","type":"Grid"},{"id":"c986fca0-15d9-4c8a-8882-6e5b71bc39e0","type":"BoxAnnotation"},{"id":"c61d2cac-746c-4c5c-916c-7d00d7d3630a","type":"Legend"},{"id":"b058e492-67e4-41d1-bbb8-f32eae7b8603","type":"GlyphRenderer"},{"id":"d83e7c85-974f-43a8-bf57-d9ab409aad72","type":"GlyphRenderer"},{"id":"bc3d125d-ffb3-4a2c-aa72-ab9513b02a62","type":"GlyphRenderer"},{"id":"996f1ef0-18fc-47d7-b849-38ded7d681b6","type":"GlyphRenderer"},{"id":"b4aad3bd-05de-476d-a05d-64797c63e4ab","type":"GlyphRenderer"},{"id":"2bac3660-5ea5-4e97-a5d4-862c9ace444c","type":"GlyphRenderer"}],"title":{"id":"c5a599b2-139c-4174-85a1-7a446a564a1a","type":"Title"},"tool_events":{"id":"3de5bc6b-ad0a-433f-9be7-a436f9ee2b06","type":"ToolEvents"},"toolbar":{"id":"44b15bda-858a-4cd2-849f-3d4bbf0c1470","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"b914e7a2-f1e0-4071-a0c2-d52db778eb69","type":"DataRange1d"},"x_scale":{"id":"4014613c-ecf3-4838-b293-ed7bce5fd9ed","type":"LogScale"},"y_range":{"id":"3e37f90e-ebea-460e-aaad-0ccc2649ad5e","type":"DataRange1d"},"y_scale":{"id":"837eab4e-6440-4689-a9ca-37aaa40b9633","type":"LogScale"}},"id":"097ea389-c4b4-49e1-9023-05ce1d47706a","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"6a93c929-ad7f-40d9-a5e5-34c1bb4065c9","type":"ToolEvents"},{"attributes":{"overlay":{"id":"b420ab84-de1a-46fb-9d5e-6e58aa4f92d7","type":"BoxAnnotation"},"plot":{"id":"44d846b0-5ac0-4386-b977-fefe682eb1fa","subtype":"Figure","type":"Plot"}},"id":"1636d618-670c-4438-b4f8-ee09061d2c5e","type":"BoxZoomTool"},{"attributes":{},"id":"4014613c-ecf3-4838-b293-ed7bce5fd9ed","type":"LogScale"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"36fbc0eb-dd8a-4425-8fe3-39a618e7adaf","type":"LogTickFormatter"},"plot":{"id":"e761952d-458d-4815-b822-4649d8dc2644","subtype":"Figure","type":"Plot"},"ticker":{"id":"594148f5-678e-4aec-922f-b845f19d9257","type":"LogTicker"}},"id":"355fbc1d-8ede-4d41-8b62-1872d926a51f","type":"LogAxis"},{"attributes":{"plot":{"id":"44d846b0-5ac0-4386-b977-fefe682eb1fa","subtype":"Figure","type":"Plot"}},"id":"de891a7c-65b8-4d1c-9b61-242f54a10f4b","type":"SaveTool"},{"attributes":{"ticker":null},"id":"6f9bfbc1-54df-45f3-9318-b72b981535ea","type":"LogTickFormatter"},{"attributes":{"plot":{"id":"44d846b0-5ac0-4386-b977-fefe682eb1fa","subtype":"Figure","type":"Plot"}},"id":"a72122d2-7c05-4909-b806-34da1879e5a5","type":"ResetTool"},{"attributes":{"callback":null,"end":26586.15297054953,"start":0},"id":"3e37f90e-ebea-460e-aaad-0ccc2649ad5e","type":"DataRange1d"},{"attributes":{"plot":{"id":"44d846b0-5ac0-4386-b977-fefe682eb1fa","subtype":"Figure","type":"Plot"}},"id":"35ae2e1d-7679-4f32-9cb4-5e5c38fc97c4","type":"HelpTool"},{"attributes":{"plot":null,"text":"Arrays: Nearest Neighbor"},"id":"c5a599b2-139c-4174-85a1-7a446a564a1a","type":"Title"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"uUoxEbZlhUBIyqgVRxWTQHa8HsxSVKFAbE0WQbrQskAfV+6MsWjBQB8gXbcI8NBAaYcHrUvT1EBXicp+gH7TQA1jrj5WlupA","dtype":"float64","shape":[9]}}},"id":"6d3d16e4-ca7a-4e07-9444-55fa9920aacc","type":"ColumnDataSource"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"531899bb-814f-4471-bec2-fa7060ea4879","type":"Line"},{"attributes":{"plot":{"id":"4a3d7c66-d194-4313-bfb7-3005c94dbc3c","subtype":"Figure","type":"Plot"},"ticker":{"id":"ff027743-2af6-4770-9184-cc03c83a71bd","type":"BasicTicker"}},"id":"a5ca3e57-e32a-4db6-8179-72c0ba47abcf","type":"Grid"},{"attributes":{},"id":"3de5bc6b-ad0a-433f-9be7-a436f9ee2b06","type":"ToolEvents"},{"attributes":{"ticker":null},"id":"b08b8553-f625-4e5e-a796-321b0764389e","type":"LogTickFormatter"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"69b885bb-30f4-426f-9d42-de58458e2ff9","type":"PanTool"},{"id":"dc2065d3-d923-4d56-8f0a-098f825d4480","type":"WheelZoomTool"},{"id":"7fb2a7a7-6e7c-4f7e-abe6-bb0b4d4547c0","type":"BoxZoomTool"},{"id":"6e62c218-37ff-42bd-806b-b4c4eaaf4155","type":"SaveTool"},{"id":"5862d233-d9e4-402f-a9b0-823b4b28838d","type":"ResetTool"},{"id":"1abec02d-e81f-4d20-8096-046d0dca57f1","type":"HelpTool"}]},"id":"44b15bda-858a-4cd2-849f-3d4bbf0c1470","type":"Toolbar"},{"attributes":{},"id":"8248c6db-dbe7-456c-ab26-c3643aa8f701","type":"LinearScale"},{"attributes":{"num_minor_ticks":10},"id":"7bed437b-39bf-4be0-8e06-06cfcfe0d424","type":"LogTicker"},{"attributes":{"items":[{"id":"a3de8dc0-abcd-4d41-8755-b95d88644b34","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"44d846b0-5ac0-4386-b977-fefe682eb1fa","subtype":"Figure","type":"Plot"}},"id":"8096a52d-2eb0-43ae-8c02-d0ccfd8764fe","type":"Legend"},{"attributes":{"data_source":{"id":"6d976f93-8361-43be-ba78-c88a9b9bd37e","type":"ColumnDataSource"},"glyph":{"id":"531899bb-814f-4471-bec2-fa7060ea4879","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"95dcdf2b-c4d6-45a7-ad97-b9c34e92a11f","type":"Line"},"selection_glyph":null},"id":"b058e492-67e4-41d1-bbb8-f32eae7b8603","type":"GlyphRenderer"},{"attributes":{"callback":null,"end":54450.69515151354,"start":0},"id":"c254a9b2-bef4-4b72-b5aa-f2c82c2caf7a","type":"DataRange1d"},{"attributes":{"child":{"id":"097ea389-c4b4-49e1-9023-05ce1d47706a","subtype":"Figure","type":"Plot"},"title":"log"},"id":"e230fb50-90d8-4680-914f-3e15f5488fb7","type":"Panel"},{"attributes":{"below":[{"id":"89620215-3410-4fc6-88c0-a36a5b8d0eff","type":"LogAxis"}],"left":[{"id":"355fbc1d-8ede-4d41-8b62-1872d926a51f","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"89620215-3410-4fc6-88c0-a36a5b8d0eff","type":"LogAxis"},{"id":"fa229f3e-b9eb-4a4d-b7aa-9a158b6e3663","type":"Grid"},{"id":"355fbc1d-8ede-4d41-8b62-1872d926a51f","type":"LogAxis"},{"id":"3f12a60a-b6bf-495c-8aab-879125782ebb","type":"Grid"},{"id":"8aa7f9a3-e0e5-419b-a8a4-5ad8e36fe88f","type":"BoxAnnotation"},{"id":"7e75b446-a5a1-42a6-a69a-c20836f1fc09","type":"Legend"},{"id":"4a30e69a-5fc9-49a9-b6d8-ba88f9921bb4","type":"GlyphRenderer"},{"id":"a1906a82-2729-4639-8244-d507183595be","type":"GlyphRenderer"},{"id":"a02779c4-dd2e-4ea8-84d2-22f9a7077d78","type":"GlyphRenderer"}],"title":{"id":"47048622-1378-44e2-99f1-de93b56bab91","type":"Title"},"tool_events":{"id":"6a93c929-ad7f-40d9-a5e5-34c1bb4065c9","type":"ToolEvents"},"toolbar":{"id":"ce98c481-bf2e-4620-8761-d1a2a476326f","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"00b584b3-d9a5-4d49-bd82-f00cbda59075","type":"DataRange1d"},"x_scale":{"id":"e698c7cc-b5b7-4fa3-a87f-4a477779ff5b","type":"LogScale"},"y_range":{"id":"b60405f4-e776-44b3-9ee4-a2c80585564e","type":"DataRange1d"},"y_scale":{"id":"c85cad71-9c17-451c-945a-363c1d6a860e","type":"LogScale"}},"id":"e761952d-458d-4815-b822-4649d8dc2644","subtype":"Figure","type":"Plot"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"c986fca0-15d9-4c8a-8882-6e5b71bc39e0","type":"BoxAnnotation"},{"attributes":{"label":{"value":"random"},"renderers":[{"id":"8eaa3582-91c7-47cd-ab32-3ebb97258b0b","type":"GlyphRenderer"}]},"id":"a3de8dc0-abcd-4d41-8755-b95d88644b34","type":"LegendItem"},{"attributes":{"plot":{"id":"097ea389-c4b4-49e1-9023-05ce1d47706a","subtype":"Figure","type":"Plot"}},"id":"69b885bb-30f4-426f-9d42-de58458e2ff9","type":"PanTool"},{"attributes":{"data_source":{"id":"7a3558a2-1290-4827-9a3d-c625a77e8b0e","type":"ColumnDataSource"},"glyph":{"id":"9b093790-fa1b-4925-b9b7-43ffc25beb47","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"426a9403-2d23-4746-a21c-cc376fd6dac6","type":"Line"},"selection_glyph":null},"id":"f054c46b-cd16-493d-a94b-9eafe0508f67","type":"GlyphRenderer"},{"attributes":{"child":{"id":"e761952d-458d-4815-b822-4649d8dc2644","subtype":"Figure","type":"Plot"},"title":"log"},"id":"0eb06f75-3f93-4b46-835e-a4511fcd840d","type":"Panel"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"5648843d-1d51-4531-80e0-14f7e42d4305","type":"Circle"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"0/MveXGpk0CwnWLotLafQFzSKK/0oatAGQzzlhiFvEBuV2Og1WzIQFHk16Cvv8pAs1osroIq1EBv606nwBHYQN/8RMqJ9tlA","dtype":"float64","shape":[9]}}},"id":"497683f3-df7a-41af-ae53-1146920e8780","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"097ea389-c4b4-49e1-9023-05ce1d47706a","subtype":"Figure","type":"Plot"}},"id":"dc2065d3-d923-4d56-8f0a-098f825d4480","type":"WheelZoomTool"},{"attributes":{"data_source":{"id":"d6b42dd1-cc84-41b8-9d72-cfc98b3531e0","type":"ColumnDataSource"},"glyph":{"id":"5648843d-1d51-4531-80e0-14f7e42d4305","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"f7f0d0b3-1b82-4b62-b6b2-5eaa8f7ac3ff","type":"Circle"},"selection_glyph":null},"id":"a54bc275-3a80-4e52-9c0c-ed5ea82ddd80","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"5765ae37-6a30-4667-90e7-73a12b390a18","type":"ColumnDataSource"},"glyph":{"id":"9e1ea25f-8e06-4321-86cf-a7eb3ae41d67","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"a7b57c45-0aa0-464d-a1f7-22b769b5acee","type":"Line"},"selection_glyph":null},"id":"4a30e69a-5fc9-49a9-b6d8-ba88f9921bb4","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"f7f0d0b3-1b82-4b62-b6b2-5eaa8f7ac3ff","type":"Circle"},{"attributes":{"overlay":{"id":"c986fca0-15d9-4c8a-8882-6e5b71bc39e0","type":"BoxAnnotation"},"plot":{"id":"097ea389-c4b4-49e1-9023-05ce1d47706a","subtype":"Figure","type":"Plot"}},"id":"7fb2a7a7-6e7c-4f7e-abe6-bb0b4d4547c0","type":"BoxZoomTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,43718.02267323494]}},"id":"efa8cf3b-3bda-4291-808a-01b3c438e313","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"097ea389-c4b4-49e1-9023-05ce1d47706a","subtype":"Figure","type":"Plot"}},"id":"6e62c218-37ff-42bd-806b-b4c4eaaf4155","type":"SaveTool"},{"attributes":{},"id":"dc7817a3-cd49-4b6a-89f8-537ebc9b77e2","type":"LinearScale"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"9634e806-9f97-4ba2-93d3-f7411b9ca83c","type":"Line"},{"attributes":{"plot":{"id":"097ea389-c4b4-49e1-9023-05ce1d47706a","subtype":"Figure","type":"Plot"}},"id":"5862d233-d9e4-402f-a9b0-823b4b28838d","type":"ResetTool"},{"attributes":{"callback":null,"start":0},"id":"4ca0bb4c-dc4d-4e88-9c27-d657bfabf665","type":"DataRange1d"},{"attributes":{"ticker":null},"id":"728fd7c7-985f-4258-b738-77558af59308","type":"LogTickFormatter"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"9b093790-fa1b-4925-b9b7-43ffc25beb47","type":"Line"},{"attributes":{"plot":{"id":"097ea389-c4b4-49e1-9023-05ce1d47706a","subtype":"Figure","type":"Plot"}},"id":"1abec02d-e81f-4d20-8096-046d0dca57f1","type":"HelpTool"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"d3c423e7-6dd5-429c-a8ea-24c463171dbc","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"426a9403-2d23-4746-a21c-cc376fd6dac6","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"0/MveXGpk0CwnWLotLafQFzSKK/0oatAGQzzlhiFvEBuV2Og1WzIQFHk16Cvv8pAs1osroIq1EBv606nwBHYQN/8RMqJ9tlA","dtype":"float64","shape":[9]}}},"id":"6d976f93-8361-43be-ba78-c88a9b9bd37e","type":"ColumnDataSource"},{"attributes":{"plot":null,"text":"Arrays: Create"},"id":"374a93fa-028c-4ced-a6cb-2f09519f5c0a","type":"Title"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"90b73c90-6751-497b-91a4-2687c0e9afff","type":"Line"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"7a39f017-1a6a-48be-b9dc-2cfcffd5a9d0","type":"FixedTicker"},{"attributes":{"ticker":null},"id":"bfbd3b84-ea65-468d-80a4-0bc507dbc1f7","type":"LogTickFormatter"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,175286.75839479806]}},"id":"7a3558a2-1290-4827-9a3d-c625a77e8b0e","type":"ColumnDataSource"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"79773228-b238-46de-a46e-225b8a19bdec","type":"Circle"},{"attributes":{"below":[{"id":"0c40f414-7115-4162-ac89-d192308d67d9","type":"LinearAxis"}],"left":[{"id":"a86681af-270c-426f-99dc-5fe3d642ee83","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"0c40f414-7115-4162-ac89-d192308d67d9","type":"LinearAxis"},{"id":"a5ca3e57-e32a-4db6-8179-72c0ba47abcf","type":"Grid"},{"id":"a86681af-270c-426f-99dc-5fe3d642ee83","type":"LinearAxis"},{"id":"cba051c4-7034-4651-af30-532eed93f314","type":"Grid"},{"id":"e66ff0e4-d114-4529-9a0c-97317efe2e91","type":"BoxAnnotation"},{"id":"981660b0-6865-4ab3-a1e0-7f28cb9baec2","type":"Legend"},{"id":"be28dc51-16ef-4fd0-8f80-c604f3e6eaad","type":"GlyphRenderer"},{"id":"557b9797-c849-4fd1-b818-da278a904030","type":"GlyphRenderer"},{"id":"a88c23d3-43d3-4a29-a8c5-20a855f54ba0","type":"GlyphRenderer"}],"title":{"id":"374a93fa-028c-4ced-a6cb-2f09519f5c0a","type":"Title"},"tool_events":{"id":"7244a45f-c85d-4125-a7ae-0f84923c5fe7","type":"ToolEvents"},"toolbar":{"id":"2bb7d9a1-8c78-4cde-af39-f8eabb690582","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"4ca0bb4c-dc4d-4e88-9c27-d657bfabf665","type":"DataRange1d"},"x_scale":{"id":"dc7817a3-cd49-4b6a-89f8-537ebc9b77e2","type":"LinearScale"},"y_range":{"id":"c254a9b2-bef4-4b72-b5aa-f2c82c2caf7a","type":"DataRange1d"},"y_scale":{"id":"8248c6db-dbe7-456c-ab26-c3643aa8f701","type":"LinearScale"}},"id":"4a3d7c66-d194-4313-bfb7-3005c94dbc3c","subtype":"Figure","type":"Plot"},{"attributes":{"items":[{"id":"11f43b07-449e-4df3-882c-15505a06b258","type":"LegendItem"},{"id":"58547ae1-621e-4d92-ae27-5c68081b16e1","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"097ea389-c4b4-49e1-9023-05ce1d47706a","subtype":"Figure","type":"Plot"}},"id":"c61d2cac-746c-4c5c-916c-7d00d7d3630a","type":"Legend"},{"attributes":{},"id":"7244a45f-c85d-4125-a7ae-0f84923c5fe7","type":"ToolEvents"},{"attributes":{"data_source":{"id":"9edfe087-a9db-4571-ad5b-4681c0b86949","type":"ColumnDataSource"},"glyph":{"id":"9939fc53-c6da-4a01-8f13-c4bd2d5353e5","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"2164bbae-a4b3-4db0-82ec-9b38abaf3bc0","type":"Circle"},"selection_glyph":null},"id":"996f1ef0-18fc-47d7-b849-38ded7d681b6","type":"GlyphRenderer"},{"attributes":{},"id":"06e67db3-4220-4bc5-9c70-f47ea8741d48","type":"BasicTickFormatter"},{"attributes":{"label":{"value":"x.std()"},"renderers":[{"id":"b058e492-67e4-41d1-bbb8-f32eae7b8603","type":"GlyphRenderer"}]},"id":"11f43b07-449e-4df3-882c-15505a06b258","type":"LegendItem"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"3940e366-9f09-44b8-97f8-2bd1a03e885c","type":"PanTool"},{"id":"285fd6ac-775c-4e1a-936a-68a5442242b7","type":"WheelZoomTool"},{"id":"8546812f-01ca-4877-98fd-c31364acbe3d","type":"BoxZoomTool"},{"id":"00dffc1d-3080-4d12-b51b-e86572d80407","type":"SaveTool"},{"id":"44cf5793-22b4-47bf-a7a1-da3521d2ceea","type":"ResetTool"},{"id":"1e78d025-bd71-4310-9185-a2857b169623","type":"HelpTool"}]},"id":"2bb7d9a1-8c78-4cde-af39-f8eabb690582","type":"Toolbar"},{"attributes":{"data_source":{"id":"2922c8ad-35b5-4e79-9792-6becb9fe41b8","type":"ColumnDataSource"},"glyph":{"id":"3ab5e293-8325-4cf4-a709-89e313909a73","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"eda3af37-fd3e-4b25-b800-c5e2cd345759","type":"Line"},"selection_glyph":null},"id":"bc3d125d-ffb3-4a2c-aa72-ab9513b02a62","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"e761952d-458d-4815-b822-4649d8dc2644","subtype":"Figure","type":"Plot"}},"id":"b9db48f5-2638-4c94-a364-8e39eba47393","type":"PanTool"},{"attributes":{"axis_label":"cores","formatter":{"id":"06e67db3-4220-4bc5-9c70-f47ea8741d48","type":"BasicTickFormatter"},"plot":{"id":"4a3d7c66-d194-4313-bfb7-3005c94dbc3c","subtype":"Figure","type":"Plot"},"ticker":{"id":"17a64fd8-de2d-413d-8fb7-7b0e2aca8c2f","type":"FixedTicker"}},"id":"0c40f414-7115-4162-ac89-d192308d67d9","type":"LinearAxis"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"140ce005-bb9d-46cc-8eda-564195a0245d","type":"Circle"},{"attributes":{"data_source":{"id":"497683f3-df7a-41af-ae53-1146920e8780","type":"ColumnDataSource"},"glyph":{"id":"140ce005-bb9d-46cc-8eda-564195a0245d","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"5887b407-1bf1-40fc-a861-ab937970ec2e","type":"Circle"},"selection_glyph":null},"id":"d83e7c85-974f-43a8-bf57-d9ab409aad72","type":"GlyphRenderer"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"efc0b3c2-8ca9-460c-ab49-b51a72221c59","type":"Line"},{"attributes":{},"id":"ff027743-2af6-4770-9184-cc03c83a71bd","type":"BasicTicker"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"5887b407-1bf1-40fc-a861-ab937970ec2e","type":"Circle"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"a7b57c45-0aa0-464d-a1f7-22b769b5acee","type":"Line"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"50315001-4f91-4b1a-b4e7-b4ef44c3ce43","type":"BasicTickFormatter"},"plot":{"id":"4a3d7c66-d194-4313-bfb7-3005c94dbc3c","subtype":"Figure","type":"Plot"},"ticker":{"id":"86a3e6f6-a663-434e-b8b7-44bbc19c24a9","type":"BasicTicker"}},"id":"a86681af-270c-426f-99dc-5fe3d642ee83","type":"LinearAxis"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"2164bbae-a4b3-4db0-82ec-9b38abaf3bc0","type":"Circle"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"dfeb258a-6b2a-4e5f-998b-e0fada92b7f4","type":"Line"},{"attributes":{"data_source":{"id":"8be1d71b-98a0-4083-9421-aeab40ecde25","type":"ColumnDataSource"},"glyph":{"id":"90b73c90-6751-497b-91a4-2687c0e9afff","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"2ed27db1-4511-40c7-9db0-12bd6aaf650b","type":"Line"},"selection_glyph":null},"id":"b4aad3bd-05de-476d-a05d-64797c63e4ab","type":"GlyphRenderer"},{"attributes":{},"id":"86a3e6f6-a663-434e-b8b7-44bbc19c24a9","type":"BasicTicker"},{"attributes":{"data_source":{"id":"7b5f2355-7a70-4f3d-8db3-05c17d0c3f53","type":"ColumnDataSource"},"glyph":{"id":"9634e806-9f97-4ba2-93d3-f7411b9ca83c","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"d3c423e7-6dd5-429c-a8ea-24c463171dbc","type":"Line"},"selection_glyph":null},"id":"07b8d079-8049-4102-a197-ff15e44f8107","type":"GlyphRenderer"},{"attributes":{"dimension":1,"plot":{"id":"4a3d7c66-d194-4313-bfb7-3005c94dbc3c","subtype":"Figure","type":"Plot"},"ticker":{"id":"86a3e6f6-a663-434e-b8b7-44bbc19c24a9","type":"BasicTicker"}},"id":"cba051c4-7034-4651-af30-532eed93f314","type":"Grid"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"3ab5e293-8325-4cf4-a709-89e313909a73","type":"Line"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"770006c8-d877-4b90-be85-1b312cc721a6","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"eda3af37-fd3e-4b25-b800-c5e2cd345759","type":"Line"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"b4587e42-522f-44d1-bdae-1770fdb06934","type":"FixedTicker"},{"attributes":{"data_source":{"id":"725dc24c-20c8-4bb8-a124-50832fa60c95","type":"ColumnDataSource"},"glyph":{"id":"770006c8-d877-4b90-be85-1b312cc721a6","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"efc0b3c2-8ca9-460c-ab49-b51a72221c59","type":"Line"},"selection_glyph":null},"id":"be28dc51-16ef-4fd0-8f80-c604f3e6eaad","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"gT1T0rcmkkBfZLyai4CeQPrpFfFJcqpAtJAQZsY2tkD936fnDJnCQEN9+GXL79BAP/LDy9Ijz0A7bbDIfRrUQJv5zSYbI9NA","dtype":"float64","shape":[9]}}},"id":"9edfe087-a9db-4571-ad5b-4681c0b86949","type":"ColumnDataSource"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"9939fc53-c6da-4a01-8f13-c4bd2d5353e5","type":"Circle"},{"attributes":{"child":{"id":"4a3d7c66-d194-4313-bfb7-3005c94dbc3c","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"da378e56-f9e8-445e-b465-2cde4901f11a","type":"Panel"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"0eb06f75-3f93-4b46-835e-a4511fcd840d","type":"Panel"},{"id":"a41999c2-103e-46f4-b55e-3c9610dd09da","type":"Panel"}]},"id":"7d705d97-1eec-49b4-bda6-9eebdb487600","type":"Tabs"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"e66ff0e4-d114-4529-9a0c-97317efe2e91","type":"BoxAnnotation"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"gT1T0rcmkkBfZLyai4CeQPrpFfFJcqpAtJAQZsY2tkD936fnDJnCQEN9+GXL79BAP/LDy9Ijz0A7bbDIfRrUQJv5zSYbI9NA","dtype":"float64","shape":[9]}}},"id":"2922c8ad-35b5-4e79-9792-6becb9fe41b8","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"e761952d-458d-4815-b822-4649d8dc2644","subtype":"Figure","type":"Plot"}},"id":"51b20535-8160-496c-986d-8baaecbb2697","type":"WheelZoomTool"},{"attributes":{"plot":{"id":"4a3d7c66-d194-4313-bfb7-3005c94dbc3c","subtype":"Figure","type":"Plot"}},"id":"3940e366-9f09-44b8-97f8-2bd1a03e885c","type":"PanTool"},{"attributes":{"label":{"value":"x.std(axis=0)"},"renderers":[{"id":"bc3d125d-ffb3-4a2c-aa72-ab9513b02a62","type":"GlyphRenderer"}]},"id":"58547ae1-621e-4d92-ae27-5c68081b16e1","type":"LegendItem"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"SA88gR1ocECqGEzZQ5R8QGxc6tAEo45AjGt/DoKRmEDGXTCinsSgQMrrGdYeO65A67hcfiEHuUA4aLWDa6XIQMM6i3W2GtBA","dtype":"float64","shape":[9]}}},"id":"5765ae37-6a30-4667-90e7-73a12b390a18","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"54df90b3-a53c-4d20-93ed-f281f7681062","type":"ColumnDataSource"},"glyph":{"id":"fcc04348-ebd5-47a9-b747-a8d384d24b91","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"ca723c3f-bcb5-410f-a5fe-59a9ac9a7008","type":"Line"},"selection_glyph":null},"id":"2bac3660-5ea5-4e97-a5d4-862c9ace444c","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"uUoxEbZlhUBIyqgVRxWTQHa8HsxSVKFAbE0WQbrQskAfV+6MsWjBQB8gXbcI8NBAaYcHrUvT1EBXicp+gH7TQA1jrj5WlupA","dtype":"float64","shape":[9]}}},"id":"0ab80c92-5372-4728-a0da-e513a7ae8e2b","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"4a3d7c66-d194-4313-bfb7-3005c94dbc3c","subtype":"Figure","type":"Plot"}},"id":"285fd6ac-775c-4e1a-936a-68a5442242b7","type":"WheelZoomTool"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"fcc04348-ebd5-47a9-b747-a8d384d24b91","type":"Line"},{"attributes":{"plot":null,"text":"DataFrames: Time Series"},"id":"47048622-1378-44e2-99f1-de93b56bab91","type":"Title"},{"attributes":{"overlay":{"id":"e66ff0e4-d114-4529-9a0c-97317efe2e91","type":"BoxAnnotation"},"plot":{"id":"4a3d7c66-d194-4313-bfb7-3005c94dbc3c","subtype":"Figure","type":"Plot"}},"id":"8546812f-01ca-4877-98fd-c31364acbe3d","type":"BoxZoomTool"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"7a2b565a-76a5-47cc-b363-6c71c9e408be","type":"Circle"},{"attributes":{"plot":{"id":"4a3d7c66-d194-4313-bfb7-3005c94dbc3c","subtype":"Figure","type":"Plot"}},"id":"00dffc1d-3080-4d12-b51b-e86572d80407","type":"SaveTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"0/MveXGpk0CwnWLotLafQFzSKK/0oatAGQzzlhiFvEBuV2Og1WzIQFHk16Cvv8pAs1osroIq1EBv606nwBHYQN/8RMqJ9tlA","dtype":"float64","shape":[9]}}},"id":"ce4017e6-9e2d-42c6-90aa-b10b01a952d1","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"4a3d7c66-d194-4313-bfb7-3005c94dbc3c","subtype":"Figure","type":"Plot"}},"id":"44cf5793-22b4-47bf-a7a1-da3521d2ceea","type":"ResetTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,322140.36834698654]}},"id":"8be1d71b-98a0-4083-9421-aeab40ecde25","type":"ColumnDataSource"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"6f728c8d-c3bd-4ac6-b3d5-e7049adc2931","type":"Line"},{"attributes":{"plot":{"id":"4a3d7c66-d194-4313-bfb7-3005c94dbc3c","subtype":"Figure","type":"Plot"}},"id":"1e78d025-bd71-4310-9185-a2857b169623","type":"HelpTool"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"2ed27db1-4511-40c7-9db0-12bd6aaf650b","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"uUoxEbZlhUBIyqgVRxWTQHa8HsxSVKFAbE0WQbrQskAfV+6MsWjBQB8gXbcI8NBAaYcHrUvT1EBXicp+gH7TQA1jrj5WlupA","dtype":"float64","shape":[9]}}},"id":"725dc24c-20c8-4bb8-a124-50832fa60c95","type":"ColumnDataSource"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"ca723c3f-bcb5-410f-a5fe-59a9ac9a7008","type":"Line"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"f240d2c0-ff6c-491a-ab4e-61eae5d526ed","type":"PanTool"},{"id":"e2ce7b1b-cf07-411c-b287-d1b33ecf17f9","type":"WheelZoomTool"},{"id":"d9cf7207-525f-49a5-9224-035cd6dfc2dc","type":"BoxZoomTool"},{"id":"e4a0baa0-5ecc-4689-a4f4-6ee954c27052","type":"SaveTool"},{"id":"58423d76-834c-42ce-9dc5-b8729ce97321","type":"ResetTool"},{"id":"9b541db3-5f39-403a-ac85-bd080ebaa38a","type":"HelpTool"}]},"id":"bcf7b40a-0310-4ba6-8071-f0219b20abff","type":"Toolbar"},{"attributes":{"child":{"id":"764739d9-0946-4e3c-92d5-e83dab730b60","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"a5b8f31d-2417-4153-b180-1655c30646ec","type":"Panel"},{"attributes":{},"id":"50315001-4f91-4b1a-b4e7-b4ef44c3ce43","type":"BasicTickFormatter"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"ec591052-6ee6-4799-aba5-cec91ccbd76c","type":"FixedTicker"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"9e1ea25f-8e06-4321-86cf-a7eb3ae41d67","type":"Line"},{"attributes":{"callback":null,"start":0},"id":"ed5a7b95-48a4-4fec-ac8f-6f16eaf0c5d8","type":"DataRange1d"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,297389.95539566135]}},"id":"54df90b3-a53c-4d20-93ed-f281f7681062","type":"ColumnDataSource"},{"attributes":{"items":[{"id":"1cb14e57-d9e9-437f-8a5d-e10f8e72cc8f","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"4a3d7c66-d194-4313-bfb7-3005c94dbc3c","subtype":"Figure","type":"Plot"}},"id":"981660b0-6865-4ab3-a1e0-7f28cb9baec2","type":"Legend"},{"attributes":{"plot":null,"text":"Arrays: Nearest Neighbor"},"id":"c165289a-4f05-43ad-a4ca-1a91390483bf","type":"Title"},{"attributes":{},"id":"83201b83-a0a4-4eec-a296-5f52f8bbd13c","type":"ToolEvents"},{"attributes":{},"id":"6b2e1f16-bda3-4246-a7a9-7bfb330fc60f","type":"ToolEvents"},{"attributes":{"below":[{"id":"9ec87ca6-9a84-4edf-84fa-bec88e938f62","type":"LinearAxis"}],"left":[{"id":"8b50a8f3-cb96-4f46-b0eb-421532888322","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"9ec87ca6-9a84-4edf-84fa-bec88e938f62","type":"LinearAxis"},{"id":"940bc7b4-5373-490f-881f-eef70bc28979","type":"Grid"},{"id":"8b50a8f3-cb96-4f46-b0eb-421532888322","type":"LinearAxis"},{"id":"badd4c30-b02a-4b7c-b266-8b9a6136e084","type":"Grid"},{"id":"6e2d097b-519f-4d25-9d8f-71339b740fee","type":"BoxAnnotation"},{"id":"12862a29-a060-4fff-b654-18b2c2b74aa7","type":"Legend"},{"id":"a881979e-2750-4ccf-b89d-e358d963f201","type":"GlyphRenderer"},{"id":"1220894e-d5c5-4272-bf99-4f2213703bda","type":"GlyphRenderer"},{"id":"81e42002-a8be-4345-b4e1-df093044c5f1","type":"GlyphRenderer"},{"id":"72786059-6377-45c0-9914-95820228350f","type":"GlyphRenderer"},{"id":"7a8d5d6f-8fb1-4d50-88a8-5a53336ee7fa","type":"GlyphRenderer"},{"id":"123e0dbf-a6dd-488a-88e8-eadf20a2fe99","type":"GlyphRenderer"}],"title":{"id":"c165289a-4f05-43ad-a4ca-1a91390483bf","type":"Title"},"tool_events":{"id":"6b2e1f16-bda3-4246-a7a9-7bfb330fc60f","type":"ToolEvents"},"toolbar":{"id":"d1d8a1b5-1532-4a74-9e7d-3554d73a6513","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"ae5f6fd6-f43f-466a-ba47-9ec627c63007","type":"DataRange1d"},"x_scale":{"id":"68486794-1408-4221-b928-fa766fa764ad","type":"LinearScale"},"y_range":{"id":"749e5bb6-c494-4e2e-9e01-e5861c6f02f1","type":"DataRange1d"},"y_scale":{"id":"d67d8d4b-835a-41e8-8bc5-d4a6faf5158f","type":"LinearScale"}},"id":"764739d9-0946-4e3c-92d5-e83dab730b60","subtype":"Figure","type":"Plot"},{"attributes":{"label":{"value":"random"},"renderers":[{"id":"be28dc51-16ef-4fd0-8f80-c604f3e6eaad","type":"GlyphRenderer"}]},"id":"1cb14e57-d9e9-437f-8a5d-e10f8e72cc8f","type":"LegendItem"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"9cdbfea0-78a1-4816-95b3-20711d60ba64","type":"PanTool"},{"id":"d917a161-3ebd-4ff0-9e66-b1890604b6c7","type":"WheelZoomTool"},{"id":"c2e5b325-c321-4c08-b3da-cdf89da70574","type":"BoxZoomTool"},{"id":"60a7d2f5-41bc-4215-ac31-864b2bf34829","type":"SaveTool"},{"id":"66346441-b239-489b-900f-c4d4660d7478","type":"ResetTool"},{"id":"9e9eec61-23ba-4987-af83-bc09e49775bd","type":"HelpTool"}]},"id":"d1d8a1b5-1532-4a74-9e7d-3554d73a6513","type":"Toolbar"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"84379b7b-549c-4cb0-ace2-a9135bc8555a","type":"Circle"},{"attributes":{"data_source":{"id":"52629581-796c-4d48-a9b9-0aaa7e448f8f","type":"ColumnDataSource"},"glyph":{"id":"5002a81e-5bc4-4477-ad34-84bc9ee553ec","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"4ae58565-4539-49f2-8237-6af4d1b7690a","type":"Line"},"selection_glyph":null},"id":"d1ea2033-418b-42dc-affa-05ab97a548a6","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"41999ae5-ee50-4180-8a07-fdec3748db8d","subtype":"Figure","type":"Plot"}},"id":"61d07d7f-f947-4d2a-b341-c4bf85907198","type":"WheelZoomTool"},{"attributes":{"overlay":{"id":"85719b8d-547b-42d8-a67b-dea193b89ff1","type":"BoxAnnotation"},"plot":{"id":"41999ae5-ee50-4180-8a07-fdec3748db8d","subtype":"Figure","type":"Plot"}},"id":"57656b7a-1720-478f-ac0a-95a597024f3f","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"41999ae5-ee50-4180-8a07-fdec3748db8d","subtype":"Figure","type":"Plot"}},"id":"15a60aeb-0901-43a5-a728-241b8af7fc12","type":"SaveTool"},{"attributes":{"plot":{"id":"41999ae5-ee50-4180-8a07-fdec3748db8d","subtype":"Figure","type":"Plot"}},"id":"a1f08da9-1313-44bd-8f3b-3a3d3c43eb46","type":"ResetTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"zYZNznMFgkAJVc0dH1aIQF6HLbAX8KBAEDiFMgobsEBOSk1Dh6G8QIcwzAFNYsdAkYwvunD+0kD/VzpMtBLaQNxDJJJ7lt5A","dtype":"float64","shape":[9]}}},"id":"0799bb10-bfb8-4366-a4c5-8c73a10400cb","type":"ColumnDataSource"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"8ddef416-c675-4c4d-b5a5-1fc5ef76f4d9","type":"Circle"},{"attributes":{"data_source":{"id":"0799bb10-bfb8-4366-a4c5-8c73a10400cb","type":"ColumnDataSource"},"glyph":{"id":"8ddef416-c675-4c4d-b5a5-1fc5ef76f4d9","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"8fac1d07-a0e0-4f17-bfbc-528790c4164b","type":"Circle"},"selection_glyph":null},"id":"43f1cd74-dc1a-4aab-8f12-2d6e8620b2f3","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"8fac1d07-a0e0-4f17-bfbc-528790c4164b","type":"Circle"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"3e1ace13-0a5c-46b9-9e15-ef3348af7d36","type":"Line"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"ff7e9255-d098-4787-a969-19126cf71054","type":"Panel"},{"id":"53f31c0f-fad8-43d0-ab0c-d7fffe937e21","type":"Panel"}]},"id":"4d110cea-f4d8-49a1-bd97-89adb074f930","type":"Tabs"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"5002a81e-5bc4-4477-ad34-84bc9ee553ec","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"4ae58565-4539-49f2-8237-6af4d1b7690a","type":"Line"},{"attributes":{"label":{"value":"df[0].std()"},"renderers":[{"id":"ef803fd0-35ee-436c-b303-af5e06365f2e","type":"GlyphRenderer"}]},"id":"2f3b9045-758c-457b-82e4-33b756b648ca","type":"LegendItem"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"VDws0AG1nUDZ+18BvWCtQLxzwc5E/bpAaCwXiat5w0Bb8Sa8jEzHQLDjXzrvA8JAB24/B96Gw0AcKVqQRQLDQPc5BRJjuMJA","dtype":"float64","shape":[9]}}},"id":"28d7c288-74fa-4713-a4c4-b5132993efaa","type":"ColumnDataSource"},{"attributes":{"below":[{"id":"47214d03-7988-4f6f-8fe2-b8516eddf652","type":"LogAxis"}],"left":[{"id":"37a11cef-799c-4edc-afd4-a9de54bf0603","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"47214d03-7988-4f6f-8fe2-b8516eddf652","type":"LogAxis"},{"id":"4474c789-1ca5-4c52-bd13-987e6a967ee2","type":"Grid"},{"id":"37a11cef-799c-4edc-afd4-a9de54bf0603","type":"LogAxis"},{"id":"6652ad33-92c0-4906-879d-ad91091a9bf1","type":"Grid"},{"id":"718a4478-4b1a-41f2-8226-379ae6e8f2bd","type":"BoxAnnotation"},{"id":"d299c857-0c66-423b-b2de-f913d01560fd","type":"Legend"},{"id":"a6793c13-f7d5-44e2-a22d-25ec492a5b20","type":"GlyphRenderer"},{"id":"fdfed4e2-9e87-4e34-b162-bd3ae7ae904c","type":"GlyphRenderer"},{"id":"21efc981-8921-4624-9d1b-0a0b59115c83","type":"GlyphRenderer"}],"title":{"id":"2134e970-89f1-4641-a947-005ad19e747d","type":"Title"},"tool_events":{"id":"55638ec9-8e5e-46b2-bf60-b73f95746859","type":"ToolEvents"},"toolbar":{"id":"558347a4-b08d-474f-ba3e-19985a988d80","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"861cd34d-128e-4bba-bdfb-68e9bf495dfb","type":"DataRange1d"},"x_scale":{"id":"16067309-af37-4e3e-b725-219f604c0294","type":"LogScale"},"y_range":{"id":"8801673f-bd19-4c63-b61b-7cb44aa99202","type":"DataRange1d"},"y_scale":{"id":"20d88806-06b2-4982-a565-582717e24c44","type":"LogScale"}},"id":"bd53afa0-afcc-4aa5-a575-afdfd9223821","subtype":"Figure","type":"Plot"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"VDws0AG1nUDZ+18BvWCtQLxzwc5E/bpAaCwXiat5w0Bb8Sa8jEzHQLDjXzrvA8JAB24/B96Gw0AcKVqQRQLDQPc5BRJjuMJA","dtype":"float64","shape":[9]}}},"id":"52629581-796c-4d48-a9b9-0aaa7e448f8f","type":"ColumnDataSource"},{"attributes":{"plot":null,"text":"DataFrames: Random Access"},"id":"8b58d2e1-940d-46bb-b62b-49993c97f583","type":"Title"},{"attributes":{"label":{"value":"arithmetic"},"renderers":[{"id":"d1ea2033-418b-42dc-affa-05ab97a548a6","type":"GlyphRenderer"}]},"id":"49216e78-0ce7-4ef2-8c38-04c33f920efb","type":"LegendItem"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"a8ef1855-8b30-4859-8c14-b570f5b7928a","type":"Line"},{"attributes":{"below":[{"id":"753f4ce9-d390-4b58-a059-c518a5e49ce1","type":"LinearAxis"}],"left":[{"id":"4d454c38-1899-4eed-9645-d410fe6182dc","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"753f4ce9-d390-4b58-a059-c518a5e49ce1","type":"LinearAxis"},{"id":"6006aa04-b603-4bf5-98ac-0f3e964532b4","type":"Grid"},{"id":"4d454c38-1899-4eed-9645-d410fe6182dc","type":"LinearAxis"},{"id":"c8db7769-7b21-4e5f-bc50-de9b79a58acb","type":"Grid"},{"id":"1fbe31f9-bc92-471f-a1c7-7c615e5a8ceb","type":"BoxAnnotation"},{"id":"36cef690-7379-487b-ab9a-3fcf6b698d56","type":"Legend"},{"id":"c98728aa-5b20-45cb-b0eb-e000c83d1e0b","type":"GlyphRenderer"},{"id":"1b7b8508-2bf1-4928-822f-ddf0e68a8e6d","type":"GlyphRenderer"},{"id":"09ee4a96-d17c-425a-963e-4577b347ca9a","type":"GlyphRenderer"}],"title":{"id":"8b58d2e1-940d-46bb-b62b-49993c97f583","type":"Title"},"tool_events":{"id":"5ba50c74-b6e8-4949-aa94-e04464dfcdf6","type":"ToolEvents"},"toolbar":{"id":"fb365b69-0937-4b12-aed7-3664926439af","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"31a37c7a-531a-4856-9c21-0374ecd89715","type":"DataRange1d"},"x_scale":{"id":"c26c51cc-fbf2-4dd2-b059-8d10d2e7d09d","type":"LinearScale"},"y_range":{"id":"20e62bf1-e0d7-446b-a801-250ba78d82d1","type":"DataRange1d"},"y_scale":{"id":"8513aafd-08a4-4bd5-a114-e57d57ae92e6","type":"LinearScale"}},"id":"a86def8f-0374-4c0f-85c5-de548a1dbba5","subtype":"Figure","type":"Plot"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"ff541f2a-7cc8-4ffc-ba5f-f7fa99a2277e","type":"Circle"},{"attributes":{"data_source":{"id":"f8412bd2-b8d7-45ae-bf83-be9f99c253c3","type":"ColumnDataSource"},"glyph":{"id":"a8ef1855-8b30-4859-8c14-b570f5b7928a","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"d966efd2-13a1-4d33-ad47-6ed8a16258a9","type":"Line"},"selection_glyph":null},"id":"1536de68-3f85-4ec4-9468-0e3fc8b2e64e","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"2ba9c818-95cb-4c26-bb18-08574a27ac76","type":"Circle"},{"attributes":{"data_source":{"id":"28d7c288-74fa-4713-a4c4-b5132993efaa","type":"ColumnDataSource"},"glyph":{"id":"ff541f2a-7cc8-4ffc-ba5f-f7fa99a2277e","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"2ba9c818-95cb-4c26-bb18-08574a27ac76","type":"Circle"},"selection_glyph":null},"id":"78d02e69-5824-47a6-92da-f24d594ffdac","type":"GlyphRenderer"},{"attributes":{},"id":"5ba50c74-b6e8-4949-aa94-e04464dfcdf6","type":"ToolEvents"},{"attributes":{"data_source":{"id":"0084dd1d-b5b6-4249-b7dd-4d81ae8d44d9","type":"ColumnDataSource"},"glyph":{"id":"e07a0b7a-e02f-4857-8704-45e71b5bc074","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"2437b245-9f04-4e74-afb9-429ff4bc323e","type":"Line"},"selection_glyph":null},"id":"66a1bbc2-db12-4787-93f9-0f0b4ed2a298","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,147630.4757338077]}},"id":"f8412bd2-b8d7-45ae-bf83-be9f99c253c3","type":"ColumnDataSource"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"d966efd2-13a1-4d33-ad47-6ed8a16258a9","type":"Line"},{"attributes":{},"id":"16067309-af37-4e3e-b725-219f604c0294","type":"LogScale"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"6d26b10e-948b-4ae1-b8a0-c48032e939cc","type":"FixedTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,123285.74943667026]}},"id":"8250fc8b-ed67-44ae-b653-795a1a058612","type":"ColumnDataSource"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"7537c165-ab7c-43e5-8751-187c14d86800","type":"FixedTicker"},{"attributes":{},"id":"55638ec9-8e5e-46b2-bf60-b73f95746859","type":"ToolEvents"},{"attributes":{"plot":null,"text":"DataFrames: Random Access"},"id":"2134e970-89f1-4641-a947-005ad19e747d","type":"Title"},{"attributes":{"callback":null,"start":0},"id":"861cd34d-128e-4bba-bdfb-68e9bf495dfb","type":"DataRange1d"},{"attributes":{"axis_label":"cores","formatter":{"id":"bfae64d2-e17a-4c51-b57c-6ac963197229","type":"LogTickFormatter"},"plot":{"id":"bd53afa0-afcc-4aa5-a575-afdfd9223821","subtype":"Figure","type":"Plot"},"ticker":{"id":"7537c165-ab7c-43e5-8751-187c14d86800","type":"FixedTicker"}},"id":"47214d03-7988-4f6f-8fe2-b8516eddf652","type":"LogAxis"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"de25869c-2180-4518-a20f-bdf692f81aa1","type":"PanTool"},{"id":"a4b243e0-e791-4a8d-aec9-365070929b02","type":"WheelZoomTool"},{"id":"b9e581e8-8947-4185-bb76-39863c3ef43d","type":"BoxZoomTool"},{"id":"bcecb8b5-28bd-4f4f-bdb3-a693d7ebae56","type":"SaveTool"},{"id":"3270958b-802a-4185-991e-b40fd50477a0","type":"ResetTool"},{"id":"ecdc3244-d890-4831-ae2f-4b9a127b2173","type":"HelpTool"}]},"id":"558347a4-b08d-474f-ba3e-19985a988d80","type":"Toolbar"},{"attributes":{"num_minor_ticks":10},"id":"d557960d-4ccd-4c90-a6c6-2d0dd129d409","type":"LogTicker"},{"attributes":{"num_minor_ticks":10},"id":"dd8262db-e0fb-4c10-b9ed-67bdec670c63","type":"LogTicker"},{"attributes":{"callback":null,"end":543.5412502227334,"start":0},"id":"8801673f-bd19-4c63-b61b-7cb44aa99202","type":"DataRange1d"},{"attributes":{"ticker":null},"id":"bfae64d2-e17a-4c51-b57c-6ac963197229","type":"LogTickFormatter"},{"attributes":{"plot":{"id":"bd53afa0-afcc-4aa5-a575-afdfd9223821","subtype":"Figure","type":"Plot"},"ticker":{"id":"d557960d-4ccd-4c90-a6c6-2d0dd129d409","type":"LogTicker"}},"id":"4474c789-1ca5-4c52-bd13-987e6a967ee2","type":"Grid"},{"attributes":{"plot":{"id":"bd53afa0-afcc-4aa5-a575-afdfd9223821","subtype":"Figure","type":"Plot"}},"id":"de25869c-2180-4518-a20f-bdf692f81aa1","type":"PanTool"},{"attributes":{"axis_label":"bytes/s","formatter":{"id":"ac0e7478-7984-42f8-af1c-1e95a6dff490","type":"LogTickFormatter"},"plot":{"id":"bd53afa0-afcc-4aa5-a575-afdfd9223821","subtype":"Figure","type":"Plot"},"ticker":{"id":"dd8262db-e0fb-4c10-b9ed-67bdec670c63","type":"LogTicker"}},"id":"37a11cef-799c-4edc-afd4-a9de54bf0603","type":"LogAxis"},{"attributes":{},"id":"20d88806-06b2-4982-a565-582717e24c44","type":"LogScale"},{"attributes":{"data_source":{"id":"f9e1f8dd-2451-4dee-8066-39e2c4b18c1d","type":"ColumnDataSource"},"glyph":{"id":"4309a4f6-e218-43e5-833f-968ad0aaeb9e","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"3e1ace13-0a5c-46b9-9e15-ef3348af7d36","type":"Line"},"selection_glyph":null},"id":"a6793c13-f7d5-44e2-a22d-25ec492a5b20","type":"GlyphRenderer"},{"attributes":{"dimension":1,"plot":{"id":"bd53afa0-afcc-4aa5-a575-afdfd9223821","subtype":"Figure","type":"Plot"},"ticker":{"id":"dd8262db-e0fb-4c10-b9ed-67bdec670c63","type":"LogTicker"}},"id":"6652ad33-92c0-4906-879d-ad91091a9bf1","type":"Grid"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"4309a4f6-e218-43e5-833f-968ad0aaeb9e","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"U06x/VsZfkD5iFMkcqh/QEM6qdQn+H5AoYHbLshYfUBM9NAUrJp+QL4s/3pU/IBApWWsI7IYfkArSDatYKF1QLe4CmYnkXZA","dtype":"float64","shape":[9]}}},"id":"2e54a943-d381-4da1-b144-b9546371fec2","type":"ColumnDataSource"},{"attributes":{"child":{"id":"bd53afa0-afcc-4aa5-a575-afdfd9223821","subtype":"Figure","type":"Plot"},"title":"log"},"id":"ff7e9255-d098-4787-a969-19126cf71054","type":"Panel"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"718a4478-4b1a-41f2-8226-379ae6e8f2bd","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"bd53afa0-afcc-4aa5-a575-afdfd9223821","subtype":"Figure","type":"Plot"}},"id":"a4b243e0-e791-4a8d-aec9-365070929b02","type":"WheelZoomTool"},{"attributes":{"plot":{"id":"bd53afa0-afcc-4aa5-a575-afdfd9223821","subtype":"Figure","type":"Plot"}},"id":"3270958b-802a-4185-991e-b40fd50477a0","type":"ResetTool"},{"attributes":{"overlay":{"id":"718a4478-4b1a-41f2-8226-379ae6e8f2bd","type":"BoxAnnotation"},"plot":{"id":"bd53afa0-afcc-4aa5-a575-afdfd9223821","subtype":"Figure","type":"Plot"}},"id":"b9e581e8-8947-4185-bb76-39863c3ef43d","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"bd53afa0-afcc-4aa5-a575-afdfd9223821","subtype":"Figure","type":"Plot"}},"id":"bcecb8b5-28bd-4f4f-bdb3-a693d7ebae56","type":"SaveTool"},{"attributes":{"plot":{"id":"bd53afa0-afcc-4aa5-a575-afdfd9223821","subtype":"Figure","type":"Plot"}},"id":"ecdc3244-d890-4831-ae2f-4b9a127b2173","type":"HelpTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"U06x/VsZfkD5iFMkcqh/QEM6qdQn+H5AoYHbLshYfUBM9NAUrJp+QL4s/3pU/IBApWWsI7IYfkArSDatYKF1QLe4CmYnkXZA","dtype":"float64","shape":[9]}}},"id":"f9e1f8dd-2451-4dee-8066-39e2c4b18c1d","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"8250fc8b-ed67-44ae-b653-795a1a058612","type":"ColumnDataSource"},"glyph":{"id":"06c3006f-0774-43f6-aadf-bae7b2da8362","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"c092a6df-51a8-4ab0-9c1c-dd94fa07f46d","type":"Line"},"selection_glyph":null},"id":"21efc981-8921-4624-9d1b-0a0b59115c83","type":"GlyphRenderer"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"c092a6df-51a8-4ab0-9c1c-dd94fa07f46d","type":"Line"},{"attributes":{"ticker":null},"id":"ac0e7478-7984-42f8-af1c-1e95a6dff490","type":"LogTickFormatter"},{"attributes":{"items":[{"id":"0c7b3508-6eb3-4183-9aa0-0751a36a881f","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"bd53afa0-afcc-4aa5-a575-afdfd9223821","subtype":"Figure","type":"Plot"}},"id":"d299c857-0c66-423b-b2de-f913d01560fd","type":"Legend"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"06c3006f-0774-43f6-aadf-bae7b2da8362","type":"Line"},{"attributes":{"label":{"value":"df.loc[123456]"},"renderers":[{"id":"a6793c13-f7d5-44e2-a22d-25ec492a5b20","type":"GlyphRenderer"}]},"id":"0c7b3508-6eb3-4183-9aa0-0751a36a881f","type":"LegendItem"},{"attributes":{"callback":null,"end":543.5412502227334,"start":0},"id":"20e62bf1-e0d7-446b-a801-250ba78d82d1","type":"DataRange1d"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"ece331ec-6fb2-4089-8b37-3920e038c682","type":"PanTool"},{"id":"01965bf4-e8c4-4ab4-ba11-d812d609dfe5","type":"WheelZoomTool"},{"id":"5bae1a67-b093-424d-9098-79c5e411c1e5","type":"BoxZoomTool"},{"id":"20d93f6e-d1ac-41f7-9eb4-5bb797a77934","type":"SaveTool"},{"id":"17d27847-a15f-4d57-8436-dd60b85219ef","type":"ResetTool"},{"id":"2c4661b2-6594-4d44-941f-b9b29a09447a","type":"HelpTool"}]},"id":"fb365b69-0937-4b12-aed7-3664926439af","type":"Toolbar"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"18656b28-aadf-4f16-a578-beee40457510","type":"Circle"},{"attributes":{"data_source":{"id":"2e54a943-d381-4da1-b144-b9546371fec2","type":"ColumnDataSource"},"glyph":{"id":"18656b28-aadf-4f16-a578-beee40457510","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"3b05f731-e4fa-442b-a87e-4a4bd72a215c","type":"Circle"},"selection_glyph":null},"id":"fdfed4e2-9e87-4e34-b162-bd3ae7ae904c","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"3b05f731-e4fa-442b-a87e-4a4bd72a215c","type":"Circle"},{"attributes":{"data_source":{"id":"c6b1dde5-997a-42d4-a3be-0099f9fafcca","type":"ColumnDataSource"},"glyph":{"id":"3ba055f8-c19e-4f01-80ac-432301ad1555","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"9dde85bf-6688-4bae-b751-cbf4ecd3c2bd","type":"Line"},"selection_glyph":null},"id":"09ee4a96-d17c-425a-963e-4577b347ca9a","type":"GlyphRenderer"},{"attributes":{"callback":null,"start":0},"id":"31a37c7a-531a-4856-9c21-0374ecd89715","type":"DataRange1d"},{"attributes":{"child":{"id":"a86def8f-0374-4c0f-85c5-de548a1dbba5","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"53f31c0f-fad8-43d0-ab0c-d7fffe937e21","type":"Panel"},{"attributes":{},"id":"c26c51cc-fbf2-4dd2-b059-8d10d2e7d09d","type":"LinearScale"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"18a4fa8f-8a4f-4a6d-93bf-f68a0417ac7e","type":"Line"},{"attributes":{},"id":"8513aafd-08a4-4bd5-a114-e57d57ae92e6","type":"LinearScale"},{"attributes":{"plot":{"id":"a86def8f-0374-4c0f-85c5-de548a1dbba5","subtype":"Figure","type":"Plot"},"ticker":{"id":"9a9dfa8c-8d3c-4808-b61c-ade03c69d709","type":"BasicTicker"}},"id":"6006aa04-b603-4bf5-98ac-0f3e964532b4","type":"Grid"},{"attributes":{"axis_label":"cores","formatter":{"id":"f6138c94-b325-4c78-9ddc-db2f542e7ad5","type":"BasicTickFormatter"},"plot":{"id":"a86def8f-0374-4c0f-85c5-de548a1dbba5","subtype":"Figure","type":"Plot"},"ticker":{"id":"dd63b41d-c89b-457f-ae85-56a166c15053","type":"FixedTicker"}},"id":"753f4ce9-d390-4b58-a059-c518a5e49ce1","type":"LinearAxis"},{"attributes":{},"id":"9a9dfa8c-8d3c-4808-b61c-ade03c69d709","type":"BasicTicker"},{"attributes":{"axis_label":"bytes/s","formatter":{"id":"da1c3229-7ed4-4b85-8dc8-f1a764009e6c","type":"BasicTickFormatter"},"plot":{"id":"a86def8f-0374-4c0f-85c5-de548a1dbba5","subtype":"Figure","type":"Plot"},"ticker":{"id":"5f0ecc27-22ea-4eb4-b717-caa026a4f56a","type":"BasicTicker"}},"id":"4d454c38-1899-4eed-9645-d410fe6182dc","type":"LinearAxis"},{"attributes":{},"id":"5f0ecc27-22ea-4eb4-b717-caa026a4f56a","type":"BasicTicker"},{"attributes":{"dimension":1,"plot":{"id":"a86def8f-0374-4c0f-85c5-de548a1dbba5","subtype":"Figure","type":"Plot"},"ticker":{"id":"5f0ecc27-22ea-4eb4-b717-caa026a4f56a","type":"BasicTicker"}},"id":"c8db7769-7b21-4e5f-bc50-de9b79a58acb","type":"Grid"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"b7a5b62c-bd62-4775-bf00-995acba88608","type":"Line"},{"attributes":{},"id":"f6138c94-b325-4c78-9ddc-db2f542e7ad5","type":"BasicTickFormatter"},{"attributes":{"data_source":{"id":"13edcb3a-0db1-45d7-b214-0bc590030c00","type":"ColumnDataSource"},"glyph":{"id":"18a4fa8f-8a4f-4a6d-93bf-f68a0417ac7e","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"b7a5b62c-bd62-4775-bf00-995acba88608","type":"Line"},"selection_glyph":null},"id":"c98728aa-5b20-45cb-b0eb-e000c83d1e0b","type":"GlyphRenderer"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"1fbe31f9-bc92-471f-a1c7-7c615e5a8ceb","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"a86def8f-0374-4c0f-85c5-de548a1dbba5","subtype":"Figure","type":"Plot"}},"id":"ece331ec-6fb2-4089-8b37-3920e038c682","type":"PanTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"U06x/VsZfkD5iFMkcqh/QEM6qdQn+H5AoYHbLshYfUBM9NAUrJp+QL4s/3pU/IBApWWsI7IYfkArSDatYKF1QLe4CmYnkXZA","dtype":"float64","shape":[9]}}},"id":"489c22a9-e13c-4c2f-bef1-fe3998541433","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"a86def8f-0374-4c0f-85c5-de548a1dbba5","subtype":"Figure","type":"Plot"}},"id":"01965bf4-e8c4-4ab4-ba11-d812d609dfe5","type":"WheelZoomTool"},{"attributes":{"overlay":{"id":"1fbe31f9-bc92-471f-a1c7-7c615e5a8ceb","type":"BoxAnnotation"},"plot":{"id":"a86def8f-0374-4c0f-85c5-de548a1dbba5","subtype":"Figure","type":"Plot"}},"id":"5bae1a67-b093-424d-9098-79c5e411c1e5","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"a86def8f-0374-4c0f-85c5-de548a1dbba5","subtype":"Figure","type":"Plot"}},"id":"20d93f6e-d1ac-41f7-9eb4-5bb797a77934","type":"SaveTool"},{"attributes":{"plot":{"id":"a86def8f-0374-4c0f-85c5-de548a1dbba5","subtype":"Figure","type":"Plot"}},"id":"17d27847-a15f-4d57-8436-dd60b85219ef","type":"ResetTool"},{"attributes":{"plot":{"id":"a86def8f-0374-4c0f-85c5-de548a1dbba5","subtype":"Figure","type":"Plot"}},"id":"2c4661b2-6594-4d44-941f-b9b29a09447a","type":"HelpTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"U06x/VsZfkD5iFMkcqh/QEM6qdQn+H5AoYHbLshYfUBM9NAUrJp+QL4s/3pU/IBApWWsI7IYfkArSDatYKF1QLe4CmYnkXZA","dtype":"float64","shape":[9]}}},"id":"13edcb3a-0db1-45d7-b214-0bc590030c00","type":"ColumnDataSource"},{"attributes":{"below":[{"id":"1e849201-56ff-4961-9fad-ad3e390b0b84","type":"LogAxis"}],"left":[{"id":"5b086bdd-94d7-4ecc-a808-62305578339a","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"1e849201-56ff-4961-9fad-ad3e390b0b84","type":"LogAxis"},{"id":"9a8a88eb-dd52-483a-990f-7450b106004e","type":"Grid"},{"id":"5b086bdd-94d7-4ecc-a808-62305578339a","type":"LogAxis"},{"id":"27788952-6cd6-474e-a0e6-72adb1dd2b27","type":"Grid"},{"id":"ae9e26bd-f1c5-4428-9cf9-a219db087cd7","type":"BoxAnnotation"},{"id":"e94cfdab-a4c2-48a2-8df7-736db028f287","type":"Legend"},{"id":"000f6e25-f829-469e-b780-cb0242e3a7b2","type":"GlyphRenderer"},{"id":"9d9f4b53-3055-4c9d-be31-5b3f8e490207","type":"GlyphRenderer"},{"id":"ef803fd0-35ee-436c-b303-af5e06365f2e","type":"GlyphRenderer"},{"id":"fab646e5-5445-44e6-a6b2-7ea6e79e8bf3","type":"GlyphRenderer"},{"id":"05ced829-496d-462f-ae2d-1f5499a1f090","type":"GlyphRenderer"},{"id":"d7a8e78a-acbf-40cf-8053-6ee0e66f2b88","type":"GlyphRenderer"},{"id":"acfdfd9a-88eb-4304-8a3e-84611f33f47b","type":"GlyphRenderer"},{"id":"671e7a2f-b80f-4a4d-9aef-365c72389908","type":"GlyphRenderer"},{"id":"0ff3ae63-4217-4531-910b-ebe12ee6c8bd","type":"GlyphRenderer"}],"title":{"id":"34326b5a-8a5b-4055-97e5-94ac3ada266b","type":"Title"},"tool_events":{"id":"2d56e258-7c42-4604-967f-0be7f343e299","type":"ToolEvents"},"toolbar":{"id":"5334e99f-7403-4675-8861-86e795329fc0","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"d4cd0f2c-4386-41df-80d9-5fda13c9c483","type":"DataRange1d"},"x_scale":{"id":"4637c9d2-5603-4d27-b014-17d8825b54b6","type":"LogScale"},"y_range":{"id":"6d8b57f3-de64-4dd0-9cf5-0f8ee6bef4e3","type":"DataRange1d"},"y_scale":{"id":"357859f4-cd8a-4d9f-b1ce-545c236f3a28","type":"LogScale"}},"id":"39d25c5a-3587-4f53-81c7-7a8e11174dcc","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"da1c3229-7ed4-4b85-8dc8-f1a764009e6c","type":"BasicTickFormatter"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"3636d692-9f9c-4c45-96c1-6e32c1ba4925","type":"Line"},{"attributes":{"items":[{"id":"f8f20edb-dbe9-4122-aac8-b804b5d834fe","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"a86def8f-0374-4c0f-85c5-de548a1dbba5","subtype":"Figure","type":"Plot"}},"id":"36cef690-7379-487b-ab9a-3fcf6b698d56","type":"Legend"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"d2cd1935-5500-4033-8dbb-e3d6802d8d0c","type":"Panel"},{"id":"26ff1b0e-0ff1-4a7e-be62-eb15afe2b542","type":"Panel"}]},"id":"682f90be-e284-4f12-a195-8e4927a7b6dc","type":"Tabs"},{"attributes":{"label":{"value":"df.loc[123456]"},"renderers":[{"id":"c98728aa-5b20-45cb-b0eb-e000c83d1e0b","type":"GlyphRenderer"}]},"id":"f8f20edb-dbe9-4122-aac8-b804b5d834fe","type":"LegendItem"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"8e2cac76-fee4-459e-9829-7f4e2e92a49d","type":"Circle"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"b00c573b-e447-44b5-8c30-f2a626f5510d","type":"Circle"},{"attributes":{"data_source":{"id":"489c22a9-e13c-4c2f-bef1-fe3998541433","type":"ColumnDataSource"},"glyph":{"id":"b00c573b-e447-44b5-8c30-f2a626f5510d","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"183ee589-72bb-4070-9023-f9eb3b5d18ea","type":"Circle"},"selection_glyph":null},"id":"1b7b8508-2bf1-4928-822f-ddf0e68a8e6d","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"183ee589-72bb-4070-9023-f9eb3b5d18ea","type":"Circle"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"dd63b41d-c89b-457f-ae85-56a166c15053","type":"FixedTicker"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"2a30a863-9480-470a-870b-5008114c481b","type":"Line"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"1f93e094-cb4f-4585-882c-274eeac06db7","type":"Circle"},{"attributes":{"data_source":{"id":"552924f5-e214-4bbf-aeb6-bf3cd02145da","type":"ColumnDataSource"},"glyph":{"id":"1f93e094-cb4f-4585-882c-274eeac06db7","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"8e2cac76-fee4-459e-9829-7f4e2e92a49d","type":"Circle"},"selection_glyph":null},"id":"fab646e5-5445-44e6-a6b2-7ea6e79e8bf3","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"8y0AKWPAZkCdUmbwFEF2QI6UudUfxYVAYqt1iZwWkUDDRzQLwfGbQB6idyjfk55A/UKAURi0l0CkSi02kcaaQNjl7PoIv5pA","dtype":"float64","shape":[9]}}},"id":"552924f5-e214-4bbf-aeb6-bf3cd02145da","type":"ColumnDataSource"},{"attributes":{"plot":null,"text":"DataFrames: Reductions"},"id":"34326b5a-8a5b-4055-97e5-94ac3ada266b","type":"Title"},{"attributes":{"callback":null,"start":0},"id":"d4cd0f2c-4386-41df-80d9-5fda13c9c483","type":"DataRange1d"},{"attributes":{},"id":"2d56e258-7c42-4604-967f-0be7f343e299","type":"ToolEvents"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"81841aa7-676d-402a-8666-6664cf5519ae","type":"PanTool"},{"id":"06fbd4d5-fe13-42d0-9421-8d0877b78efa","type":"WheelZoomTool"},{"id":"77426e4c-cc31-4908-940c-e24d10e80abb","type":"BoxZoomTool"},{"id":"da292bcf-3c4c-4796-abd6-94dc90c9c8bb","type":"SaveTool"},{"id":"262cfe98-756b-4aed-b2e9-fcd9a7fcb524","type":"ResetTool"},{"id":"48e2c614-a86a-4802-b12a-53e7d558caa8","type":"HelpTool"}]},"id":"5334e99f-7403-4675-8861-86e795329fc0","type":"Toolbar"},{"attributes":{"num_minor_ticks":10},"id":"a9bfe4d8-3b3a-4162-aaf4-35869eba98a4","type":"LogTicker"},{"attributes":{"num_minor_ticks":10},"id":"7f323a76-701b-4ca0-93cc-d0792f265917","type":"LogTicker"},{"attributes":{"callback":null,"end":31167.568472113522,"start":0},"id":"6d8b57f3-de64-4dd0-9cf5-0f8ee6bef4e3","type":"DataRange1d"},{"attributes":{"ticker":null},"id":"83ba2660-26bc-4459-ac79-33ce297747bd","type":"LogTickFormatter"},{"attributes":{"plot":{"id":"39d25c5a-3587-4f53-81c7-7a8e11174dcc","subtype":"Figure","type":"Plot"},"ticker":{"id":"a9bfe4d8-3b3a-4162-aaf4-35869eba98a4","type":"LogTicker"}},"id":"9a8a88eb-dd52-483a-990f-7450b106004e","type":"Grid"},{"attributes":{},"id":"4637c9d2-5603-4d27-b014-17d8825b54b6","type":"LogScale"},{"attributes":{"axis_label":"cores","formatter":{"id":"83ba2660-26bc-4459-ac79-33ce297747bd","type":"LogTickFormatter"},"plot":{"id":"39d25c5a-3587-4f53-81c7-7a8e11174dcc","subtype":"Figure","type":"Plot"},"ticker":{"id":"51bde773-d544-40ca-95ac-f509ded2ce8f","type":"FixedTicker"}},"id":"1e849201-56ff-4961-9fad-ad3e390b0b84","type":"LogAxis"},{"attributes":{},"id":"357859f4-cd8a-4d9f-b1ce-545c236f3a28","type":"LogScale"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"f200cb3d-4d8b-4a61-8b15-ef98d3ee5c47","type":"LogTickFormatter"},"plot":{"id":"39d25c5a-3587-4f53-81c7-7a8e11174dcc","subtype":"Figure","type":"Plot"},"ticker":{"id":"7f323a76-701b-4ca0-93cc-d0792f265917","type":"LogTicker"}},"id":"5b086bdd-94d7-4ecc-a808-62305578339a","type":"LogAxis"},{"attributes":{"dimension":1,"plot":{"id":"39d25c5a-3587-4f53-81c7-7a8e11174dcc","subtype":"Figure","type":"Plot"},"ticker":{"id":"7f323a76-701b-4ca0-93cc-d0792f265917","type":"LogTicker"}},"id":"27788952-6cd6-474e-a0e6-72adb1dd2b27","type":"Grid"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"ab0cdf84-d38b-4e8f-961c-ae46e8d5792e","type":"Line"},{"attributes":{"data_source":{"id":"cde0055d-6f08-4d55-bc20-b657c5ceefab","type":"ColumnDataSource"},"glyph":{"id":"ab0cdf84-d38b-4e8f-961c-ae46e8d5792e","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"3636d692-9f9c-4c45-96c1-6e32c1ba4925","type":"Line"},"selection_glyph":null},"id":"000f6e25-f829-469e-b780-cb0242e3a7b2","type":"GlyphRenderer"},{"attributes":{"child":{"id":"39d25c5a-3587-4f53-81c7-7a8e11174dcc","subtype":"Figure","type":"Plot"},"title":"log"},"id":"d2cd1935-5500-4033-8dbb-e3d6802d8d0c","type":"Panel"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"ae9e26bd-f1c5-4428-9cf9-a219db087cd7","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"39d25c5a-3587-4f53-81c7-7a8e11174dcc","subtype":"Figure","type":"Plot"}},"id":"81841aa7-676d-402a-8666-6664cf5519ae","type":"PanTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"CPDOA7wEdkDPvmrzVKmGQPPjMBfneZZAL4RSELzboUAewsNdADiwQGQLuqvxDLlAiDO3aHUuwUBjrjECPpnGQMOz+hDDrsRA","dtype":"float64","shape":[9]}}},"id":"dbe7a236-6338-49bd-81dd-e948ede2c4e4","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"39d25c5a-3587-4f53-81c7-7a8e11174dcc","subtype":"Figure","type":"Plot"}},"id":"06fbd4d5-fe13-42d0-9421-8d0877b78efa","type":"WheelZoomTool"},{"attributes":{"overlay":{"id":"ae9e26bd-f1c5-4428-9cf9-a219db087cd7","type":"BoxAnnotation"},"plot":{"id":"39d25c5a-3587-4f53-81c7-7a8e11174dcc","subtype":"Figure","type":"Plot"}},"id":"77426e4c-cc31-4908-940c-e24d10e80abb","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"39d25c5a-3587-4f53-81c7-7a8e11174dcc","subtype":"Figure","type":"Plot"}},"id":"da292bcf-3c4c-4796-abd6-94dc90c9c8bb","type":"SaveTool"},{"attributes":{"plot":{"id":"39d25c5a-3587-4f53-81c7-7a8e11174dcc","subtype":"Figure","type":"Plot"}},"id":"262cfe98-756b-4aed-b2e9-fcd9a7fcb524","type":"ResetTool"},{"attributes":{"plot":{"id":"39d25c5a-3587-4f53-81c7-7a8e11174dcc","subtype":"Figure","type":"Plot"}},"id":"48e2c614-a86a-4802-b12a-53e7d558caa8","type":"HelpTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"CPDOA7wEdkDPvmrzVKmGQPPjMBfneZZAL4RSELzboUAewsNdADiwQGQLuqvxDLlAiDO3aHUuwUBjrjECPpnGQMOz+hDDrsRA","dtype":"float64","shape":[9]}}},"id":"cde0055d-6f08-4d55-bc20-b657c5ceefab","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,46595.0987549684]}},"id":"93d1534e-16d4-4453-84aa-356802b328a8","type":"ColumnDataSource"},{"attributes":{"ticker":null},"id":"f200cb3d-4d8b-4a61-8b15-ef98d3ee5c47","type":"LogTickFormatter"},{"attributes":{"data_source":{"id":"7b276bde-624a-404d-a0c9-3b5647fbe4c5","type":"ColumnDataSource"},"glyph":{"id":"cc5db39a-16cb-4e1b-86d6-8f12c85cc5af","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"1593145e-d449-43ce-8a39-249cfa0ecae8","type":"Line"},"selection_glyph":null},"id":"ef803fd0-35ee-436c-b303-af5e06365f2e","type":"GlyphRenderer"},{"attributes":{"items":[{"id":"674c14ce-8b11-47bc-8e6f-e722b96f9cc0","type":"LegendItem"},{"id":"2f3b9045-758c-457b-82e4-33b756b648ca","type":"LegendItem"},{"id":"0dee6c19-e80f-4b2a-8572-c5cd37b26727","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"39d25c5a-3587-4f53-81c7-7a8e11174dcc","subtype":"Figure","type":"Plot"}},"id":"e94cfdab-a4c2-48a2-8df7-736db028f287","type":"Legend"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"1593145e-d449-43ce-8a39-249cfa0ecae8","type":"Line"},{"attributes":{"label":{"value":"df.std()"},"renderers":[{"id":"000f6e25-f829-469e-b780-cb0242e3a7b2","type":"GlyphRenderer"}]},"id":"674c14ce-8b11-47bc-8e6f-e722b96f9cc0","type":"LegendItem"},{"attributes":{"data_source":{"id":"93d1534e-16d4-4453-84aa-356802b328a8","type":"ColumnDataSource"},"glyph":{"id":"2a30a863-9480-470a-870b-5008114c481b","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"cd58c1ae-ad48-4da0-9d98-64682209015a","type":"Line"},"selection_glyph":null},"id":"671e7a2f-b80f-4a4d-9aef-365c72389908","type":"GlyphRenderer"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"0e35e0e3-5612-4276-af4f-cb0f1f879608","type":"Circle"},{"attributes":{"data_source":{"id":"dbe7a236-6338-49bd-81dd-e948ede2c4e4","type":"ColumnDataSource"},"glyph":{"id":"0e35e0e3-5612-4276-af4f-cb0f1f879608","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"cb5efd96-7569-49c6-8114-bb34a68c2c37","type":"Circle"},"selection_glyph":null},"id":"9d9f4b53-3055-4c9d-be31-5b3f8e490207","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"cb5efd96-7569-49c6-8114-bb34a68c2c37","type":"Circle"},{"attributes":{"data_source":{"id":"7a8cb52b-1e55-4941-9272-38c7c42a078e","type":"ColumnDataSource"},"glyph":{"id":"89bcb074-f1d6-4c3a-b59e-9fe2c2eee9c3","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"fa5c78d3-8730-46d6-a131-640dea12d81a","type":"Line"},"selection_glyph":null},"id":"a668550a-b6bd-420e-94a0-5fcd299635fa","type":"GlyphRenderer"},{"attributes":{"below":[{"id":"d8b4966f-761f-4a13-b995-c378cf55e5f6","type":"LogAxis"}],"left":[{"id":"9063c47a-11f6-49e9-b286-6bc1acdaadc0","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"d8b4966f-761f-4a13-b995-c378cf55e5f6","type":"LogAxis"},{"id":"e9eccd09-1dff-4c05-b140-f16299e24d69","type":"Grid"},{"id":"9063c47a-11f6-49e9-b286-6bc1acdaadc0","type":"LogAxis"},{"id":"9f2da7a3-eee1-43b8-a823-2f05894d53d8","type":"Grid"},{"id":"f6baaf11-ce1c-486c-b8d5-0c1632ec6adb","type":"BoxAnnotation"},{"id":"bbc252c1-f586-46a1-b558-90d283b221dc","type":"Legend"},{"id":"5c85f6ef-b891-4ea1-9db3-e6ac8bdb4f0b","type":"GlyphRenderer"},{"id":"a8e83bb1-625e-4332-8c28-68b807e4ac87","type":"GlyphRenderer"},{"id":"ec695afc-00cb-4dd8-b886-1ec3953fb9f2","type":"GlyphRenderer"}],"title":{"id":"c733c949-e562-42bd-a803-a765568c71ba","type":"Title"},"tool_events":{"id":"957c312c-ee88-4507-8e73-02c85ab9c662","type":"ToolEvents"},"toolbar":{"id":"f9b3b0dc-97d0-4d34-aa6b-85ea48314f4b","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"f81c0f54-5258-4806-90a9-f4c31ed03309","type":"DataRange1d"},"x_scale":{"id":"2309e548-4929-4624-83fb-040c36c8aeef","type":"LogScale"},"y_range":{"id":"5878d1b4-cf07-46a6-b641-549870ec2d2e","type":"DataRange1d"},"y_scale":{"id":"006ec2d2-dc9c-403c-8195-2f79429db69e","type":"LogScale"}},"id":"d49fa646-bb9d-4704-9105-469647c7e6e2","subtype":"Figure","type":"Plot"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"bce31b7e-e4f5-4769-aefc-2486ec2cf37c","type":"Line"},{"attributes":{"label":{"value":"random"},"renderers":[{"id":"5c85f6ef-b891-4ea1-9db3-e6ac8bdb4f0b","type":"GlyphRenderer"}]},"id":"870dc000-14dd-48c8-8cdb-ab8ba9ff5849","type":"LegendItem"},{"attributes":{"data_source":{"id":"1e2b523e-fc0c-4a27-821f-49d321415b62","type":"ColumnDataSource"},"glyph":{"id":"7f499213-0134-4c5c-9235-126158ca7eed","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"bce31b7e-e4f5-4769-aefc-2486ec2cf37c","type":"Line"},"selection_glyph":null},"id":"9f288aaa-82ff-46f4-8bf4-3cf335875256","type":"GlyphRenderer"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"2b82eb62-e193-41f6-836a-f70e97ad6b5a","type":"PanTool"},{"id":"c13542c5-c8f2-4b1d-a5ce-55f0dde4c758","type":"WheelZoomTool"},{"id":"d7b8a053-c9f1-4fb7-bed4-a3df788f3ed2","type":"BoxZoomTool"},{"id":"13000b57-843e-4b11-98c2-6f02983aa18e","type":"SaveTool"},{"id":"e1966624-3bba-4c3a-86cf-1106a6800dc3","type":"ResetTool"},{"id":"08f094cf-dc4b-463e-9621-abdff60d0163","type":"HelpTool"}]},"id":"f9b3b0dc-97d0-4d34-aa6b-85ea48314f4b","type":"Toolbar"},{"attributes":{"plot":{"id":"bd7a4267-cf17-4206-a88f-6685cd6fdc61","subtype":"Figure","type":"Plot"},"ticker":{"id":"0519fcce-0843-4297-81d2-3938461dc021","type":"BasicTicker"}},"id":"eb20f2d2-177b-4983-8352-8bfbceddd49c","type":"Grid"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"89bcb074-f1d6-4c3a-b59e-9fe2c2eee9c3","type":"Line"},{"attributes":{"callback":null,"start":0},"id":"f81c0f54-5258-4806-90a9-f4c31ed03309","type":"DataRange1d"},{"attributes":{},"id":"1bb1ad9c-effe-4731-9798-9d4a4aa13fa5","type":"LinearScale"},{"attributes":{"child":{"id":"d49fa646-bb9d-4704-9105-469647c7e6e2","subtype":"Figure","type":"Plot"},"title":"log"},"id":"a6bff001-54c7-4f7e-bdea-e243ae2bff4c","type":"Panel"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"fa5c78d3-8730-46d6-a131-640dea12d81a","type":"Line"},{"attributes":{},"id":"957c312c-ee88-4507-8e73-02c85ab9c662","type":"ToolEvents"},{"attributes":{},"id":"1e10a22c-12aa-4a41-915c-f85a8ed32090","type":"LinearScale"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"a6bff001-54c7-4f7e-bdea-e243ae2bff4c","type":"Panel"},{"id":"60def97a-ac42-4a56-a9d1-0607993c2e62","type":"Panel"}]},"id":"bddfdd21-3692-4f09-a839-053ec2e8b96e","type":"Tabs"},{"attributes":{"callback":null,"end":6021.609676148496,"start":0},"id":"e4513607-4e88-4611-b52e-319f23abdd30","type":"DataRange1d"},{"attributes":{"items":[{"id":"870dc000-14dd-48c8-8cdb-ab8ba9ff5849","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"d49fa646-bb9d-4704-9105-469647c7e6e2","subtype":"Figure","type":"Plot"}},"id":"bbc252c1-f586-46a1-b558-90d283b221dc","type":"Legend"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,269795.901835003]}},"id":"7a8cb52b-1e55-4941-9272-38c7c42a078e","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"iDmAdOYzhED3jGZaeR+UQMzGF5CAyKNAqQZ49/jWsUDabo5ZahbBQPB5MuwHU9BAhZDfGJtW10Al2QgzKAzZQOndxaYoYuhA","dtype":"float64","shape":[9]}}},"id":"1b801c3f-37c6-4605-b059-12d57dbdb1c3","type":"ColumnDataSource"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"5e029128-a514-44ed-b3e0-5887022236ac","type":"Line"},{"attributes":{"axis_label":"cores","formatter":{"id":"11c0ae3b-3f61-48c8-8fd3-8f948edeb699","type":"LogTickFormatter"},"plot":{"id":"d49fa646-bb9d-4704-9105-469647c7e6e2","subtype":"Figure","type":"Plot"},"ticker":{"id":"55c37866-4dd7-43c4-a37f-ddd611adba37","type":"FixedTicker"}},"id":"d8b4966f-761f-4a13-b995-c378cf55e5f6","type":"LogAxis"},{"attributes":{"axis_label":"cores","formatter":{"id":"8859a918-4868-4c59-aace-0b98dacfb605","type":"BasicTickFormatter"},"plot":{"id":"bd7a4267-cf17-4206-a88f-6685cd6fdc61","subtype":"Figure","type":"Plot"},"ticker":{"id":"af5ddceb-be3b-4264-b6d6-570331edf696","type":"FixedTicker"}},"id":"5782e52f-d045-4066-8e67-4ff6b658a5b9","type":"LinearAxis"},{"attributes":{"callback":null,"end":49937.270358022135,"start":0},"id":"5878d1b4-cf07-46a6-b641-549870ec2d2e","type":"DataRange1d"},{"attributes":{},"id":"0519fcce-0843-4297-81d2-3938461dc021","type":"BasicTicker"},{"attributes":{"items":[{"id":"d94ad771-b4da-4cb2-8f9c-bac51b23c3c7","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"5abed06b-e288-46e8-9011-7e573975edd0","subtype":"Figure","type":"Plot"}},"id":"a21531e6-2df4-4a46-b1a3-fb0c270ed7fd","type":"Legend"},{"attributes":{"axis_label":"tasks/s","formatter":{"id":"079e65d5-53f5-4710-b4c5-17223bfa988e","type":"BasicTickFormatter"},"plot":{"id":"bd7a4267-cf17-4206-a88f-6685cd6fdc61","subtype":"Figure","type":"Plot"},"ticker":{"id":"1aded654-e20e-4b2f-b7c2-7486dc59dafd","type":"BasicTicker"}},"id":"b6d97613-9b92-4c51-93c9-494f3435f0be","type":"LinearAxis"},{"attributes":{"plot":null,"text":"DataFrames: Create"},"id":"c733c949-e562-42bd-a803-a765568c71ba","type":"Title"},{"attributes":{},"id":"1aded654-e20e-4b2f-b7c2-7486dc59dafd","type":"BasicTicker"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"e07a0b7a-e02f-4857-8704-45e71b5bc074","type":"Line"},{"attributes":{"dimension":1,"plot":{"id":"bd7a4267-cf17-4206-a88f-6685cd6fdc61","subtype":"Figure","type":"Plot"},"ticker":{"id":"1aded654-e20e-4b2f-b7c2-7486dc59dafd","type":"BasicTicker"}},"id":"46f80ab7-5004-455a-a548-55ec92fffb80","type":"Grid"},{"attributes":{"ticker":null},"id":"36fbc0eb-dd8a-4425-8fe3-39a618e7adaf","type":"LogTickFormatter"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"864c156a-c259-4dd5-b34f-4cebcafb8c38","type":"Line"},{"attributes":{"axis_label":"cores","formatter":{"id":"31ef9d8d-cddd-41f2-bddf-f04cfd9ccc47","type":"BasicTickFormatter"},"plot":{"id":"5ef06628-9afd-4f0f-a198-ba83837bf919","subtype":"Figure","type":"Plot"},"ticker":{"id":"73babae4-8e06-44f1-9d83-c6eb0618f4ae","type":"FixedTicker"}},"id":"1085eb81-6287-44f6-b8f0-7321717d7103","type":"LinearAxis"},{"attributes":{},"id":"8859a918-4868-4c59-aace-0b98dacfb605","type":"BasicTickFormatter"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"SA88gR1ocECqGEzZQ5R8QGxc6tAEo45AjGt/DoKRmEDGXTCinsSgQMrrGdYeO65A67hcfiEHuUA4aLWDa6XIQMM6i3W2GtBA","dtype":"float64","shape":[9]}}},"id":"264e02f4-93f6-4f11-91ee-752faff7d2b7","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"1867f265-98e5-4f56-bedc-c5196d3a4107","type":"ColumnDataSource"},"glyph":{"id":"5e029128-a514-44ed-b3e0-5887022236ac","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"864c156a-c259-4dd5-b34f-4cebcafb8c38","type":"Line"},"selection_glyph":null},"id":"4bf1be6f-0524-4691-a469-c271ed97e0b9","type":"GlyphRenderer"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"74e4b1eb-e4e1-491a-9967-4fdbded4795d","type":"Line"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"656cf7cc-3748-4c60-ad17-48948d4f1067","type":"BoxAnnotation"},{"attributes":{"num_minor_ticks":10},"id":"594148f5-678e-4aec-922f-b845f19d9257","type":"LogTicker"},{"attributes":{"plot":{"id":"bd7a4267-cf17-4206-a88f-6685cd6fdc61","subtype":"Figure","type":"Plot"}},"id":"ef963147-69dd-4316-824e-c39236f8374e","type":"PanTool"},{"attributes":{"plot":null,"text":"DataFrames: Time Series"},"id":"56f1830a-a13a-468a-a108-d6d2b2595e01","type":"Title"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"UGdETRRGSEAF/cD9b45XQEf6VqPzsGlAWonEk7n0ekB6ZUoa/4eOQF78OcpRNp1AikTHrPdfqkBkyoxeIYGzQPFuvBOchbdA","dtype":"float64","shape":[9]}}},"id":"41eeabab-8df7-4d43-ad9c-8df4f7dcc533","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"5ef06628-9afd-4f0f-a198-ba83837bf919","subtype":"Figure","type":"Plot"}},"id":"f4d6378e-99e2-41ca-9077-4e1fa1c98975","type":"HelpTool"},{"attributes":{"plot":{"id":"bd7a4267-cf17-4206-a88f-6685cd6fdc61","subtype":"Figure","type":"Plot"}},"id":"dd1840f2-3a3f-4ee8-96d7-274b0b58a153","type":"WheelZoomTool"},{"attributes":{"data_source":{"id":"264e02f4-93f6-4f11-91ee-752faff7d2b7","type":"ColumnDataSource"},"glyph":{"id":"cb0b67ef-d70b-4bff-80a5-abbdbb66fa92","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"52f412a0-ec50-4aa8-a4fd-37ece7bd1c2c","type":"Circle"},"selection_glyph":null},"id":"a1906a82-2729-4639-8244-d507183595be","type":"GlyphRenderer"},{"attributes":{"overlay":{"id":"656cf7cc-3748-4c60-ad17-48948d4f1067","type":"BoxAnnotation"},"plot":{"id":"bd7a4267-cf17-4206-a88f-6685cd6fdc61","subtype":"Figure","type":"Plot"}},"id":"cc2ba96b-474a-45b8-a0f7-c979a271d7f6","type":"BoxZoomTool"},{"attributes":{"child":{"id":"5ef06628-9afd-4f0f-a198-ba83837bf919","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"a41999c2-103e-46f4-b55e-3c9610dd09da","type":"Panel"},{"attributes":{"plot":{"id":"bd7a4267-cf17-4206-a88f-6685cd6fdc61","subtype":"Figure","type":"Plot"}},"id":"89c7e6f2-37c0-4e30-8273-cc364867a06e","type":"SaveTool"},{"attributes":{"dimension":1,"plot":{"id":"5ef06628-9afd-4f0f-a198-ba83837bf919","subtype":"Figure","type":"Plot"},"ticker":{"id":"38a20dfd-2b36-4a35-9899-ead569342ca0","type":"BasicTicker"}},"id":"b7a84748-2be1-4462-bad2-06f8041e2835","type":"Grid"},{"attributes":{"plot":{"id":"bd7a4267-cf17-4206-a88f-6685cd6fdc61","subtype":"Figure","type":"Plot"}},"id":"5f24c536-7893-4741-b8a6-f4e717ad6684","type":"ResetTool"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"542fccb4-83c9-48a8-b58c-5e1c23974b3e","type":"Line"},{"attributes":{"plot":{"id":"bd7a4267-cf17-4206-a88f-6685cd6fdc61","subtype":"Figure","type":"Plot"}},"id":"7ec0ca47-849b-4454-a2f0-7708648ed7be","type":"HelpTool"},{"attributes":{},"id":"c058f40c-6a7e-46d6-9050-e144151e8f23","type":"BasicTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"UGdETRRGSEAF/cD9b45XQEf6VqPzsGlAWonEk7n0ekB6ZUoa/4eOQF78OcpRNp1AikTHrPdfqkBkyoxeIYGzQPFuvBOchbdA","dtype":"float64","shape":[9]}}},"id":"1867f265-98e5-4f56-bedc-c5196d3a4107","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"d938cf74-add6-42d9-89fc-63789c1fb62b","type":"ColumnDataSource"},"glyph":{"id":"56b144fb-bcd8-4bbe-9987-2d7c7c23e596","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"542fccb4-83c9-48a8-b58c-5e1c23974b3e","type":"Line"},"selection_glyph":null},"id":"d2516bde-0777-46dc-9686-074f0d0dbcc2","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"bd6c9fed-c0f1-4374-be42-8e2c6dfad3be","type":"Circle"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"56b144fb-bcd8-4bbe-9987-2d7c7c23e596","type":"Line"},{"attributes":{},"id":"079e65d5-53f5-4710-b4c5-17223bfa988e","type":"BasicTickFormatter"},{"attributes":{"label":{"value":"df.rolling(...).mean()"},"renderers":[{"id":"4a30e69a-5fc9-49a9-b6d8-ba88f9921bb4","type":"GlyphRenderer"}]},"id":"7685730b-8ebb-488e-81d8-5139a089687a","type":"LegendItem"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"8128e3b4-a11a-4820-936b-e495d291a190","type":"Line"},{"attributes":{"data_source":{"id":"dacc35f2-c0bb-4520-9de0-84e27b734f0f","type":"ColumnDataSource"},"glyph":{"id":"74e4b1eb-e4e1-491a-9967-4fdbded4795d","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"4aa9bd5c-fd76-4924-9e89-faf68732453a","type":"Line"},"selection_glyph":null},"id":"a02779c4-dd2e-4ea8-84d2-22f9a7077d78","type":"GlyphRenderer"},{"attributes":{"items":[{"id":"eb28207e-8b56-4776-814f-6f5d3cf2df7c","type":"LegendItem"},{"id":"a1e70810-baa2-45a7-a129-f90dcec0b28c","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"bd7a4267-cf17-4206-a88f-6685cd6fdc61","subtype":"Figure","type":"Plot"}},"id":"e428b65c-bedd-4afe-b852-3e0e275615e3","type":"Legend"},{"attributes":{},"id":"38a20dfd-2b36-4a35-9899-ead569342ca0","type":"BasicTicker"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"0b677418-2978-4d3d-8c5f-de569cba996c","type":"Circle"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"SA88gR1ocECqGEzZQ5R8QGxc6tAEo45AjGt/DoKRmEDGXTCinsSgQMrrGdYeO65A67hcfiEHuUA4aLWDa6XIQMM6i3W2GtBA","dtype":"float64","shape":[9]}}},"id":"6bf174bd-5b20-431b-80a7-891992f72f3d","type":"ColumnDataSource"},{"attributes":{"label":{"value":"100ms"},"renderers":[{"id":"4bf1be6f-0524-4691-a469-c271ed97e0b9","type":"GlyphRenderer"}]},"id":"eb28207e-8b56-4776-814f-6f5d3cf2df7c","type":"LegendItem"},{"attributes":{"callback":null,"end":16490.85092430819,"start":0},"id":"59b4720e-8629-484b-8df1-f7a9c014d689","type":"DataRange1d"},{"attributes":{"data_source":{"id":"c1baa2d8-4f45-4192-a8a8-50e8b998b0e4","type":"ColumnDataSource"},"glyph":{"id":"18ddde02-fbb5-40ef-bed2-1096bcfbc37b","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"f09ae9dd-a80b-4e17-a95c-028c5bde6c81","type":"Line"},"selection_glyph":null},"id":"c100652d-4b2f-441d-a19b-1d5d96238306","type":"GlyphRenderer"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"80cd19c6-3bfd-4480-85a6-850a04dec7ab","type":"BasicTickFormatter"},"plot":{"id":"5ef06628-9afd-4f0f-a198-ba83837bf919","subtype":"Figure","type":"Plot"},"ticker":{"id":"38a20dfd-2b36-4a35-9899-ead569342ca0","type":"BasicTicker"}},"id":"ccdf9d3b-fa7d-4446-9621-e51f5683f88a","type":"LinearAxis"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"65b26a3f-5cd7-4899-9f01-e91ff91e36fe","type":"Circle"},{"attributes":{},"id":"93c84cb3-e191-43b3-a8f9-ca0ad1d2b0c2","type":"LinearScale"},{"attributes":{"data_source":{"id":"41eeabab-8df7-4d43-ad9c-8df4f7dcc533","type":"ColumnDataSource"},"glyph":{"id":"65b26a3f-5cd7-4899-9f01-e91ff91e36fe","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"d1eaea75-1a55-4319-bc70-b94cff1e37e4","type":"Circle"},"selection_glyph":null},"id":"fad64154-6abb-410f-a32f-9691b20da3f9","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"5ef06628-9afd-4f0f-a198-ba83837bf919","subtype":"Figure","type":"Plot"},"ticker":{"id":"c058f40c-6a7e-46d6-9050-e144151e8f23","type":"BasicTicker"}},"id":"c25d5f02-8e42-42c4-8580-4ec796e24b4b","type":"Grid"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"d1eaea75-1a55-4319-bc70-b94cff1e37e4","type":"Circle"},{"attributes":{"plot":{"id":"e761952d-458d-4815-b822-4649d8dc2644","subtype":"Figure","type":"Plot"},"ticker":{"id":"43f23bff-cadb-4d03-bbc9-af672ade5478","type":"LogTicker"}},"id":"fa229f3e-b9eb-4a4d-b7aa-9a158b6e3663","type":"Grid"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"2459dc7b-dc2a-422d-930c-6d5a873a89ba","type":"Circle"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"cb0b67ef-d70b-4bff-80a5-abbdbb66fa92","type":"Circle"},{"attributes":{"data_source":{"id":"188daee3-63a9-4d9c-af21-e5233113b8f3","type":"ColumnDataSource"},"glyph":{"id":"8128e3b4-a11a-4820-936b-e495d291a190","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"9b667962-f88d-4d40-83a9-a392a6bfdf49","type":"Line"},"selection_glyph":null},"id":"610e6598-84c3-4377-bce3-4258e91e8129","type":"GlyphRenderer"},{"attributes":{},"id":"c85cad71-9c17-451c-945a-363c1d6a860e","type":"LogScale"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"18ddde02-fbb5-40ef-bed2-1096bcfbc37b","type":"Line"},{"attributes":{"below":[{"id":"1085eb81-6287-44f6-b8f0-7321717d7103","type":"LinearAxis"}],"left":[{"id":"ccdf9d3b-fa7d-4446-9621-e51f5683f88a","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"1085eb81-6287-44f6-b8f0-7321717d7103","type":"LinearAxis"},{"id":"c25d5f02-8e42-42c4-8580-4ec796e24b4b","type":"Grid"},{"id":"ccdf9d3b-fa7d-4446-9621-e51f5683f88a","type":"LinearAxis"},{"id":"b7a84748-2be1-4462-bad2-06f8041e2835","type":"Grid"},{"id":"3fce8c9b-c4c4-48ec-8b2b-6f449be0a6bf","type":"BoxAnnotation"},{"id":"f39c1e83-6b71-463b-805a-cc114061c218","type":"Legend"},{"id":"d2516bde-0777-46dc-9686-074f0d0dbcc2","type":"GlyphRenderer"},{"id":"8ce713be-1279-437b-9080-e9b23af20017","type":"GlyphRenderer"},{"id":"57089827-d1b4-45cd-9546-ec865f69abc4","type":"GlyphRenderer"}],"title":{"id":"56f1830a-a13a-468a-a108-d6d2b2595e01","type":"Title"},"tool_events":{"id":"3a7714bc-f25c-4d6b-811c-9ba122d76979","type":"ToolEvents"},"toolbar":{"id":"bf8efa5f-cbc2-4485-9c3f-b7d8a0ca4b8c","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"b5f33caf-5956-43d0-bc8c-1a9ea8fdf0b3","type":"DataRange1d"},"x_scale":{"id":"bf8e56ab-72a0-4514-a426-e3a8ae1f25e8","type":"LinearScale"},"y_range":{"id":"59b4720e-8629-484b-8df1-f7a9c014d689","type":"DataRange1d"},"y_scale":{"id":"93c84cb3-e191-43b3-a8f9-ca0ad1d2b0c2","type":"LinearScale"}},"id":"5ef06628-9afd-4f0f-a198-ba83837bf919","subtype":"Figure","type":"Plot"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"f09ae9dd-a80b-4e17-a95c-028c5bde6c81","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"4aa9bd5c-fd76-4924-9e89-faf68732453a","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"kaJ6m493kEDnCG6oG3iZQDH65WfdO51AN9IVWq5NnEATlZAk2WCfQArn1nJmT6JAEu2KAqxsokCrK/Acec2hQLjq14Ix6JZA","dtype":"float64","shape":[9]}}},"id":"7be7c900-1988-407b-904f-421d124c4462","type":"ColumnDataSource"},{"attributes":{"children":[{"id":"5c8cf25d-0f4c-46d7-a10b-af71e65fd1f4","type":"WidgetBox"},{"id":"53cea20e-2f14-4b85-8239-4e58ae53738b","type":"WidgetBox"}],"sizing_mode":"scale_width"},"id":"832b3d76-7254-415f-99bd-28ab20af0ab2","type":"Row"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"460ef359-dfb1-403c-837f-2f484f0985b2","type":"FixedTicker"},{"attributes":{"child":{"id":"d83086bf-c820-4079-b0fb-49267a76c0bb","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"41db98d3-1eb6-401a-a3e8-97326b42d74d","type":"Panel"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"kaJ6m493kEDnCG6oG3iZQDH65WfdO51AN9IVWq5NnEATlZAk2WCfQArn1nJmT6JAEu2KAqxsokCrK/Acec2hQLjq14Ix6JZA","dtype":"float64","shape":[9]}}},"id":"c1baa2d8-4f45-4192-a8a8-50e8b998b0e4","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,67201.84405141801]}},"id":"dacc35f2-c0bb-4520-9de0-84e27b734f0f","type":"ColumnDataSource"},{"attributes":{"label":{"value":"1us"},"renderers":[{"id":"c100652d-4b2f-441d-a19b-1d5d96238306","type":"GlyphRenderer"}]},"id":"a1e70810-baa2-45a7-a129-f90dcec0b28c","type":"LegendItem"},{"attributes":{},"id":"3a7714bc-f25c-4d6b-811c-9ba122d76979","type":"ToolEvents"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,12428.158608007856]}},"id":"188daee3-63a9-4d9c-af21-e5233113b8f3","type":"ColumnDataSource"},{"attributes":{},"id":"31ef9d8d-cddd-41f2-bddf-f04cfd9ccc47","type":"BasicTickFormatter"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"1cde6262-d06e-4f36-97cc-40d1a028fec4","type":"Circle"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"04a7a51b-fbbb-425a-8cfe-6dd02962e744","type":"PanTool"},{"id":"54250a06-b278-4bf6-b85a-f689f5021979","type":"WheelZoomTool"},{"id":"1a20d206-c661-4312-9339-cee7e4878f9b","type":"BoxZoomTool"},{"id":"29feaaed-05a7-4359-9eed-5e7cb8487502","type":"SaveTool"},{"id":"93e59c29-b745-43c4-a2ec-157e1523e2dc","type":"ResetTool"},{"id":"f4d6378e-99e2-41ca-9077-4e1fa1c98975","type":"HelpTool"}]},"id":"bf8efa5f-cbc2-4485-9c3f-b7d8a0ca4b8c","type":"Toolbar"},{"attributes":{"data_source":{"id":"25ad1df3-af70-4961-90ad-b55b1a1b4ca6","type":"ColumnDataSource"},"glyph":{"id":"1cde6262-d06e-4f36-97cc-40d1a028fec4","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"0b677418-2978-4d3d-8c5f-de569cba996c","type":"Circle"},"selection_glyph":null},"id":"7968e288-abf3-4d02-8ac6-054b992cfb1c","type":"GlyphRenderer"},{"attributes":{},"id":"bf8e56ab-72a0-4514-a426-e3a8ae1f25e8","type":"LinearScale"},{"attributes":{"data_source":{"id":"7be7c900-1988-407b-904f-421d124c4462","type":"ColumnDataSource"},"glyph":{"id":"2459dc7b-dc2a-422d-930c-6d5a873a89ba","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"bd6c9fed-c0f1-4374-be42-8e2c6dfad3be","type":"Circle"},"selection_glyph":null},"id":"6216cd53-1ee7-4d64-bed7-99aa32b36d65","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"52f412a0-ec50-4aa8-a4fd-37ece7bd1c2c","type":"Circle"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"9b667962-f88d-4d40-83a9-a392a6bfdf49","type":"Line"},{"attributes":{"plot":{"id":"5ef06628-9afd-4f0f-a198-ba83837bf919","subtype":"Figure","type":"Plot"}},"id":"29feaaed-05a7-4359-9eed-5e7cb8487502","type":"SaveTool"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"c542b8bb-269b-41dd-9e33-ba4f3c2babae","type":"Line"},{"attributes":{"plot":{"id":"5ef06628-9afd-4f0f-a198-ba83837bf919","subtype":"Figure","type":"Plot"}},"id":"93e59c29-b745-43c4-a2ec-157e1523e2dc","type":"ResetTool"},{"attributes":{"data_source":{"id":"8a768c37-9fbd-4188-add4-f4d940f2080a","type":"ColumnDataSource"},"glyph":{"id":"c542b8bb-269b-41dd-9e33-ba4f3c2babae","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"75410fff-4401-4f8a-b39f-b6116d253c21","type":"Line"},"selection_glyph":null},"id":"eae59b8c-4aae-42b2-af00-907169e4c2d0","type":"GlyphRenderer"},{"attributes":{"dimension":1,"plot":{"id":"e761952d-458d-4815-b822-4649d8dc2644","subtype":"Figure","type":"Plot"},"ticker":{"id":"594148f5-678e-4aec-922f-b845f19d9257","type":"LogTicker"}},"id":"3f12a60a-b6bf-495c-8aab-879125782ebb","type":"Grid"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"75410fff-4401-4f8a-b39f-b6116d253c21","type":"Line"},{"attributes":{"overlay":{"id":"3fce8c9b-c4c4-48ec-8b2b-6f449be0a6bf","type":"BoxAnnotation"},"plot":{"id":"5ef06628-9afd-4f0f-a198-ba83837bf919","subtype":"Figure","type":"Plot"}},"id":"1a20d206-c661-4312-9339-cee7e4878f9b","type":"BoxZoomTool"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"3e2d4b46-09ac-405a-9276-b907e8df69bb","type":"Panel"},{"id":"b23801a4-be5a-492c-ab93-ece3370c2047","type":"Panel"}]},"id":"b175b9d7-0992-4013-a3da-e7da9e7115e2","type":"Tabs"},{"attributes":{"items":[{"id":"f049aeab-54bf-42b7-82be-ac76f7c60d9e","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"5ef06628-9afd-4f0f-a198-ba83837bf919","subtype":"Figure","type":"Plot"}},"id":"f39c1e83-6b71-463b-805a-cc114061c218","type":"Legend"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"af5ddceb-be3b-4264-b6d6-570331edf696","type":"FixedTicker"},{"attributes":{"plot":{"id":"5ef06628-9afd-4f0f-a198-ba83837bf919","subtype":"Figure","type":"Plot"}},"id":"54250a06-b278-4bf6-b85a-f689f5021979","type":"WheelZoomTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,269795.901835003]}},"id":"8a768c37-9fbd-4188-add4-f4d940f2080a","type":"ColumnDataSource"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"e5236ef3-0528-4dea-8c9b-304d9481bb44","type":"Line"},{"attributes":{"child":{"id":"b1f9c2ed-12a3-425c-a852-37fafe3ee9c8","subtype":"Figure","type":"Plot"},"title":"log"},"id":"3e2d4b46-09ac-405a-9276-b907e8df69bb","type":"Panel"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,67201.84405141801]}},"id":"9825c8e5-f8fc-4439-bb82-8b65ae769606","type":"ColumnDataSource"},{"attributes":{"below":[{"id":"bed1583f-c1c4-4b10-b258-4317f7933bee","type":"LogAxis"}],"left":[{"id":"7e0cec7c-15ef-4709-acbb-933eb9238e70","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"bed1583f-c1c4-4b10-b258-4317f7933bee","type":"LogAxis"},{"id":"22a51e72-b540-4b2c-8c88-9ade9d03d005","type":"Grid"},{"id":"7e0cec7c-15ef-4709-acbb-933eb9238e70","type":"LogAxis"},{"id":"389f6e50-4229-4968-b92d-2704548be1fd","type":"Grid"},{"id":"3dc67210-eb0f-4c45-8bf3-9ac6b35a51c3","type":"BoxAnnotation"},{"id":"88daa403-982e-4d7e-a63e-e5f4401d6e08","type":"Legend"},{"id":"b11cf862-e14d-410d-a091-23cafd5f1930","type":"GlyphRenderer"},{"id":"7968e288-abf3-4d02-8ac6-054b992cfb1c","type":"GlyphRenderer"},{"id":"69bd697e-0832-457a-8127-aaea5640b266","type":"GlyphRenderer"}],"title":{"id":"e505d900-f832-47d6-938f-205f40ea5122","type":"Title"},"tool_events":{"id":"99a796cb-62aa-439c-9884-e0ebc38f5e70","type":"ToolEvents"},"toolbar":{"id":"7eba6133-b827-4508-9767-d20d180afb83","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"395f10f4-541f-48cb-9c4c-812c50bc3614","type":"DataRange1d"},"x_scale":{"id":"69573f14-e33e-461f-8ec4-bce6061bf3f6","type":"LogScale"},"y_range":{"id":"47dd3cd0-96e7-4c0a-ae88-992bcd68a53a","type":"DataRange1d"},"y_scale":{"id":"9504112c-f960-4fb5-bd9c-0723c7c7fa72","type":"LogScale"}},"id":"b1f9c2ed-12a3-425c-a852-37fafe3ee9c8","subtype":"Figure","type":"Plot"},{"attributes":{"data_source":{"id":"9825c8e5-f8fc-4439-bb82-8b65ae769606","type":"ColumnDataSource"},"glyph":{"id":"ead5269f-b61f-4110-aa9d-1f8609ccb12b","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"e5236ef3-0528-4dea-8c9b-304d9481bb44","type":"Line"},"selection_glyph":null},"id":"57089827-d1b4-45cd-9546-ec865f69abc4","type":"GlyphRenderer"},{"attributes":{"label":{"value":"fast"},"renderers":[{"id":"b11cf862-e14d-410d-a091-23cafd5f1930","type":"GlyphRenderer"}]},"id":"8750c30b-8f3b-4482-917f-12aade87bf36","type":"LegendItem"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"75a7f12d-788a-43ab-abc3-aeb0b17ab640","type":"Circle"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"sAHL3O3PZEALehwq1T1lQCHeSBOVVGVARLJVKdkUZUBZ9cB/YLBjQCWoVrQJiGBA4ishF0sKXUCMae/xfoRaQC1Vmopl8FJA","dtype":"float64","shape":[9]}}},"id":"25ad1df3-af70-4961-90ad-b55b1a1b4ca6","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"6bf174bd-5b20-431b-80a7-891992f72f3d","type":"ColumnDataSource"},"glyph":{"id":"a5244703-1e09-4eac-a525-6ef65a22a342","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"75a7f12d-788a-43ab-abc3-aeb0b17ab640","type":"Circle"},"selection_glyph":null},"id":"8ce713be-1279-437b-9080-e9b23af20017","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"sAHL3O3PZEALehwq1T1lQCHeSBOVVGVARLJVKdkUZUBZ9cB/YLBjQCWoVrQJiGBA4ishF0sKXUCMae/xfoRaQC1Vmopl8FJA","dtype":"float64","shape":[9]}}},"id":"85c2b4e9-0196-44c3-8ef9-407e77eae0fe","type":"ColumnDataSource"},{"attributes":{},"id":"80cd19c6-3bfd-4480-85a6-850a04dec7ab","type":"BasicTickFormatter"},{"attributes":{"plot":null,"text":"Tasks: Sequential"},"id":"e505d900-f832-47d6-938f-205f40ea5122","type":"Title"},{"attributes":{"plot":{"id":"5ef06628-9afd-4f0f-a198-ba83837bf919","subtype":"Figure","type":"Plot"}},"id":"04a7a51b-fbbb-425a-8cfe-6dd02962e744","type":"PanTool"},{"attributes":{"callback":null,"start":0},"id":"395f10f4-541f-48cb-9c4c-812c50bc3614","type":"DataRange1d"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"a5244703-1e09-4eac-a525-6ef65a22a342","type":"Circle"},{"attributes":{},"id":"99a796cb-62aa-439c-9884-e0ebc38f5e70","type":"ToolEvents"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"3fce8c9b-c4c4-48ec-8b2b-6f449be0a6bf","type":"BoxAnnotation"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"5e757e75-742d-4e2b-a209-c24a8e6f6f09","type":"PanTool"},{"id":"9d0d89f3-9fda-4c43-a80f-f4e61bcf7ae2","type":"WheelZoomTool"},{"id":"96a77bc3-3076-4302-83ae-ca23b110ad65","type":"BoxZoomTool"},{"id":"52d138cc-73e1-4c54-9808-4c7479025432","type":"SaveTool"},{"id":"542aceac-c73e-4206-9e7d-a2e326d4112f","type":"ResetTool"},{"id":"aecac00b-e41f-415c-86ad-223f8986f0f5","type":"HelpTool"}]},"id":"7eba6133-b827-4508-9767-d20d180afb83","type":"Toolbar"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"ead5269f-b61f-4110-aa9d-1f8609ccb12b","type":"Line"},{"attributes":{"axis_label":"cores","formatter":{"id":"31b24d74-6828-4e72-a0b1-b9ff3ea2812d","type":"LogTickFormatter"},"plot":{"id":"b1f9c2ed-12a3-425c-a852-37fafe3ee9c8","subtype":"Figure","type":"Plot"},"ticker":{"id":"497eb2a8-9f4f-4a33-8db7-0bf2b0729624","type":"FixedTicker"}},"id":"bed1583f-c1c4-4b10-b258-4317f7933bee","type":"LogAxis"},{"attributes":{"label":{"value":"df.rolling(...).mean()"},"renderers":[{"id":"d2516bde-0777-46dc-9686-074f0d0dbcc2","type":"GlyphRenderer"}]},"id":"f049aeab-54bf-42b7-82be-ac76f7c60d9e","type":"LegendItem"},{"attributes":{"num_minor_ticks":10},"id":"1f17c79c-958a-4b8d-b13a-5a1b3fe37259","type":"LogTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"SA88gR1ocECqGEzZQ5R8QGxc6tAEo45AjGt/DoKRmEDGXTCinsSgQMrrGdYeO65A67hcfiEHuUA4aLWDa6XIQMM6i3W2GtBA","dtype":"float64","shape":[9]}}},"id":"d938cf74-add6-42d9-89fc-63789c1fb62b","type":"ColumnDataSource"},{"attributes":{"callback":null,"end":170.64319767219516,"start":0},"id":"47dd3cd0-96e7-4c0a-ae88-992bcd68a53a","type":"DataRange1d"},{"attributes":{"children":[{"id":"51125763-50b1-42e3-a52f-a429803a4aa3","type":"WidgetBox"},{"id":"48036047-1f68-42ed-9c8e-881e34e44386","type":"WidgetBox"}],"sizing_mode":"scale_width"},"id":"dd1c4ad1-e50c-417c-a615-b3d7acdf96c8","type":"Row"},{"attributes":{"items":[{"id":"7685730b-8ebb-488e-81d8-5139a089687a","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"e761952d-458d-4815-b822-4649d8dc2644","subtype":"Figure","type":"Plot"}},"id":"7e75b446-a5a1-42a6-a69a-c20836f1fc09","type":"Legend"},{"attributes":{"axis_label":"tasks/s","formatter":{"id":"50eadc2e-195e-4477-a4bd-aa0597ac1245","type":"LogTickFormatter"},"plot":{"id":"b1f9c2ed-12a3-425c-a852-37fafe3ee9c8","subtype":"Figure","type":"Plot"},"ticker":{"id":"3e1785c2-5406-42b7-b954-df12211fa100","type":"LogTicker"}},"id":"7e0cec7c-15ef-4709-acbb-933eb9238e70","type":"LogAxis"},{"attributes":{"callback":null,"start":0},"id":"b5f33caf-5956-43d0-bc8c-1a9ea8fdf0b3","type":"DataRange1d"},{"attributes":{},"id":"69573f14-e33e-461f-8ec4-bce6061bf3f6","type":"LogScale"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"8aa7f9a3-e0e5-419b-a8a4-5ad8e36fe88f","type":"BoxAnnotation"},{"attributes":{"overlay":{"id":"8aa7f9a3-e0e5-419b-a8a4-5ad8e36fe88f","type":"BoxAnnotation"},"plot":{"id":"e761952d-458d-4815-b822-4649d8dc2644","subtype":"Figure","type":"Plot"}},"id":"e97e20d3-03e2-428d-aa0f-d00fd9bde72e","type":"BoxZoomTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"LbJ3pRAWX0BbUHF3/95iQIiYAtArN3BAAzvcglmDd0A7RNmNS3J/QPRV3VjCgItAGs1hBVAyk0A6pgUWMmqYQM/xIEy8hpZA","dtype":"float64","shape":[9]}}},"id":"eb1ddd02-cb71-479e-bc49-7b1b8c0b5761","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"eb1ddd02-cb71-479e-bc49-7b1b8c0b5761","type":"ColumnDataSource"},"glyph":{"id":"aae9a1bc-6c14-43a2-bb8a-4ee12d60e597","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"2148100e-a957-4c41-9858-ac0916291cc9","type":"Circle"},"selection_glyph":null},"id":"399be48b-b05d-48ac-ad82-8bb5171ea982","type":"GlyphRenderer"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"aae9a1bc-6c14-43a2-bb8a-4ee12d60e597","type":"Circle"},{"attributes":{"label":{"value":"df.set_index(...)"},"renderers":[{"id":"a209df9a-abf2-4be6-83bc-504ea82e6cdc","type":"GlyphRenderer"}]},"id":"4f9b7b11-905b-4212-8a29-e4342a009bef","type":"LegendItem"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"2148100e-a957-4c41-9858-ac0916291cc9","type":"Circle"},{"attributes":{"plot":{"id":"d49fa646-bb9d-4704-9105-469647c7e6e2","subtype":"Figure","type":"Plot"}},"id":"08f094cf-dc4b-463e-9621-abdff60d0163","type":"HelpTool"},{"attributes":{"plot":{"id":"d49fa646-bb9d-4704-9105-469647c7e6e2","subtype":"Figure","type":"Plot"}},"id":"13000b57-843e-4b11-98c2-6f02983aa18e","type":"SaveTool"},{"attributes":{"data_source":{"id":"9dce6d6a-a1f8-4fe1-a323-dee8ffeb69ab","type":"ColumnDataSource"},"glyph":{"id":"f9899b33-cee8-4e92-8bde-98976f515e97","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"ceafc98c-ebe1-4bd1-be3c-67d14086a66f","type":"Circle"},"selection_glyph":null},"id":"48a027b0-fa86-43fd-86fc-e465a24274e1","type":"GlyphRenderer"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"f9899b33-cee8-4e92-8bde-98976f515e97","type":"Circle"},{"attributes":{"data_source":{"id":"8b8c8a62-e1b0-4930-847e-e46e8c9c1a64","type":"ColumnDataSource"},"glyph":{"id":"ba01f278-86ed-4144-8e23-212b673f5235","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"ffb32e57-b8a9-45fd-b011-ef75303ec8ad","type":"Circle"},"selection_glyph":null},"id":"53e417c9-c14d-4f62-b71e-b64f00b18b47","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"ceafc98c-ebe1-4bd1-be3c-67d14086a66f","type":"Circle"},{"attributes":{},"id":"cc86455c-5d1e-4abc-86d1-b890845eeceb","type":"BasicTickFormatter"},{"attributes":{"data_source":{"id":"c756c34c-f5c3-4a6b-ba9a-e7bc550d24a1","type":"ColumnDataSource"},"glyph":{"id":"5ef2b5ec-e447-4e9e-817e-4be835aa6437","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"12b11cde-0643-4d2b-972c-a05b6da33c2e","type":"Line"},"selection_glyph":null},"id":"c97583c4-51e9-4cff-9769-b4b540deaac8","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"8y0AKWPAZkCdUmbwFEF2QI6UudUfxYVAYqt1iZwWkUDDRzQLwfGbQB6idyjfk55A/UKAURi0l0CkSi02kcaaQNjl7PoIv5pA","dtype":"float64","shape":[9]}}},"id":"ac3232fd-36ce-4a71-b1ec-d2c11df0dab8","type":"ColumnDataSource"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"0cc6521b-a666-4a7b-a5d3-503c5254fe2c","type":"BoxAnnotation"},{"attributes":{"data_source":{"id":"ac3232fd-36ce-4a71-b1ec-d2c11df0dab8","type":"ColumnDataSource"},"glyph":{"id":"96ada5fa-cc77-4370-8f95-57ba750e765d","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"2d2f0701-ffed-4624-9292-adcf6eb13b76","type":"Line"},"selection_glyph":null},"id":"a48aa2d3-5a7c-4537-b210-38f11a1832f4","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"32468298-7c8b-4961-a1e3-2247d13004a8","subtype":"Figure","type":"Plot"}},"id":"d7e00d52-b6a0-43d3-b2ce-aba245b42337","type":"PanTool"},{"attributes":{"line_color":{"value":"#2ca02c"},"x":{"field":"x"},"y":{"field":"y"}},"id":"851688a4-7809-4398-a203-451c229ef21e","type":"Line"},{"attributes":{"ticker":null},"id":"ebe40921-ac2f-4cb2-a5bb-e6bd77e82fa3","type":"LogTickFormatter"},{"attributes":{"items":[{"id":"2fd34b86-5d5a-4e52-92bd-7a34844e7868","type":"LegendItem"},{"id":"5ab2bac9-cdf6-4edd-a327-9ec68c7b545f","type":"LegendItem"},{"id":"4ba13fdc-0b78-45bc-ba23-18a594c16715","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"dbde8c5e-a16f-4c03-9c99-8f0f50a1bbd4","subtype":"Figure","type":"Plot"}},"id":"ce4c2571-807d-49e2-86ee-7b9ef2271d9c","type":"Legend"},{"attributes":{"plot":{"id":"32468298-7c8b-4961-a1e3-2247d13004a8","subtype":"Figure","type":"Plot"}},"id":"b288bf15-ef2e-4945-9eb4-ccaf8cd6c334","type":"WheelZoomTool"},{"attributes":{"data_source":{"id":"ec1816cc-3f9f-4512-8e57-4a83cf83db87","type":"ColumnDataSource"},"glyph":{"id":"b88406b9-d608-401e-afa2-3ee7f2a586df","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"a2581d1d-0772-415b-9db6-5e8ea1cd3e40","type":"Circle"},"selection_glyph":null},"id":"297420ff-5aff-4487-b0f0-78eba07a1421","type":"GlyphRenderer"},{"attributes":{"overlay":{"id":"0cc6521b-a666-4a7b-a5d3-503c5254fe2c","type":"BoxAnnotation"},"plot":{"id":"32468298-7c8b-4961-a1e3-2247d13004a8","subtype":"Figure","type":"Plot"}},"id":"6007b073-348b-41bc-a701-31198057ecfe","type":"BoxZoomTool"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"7df733ee-85ac-443a-a6f5-d4cf488ea15c","type":"Circle"},{"attributes":{"plot":{"id":"32468298-7c8b-4961-a1e3-2247d13004a8","subtype":"Figure","type":"Plot"}},"id":"2a2d4a72-6109-438e-aa25-4a5692f74faf","type":"SaveTool"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"009c1ead-6a21-4251-baaa-32ce7645bc5a","type":"Panel"},{"id":"41db98d3-1eb6-401a-a3e8-97326b42d74d","type":"Panel"}]},"id":"bc2a925a-2e83-48cf-a2c4-b3b85a7804f9","type":"Tabs"},{"attributes":{"plot":{"id":"32468298-7c8b-4961-a1e3-2247d13004a8","subtype":"Figure","type":"Plot"}},"id":"d377ce77-c83a-4022-83a8-021b1f1aa6b0","type":"ResetTool"},{"attributes":{"axis_label":"cores","formatter":{"id":"e5dad457-ce21-46f9-a4bf-b881885257b3","type":"LogTickFormatter"},"plot":{"id":"2e73804a-3798-4c2c-8afb-a230f37ff380","subtype":"Figure","type":"Plot"},"ticker":{"id":"07998121-7f06-437a-b1cb-c2c6736f6931","type":"FixedTicker"}},"id":"4d4fb43e-1d9c-4092-b88e-0d051451169a","type":"LogAxis"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"11d6ffa4-ab28-4bc2-b71f-bdb2f16a1e20","type":"Line"},{"attributes":{"plot":{"id":"32468298-7c8b-4961-a1e3-2247d13004a8","subtype":"Figure","type":"Plot"}},"id":"16a5b5ca-891a-4048-9e4c-749d27fa4e99","type":"HelpTool"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"028955e6-2170-4e3c-b1c5-91d2e6c7cea2","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"12b11cde-0643-4d2b-972c-a05b6da33c2e","type":"Line"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"617cd638-9d5f-4765-89eb-7472275a2a0e","type":"Line"},{"attributes":{"items":[{"id":"466c273c-d0ae-4b38-b913-71c48c943226","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"e7f5d259-31f1-49a6-bf6b-7e56a0ac86b4","subtype":"Figure","type":"Plot"}},"id":"7cc450a8-4515-40be-bfe1-4ef1aebb9bb2","type":"Legend"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"426bd2ab-0882-4386-b2c7-83a24a666182","type":"FixedTicker"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"5ef2b5ec-e447-4e9e-817e-4be835aa6437","type":"Line"},{"attributes":{"below":[{"id":"4d4fb43e-1d9c-4092-b88e-0d051451169a","type":"LogAxis"}],"left":[{"id":"8bf2acd7-c0e3-4b9f-9ec7-6bbfbf02bc94","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"4d4fb43e-1d9c-4092-b88e-0d051451169a","type":"LogAxis"},{"id":"e6575946-5eda-4943-89de-ed775cb0e659","type":"Grid"},{"id":"8bf2acd7-c0e3-4b9f-9ec7-6bbfbf02bc94","type":"LogAxis"},{"id":"c27ae204-4506-4af7-a62e-eb46a04d85b5","type":"Grid"},{"id":"efc5e5bd-a54b-4f82-91ac-81f1452b4983","type":"BoxAnnotation"},{"id":"65412682-021f-4cbc-8a36-490fb1d821b7","type":"Legend"},{"id":"ac6c1b73-a582-457c-9f3d-112602cdb911","type":"GlyphRenderer"},{"id":"cf7e8366-7581-4e12-93ef-b6b229126bc9","type":"GlyphRenderer"},{"id":"fb4f1318-40bf-4b51-8d8c-b441c1eed3c0","type":"GlyphRenderer"},{"id":"f3e364a8-c1f0-40ba-aed0-daf9ceae878e","type":"GlyphRenderer"},{"id":"d3f0205b-45dc-444f-9485-3a821bd6f1e5","type":"GlyphRenderer"},{"id":"b614a223-bbd2-40b1-9145-a3094667aad7","type":"GlyphRenderer"}],"title":{"id":"113a0520-9b86-41e7-bf29-b849f4de495e","type":"Title"},"tool_events":{"id":"4004fca6-fd9e-45e1-943d-648c8e1c4be6","type":"ToolEvents"},"toolbar":{"id":"71b65dcc-e835-421d-9a7b-a6e75eaa29aa","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"494e2192-f731-4da3-a551-f5a32aaa3e68","type":"DataRange1d"},"x_scale":{"id":"81e0367e-101e-4005-93ba-f836e0b92727","type":"LogScale"},"y_range":{"id":"757ebc94-e691-40a6-90a0-32ac7eca4bea","type":"DataRange1d"},"y_scale":{"id":"cbb9bc6c-1f65-4274-bf49-6c454c02a800","type":"LogScale"}},"id":"2e73804a-3798-4c2c-8afb-a230f37ff380","subtype":"Figure","type":"Plot"},{"attributes":{"plot":null,"text":"Arrays: Random Access"},"id":"eb1ce7e7-012f-4e9b-b610-4abaf35beff5","type":"Title"},{"attributes":{"plot":{"id":"2e73804a-3798-4c2c-8afb-a230f37ff380","subtype":"Figure","type":"Plot"},"ticker":{"id":"1670de7c-cc5d-48c3-9569-992918297fa5","type":"LogTicker"}},"id":"e6575946-5eda-4943-89de-ed775cb0e659","type":"Grid"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"481c2585-6dc3-4cc1-bd72-f61b21d604e8","type":"FixedTicker"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"0a12f286-72b9-40f7-9747-cce4fca208b7","type":"PanTool"},{"id":"6c2a2f99-b315-4295-938e-4182eefbf3a8","type":"WheelZoomTool"},{"id":"2d6b0dc4-acfe-49bb-9dda-551a36dcf187","type":"BoxZoomTool"},{"id":"aaeac710-c162-4542-bc3a-0a5dabae8022","type":"SaveTool"},{"id":"541700d3-9318-4921-895f-bc6a03b02ecb","type":"ResetTool"},{"id":"21b20b53-1ceb-423e-a2ec-b801c3464c1e","type":"HelpTool"}]},"id":"71b65dcc-e835-421d-9a7b-a6e75eaa29aa","type":"Toolbar"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,178949.5144368985]}},"id":"c756c34c-f5c3-4a6b-ba9a-e7bc550d24a1","type":"ColumnDataSource"},{"attributes":{"callback":null,"start":0},"id":"494e2192-f731-4da3-a551-f5a32aaa3e68","type":"DataRange1d"},{"attributes":{"below":[{"id":"dc3bbb53-429d-4260-9aa6-45e98a2be775","type":"LinearAxis"}],"left":[{"id":"e9bd950f-62b2-4765-a98e-db6b1c8eeeec","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"dc3bbb53-429d-4260-9aa6-45e98a2be775","type":"LinearAxis"},{"id":"57036595-e514-4b1b-8b10-d384620b1d78","type":"Grid"},{"id":"e9bd950f-62b2-4765-a98e-db6b1c8eeeec","type":"LinearAxis"},{"id":"877a93e9-1cca-4b09-9d05-278c94dfd138","type":"Grid"},{"id":"3d408590-153d-4681-85ae-4708f22cd483","type":"BoxAnnotation"},{"id":"7cc450a8-4515-40be-bfe1-4ef1aebb9bb2","type":"Legend"},{"id":"dee22ce0-66ba-4649-9c74-f6b14deb8d62","type":"GlyphRenderer"},{"id":"1dfa1dab-2828-4f9f-bce9-5673cb6dd956","type":"GlyphRenderer"},{"id":"bb8ed46b-3283-4bcb-8123-2e7d95d77ab9","type":"GlyphRenderer"}],"title":{"id":"eb1ce7e7-012f-4e9b-b610-4abaf35beff5","type":"Title"},"tool_events":{"id":"9fd483d9-9788-43e2-8845-79eb10d0ff13","type":"ToolEvents"},"toolbar":{"id":"5669ca56-cc7e-40f3-9b2f-54d16ea24417","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"3135a211-0b5d-4d2e-bb6f-a1c96094b605","type":"DataRange1d"},"x_scale":{"id":"74a9a570-f7ef-495f-97a1-9203d57007e1","type":"LinearScale"},"y_range":{"id":"42206c4b-3441-4ae3-b10d-fba4fc6c6863","type":"DataRange1d"},"y_scale":{"id":"4de585f3-cef4-4f3e-a987-72340dbd60a6","type":"LinearScale"}},"id":"e7f5d259-31f1-49a6-bf6b-7e56a0ac86b4","subtype":"Figure","type":"Plot"},{"attributes":{"num_minor_ticks":10},"id":"1670de7c-cc5d-48c3-9569-992918297fa5","type":"LogTicker"},{"attributes":{},"id":"9fd483d9-9788-43e2-8845-79eb10d0ff13","type":"ToolEvents"},{"attributes":{"callback":null,"end":1846.5368297047141,"start":0},"id":"757ebc94-e691-40a6-90a0-32ac7eca4bea","type":"DataRange1d"},{"attributes":{},"id":"d517dc70-e6bf-4630-b959-be0ec2898fbc","type":"BasicTickFormatter"},{"attributes":{},"id":"4004fca6-fd9e-45e1-943d-648c8e1c4be6","type":"ToolEvents"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"6c0f4cd7-3b7f-450a-8ec5-5ad54dee9602","type":"PanTool"},{"id":"268bb071-8229-41d7-81ce-b1a2b2b810d9","type":"WheelZoomTool"},{"id":"8af95738-37f1-4d91-bff1-a0ad5a3357a7","type":"BoxZoomTool"},{"id":"45ddb6b8-b536-4147-a7e6-f7302b12e46a","type":"SaveTool"},{"id":"19085b66-4288-4213-ba84-7d22eb1a99ac","type":"ResetTool"},{"id":"8c843b78-9689-42c8-a19b-383f83ed3cae","type":"HelpTool"}]},"id":"5669ca56-cc7e-40f3-9b2f-54d16ea24417","type":"Toolbar"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"dc048a49-a86c-421d-85b3-866e9aaaa8a9","type":"Line"},{"attributes":{"callback":null,"start":0},"id":"3135a211-0b5d-4d2e-bb6f-a1c96094b605","type":"DataRange1d"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,231102.94226687078]}},"id":"fc5c219c-e1a7-41be-b63c-2f2ba81ef1b9","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"5649e48b-92ca-4927-a7df-20ef2fd756e0","type":"ColumnDataSource"},"glyph":{"id":"a2174dc6-19a1-44b0-811a-4d939b79bdec","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"a9237ee5-e9fc-42ae-9f49-5545ef0a7673","type":"Line"},"selection_glyph":null},"id":"dee22ce0-66ba-4649-9c74-f6b14deb8d62","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"a2581d1d-0772-415b-9db6-5e8ea1cd3e40","type":"Circle"},{"attributes":{},"id":"74a9a570-f7ef-495f-97a1-9203d57007e1","type":"LinearScale"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"afbb3500-3074-4c50-9738-84e98d948aa5","type":"Line"},{"attributes":{"callback":null,"end":860.5686440460619,"start":0},"id":"42206c4b-3441-4ae3-b10d-fba4fc6c6863","type":"DataRange1d"},{"attributes":{"data_source":{"id":"15c59cd6-40c0-409b-b871-6db811ffa5b0","type":"ColumnDataSource"},"glyph":{"id":"4ee4b42b-979d-4f58-9941-87e79c4e041c","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"91d84762-ad3e-4142-b0e5-a8a0e376824b","type":"Circle"},"selection_glyph":null},"id":"d385e74e-efcb-4bd7-b948-857f407fe29f","type":"GlyphRenderer"},{"attributes":{},"id":"4de585f3-cef4-4f3e-a987-72340dbd60a6","type":"LinearScale"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"62bf1e99-6372-4c2a-9a19-f71bae085fa1","type":"Line"},{"attributes":{"plot":{"id":"e7f5d259-31f1-49a6-bf6b-7e56a0ac86b4","subtype":"Figure","type":"Plot"},"ticker":{"id":"cb44216f-bb64-4ff1-aa03-65192d7c0b92","type":"BasicTicker"}},"id":"57036595-e514-4b1b-8b10-d384620b1d78","type":"Grid"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"LbJ3pRAWX0BbUHF3/95iQIiYAtArN3BAAzvcglmDd0A7RNmNS3J/QPRV3VjCgItAGs1hBVAyk0A6pgUWMmqYQM/xIEy8hpZA","dtype":"float64","shape":[9]}}},"id":"6f91da52-632c-4841-85ce-3bd6defdeda0","type":"ColumnDataSource"},{"attributes":{"axis_label":"cores","formatter":{"id":"21d94c0f-121b-4b1a-9153-81a71b1a31ab","type":"BasicTickFormatter"},"plot":{"id":"e7f5d259-31f1-49a6-bf6b-7e56a0ac86b4","subtype":"Figure","type":"Plot"},"ticker":{"id":"42f1c34b-c7c5-4121-b751-aa05a3239087","type":"FixedTicker"}},"id":"dc3bbb53-429d-4260-9aa6-45e98a2be775","type":"LinearAxis"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,90187.75092977297]}},"id":"d7e0d6b8-fb4a-4e54-b015-7407a31794f1","type":"ColumnDataSource"},{"attributes":{},"id":"cb44216f-bb64-4ff1-aa03-65192d7c0b92","type":"BasicTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"kTbDifc1jEA5B/1PBoKaQOuHU50ntahAZ9QJI5VLtkAx9mM3CDnEQLuRZacBKdBAqmH6cOlv00AR3Nhh5G/eQEoAPNDmhd1A","dtype":"float64","shape":[9]}}},"id":"ec1816cc-3f9f-4512-8e57-4a83cf83db87","type":"ColumnDataSource"},{"attributes":{"axis_label":"bytes/s","formatter":{"id":"d517dc70-e6bf-4630-b959-be0ec2898fbc","type":"BasicTickFormatter"},"plot":{"id":"e7f5d259-31f1-49a6-bf6b-7e56a0ac86b4","subtype":"Figure","type":"Plot"},"ticker":{"id":"c1a8f09c-a6ff-405b-be06-782c4558eba6","type":"BasicTicker"}},"id":"e9bd950f-62b2-4765-a98e-db6b1c8eeeec","type":"LinearAxis"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"kTbDifc1jEA5B/1PBoKaQOuHU50ntahAZ9QJI5VLtkAx9mM3CDnEQLuRZacBKdBAqmH6cOlv00AR3Nhh5G/eQEoAPNDmhd1A","dtype":"float64","shape":[9]}}},"id":"ee803efa-eb3b-4bc8-a874-e4f2424a1e2b","type":"ColumnDataSource"},{"attributes":{},"id":"c1a8f09c-a6ff-405b-be06-782c4558eba6","type":"BasicTicker"},{"attributes":{"child":{"id":"2e73804a-3798-4c2c-8afb-a230f37ff380","subtype":"Figure","type":"Plot"},"title":"log"},"id":"009c1ead-6a21-4251-baaa-32ce7645bc5a","type":"Panel"},{"attributes":{"dimension":1,"plot":{"id":"e7f5d259-31f1-49a6-bf6b-7e56a0ac86b4","subtype":"Figure","type":"Plot"},"ticker":{"id":"c1a8f09c-a6ff-405b-be06-782c4558eba6","type":"BasicTicker"}},"id":"877a93e9-1cca-4b09-9d05-278c94dfd138","type":"Grid"},{"attributes":{"label":{"value":"df.groupby(0)[1].mean()"},"renderers":[{"id":"212c18d2-a6f0-4ad8-9c48-a97150375720","type":"GlyphRenderer"}]},"id":"4ba13fdc-0b78-45bc-ba23-18a594c16715","type":"LegendItem"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"a9237ee5-e9fc-42ae-9f49-5545ef0a7673","type":"Line"},{"attributes":{"data_source":{"id":"d7e0d6b8-fb4a-4e54-b015-7407a31794f1","type":"ColumnDataSource"},"glyph":{"id":"617cd638-9d5f-4765-89eb-7472275a2a0e","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"028955e6-2170-4e3c-b1c5-91d2e6c7cea2","type":"Line"},"selection_glyph":null},"id":"cc17ce54-c27b-4773-ace9-fd26405a4b4d","type":"GlyphRenderer"},{"attributes":{"ticker":null},"id":"d8d73696-8fda-409c-8671-bfbd8fe8cbbb","type":"LogTickFormatter"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"b88406b9-d608-401e-afa2-3ee7f2a586df","type":"Circle"},{"attributes":{"child":{"id":"e7f5d259-31f1-49a6-bf6b-7e56a0ac86b4","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"51a14984-ff50-4ffd-9f85-404e8b4bfba6","type":"Panel"},{"attributes":{"dimension":1,"plot":{"id":"dbde8c5e-a16f-4c03-9c99-8f0f50a1bbd4","subtype":"Figure","type":"Plot"},"ticker":{"id":"fbdb2ec2-a20a-4ec8-9f05-3eafb3435ded","type":"BasicTicker"}},"id":"af6933eb-7eb8-4827-8ca1-0222eae27f31","type":"Grid"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"3d408590-153d-4681-85ae-4708f22cd483","type":"BoxAnnotation"},{"attributes":{"data_source":{"id":"fc5c219c-e1a7-41be-b63c-2f2ba81ef1b9","type":"ColumnDataSource"},"glyph":{"id":"0d2c79e4-0457-4d22-9d58-eeb7ea9740bc","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"dc048a49-a86c-421d-85b3-866e9aaaa8a9","type":"Line"},"selection_glyph":null},"id":"4f423387-fdc4-4b44-b254-02e16e1ed7d5","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"e7f5d259-31f1-49a6-bf6b-7e56a0ac86b4","subtype":"Figure","type":"Plot"}},"id":"6c0f4cd7-3b7f-450a-8ec5-5ad54dee9602","type":"PanTool"},{"attributes":{"plot":{"id":"dbde8c5e-a16f-4c03-9c99-8f0f50a1bbd4","subtype":"Figure","type":"Plot"},"ticker":{"id":"2271dbe4-edd0-41c8-b581-78f6706f05cc","type":"BasicTicker"}},"id":"7d32d4ca-4bcb-4b99-80f8-4754c9adb392","type":"Grid"},{"attributes":{},"id":"21d94c0f-121b-4b1a-9153-81a71b1a31ab","type":"BasicTickFormatter"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"0d2c79e4-0457-4d22-9d58-eeb7ea9740bc","type":"Line"},{"attributes":{"plot":{"id":"e7f5d259-31f1-49a6-bf6b-7e56a0ac86b4","subtype":"Figure","type":"Plot"}},"id":"268bb071-8229-41d7-81ce-b1a2b2b810d9","type":"WheelZoomTool"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"ffb32e57-b8a9-45fd-b011-ef75303ec8ad","type":"Circle"},{"attributes":{"overlay":{"id":"3d408590-153d-4681-85ae-4708f22cd483","type":"BoxAnnotation"},"plot":{"id":"e7f5d259-31f1-49a6-bf6b-7e56a0ac86b4","subtype":"Figure","type":"Plot"}},"id":"8af95738-37f1-4d91-bff1-a0ad5a3357a7","type":"BoxZoomTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,46595.0987549684]}},"id":"a3b7e089-5ad8-4ebf-9a64-c1d09598606a","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"e7f5d259-31f1-49a6-bf6b-7e56a0ac86b4","subtype":"Figure","type":"Plot"}},"id":"45ddb6b8-b536-4147-a7e6-f7302b12e46a","type":"SaveTool"},{"attributes":{"items":[{"id":"b8084d43-5b05-43f2-845a-d9bcf1c4ada1","type":"LegendItem"},{"id":"4f9b7b11-905b-4212-8a29-e4342a009bef","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"d83086bf-c820-4079-b0fb-49267a76c0bb","subtype":"Figure","type":"Plot"}},"id":"115c68a3-af39-4a59-8199-23ad4a0f78ba","type":"Legend"},{"attributes":{"plot":{"id":"e7f5d259-31f1-49a6-bf6b-7e56a0ac86b4","subtype":"Figure","type":"Plot"}},"id":"19085b66-4288-4213-ba84-7d22eb1a99ac","type":"ResetTool"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"15228f53-57af-4a54-b48e-e18360e7c5bd","type":"Circle"},{"attributes":{"plot":{"id":"e7f5d259-31f1-49a6-bf6b-7e56a0ac86b4","subtype":"Figure","type":"Plot"}},"id":"8c843b78-9689-42c8-a19b-383f83ed3cae","type":"HelpTool"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"d81ade6e-a134-4569-88f9-6710ed145dad","type":"Line"},{"attributes":{"data_source":{"id":"ea9596b4-c596-4b0d-b364-3758052a9220","type":"ColumnDataSource"},"glyph":{"id":"1bbb3080-7d77-4404-8457-8378ba7d0a21","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"c72b98cc-c4dc-4b7b-ac2a-bf554d0aefd9","type":"Line"},"selection_glyph":null},"id":"bb8ed46b-3283-4bcb-8123-2e7d95d77ab9","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"a3b7e089-5ad8-4ebf-9a64-c1d09598606a","type":"ColumnDataSource"},"glyph":{"id":"d81ade6e-a134-4569-88f9-6710ed145dad","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"afbb3500-3074-4c50-9738-84e98d948aa5","type":"Line"},"selection_glyph":null},"id":"0e99105b-1eec-4047-8453-f246267675b9","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"txeRHSzYhUCjQkyym/ODQDXnP5WM5IpAL2chJm3jhEB0ZkdqW+WBQI2SGi6KzoBAUViGGKc/hEDneAuUg4eCQAnnNrBvOH1A","dtype":"float64","shape":[9]}}},"id":"3e90ab5c-cb11-44b4-ae01-440fab2130a0","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"ee803efa-eb3b-4bc8-a874-e4f2424a1e2b","type":"ColumnDataSource"},"glyph":{"id":"851688a4-7809-4398-a203-451c229ef21e","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"62bf1e99-6372-4c2a-9a19-f71bae085fa1","type":"Line"},"selection_glyph":null},"id":"212c18d2-a6f0-4ad8-9c48-a97150375720","type":"GlyphRenderer"},{"attributes":{"label":{"value":"x[12345, 23456]"},"renderers":[{"id":"dee22ce0-66ba-4649-9c74-f6b14deb8d62","type":"GlyphRenderer"}]},"id":"466c273c-d0ae-4b38-b913-71c48c943226","type":"LegendItem"},{"attributes":{},"id":"fbdb2ec2-a20a-4ec8-9f05-3eafb3435ded","type":"BasicTicker"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"bc1d6ca4-2165-4fa9-8827-0e0ae50663c7","type":"Circle"},{"attributes":{"label":{"value":"df[0].std()"},"renderers":[{"id":"a48aa2d3-5a7c-4537-b210-38f11a1832f4","type":"GlyphRenderer"}]},"id":"5ab2bac9-cdf6-4edd-a327-9ec68c7b545f","type":"LegendItem"},{"attributes":{"data_source":{"id":"3e90ab5c-cb11-44b4-ae01-440fab2130a0","type":"ColumnDataSource"},"glyph":{"id":"bc1d6ca4-2165-4fa9-8827-0e0ae50663c7","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"8e298fbf-32ee-4dd1-abff-c26acdedacc5","type":"Circle"},"selection_glyph":null},"id":"1dfa1dab-2828-4f9f-bce9-5673cb6dd956","type":"GlyphRenderer"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"ba01f278-86ed-4144-8e23-212b673f5235","type":"Circle"},{"attributes":{},"id":"13eb8f2b-7872-4adc-a796-69ac87b9efe0","type":"LogScale"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"8e298fbf-32ee-4dd1-abff-c26acdedacc5","type":"Circle"},{"attributes":{"child":{"id":"354d44e6-7d5b-46b5-add8-5e0812ed0ef4","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"6fc4847b-fea2-49c6-944c-3c087f03b08a","type":"Panel"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"8y0AKWPAZkCdUmbwFEF2QI6UudUfxYVAYqt1iZwWkUDDRzQLwfGbQB6idyjfk55A/UKAURi0l0CkSi02kcaaQNjl7PoIv5pA","dtype":"float64","shape":[9]}}},"id":"8b8c8a62-e1b0-4930-847e-e46e8c9c1a64","type":"ColumnDataSource"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"744c6c49-c239-4b07-9a35-b96a1c0c4fd3","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"2d2f0701-ffed-4624-9292-adcf6eb13b76","type":"Line"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"1bbb3080-7d77-4404-8457-8378ba7d0a21","type":"Line"},{"attributes":{"label":{"value":"df.set_index(...)"},"renderers":[{"id":"fb4f1318-40bf-4b51-8d8c-b441c1eed3c0","type":"GlyphRenderer"}]},"id":"b0557011-7268-4e67-bce4-b6b09d62330d","type":"LegendItem"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"c72b98cc-c4dc-4b7b-ac2a-bf554d0aefd9","type":"Line"},{"attributes":{},"id":"81e0367e-101e-4005-93ba-f836e0b92727","type":"LogScale"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"5mBoSrF9l0Bna2xc6/uUQD2eEyk3/Z1A41+WwWA8rkCg3qY5Aga3QAVA74snN8NAncj380uyxEDOuHXJpNfQQN8hVioyK9NA","dtype":"float64","shape":[9]}}},"id":"1b752d80-be80-4b1d-a56a-695e21cd9d48","type":"ColumnDataSource"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"96ada5fa-cc77-4370-8f95-57ba750e765d","type":"Line"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"42f1c34b-c7c5-4121-b751-aa05a3239087","type":"FixedTicker"},{"attributes":{"plot":{"id":"dbde8c5e-a16f-4c03-9c99-8f0f50a1bbd4","subtype":"Figure","type":"Plot"}},"id":"ff50f331-ff6c-4fc5-9f0d-272907ba7131","type":"ResetTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,178949.5144368985]}},"id":"ea9596b4-c596-4b0d-b364-3758052a9220","type":"ColumnDataSource"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"b5b3cacd-17da-4b7a-8255-21f22b67b039","type":"LogTickFormatter"},"plot":{"id":"2e73804a-3798-4c2c-8afb-a230f37ff380","subtype":"Figure","type":"Plot"},"ticker":{"id":"bf227976-4636-4ae8-b651-79ca2bb13121","type":"LogTicker"}},"id":"8bf2acd7-c0e3-4b9f-9ec7-6bbfbf02bc94","type":"LogAxis"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"70ff432f-e2c0-4f3d-8740-3a64d1b9851b","type":"BoxAnnotation"},{"attributes":{"label":{"value":"df.std()"},"renderers":[{"id":"3543525d-36fc-463f-b01a-5ace3e3a5e70","type":"GlyphRenderer"}]},"id":"2fd34b86-5d5a-4e52-92bd-7a34844e7868","type":"LegendItem"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"5f82e741-c2af-4053-a06d-82cf9834966a","type":"Panel"},{"id":"6fc4847b-fea2-49c6-944c-3c087f03b08a","type":"Panel"}]},"id":"d5eef40f-39b6-49cb-9230-302039c21841","type":"Tabs"},{"attributes":{"plot":null,"text":"DataFrames: Full Shuffle"},"id":"113a0520-9b86-41e7-bf29-b849f4de495e","type":"Title"},{"attributes":{"child":{"id":"bf138360-c752-43f5-8fb6-8e184f623a29","subtype":"Figure","type":"Plot"},"title":"log"},"id":"5f82e741-c2af-4053-a06d-82cf9834966a","type":"Panel"},{"attributes":{"plot":{"id":"dbde8c5e-a16f-4c03-9c99-8f0f50a1bbd4","subtype":"Figure","type":"Plot"}},"id":"0a9d28ae-0c46-4fa8-93b9-7586884f78b2","type":"HelpTool"},{"attributes":{"below":[{"id":"6b275538-7dee-47c4-ae09-9c61a84016b8","type":"LogAxis"}],"left":[{"id":"f2d4171a-9750-4a70-9801-267c206f0394","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"6b275538-7dee-47c4-ae09-9c61a84016b8","type":"LogAxis"},{"id":"faf2ba21-d806-4401-b08c-f410365985c0","type":"Grid"},{"id":"f2d4171a-9750-4a70-9801-267c206f0394","type":"LogAxis"},{"id":"8960a848-f444-45e4-b72b-b24fd0383d3e","type":"Grid"},{"id":"70ff432f-e2c0-4f3d-8740-3a64d1b9851b","type":"BoxAnnotation"},{"id":"9d47119f-4b80-4765-8852-50252e2f4aca","type":"Legend"},{"id":"d5fcae1d-47da-48ad-b0be-3810b2422c05","type":"GlyphRenderer"},{"id":"52752298-39d9-4847-892a-1eba3a73b48b","type":"GlyphRenderer"},{"id":"0845f557-d6e7-420f-9522-e6c54163a1cf","type":"GlyphRenderer"}],"title":{"id":"83c4bfa4-84cf-402d-bd2b-bd1eebde8506","type":"Title"},"tool_events":{"id":"7d1619c2-3b48-40cc-91a8-fc88f0c9c24f","type":"ToolEvents"},"toolbar":{"id":"c88d472f-bcc7-4929-a31d-b99ef69ef771","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"1110dafa-5b80-4374-9250-da23305c877f","type":"DataRange1d"},"x_scale":{"id":"40d865ab-754f-4202-8819-1d8775b4cfcb","type":"LogScale"},"y_range":{"id":"487dbaf5-62a1-4041-8d2c-2624bdbd0abc","type":"DataRange1d"},"y_scale":{"id":"ac0e2ba7-9ee1-4bea-91e9-941fa39bd039","type":"LogScale"}},"id":"bf138360-c752-43f5-8fb6-8e184f623a29","subtype":"Figure","type":"Plot"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"CPDOA7wEdkDPvmrzVKmGQPPjMBfneZZAL4RSELzboUAewsNdADiwQGQLuqvxDLlAiDO3aHUuwUBjrjECPpnGQMOz+hDDrsRA","dtype":"float64","shape":[9]}}},"id":"e453ea94-6c1c-456c-ab14-dbe4a81678b2","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"5mBoSrF9l0Bna2xc6/uUQD2eEyk3/Z1A41+WwWA8rkCg3qY5Aga3QAVA74snN8NAncj380uyxEDOuHXJpNfQQN8hVioyK9NA","dtype":"float64","shape":[9]}}},"id":"b005e721-b1fb-450b-a107-a2ca30c9db54","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"dbde8c5e-a16f-4c03-9c99-8f0f50a1bbd4","subtype":"Figure","type":"Plot"}},"id":"9d53358f-a4b9-4b60-ac35-5f0e5f26d9fe","type":"PanTool"},{"attributes":{"data_source":{"id":"b005e721-b1fb-450b-a107-a2ca30c9db54","type":"ColumnDataSource"},"glyph":{"id":"d072ced0-ff65-4d88-b5e2-57c72529e674","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"744c6c49-c239-4b07-9a35-b96a1c0c4fd3","type":"Line"},"selection_glyph":null},"id":"d5fcae1d-47da-48ad-b0be-3810b2422c05","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"91d84762-ad3e-4142-b0e5-a8a0e376824b","type":"Circle"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"d072ced0-ff65-4d88-b5e2-57c72529e674","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"f73db528-2c8c-4e66-af7b-7091dc77227d","type":"Line"},{"attributes":{"plot":null,"text":"Arrays: Bulk Communication"},"id":"83c4bfa4-84cf-402d-bd2b-bd1eebde8506","type":"Title"},{"attributes":{"overlay":{"id":"31e613bb-109b-4dd8-a7b8-dedb6ac90927","type":"BoxAnnotation"},"plot":{"id":"dbde8c5e-a16f-4c03-9c99-8f0f50a1bbd4","subtype":"Figure","type":"Plot"}},"id":"09ada8fa-169d-49df-abc7-0703b7616c81","type":"BoxZoomTool"},{"attributes":{"callback":null,"start":0},"id":"1110dafa-5b80-4374-9250-da23305c877f","type":"DataRange1d"},{"attributes":{"plot":{"id":"dbde8c5e-a16f-4c03-9c99-8f0f50a1bbd4","subtype":"Figure","type":"Plot"}},"id":"168f662e-50de-4018-b25f-82741503df6e","type":"SaveTool"},{"attributes":{},"id":"7d1619c2-3b48-40cc-91a8-fc88f0c9c24f","type":"ToolEvents"},{"attributes":{"plot":{"id":"dbde8c5e-a16f-4c03-9c99-8f0f50a1bbd4","subtype":"Figure","type":"Plot"}},"id":"71b8f90f-69b5-4a11-b22e-9c8037bd5bef","type":"WheelZoomTool"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"8361b22b-3ff7-4956-88ad-047a13352874","type":"PanTool"},{"id":"775682b6-14e9-4776-8728-d6245765d7d7","type":"WheelZoomTool"},{"id":"ed4b4a35-3f55-4984-b0c8-a056878f6a04","type":"BoxZoomTool"},{"id":"470cd535-e7e1-4478-b54d-b49a97059204","type":"SaveTool"},{"id":"3ce9e121-6fd7-401f-a547-da593e4dc77b","type":"ResetTool"},{"id":"16d88d36-65a2-4f3e-b237-11a79023b49a","type":"HelpTool"}]},"id":"c88d472f-bcc7-4929-a31d-b99ef69ef771","type":"Toolbar"},{"attributes":{"data_source":{"id":"e453ea94-6c1c-456c-ab14-dbe4a81678b2","type":"ColumnDataSource"},"glyph":{"id":"d2fb3748-d2e6-4025-8afe-e77a383f3c50","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"f73db528-2c8c-4e66-af7b-7091dc77227d","type":"Line"},"selection_glyph":null},"id":"3543525d-36fc-463f-b01a-5ace3e3a5e70","type":"GlyphRenderer"},{"attributes":{},"id":"40d865ab-754f-4202-8819-1d8775b4cfcb","type":"LogScale"},{"attributes":{},"id":"f7bbd14e-534b-45b2-aa72-80a33d3d048e","type":"BasicTickFormatter"},{"attributes":{"axis_label":"cores","formatter":{"id":"ebe40921-ac2f-4cb2-a5bb-e6bd77e82fa3","type":"LogTickFormatter"},"plot":{"id":"bf138360-c752-43f5-8fb6-8e184f623a29","subtype":"Figure","type":"Plot"},"ticker":{"id":"baf0ecde-67cc-4c97-afc6-2208995fddb4","type":"FixedTicker"}},"id":"6b275538-7dee-47c4-ae09-9c61a84016b8","type":"LogAxis"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"d2fb3748-d2e6-4025-8afe-e77a383f3c50","type":"Line"},{"attributes":{"callback":null,"end":19628.783834012105,"start":0},"id":"487dbaf5-62a1-4041-8d2c-2624bdbd0abc","type":"DataRange1d"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"31e613bb-109b-4dd8-a7b8-dedb6ac90927","type":"BoxAnnotation"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"d8d73696-8fda-409c-8671-bfbd8fe8cbbb","type":"LogTickFormatter"},"plot":{"id":"bf138360-c752-43f5-8fb6-8e184f623a29","subtype":"Figure","type":"Plot"},"ticker":{"id":"417d361c-4a40-4733-9159-5f1811af26b7","type":"LogTicker"}},"id":"f2d4171a-9750-4a70-9801-267c206f0394","type":"LogAxis"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"cc86455c-5d1e-4abc-86d1-b890845eeceb","type":"BasicTickFormatter"},"plot":{"id":"dbde8c5e-a16f-4c03-9c99-8f0f50a1bbd4","subtype":"Figure","type":"Plot"},"ticker":{"id":"fbdb2ec2-a20a-4ec8-9f05-3eafb3435ded","type":"BasicTicker"}},"id":"7f7381b1-b3c1-442b-9895-0807a0393276","type":"LinearAxis"},{"attributes":{"items":[{"id":"6eb5feba-11a0-471c-b3e9-b6c749e1f6de","type":"LegendItem"},{"id":"ae1c749a-1e1d-4508-bc62-96b7e7b8b472","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"d596527a-b70a-42bb-897a-44a65ed6ae5d","subtype":"Figure","type":"Plot"}},"id":"8b367327-372a-448e-99a6-fd80f7367292","type":"Legend"},{"attributes":{"label":{"value":"100ms"},"renderers":[{"id":"3e3a4880-9622-4757-8216-1579e675e963","type":"GlyphRenderer"}]},"id":"3dc9f913-a083-4fee-9bdb-73613184eac0","type":"LegendItem"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"deecb97b-ec64-461a-bed8-a7fec272a973","type":"FixedTicker"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"9720958c-b388-4629-b5bf-1f10a75bb572","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"YiGd3kr1M0C4DswHLGBDQAJlPBqPJVNAF1KOo6LJYkAH1cQ/wblxQALKM0XtrH9ADDdmn9fbi0DY4pMc/3KVQBZyRig+NJpA","dtype":"float64","shape":[9]}}},"id":"24450ec2-23b0-43b9-a45e-645b0d14b4e0","type":"ColumnDataSource"},{"attributes":{"below":[{"id":"da0293e6-8787-4a4b-acc4-54f1b66b0e15","type":"LogAxis"}],"left":[{"id":"99c0ff9d-5883-4247-8181-85ebbc60dbdb","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"da0293e6-8787-4a4b-acc4-54f1b66b0e15","type":"LogAxis"},{"id":"92f0df92-fb07-4101-b4c1-bd10f1501839","type":"Grid"},{"id":"99c0ff9d-5883-4247-8181-85ebbc60dbdb","type":"LogAxis"},{"id":"983d1f9d-e177-4bcc-a1a8-d9e40e414ce6","type":"Grid"},{"id":"48fde0e7-a80a-4e9e-bd2f-436c8a884fd1","type":"BoxAnnotation"},{"id":"7a0525cc-55d6-48fa-9055-850bc3b16a13","type":"Legend"},{"id":"cd853902-f97c-4992-8384-5ed93deffbc5","type":"GlyphRenderer"},{"id":"611ba4cf-ffd4-4546-88e1-08aeb3b14a35","type":"GlyphRenderer"},{"id":"76953deb-9860-4763-8a94-ac5bb647cab4","type":"GlyphRenderer"},{"id":"cf6d03a5-0a25-421f-8c94-20b344fc52cb","type":"GlyphRenderer"},{"id":"eca0ff91-84df-49e0-bed0-a5d1a0fbe0d6","type":"GlyphRenderer"},{"id":"4f0b25ee-dbc0-47f9-83c7-ed0e3b632ddd","type":"GlyphRenderer"}],"title":{"id":"7720a27e-333d-4dd8-af6f-a50c85953607","type":"Title"},"tool_events":{"id":"190f18d3-cf9a-4768-8341-9e6b68050861","type":"ToolEvents"},"toolbar":{"id":"a071c649-6e58-40d2-b509-d7b4aab2938b","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"34043d67-f8a7-44b4-a686-034d27f0c1af","type":"DataRange1d"},"x_scale":{"id":"b046beaf-bece-4af0-bda4-f549e8d43b24","type":"LogScale"},"y_range":{"id":"d9a2eb86-4659-4ef4-bdbd-78a8fbeb77c3","type":"DataRange1d"},"y_scale":{"id":"9d3f8f44-3c13-4901-944a-cd5ab8f7128e","type":"LogScale"}},"id":"8ac92914-8e42-465b-b484-6acf7a09c79f","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"da37c611-bedc-4bf4-b960-9f180884f70d","type":"ToolEvents"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"PhAgSQGEM0BmSs6U6HdDQAwo6ea6PVNAoNqNGAnLYkDbj4LPiEVyQN1kKI9W54BAZTy4mhPIjED7LEuHcqKWQMQ4SLgPgZ5A","dtype":"float64","shape":[9]}}},"id":"226b848b-78d0-47a7-8136-381095ca2605","type":"ColumnDataSource"},{"attributes":{"label":{"value":"100ms"},"renderers":[{"id":"0e811120-b5fb-4b6a-acaf-28b0e4a6c1d1","type":"GlyphRenderer"}]},"id":"6eb5feba-11a0-471c-b3e9-b6c749e1f6de","type":"LegendItem"},{"attributes":{"plot":null,"text":"DataFrames: Elementwise"},"id":"7720a27e-333d-4dd8-af6f-a50c85953607","type":"Title"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"73babae4-8e06-44f1-9d83-c6eb0618f4ae","type":"FixedTicker"},{"attributes":{"data_source":{"id":"2ac7d06b-3cf9-49d7-9bc5-b0cdd1c04e06","type":"ColumnDataSource"},"glyph":{"id":"97216997-39de-4b2a-9536-0b0f61e8156d","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"8ef089b6-3e17-4882-8fb0-80ff34ab8b69","type":"Line"},"selection_glyph":null},"id":"34a85c78-f3f8-4f21-91e8-1fe0f4fae357","type":"GlyphRenderer"},{"attributes":{"children":[{"id":"bddfdd21-3692-4f09-a839-053ec2e8b96e","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"51125763-50b1-42e3-a52f-a429803a4aa3","type":"WidgetBox"},{"attributes":{"callback":null,"start":0},"id":"34043d67-f8a7-44b4-a686-034d27f0c1af","type":"DataRange1d"},{"attributes":{"line_color":{"value":"#2ca02c"},"x":{"field":"x"},"y":{"field":"y"}},"id":"5ba8039d-868c-4c4e-93d9-acde699026b7","type":"Line"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"97216997-39de-4b2a-9536-0b0f61e8156d","type":"Line"},{"attributes":{},"id":"190f18d3-cf9a-4768-8341-9e6b68050861","type":"ToolEvents"},{"attributes":{"callback":null,"start":0},"id":"8ec9bb53-f780-4baa-8ef5-a50a56dc6be3","type":"DataRange1d"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"ef963147-69dd-4316-824e-c39236f8374e","type":"PanTool"},{"id":"dd1840f2-3a3f-4ee8-96d7-274b0b58a153","type":"WheelZoomTool"},{"id":"cc2ba96b-474a-45b8-a0f7-c979a271d7f6","type":"BoxZoomTool"},{"id":"89c7e6f2-37c0-4e30-8273-cc364867a06e","type":"SaveTool"},{"id":"5f24c536-7893-4741-b8a6-f4e717ad6684","type":"ResetTool"},{"id":"7ec0ca47-849b-4454-a2f0-7708648ed7be","type":"HelpTool"}]},"id":"344426e7-d635-4e04-a5de-4ffd275f71c6","type":"Toolbar"},{"attributes":{"children":[{"id":"4d110cea-f4d8-49a1-bd97-89adb074f930","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"5c8cf25d-0f4c-46d7-a10b-af71e65fd1f4","type":"WidgetBox"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"c869dddf-026b-4298-848c-9488ecfbb0d0","type":"PanTool"},{"id":"06f0f153-fe4f-4f04-bd9b-59cd409d355c","type":"WheelZoomTool"},{"id":"923715ca-d193-4f9d-944c-58b88c7822e4","type":"BoxZoomTool"},{"id":"71367c3f-65b3-44c2-bc8b-9fc765bc5439","type":"SaveTool"},{"id":"6e79b2a1-f12d-421b-97e1-436f5f3e9ab2","type":"ResetTool"},{"id":"06f3f5fc-0cde-4733-a283-d09e63fb8e80","type":"HelpTool"}]},"id":"a071c649-6e58-40d2-b509-d7b4aab2938b","type":"Toolbar"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"ee3537e5-86e7-4a9d-b711-c2af4eba781f","type":"Circle"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"98d27QUzmkDK8eq3P0GeQDr+4mLrCqlAVTMqiTA6rEBCN3IZNmOuQOX8onhe0a1A7S7Drxhtq0DDEdAa+eaqQHPoMPdNM6lA","dtype":"float64","shape":[9]}}},"id":"216e410a-eab9-4916-8e8e-cdb763349136","type":"ColumnDataSource"},{"attributes":{"num_minor_ticks":10},"id":"2b14d7dd-d2f6-4a27-847e-77b0bf60361e","type":"LogTicker"},{"attributes":{"plot":null,"text":"Tasks: Tree Reduction"},"id":"6ceb9969-20b5-49b8-af74-e7e4340a10dc","type":"Title"},{"attributes":{"data_source":{"id":"d29b555c-87cc-486b-b2ff-f337f857ec70","type":"ColumnDataSource"},"glyph":{"id":"5ba8039d-868c-4c4e-93d9-acde699026b7","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"b2fed8be-3137-416d-b963-f115b6352fcb","type":"Line"},"selection_glyph":null},"id":"d1ebdf62-fdde-4a98-b4cb-020bb3f8a1a8","type":"GlyphRenderer"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"8ef089b6-3e17-4882-8fb0-80ff34ab8b69","type":"Line"},{"attributes":{"children":[{"id":"bc2a925a-2e83-48cf-a2c4-b3b85a7804f9","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"b2d598c9-d4df-4a68-8f1d-68cffc6f1164","type":"WidgetBox"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"08a42295-f619-42c9-b9b2-ebe2f84a8490","type":"Circle"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"f674b4e6-f1c4-4679-8225-4004f87731fd","type":"FixedTicker"},{"attributes":{"num_minor_ticks":10},"id":"e81dc0e0-36f0-41d3-9456-ad2840ba21f8","type":"LogTicker"},{"attributes":{"callback":null,"end":31321.93079477908,"start":0},"id":"d9a2eb86-4659-4ef4-bdbd-78a8fbeb77c3","type":"DataRange1d"},{"attributes":{"data_source":{"id":"226b848b-78d0-47a7-8136-381095ca2605","type":"ColumnDataSource"},"glyph":{"id":"ee3537e5-86e7-4a9d-b711-c2af4eba781f","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"08a42295-f619-42c9-b9b2-ebe2f84a8490","type":"Circle"},"selection_glyph":null},"id":"6cdd9453-036b-46a1-8702-56743bbbbb70","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"98d27QUzmkDK8eq3P0GeQDr+4mLrCqlAVTMqiTA6rEBCN3IZNmOuQOX8onhe0a1A7S7Drxhtq0DDEdAa+eaqQHPoMPdNM6lA","dtype":"float64","shape":[9]}}},"id":"2ac7d06b-3cf9-49d7-9bc5-b0cdd1c04e06","type":"ColumnDataSource"},{"attributes":{"ticker":null},"id":"adfc28a7-c9ef-4dba-b554-717dbda721ec","type":"LogTickFormatter"},{"attributes":{"label":{"value":"1us"},"renderers":[{"id":"34a85c78-f3f8-4f21-91e8-1fe0f4fae357","type":"GlyphRenderer"}]},"id":"ae1c749a-1e1d-4508-bc62-96b7e7b8b472","type":"LegendItem"},{"attributes":{"plot":{"id":"8ac92914-8e42-465b-b484-6acf7a09c79f","subtype":"Figure","type":"Plot"},"ticker":{"id":"2b14d7dd-d2f6-4a27-847e-77b0bf60361e","type":"LogTicker"}},"id":"92f0df92-fb07-4101-b4c1-bd10f1501839","type":"Grid"},{"attributes":{"callback":null,"tabs":[{"id":"53351d4a-2cee-4165-8f1c-c86f897bd418","type":"Panel"},{"id":"890493da-ef86-4d46-938a-2dd5a182a384","type":"Panel"}]},"id":"515cf591-ccc9-4d46-8771-c0b31a55cb97","type":"Tabs"},{"attributes":{"callback":null,"start":0},"id":"4e708514-46ec-4b98-946d-1599deda79a7","type":"DataRange1d"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"1Kb//XpDnED+4P/fNNapQHf4V3ait6lAprCFQjc6r0DGsrWLmNarQOaB18HP4q1ArjxETcD/rEC2aNo+EvGrQGxqBt5Qg6tA","dtype":"float64","shape":[9]}}},"id":"bc931ae2-6111-4082-9ad6-fc2397b13e0d","type":"ColumnDataSource"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"d0cc3968-e9bd-45dc-a1c6-7082cb75b4ea","type":"Line"},{"attributes":{"children":[{"id":"067958e4-18bc-4159-852e-fc2a375a27cb","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"48036047-1f68-42ed-9c8e-881e34e44386","type":"WidgetBox"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"1Kb//XpDnED+4P/fNNapQHf4V3ait6lAprCFQjc6r0DGsrWLmNarQOaB18HP4q1ArjxETcD/rEC2aNo+EvGrQGxqBt5Qg6tA","dtype":"float64","shape":[9]}}},"id":"d29b555c-87cc-486b-b2ff-f337f857ec70","type":"ColumnDataSource"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"b18f555f-df99-4a18-97da-08065ef87eac","type":"Circle"},{"attributes":{},"id":"b046beaf-bece-4af0-bda4-f549e8d43b24","type":"LogScale"},{"attributes":{"children":[{"id":"682f90be-e284-4f12-a195-8e4927a7b6dc","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"53cea20e-2f14-4b85-8239-4e58ae53738b","type":"WidgetBox"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"b2fed8be-3137-416d-b963-f115b6352fcb","type":"Line"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"d0bd021d-6c9f-4b97-82d6-e9dfdb6f3fc7","type":"Circle"},{"attributes":{"axis_label":"cores","formatter":{"id":"adfc28a7-c9ef-4dba-b554-717dbda721ec","type":"LogTickFormatter"},"plot":{"id":"8ac92914-8e42-465b-b484-6acf7a09c79f","subtype":"Figure","type":"Plot"},"ticker":{"id":"8ba39ea0-6825-4ab7-8b3f-7c1ea62a5c1f","type":"FixedTicker"}},"id":"da0293e6-8787-4a4b-acc4-54f1b66b0e15","type":"LogAxis"},{"attributes":{"data_source":{"id":"47bc40dc-b545-48af-bd85-d901454eb062","type":"ColumnDataSource"},"glyph":{"id":"d0cc3968-e9bd-45dc-a1c6-7082cb75b4ea","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"196b337e-cf37-461b-a2f5-a405e2edcb0b","type":"Line"},"selection_glyph":null},"id":"8fca08fa-ae6e-4a96-b377-c51a5fac6c86","type":"GlyphRenderer"},{"attributes":{},"id":"9d3f8f44-3c13-4901-944a-cd5ab8f7128e","type":"LogScale"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"1f4d4bb4-e030-45fe-9404-a338c9ff5df1","type":"LogTickFormatter"},"plot":{"id":"8ac92914-8e42-465b-b484-6acf7a09c79f","subtype":"Figure","type":"Plot"},"ticker":{"id":"e81dc0e0-36f0-41d3-9456-ad2840ba21f8","type":"LogTicker"}},"id":"99c0ff9d-5883-4247-8181-85ebbc60dbdb","type":"LogAxis"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"3cac42c0-6194-4169-9716-4dec8c4aa05a","type":"Line"},{"attributes":{"label":{"value":"1us"},"renderers":[{"id":"d1ebdf62-fdde-4a98-b4cb-020bb3f8a1a8","type":"GlyphRenderer"}]},"id":"a8dc3374-6ebe-49de-b7b0-c9600eb77fb9","type":"LegendItem"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"a291aa4a-9f10-4223-8b97-4859c884adc0","type":"Circle"},{"attributes":{"children":[{"id":"b2d598c9-d4df-4a68-8f1d-68cffc6f1164","type":"WidgetBox"},{"id":"c042a9dd-3b37-48f3-be88-5992ed5c87a3","type":"WidgetBox"}],"sizing_mode":"scale_width"},"id":"609029c0-8bfc-4ee4-accf-af70114ed2af","type":"Row"},{"attributes":{"data_source":{"id":"58d2c711-9c56-4160-b4fc-688acfb61595","type":"ColumnDataSource"},"glyph":{"id":"1f486cb7-4d22-4c1f-a311-6b93b01ec372","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"74a558cd-fb53-4650-a6d9-bd590e706dc8","type":"Line"},"selection_glyph":null},"id":"709df9d8-d49b-4549-8089-964dc4ed02d1","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"216e410a-eab9-4916-8e8e-cdb763349136","type":"ColumnDataSource"},"glyph":{"id":"d0bd021d-6c9f-4b97-82d6-e9dfdb6f3fc7","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"a291aa4a-9f10-4223-8b97-4859c884adc0","type":"Circle"},"selection_glyph":null},"id":"c3588a2b-2bad-4eac-a8e9-cdba1cb268e2","type":"GlyphRenderer"},{"attributes":{"dimension":1,"plot":{"id":"8ac92914-8e42-465b-b484-6acf7a09c79f","subtype":"Figure","type":"Plot"},"ticker":{"id":"e81dc0e0-36f0-41d3-9456-ad2840ba21f8","type":"LogTicker"}},"id":"983d1f9d-e177-4bcc-a1a8-d9e40e414ce6","type":"Grid"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"309755c0-e6ae-4ef6-92ac-88ecfed33a2c","type":"Line"},{"attributes":{"data_source":{"id":"ae076018-ffa2-41fa-aa1e-d4f975b654a4","type":"ColumnDataSource"},"glyph":{"id":"e369f3b7-fe55-4643-a276-feddb15f54a5","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"b18f555f-df99-4a18-97da-08065ef87eac","type":"Circle"},"selection_glyph":null},"id":"cbe97b25-e92c-4fa4-95f7-c625d8f6e111","type":"GlyphRenderer"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"0f0a75f0-9615-4598-ad77-dcac0abfc7f6","type":"Circle"},{"attributes":{"sizing_mode":"scale_width","toolbar_location":"above","tools":[{"id":"2b82eb62-e193-41f6-836a-f70e97ad6b5a","type":"PanTool"},{"id":"c13542c5-c8f2-4b1d-a5ce-55f0dde4c758","type":"WheelZoomTool"},{"id":"d7b8a053-c9f1-4fb7-bed4-a3df788f3ed2","type":"BoxZoomTool"},{"id":"13000b57-843e-4b11-98c2-6f02983aa18e","type":"SaveTool"},{"id":"e1966624-3bba-4c3a-86cf-1106a6800dc3","type":"ResetTool"},{"id":"08f094cf-dc4b-463e-9621-abdff60d0163","type":"HelpTool"},{"id":"76cf33d1-1336-4737-8cad-79fdcc322c10","type":"PanTool"},{"id":"f8834acc-9897-441d-9102-1f616408a50e","type":"WheelZoomTool"},{"id":"e7ac849f-5c88-4646-ae8c-6a4bc419f016","type":"BoxZoomTool"},{"id":"2c1a56f5-2c0d-4758-a7cc-eb45971629aa","type":"SaveTool"},{"id":"43e9c244-f5b8-4240-a21d-ba93dd66d881","type":"ResetTool"},{"id":"a0a42aa4-41ea-4099-a082-ff6b3e0b4400","type":"HelpTool"},{"id":"c869dddf-026b-4298-848c-9488ecfbb0d0","type":"PanTool"},{"id":"06f0f153-fe4f-4f04-bd9b-59cd409d355c","type":"WheelZoomTool"},{"id":"923715ca-d193-4f9d-944c-58b88c7822e4","type":"BoxZoomTool"},{"id":"71367c3f-65b3-44c2-bc8b-9fc765bc5439","type":"SaveTool"},{"id":"6e79b2a1-f12d-421b-97e1-436f5f3e9ab2","type":"ResetTool"},{"id":"06f3f5fc-0cde-4733-a283-d09e63fb8e80","type":"HelpTool"},{"id":"0b91a400-e7e6-479c-ab8d-dd80448d3b0e","type":"PanTool"},{"id":"61d07d7f-f947-4d2a-b341-c4bf85907198","type":"WheelZoomTool"},{"id":"57656b7a-1720-478f-ac0a-95a597024f3f","type":"BoxZoomTool"},{"id":"15a60aeb-0901-43a5-a728-241b8af7fc12","type":"SaveTool"},{"id":"a1f08da9-1313-44bd-8f3b-3a3d3c43eb46","type":"ResetTool"},{"id":"38f21f50-12f8-49d4-88f0-a7d648ec7347","type":"HelpTool"},{"id":"de25869c-2180-4518-a20f-bdf692f81aa1","type":"PanTool"},{"id":"a4b243e0-e791-4a8d-aec9-365070929b02","type":"WheelZoomTool"},{"id":"b9e581e8-8947-4185-bb76-39863c3ef43d","type":"BoxZoomTool"},{"id":"bcecb8b5-28bd-4f4f-bdb3-a693d7ebae56","type":"SaveTool"},{"id":"3270958b-802a-4185-991e-b40fd50477a0","type":"ResetTool"},{"id":"ecdc3244-d890-4831-ae2f-4b9a127b2173","type":"HelpTool"},{"id":"ece331ec-6fb2-4089-8b37-3920e038c682","type":"PanTool"},{"id":"01965bf4-e8c4-4ab4-ba11-d812d609dfe5","type":"WheelZoomTool"},{"id":"5bae1a67-b093-424d-9098-79c5e411c1e5","type":"BoxZoomTool"},{"id":"20d93f6e-d1ac-41f7-9eb4-5bb797a77934","type":"SaveTool"},{"id":"17d27847-a15f-4d57-8436-dd60b85219ef","type":"ResetTool"},{"id":"2c4661b2-6594-4d44-941f-b9b29a09447a","type":"HelpTool"},{"id":"9d53358f-a4b9-4b60-ac35-5f0e5f26d9fe","type":"PanTool"},{"id":"71b8f90f-69b5-4a11-b22e-9c8037bd5bef","type":"WheelZoomTool"},{"id":"09ada8fa-169d-49df-abc7-0703b7616c81","type":"BoxZoomTool"},{"id":"168f662e-50de-4018-b25f-82741503df6e","type":"SaveTool"},{"id":"ff50f331-ff6c-4fc5-9f0d-272907ba7131","type":"ResetTool"},{"id":"0a9d28ae-0c46-4fa8-93b9-7586884f78b2","type":"HelpTool"},{"id":"81841aa7-676d-402a-8666-6664cf5519ae","type":"PanTool"},{"id":"06fbd4d5-fe13-42d0-9421-8d0877b78efa","type":"WheelZoomTool"},{"id":"77426e4c-cc31-4908-940c-e24d10e80abb","type":"BoxZoomTool"},{"id":"da292bcf-3c4c-4796-abd6-94dc90c9c8bb","type":"SaveTool"},{"id":"262cfe98-756b-4aed-b2e9-fcd9a7fcb524","type":"ResetTool"},{"id":"48e2c614-a86a-4802-b12a-53e7d558caa8","type":"HelpTool"},{"id":"9bfe2281-8649-489d-b24b-7d071f5e0448","type":"PanTool"},{"id":"04994c9c-5fd6-43a7-8350-ba05388e5dda","type":"WheelZoomTool"},{"id":"cd115f44-a4b3-4792-b271-ff55420fa101","type":"BoxZoomTool"},{"id":"1867925c-2af4-4db7-8e7f-f5ebe7c04b2c","type":"SaveTool"},{"id":"139b60e4-50a4-4e7d-859b-078ddbdbdcc7","type":"ResetTool"},{"id":"c6324a86-b7a0-4476-8559-7e4c969cbc1d","type":"HelpTool"},{"id":"0a12f286-72b9-40f7-9747-cce4fca208b7","type":"PanTool"},{"id":"6c2a2f99-b315-4295-938e-4182eefbf3a8","type":"WheelZoomTool"},{"id":"2d6b0dc4-acfe-49bb-9dda-551a36dcf187","type":"BoxZoomTool"},{"id":"aaeac710-c162-4542-bc3a-0a5dabae8022","type":"SaveTool"},{"id":"541700d3-9318-4921-895f-bc6a03b02ecb","type":"ResetTool"},{"id":"21b20b53-1ceb-423e-a2ec-b801c3464c1e","type":"HelpTool"},{"id":"04a7a51b-fbbb-425a-8cfe-6dd02962e744","type":"PanTool"},{"id":"54250a06-b278-4bf6-b85a-f689f5021979","type":"WheelZoomTool"},{"id":"1a20d206-c661-4312-9339-cee7e4878f9b","type":"BoxZoomTool"},{"id":"29feaaed-05a7-4359-9eed-5e7cb8487502","type":"SaveTool"},{"id":"93e59c29-b745-43c4-a2ec-157e1523e2dc","type":"ResetTool"},{"id":"f4d6378e-99e2-41ca-9077-4e1fa1c98975","type":"HelpTool"},{"id":"b9db48f5-2638-4c94-a364-8e39eba47393","type":"PanTool"},{"id":"51b20535-8160-496c-986d-8baaecbb2697","type":"WheelZoomTool"},{"id":"e97e20d3-03e2-428d-aa0f-d00fd9bde72e","type":"BoxZoomTool"},{"id":"86256809-cc92-4e15-ae4a-6278598dfde4","type":"SaveTool"},{"id":"ee40caee-98db-40f4-b0db-7656427ffd03","type":"ResetTool"},{"id":"de4229e6-b7b3-4b99-a9cf-99e8e2addfe1","type":"HelpTool"}]},"id":"28b76cb2-6ef1-4876-bb6b-4ad74fe7ff10","type":"ToolbarBox"},{"attributes":{"data_source":{"id":"bc931ae2-6111-4082-9ad6-fc2397b13e0d","type":"ColumnDataSource"},"glyph":{"id":"0f0a75f0-9615-4598-ad77-dcac0abfc7f6","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"02400165-226a-443f-bd23-9a95c5960140","type":"Circle"},"selection_glyph":null},"id":"456a4a3c-1e23-4e6a-bedd-81c07d21ef13","type":"GlyphRenderer"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"e369f3b7-fe55-4643-a276-feddb15f54a5","type":"Circle"},{"attributes":{"data_source":{"id":"f2fa5cbc-9ff1-43ae-8dd7-30ea0c8a8ecb","type":"ColumnDataSource"},"glyph":{"id":"309755c0-e6ae-4ef6-92ac-88ecfed33a2c","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"9720958c-b388-4629-b5bf-1f10a75bb572","type":"Line"},"selection_glyph":null},"id":"cd853902-f97c-4992-8384-5ed93deffbc5","type":"GlyphRenderer"},{"attributes":{"child":{"id":"8ac92914-8e42-465b-b484-6acf7a09c79f","subtype":"Figure","type":"Plot"},"title":"log"},"id":"d02370bb-d02f-4e1b-ba87-d899301df6bf","type":"Panel"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"02400165-226a-443f-bd23-9a95c5960140","type":"Circle"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,5109.292459316874]}},"id":"47bc40dc-b545-48af-bd85-d901454eb062","type":"ColumnDataSource"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"48fde0e7-a80a-4e9e-bd2f-436c8a884fd1","type":"BoxAnnotation"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"196b337e-cf37-461b-a2f5-a405e2edcb0b","type":"Line"},{"attributes":{"children":[{"id":"7d705d97-1eec-49b4-bda6-9eebdb487600","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"c042a9dd-3b37-48f3-be88-5992ed5c87a3","type":"WidgetBox"},{"attributes":{"children":[{"id":"28b76cb2-6ef1-4876-bb6b-4ad74fe7ff10","type":"ToolbarBox"},{"id":"3f03d566-49e4-49c5-a975-a6991af0707f","type":"Column"}],"sizing_mode":"scale_width"},"id":"8cef5b34-d5d8-47bf-9055-bdcedac89484","type":"Column"},{"attributes":{"plot":{"id":"8ac92914-8e42-465b-b484-6acf7a09c79f","subtype":"Figure","type":"Plot"}},"id":"c869dddf-026b-4298-848c-9488ecfbb0d0","type":"PanTool"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"b1eaf9f9-78f0-4828-ac0e-59daee40d548","type":"Line"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"80bc6a21-9b42-485f-9c21-6fe45a77ce81","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,9558.858417964195]}},"id":"2f0e9839-fadf-4d77-8731-a9c5631134a1","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"b2ac3fe0-a9e8-4336-a6eb-a9c99bb77f6b","type":"ColumnDataSource"},"glyph":{"id":"b1eaf9f9-78f0-4828-ac0e-59daee40d548","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"fa6225cd-58ee-4a38-b0d6-ad3ea9bd0c62","type":"Line"},"selection_glyph":null},"id":"68b04869-9d0b-402b-8294-fb51737caed5","type":"GlyphRenderer"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"1f486cb7-4d22-4c1f-a311-6b93b01ec372","type":"Line"},{"attributes":{"plot":{"id":"8ac92914-8e42-465b-b484-6acf7a09c79f","subtype":"Figure","type":"Plot"}},"id":"06f0f153-fe4f-4f04-bd9b-59cd409d355c","type":"WheelZoomTool"},{"attributes":{"below":[{"id":"b50fcfc7-852e-4175-a484-f0c7929d0be6","type":"LogAxis"}],"left":[{"id":"3c6f921a-e1e0-4d53-bf09-070318c1480b","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"b50fcfc7-852e-4175-a484-f0c7929d0be6","type":"LogAxis"},{"id":"a91e731e-5176-484f-bdeb-cba448ed06ed","type":"Grid"},{"id":"3c6f921a-e1e0-4d53-bf09-070318c1480b","type":"LogAxis"},{"id":"7556ee27-b6ed-4f28-b58c-9f8233cee83f","type":"Grid"},{"id":"d74aa3a3-c638-44f8-bc84-1e0d355ed4fb","type":"BoxAnnotation"},{"id":"f5c62aec-87be-4702-b5af-4dea433eeeda","type":"Legend"},{"id":"45795d30-20cf-421a-90d5-552a1df6c804","type":"GlyphRenderer"},{"id":"f2343a90-0f7a-4366-8d1f-79df80822bfe","type":"GlyphRenderer"},{"id":"4a334960-fb43-4949-8e13-5cc44f095565","type":"GlyphRenderer"},{"id":"644a8b13-3361-41d5-b179-460630e1a06d","type":"GlyphRenderer"},{"id":"d580f476-985c-4210-a9c9-b6ebbdbe8ef8","type":"GlyphRenderer"},{"id":"39e06dbc-9a1e-441a-b66e-387a27314ce3","type":"GlyphRenderer"}],"title":{"id":"6ceb9969-20b5-49b8-af74-e7e4340a10dc","type":"Title"},"tool_events":{"id":"da37c611-bedc-4bf4-b960-9f180884f70d","type":"ToolEvents"},"toolbar":{"id":"32c5998f-e289-477a-a7f0-ec9c1b401522","type":"Toolbar"},"x_range":{"id":"8ec9bb53-f780-4baa-8ef5-a50a56dc6be3","type":"DataRange1d"},"x_scale":{"id":"ea6823ec-e386-4d86-9551-8b80fd1c0929","type":"LogScale"},"y_range":{"id":"2a2bf995-7f81-4c01-bdf4-737b49bb5434","type":"DataRange1d"},"y_scale":{"id":"693b8e3a-6242-45c9-813e-6c352e54feed","type":"LogScale"}},"id":"eb49fe05-6fcc-4349-974f-6d397bc15a51","subtype":"Figure","type":"Plot"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"74a558cd-fb53-4650-a6d9-bd590e706dc8","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"fa6225cd-58ee-4a38-b0d6-ad3ea9bd0c62","type":"Line"},{"attributes":{"overlay":{"id":"48fde0e7-a80a-4e9e-bd2f-436c8a884fd1","type":"BoxAnnotation"},"plot":{"id":"8ac92914-8e42-465b-b484-6acf7a09c79f","subtype":"Figure","type":"Plot"}},"id":"923715ca-d193-4f9d-944c-58b88c7822e4","type":"BoxZoomTool"},{"attributes":{"data_source":{"id":"14a6dd3a-ba9e-4451-92da-0b1a6f12b321","type":"ColumnDataSource"},"glyph":{"id":"549b8eda-aa71-44d5-9e0b-9b0ea1f30836","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"0e661795-1e40-42fd-bfb2-49c0e5851826","type":"Line"},"selection_glyph":null},"id":"7d8cbf49-f0bf-4739-979a-7f055e17f277","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"8ac92914-8e42-465b-b484-6acf7a09c79f","subtype":"Figure","type":"Plot"}},"id":"71367c3f-65b3-44c2-bc8b-9fc765bc5439","type":"SaveTool"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"60deee68-cb1a-48f9-b86a-aa720e9fdc6f","type":"Circle"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"7b38614f-09f7-4738-8ebe-089febc14dd8","type":"Line"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"acdabbcf-d0bc-4bc7-a81d-cbfd88cbe15e","type":"Line"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"0e46acf9-09da-417d-a968-159dea7e0873","type":"FixedTicker"},{"attributes":{"plot":{"id":"8ac92914-8e42-465b-b484-6acf7a09c79f","subtype":"Figure","type":"Plot"}},"id":"6e79b2a1-f12d-421b-97e1-436f5f3e9ab2","type":"ResetTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,506.8406659004797]}},"id":"58d2c711-9c56-4160-b4fc-688acfb61595","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,429249.4818984265]}},"id":"b2ac3fe0-a9e8-4336-a6eb-a9c99bb77f6b","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"8ac92914-8e42-465b-b484-6acf7a09c79f","subtype":"Figure","type":"Plot"}},"id":"06f3f5fc-0cde-4733-a283-d09e63fb8e80","type":"HelpTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"BSBNWCE2dEC9BpWUVImFQDzSCcCUkJBAVLgx0fyxnkCFGyLiy4+jQE7fanACZ6xAEeby5mMgsEDryZN7k+GwQFqGlput7rBA","dtype":"float64","shape":[9]}}},"id":"8ad7615f-51ad-41c7-b1e0-337bd8c82673","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"30917702-470e-4685-9830-7bc92131827d","type":"ColumnDataSource"},"glyph":{"id":"acdabbcf-d0bc-4bc7-a81d-cbfd88cbe15e","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"2401d944-e5a3-419a-870f-549c6f410968","type":"Line"},"selection_glyph":null},"id":"0d73ec87-f247-42dd-aa8e-6a30c4849c0e","type":"GlyphRenderer"},{"attributes":{"label":{"value":"100ms"},"renderers":[{"id":"f9a639d4-0a82-4749-88a4-bf025f7348ff","type":"GlyphRenderer"}]},"id":"e1eb6940-c616-4216-9f29-0b7b8d30694e","type":"LegendItem"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"zYZNznMFgkAJVc0dH1aIQF6HLbAX8KBAEDiFMgobsEBOSk1Dh6G8QIcwzAFNYsdAkYwvunD+0kD/VzpMtBLaQNxDJJJ7lt5A","dtype":"float64","shape":[9]}}},"id":"f2fa5cbc-9ff1-43ae-8dd7-30ea0c8a8ecb","type":"ColumnDataSource"},{"attributes":{"below":[{"id":"7b38a3fe-f342-4f09-8633-895cd6f78c16","type":"LogAxis"}],"left":[{"id":"09dc88c0-411b-4925-a5e3-21de90889400","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"7b38a3fe-f342-4f09-8633-895cd6f78c16","type":"LogAxis"},{"id":"f4fdfa42-c255-4537-8c5f-3fe3314db93c","type":"Grid"},{"id":"09dc88c0-411b-4925-a5e3-21de90889400","type":"LogAxis"},{"id":"98503142-2dad-473a-92da-a2b1090cbfd6","type":"Grid"},{"id":"bf7a1ecf-5767-45a0-bf2e-6acf9643a458","type":"BoxAnnotation"},{"id":"7af3ef08-9d2f-44e8-a096-ffed0ce42b15","type":"Legend"},{"id":"f9a639d4-0a82-4749-88a4-bf025f7348ff","type":"GlyphRenderer"},{"id":"cbe97b25-e92c-4fa4-95f7-c625d8f6e111","type":"GlyphRenderer"},{"id":"7d8cbf49-f0bf-4739-979a-7f055e17f277","type":"GlyphRenderer"},{"id":"80a40327-c430-40e1-8f2c-dac700777eb2","type":"GlyphRenderer"},{"id":"9f288aaa-82ff-46f4-8bf4-3cf335875256","type":"GlyphRenderer"},{"id":"a668550a-b6bd-420e-94a0-5fcd299635fa","type":"GlyphRenderer"}],"title":{"id":"b08c83bf-a84f-4c5c-895a-b473fdead52f","type":"Title"},"tool_events":{"id":"0d081c86-7571-47f1-bfc6-de24820edfb7","type":"ToolEvents"},"toolbar":{"id":"cf9a0a0c-6d28-4149-ba79-9a58f04c5061","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"ec1d775c-dfa5-4d0e-a787-98c358e3480e","type":"DataRange1d"},"x_scale":{"id":"c03284a4-5733-407b-ab0f-d4d81f920a2d","type":"LogScale"},"y_range":{"id":"9656f774-8191-411a-b3f6-3c3e54127af3","type":"DataRange1d"},"y_scale":{"id":"34f5a12d-d155-4614-b981-4c177fe227d2","type":"LogScale"}},"id":"c8d3b1c8-30be-40bf-af59-2f7817aaab49","subtype":"Figure","type":"Plot"},{"attributes":{"child":{"id":"c8d3b1c8-30be-40bf-af59-2f7817aaab49","subtype":"Figure","type":"Plot"},"title":"log"},"id":"80c3be4c-8081-480d-9da3-addc890ff49c","type":"Panel"},{"attributes":{"data_source":{"id":"538a97ac-2e34-4abf-a5e2-7570e274d3f5","type":"ColumnDataSource"},"glyph":{"id":"80bc6a21-9b42-485f-9c21-6fe45a77ce81","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"f199459f-7714-4eb0-a4e0-3b0712109a57","type":"Line"},"selection_glyph":null},"id":"76953deb-9860-4763-8a94-ac5bb647cab4","type":"GlyphRenderer"},{"attributes":{"items":[{"id":"9ec899a7-17d1-4036-b04b-f8571d6c4301","type":"LegendItem"},{"id":"51b3fff7-3940-4500-9667-260f70667185","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"1d34c2b9-9f2b-48c4-b9c6-c2c8e72691c5","subtype":"Figure","type":"Plot"}},"id":"e17a2f1c-37b7-4731-9b71-3c5f5de1b30e","type":"Legend"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"80c3be4c-8081-480d-9da3-addc890ff49c","type":"Panel"},{"id":"08641f92-85d2-4c9d-a11c-21a93031ddde","type":"Panel"}]},"id":"aeaa4543-6cc0-4ed6-bf28-929d3bf8399f","type":"Tabs"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,4996.005022052732]}},"id":"30917702-470e-4685-9830-7bc92131827d","type":"ColumnDataSource"},{"attributes":{"ticker":null},"id":"1f4d4bb4-e030-45fe-9404-a338c9ff5df1","type":"LogTickFormatter"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"2401d944-e5a3-419a-870f-549c6f410968","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"43c66efd-7698-49b9-b680-0d10546b250f","type":"Line"},{"attributes":{},"id":"548c551b-c483-4441-98cd-2ceea8cd15a2","type":"BasicTickFormatter"},{"attributes":{"overlay":{"id":"ab10fe90-cae8-447a-af95-b3c5e014c4af","type":"BoxAnnotation"},"plot":{"id":"1d34c2b9-9f2b-48c4-b9c6-c2c8e72691c5","subtype":"Figure","type":"Plot"}},"id":"dd8d9f16-c2ba-4ca6-91e3-6cc6c3606729","type":"BoxZoomTool"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"d2907638-c321-4c0f-9ca4-92f292a0e95d","type":"Line"},{"attributes":{"children":[{"id":"dd1c4ad1-e50c-417c-a615-b3d7acdf96c8","type":"Row"},{"id":"832b3d76-7254-415f-99bd-28ab20af0ab2","type":"Row"},{"id":"609029c0-8bfc-4ee4-accf-af70114ed2af","type":"Row"}],"sizing_mode":"scale_width"},"id":"3f03d566-49e4-49c5-a975-a6991af0707f","type":"Column"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"f199459f-7714-4eb0-a4e0-3b0712109a57","type":"Line"},{"attributes":{"data_source":{"id":"e0a1ea65-7844-4521-8824-7ef2b3b51172","type":"ColumnDataSource"},"glyph":{"id":"d2907638-c321-4c0f-9ca4-92f292a0e95d","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"226d8845-d933-459e-bfd1-4c5edd23fc80","type":"Line"},"selection_glyph":null},"id":"36248ec8-5457-4eb1-adb0-908e0f6b885e","type":"GlyphRenderer"},{"attributes":{"items":[{"id":"e1eb6940-c616-4216-9f29-0b7b8d30694e","type":"LegendItem"},{"id":"016140c0-733a-42e7-a6fd-9596879785b7","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"c8d3b1c8-30be-40bf-af59-2f7817aaab49","subtype":"Figure","type":"Plot"}},"id":"7af3ef08-9d2f-44e8-a096-ffed0ce42b15","type":"Legend"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"zYZNznMFgkAJVc0dH1aIQF6HLbAX8KBAEDiFMgobsEBOSk1Dh6G8QIcwzAFNYsdAkYwvunD+0kD/VzpMtBLaQNxDJJJ7lt5A","dtype":"float64","shape":[9]}}},"id":"dd527476-61b0-4a8a-b9d2-567373d3b727","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"1d34c2b9-9f2b-48c4-b9c6-c2c8e72691c5","subtype":"Figure","type":"Plot"}},"id":"ddb6422a-3fc9-47a5-885a-6a6dd8f99f1a","type":"ResetTool"},{"attributes":{"overlay":{"id":"add7c6e9-6c2c-44db-a848-5316f1f5604e","type":"BoxAnnotation"},"plot":{"id":"235a3d83-e4c1-4855-b3f3-96247ff0b146","subtype":"Figure","type":"Plot"}},"id":"f50914d8-200a-4f31-abfd-26c17a885c0c","type":"BoxZoomTool"},{"attributes":{},"id":"19779bf4-dc96-4d5f-a96c-650727df217c","type":"ToolEvents"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"VDws0AG1nUDZ+18BvWCtQLxzwc5E/bpAaCwXiat5w0Bb8Sa8jEzHQLDjXzrvA8JAB24/B96Gw0AcKVqQRQLDQPc5BRJjuMJA","dtype":"float64","shape":[9]}}},"id":"3013aa6d-021d-4bc4-a8db-8a82ff575182","type":"ColumnDataSource"},{"attributes":{"plot":null,"text":"Tasks: Dynamic Reduction"},"id":"b08c83bf-a84f-4c5c-895a-b473fdead52f","type":"Title"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,463070.74804554624]}},"id":"e0a1ea65-7844-4521-8824-7ef2b3b51172","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"VDws0AG1nUDZ+18BvWCtQLxzwc5E/bpAaCwXiat5w0Bb8Sa8jEzHQLDjXzrvA8JAB24/B96Gw0AcKVqQRQLDQPc5BRJjuMJA","dtype":"float64","shape":[9]}}},"id":"538a97ac-2e34-4abf-a5e2-7570e274d3f5","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"8ad7615f-51ad-41c7-b1e0-337bd8c82673","type":"ColumnDataSource"},"glyph":{"id":"d3712cc3-12bb-434e-9368-dcb219e339ed","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"65eaf9ce-00b9-49b4-97fb-bab4f40f5201","type":"Circle"},"selection_glyph":null},"id":"24b55eb1-0e53-4069-b743-6ac9c23fb075","type":"GlyphRenderer"},{"attributes":{"callback":null,"start":0},"id":"ec1d775c-dfa5-4d0e-a787-98c358e3480e","type":"DataRange1d"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"226d8845-d933-459e-bfd1-4c5edd23fc80","type":"Line"},{"attributes":{"label":{"value":"arithmetic"},"renderers":[{"id":"76953deb-9860-4763-8a94-ac5bb647cab4","type":"GlyphRenderer"}]},"id":"9707ea10-1988-4219-a460-b3ef4112a1be","type":"LegendItem"},{"attributes":{},"id":"0d081c86-7571-47f1-bfc6-de24820edfb7","type":"ToolEvents"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"528c4458-e43f-4ee6-909e-58cbbbeab9a0","type":"Line"},{"attributes":{"num_minor_ticks":10},"id":"ff96c4b6-a677-4cb7-a9c6-0efb1eb71431","type":"LogTicker"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"d5103372-bb6a-4b74-87c3-e3cc397eddc7","type":"PanTool"},{"id":"d17ca258-ae85-49ed-873d-53dfb55bdf92","type":"WheelZoomTool"},{"id":"50075953-0cab-4112-abce-afe021f330ea","type":"BoxZoomTool"},{"id":"216cc837-f8a2-4718-a2b9-ab6aae1188f5","type":"SaveTool"},{"id":"497251b6-85e2-450e-a3df-dc513d4623d8","type":"ResetTool"},{"id":"da51f522-93e8-4265-a989-c54b2d742198","type":"HelpTool"}]},"id":"ec2981f8-3c0a-461f-890b-8c6afd32553a","type":"Toolbar"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"06d1c4f3-2e25-4e2c-87d9-5dcdfdef1c9f","type":"PanTool"},{"id":"711ed14b-0adc-44af-9a23-d8e69826940d","type":"WheelZoomTool"},{"id":"92b4441f-a58a-4aa2-b658-2032f50692c7","type":"BoxZoomTool"},{"id":"82e9c665-d127-48d2-8b1d-81259c5c59c1","type":"SaveTool"},{"id":"4a8c0673-09a5-4a8b-9ec7-2c6b9e9a1e2f","type":"ResetTool"},{"id":"c58059d9-7ec6-40de-a9a1-e030da7a5db5","type":"HelpTool"}]},"id":"cf9a0a0c-6d28-4149-ba79-9a58f04c5061","type":"Toolbar"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"20d3ccb9-5f8a-499b-baf5-3dcdf25ae187","type":"FixedTicker"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"2437b245-9f04-4e74-afb9-429ff4bc323e","type":"Line"},{"attributes":{"num_minor_ticks":10},"id":"0901eb15-3d3b-4a56-a833-432c0e388578","type":"LogTicker"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"ba129916-b144-45bf-839b-308cc2fe0cca","type":"Circle"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"d3e6561e-37f2-4687-a2f0-956cf8e99078","type":"PanTool"},{"id":"3829dd1c-6699-4426-aa8e-c268d6ae79ea","type":"WheelZoomTool"},{"id":"a6655d1f-e619-4386-b6be-6edc76a7f639","type":"BoxZoomTool"},{"id":"fac5489e-3ff1-49ab-a77c-b72534911361","type":"SaveTool"},{"id":"d9c2feef-fc71-4e4c-acb8-81fe30e4f4ab","type":"ResetTool"},{"id":"8b0fce7e-8494-4b5e-ac5a-6d669d058969","type":"HelpTool"}]},"id":"32c5998f-e289-477a-a7f0-ec9c1b401522","type":"Toolbar"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"46f9230c-1513-48e6-85cc-89e281677acd","type":"Line"},{"attributes":{"num_minor_ticks":10},"id":"5989b85d-d7f3-4a9f-962f-058427056f5e","type":"LogTicker"},{"attributes":{"data_source":{"id":"7fff3c49-e098-4ed2-ab2c-6d988fa57cd3","type":"ColumnDataSource"},"glyph":{"id":"528c4458-e43f-4ee6-909e-58cbbbeab9a0","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"4cb0f6b1-b7e8-4d2f-92a1-ba4357a5e784","type":"Line"},"selection_glyph":null},"id":"eca0ff91-84df-49e0-bed0-a5d1a0fbe0d6","type":"GlyphRenderer"},{"attributes":{"num_minor_ticks":10},"id":"6b009896-2f97-4cc5-a515-1fc7534e4b1a","type":"LogTicker"},{"attributes":{"callback":null,"end":6021.609676148496,"start":0},"id":"9656f774-8191-411a-b3f6-3c3e54127af3","type":"DataRange1d"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"4814193b-7f49-47ad-a889-ad29dffb1e6f","type":"Circle"},{"attributes":{"callback":null,"end":3596.14811695509,"start":0},"id":"2a2bf995-7f81-4c01-bdf4-737b49bb5434","type":"DataRange1d"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"Gmr7TkyPlEBRf7wkoNCfQH9VJgZniqdALjvyvyi0rECvSL87J26wQO0MDDGQELJANhKvi9UxskBz0zc+QhSyQD61nBkBcrFA","dtype":"float64","shape":[9]}}},"id":"db943d83-03cd-4fea-b185-6dd6ef1b2660","type":"ColumnDataSource"},{"attributes":{"ticker":null},"id":"486c670a-fb22-4ffd-98c1-16474d37d388","type":"LogTickFormatter"},{"attributes":{"data_source":{"id":"3013aa6d-021d-4bc4-a8db-8a82ff575182","type":"ColumnDataSource"},"glyph":{"id":"ba129916-b144-45bf-839b-308cc2fe0cca","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"4814193b-7f49-47ad-a889-ad29dffb1e6f","type":"Circle"},"selection_glyph":null},"id":"cf6d03a5-0a25-421f-8c94-20b344fc52cb","type":"GlyphRenderer"},{"attributes":{"ticker":null},"id":"44bf66b5-b040-43f3-b613-7d8c1f271042","type":"LogTickFormatter"},{"attributes":{"plot":{"id":"c8d3b1c8-30be-40bf-af59-2f7817aaab49","subtype":"Figure","type":"Plot"},"ticker":{"id":"0901eb15-3d3b-4a56-a833-432c0e388578","type":"LogTicker"}},"id":"f4fdfa42-c255-4537-8c5f-3fe3314db93c","type":"Grid"},{"attributes":{"items":[{"id":"8ae41a66-88b0-4423-b9a9-153148095b06","type":"LegendItem"},{"id":"49216e78-0ce7-4ef2-8c38-04c33f920efb","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"41999ae5-ee50-4180-8a07-fdec3748db8d","subtype":"Figure","type":"Plot"}},"id":"202e406d-1c86-442e-a9a2-7eb69070d709","type":"Legend"},{"attributes":{"plot":{"id":"eb49fe05-6fcc-4349-974f-6d397bc15a51","subtype":"Figure","type":"Plot"},"ticker":{"id":"ff96c4b6-a677-4cb7-a9c6-0efb1eb71431","type":"LogTicker"}},"id":"a91e731e-5176-484f-bdeb-cba448ed06ed","type":"Grid"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"Gmr7TkyPlEBRf7wkoNCfQH9VJgZniqdALjvyvyi0rECvSL87J26wQO0MDDGQELJANhKvi9UxskBz0zc+QhSyQD61nBkBcrFA","dtype":"float64","shape":[9]}}},"id":"7b03661a-da30-48ae-baf6-c203e36ab1c9","type":"ColumnDataSource"},{"attributes":{},"id":"c03284a4-5733-407b-ab0f-d4d81f920a2d","type":"LogScale"},{"attributes":{"plot":{"id":"41999ae5-ee50-4180-8a07-fdec3748db8d","subtype":"Figure","type":"Plot"}},"id":"38f21f50-12f8-49d4-88f0-a7d648ec7347","type":"HelpTool"},{"attributes":{},"id":"ea6823ec-e386-4d86-9551-8b80fd1c0929","type":"LogScale"},{"attributes":{"axis_label":"cores","formatter":{"id":"486c670a-fb22-4ffd-98c1-16474d37d388","type":"LogTickFormatter"},"plot":{"id":"c8d3b1c8-30be-40bf-af59-2f7817aaab49","subtype":"Figure","type":"Plot"},"ticker":{"id":"f674b4e6-f1c4-4679-8225-4004f87731fd","type":"FixedTicker"}},"id":"7b38a3fe-f342-4f09-8633-895cd6f78c16","type":"LogAxis"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,147630.4757338077]}},"id":"7fff3c49-e098-4ed2-ab2c-6d988fa57cd3","type":"ColumnDataSource"},{"attributes":{"axis_label":"cores","formatter":{"id":"44bf66b5-b040-43f3-b613-7d8c1f271042","type":"LogTickFormatter"},"plot":{"id":"eb49fe05-6fcc-4349-974f-6d397bc15a51","subtype":"Figure","type":"Plot"},"ticker":{"id":"78e8d05c-b35c-4703-ae94-e6433b0aa44e","type":"FixedTicker"}},"id":"b50fcfc7-852e-4175-a484-f0c7929d0be6","type":"LogAxis"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"f891b164-8ac4-4312-9ada-fe4e94977263","type":"Line"},{"attributes":{},"id":"34f5a12d-d155-4614-b981-4c177fe227d2","type":"LogScale"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"4cb0f6b1-b7e8-4d2f-92a1-ba4357a5e784","type":"Line"},{"attributes":{},"id":"693b8e3a-6242-45c9-813e-6c352e54feed","type":"LogScale"},{"attributes":{"axis_label":"tasks/s","formatter":{"id":"0ec83177-9fe3-476b-aef4-91a0528ef45d","type":"LogTickFormatter"},"plot":{"id":"c8d3b1c8-30be-40bf-af59-2f7817aaab49","subtype":"Figure","type":"Plot"},"ticker":{"id":"5989b85d-d7f3-4a9f-962f-058427056f5e","type":"LogTicker"}},"id":"09dc88c0-411b-4925-a5e3-21de90889400","type":"LogAxis"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"0f2ed422-3629-4fce-b8e3-edef0474934a","type":"Line"},{"attributes":{"axis_label":"tasks/s","formatter":{"id":"33fbffc0-4674-4d29-95d1-a416b0a33583","type":"LogTickFormatter"},"plot":{"id":"eb49fe05-6fcc-4349-974f-6d397bc15a51","subtype":"Figure","type":"Plot"},"ticker":{"id":"6b009896-2f97-4cc5-a515-1fc7534e4b1a","type":"LogTicker"}},"id":"3c6f921a-e1e0-4d53-bf09-070318c1480b","type":"LogAxis"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"a585a097-5244-44ed-bb7d-624ca56c71b8","type":"Circle"},{"attributes":{"dimension":1,"plot":{"id":"c8d3b1c8-30be-40bf-af59-2f7817aaab49","subtype":"Figure","type":"Plot"},"ticker":{"id":"5989b85d-d7f3-4a9f-962f-058427056f5e","type":"LogTicker"}},"id":"98503142-2dad-473a-92da-a2b1090cbfd6","type":"Grid"},{"attributes":{"data_source":{"id":"864676dc-13f4-4b00-ad79-d0fb6cc50da1","type":"ColumnDataSource"},"glyph":{"id":"0f2ed422-3629-4fce-b8e3-edef0474934a","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"31a91e25-186a-4e36-b5ca-2c1eb51fa949","type":"Line"},"selection_glyph":null},"id":"4f0b25ee-dbc0-47f9-83c7-ed0e3b632ddd","type":"GlyphRenderer"},{"attributes":{"dimension":1,"plot":{"id":"eb49fe05-6fcc-4349-974f-6d397bc15a51","subtype":"Figure","type":"Plot"},"ticker":{"id":"6b009896-2f97-4cc5-a515-1fc7534e4b1a","type":"LogTicker"}},"id":"7556ee27-b6ed-4f28-b58c-9f8233cee83f","type":"Grid"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"0e6b8974-995b-41bd-87b3-95b0e260a731","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"31a91e25-186a-4e36-b5ca-2c1eb51fa949","type":"Line"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"db5ad287-33f6-4378-bec4-28e10514e436","type":"Line"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"b67e2512-2857-4f42-98a2-f65fdcc2066c","type":"Circle"},{"attributes":{"data_source":{"id":"c7abd770-347f-4b48-b018-4423b55e33bc","type":"ColumnDataSource"},"glyph":{"id":"0e6b8974-995b-41bd-87b3-95b0e260a731","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"43c66efd-7698-49b9-b680-0d10546b250f","type":"Line"},"selection_glyph":null},"id":"f9a639d4-0a82-4749-88a4-bf025f7348ff","type":"GlyphRenderer"},{"attributes":{"callback":null,"start":0},"id":"b19a05ed-da08-4c4e-bd1e-3d79b9604fc2","type":"DataRange1d"},{"attributes":{"data_source":{"id":"374a9186-f058-4688-a098-11394db69d3b","type":"ColumnDataSource"},"glyph":{"id":"db5ad287-33f6-4378-bec4-28e10514e436","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"3cac42c0-6194-4169-9716-4dec8c4aa05a","type":"Line"},"selection_glyph":null},"id":"45795d30-20cf-421a-90d5-552a1df6c804","type":"GlyphRenderer"},{"attributes":{"child":{"id":"bd7a4267-cf17-4206-a88f-6685cd6fdc61","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"08641f92-85d2-4c9d-a11c-21a93031ddde","type":"Panel"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"8ba39ea0-6825-4ab7-8b3f-7c1ea62a5c1f","type":"FixedTicker"},{"attributes":{"child":{"id":"eb49fe05-6fcc-4349-974f-6d397bc15a51","subtype":"Figure","type":"Plot"},"title":"log"},"id":"53351d4a-2cee-4165-8f1c-c86f897bd418","type":"Panel"},{"attributes":{"callback":null,"start":0},"id":"a0a586bf-9420-4eae-b3fe-b9400c461017","type":"DataRange1d"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"bf7a1ecf-5767-45a0-bf2e-6acf9643a458","type":"BoxAnnotation"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,486720.45329374564]}},"id":"864676dc-13f4-4b00-ad79-d0fb6cc50da1","type":"ColumnDataSource"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"d74aa3a3-c638-44f8-bc84-1e0d355ed4fb","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"c8d3b1c8-30be-40bf-af59-2f7817aaab49","subtype":"Figure","type":"Plot"}},"id":"06d1c4f3-2e25-4e2c-87d9-5dcdfdef1c9f","type":"PanTool"},{"attributes":{"plot":null,"text":"DataFrames: Elementwise"},"id":"36d0c6cb-6834-4209-9c6a-f394c6cf7730","type":"Title"},{"attributes":{"plot":{"id":"eb49fe05-6fcc-4349-974f-6d397bc15a51","subtype":"Figure","type":"Plot"}},"id":"d3e6561e-37f2-4687-a2f0-956cf8e99078","type":"PanTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,82786.0840579272]}},"id":"368ebbd7-bae3-4d68-8617-71a85eab8635","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"kaJ6m493kEDnCG6oG3iZQDH65WfdO51AN9IVWq5NnEATlZAk2WCfQArn1nJmT6JAEu2KAqxsokCrK/Acec2hQLjq14Ix6JZA","dtype":"float64","shape":[9]}}},"id":"21c6a951-806f-4a05-862c-8b9c54656803","type":"ColumnDataSource"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"0b91a400-e7e6-479c-ab8d-dd80448d3b0e","type":"PanTool"},{"id":"61d07d7f-f947-4d2a-b341-c4bf85907198","type":"WheelZoomTool"},{"id":"57656b7a-1720-478f-ac0a-95a597024f3f","type":"BoxZoomTool"},{"id":"15a60aeb-0901-43a5-a728-241b8af7fc12","type":"SaveTool"},{"id":"a1f08da9-1313-44bd-8f3b-3a3d3c43eb46","type":"ResetTool"},{"id":"38f21f50-12f8-49d4-88f0-a7d648ec7347","type":"HelpTool"}]},"id":"90856cc7-bb7d-439a-981c-07dc22c423ec","type":"Toolbar"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"Qs2j4G2rQkAGPm5LixNSQAEIVOXlumFAHhOcgak6cUC7DoHNWWyAQGIjzN8v5o5Auzk4xMOJnECcfp9Q3SOlQDBM/NVLGKxA","dtype":"float64","shape":[9]}}},"id":"0ed2e546-c35d-4985-b91e-c9ce4a99aed8","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"c8d3b1c8-30be-40bf-af59-2f7817aaab49","subtype":"Figure","type":"Plot"}},"id":"711ed14b-0adc-44af-9a23-d8e69826940d","type":"WheelZoomTool"},{"attributes":{},"id":"65bdd6d6-a9a1-4b37-ad15-96e017bfd116","type":"ToolEvents"},{"attributes":{"plot":{"id":"eb49fe05-6fcc-4349-974f-6d397bc15a51","subtype":"Figure","type":"Plot"}},"id":"3829dd1c-6699-4426-aa8e-c268d6ae79ea","type":"WheelZoomTool"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"e14090af-ea5a-42b8-8321-42ea86824188","type":"Line"},{"attributes":{"overlay":{"id":"bf7a1ecf-5767-45a0-bf2e-6acf9643a458","type":"BoxAnnotation"},"plot":{"id":"c8d3b1c8-30be-40bf-af59-2f7817aaab49","subtype":"Figure","type":"Plot"}},"id":"92b4441f-a58a-4aa2-b658-2032f50692c7","type":"BoxZoomTool"},{"attributes":{},"id":"8275f929-b9b2-4d3f-b293-df539ed78bd9","type":"BasicTickFormatter"},{"attributes":{"overlay":{"id":"d74aa3a3-c638-44f8-bc84-1e0d355ed4fb","type":"BoxAnnotation"},"plot":{"id":"eb49fe05-6fcc-4349-974f-6d397bc15a51","subtype":"Figure","type":"Plot"}},"id":"a6655d1f-e619-4386-b6be-6edc76a7f639","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"c8d3b1c8-30be-40bf-af59-2f7817aaab49","subtype":"Figure","type":"Plot"}},"id":"82e9c665-d127-48d2-8b1d-81259c5c59c1","type":"SaveTool"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"9f47b1ec-bc60-4e62-9f08-ef21fbf4782d","type":"Line"},{"attributes":{"plot":{"id":"eb49fe05-6fcc-4349-974f-6d397bc15a51","subtype":"Figure","type":"Plot"}},"id":"fac5489e-3ff1-49ab-a77c-b72534911361","type":"SaveTool"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"a8d36c07-1d97-49e1-9d58-015a82fc942a","type":"Line"},{"attributes":{"plot":{"id":"c8d3b1c8-30be-40bf-af59-2f7817aaab49","subtype":"Figure","type":"Plot"}},"id":"4a8c0673-09a5-4a8b-9ec7-2c6b9e9a1e2f","type":"ResetTool"},{"attributes":{"below":[{"id":"edbdefdf-1c66-4d43-8239-0c88a3660748","type":"LinearAxis"}],"left":[{"id":"9b421310-47a1-477b-96c2-0c768ef21e4e","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"edbdefdf-1c66-4d43-8239-0c88a3660748","type":"LinearAxis"},{"id":"f52e9572-9aa3-47c9-be00-643f1657cf2d","type":"Grid"},{"id":"9b421310-47a1-477b-96c2-0c768ef21e4e","type":"LinearAxis"},{"id":"89626384-979e-4411-9ae9-1fc932204cb8","type":"Grid"},{"id":"85719b8d-547b-42d8-a67b-dea193b89ff1","type":"BoxAnnotation"},{"id":"202e406d-1c86-442e-a9a2-7eb69070d709","type":"Legend"},{"id":"a2e3d51d-d8bb-41ac-8515-3f1154af5d97","type":"GlyphRenderer"},{"id":"43f1cd74-dc1a-4aab-8f12-2d6e8620b2f3","type":"GlyphRenderer"},{"id":"d1ea2033-418b-42dc-affa-05ab97a548a6","type":"GlyphRenderer"},{"id":"78d02e69-5824-47a6-92da-f24d594ffdac","type":"GlyphRenderer"},{"id":"1536de68-3f85-4ec4-9468-0e3fc8b2e64e","type":"GlyphRenderer"},{"id":"66a1bbc2-db12-4787-93f9-0f0b4ed2a298","type":"GlyphRenderer"}],"title":{"id":"36d0c6cb-6834-4209-9c6a-f394c6cf7730","type":"Title"},"tool_events":{"id":"65bdd6d6-a9a1-4b37-ad15-96e017bfd116","type":"ToolEvents"},"toolbar":{"id":"90856cc7-bb7d-439a-981c-07dc22c423ec","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"b19a05ed-da08-4c4e-bd1e-3d79b9604fc2","type":"DataRange1d"},"x_scale":{"id":"c1aabede-839b-41c1-bb94-f0de84f0848f","type":"LinearScale"},"y_range":{"id":"7a5bc9bf-1185-45ba-9913-11cdcb243a9f","type":"DataRange1d"},"y_scale":{"id":"d8ad20a7-5cd9-4996-9df2-57a9e389fa32","type":"LinearScale"}},"id":"41999ae5-ee50-4180-8a07-fdec3748db8d","subtype":"Figure","type":"Plot"},{"attributes":{"plot":{"id":"eb49fe05-6fcc-4349-974f-6d397bc15a51","subtype":"Figure","type":"Plot"}},"id":"d9c2feef-fc71-4e4c-acb8-81fe30e4f4ab","type":"ResetTool"},{"attributes":{"plot":{"id":"c8d3b1c8-30be-40bf-af59-2f7817aaab49","subtype":"Figure","type":"Plot"}},"id":"c58059d9-7ec6-40de-a9a1-e030da7a5db5","type":"HelpTool"},{"attributes":{},"id":"c1aabede-839b-41c1-bb94-f0de84f0848f","type":"LinearScale"},{"attributes":{"plot":{"id":"eb49fe05-6fcc-4349-974f-6d397bc15a51","subtype":"Figure","type":"Plot"}},"id":"8b0fce7e-8494-4b5e-ac5a-6d669d058969","type":"HelpTool"},{"attributes":{},"id":"066b97e1-b17a-43b2-bdf7-56c504f64209","type":"LinearScale"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"UGdETRRGSEAF/cD9b45XQEf6VqPzsGlAWonEk7n0ekB6ZUoa/4eOQF78OcpRNp1AikTHrPdfqkBkyoxeIYGzQPFuvBOchbdA","dtype":"float64","shape":[9]}}},"id":"c7abd770-347f-4b48-b018-4423b55e33bc","type":"ColumnDataSource"},{"attributes":{"callback":null,"end":31321.93079477908,"start":0},"id":"7a5bc9bf-1185-45ba-9913-11cdcb243a9f","type":"DataRange1d"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"Qs2j4G2rQkAGPm5LixNSQAEIVOXlumFAHhOcgak6cUC7DoHNWWyAQGIjzN8v5o5Auzk4xMOJnECcfp9Q3SOlQDBM/NVLGKxA","dtype":"float64","shape":[9]}}},"id":"374a9186-f058-4688-a098-11394db69d3b","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"UGdETRRGSEAF/cD9b45XQEf6VqPzsGlAWonEk7n0ekB6ZUoa/4eOQF78OcpRNp1AikTHrPdfqkBkyoxeIYGzQPFuvBOchbdA","dtype":"float64","shape":[9]}}},"id":"ae076018-ffa2-41fa-aa1e-d4f975b654a4","type":"ColumnDataSource"},{"attributes":{},"id":"d8ad20a7-5cd9-4996-9df2-57a9e389fa32","type":"LinearScale"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"2b200e79-13ef-4614-a86b-0c5ac8e51e1a","type":"Circle"},{"attributes":{},"id":"35297baa-7835-45c5-904a-1e2b7a405af7","type":"LinearScale"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"549b8eda-aa71-44d5-9e0b-9b0ea1f30836","type":"Line"},{"attributes":{"plot":{"id":"41999ae5-ee50-4180-8a07-fdec3748db8d","subtype":"Figure","type":"Plot"},"ticker":{"id":"5d1fbe0b-b4b7-424b-8fa8-0deacdafab57","type":"BasicTicker"}},"id":"f52e9572-9aa3-47c9-be00-643f1657cf2d","type":"Grid"},{"attributes":{"ticker":null},"id":"33fbffc0-4674-4d29-95d1-a416b0a33583","type":"LogTickFormatter"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"0e661795-1e40-42fd-bfb2-49c0e5851826","type":"Line"},{"attributes":{"axis_label":"cores","formatter":{"id":"8275f929-b9b2-4d3f-b293-df539ed78bd9","type":"BasicTickFormatter"},"plot":{"id":"41999ae5-ee50-4180-8a07-fdec3748db8d","subtype":"Figure","type":"Plot"},"ticker":{"id":"6d26b10e-948b-4ae1-b8a0-c48032e939cc","type":"FixedTicker"}},"id":"edbdefdf-1c66-4d43-8239-0c88a3660748","type":"LinearAxis"},{"attributes":{"data_source":{"id":"57cae494-21bb-46fc-b4c5-bff8741d5e59","type":"ColumnDataSource"},"glyph":{"id":"60deee68-cb1a-48f9-b86a-aa720e9fdc6f","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"2b200e79-13ef-4614-a86b-0c5ac8e51e1a","type":"Circle"},"selection_glyph":null},"id":"644a8b13-3361-41d5-b179-460630e1a06d","type":"GlyphRenderer"},{"attributes":{"axis_label":"cores","formatter":{"id":"4a2078e2-dc0a-4ee9-8148-99c308724a80","type":"BasicTickFormatter"},"plot":{"id":"4fab0e63-76cb-46a6-843e-af38f25fd676","subtype":"Figure","type":"Plot"},"ticker":{"id":"e252e15d-c6d5-4642-8016-3528d9a053b7","type":"FixedTicker"}},"id":"42bbee2c-121b-411e-9428-94c80306a923","type":"LinearAxis"},{"attributes":{"plot":null,"text":"Tasks: Dynamic Reduction"},"id":"5ed47176-1165-43e2-acb6-dcbc4ffee512","type":"Title"},{"attributes":{},"id":"5d1fbe0b-b4b7-424b-8fa8-0deacdafab57","type":"BasicTicker"},{"attributes":{"items":[{"id":"c873e202-11e2-4533-9862-daf563b91c32","type":"LegendItem"},{"id":"2cd69d4e-ce9d-4ffe-a8da-73b9a4a76f70","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"eb49fe05-6fcc-4349-974f-6d397bc15a51","subtype":"Figure","type":"Plot"}},"id":"f5c62aec-87be-4702-b5af-4dea433eeeda","type":"Legend"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"kaJ6m493kEDnCG6oG3iZQDH65WfdO51AN9IVWq5NnEATlZAk2WCfQArn1nJmT6JAEu2KAqxsokCrK/Acec2hQLjq14Ix6JZA","dtype":"float64","shape":[9]}}},"id":"14a6dd3a-ba9e-4451-92da-0b1a6f12b321","type":"ColumnDataSource"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"548c551b-c483-4441-98cd-2ceea8cd15a2","type":"BasicTickFormatter"},"plot":{"id":"41999ae5-ee50-4180-8a07-fdec3748db8d","subtype":"Figure","type":"Plot"},"ticker":{"id":"9765a573-ce07-4dac-8d25-f5c756725fe1","type":"BasicTicker"}},"id":"9b421310-47a1-477b-96c2-0c768ef21e4e","type":"LinearAxis"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"q5SRkbXHk0C5OgbIBTCbQNOi7PG6oZ9AYTvcNx2BokB7bWhsEu6mQOBg60Yg8aVA9irH/+SCpUCs8pL575ilQPIu2GiALKRA","dtype":"float64","shape":[9]}}},"id":"57cae494-21bb-46fc-b4c5-bff8741d5e59","type":"ColumnDataSource"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"94f6bf21-e27e-4108-87fb-96b57789a1b2","type":"BasicTickFormatter"},"plot":{"id":"4fab0e63-76cb-46a6-843e-af38f25fd676","subtype":"Figure","type":"Plot"},"ticker":{"id":"78519e65-2c2d-4424-83a9-9dbd90af22d8","type":"BasicTicker"}},"id":"44a176de-89c2-42ad-8f87-ee0986553174","type":"LinearAxis"},{"attributes":{"label":{"value":"1us"},"renderers":[{"id":"7d8cbf49-f0bf-4739-979a-7f055e17f277","type":"GlyphRenderer"}]},"id":"016140c0-733a-42e7-a6fd-9596879785b7","type":"LegendItem"},{"attributes":{},"id":"9765a573-ce07-4dac-8d25-f5c756725fe1","type":"BasicTicker"},{"attributes":{"label":{"value":"100ms"},"renderers":[{"id":"45795d30-20cf-421a-90d5-552a1df6c804","type":"GlyphRenderer"}]},"id":"c873e202-11e2-4533-9862-daf563b91c32","type":"LegendItem"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"7f499213-0134-4c5c-9235-126158ca7eed","type":"Line"},{"attributes":{"dimension":1,"plot":{"id":"41999ae5-ee50-4180-8a07-fdec3748db8d","subtype":"Figure","type":"Plot"},"ticker":{"id":"9765a573-ce07-4dac-8d25-f5c756725fe1","type":"BasicTicker"}},"id":"89626384-979e-4411-9ae9-1fc932204cb8","type":"Grid"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"q5SRkbXHk0C5OgbIBTCbQNOi7PG6oZ9AYTvcNx2BokB7bWhsEu6mQOBg60Yg8aVA9irH/+SCpUCs8pL575ilQPIu2GiALKRA","dtype":"float64","shape":[9]}}},"id":"6a885f9a-7765-41fa-af8a-dc7292f329df","type":"ColumnDataSource"},{"attributes":{"callback":null,"tabs":[{"id":"46d7408d-edb4-4a99-a135-415f0feaf6aa","type":"Panel"},{"id":"f8aa2e3b-14d5-4851-9c16-0c58862b0b9c","type":"Panel"}]},"id":"c0b81eb9-68c0-4b71-926b-0b5bcab775b1","type":"Tabs"},{"attributes":{"callback":null,"start":0},"id":"8e2a6dae-41dd-4920-8ff6-8780ed089f0a","type":"DataRange1d"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"1171b53b-8f58-4cb0-9e73-ba2526025d8e","type":"Line"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"9ffd45a0-96fc-4cdc-87b8-5f0d6ce79da3","type":"Circle"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"464981ae-ca40-475a-b516-bdbb6d4ec644","type":"Circle"},{"attributes":{"data_source":{"id":"dd527476-61b0-4a8a-b9d2-567373d3b727","type":"ColumnDataSource"},"glyph":{"id":"1171b53b-8f58-4cb0-9e73-ba2526025d8e","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"9f47b1ec-bc60-4e62-9f08-ef21fbf4782d","type":"Line"},"selection_glyph":null},"id":"a2e3d51d-d8bb-41ac-8515-3f1154af5d97","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"0ed2e546-c35d-4985-b91e-c9ce4a99aed8","type":"ColumnDataSource"},"glyph":{"id":"9ffd45a0-96fc-4cdc-87b8-5f0d6ce79da3","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"9bf667be-10d2-4824-837d-262ff7baacab","type":"Circle"},"selection_glyph":null},"id":"f2343a90-0f7a-4366-8d1f-79df80822bfe","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"7537baba-bde5-45e0-9cb4-eabbcb1b539c","type":"ColumnDataSource"},"glyph":{"id":"71ac58c9-55fe-4631-b9ec-79f55dccf3f8","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"a8d36c07-1d97-49e1-9d58-015a82fc942a","type":"Line"},"selection_glyph":null},"id":"47190973-cc0e-4e9f-9e83-1c6099856fa6","type":"GlyphRenderer"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,12428.158608007856]}},"id":"1e2b523e-fc0c-4a27-821f-49d321415b62","type":"ColumnDataSource"},{"attributes":{"child":{"id":"41999ae5-ee50-4180-8a07-fdec3748db8d","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"a1bcbfdd-7db0-4ae7-9ce5-61153092ac99","type":"Panel"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"9bf667be-10d2-4824-837d-262ff7baacab","type":"Circle"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"f7621818-2e3f-424e-b6ba-eacd12cd13ce","type":"Circle"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"85719b8d-547b-42d8-a67b-dea193b89ff1","type":"BoxAnnotation"},{"attributes":{"label":{"value":"100ms"},"renderers":[{"id":"0ea10786-847d-49d5-b396-c435c2c06fdb","type":"GlyphRenderer"}]},"id":"d82896f9-20c8-4dde-9f78-84e02a60992d","type":"LegendItem"},{"attributes":{},"id":"ccca75ae-9a90-470d-8341-41cbc1494d9f","type":"ToolEvents"},{"attributes":{"data_source":{"id":"21c6a951-806f-4a05-862c-8b9c54656803","type":"ColumnDataSource"},"glyph":{"id":"464981ae-ca40-475a-b516-bdbb6d4ec644","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"f7621818-2e3f-424e-b6ba-eacd12cd13ce","type":"Circle"},"selection_glyph":null},"id":"80a40327-c430-40e1-8f2c-dac700777eb2","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"41999ae5-ee50-4180-8a07-fdec3748db8d","subtype":"Figure","type":"Plot"}},"id":"0b91a400-e7e6-479c-ab8d-dd80448d3b0e","type":"PanTool"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"b6b35bed-a3cf-4ff2-94af-733dab6ed569","type":"Line"},{"attributes":{"plot":{"id":"4fab0e63-76cb-46a6-843e-af38f25fd676","subtype":"Figure","type":"Plot"}},"id":"d5103372-bb6a-4b74-87c3-e3cc397eddc7","type":"PanTool"},{"attributes":{"plot":{"id":"4fab0e63-76cb-46a6-843e-af38f25fd676","subtype":"Figure","type":"Plot"}},"id":"d17ca258-ae85-49ed-873d-53dfb55bdf92","type":"WheelZoomTool"},{"attributes":{"plot":{"id":"e761952d-458d-4815-b822-4649d8dc2644","subtype":"Figure","type":"Plot"}},"id":"ee40caee-98db-40f4-b0db-7656427ffd03","type":"ResetTool"},{"attributes":{"data_source":{"id":"7b03661a-da30-48ae-baf6-c203e36ab1c9","type":"ColumnDataSource"},"glyph":{"id":"46f9230c-1513-48e6-85cc-89e281677acd","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"dcddc97a-a121-4424-bba4-06e32f3ffef4","type":"Line"},"selection_glyph":null},"id":"199b004a-feec-4a2b-a46f-04133c01e7f4","type":"GlyphRenderer"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"d3712cc3-12bb-434e-9368-dcb219e339ed","type":"Circle"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"LbJ3pRAWX0BbUHF3/95iQIiYAtArN3BAAzvcglmDd0A7RNmNS3J/QPRV3VjCgItAGs1hBVAyk0A6pgUWMmqYQM/xIEy8hpZA","dtype":"float64","shape":[9]}}},"id":"eea30ed3-c5f9-448f-b8f6-2e5a5fa90822","type":"ColumnDataSource"},{"attributes":{"below":[{"id":"976239da-0767-49f8-b67b-b3f818503085","type":"LogAxis"}],"left":[{"id":"a35de6cf-b211-46f4-88a4-9dbd93884729","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"976239da-0767-49f8-b67b-b3f818503085","type":"LogAxis"},{"id":"eea18fa0-c581-444d-9b1c-2ad38ec878ba","type":"Grid"},{"id":"a35de6cf-b211-46f4-88a4-9dbd93884729","type":"LogAxis"},{"id":"64a92885-1953-4217-ae30-fc73634b3d3a","type":"Grid"},{"id":"ab10fe90-cae8-447a-af95-b3c5e014c4af","type":"BoxAnnotation"},{"id":"e17a2f1c-37b7-4731-9b71-3c5f5de1b30e","type":"Legend"},{"id":"69263cca-41e2-48f2-8397-a6f5bd0b9575","type":"GlyphRenderer"},{"id":"24b55eb1-0e53-4069-b743-6ac9c23fb075","type":"GlyphRenderer"},{"id":"199b004a-feec-4a2b-a46f-04133c01e7f4","type":"GlyphRenderer"},{"id":"1396869b-480b-4403-bbc4-7a8c68f073fc","type":"GlyphRenderer"},{"id":"f70cce5b-2d5f-4a24-a2bc-810b27a6590b","type":"GlyphRenderer"},{"id":"721e5bec-22ab-43ef-abf1-4dc8664c0b20","type":"GlyphRenderer"}],"title":{"id":"ffd420f2-8ace-48ac-8eab-0093cbfa7d9d","type":"Title"},"tool_events":{"id":"2301c8d7-0a31-45fa-8aae-799e249007b9","type":"ToolEvents"},"toolbar":{"id":"aa3a240b-2cda-4618-9428-3dd1c9a81388","type":"Toolbar"},"x_range":{"id":"4e708514-46ec-4b98-946d-1599deda79a7","type":"DataRange1d"},"x_scale":{"id":"ba21b478-91dd-40c6-9b8d-8f7eb1b61ee9","type":"LogScale"},"y_range":{"id":"5b78f537-da57-49d6-b94c-4f6ceaf00b8c","type":"DataRange1d"},"y_scale":{"id":"235f9377-d769-48bc-b536-85c4458f48aa","type":"LogScale"}},"id":"1d34c2b9-9f2b-48c4-b9c6-c2c8e72691c5","subtype":"Figure","type":"Plot"},{"attributes":{"ticker":null},"id":"7bbeb255-750d-4f2b-834b-0d208c1f918a","type":"LogTickFormatter"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"fd037a13-f03b-4366-a086-1cc9bfd29a68","type":"Line"},{"attributes":{"label":{"value":"100ms"},"renderers":[{"id":"69263cca-41e2-48f2-8397-a6f5bd0b9575","type":"GlyphRenderer"}]},"id":"9ec899a7-17d1-4036-b04b-f8571d6c4301","type":"LegendItem"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"PhAgSQGEM0BmSs6U6HdDQAwo6ea6PVNAoNqNGAnLYkDbj4LPiEVyQN1kKI9W54BAZTy4mhPIjED7LEuHcqKWQMQ4SLgPgZ5A","dtype":"float64","shape":[9]}}},"id":"b0869052-3aa1-458f-84c4-0ad2af85c8a4","type":"ColumnDataSource"},{"attributes":{},"id":"8b140c20-6d76-44f3-91e6-e42463f894b7","type":"LogScale"},{"attributes":{"ticker":null},"id":"fdbaaf97-6b9a-4d53-9fdb-04191c08b390","type":"LogTickFormatter"},{"attributes":{"plot":null,"text":"Arrays: Map Overlap"},"id":"ffd420f2-8ace-48ac-8eab-0093cbfa7d9d","type":"Title"},{"attributes":{"items":[{"id":"72fc77ae-e398-4f34-aa4c-24de5ab25546","type":"LegendItem"},{"id":"f9533df3-a6f2-407a-a408-6a557236dd27","type":"LegendItem"},{"id":"0cb38f80-7295-488c-94f0-6da0bc31d650","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"9ddf2da6-70f9-4a0c-bd6d-3969e84b1b1a","subtype":"Figure","type":"Plot"}},"id":"ed7d6a7d-f1ff-4745-8068-712ebf456009","type":"Legend"},{"attributes":{"label":{"value":"1us"},"renderers":[{"id":"4a334960-fb43-4949-8e13-5cc44f095565","type":"GlyphRenderer"}]},"id":"2cd69d4e-ce9d-4ffe-a8da-73b9a4a76f70","type":"LegendItem"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"w1YWXnOt/z84PQvNaIsPQIhEPP1NRB9ArYqH3f3CLkDSt1Z9g8U9QH5TAs78FU1AkemwSLPlVkAELKWLNq1mQC1uH4c19HBA","dtype":"float64","shape":[9]}}},"id":"8b74d8e5-a13e-4955-bfca-bcab0e2df8a7","type":"ColumnDataSource"},{"attributes":{"below":[{"id":"62620c35-fbf6-4b85-83ff-8772227ebb77","type":"LogAxis"}],"left":[{"id":"6a3c8e27-4a29-4f03-8bdd-09b6029954d4","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"62620c35-fbf6-4b85-83ff-8772227ebb77","type":"LogAxis"},{"id":"62848c21-8c52-4034-857a-5e05d13ccf58","type":"Grid"},{"id":"6a3c8e27-4a29-4f03-8bdd-09b6029954d4","type":"LogAxis"},{"id":"87babc94-0e03-40e1-8659-332b33b52fb3","type":"Grid"},{"id":"56067dd7-a2b6-4ff7-8cd3-f8c1cc51c25b","type":"BoxAnnotation"},{"id":"ed7d6a7d-f1ff-4745-8068-712ebf456009","type":"Legend"},{"id":"a5327d44-b21a-412c-8c94-f9363f357d96","type":"GlyphRenderer"},{"id":"72ff53d1-eae1-44b0-9e17-1576c8ed8435","type":"GlyphRenderer"},{"id":"a6a7460d-9127-40a4-a3b5-041c6162c91d","type":"GlyphRenderer"},{"id":"a6539ac4-c610-470f-a3a3-c918c3b7e923","type":"GlyphRenderer"},{"id":"494cd700-056c-434d-bb7e-5fd639a6400e","type":"GlyphRenderer"},{"id":"ace2b496-86ba-4ad5-a5b7-25ca170cf0ff","type":"GlyphRenderer"},{"id":"7ab82ebf-e918-475c-bc9e-b885ae95fd38","type":"GlyphRenderer"},{"id":"60b83405-6ab4-4cc9-a41c-02803e9efe91","type":"GlyphRenderer"},{"id":"abc7b321-64df-4b9f-af6b-1c535dfa2c81","type":"GlyphRenderer"}],"title":{"id":"671592e4-7a2a-4c2d-bd41-5dbb5497a950","type":"Title"},"tool_events":{"id":"63fd8daa-6710-4f4f-804d-d448df9409b4","type":"ToolEvents"},"toolbar":{"id":"c496e8c6-9922-4131-a4ab-3b56e9a8735d","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"42cfc0dd-2373-4fce-86da-8246e3defb5f","type":"DataRange1d"},"x_scale":{"id":"17f181e3-d864-45a9-812a-e4f38a97a5f2","type":"LogScale"},"y_range":{"id":"93c9efee-2572-41c6-86a1-e2b5871b2dee","type":"DataRange1d"},"y_scale":{"id":"b9184cb6-ecb5-470e-b07d-c48cf33cee83","type":"LogScale"}},"id":"9ddf2da6-70f9-4a0c-bd6d-3969e84b1b1a","subtype":"Figure","type":"Plot"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"bf1999e5-e42b-464b-9db0-ea3083c3c1d1","type":"Line"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"1c9d2576-92de-4aae-bc21-4ac781464cad","type":"Panel"},{"id":"e36df08d-6b52-45fd-90e8-47b2576ad722","type":"Panel"}]},"id":"59c15cdf-cad3-4afc-a12b-458d2a0a5b13","type":"Tabs"},{"attributes":{"plot":null,"text":"Tasks: Embarrassingly Parallel"},"id":"671592e4-7a2a-4c2d-bd41-5dbb5497a950","type":"Title"},{"attributes":{},"id":"63fd8daa-6710-4f4f-804d-d448df9409b4","type":"ToolEvents"},{"attributes":{"dimension":1,"plot":{"id":"9ddf2da6-70f9-4a0c-bd6d-3969e84b1b1a","subtype":"Figure","type":"Plot"},"ticker":{"id":"3f8e6358-0297-4cf0-a05a-0078a8ee58db","type":"LogTicker"}},"id":"87babc94-0e03-40e1-8659-332b33b52fb3","type":"Grid"},{"attributes":{},"id":"2301c8d7-0a31-45fa-8aae-799e249007b9","type":"ToolEvents"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"56067dd7-a2b6-4ff7-8cd3-f8c1cc51c25b","type":"BoxAnnotation"},{"attributes":{"callback":null,"start":0},"id":"42cfc0dd-2373-4fce-86da-8246e3defb5f","type":"DataRange1d"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"73c8ebe1-fa92-4ff6-ac89-ed9778c71129","type":"PanTool"},{"id":"256f01fa-c5a6-4e2d-9723-781db6e85fc8","type":"WheelZoomTool"},{"id":"36eac939-56b7-4b14-8629-bde3b0ed0b72","type":"BoxZoomTool"},{"id":"c3009076-d910-41e9-9d89-8c3229a09dcb","type":"SaveTool"},{"id":"a900c3fe-2975-4b0c-bd7b-39871891269e","type":"ResetTool"},{"id":"8c3dcf24-6388-41fd-a1b4-b7d803c9a77b","type":"HelpTool"}]},"id":"c496e8c6-9922-4131-a4ab-3b56e9a8735d","type":"Toolbar"},{"attributes":{"axis_label":"cores","formatter":{"id":"fdbaaf97-6b9a-4d53-9fdb-04191c08b390","type":"LogTickFormatter"},"plot":{"id":"9ddf2da6-70f9-4a0c-bd6d-3969e84b1b1a","subtype":"Figure","type":"Plot"},"ticker":{"id":"78faa5a9-3f86-4363-9f9e-f3ec7d2e2979","type":"FixedTicker"}},"id":"62620c35-fbf6-4b85-83ff-8772227ebb77","type":"LogAxis"},{"attributes":{"num_minor_ticks":10},"id":"ac3b4e3e-b8ae-4a9c-80e8-d7da53152181","type":"LogTicker"},{"attributes":{},"id":"17f181e3-d864-45a9-812a-e4f38a97a5f2","type":"LogScale"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"6d57a634-c93d-41d9-a9a6-7c9cea68c3d6","type":"PanTool"},{"id":"3db0df46-85de-46c4-a354-e601943c8be3","type":"WheelZoomTool"},{"id":"dd8d9f16-c2ba-4ca6-91e3-6cc6c3606729","type":"BoxZoomTool"},{"id":"3ae29d94-89c6-4214-a2f5-4405c02dfb8b","type":"SaveTool"},{"id":"ddb6422a-3fc9-47a5-885a-6a6dd8f99f1a","type":"ResetTool"},{"id":"33efd2d6-d965-428f-8c5c-efdcb84a3bc2","type":"HelpTool"}]},"id":"aa3a240b-2cda-4618-9428-3dd1c9a81388","type":"Toolbar"},{"attributes":{"callback":null,"end":3997.1079293993125,"start":0},"id":"93c9efee-2572-41c6-86a1-e2b5871b2dee","type":"DataRange1d"},{"attributes":{"num_minor_ticks":10},"id":"3f8e6358-0297-4cf0-a05a-0078a8ee58db","type":"LogTicker"},{"attributes":{"num_minor_ticks":10},"id":"1c284cec-7c05-4fec-be42-6fe7fecc5341","type":"LogTicker"},{"attributes":{"plot":{"id":"9ddf2da6-70f9-4a0c-bd6d-3969e84b1b1a","subtype":"Figure","type":"Plot"},"ticker":{"id":"ac3b4e3e-b8ae-4a9c-80e8-d7da53152181","type":"LogTicker"}},"id":"62848c21-8c52-4034-857a-5e05d13ccf58","type":"Grid"},{"attributes":{},"id":"b9184cb6-ecb5-470e-b07d-c48cf33cee83","type":"LogScale"},{"attributes":{"axis_label":"tasks/s","formatter":{"id":"cc6bc177-2234-4097-8df3-d61c3f4ee5dd","type":"LogTickFormatter"},"plot":{"id":"9ddf2da6-70f9-4a0c-bd6d-3969e84b1b1a","subtype":"Figure","type":"Plot"},"ticker":{"id":"3f8e6358-0297-4cf0-a05a-0078a8ee58db","type":"LogTicker"}},"id":"6a3c8e27-4a29-4f03-8bdd-09b6029954d4","type":"LogAxis"},{"attributes":{"child":{"id":"9ddf2da6-70f9-4a0c-bd6d-3969e84b1b1a","subtype":"Figure","type":"Plot"},"title":"log"},"id":"1c9d2576-92de-4aae-bc21-4ac781464cad","type":"Panel"},{"attributes":{"num_minor_ticks":10},"id":"a5f4fff5-9473-482a-b773-100a25273fa8","type":"LogTicker"},{"attributes":{"plot":{"id":"9ddf2da6-70f9-4a0c-bd6d-3969e84b1b1a","subtype":"Figure","type":"Plot"}},"id":"c3009076-d910-41e9-9d89-8c3229a09dcb","type":"SaveTool"},{"attributes":{"plot":{"id":"9ddf2da6-70f9-4a0c-bd6d-3969e84b1b1a","subtype":"Figure","type":"Plot"}},"id":"73c8ebe1-fa92-4ff6-ac89-ed9778c71129","type":"PanTool"},{"attributes":{"overlay":{"id":"56067dd7-a2b6-4ff7-8cd3-f8c1cc51c25b","type":"BoxAnnotation"},"plot":{"id":"9ddf2da6-70f9-4a0c-bd6d-3969e84b1b1a","subtype":"Figure","type":"Plot"}},"id":"36eac939-56b7-4b14-8629-bde3b0ed0b72","type":"BoxZoomTool"},{"attributes":{"callback":null,"end":4657.834162656733,"start":0},"id":"5b78f537-da57-49d6-b94c-4f6ceaf00b8c","type":"DataRange1d"},{"attributes":{"plot":{"id":"9ddf2da6-70f9-4a0c-bd6d-3969e84b1b1a","subtype":"Figure","type":"Plot"}},"id":"256f01fa-c5a6-4e2d-9723-781db6e85fc8","type":"WheelZoomTool"},{"attributes":{"plot":{"id":"9ddf2da6-70f9-4a0c-bd6d-3969e84b1b1a","subtype":"Figure","type":"Plot"}},"id":"a900c3fe-2975-4b0c-bd7b-39871891269e","type":"ResetTool"},{"attributes":{"plot":{"id":"9ddf2da6-70f9-4a0c-bd6d-3969e84b1b1a","subtype":"Figure","type":"Plot"}},"id":"8c3dcf24-6388-41fd-a1b4-b7d803c9a77b","type":"HelpTool"},{"attributes":{"ticker":null},"id":"cc6bc177-2234-4097-8df3-d61c3f4ee5dd","type":"LogTickFormatter"},{"attributes":{"data_source":{"id":"529ef473-0ccc-45f4-861a-7b66c8fc4d8d","type":"ColumnDataSource"},"glyph":{"id":"4b247019-ac32-4471-98a2-5ed1fc275453","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"7b38614f-09f7-4738-8ebe-089febc14dd8","type":"Line"},"selection_glyph":null},"id":"69263cca-41e2-48f2-8397-a6f5bd0b9575","type":"GlyphRenderer"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"fe27d651-f139-49a9-8553-62e6d681be99","type":"Circle"},{"attributes":{"label":{"value":"1s"},"renderers":[{"id":"a5327d44-b21a-412c-8c94-f9363f357d96","type":"GlyphRenderer"}]},"id":"72fc77ae-e398-4f34-aa4c-24de5ab25546","type":"LegendItem"},{"attributes":{"plot":{"id":"1d34c2b9-9f2b-48c4-b9c6-c2c8e72691c5","subtype":"Figure","type":"Plot"},"ticker":{"id":"1c284cec-7c05-4fec-be42-6fe7fecc5341","type":"LogTicker"}},"id":"eea18fa0-c581-444d-9b1c-2ad38ec878ba","type":"Grid"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"7abab916-ce13-45b2-b8d4-a4898c863064","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"w1YWXnOt/z84PQvNaIsPQIhEPP1NRB9ArYqH3f3CLkDSt1Z9g8U9QH5TAs78FU1AkemwSLPlVkAELKWLNq1mQC1uH4c19HBA","dtype":"float64","shape":[9]}}},"id":"798b04c0-7810-433c-9d10-e52806569ced","type":"ColumnDataSource"},{"attributes":{},"id":"ba21b478-91dd-40c6-9b8d-8f7eb1b61ee9","type":"LogScale"},{"attributes":{"axis_label":"cores","formatter":{"id":"7bbeb255-750d-4f2b-834b-0d208c1f918a","type":"LogTickFormatter"},"plot":{"id":"1d34c2b9-9f2b-48c4-b9c6-c2c8e72691c5","subtype":"Figure","type":"Plot"},"ticker":{"id":"ef5e1e26-3fce-47b3-8daf-d463705887ce","type":"FixedTicker"}},"id":"976239da-0767-49f8-b67b-b3f818503085","type":"LogAxis"},{"attributes":{},"id":"235f9377-d769-48bc-b536-85c4458f48aa","type":"LogScale"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"320ffe66-cf3a-4715-99c8-4331b3f8f028","type":"LogTickFormatter"},"plot":{"id":"1d34c2b9-9f2b-48c4-b9c6-c2c8e72691c5","subtype":"Figure","type":"Plot"},"ticker":{"id":"a5f4fff5-9473-482a-b773-100a25273fa8","type":"LogTicker"}},"id":"a35de6cf-b211-46f4-88a4-9dbd93884729","type":"LogAxis"},{"attributes":{"dimension":1,"plot":{"id":"1d34c2b9-9f2b-48c4-b9c6-c2c8e72691c5","subtype":"Figure","type":"Plot"},"ticker":{"id":"a5f4fff5-9473-482a-b773-100a25273fa8","type":"LogTicker"}},"id":"64a92885-1953-4217-ae30-fc73634b3d3a","type":"Grid"},{"attributes":{"ticker":null},"id":"320ffe66-cf3a-4715-99c8-4331b3f8f028","type":"LogTickFormatter"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"BSBNWCE2dEC9BpWUVImFQDzSCcCUkJBAVLgx0fyxnkCFGyLiy4+jQE7fanACZ6xAEeby5mMgsEDryZN7k+GwQFqGlput7rBA","dtype":"float64","shape":[9]}}},"id":"7537baba-bde5-45e0-9cb4-eabbcb1b539c","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"1d34c2b9-9f2b-48c4-b9c6-c2c8e72691c5","subtype":"Figure","type":"Plot"}},"id":"6d57a634-c93d-41d9-a9a6-7c9cea68c3d6","type":"PanTool"},{"attributes":{"plot":{"id":"1d34c2b9-9f2b-48c4-b9c6-c2c8e72691c5","subtype":"Figure","type":"Plot"}},"id":"3db0df46-85de-46c4-a354-e601943c8be3","type":"WheelZoomTool"},{"attributes":{"plot":{"id":"1d34c2b9-9f2b-48c4-b9c6-c2c8e72691c5","subtype":"Figure","type":"Plot"}},"id":"3ae29d94-89c6-4214-a2f5-4405c02dfb8b","type":"SaveTool"},{"attributes":{"data_source":{"id":"fa916c00-5834-4d8f-81b6-4b4f9d09e1e8","type":"ColumnDataSource"},"glyph":{"id":"4b506720-d38f-4e6f-8f8a-a33ca1b77473","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"66773eb1-a562-405d-af87-f22eb4215a62","type":"Line"},"selection_glyph":null},"id":"a88c23d3-43d3-4a29-a8c5-20a855f54ba0","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"2f758d81-8533-4e97-a53c-041f1db98fae","type":"Circle"},{"attributes":{"plot":{"id":"1d34c2b9-9f2b-48c4-b9c6-c2c8e72691c5","subtype":"Figure","type":"Plot"}},"id":"33efd2d6-d965-428f-8c5c-efdcb84a3bc2","type":"HelpTool"},{"attributes":{"data_source":{"id":"0ab80c92-5372-4728-a0da-e513a7ae8e2b","type":"ColumnDataSource"},"glyph":{"id":"84379b7b-549c-4cb0-ace2-a9135bc8555a","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"2f758d81-8533-4e97-a53c-041f1db98fae","type":"Circle"},"selection_glyph":null},"id":"557b9797-c849-4fd1-b818-da278a904030","type":"GlyphRenderer"},{"attributes":{},"id":"387a0733-5b81-4503-81a8-d7264a71bdb4","type":"LogScale"},{"attributes":{"axis_label":"cores","formatter":{"id":"2713700e-4b8e-48b0-b334-2cbddf0387da","type":"LogTickFormatter"},"plot":{"id":"3618c651-3cdd-4455-a504-ff10836a058a","subtype":"Figure","type":"Plot"},"ticker":{"id":"850d2f6c-e3b2-4ea7-b9cc-79d306793f43","type":"FixedTicker"}},"id":"b1a4b554-ebdb-4cfe-8816-e965d3551cb6","type":"LogAxis"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"4b506720-d38f-4e6f-8f8a-a33ca1b77473","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"66773eb1-a562-405d-af87-f22eb4215a62","type":"Line"},{"attributes":{"plot":{"id":"3618c651-3cdd-4455-a504-ff10836a058a","subtype":"Figure","type":"Plot"},"ticker":{"id":"7cc02c81-8117-4e1a-9f0d-a555c82109a7","type":"LogTicker"}},"id":"a43669eb-fdb1-4342-8acf-5b451bc715f8","type":"Grid"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"17a64fd8-de2d-413d-8fb7-7b0e2aca8c2f","type":"FixedTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,175286.75839479806]}},"id":"fa916c00-5834-4d8f-81b6-4b4f9d09e1e8","type":"ColumnDataSource"},{"attributes":{"num_minor_ticks":10},"id":"975992b0-97bd-4812-a84b-1e893068c778","type":"LogTicker"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"47619def-c558-4026-a503-09d9f7f9c819","type":"Panel"},{"id":"f2fff79f-5732-487e-9d7f-1275394f234a","type":"Panel"}]},"id":"d5449472-9151-45a2-82ef-636c1e6e7fad","type":"Tabs"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"65eaf9ce-00b9-49b4-97fb-bab4f40f5201","type":"Circle"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"4f55baa9-9c09-4246-8a1c-6c137081c1e9","type":"Line"},{"attributes":{"below":[{"id":"b1a4b554-ebdb-4cfe-8816-e965d3551cb6","type":"LogAxis"}],"left":[{"id":"646d0ce3-67d2-4c55-90dd-73081617b0b7","type":"LogAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"b1a4b554-ebdb-4cfe-8816-e965d3551cb6","type":"LogAxis"},{"id":"a43669eb-fdb1-4342-8acf-5b451bc715f8","type":"Grid"},{"id":"646d0ce3-67d2-4c55-90dd-73081617b0b7","type":"LogAxis"},{"id":"142a5ef0-45db-448e-bdf5-6bc1af40dbfd","type":"Grid"},{"id":"03993200-55da-4eb3-a474-14a6cc4b0285","type":"BoxAnnotation"},{"id":"5338ed5e-1297-491b-ab37-c351da40ae08","type":"Legend"},{"id":"cfa2bcc6-8adc-4e87-94e0-c2637340e2d9","type":"GlyphRenderer"},{"id":"2d85a19d-e6ef-45e9-9204-ca1ae2bb73da","type":"GlyphRenderer"},{"id":"5cf2b740-4cdb-4f37-9f1f-2727fd54866e","type":"GlyphRenderer"}],"title":{"id":"70275d6c-23e2-4ef7-81a4-fce074c48cf6","type":"Title"},"tool_events":{"id":"83201b83-a0a4-4eec-a296-5f52f8bbd13c","type":"ToolEvents"},"toolbar":{"id":"bcf7b40a-0310-4ba6-8071-f0219b20abff","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"ed5a7b95-48a4-4fec-ac8f-6f16eaf0c5d8","type":"DataRange1d"},"x_scale":{"id":"6c655239-a968-4c12-a022-1ddf3d599681","type":"LogScale"},"y_range":{"id":"2d463c30-1176-4f36-9bed-64d13295e7c1","type":"DataRange1d"},"y_scale":{"id":"387a0733-5b81-4503-81a8-d7264a71bdb4","type":"LogScale"}},"id":"3618c651-3cdd-4455-a504-ff10836a058a","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"6c655239-a968-4c12-a022-1ddf3d599681","type":"LogScale"},{"attributes":{"ticker":null},"id":"2713700e-4b8e-48b0-b334-2cbddf0387da","type":"LogTickFormatter"},{"attributes":{"callback":null,"end":2404.1774131328957,"start":0},"id":"2d463c30-1176-4f36-9bed-64d13295e7c1","type":"DataRange1d"},{"attributes":{"plot":null,"text":"Arrays: Elementwise"},"id":"70275d6c-23e2-4ef7-81a4-fce074c48cf6","type":"Title"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"8deb4c17-6422-4a63-a312-295c749663ba","type":"LogTickFormatter"},"plot":{"id":"3618c651-3cdd-4455-a504-ff10836a058a","subtype":"Figure","type":"Plot"},"ticker":{"id":"975992b0-97bd-4812-a84b-1e893068c778","type":"LogTicker"}},"id":"646d0ce3-67d2-4c55-90dd-73081617b0b7","type":"LogAxis"},{"attributes":{"num_minor_ticks":10},"id":"7cc02c81-8117-4e1a-9f0d-a555c82109a7","type":"LogTicker"},{"attributes":{"dimension":1,"plot":{"id":"3618c651-3cdd-4455-a504-ff10836a058a","subtype":"Figure","type":"Plot"},"ticker":{"id":"975992b0-97bd-4812-a84b-1e893068c778","type":"LogTicker"}},"id":"142a5ef0-45db-448e-bdf5-6bc1af40dbfd","type":"Grid"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"b5818651-f465-4a4e-b56d-d88fccd70330","type":"Line"},{"attributes":{"data_source":{"id":"97e682ba-d0e2-401f-98d3-0d4f783335b1","type":"ColumnDataSource"},"glyph":{"id":"811b80b5-39df-4ff0-a1a8-9e8a324990cf","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"e14090af-ea5a-42b8-8321-42ea86824188","type":"Line"},"selection_glyph":null},"id":"721e5bec-22ab-43ef-abf1-4dc8664c0b20","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"672963d6-239d-4bac-ae4f-fa0f80c2c98f","type":"ColumnDataSource"},"glyph":{"id":"b5818651-f465-4a4e-b56d-d88fccd70330","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"4f55baa9-9c09-4246-8a1c-6c137081c1e9","type":"Line"},"selection_glyph":null},"id":"cfa2bcc6-8adc-4e87-94e0-c2637340e2d9","type":"GlyphRenderer"},{"attributes":{"child":{"id":"3618c651-3cdd-4455-a504-ff10836a058a","subtype":"Figure","type":"Plot"},"title":"log"},"id":"47619def-c558-4026-a503-09d9f7f9c819","type":"Panel"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"03993200-55da-4eb3-a474-14a6cc4b0285","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"3618c651-3cdd-4455-a504-ff10836a058a","subtype":"Figure","type":"Plot"}},"id":"f240d2c0-ff6c-491a-ab4e-61eae5d526ed","type":"PanTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"epLhQ0J1YkAd5AFIZzFxQJg0vPmU7ntAyXEzRLI4iUCSgRntGWCSQEF51ldnFptA8ymMPoR+nkBOxBAU3zWiQFbn5NVayKJA","dtype":"float64","shape":[9]}}},"id":"e97fa81b-6cd9-47c7-a0cb-4d5e48994294","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"3618c651-3cdd-4455-a504-ff10836a058a","subtype":"Figure","type":"Plot"}},"id":"e2ce7b1b-cf07-411c-b287-d1b33ecf17f9","type":"WheelZoomTool"},{"attributes":{"overlay":{"id":"03993200-55da-4eb3-a474-14a6cc4b0285","type":"BoxAnnotation"},"plot":{"id":"3618c651-3cdd-4455-a504-ff10836a058a","subtype":"Figure","type":"Plot"}},"id":"d9cf7207-525f-49a5-9224-035cd6dfc2dc","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"3618c651-3cdd-4455-a504-ff10836a058a","subtype":"Figure","type":"Plot"}},"id":"e4a0baa0-5ecc-4689-a4f4-6ee954c27052","type":"SaveTool"},{"attributes":{"plot":{"id":"3618c651-3cdd-4455-a504-ff10836a058a","subtype":"Figure","type":"Plot"}},"id":"58423d76-834c-42ce-9dc5-b8729ce97321","type":"ResetTool"},{"attributes":{"plot":{"id":"3618c651-3cdd-4455-a504-ff10836a058a","subtype":"Figure","type":"Plot"}},"id":"9b541db3-5f39-403a-ac85-bd080ebaa38a","type":"HelpTool"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"dcddc97a-a121-4424-bba4-06e32f3ffef4","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"epLhQ0J1YkAd5AFIZzFxQJg0vPmU7ntAyXEzRLI4iUCSgRntGWCSQEF51ldnFptA8ymMPoR+nkBOxBAU3zWiQFbn5NVayKJA","dtype":"float64","shape":[9]}}},"id":"672963d6-239d-4bac-ae4f-fa0f80c2c98f","type":"ColumnDataSource"},{"attributes":{},"id":"fc5d57cc-4be5-470d-818c-b8e1c88d4b27","type":"BasicTicker"},{"attributes":{"ticker":null},"id":"8deb4c17-6422-4a63-a312-295c749663ba","type":"LogTickFormatter"},{"attributes":{"axis_label":"cores","formatter":{"id":"2c2c32d2-a2d8-44ea-8cac-2f8ac31adc31","type":"BasicTickFormatter"},"plot":{"id":"3f1de3ec-80fb-4cbf-bbd3-870be4431819","subtype":"Figure","type":"Plot"},"ticker":{"id":"5fac156b-7f91-4578-b83b-8858f80abfd0","type":"FixedTicker"}},"id":"00df0b68-7f3b-4b44-86e3-d993159cf3fd","type":"LinearAxis"},{"attributes":{"items":[{"id":"8fba1edc-7ad8-42a7-824e-3ebfd935345d","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"3618c651-3cdd-4455-a504-ff10836a058a","subtype":"Figure","type":"Plot"}},"id":"5338ed5e-1297-491b-ab37-c351da40ae08","type":"Legend"},{"attributes":{"plot":{"id":"3f1de3ec-80fb-4cbf-bbd3-870be4431819","subtype":"Figure","type":"Plot"},"ticker":{"id":"fc5d57cc-4be5-470d-818c-b8e1c88d4b27","type":"BasicTicker"}},"id":"11f328a6-a429-4bc9-bfc3-160584c1307f","type":"Grid"},{"attributes":{"label":{"value":"sin(x)**2 + cos(x)**2"},"renderers":[{"id":"cfa2bcc6-8adc-4e87-94e0-c2637340e2d9","type":"GlyphRenderer"}]},"id":"8fba1edc-7ad8-42a7-824e-3ebfd935345d","type":"LegendItem"},{"attributes":{"data_source":{"id":"1baae952-180c-493c-8d5d-14fe7bb1c1a5","type":"ColumnDataSource"},"glyph":{"id":"56955a48-ec14-45e7-8d88-9059d9102d6a","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"c8f397d9-ee85-4e07-9a6c-f07e5a82b8c3","type":"Line"},"selection_glyph":null},"id":"5cf2b740-4cdb-4f37-9f1f-2727fd54866e","type":"GlyphRenderer"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"f8f40c92-11f2-4003-84c4-6b69f89e39ee","type":"Circle"},{"attributes":{"data_source":{"id":"e97fa81b-6cd9-47c7-a0cb-4d5e48994294","type":"ColumnDataSource"},"glyph":{"id":"f8f40c92-11f2-4003-84c4-6b69f89e39ee","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"90f18ef6-70ef-436c-a31c-cd3ad942f1db","type":"Circle"},"selection_glyph":null},"id":"2d85a19d-e6ef-45e9-9204-ca1ae2bb73da","type":"GlyphRenderer"},{"attributes":{"plot":null,"text":"Arrays: Map Overlap"},"id":"bb55573e-1fd0-4f35-8357-230d93a1d391","type":"Title"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"90f18ef6-70ef-436c-a31c-cd3ad942f1db","type":"Circle"},{"attributes":{},"id":"10a1a392-095c-4fb2-9881-8a51553cf66c","type":"LinearScale"},{"attributes":{"callback":null,"end":2404.1774131328957,"start":0},"id":"10bd0dfa-c230-40e0-834f-357bfe3b11cd","type":"DataRange1d"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"56955a48-ec14-45e7-8d88-9059d9102d6a","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"c8f397d9-ee85-4e07-9a6c-f07e5a82b8c3","type":"Line"},{"attributes":{"plot":null,"text":"Arrays: Elementwise"},"id":"d5492012-a955-4369-a877-7ec4bd84d33a","type":"Title"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"850d2f6c-e3b2-4ea7-b9cc-79d306793f43","type":"FixedTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,37802.07078627213]}},"id":"1baae952-180c-493c-8d5d-14fe7bb1c1a5","type":"ColumnDataSource"},{"attributes":{"below":[{"id":"00df0b68-7f3b-4b44-86e3-d993159cf3fd","type":"LinearAxis"}],"left":[{"id":"9fbbdea7-a42d-4d77-9a04-ee5e721ac1fc","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"00df0b68-7f3b-4b44-86e3-d993159cf3fd","type":"LinearAxis"},{"id":"11f328a6-a429-4bc9-bfc3-160584c1307f","type":"Grid"},{"id":"9fbbdea7-a42d-4d77-9a04-ee5e721ac1fc","type":"LinearAxis"},{"id":"ff0cd646-1110-4ba4-abdd-16a97ab481e5","type":"Grid"},{"id":"34632c3e-798e-4c84-b4f3-96c4a5e0d99c","type":"BoxAnnotation"},{"id":"e7cd8d6c-fd1b-47c5-9046-bb0dd0d37454","type":"Legend"},{"id":"f7346b9b-f65b-4d3c-9113-394fac1afe19","type":"GlyphRenderer"},{"id":"17c349ae-33d2-40cd-9f5b-e7687a90507d","type":"GlyphRenderer"},{"id":"9318cc6d-b0c6-40cb-b3dc-d1660534ca98","type":"GlyphRenderer"}],"title":{"id":"d5492012-a955-4369-a877-7ec4bd84d33a","type":"Title"},"tool_events":{"id":"c644633b-1916-42d1-b95e-2a8e1991afc8","type":"ToolEvents"},"toolbar":{"id":"11e3f717-a81d-4f86-8cd5-25b1f8676304","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"ad04e308-9865-49e5-b635-493d64bd0cbf","type":"DataRange1d"},"x_scale":{"id":"adf71b1d-b8fc-4cc6-bc1f-70762e963cda","type":"LinearScale"},"y_range":{"id":"10bd0dfa-c230-40e0-834f-357bfe3b11cd","type":"DataRange1d"},"y_scale":{"id":"10a1a392-095c-4fb2-9881-8a51553cf66c","type":"LinearScale"}},"id":"3f1de3ec-80fb-4cbf-bbd3-870be4431819","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"c644633b-1916-42d1-b95e-2a8e1991afc8","type":"ToolEvents"},{"attributes":{"label":{"value":"1us"},"renderers":[{"id":"199b004a-feec-4a2b-a46f-04133c01e7f4","type":"GlyphRenderer"}]},"id":"51b3fff7-3940-4500-9667-260f70667185","type":"LegendItem"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"epLhQ0J1YkAd5AFIZzFxQJg0vPmU7ntAyXEzRLI4iUCSgRntGWCSQEF51ldnFptA8ymMPoR+nkBOxBAU3zWiQFbn5NVayKJA","dtype":"float64","shape":[9]}}},"id":"cc8b7364-0e31-40d1-9f4b-287a10a07cfc","type":"ColumnDataSource"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"ee8f3c74-9606-467b-9649-496ff3790d92","type":"PanTool"},{"id":"20be3e8f-d733-4775-88b3-846d52aa9ab7","type":"WheelZoomTool"},{"id":"157c4d22-7e1e-4cb9-b5de-51771a77c65b","type":"BoxZoomTool"},{"id":"1d85faba-8a6b-4095-961d-e7b90412bc93","type":"SaveTool"},{"id":"c39730d2-c985-4f53-97f8-e14e1accf06e","type":"ResetTool"},{"id":"b5fe7351-f4ac-4095-818c-83276790a403","type":"HelpTool"}]},"id":"11e3f717-a81d-4f86-8cd5-25b1f8676304","type":"Toolbar"},{"attributes":{"callback":null,"start":0},"id":"ad04e308-9865-49e5-b635-493d64bd0cbf","type":"DataRange1d"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"aac4d0f9-afb8-4aec-a718-d811d8bde39c","type":"Line"},{"attributes":{},"id":"adf71b1d-b8fc-4cc6-bc1f-70762e963cda","type":"LinearScale"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"0abf6ccc-6de3-47eb-bbb9-5d656a49d2f8","type":"BasicTickFormatter"},"plot":{"id":"3f1de3ec-80fb-4cbf-bbd3-870be4431819","subtype":"Figure","type":"Plot"},"ticker":{"id":"52e3bb9d-b9b8-45fe-94f5-2607cefb09a4","type":"BasicTicker"}},"id":"9fbbdea7-a42d-4d77-9a04-ee5e721ac1fc","type":"LinearAxis"},{"attributes":{},"id":"52e3bb9d-b9b8-45fe-94f5-2607cefb09a4","type":"BasicTicker"},{"attributes":{"dimension":1,"plot":{"id":"3f1de3ec-80fb-4cbf-bbd3-870be4431819","subtype":"Figure","type":"Plot"},"ticker":{"id":"52e3bb9d-b9b8-45fe-94f5-2607cefb09a4","type":"BasicTicker"}},"id":"ff0cd646-1110-4ba4-abdd-16a97ab481e5","type":"Grid"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"b446cf6e-495f-4ff7-afa8-02afa39b6a6d","type":"Line"},{"attributes":{"data_source":{"id":"cc8b7364-0e31-40d1-9f4b-287a10a07cfc","type":"ColumnDataSource"},"glyph":{"id":"b446cf6e-495f-4ff7-afa8-02afa39b6a6d","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"aac4d0f9-afb8-4aec-a718-d811d8bde39c","type":"Line"},"selection_glyph":null},"id":"f7346b9b-f65b-4d3c-9113-394fac1afe19","type":"GlyphRenderer"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"ef5e1e26-3fce-47b3-8daf-d463705887ce","type":"FixedTicker"},{"attributes":{"child":{"id":"3f1de3ec-80fb-4cbf-bbd3-870be4431819","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"f2fff79f-5732-487e-9d7f-1275394f234a","type":"Panel"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"34632c3e-798e-4c84-b4f3-96c4a5e0d99c","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"3f1de3ec-80fb-4cbf-bbd3-870be4431819","subtype":"Figure","type":"Plot"}},"id":"ee8f3c74-9606-467b-9649-496ff3790d92","type":"PanTool"},{"attributes":{},"id":"2c2c32d2-a2d8-44ea-8cac-2f8ac31adc31","type":"BasicTickFormatter"},{"attributes":{"plot":{"id":"3f1de3ec-80fb-4cbf-bbd3-870be4431819","subtype":"Figure","type":"Plot"}},"id":"20be3e8f-d733-4775-88b3-846d52aa9ab7","type":"WheelZoomTool"},{"attributes":{"overlay":{"id":"34632c3e-798e-4c84-b4f3-96c4a5e0d99c","type":"BoxAnnotation"},"plot":{"id":"3f1de3ec-80fb-4cbf-bbd3-870be4431819","subtype":"Figure","type":"Plot"}},"id":"157c4d22-7e1e-4cb9-b5de-51771a77c65b","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"3f1de3ec-80fb-4cbf-bbd3-870be4431819","subtype":"Figure","type":"Plot"}},"id":"1d85faba-8a6b-4095-961d-e7b90412bc93","type":"SaveTool"},{"attributes":{"plot":{"id":"3f1de3ec-80fb-4cbf-bbd3-870be4431819","subtype":"Figure","type":"Plot"}},"id":"c39730d2-c985-4f53-97f8-e14e1accf06e","type":"ResetTool"},{"attributes":{"plot":{"id":"3f1de3ec-80fb-4cbf-bbd3-870be4431819","subtype":"Figure","type":"Plot"}},"id":"b5fe7351-f4ac-4095-818c-83276790a403","type":"HelpTool"},{"attributes":{"callback":null,"start":0},"id":"b914e7a2-f1e0-4071-a0c2-d52db778eb69","type":"DataRange1d"},{"attributes":{"data_source":{"id":"368ebbd7-bae3-4d68-8617-71a85eab8635","type":"ColumnDataSource"},"glyph":{"id":"f891b164-8ac4-4312-9ada-fe4e94977263","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"1cb57ebf-bed8-4db8-b71f-77aa1312af5c","type":"Line"},"selection_glyph":null},"id":"f70cce5b-2d5f-4a24-a2bc-810b27a6590b","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"db943d83-03cd-4fea-b185-6dd6ef1b2660","type":"ColumnDataSource"},"glyph":{"id":"a585a097-5244-44ed-bb7d-624ca56c71b8","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"b67e2512-2857-4f42-98a2-f65fdcc2066c","type":"Circle"},"selection_glyph":null},"id":"1396869b-480b-4403-bbc4-7a8c68f073fc","type":"GlyphRenderer"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"811b80b5-39df-4ff0-a1a8-9e8a324990cf","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"1cb57ebf-bed8-4db8-b71f-77aa1312af5c","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"Gmr7TkyPlEBRf7wkoNCfQH9VJgZniqdALjvyvyi0rECvSL87J26wQO0MDDGQELJANhKvi9UxskBz0zc+QhSyQD61nBkBcrFA","dtype":"float64","shape":[9]}}},"id":"540ea8dd-e79f-4d23-8d1d-2260fd9c9cc4","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"d49fa646-bb9d-4704-9105-469647c7e6e2","subtype":"Figure","type":"Plot"}},"id":"e1966624-3bba-4c3a-86cf-1106a6800dc3","type":"ResetTool"},{"attributes":{"below":[{"id":"42bbee2c-121b-411e-9428-94c80306a923","type":"LinearAxis"}],"left":[{"id":"44a176de-89c2-42ad-8f87-ee0986553174","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"42bbee2c-121b-411e-9428-94c80306a923","type":"LinearAxis"},{"id":"96443dd1-a71d-4562-bb9c-d7381e117c4b","type":"Grid"},{"id":"44a176de-89c2-42ad-8f87-ee0986553174","type":"LinearAxis"},{"id":"77e0942d-9264-4240-8e68-8926967393ba","type":"Grid"},{"id":"16d3717d-7fcf-4009-b087-43fb39f20e1e","type":"BoxAnnotation"},{"id":"c5106ceb-af0f-41eb-a0a3-00686ab2f704","type":"Legend"},{"id":"47190973-cc0e-4e9f-9e83-1c6099856fa6","type":"GlyphRenderer"},{"id":"75535fe1-cdc3-44dd-af24-2d0e3adb77e1","type":"GlyphRenderer"},{"id":"0c8fbb3b-c3b4-497a-95d9-2952c6e6eeaf","type":"GlyphRenderer"},{"id":"0310db36-3f75-438c-be17-cdf7fd57fe79","type":"GlyphRenderer"},{"id":"ddc24851-3ca0-4a00-b6cf-df22004ede8c","type":"GlyphRenderer"},{"id":"6551e6ea-0811-4028-ad05-fef84dbb3abf","type":"GlyphRenderer"}],"title":{"id":"bb55573e-1fd0-4f35-8357-230d93a1d391","type":"Title"},"tool_events":{"id":"ccca75ae-9a90-470d-8341-41cbc1494d9f","type":"ToolEvents"},"toolbar":{"id":"ec2981f8-3c0a-461f-890b-8c6afd32553a","type":"Toolbar"},"x_range":{"id":"a0a586bf-9420-4eae-b3fe-b9400c461017","type":"DataRange1d"},"x_scale":{"id":"066b97e1-b17a-43b2-bdf7-56c504f64209","type":"LinearScale"},"y_range":{"id":"dd195aa8-2909-45be-a42f-97f0b117af1a","type":"DataRange1d"},"y_scale":{"id":"35297baa-7835-45c5-904a-1e2b7a405af7","type":"LinearScale"}},"id":"4fab0e63-76cb-46a6-843e-af38f25fd676","subtype":"Figure","type":"Plot"},{"attributes":{"callback":null,"end":4657.834162656733,"start":0},"id":"dd195aa8-2909-45be-a42f-97f0b117af1a","type":"DataRange1d"},{"attributes":{"plot":{"id":"4fab0e63-76cb-46a6-843e-af38f25fd676","subtype":"Figure","type":"Plot"},"ticker":{"id":"e2f877e7-d691-4e32-a8bd-8b34c2496036","type":"BasicTicker"}},"id":"96443dd1-a71d-4562-bb9c-d7381e117c4b","type":"Grid"},{"attributes":{},"id":"2309e548-4929-4624-83fb-040c36c8aeef","type":"LogScale"},{"attributes":{},"id":"006ec2d2-dc9c-403c-8195-2f79429db69e","type":"LogScale"},{"attributes":{"num_minor_ticks":10},"id":"2bb352e3-c945-4242-9341-6cead1c6ee1c","type":"LogTicker"},{"attributes":{"plot":{"id":"d49fa646-bb9d-4704-9105-469647c7e6e2","subtype":"Figure","type":"Plot"},"ticker":{"id":"c7f8c4c0-ccc9-4746-9876-e5a51098f8c8","type":"LogTicker"}},"id":"e9eccd09-1dff-4c05-b140-f16299e24d69","type":"Grid"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"487e0adf-1885-4d2f-bb37-d011d1e6f33c","type":"LogTickFormatter"},"plot":{"id":"d49fa646-bb9d-4704-9105-469647c7e6e2","subtype":"Figure","type":"Plot"},"ticker":{"id":"2bb352e3-c945-4242-9341-6cead1c6ee1c","type":"LogTicker"}},"id":"9063c47a-11f6-49e9-b286-6bc1acdaadc0","type":"LogAxis"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"38318980-d796-4290-a92b-136a4aab8577","type":"Line"},{"attributes":{},"id":"e2f877e7-d691-4e32-a8bd-8b34c2496036","type":"BasicTicker"},{"attributes":{"dimension":1,"plot":{"id":"d49fa646-bb9d-4704-9105-469647c7e6e2","subtype":"Figure","type":"Plot"},"ticker":{"id":"2bb352e3-c945-4242-9341-6cead1c6ee1c","type":"LogTicker"}},"id":"9f2da7a3-eee1-43b8-a823-2f05894d53d8","type":"Grid"},{"attributes":{"plot":{"id":"d49fa646-bb9d-4704-9105-469647c7e6e2","subtype":"Figure","type":"Plot"}},"id":"2b82eb62-e193-41f6-836a-f70e97ad6b5a","type":"PanTool"},{"attributes":{"data_source":{"id":"a7b16262-b813-4400-9275-3ff27790469e","type":"ColumnDataSource"},"glyph":{"id":"92abfb32-e7d6-4493-9b5b-d9fb88a34090","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"38318980-d796-4290-a92b-136a4aab8577","type":"Line"},"selection_glyph":null},"id":"5c85f6ef-b891-4ea1-9db3-e6ac8bdb4f0b","type":"GlyphRenderer"},{"attributes":{"ticker":null},"id":"11c0ae3b-3f61-48c8-8fd3-8f948edeb699","type":"LogTickFormatter"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"f6baaf11-ce1c-486c-b8d5-0c1632ec6adb","type":"BoxAnnotation"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"iDmAdOYzhED3jGZaeR+UQMzGF5CAyKNAqQZ49/jWsUDabo5ZahbBQPB5MuwHU9BAhZDfGJtW10Al2QgzKAzZQOndxaYoYuhA","dtype":"float64","shape":[9]}}},"id":"a7b16262-b813-4400-9275-3ff27790469e","type":"ColumnDataSource"},{"attributes":{"ticker":null},"id":"487e0adf-1885-4d2f-bb37-d011d1e6f33c","type":"LogTickFormatter"},{"attributes":{"overlay":{"id":"f6baaf11-ce1c-486c-b8d5-0c1632ec6adb","type":"BoxAnnotation"},"plot":{"id":"d49fa646-bb9d-4704-9105-469647c7e6e2","subtype":"Figure","type":"Plot"}},"id":"d7b8a053-c9f1-4fb7-bed4-a3df788f3ed2","type":"BoxZoomTool"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"76cf33d1-1336-4737-8cad-79fdcc322c10","type":"PanTool"},{"id":"f8834acc-9897-441d-9102-1f616408a50e","type":"WheelZoomTool"},{"id":"e7ac849f-5c88-4646-ae8c-6a4bc419f016","type":"BoxZoomTool"},{"id":"2c1a56f5-2c0d-4758-a7cc-eb45971629aa","type":"SaveTool"},{"id":"43e9c244-f5b8-4240-a21d-ba93dd66d881","type":"ResetTool"},{"id":"a0a42aa4-41ea-4099-a082-ff6b3e0b4400","type":"HelpTool"}]},"id":"eb086fb2-cda6-4992-9763-f203826c3f7e","type":"Toolbar"},{"attributes":{"plot":{"id":"d49fa646-bb9d-4704-9105-469647c7e6e2","subtype":"Figure","type":"Plot"}},"id":"c13542c5-c8f2-4b1d-a5ce-55f0dde4c758","type":"WheelZoomTool"},{"attributes":{"data_source":{"id":"3cf019ba-9b1f-42ed-b647-fc9001a82168","type":"ColumnDataSource"},"glyph":{"id":"eb4d7887-70ac-4a30-86f1-1f40e2b49fe4","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"7b5af66e-5b5f-4f69-9990-b49719f2e0b6","type":"Line"},"selection_glyph":null},"id":"ec695afc-00cb-4dd8-b886-1ec3953fb9f2","type":"GlyphRenderer"},{"attributes":{},"id":"78519e65-2c2d-4424-83a9-9dbd90af22d8","type":"BasicTicker"},{"attributes":{"dimension":1,"plot":{"id":"5abed06b-e288-46e8-9011-7e573975edd0","subtype":"Figure","type":"Plot"},"ticker":{"id":"6edfdffb-7ce2-4cdd-9a8b-187af55764c4","type":"BasicTicker"}},"id":"0b786862-0fbb-4be6-a9cf-9bc5c59562b8","type":"Grid"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"e9e30041-e0e2-4080-9d31-19b7ea2c3a9e","type":"Line"},{"attributes":{"callback":null,"start":0},"id":"47f2536d-f362-4e46-a6be-2c6ff9fd2795","type":"DataRange1d"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"32f83fd4-91dc-4dff-a7ce-4d79c6224e86","type":"Circle"},{"attributes":{},"id":"6edfdffb-7ce2-4cdd-9a8b-187af55764c4","type":"BasicTicker"},{"attributes":{"dimension":1,"plot":{"id":"4fab0e63-76cb-46a6-843e-af38f25fd676","subtype":"Figure","type":"Plot"},"ticker":{"id":"78519e65-2c2d-4424-83a9-9dbd90af22d8","type":"BasicTicker"}},"id":"77e0942d-9264-4240-8e68-8926967393ba","type":"Grid"},{"attributes":{},"id":"f4da7451-41df-4758-b1fb-6f6434e03f8b","type":"LinearScale"},{"attributes":{"plot":{"id":"5abed06b-e288-46e8-9011-7e573975edd0","subtype":"Figure","type":"Plot"},"ticker":{"id":"3b63ade1-d387-45e1-90da-265ac9592e72","type":"BasicTicker"}},"id":"3a00a1b6-fe6b-4538-bd67-533a0082b73c","type":"Grid"},{"attributes":{},"id":"3c28f81c-0697-47c1-ad14-2e0bfd73ee41","type":"LinearScale"},{"attributes":{"callback":null,"end":49937.270358022135,"start":0},"id":"2c3b2c68-cfe2-4e65-a9ee-60bd310a543a","type":"DataRange1d"},{"attributes":{},"id":"3b63ade1-d387-45e1-90da-265ac9592e72","type":"BasicTicker"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"71ac58c9-55fe-4631-b9ec-79f55dccf3f8","type":"Line"},{"attributes":{"axis_label":"cores","formatter":{"id":"f8c8a5cb-4fe7-46f9-9de3-c35644b04d92","type":"BasicTickFormatter"},"plot":{"id":"5abed06b-e288-46e8-9011-7e573975edd0","subtype":"Figure","type":"Plot"},"ticker":{"id":"335a9b38-a043-480c-9973-47def7c903b1","type":"FixedTicker"}},"id":"5215e61c-0d48-4011-8448-f9324be78a6c","type":"LinearAxis"},{"attributes":{"num_minor_ticks":10},"id":"c7f8c4c0-ccc9-4746-9876-e5a51098f8c8","type":"LogTicker"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"337a130a-12c7-4349-9504-9d093b482931","type":"BoxAnnotation"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"41323f6e-74d9-490d-b94a-b1bb7474c3ff","type":"BasicTickFormatter"},"plot":{"id":"5abed06b-e288-46e8-9011-7e573975edd0","subtype":"Figure","type":"Plot"},"ticker":{"id":"6edfdffb-7ce2-4cdd-9a8b-187af55764c4","type":"BasicTicker"}},"id":"c1db760e-8d31-4e13-b508-a853ff778a89","type":"LinearAxis"},{"attributes":{"data_source":{"id":"10842f42-4f23-49a9-85b5-8be77dc747a3","type":"ColumnDataSource"},"glyph":{"id":"abda4121-416d-4332-9411-83986320f042","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"e9e30041-e0e2-4080-9d31-19b7ea2c3a9e","type":"Line"},"selection_glyph":null},"id":"46e3e75e-6c2b-44a0-9c65-9b675d41ad2f","type":"GlyphRenderer"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"abda4121-416d-4332-9411-83986320f042","type":"Line"},{"attributes":{"child":{"id":"5abed06b-e288-46e8-9011-7e573975edd0","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"60def97a-ac42-4a56-a9d1-0607993c2e62","type":"Panel"},{"attributes":{"overlay":{"id":"337a130a-12c7-4349-9504-9d093b482931","type":"BoxAnnotation"},"plot":{"id":"5abed06b-e288-46e8-9011-7e573975edd0","subtype":"Figure","type":"Plot"}},"id":"e7ac849f-5c88-4646-ae8c-6a4bc419f016","type":"BoxZoomTool"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"fbdb8d93-5848-4381-a2cf-1d63541f10f6","type":"Circle"},{"attributes":{"plot":{"id":"5abed06b-e288-46e8-9011-7e573975edd0","subtype":"Figure","type":"Plot"}},"id":"f8834acc-9897-441d-9102-1f616408a50e","type":"WheelZoomTool"},{"attributes":{"child":{"id":"4fab0e63-76cb-46a6-843e-af38f25fd676","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"f8aa2e3b-14d5-4851-9c16-0c58862b0b9c","type":"Panel"},{"attributes":{},"id":"f8c8a5cb-4fe7-46f9-9de3-c35644b04d92","type":"BasicTickFormatter"},{"attributes":{"plot":{"id":"5abed06b-e288-46e8-9011-7e573975edd0","subtype":"Figure","type":"Plot"}},"id":"76cf33d1-1336-4737-8cad-79fdcc322c10","type":"PanTool"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"92abfb32-e7d6-4493-9b5b-d9fb88a34090","type":"Line"},{"attributes":{"callback":null,"sizing_mode":"scale_width","tabs":[{"id":"d02370bb-d02f-4e1b-ba87-d899301df6bf","type":"Panel"},{"id":"a1bcbfdd-7db0-4ae7-9ce5-61153092ac99","type":"Panel"}]},"id":"067958e4-18bc-4159-852e-fc2a375a27cb","type":"Tabs"},{"attributes":{"plot":{"id":"5abed06b-e288-46e8-9011-7e573975edd0","subtype":"Figure","type":"Plot"}},"id":"2c1a56f5-2c0d-4758-a7cc-eb45971629aa","type":"SaveTool"},{"attributes":{},"id":"4a2078e2-dc0a-4ee9-8148-99c308724a80","type":"BasicTickFormatter"},{"attributes":{},"id":"85fa896a-0164-445b-8199-17465b47072f","type":"ToolEvents"},{"attributes":{"below":[{"id":"5215e61c-0d48-4011-8448-f9324be78a6c","type":"LinearAxis"}],"left":[{"id":"c1db760e-8d31-4e13-b508-a853ff778a89","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"5215e61c-0d48-4011-8448-f9324be78a6c","type":"LinearAxis"},{"id":"3a00a1b6-fe6b-4538-bd67-533a0082b73c","type":"Grid"},{"id":"c1db760e-8d31-4e13-b508-a853ff778a89","type":"LinearAxis"},{"id":"0b786862-0fbb-4be6-a9cf-9bc5c59562b8","type":"Grid"},{"id":"337a130a-12c7-4349-9504-9d093b482931","type":"BoxAnnotation"},{"id":"a21531e6-2df4-4a46-b1a3-fb0c270ed7fd","type":"Legend"},{"id":"46e3e75e-6c2b-44a0-9c65-9b675d41ad2f","type":"GlyphRenderer"},{"id":"7c3e3106-8a5d-4189-9e5b-e5490e9a411b","type":"GlyphRenderer"},{"id":"32dfa032-2760-481a-a16b-3a70d989c048","type":"GlyphRenderer"}],"title":{"id":"ab1e2224-d79e-44eb-94ae-38e6722fb5a0","type":"Title"},"tool_events":{"id":"85fa896a-0164-445b-8199-17465b47072f","type":"ToolEvents"},"toolbar":{"id":"eb086fb2-cda6-4992-9763-f203826c3f7e","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"47f2536d-f362-4e46-a6be-2c6ff9fd2795","type":"DataRange1d"},"x_scale":{"id":"f4da7451-41df-4758-b1fb-6f6434e03f8b","type":"LinearScale"},"y_range":{"id":"2c3b2c68-cfe2-4e65-a9ee-60bd310a543a","type":"DataRange1d"},"y_scale":{"id":"3c28f81c-0697-47c1-ad14-2e0bfd73ee41","type":"LinearScale"}},"id":"5abed06b-e288-46e8-9011-7e573975edd0","subtype":"Figure","type":"Plot"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,165500.80688519427]}},"id":"3cf019ba-9b1f-42ed-b647-fc9001a82168","type":"ColumnDataSource"},{"attributes":{"line_color":{"value":"#ff7f0e"},"x":{"field":"x"},"y":{"field":"y"}},"id":"cc5db39a-16cb-4e1b-86d6-8f12c85cc5af","type":"Line"},{"attributes":{"plot":{"id":"5abed06b-e288-46e8-9011-7e573975edd0","subtype":"Figure","type":"Plot"}},"id":"a0a42aa4-41ea-4099-a082-ff6b3e0b4400","type":"HelpTool"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"55c37866-4dd7-43c4-a37f-ddd611adba37","type":"FixedTicker"},{"attributes":{"plot":null,"text":"DataFrames: Create"},"id":"ab1e2224-d79e-44eb-94ae-38e6722fb5a0","type":"Title"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"7b5af66e-5b5f-4f69-9990-b49719f2e0b6","type":"Line"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"eb4d7887-70ac-4a30-86f1-1f40e2b49fe4","type":"Line"},{"attributes":{"label":{"value":"random"},"renderers":[{"id":"46e3e75e-6c2b-44a0-9c65-9b675d41ad2f","type":"GlyphRenderer"}]},"id":"d94ad771-b4da-4cb2-8f9c-bac51b23c3c7","type":"LegendItem"},{"attributes":{"data_source":{"id":"a0f874ec-f5cf-46c0-ad48-25e382cb8e43","type":"ColumnDataSource"},"glyph":{"id":"70c98845-9cca-4d40-ac86-29049566c1e5","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"8e526ab1-2830-4487-9596-68e523596731","type":"Circle"},"selection_glyph":null},"id":"611ba4cf-ffd4-4546-88e1-08aeb3b14a35","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"4081f139-8263-4831-af45-7d73bd36d181","type":"Circle"},{"attributes":{"data_source":{"id":"1b801c3f-37c6-4605-b059-12d57dbdb1c3","type":"ColumnDataSource"},"glyph":{"id":"32f83fd4-91dc-4dff-a7ce-4d79c6224e86","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"4081f139-8263-4831-af45-7d73bd36d181","type":"Circle"},"selection_glyph":null},"id":"a8e83bb1-625e-4332-8c28-68b807e4ac87","type":"GlyphRenderer"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"70c98845-9cca-4d40-ac86-29049566c1e5","type":"Circle"},{"attributes":{"plot":{"id":"5abed06b-e288-46e8-9011-7e573975edd0","subtype":"Figure","type":"Plot"}},"id":"43e9c244-f5b8-4240-a21d-ba93dd66d881","type":"ResetTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"zYZNznMFgkAJVc0dH1aIQF6HLbAX8KBAEDiFMgobsEBOSk1Dh6G8QIcwzAFNYsdAkYwvunD+0kD/VzpMtBLaQNxDJJJ7lt5A","dtype":"float64","shape":[9]}}},"id":"a0f874ec-f5cf-46c0-ad48-25e382cb8e43","type":"ColumnDataSource"},{"attributes":{},"id":"41323f6e-74d9-490d-b94a-b1bb7474c3ff","type":"BasicTickFormatter"},{"attributes":{"items":[{"id":"1e0efdec-b408-4fd1-917c-8b4abef0c32f","type":"LegendItem"},{"id":"9707ea10-1988-4219-a460-b3ef4112a1be","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"8ac92914-8e42-465b-b484-6acf7a09c79f","subtype":"Figure","type":"Plot"}},"id":"7a0525cc-55d6-48fa-9055-850bc3b16a13","type":"Legend"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,486720.45329374564]}},"id":"0084dd1d-b5b6-4249-b7dd-4d81ae8d44d9","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"iDmAdOYzhED3jGZaeR+UQMzGF5CAyKNAqQZ49/jWsUDabo5ZahbBQPB5MuwHU9BAhZDfGJtW10Al2QgzKAzZQOndxaYoYuhA","dtype":"float64","shape":[9]}}},"id":"10842f42-4f23-49a9-85b5-8be77dc747a3","type":"ColumnDataSource"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"2194ad66-1e38-4dd6-ba9c-50d5ba5b5e58","type":"Circle"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"iDmAdOYzhED3jGZaeR+UQMzGF5CAyKNAqQZ49/jWsUDabo5ZahbBQPB5MuwHU9BAhZDfGJtW10Al2QgzKAzZQOndxaYoYuhA","dtype":"float64","shape":[9]}}},"id":"b6dc27ed-5b0b-41a2-8d69-b5f301ba71bf","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"1e7af976-289b-4c4e-bf70-714e2aa9e95c","type":"ColumnDataSource"},"glyph":{"id":"67ed2335-c348-4d5d-9031-7a893dfbafcc","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"53f5e00e-5c20-4a96-8e13-0995e030ab6d","type":"Line"},"selection_glyph":null},"id":"32dfa032-2760-481a-a16b-3a70d989c048","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"b6dc27ed-5b0b-41a2-8d69-b5f301ba71bf","type":"ColumnDataSource"},"glyph":{"id":"fbdb8d93-5848-4381-a2cf-1d63541f10f6","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"2194ad66-1e38-4dd6-ba9c-50d5ba5b5e58","type":"Circle"},"selection_glyph":null},"id":"7c3e3106-8a5d-4189-9e5b-e5490e9a411b","type":"GlyphRenderer"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"335a9b38-a043-480c-9973-47def7c903b1","type":"FixedTicker"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"8e526ab1-2830-4487-9596-68e523596731","type":"Circle"},{"attributes":{"label":{"value":"100ms"},"renderers":[{"id":"a2e3d51d-d8bb-41ac-8515-3f1154af5d97","type":"GlyphRenderer"}]},"id":"8ae41a66-88b0-4423-b9a9-153148095b06","type":"LegendItem"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"53f5e00e-5c20-4a96-8e13-0995e030ab6d","type":"Line"},{"attributes":{"label":{"value":"100ms"},"renderers":[{"id":"cd853902-f97c-4992-8384-5ed93deffbc5","type":"GlyphRenderer"}]},"id":"1e0efdec-b408-4fd1-917c-8b4abef0c32f","type":"LegendItem"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"67ed2335-c348-4d5d-9031-7a893dfbafcc","type":"Line"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,165500.80688519427]}},"id":"1e7af976-289b-4c4e-bf70-714e2aa9e95c","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"b1f9c2ed-12a3-425c-a852-37fafe3ee9c8","subtype":"Figure","type":"Plot"},"ticker":{"id":"1f17c79c-958a-4b8d-b13a-5a1b3fe37259","type":"LogTicker"}},"id":"22a51e72-b540-4b2c-8c88-9ade9d03d005","type":"Grid"},{"attributes":{"num_minor_ticks":10},"id":"3e1785c2-5406-42b7-b954-df12211fa100","type":"LogTicker"},{"attributes":{},"id":"9504112c-f960-4fb5-bd9c-0723c7c7fa72","type":"LogScale"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"851b8c01-8fff-4950-802c-c850631d44c2","type":"Line"},{"attributes":{"dimension":1,"plot":{"id":"b1f9c2ed-12a3-425c-a852-37fafe3ee9c8","subtype":"Figure","type":"Plot"},"ticker":{"id":"3e1785c2-5406-42b7-b954-df12211fa100","type":"LogTicker"}},"id":"389f6e50-4229-4968-b92d-2704548be1fd","type":"Grid"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"46c9f7fd-1c9e-4c19-916e-82699db390aa","type":"Line"},{"attributes":{"ticker":null},"id":"31b24d74-6828-4e72-a0b1-b9ff3ea2812d","type":"LogTickFormatter"},{"attributes":{"data_source":{"id":"6b415d1b-4f2d-4b54-9905-a68e448bd719","type":"ColumnDataSource"},"glyph":{"id":"851b8c01-8fff-4950-802c-c850631d44c2","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"46c9f7fd-1c9e-4c19-916e-82699db390aa","type":"Line"},"selection_glyph":null},"id":"b11cf862-e14d-410d-a091-23cafd5f1930","type":"GlyphRenderer"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"3dc67210-eb0f-4c45-8bf3-9ac6b35a51c3","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"b1f9c2ed-12a3-425c-a852-37fafe3ee9c8","subtype":"Figure","type":"Plot"}},"id":"5e757e75-742d-4e2b-a209-c24a8e6f6f09","type":"PanTool"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"e52748fa-0c22-477b-9c44-f35955ece350","type":"Line"},{"attributes":{"plot":{"id":"b1f9c2ed-12a3-425c-a852-37fafe3ee9c8","subtype":"Figure","type":"Plot"}},"id":"9d0d89f3-9fda-4c43-a80f-f4e61bcf7ae2","type":"WheelZoomTool"},{"attributes":{"overlay":{"id":"3dc67210-eb0f-4c45-8bf3-9ac6b35a51c3","type":"BoxAnnotation"},"plot":{"id":"b1f9c2ed-12a3-425c-a852-37fafe3ee9c8","subtype":"Figure","type":"Plot"}},"id":"96a77bc3-3076-4302-83ae-ca23b110ad65","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"b1f9c2ed-12a3-425c-a852-37fafe3ee9c8","subtype":"Figure","type":"Plot"}},"id":"52d138cc-73e1-4c54-9808-4c7479025432","type":"SaveTool"},{"attributes":{"plot":{"id":"b1f9c2ed-12a3-425c-a852-37fafe3ee9c8","subtype":"Figure","type":"Plot"}},"id":"542aceac-c73e-4206-9e7d-a2e326d4112f","type":"ResetTool"},{"attributes":{"plot":{"id":"b1f9c2ed-12a3-425c-a852-37fafe3ee9c8","subtype":"Figure","type":"Plot"}},"id":"aecac00b-e41f-415c-86ad-223f8986f0f5","type":"HelpTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"sAHL3O3PZEALehwq1T1lQCHeSBOVVGVARLJVKdkUZUBZ9cB/YLBjQCWoVrQJiGBA4ishF0sKXUCMae/xfoRaQC1Vmopl8FJA","dtype":"float64","shape":[9]}}},"id":"6b415d1b-4f2d-4b54-9905-a68e448bd719","type":"ColumnDataSource"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"8y0AKWPAZkCdUmbwFEF2QI6UudUfxYVAYqt1iZwWkUDDRzQLwfGbQB6idyjfk55A/UKAURi0l0CkSi02kcaaQNjl7PoIv5pA","dtype":"float64","shape":[9]}}},"id":"7b276bde-624a-404d-a0c9-3b5647fbe4c5","type":"ColumnDataSource"},{"attributes":{"ticker":null},"id":"50eadc2e-195e-4477-a4bd-aa0597ac1245","type":"LogTickFormatter"},{"attributes":{"plot":{"id":"e761952d-458d-4815-b822-4649d8dc2644","subtype":"Figure","type":"Plot"}},"id":"86256809-cc92-4e15-ae4a-6278598dfde4","type":"SaveTool"},{"attributes":{"data_source":{"id":"5bd413e9-d12b-4922-9b6b-f0a2b43b0622","type":"ColumnDataSource"},"glyph":{"id":"e52748fa-0c22-477b-9c44-f35955ece350","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"ff3bd972-17cf-4f77-8c33-acc754cd4117","type":"Line"},"selection_glyph":null},"id":"69bd697e-0832-457a-8127-aaea5640b266","type":"GlyphRenderer"},{"attributes":{"plot":null,"text":"Tasks: Sequential"},"id":"dfa56f3f-a156-4193-83ff-40eb8c67f4ed","type":"Title"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"ff3bd972-17cf-4f77-8c33-acc754cd4117","type":"Line"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"497eb2a8-9f4f-4a33-8db7-0bf2b0729624","type":"FixedTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,42623.433202269836]}},"id":"5bd413e9-d12b-4922-9b6b-f0a2b43b0622","type":"ColumnDataSource"},{"attributes":{"below":[{"id":"912de431-8b1d-46d6-af89-e561bde867a4","type":"LinearAxis"}],"left":[{"id":"da6437c6-3b81-4eed-9a7a-58602ad66226","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"912de431-8b1d-46d6-af89-e561bde867a4","type":"LinearAxis"},{"id":"66e96303-5198-445b-9a3b-86f7e3ceff8c","type":"Grid"},{"id":"da6437c6-3b81-4eed-9a7a-58602ad66226","type":"LinearAxis"},{"id":"0f416ece-5b5f-4dab-99b5-c95edd50860a","type":"Grid"},{"id":"a206be90-515a-4182-b323-5f422be9232e","type":"BoxAnnotation"},{"id":"bd46bbf9-7e52-4d61-9be4-1f608610f06a","type":"Legend"},{"id":"5099c155-3073-4833-b7aa-b081f8daa2ff","type":"GlyphRenderer"},{"id":"8e226454-922a-4805-8aec-847436379896","type":"GlyphRenderer"},{"id":"4cc54198-52da-4f96-a5e4-bafc00a584e6","type":"GlyphRenderer"}],"title":{"id":"dfa56f3f-a156-4193-83ff-40eb8c67f4ed","type":"Title"},"tool_events":{"id":"0e4e2222-adbf-49f6-acd2-33308b2ca3ba","type":"ToolEvents"},"toolbar":{"id":"d63e6e35-658f-4878-8b6c-d2aa06097609","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"9e2489a1-99dd-4b02-87d9-a898db578105","type":"DataRange1d"},"x_scale":{"id":"385c7cce-2fea-488f-827f-4ca5b3b70083","type":"LinearScale"},"y_range":{"id":"6e9f69c9-63ac-4f86-b56a-326d53713801","type":"DataRange1d"},"y_scale":{"id":"feffa6ec-8962-4c41-8d23-9a0262def705","type":"LinearScale"}},"id":"c3993f67-2d60-4740-8c46-23091463e0ff","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"0e4e2222-adbf-49f6-acd2-33308b2ca3ba","type":"ToolEvents"},{"attributes":{},"id":"ff84d567-e4cf-48b2-8020-53c15130ecaa","type":"BasicTickFormatter"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"a8554471-d5ef-4589-a321-913aa6bb28a1","type":"PanTool"},{"id":"e7d3bd37-e75c-42d9-a4b8-4e76df6bec36","type":"WheelZoomTool"},{"id":"b69eb5c0-544d-428c-96fd-a2f8b725d9dc","type":"BoxZoomTool"},{"id":"0628cc13-598c-4da3-a089-50a5bd679f56","type":"SaveTool"},{"id":"c1e43ec1-5eec-450c-a783-78ee199ef975","type":"ResetTool"},{"id":"2852fd51-fc13-4e7e-b08d-8fc2c80a47cb","type":"HelpTool"}]},"id":"d63e6e35-658f-4878-8b6c-d2aa06097609","type":"Toolbar"},{"attributes":{"callback":null,"start":0},"id":"9e2489a1-99dd-4b02-87d9-a898db578105","type":"DataRange1d"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"f5f383a4-e75d-46f0-86d1-1aa89511616d","type":"Line"},{"attributes":{},"id":"385c7cce-2fea-488f-827f-4ca5b3b70083","type":"LinearScale"},{"attributes":{"callback":null,"end":170.64319767219516,"start":0},"id":"6e9f69c9-63ac-4f86-b56a-326d53713801","type":"DataRange1d"},{"attributes":{},"id":"feffa6ec-8962-4c41-8d23-9a0262def705","type":"LinearScale"},{"attributes":{"plot":{"id":"c3993f67-2d60-4740-8c46-23091463e0ff","subtype":"Figure","type":"Plot"},"ticker":{"id":"60aa3ecf-fbe6-4738-8e0b-d48824959a91","type":"BasicTicker"}},"id":"66e96303-5198-445b-9a3b-86f7e3ceff8c","type":"Grid"},{"attributes":{"axis_label":"cores","formatter":{"id":"ff84d567-e4cf-48b2-8020-53c15130ecaa","type":"BasicTickFormatter"},"plot":{"id":"c3993f67-2d60-4740-8c46-23091463e0ff","subtype":"Figure","type":"Plot"},"ticker":{"id":"dfe7fa67-6ff7-49c0-b16c-2a299864d0b9","type":"FixedTicker"}},"id":"912de431-8b1d-46d6-af89-e561bde867a4","type":"LinearAxis"},{"attributes":{},"id":"60aa3ecf-fbe6-4738-8e0b-d48824959a91","type":"BasicTicker"},{"attributes":{"axis_label":"tasks/s","formatter":{"id":"f67545e2-709d-4141-8402-d98300497037","type":"BasicTickFormatter"},"plot":{"id":"c3993f67-2d60-4740-8c46-23091463e0ff","subtype":"Figure","type":"Plot"},"ticker":{"id":"44cac71f-3f6c-4055-aa05-f96a6adaea1d","type":"BasicTicker"}},"id":"da6437c6-3b81-4eed-9a7a-58602ad66226","type":"LinearAxis"},{"attributes":{},"id":"44cac71f-3f6c-4055-aa05-f96a6adaea1d","type":"BasicTicker"},{"attributes":{"dimension":1,"plot":{"id":"c3993f67-2d60-4740-8c46-23091463e0ff","subtype":"Figure","type":"Plot"},"ticker":{"id":"44cac71f-3f6c-4055-aa05-f96a6adaea1d","type":"BasicTicker"}},"id":"0f416ece-5b5f-4dab-99b5-c95edd50860a","type":"Grid"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"17f0b01f-853a-465d-ac8e-b38fdb0c9714","type":"Line"},{"attributes":{"data_source":{"id":"f1831457-0eb1-4ab3-857d-0d85e71eff8b","type":"ColumnDataSource"},"glyph":{"id":"17f0b01f-853a-465d-ac8e-b38fdb0c9714","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"f5f383a4-e75d-46f0-86d1-1aa89511616d","type":"Line"},"selection_glyph":null},"id":"5099c155-3073-4833-b7aa-b081f8daa2ff","type":"GlyphRenderer"},{"attributes":{"child":{"id":"c3993f67-2d60-4740-8c46-23091463e0ff","subtype":"Figure","type":"Plot"},"title":"linear"},"id":"b23801a4-be5a-492c-ab93-ece3370c2047","type":"Panel"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"a206be90-515a-4182-b323-5f422be9232e","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"c3993f67-2d60-4740-8c46-23091463e0ff","subtype":"Figure","type":"Plot"}},"id":"a8554471-d5ef-4589-a321-913aa6bb28a1","type":"PanTool"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"8f3a791e-d512-42e7-93ee-6484f449844c","type":"Circle"},{"attributes":{"plot":{"id":"c3993f67-2d60-4740-8c46-23091463e0ff","subtype":"Figure","type":"Plot"}},"id":"e7d3bd37-e75c-42d9-a4b8-4e76df6bec36","type":"WheelZoomTool"},{"attributes":{"overlay":{"id":"a206be90-515a-4182-b323-5f422be9232e","type":"BoxAnnotation"},"plot":{"id":"c3993f67-2d60-4740-8c46-23091463e0ff","subtype":"Figure","type":"Plot"}},"id":"b69eb5c0-544d-428c-96fd-a2f8b725d9dc","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"c3993f67-2d60-4740-8c46-23091463e0ff","subtype":"Figure","type":"Plot"}},"id":"0628cc13-598c-4da3-a089-50a5bd679f56","type":"SaveTool"},{"attributes":{"plot":{"id":"c3993f67-2d60-4740-8c46-23091463e0ff","subtype":"Figure","type":"Plot"}},"id":"c1e43ec1-5eec-450c-a783-78ee199ef975","type":"ResetTool"},{"attributes":{"plot":{"id":"c3993f67-2d60-4740-8c46-23091463e0ff","subtype":"Figure","type":"Plot"}},"id":"2852fd51-fc13-4e7e-b08d-8fc2c80a47cb","type":"HelpTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"sAHL3O3PZEALehwq1T1lQCHeSBOVVGVARLJVKdkUZUBZ9cB/YLBjQCWoVrQJiGBA4ishF0sKXUCMae/xfoRaQC1Vmopl8FJA","dtype":"float64","shape":[9]}}},"id":"f1831457-0eb1-4ab3-857d-0d85e71eff8b","type":"ColumnDataSource"},{"attributes":{"children":[{"id":"948f0ba1-2615-4ca3-9634-5eade5d4eab8","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"399a5322-c563-42d3-88eb-636543214ee9","type":"WidgetBox"},{"attributes":{"data_source":{"id":"85c2b4e9-0196-44c3-8ef9-407e77eae0fe","type":"ColumnDataSource"},"glyph":{"id":"bf09f5bb-4f6e-4975-9ad4-5f256c231f78","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"8f3a791e-d512-42e7-93ee-6484f449844c","type":"Circle"},"selection_glyph":null},"id":"8e226454-922a-4805-8aec-847436379896","type":"GlyphRenderer"},{"attributes":{"children":[{"id":"59c15cdf-cad3-4afc-a12b-458d2a0a5b13","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"c3bcc172-5e0f-4658-96a7-e7aab967837c","type":"WidgetBox"},{"attributes":{},"id":"f67545e2-709d-4141-8402-d98300497037","type":"BasicTickFormatter"},{"attributes":{"children":[{"id":"399a5322-c563-42d3-88eb-636543214ee9","type":"WidgetBox"},{"id":"bb4e1bc3-7561-4489-9c6b-cca718a45554","type":"WidgetBox"}],"sizing_mode":"scale_width"},"id":"8b215b4b-e365-462a-ae56-01bea5fb44cd","type":"Row"},{"attributes":{"data_source":{"id":"c552bbe2-ea34-4502-818d-72d8eaef0893","type":"ColumnDataSource"},"glyph":{"id":"ed50651e-a87f-4ac9-b730-0a57804c8b90","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"0d784b33-8c4a-4ff1-8af5-955ac930029c","type":"Line"},"selection_glyph":null},"id":"4cc54198-52da-4f96-a5e4-bafc00a584e6","type":"GlyphRenderer"},{"attributes":{"items":[{"id":"72f1eca7-242c-4d72-90fe-d93812baa434","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"c3993f67-2d60-4740-8c46-23091463e0ff","subtype":"Figure","type":"Plot"}},"id":"bd46bbf9-7e52-4d61-9be4-1f608610f06a","type":"Legend"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"bf09f5bb-4f6e-4975-9ad4-5f256c231f78","type":"Circle"},{"attributes":{"label":{"value":"fast"},"renderers":[{"id":"5099c155-3073-4833-b7aa-b081f8daa2ff","type":"GlyphRenderer"}]},"id":"72f1eca7-242c-4d72-90fe-d93812baa434","type":"LegendItem"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"0d784b33-8c4a-4ff1-8af5-955ac930029c","type":"Line"},{"attributes":{"children":[{"id":"299ee94e-2b3f-41aa-b083-e0941f4210b7","type":"Row"},{"id":"8b215b4b-e365-462a-ae56-01bea5fb44cd","type":"Row"}],"sizing_mode":"scale_width"},"id":"ac7896b3-c871-4de2-8749-25da643eb97e","type":"Column"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"cd8b2f63-4971-461d-a48d-723dd0606040","type":"Line"},{"attributes":{"children":[{"id":"c3bcc172-5e0f-4658-96a7-e7aab967837c","type":"WidgetBox"},{"id":"2b7a29fc-991a-41b4-9a9b-4884221d1fec","type":"WidgetBox"}],"sizing_mode":"scale_width"},"id":"299ee94e-2b3f-41aa-b083-e0941f4210b7","type":"Row"},{"attributes":{"sizing_mode":"scale_width","toolbar_location":"above","tools":[{"id":"8946c13e-3fd9-4cd2-ad6b-048b3f3d3184","type":"PanTool"},{"id":"18467990-49d6-4857-8698-0ecf850b8dae","type":"WheelZoomTool"},{"id":"f12f9873-2e5a-4d0e-b72a-022169603cba","type":"BoxZoomTool"},{"id":"70ca44b0-73ae-499b-bf6e-a24ee883237a","type":"SaveTool"},{"id":"0888f93d-b6d7-486b-b5f6-52007244ea56","type":"ResetTool"},{"id":"12b05cc5-f225-412a-a571-bb01258df41c","type":"HelpTool"},{"id":"73c8ebe1-fa92-4ff6-ac89-ed9778c71129","type":"PanTool"},{"id":"256f01fa-c5a6-4e2d-9723-781db6e85fc8","type":"WheelZoomTool"},{"id":"36eac939-56b7-4b14-8629-bde3b0ed0b72","type":"BoxZoomTool"},{"id":"c3009076-d910-41e9-9d89-8c3229a09dcb","type":"SaveTool"},{"id":"a900c3fe-2975-4b0c-bd7b-39871891269e","type":"ResetTool"},{"id":"8c3dcf24-6388-41fd-a1b4-b7d803c9a77b","type":"HelpTool"},{"id":"5e757e75-742d-4e2b-a209-c24a8e6f6f09","type":"PanTool"},{"id":"9d0d89f3-9fda-4c43-a80f-f4e61bcf7ae2","type":"WheelZoomTool"},{"id":"96a77bc3-3076-4302-83ae-ca23b110ad65","type":"BoxZoomTool"},{"id":"52d138cc-73e1-4c54-9808-4c7479025432","type":"SaveTool"},{"id":"542aceac-c73e-4206-9e7d-a2e326d4112f","type":"ResetTool"},{"id":"aecac00b-e41f-415c-86ad-223f8986f0f5","type":"HelpTool"},{"id":"a8554471-d5ef-4589-a321-913aa6bb28a1","type":"PanTool"},{"id":"e7d3bd37-e75c-42d9-a4b8-4e76df6bec36","type":"WheelZoomTool"},{"id":"b69eb5c0-544d-428c-96fd-a2f8b725d9dc","type":"BoxZoomTool"},{"id":"0628cc13-598c-4da3-a089-50a5bd679f56","type":"SaveTool"},{"id":"c1e43ec1-5eec-450c-a783-78ee199ef975","type":"ResetTool"},{"id":"2852fd51-fc13-4e7e-b08d-8fc2c80a47cb","type":"HelpTool"},{"id":"e0ba9b60-db70-4b11-90ea-2868bc56108f","type":"PanTool"},{"id":"119970ac-6463-4f32-a7e2-f3bb89fcae56","type":"WheelZoomTool"},{"id":"f50914d8-200a-4f31-abfd-26c17a885c0c","type":"BoxZoomTool"},{"id":"25aa7dd9-490d-4122-9f8a-fa60ccf8b1ac","type":"SaveTool"},{"id":"f26e3fe2-102a-4632-99cc-17710686c86a","type":"ResetTool"},{"id":"3b7b47f2-f735-4ce9-94a7-152fe76da795","type":"HelpTool"},{"id":"3a7b15e0-3597-4a99-8089-6ecce1d9ca49","type":"PanTool"},{"id":"cd0d9904-16bc-4916-b074-b04f42c3c0c8","type":"WheelZoomTool"},{"id":"a0edbc21-c056-4cdc-a171-0c99928b8ab7","type":"BoxZoomTool"},{"id":"8b476975-4b0a-41e2-b278-f7b49bc29967","type":"SaveTool"},{"id":"fbef2fb6-1564-4b5c-b1ab-205dd4802aea","type":"ResetTool"},{"id":"b34e007a-6220-4ded-a494-4c692ef5e34e","type":"HelpTool"},{"id":"06d1c4f3-2e25-4e2c-87d9-5dcdfdef1c9f","type":"PanTool"},{"id":"711ed14b-0adc-44af-9a23-d8e69826940d","type":"WheelZoomTool"},{"id":"92b4441f-a58a-4aa2-b658-2032f50692c7","type":"BoxZoomTool"},{"id":"82e9c665-d127-48d2-8b1d-81259c5c59c1","type":"SaveTool"},{"id":"4a8c0673-09a5-4a8b-9ec7-2c6b9e9a1e2f","type":"ResetTool"},{"id":"c58059d9-7ec6-40de-a9a1-e030da7a5db5","type":"HelpTool"},{"id":"ef963147-69dd-4316-824e-c39236f8374e","type":"PanTool"},{"id":"dd1840f2-3a3f-4ee8-96d7-274b0b58a153","type":"WheelZoomTool"},{"id":"cc2ba96b-474a-45b8-a0f7-c979a271d7f6","type":"BoxZoomTool"},{"id":"89c7e6f2-37c0-4e30-8273-cc364867a06e","type":"SaveTool"},{"id":"5f24c536-7893-4741-b8a6-f4e717ad6684","type":"ResetTool"},{"id":"7ec0ca47-849b-4454-a2f0-7708648ed7be","type":"HelpTool"}]},"id":"ffc984fd-7ea1-4dfa-96f3-005a9664af52","type":"ToolbarBox"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"ed50651e-a87f-4ac9-b730-0a57804c8b90","type":"Line"},{"attributes":{"children":[{"id":"b175b9d7-0992-4013-a3da-e7da9e7115e2","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"2b7a29fc-991a-41b4-9a9b-4884221d1fec","type":"WidgetBox"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"dfe7fa67-6ff7-49c0-b16c-2a299864d0b9","type":"FixedTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,42623.433202269836]}},"id":"c552bbe2-ea34-4502-818d-72d8eaef0893","type":"ColumnDataSource"},{"attributes":{"children":[{"id":"aeaa4543-6cc0-4ed6-bf28-929d3bf8399f","type":"Tabs"}],"sizing_mode":"scale_width"},"id":"bb4e1bc3-7561-4489-9c6b-cca718a45554","type":"WidgetBox"},{"attributes":{"data_source":{"id":"eea30ed3-c5f9-448f-b8f6-2e5a5fa90822","type":"ColumnDataSource"},"glyph":{"id":"fd037a13-f03b-4366-a086-1cc9bfd29a68","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"cd8b2f63-4971-461d-a48d-723dd0606040","type":"Line"},"selection_glyph":null},"id":"a209df9a-abf2-4be6-83bc-504ea82e6cdc","type":"GlyphRenderer"},{"attributes":{"children":[{"id":"ffc984fd-7ea1-4dfa-96f3-005a9664af52","type":"ToolbarBox"},{"id":"ac7896b3-c871-4de2-8749-25da643eb97e","type":"Column"}],"sizing_mode":"scale_width"},"id":"3328a090-74cd-4548-b396-7ab5adb364e6","type":"Column"},{"attributes":{},"id":"9f244216-2c49-46aa-86e7-c7a71dc00710","type":"ToolEvents"},{"attributes":{"callback":null,"start":0},"id":"87b21d20-ff5f-42fd-8677-3f4c6a3905e5","type":"DataRange1d"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"6b4da656-890a-4f15-8308-bba17bf46855","type":"PanTool"},{"id":"a81c27f4-60d4-49ea-a1a6-2ac54ae5f301","type":"WheelZoomTool"},{"id":"62446d6b-d3d2-4d9b-91a7-1846e468a714","type":"BoxZoomTool"},{"id":"64b8c2ea-0b92-4c14-9235-028576947076","type":"SaveTool"},{"id":"9b3a52e3-e27a-4f6b-b6ba-e0db55d8bed9","type":"ResetTool"},{"id":"92a69e8a-7390-4350-9ad6-284193c2055f","type":"HelpTool"}]},"id":"0084c66b-ce81-4d6d-bee6-69dffd00663c","type":"Toolbar"},{"attributes":{"num_minor_ticks":10},"id":"c6189eaa-7e2e-47a5-bf27-b39080a1c5a0","type":"LogTicker"},{"attributes":{"num_minor_ticks":10},"id":"b8d42256-14e2-499e-ab3d-d45b852369b9","type":"LogTicker"},{"attributes":{"callback":null,"end":5234.083519499139,"start":0},"id":"b3a359d4-53fe-4c66-9a9d-198781a27d5d","type":"DataRange1d"},{"attributes":{"label":{"value":"x.rechunk(...)"},"renderers":[{"id":"6d17249c-9696-4c57-814c-e07fcba42a96","type":"GlyphRenderer"}]},"id":"c31a4625-e1ca-48cf-a30a-239eb1cb936d","type":"LegendItem"},{"attributes":{"plot":{"id":"6283fd05-bec6-41be-819f-32dcc5714915","subtype":"Figure","type":"Plot"},"ticker":{"id":"c6189eaa-7e2e-47a5-bf27-b39080a1c5a0","type":"LogTicker"}},"id":"a80b5d2e-a365-49b7-815c-151a0ac219f5","type":"Grid"},{"attributes":{},"id":"e188ceed-f72f-4184-827b-8cf9dcd2016b","type":"LogScale"},{"attributes":{"axis_label":"cores","formatter":{"id":"fb147d32-4323-41ae-9418-c219938a98fc","type":"LogTickFormatter"},"plot":{"id":"6283fd05-bec6-41be-819f-32dcc5714915","subtype":"Figure","type":"Plot"},"ticker":{"id":"c1caeac4-3154-4361-9e12-436af3f29160","type":"FixedTicker"}},"id":"69e7f477-33dd-49bb-8742-8a4acc5e5cba","type":"LogAxis"},{"attributes":{},"id":"098ce152-b87a-43ed-962e-e78371796667","type":"LogScale"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"cb7c8cb6-9623-4908-96f9-761c89176a89","type":"LogTickFormatter"},"plot":{"id":"6283fd05-bec6-41be-819f-32dcc5714915","subtype":"Figure","type":"Plot"},"ticker":{"id":"b8d42256-14e2-499e-ab3d-d45b852369b9","type":"LogTicker"}},"id":"6630e827-9e7c-490e-a37e-85408a6a4ae2","type":"LogAxis"},{"attributes":{"dimension":1,"plot":{"id":"6283fd05-bec6-41be-819f-32dcc5714915","subtype":"Figure","type":"Plot"},"ticker":{"id":"b8d42256-14e2-499e-ab3d-d45b852369b9","type":"LogTicker"}},"id":"d0f0b675-2b8e-4fd7-9f32-6cbcb749cf23","type":"Grid"},{"attributes":{"label":{"value":"x.rechunk(...)"},"renderers":[{"id":"0d08fe1d-5742-4c92-84b6-7ecbbbf5f300","type":"GlyphRenderer"}]},"id":"64bf9c19-8a17-4233-b901-97145d31cd9e","type":"LegendItem"},{"attributes":{"items":[{"id":"64bf9c19-8a17-4233-b901-97145d31cd9e","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"6283fd05-bec6-41be-819f-32dcc5714915","subtype":"Figure","type":"Plot"}},"id":"ae2f1fe7-e7ad-4a3c-9994-cd087b537257","type":"Legend"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"s8bqjTzBiUBTem6Z0ZyGQFtOQa+MMZNAS+oBg8Twl0Arqf2CF8SjQIPxxSmeUatANdBkm/kbrkBhrYhhFXK0QGeZb4e3brJA","dtype":"float64","shape":[9]}}},"id":"362f2dd6-9136-4c7c-a7a1-70a04e76a875","type":"ColumnDataSource"},{"attributes":{"data_source":{"id":"c4a73cdb-83e0-4105-8387-69c5f81b6312","type":"ColumnDataSource"},"glyph":{"id":"3fafd503-c9c9-4f95-bee1-8492be01dea0","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"fed144f7-83c4-4242-935a-c4908966c807","type":"Line"},"selection_glyph":null},"id":"ec6a1d09-8449-46cf-b1be-3e11e093564e","type":"GlyphRenderer"},{"attributes":{"plot":{"id":"6283fd05-bec6-41be-819f-32dcc5714915","subtype":"Figure","type":"Plot"}},"id":"6b4da656-890a-4f15-8308-bba17bf46855","type":"PanTool"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"6963c4fc-c97f-4d0b-9cf2-2be605fff08d","type":"Circle"},{"attributes":{"plot":{"id":"6283fd05-bec6-41be-819f-32dcc5714915","subtype":"Figure","type":"Plot"}},"id":"a81c27f4-60d4-49ea-a1a6-2ac54ae5f301","type":"WheelZoomTool"},{"attributes":{"overlay":{"id":"dde7e77b-1917-4489-9e8d-103c92197c86","type":"BoxAnnotation"},"plot":{"id":"6283fd05-bec6-41be-819f-32dcc5714915","subtype":"Figure","type":"Plot"}},"id":"62446d6b-d3d2-4d9b-91a7-1846e468a714","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"6283fd05-bec6-41be-819f-32dcc5714915","subtype":"Figure","type":"Plot"}},"id":"64b8c2ea-0b92-4c14-9235-028576947076","type":"SaveTool"},{"attributes":{"plot":{"id":"6283fd05-bec6-41be-819f-32dcc5714915","subtype":"Figure","type":"Plot"}},"id":"9b3a52e3-e27a-4f6b-b6ba-e0db55d8bed9","type":"ResetTool"},{"attributes":{"plot":{"id":"6283fd05-bec6-41be-819f-32dcc5714915","subtype":"Figure","type":"Plot"}},"id":"92a69e8a-7390-4350-9ad6-284193c2055f","type":"HelpTool"},{"attributes":{"data_source":{"id":"362f2dd6-9136-4c7c-a7a1-70a04e76a875","type":"ColumnDataSource"},"glyph":{"id":"6963c4fc-c97f-4d0b-9cf2-2be605fff08d","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"22dbf6a7-d3aa-44ab-98ac-12274725f5f9","type":"Circle"},"selection_glyph":null},"id":"6af73c8b-fc8e-4c8b-a659-a7220854b1e0","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"22dbf6a7-d3aa-44ab-98ac-12274725f5f9","type":"Circle"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,336851.07713094505]}},"id":"97e682ba-d0e2-401f-98d3-0d4f783335b1","type":"ColumnDataSource"},{"attributes":{"items":[{"id":"c31a4625-e1ca-48cf-a30a-239eb1cb936d","type":"LegendItem"}],"location":"bottom_right","plot":{"id":"26cfe895-d550-439a-9fd8-f8b33f7650c4","subtype":"Figure","type":"Plot"}},"id":"3df41605-9fba-40d8-86bb-4971ef0ac67c","type":"Legend"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"3fafd503-c9c9-4f95-bee1-8492be01dea0","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"fed144f7-83c4-4242-935a-c4908966c807","type":"Line"},{"attributes":{"plot":null,"text":"Arrays: Rechunking"},"id":"cab85b81-e7db-4f7b-a3ac-1734f990ff9e","type":"Title"},{"attributes":{"ticks":[2,4,8,16,32,64,128,256,512]},"id":"c1caeac4-3154-4361-9e12-436af3f29160","type":"FixedTicker"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,210983.56929545625]}},"id":"c4a73cdb-83e0-4105-8387-69c5f81b6312","type":"ColumnDataSource"},{"attributes":{"below":[{"id":"94a8080b-1f64-45a9-840b-0d89d2ad1ae8","type":"LinearAxis"}],"left":[{"id":"ddbfd72e-8a85-4061-9a3b-e19d9aaa07e2","type":"LinearAxis"}],"plot_height":400,"plot_width":400,"renderers":[{"id":"94a8080b-1f64-45a9-840b-0d89d2ad1ae8","type":"LinearAxis"},{"id":"53d3e7d1-ec27-4790-acd8-6afb80e548b9","type":"Grid"},{"id":"ddbfd72e-8a85-4061-9a3b-e19d9aaa07e2","type":"LinearAxis"},{"id":"d82b6b00-db6f-45a4-b0e7-968b580bb9d4","type":"Grid"},{"id":"fe061748-d906-414d-b8a1-8e29feca63d2","type":"BoxAnnotation"},{"id":"3df41605-9fba-40d8-86bb-4971ef0ac67c","type":"Legend"},{"id":"6d17249c-9696-4c57-814c-e07fcba42a96","type":"GlyphRenderer"},{"id":"ed59cd0e-8894-41f1-a48a-5f7a4c88c6ed","type":"GlyphRenderer"},{"id":"f45c7b73-fad4-4acb-b1ad-8b3ed1e0fcb2","type":"GlyphRenderer"}],"title":{"id":"cab85b81-e7db-4f7b-a3ac-1734f990ff9e","type":"Title"},"tool_events":{"id":"30c90173-81f0-47ca-877c-6c6848aebf36","type":"ToolEvents"},"toolbar":{"id":"ca681ecd-4719-4397-96dc-1986a57e5d53","type":"Toolbar"},"toolbar_location":null,"x_range":{"id":"b5d18a40-eaf0-4995-ab94-30e91d9620c7","type":"DataRange1d"},"x_scale":{"id":"83955f72-021e-4038-ae89-1b6f25fb0c08","type":"LinearScale"},"y_range":{"id":"976385a8-d4d8-4a4e-8589-c4594fee92bc","type":"DataRange1d"},"y_scale":{"id":"7eb5bb32-7a67-42fb-9d36-2ab66e73912a","type":"LinearScale"}},"id":"26cfe895-d550-439a-9fd8-f8b33f7650c4","subtype":"Figure","type":"Plot"},{"attributes":{},"id":"30c90173-81f0-47ca-877c-6c6848aebf36","type":"ToolEvents"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"dd4c646b-94ab-4c76-88ce-f8a47591f0a5","type":"Line"},{"attributes":{"active_drag":"auto","active_inspect":"auto","active_scroll":"auto","active_tap":"auto","tools":[{"id":"dc08ebd3-6a70-4b66-9ed5-ab2033b02617","type":"PanTool"},{"id":"261f447f-6e57-4caa-9612-990ac53104cc","type":"WheelZoomTool"},{"id":"ef6a0b65-2aec-4574-8101-1c476620c1ed","type":"BoxZoomTool"},{"id":"4c7d084f-250c-4af4-beb5-bd802fa783d4","type":"SaveTool"},{"id":"6df2cb4a-cb98-4ae4-bb00-2a88c169f1d5","type":"ResetTool"},{"id":"217745ac-c1b5-4fac-a108-b550990c8efa","type":"HelpTool"}]},"id":"ca681ecd-4719-4397-96dc-1986a57e5d53","type":"Toolbar"},{"attributes":{"callback":null,"start":0},"id":"b5d18a40-eaf0-4995-ab94-30e91d9620c7","type":"DataRange1d"},{"attributes":{},"id":"75726b0c-3595-4721-8cfa-a3cd9858914b","type":"BasicTickFormatter"},{"attributes":{},"id":"83955f72-021e-4038-ae89-1b6f25fb0c08","type":"LinearScale"},{"attributes":{"callback":null,"end":5234.083519499139,"start":0},"id":"976385a8-d4d8-4a4e-8589-c4594fee92bc","type":"DataRange1d"},{"attributes":{},"id":"7eb5bb32-7a67-42fb-9d36-2ab66e73912a","type":"LinearScale"},{"attributes":{"plot":{"id":"26cfe895-d550-439a-9fd8-f8b33f7650c4","subtype":"Figure","type":"Plot"},"ticker":{"id":"95d5e569-513a-4702-866c-58118ee9081d","type":"BasicTicker"}},"id":"53d3e7d1-ec27-4790-acd8-6afb80e548b9","type":"Grid"},{"attributes":{"axis_label":"cores","formatter":{"id":"312e64ce-921b-4a25-8103-2b92e4211342","type":"BasicTickFormatter"},"plot":{"id":"26cfe895-d550-439a-9fd8-f8b33f7650c4","subtype":"Figure","type":"Plot"},"ticker":{"id":"deecb97b-ec64-461a-bed8-a7fec272a973","type":"FixedTicker"}},"id":"94a8080b-1f64-45a9-840b-0d89d2ad1ae8","type":"LinearAxis"},{"attributes":{},"id":"95d5e569-513a-4702-866c-58118ee9081d","type":"BasicTicker"},{"attributes":{"axis_label":"MB/s","formatter":{"id":"75726b0c-3595-4721-8cfa-a3cd9858914b","type":"BasicTickFormatter"},"plot":{"id":"26cfe895-d550-439a-9fd8-f8b33f7650c4","subtype":"Figure","type":"Plot"},"ticker":{"id":"7819956c-57d5-445d-b5f5-0f78a771a2f3","type":"BasicTicker"}},"id":"ddbfd72e-8a85-4061-9a3b-e19d9aaa07e2","type":"LinearAxis"},{"attributes":{},"id":"7819956c-57d5-445d-b5f5-0f78a771a2f3","type":"BasicTicker"},{"attributes":{"dimension":1,"plot":{"id":"26cfe895-d550-439a-9fd8-f8b33f7650c4","subtype":"Figure","type":"Plot"},"ticker":{"id":"7819956c-57d5-445d-b5f5-0f78a771a2f3","type":"BasicTicker"}},"id":"d82b6b00-db6f-45a4-b0e7-968b580bb9d4","type":"Grid"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"s8bqjTzBiUBTem6Z0ZyGQFtOQa+MMZNAS+oBg8Twl0Arqf2CF8SjQIPxxSmeUatANdBkm/kbrkBhrYhhFXK0QGeZb4e3brJA","dtype":"float64","shape":[9]}}},"id":"4a0b502c-d137-472f-afc6-726e1982cc6c","type":"ColumnDataSource"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"13b721fb-b1f2-4009-8290-e29352972ba3","type":"Line"},{"attributes":{},"id":"312e64ce-921b-4a25-8103-2b92e4211342","type":"BasicTickFormatter"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"fe061748-d906-414d-b8a1-8e29feca63d2","type":"BoxAnnotation"},{"attributes":{"plot":{"id":"26cfe895-d550-439a-9fd8-f8b33f7650c4","subtype":"Figure","type":"Plot"}},"id":"dc08ebd3-6a70-4b66-9ed5-ab2033b02617","type":"PanTool"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[2,4,8,16,32,64,128,256,512],"y":{"__ndarray__":"BSBNWCE2dEC9BpWUVImFQDzSCcCUkJBAVLgx0fyxnkCFGyLiy4+jQE7fanACZ6xAEeby5mMgsEDryZN7k+GwQFqGlput7rBA","dtype":"float64","shape":[9]}}},"id":"529ef473-0ccc-45f4-861a-7b66c8fc4d8d","type":"ColumnDataSource"},{"attributes":{"plot":{"id":"26cfe895-d550-439a-9fd8-f8b33f7650c4","subtype":"Figure","type":"Plot"}},"id":"261f447f-6e57-4caa-9612-990ac53104cc","type":"WheelZoomTool"},{"attributes":{"overlay":{"id":"fe061748-d906-414d-b8a1-8e29feca63d2","type":"BoxAnnotation"},"plot":{"id":"26cfe895-d550-439a-9fd8-f8b33f7650c4","subtype":"Figure","type":"Plot"}},"id":"ef6a0b65-2aec-4574-8101-1c476620c1ed","type":"BoxZoomTool"},{"attributes":{"plot":{"id":"26cfe895-d550-439a-9fd8-f8b33f7650c4","subtype":"Figure","type":"Plot"}},"id":"4c7d084f-250c-4af4-beb5-bd802fa783d4","type":"SaveTool"},{"attributes":{"plot":{"id":"26cfe895-d550-439a-9fd8-f8b33f7650c4","subtype":"Figure","type":"Plot"}},"id":"6df2cb4a-cb98-4ae4-bb00-2a88c169f1d5","type":"ResetTool"},{"attributes":{"plot":{"id":"26cfe895-d550-439a-9fd8-f8b33f7650c4","subtype":"Figure","type":"Plot"}},"id":"217745ac-c1b5-4fac-a108-b550990c8efa","type":"HelpTool"},{"attributes":{"data_source":{"id":"fb7edcec-75b7-47e2-85aa-644370275686","type":"ColumnDataSource"},"glyph":{"id":"35ca343c-672d-4ef4-b159-836e4a5d8148","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"47e5a2c0-2ed6-4b5b-9195-d1deb97741dd","type":"Line"},"selection_glyph":null},"id":"f45c7b73-fad4-4acb-b1ad-8b3ed1e0fcb2","type":"GlyphRenderer"},{"attributes":{"data_source":{"id":"a1335509-0bcd-498a-bb3c-5af46436f736","type":"ColumnDataSource"},"glyph":{"id":"13b721fb-b1f2-4009-8290-e29352972ba3","type":"Line"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"dd4c646b-94ab-4c76-88ce-f8a47591f0a5","type":"Line"},"selection_glyph":null},"id":"6d17249c-9696-4c57-814c-e07fcba42a96","type":"GlyphRenderer"},{"attributes":{"fill_color":{"value":"#1f77b4"},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"9d7e0692-1553-4f44-a38e-7d9449d4d1ce","type":"Circle"},{"attributes":{"data_source":{"id":"4a0b502c-d137-472f-afc6-726e1982cc6c","type":"ColumnDataSource"},"glyph":{"id":"9d7e0692-1553-4f44-a38e-7d9449d4d1ce","type":"Circle"},"hover_glyph":null,"muted_glyph":null,"nonselection_glyph":{"id":"a2d1cdcc-1cfa-4746-be4c-ed27758443dc","type":"Circle"},"selection_glyph":null},"id":"ed59cd0e-8894-41f1-a48a-5f7a4c88c6ed","type":"GlyphRenderer"},{"attributes":{"fill_alpha":{"value":0.1},"fill_color":{"value":"#1f77b4"},"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"a2d1cdcc-1cfa-4746-be4c-ed27758443dc","type":"Circle"},{"attributes":{"child":{"id":"1d34c2b9-9f2b-48c4-b9c6-c2c8e72691c5","subtype":"Figure","type":"Plot"},"title":"log"},"id":"46d7408d-edb4-4a99-a135-415f0feaf6aa","type":"Panel"},{"attributes":{"callback":null,"column_names":["x","y"],"data":{"x":[0,512],"y":[0,210983.56929545625]}},"id":"fb7edcec-75b7-47e2-85aa-644370275686","type":"ColumnDataSource"},{"attributes":{"line_color":{"value":"gray"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"35ca343c-672d-4ef4-b159-836e4a5d8148","type":"Line"},{"attributes":{"line_alpha":{"value":0.1},"line_color":{"value":"#1f77b4"},"line_dash":[6],"x":{"field":"x"},"y":{"field":"y"}},"id":"47e5a2c0-2ed6-4b5b-9195-d1deb97741dd","type":"Line"},{"attributes":{"children":[{"id":"fb713db1-aa51-4406-b604-cd896be47fdf","type":"Row"},{"id":"5329528e-3df4-406e-b886-28d673e4447c","type":"Row"},{"id":"2562d80e-1e0a-4237-9f75-df3916b837df","type":"Row"}],"sizing_mode":"scale_width"},"id":"344a1d6e-41e7-4fa4-86a6-6537dbb6c9da","type":"Column"},{"attributes":{"bottom_units":"screen","fill_alpha":{"value":0.5},"fill_color":{"value":"lightgrey"},"left_units":"screen","level":"overlay","line_alpha":{"value":1.0},"line_color":{"value":"black"},"line_dash":[4,4],"line_width":{"value":2},"plot":null,"render_mode":"css","right_units":"screen","top_units":"screen"},"id":"ab10fe90-cae8-447a-af95-b3c5e014c4af","type":"BoxAnnotation"},{"attributes":{"line_color":{"value":"#1f77b4"},"x":{"field":"x"},"y":{"field":"y"}},"id":"4b247019-ac32-4471-98a2-5ed1fc275453","type":"Line"}],"root_ids":["3328a090-74cd-4548-b396-7ab5adb364e6","ed39ed3e-c52c-4159-85ae-c61b8a19409c","8cef5b34-d5d8-47bf-9055-bdcedac89484","515cf591-ccc9-4d46-8771-c0b31a55cb97","c0b81eb9-68c0-4b71-926b-0b5bcab775b1"]},"title":"Bokeh Application","version":"0.12.6"}};
      var render_items = [{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"36dc768f-23a6-4b1c-8cbe-a0346af1d182","modelid":"59c15cdf-cad3-4afc-a12b-458d2a0a5b13"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"90a82d6b-b090-49df-8900-62ca8188d39a","modelid":"515cf591-ccc9-4d46-8771-c0b31a55cb97"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"11288c56-ea8f-45d3-9629-62c82b24eb54","modelid":"948f0ba1-2615-4ca3-9634-5eade5d4eab8"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"3c6ec7c3-ffd7-4ec9-8ef7-1e8b21a044f5","modelid":"aeaa4543-6cc0-4ed6-bf28-929d3bf8399f"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"733be4a9-6378-4112-bcaf-bfb1a4217c9c","modelid":"b175b9d7-0992-4013-a3da-e7da9e7115e2"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"30baf2d4-57bd-4393-8a2c-c9deb1a1edc9","modelid":"3328a090-74cd-4548-b396-7ab5adb364e6"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"bdce2a6b-8576-4724-8736-24f4b86d6487","modelid":"50a686fd-e2c5-440a-92af-bbfc9d851b96"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"d919e07f-2795-4bfd-9f97-d0aaf24c0a2a","modelid":"d5449472-9151-45a2-82ef-636c1e6e7fad"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"8f38e6fe-fab7-480b-9eb8-0f72781e24d7","modelid":"716ab0cd-3d4a-4000-afeb-483507e81870"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"88e05fb2-abac-4b1e-bcf9-a04d9148a6bf","modelid":"4311ab7e-c88e-42cf-b077-bfd6acbda2f0"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"7b3301e3-8beb-4da4-bbef-9eb52793f8c1","modelid":"d5eef40f-39b6-49cb-9230-302039c21841"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"a1a28d56-81f7-4580-8400-f7aec3cd4f59","modelid":"7fc598d6-b20a-457d-80b4-34bf7c5ca49e"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"f04e6ad2-5317-4dc8-babd-f3b8d85ae09c","modelid":"c0b81eb9-68c0-4b71-926b-0b5bcab775b1"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"ea30e0d3-0a1b-40bb-be2f-e0b54af96aab","modelid":"ed39ed3e-c52c-4159-85ae-c61b8a19409c"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"c6417fff-a706-4e33-9337-548f59f230f9","modelid":"bddfdd21-3692-4f09-a839-053ec2e8b96e"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"3d2dcdc7-c91f-4e16-9a6f-88b4837e6375","modelid":"067958e4-18bc-4159-852e-fc2a375a27cb"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"e2d30245-cca5-41bc-8691-59037c33c9f2","modelid":"4d110cea-f4d8-49a1-bd97-89adb074f930"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"055fa730-cc11-4b5d-8237-c067f0fcb5d5","modelid":"682f90be-e284-4f12-a195-8e4927a7b6dc"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"a46691fb-a1b1-47fb-9d8f-9a6933ba78b7","modelid":"bc2a925a-2e83-48cf-a2c4-b3b85a7804f9"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"ea096ed4-c553-404e-b18f-a1336a4d4156","modelid":"7d705d97-1eec-49b4-bda6-9eebdb487600"},{"docid":"c37a7732-0c94-4514-bc5b-189fa40a390f","elementid":"3bad34c6-9772-45c7-88d7-62c3eaa3fa97","modelid":"8cef5b34-d5d8-47bf-9055-bdcedac89484"}];

      Bokeh.embed.embed_items(docs_json, render_items);
    });
  };
  if (document.readyState != "loading") fn();
  else document.addEventListener("DOMContentLoaded", fn);
})();

&lt;/script&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/07/03/scaling/"/>
    <summary>This work is supported by Continuum Analytics
and the Data Driven Discovery Initiative from the Moore
Foundation.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2017-07-03T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/06/28/use-parquet/</id>
    <title>Use Apache Parquet</title>
    <updated>2017-06-28T00: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 Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;This is a tiny blogpost to encourage you to use
&lt;a class="reference external" href="http://parquet.apache.org/"&gt;Parquet&lt;/a&gt; instead of CSV for your dataframe
computations. I’ll use Dask.dataframe here but Pandas would work just as well.
I’ll also use my local laptop here, but Parquet is an excellent format to use
on a cluster.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2017/06/28/use-parquet.md&lt;/span&gt;, line 18)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="csv-is-convenient-but-slow"&gt;

&lt;p&gt;I have the NYC taxi cab dataset on my laptop stored as CSV&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;mrocklin@carbon:~/data/nyc/csv$ ls
yellow_tripdata_2015-01.csv  yellow_tripdata_2015-07.csv
yellow_tripdata_2015-02.csv  yellow_tripdata_2015-08.csv
yellow_tripdata_2015-03.csv  yellow_tripdata_2015-09.csv
yellow_tripdata_2015-04.csv  yellow_tripdata_2015-10.csv
yellow_tripdata_2015-05.csv  yellow_tripdata_2015-11.csv
yellow_tripdata_2015-06.csv  yellow_tripdata_2015-12.csv
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is a convenient format for humans because we can read it directly.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;mrocklin@carbon:~/data/nyc/csv$ head yellow_tripdata_2015-01.csv
VendorID,tpep_pickup_datetime,tpep_dropoff_datetime,passenger_count,trip_distance,pickup_longitude,pickup_latitude,RateCodeID,store_and_fwd_flag,dropoff_longitude,dropoff_latitude,payment_type,fare_amount,extra,mta_tax,tip_amount,tolls_amount,improvement_surcharge,total_amount
2,2015-01-15 19:05:39,2015-01-15
19:23:42,1,1.59,-73.993896484375,40.750110626220703,1,N,-73.974784851074219,40.750617980957031,1,12,1,0.5,3.25,0,0.3,17.05
1,2015-01-10 20:33:38,2015-01-10
20:53:28,1,3.30,-74.00164794921875,40.7242431640625,1,N,-73.994415283203125,40.759109497070313,1,14.5,0.5,0.5,2,0,0.3,17.8
1,2015-01-10 20:33:38,2015-01-10
20:43:41,1,1.80,-73.963340759277344,40.802787780761719,1,N,-73.951820373535156,40.824413299560547,2,9.5,0.5,0.5,0,0,0.3,10.8
1,2015-01-10 20:33:39,2015-01-10
20:35:31,1,.50,-74.009086608886719,40.713817596435547,1,N,-74.004325866699219,40.719985961914063,2,3.5,0.5,0.5,0,0,0.3,4.8
1,2015-01-10 20:33:39,2015-01-10
20:52:58,1,3.00,-73.971176147460938,40.762428283691406,1,N,-74.004180908203125,40.742652893066406,2,15,0.5,0.5,0,0,0.3,16.3
1,2015-01-10 20:33:39,2015-01-10
20:53:52,1,9.00,-73.874374389648438,40.7740478515625,1,N,-73.986976623535156,40.758193969726563,1,27,0.5,0.5,6.7,5.33,0.3,40.33
1,2015-01-10 20:33:39,2015-01-10
20:58:31,1,2.20,-73.9832763671875,40.726009368896484,1,N,-73.992469787597656,40.7496337890625,2,14,0.5,0.5,0,0,0.3,15.3
1,2015-01-10 20:33:39,2015-01-10
20:42:20,3,.80,-74.002662658691406,40.734142303466797,1,N,-73.995010375976563,40.726325988769531,1,7,0.5,0.5,1.66,0,0.3,9.96
1,2015-01-10 20:33:39,2015-01-10
21:11:35,3,18.20,-73.783042907714844,40.644355773925781,2,N,-73.987594604492187,40.759357452392578,2,52,0,0.5,0,5.33,0.3,58.13
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We can use tools like Pandas or Dask.dataframe to read in all of this data.
Because the data is large-ish, I’ll use Dask.dataframe&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;mrocklin@carbon:~/data/nyc/csv$ du -hs .
22G .
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;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.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="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="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;dd&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;yellow_tripdata_2015-*.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;340&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;12&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;352&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;377&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;3&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;head&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;3&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt;
&lt;span class="n"&gt;VendorID&lt;/span&gt; &lt;span class="n"&gt;tpep_pickup_datetime&lt;/span&gt; &lt;span class="n"&gt;tpep_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="mi"&gt;2&lt;/span&gt;  &lt;span class="mi"&gt;2015&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;15&lt;/span&gt; &lt;span class="mi"&gt;19&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;39&lt;/span&gt;   &lt;span class="mi"&gt;2015&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;15&lt;/span&gt; &lt;span class="mi"&gt;19&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;42&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;1&lt;/span&gt;  &lt;span class="mi"&gt;2015&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;10&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;33&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;38&lt;/span&gt;   &lt;span class="mi"&gt;2015&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;10&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;53&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;28&lt;/span&gt;                &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;         &lt;span class="mi"&gt;1&lt;/span&gt;  &lt;span class="mi"&gt;2015&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;10&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;33&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;38&lt;/span&gt;   &lt;span class="mi"&gt;2015&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;10&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;43&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;41&lt;/span&gt;                &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;         &lt;span class="mi"&gt;1&lt;/span&gt;  &lt;span class="mi"&gt;2015&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;10&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;33&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;39&lt;/span&gt;   &lt;span class="mi"&gt;2015&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;10&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;35&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;31&lt;/span&gt;                &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;4&lt;/span&gt;         &lt;span class="mi"&gt;1&lt;/span&gt;  &lt;span class="mi"&gt;2015&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;10&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;33&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;39&lt;/span&gt;   &lt;span class="mi"&gt;2015&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;10&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;52&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;58&lt;/span&gt;                &lt;span class="mi"&gt;1&lt;/span&gt;

   &lt;span class="n"&gt;trip_distance&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;RateCodeID&lt;/span&gt;  \
&lt;span class="mi"&gt;0&lt;/span&gt;           &lt;span class="mf"&gt;1.59&lt;/span&gt;        &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.993896&lt;/span&gt;        &lt;span class="mf"&gt;40.750111&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;3.30&lt;/span&gt;        &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;74.001648&lt;/span&gt;        &lt;span class="mf"&gt;40.724243&lt;/span&gt;           &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;           &lt;span class="mf"&gt;1.80&lt;/span&gt;        &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.963341&lt;/span&gt;        &lt;span class="mf"&gt;40.802788&lt;/span&gt;           &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;           &lt;span class="mf"&gt;0.50&lt;/span&gt;        &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;74.009087&lt;/span&gt;        &lt;span class="mf"&gt;40.713818&lt;/span&gt;           &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;4&lt;/span&gt;           &lt;span class="mf"&gt;3.00&lt;/span&gt;        &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.971176&lt;/span&gt;        &lt;span class="mf"&gt;40.762428&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;dropoff_longitude&lt;/span&gt;  &lt;span class="n"&gt;dropoff_latitude&lt;/span&gt;  &lt;span class="n"&gt;payment_type&lt;/span&gt; \
&lt;span class="mi"&gt;0&lt;/span&gt;                  &lt;span class="n"&gt;N&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.974785&lt;/span&gt;         &lt;span class="mf"&gt;40.750618&lt;/span&gt;             &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                  &lt;span class="n"&gt;N&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.994415&lt;/span&gt;         &lt;span class="mf"&gt;40.759109&lt;/span&gt;             &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;                  &lt;span class="n"&gt;N&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.951820&lt;/span&gt;         &lt;span class="mf"&gt;40.824413&lt;/span&gt;             &lt;span class="mi"&gt;2&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;                  &lt;span class="n"&gt;N&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;74.004326&lt;/span&gt;         &lt;span class="mf"&gt;40.719986&lt;/span&gt;             &lt;span class="mi"&gt;2&lt;/span&gt;
&lt;span class="mi"&gt;4&lt;/span&gt;                  &lt;span class="n"&gt;N&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;74.004181&lt;/span&gt;         &lt;span class="mf"&gt;40.742653&lt;/span&gt;             &lt;span class="mi"&gt;2&lt;/span&gt;

   &lt;span class="n"&gt;fare_amount&lt;/span&gt;  &lt;span class="n"&gt;extra&lt;/span&gt;  &lt;span class="n"&gt;mta_tax&lt;/span&gt;  &lt;span class="n"&gt;tip_amount&lt;/span&gt;  &lt;span class="n"&gt;tolls_amount&lt;/span&gt;  \
&lt;span class="mi"&gt;0&lt;/span&gt;         &lt;span class="mf"&gt;12.0&lt;/span&gt;    &lt;span class="mf"&gt;1.0&lt;/span&gt;      &lt;span class="mf"&gt;0.5&lt;/span&gt;        &lt;span class="mf"&gt;3.25&lt;/span&gt;           &lt;span class="mf"&gt;0.0&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;         &lt;span class="mf"&gt;14.5&lt;/span&gt;    &lt;span class="mf"&gt;0.5&lt;/span&gt;      &lt;span class="mf"&gt;0.5&lt;/span&gt;        &lt;span class="mf"&gt;2.00&lt;/span&gt;           &lt;span class="mf"&gt;0.0&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;          &lt;span class="mf"&gt;9.5&lt;/span&gt;    &lt;span class="mf"&gt;0.5&lt;/span&gt;      &lt;span class="mf"&gt;0.5&lt;/span&gt;        &lt;span class="mf"&gt;0.00&lt;/span&gt;           &lt;span class="mf"&gt;0.0&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;          &lt;span class="mf"&gt;3.5&lt;/span&gt;    &lt;span class="mf"&gt;0.5&lt;/span&gt;      &lt;span class="mf"&gt;0.5&lt;/span&gt;        &lt;span class="mf"&gt;0.00&lt;/span&gt;           &lt;span class="mf"&gt;0.0&lt;/span&gt;
&lt;span class="mi"&gt;4&lt;/span&gt;         &lt;span class="mf"&gt;15.0&lt;/span&gt;    &lt;span class="mf"&gt;0.5&lt;/span&gt;      &lt;span class="mf"&gt;0.5&lt;/span&gt;        &lt;span class="mf"&gt;0.00&lt;/span&gt;           &lt;span class="mf"&gt;0.0&lt;/span&gt;

   &lt;span class="n"&gt;improvement_surcharge&lt;/span&gt;  &lt;span class="n"&gt;total_amount&lt;/span&gt;
&lt;span class="mi"&gt;0&lt;/span&gt;                    &lt;span class="mf"&gt;0.3&lt;/span&gt;         &lt;span class="mf"&gt;17.05&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                    &lt;span class="mf"&gt;0.3&lt;/span&gt;         &lt;span class="mf"&gt;17.80&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;                    &lt;span class="mf"&gt;0.3&lt;/span&gt;         &lt;span class="mf"&gt;10.80&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;                    &lt;span class="mf"&gt;0.3&lt;/span&gt;          &lt;span class="mf"&gt;4.80&lt;/span&gt;
&lt;span class="mi"&gt;4&lt;/span&gt;                    &lt;span class="mf"&gt;0.3&lt;/span&gt;         &lt;span class="mf"&gt;16.30&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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.diagnostics&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;ProgressBar&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;ProgressBar&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;passenger_count&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="p"&gt;[&lt;/span&gt;&lt;span class="c1"&gt;########################################] | 100% Completed |&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="nb"&gt;min&lt;/span&gt; &lt;span class="mf"&gt;58.8&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;6&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="mi"&gt;245566747&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We were able to ask questions about this data (and learn that 250 million
people rode cabs in 2016) even though it is too large to fit into memory. This
is because Dask is able to operate lazily from disk. It reads in the data on
an as-needed basis and then forgets it when it no longer needs it. This takes
a while (4 minutes) but does just work.&lt;/p&gt;
&lt;p&gt;However, when we read this data many times from disk we start to become
frustrated by this four minute cost. In Pandas we suffered this cost once as
we moved data from disk to memory. On larger datasets when we don’t have
enough RAM we suffer this cost many times.&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/2017/06/28/use-parquet.md&lt;/span&gt;, line 130)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="parquet-is-faster"&gt;
&lt;h1&gt;Parquet is faster&lt;/h1&gt;
&lt;p&gt;Lets try this same process with Parquet. I happen to have the same exact data
stored in Parquet format on my hard drive.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;mrocklin@carbon:~/data/nyc$ du -hs nyc-2016.parquet/
17G nyc-2016.parquet/
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;It is stored as a bunch of individual files, but we don’t actually care about
that. We’ll always refer to the directory as the dataset. These files are
stored in binary format. We can’t read them as humans&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;mrocklin&lt;/span&gt;&lt;span class="nd"&gt;@carbon&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="o"&gt;~/&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;nyc&lt;/span&gt;&lt;span class="err"&gt;$&lt;/span&gt; &lt;span class="n"&gt;head&lt;/span&gt; &lt;span class="n"&gt;nyc&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;2016.&lt;/span&gt;&lt;span class="n"&gt;parquet&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;part&lt;/span&gt;&lt;span class="mf"&gt;.0&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;parquet&lt;/span&gt;
&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;a&lt;/span&gt; &lt;span class="n"&gt;bunch&lt;/span&gt; &lt;span class="n"&gt;of&lt;/span&gt; &lt;span class="n"&gt;illegible&lt;/span&gt; &lt;span class="nb"&gt;bytes&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;But computers are much more able to both read and navigate this data. Lets do
the same experiment from before:&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.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;nyc-2016.parquet/&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="n"&gt;df&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="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;tpep_pickup_datetime&lt;/span&gt;  &lt;span class="n"&gt;VendorID&lt;/span&gt; &lt;span class="n"&gt;tpep_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="mi"&gt;2015&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;00&lt;/span&gt;&lt;span class="p"&gt;:&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;00&lt;/span&gt;         &lt;span class="mi"&gt;2&lt;/span&gt;   &lt;span class="mi"&gt;2015&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;00&lt;/span&gt;&lt;span class="p"&gt;:&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;00&lt;/span&gt;                &lt;span class="mi"&gt;3&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;  &lt;span class="mi"&gt;2015&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;00&lt;/span&gt;&lt;span class="p"&gt;:&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;00&lt;/span&gt;         &lt;span class="mi"&gt;2&lt;/span&gt;   &lt;span class="mi"&gt;2015&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;00&lt;/span&gt;&lt;span class="p"&gt;:&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;00&lt;/span&gt;                &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;  &lt;span class="mi"&gt;2015&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;00&lt;/span&gt;&lt;span class="p"&gt;:&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;00&lt;/span&gt;         &lt;span class="mi"&gt;1&lt;/span&gt;   &lt;span class="mi"&gt;2015&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;00&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;26&lt;/span&gt;                &lt;span class="mi"&gt;5&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;  &lt;span class="mi"&gt;2015&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;00&lt;/span&gt;&lt;span class="p"&gt;:&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;01&lt;/span&gt;         &lt;span class="mi"&gt;1&lt;/span&gt;   &lt;span class="mi"&gt;2015&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;00&lt;/span&gt;&lt;span class="p"&gt;:&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;49&lt;/span&gt;                &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;4&lt;/span&gt;  &lt;span class="mi"&gt;2015&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;00&lt;/span&gt;&lt;span class="p"&gt;:&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;03&lt;/span&gt;         &lt;span class="mi"&gt;2&lt;/span&gt;   &lt;span class="mi"&gt;2015&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;00&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;21&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;2&lt;/span&gt;

    &lt;span class="n"&gt;trip_distance&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;RateCodeID&lt;/span&gt;  \
&lt;span class="mi"&gt;0&lt;/span&gt;           &lt;span class="mf"&gt;1.56&lt;/span&gt;        &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;74.001320&lt;/span&gt;        &lt;span class="mf"&gt;40.729057&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;1.68&lt;/span&gt;        &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.991547&lt;/span&gt;        &lt;span class="mf"&gt;40.750069&lt;/span&gt;           &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;           &lt;span class="mf"&gt;4.00&lt;/span&gt;        &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.971436&lt;/span&gt;        &lt;span class="mf"&gt;40.760201&lt;/span&gt;           &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;           &lt;span class="mf"&gt;0.80&lt;/span&gt;        &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.860847&lt;/span&gt;        &lt;span class="mf"&gt;40.757294&lt;/span&gt;           &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;4&lt;/span&gt;           &lt;span class="mf"&gt;2.57&lt;/span&gt;        &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.969017&lt;/span&gt;        &lt;span class="mf"&gt;40.754269&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;dropoff_longitude&lt;/span&gt;  &lt;span class="n"&gt;dropoff_latitude&lt;/span&gt;  &lt;span class="n"&gt;payment_type&lt;/span&gt;  \
&lt;span class="mi"&gt;0&lt;/span&gt;                  &lt;span class="n"&gt;N&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;74.010208&lt;/span&gt;         &lt;span class="mf"&gt;40.719662&lt;/span&gt;             &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                  &lt;span class="n"&gt;N&lt;/span&gt;           &lt;span class="mf"&gt;0.000000&lt;/span&gt;          &lt;span class="mf"&gt;0.000000&lt;/span&gt;             &lt;span class="mi"&gt;2&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;                  &lt;span class="n"&gt;N&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.921181&lt;/span&gt;         &lt;span class="mf"&gt;40.768269&lt;/span&gt;             &lt;span class="mi"&gt;2&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;                  &lt;span class="n"&gt;N&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.868111&lt;/span&gt;         &lt;span class="mf"&gt;40.752285&lt;/span&gt;             &lt;span class="mi"&gt;2&lt;/span&gt;
&lt;span class="mi"&gt;4&lt;/span&gt;                  &lt;span class="n"&gt;N&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.994133&lt;/span&gt;         &lt;span class="mf"&gt;40.761600&lt;/span&gt;             &lt;span class="mi"&gt;2&lt;/span&gt;

   &lt;span class="n"&gt;fare_amount&lt;/span&gt;  &lt;span class="n"&gt;extra&lt;/span&gt;  &lt;span class="n"&gt;mta_tax&lt;/span&gt;  &lt;span class="n"&gt;tip_amount&lt;/span&gt;  &lt;span class="n"&gt;tolls_amount&lt;/span&gt;  \
&lt;span class="mi"&gt;0&lt;/span&gt;          &lt;span class="mf"&gt;7.5&lt;/span&gt;    &lt;span class="mf"&gt;0.5&lt;/span&gt;      &lt;span class="mf"&gt;0.5&lt;/span&gt;         &lt;span class="mf"&gt;0.0&lt;/span&gt;           &lt;span class="mf"&gt;0.0&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;         &lt;span class="mf"&gt;10.0&lt;/span&gt;    &lt;span class="mf"&gt;0.0&lt;/span&gt;      &lt;span class="mf"&gt;0.5&lt;/span&gt;         &lt;span class="mf"&gt;0.0&lt;/span&gt;           &lt;span class="mf"&gt;0.0&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;         &lt;span class="mf"&gt;13.5&lt;/span&gt;    &lt;span class="mf"&gt;0.5&lt;/span&gt;      &lt;span class="mf"&gt;0.5&lt;/span&gt;         &lt;span class="mf"&gt;0.0&lt;/span&gt;           &lt;span class="mf"&gt;0.0&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;          &lt;span class="mf"&gt;5.0&lt;/span&gt;    &lt;span class="mf"&gt;0.5&lt;/span&gt;      &lt;span class="mf"&gt;0.5&lt;/span&gt;         &lt;span class="mf"&gt;0.0&lt;/span&gt;           &lt;span class="mf"&gt;0.0&lt;/span&gt;
&lt;span class="mi"&gt;4&lt;/span&gt;         &lt;span class="mf"&gt;14.5&lt;/span&gt;    &lt;span class="mf"&gt;0.5&lt;/span&gt;      &lt;span class="mf"&gt;0.5&lt;/span&gt;         &lt;span class="mf"&gt;0.0&lt;/span&gt;           &lt;span class="mf"&gt;0.0&lt;/span&gt;

   &lt;span class="n"&gt;improvement_surcharge&lt;/span&gt;  &lt;span class="n"&gt;total_amount&lt;/span&gt;
&lt;span class="mi"&gt;0&lt;/span&gt;                    &lt;span class="mf"&gt;0.3&lt;/span&gt;           &lt;span class="mf"&gt;8.8&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                    &lt;span class="mf"&gt;0.3&lt;/span&gt;          &lt;span class="mf"&gt;10.8&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;                    &lt;span class="mf"&gt;0.0&lt;/span&gt;          &lt;span class="mf"&gt;14.5&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;                    &lt;span class="mf"&gt;0.0&lt;/span&gt;           &lt;span class="mf"&gt;6.3&lt;/span&gt;
&lt;span class="mi"&gt;4&lt;/span&gt;                    &lt;span class="mf"&gt;0.3&lt;/span&gt;          &lt;span class="mf"&gt;15.8&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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.diagnostics&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;ProgressBar&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;ProgressBar&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;passenger_count&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="p"&gt;[&lt;/span&gt;&lt;span class="c1"&gt;########################################] | 100% Completed |&lt;/span&gt;
&lt;span class="mf"&gt;2.8&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;6&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="mi"&gt;245566747&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Same values, but now our computation happens in three seconds, rather than four
minutes. We’re cheating a little bit here (pulling out the passenger count
column is especially easy for Parquet) but generally Parquet will be &lt;em&gt;much&lt;/em&gt;
faster than CSV. This lets us work from disk comfortably without worrying
about how much memory we have.&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/2017/06/28/use-parquet.md&lt;/span&gt;, line 210)&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="convert"&gt;
&lt;h1&gt;Convert&lt;/h1&gt;
&lt;p&gt;So do yourself a favor and convert your 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="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.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
&lt;span class="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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;csv/yellow_tripdata_2015-*.csv&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;dask.diagnostics&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;ProgressBar&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;ProgressBar&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;yellow_tripdata.parquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="c1"&gt;############                            ] | 30% Completed |  1min 54.7s&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If you want to be more clever you can specify dtypes and compression when
converting. This can definitely help give you significantly greater speedups,
but just using the default settings will still be a large improvement.&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/2017/06/28/use-parquet.md&lt;/span&gt;, line 227)&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="advantages"&gt;
&lt;h1&gt;Advantages&lt;/h1&gt;
&lt;p&gt;Parquet enables the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Binary representation of data, allowing for speedy conversion of
bytes-on-disk to bytes-in-memory&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Columnar storage, meaning that you can load in as few columns as you need
without loading the entire dataset&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Row-chunked storage so that you can pull out data from a particular range
without touching the others&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Per-chunk statistics so that you can find subsets quickly&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Compression&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/2017/06/28/use-parquet.md&lt;/span&gt;, line 240)&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="parquet-versions"&gt;
&lt;h1&gt;Parquet Versions&lt;/h1&gt;
&lt;p&gt;There are two nice Python packages with support for the Parquet format:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://arrow.apache.org/docs/python/parquet.html"&gt;pyarrow&lt;/a&gt;:
Python bindings for the Apache Arrow and Apache Parquet C++ libraries&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://fastparquet.readthedocs.io/en/latest/"&gt;fastparquet&lt;/a&gt;: a direct
NumPy + Numba implementation of the Parquet format&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Both are good. Both can do most things. Each has separate strengths. The
code above used &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;fastparquet&lt;/span&gt;&lt;/code&gt; by default but you can change this in Dask with
the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;engine='arrow'&lt;/span&gt;&lt;/code&gt; keyword if desired.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/06/28/use-parquet/"/>
    <summary>This work is supported by Continuum Analytics
and the Data Driven Discovery Initiative from the Moore
Foundation.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2017-06-28T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/06/15/dask-0.15.0/</id>
    <title>Dask Release 0.15.0</title>
    <updated>2017-06-15T00: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 Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;I’m pleased to announce the release of Dask version 0.15.0. This release
contains performance and stability enhancements as well as some breaking
changes. This blogpost outlines notable changes since the last release on May
5th.&lt;/p&gt;
&lt;p&gt;As always you can conda install Dask:&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 distributed
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or pip install from PyPI&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[complete] --upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Conda packages are available both on the defaults and conda-forge channels.&lt;/p&gt;
&lt;p&gt;Full changelogs are available here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/blob/master/docs/source/changelog.rst"&gt;dask/dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/docs/source/changelog.rst"&gt;dask/distributed&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Some notable changes follow.&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/2017/06/15/dask-0.15.0.md&lt;/span&gt;, line 34)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="numpy-ufuncs-operate-as-dask-array-ufuncs"&gt;

&lt;p&gt;Thanks to recent changes in NumPy 1.13.0, NumPy ufuncs now operate as
Dask.array ufuncs. Previously they would convert their arguments into Numpy
arrays and then operate concretely.&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;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="kn"&gt;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="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;10&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;# Before&lt;/span&gt;
&lt;span class="o"&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;negative&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;array&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="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="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="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&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="o"&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="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="mi"&gt;7&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;8&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;9&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;

&lt;span class="c1"&gt;# Now&lt;/span&gt;
&lt;span class="o"&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;negative&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;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;negative&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;10&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="n"&gt;int64&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="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;&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;To celebrate this change we’ve also improved support for more of the NumPy
ufunc and reduction API, such as support for out parameters. This means that a
non-trivial subset of the actual NumPy API works directly out-of-the box with
dask.arrays. This makes it easier to write code that seamlessly works with
either array type.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: the ufunc feature requires that you update NumPy to 1.13.0 or later.
Packages are available through PyPI and conda on the defaults and conda-forge
channels.&lt;/em&gt;&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/numpy/numpy/pull/8247"&gt;numpy/numpy #8247&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/2438"&gt;dask/dask #2438&lt;/a&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/2017/06/15/dask-0.15.0.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="asynchronous-clients"&gt;
&lt;h1&gt;Asynchronous Clients&lt;/h1&gt;
&lt;p&gt;The Dask.distributed API is capable of operating within a Tornado or Asyncio
event loop, which can be useful when integrating with other concurrent systems
like web servers or when building some more advanced algorithms in machine
learning and other fields. The API to do this used to be somewhat hidden and
only known to a few and used underscores to signify that methods were
asynchronous.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Before&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;start&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_start&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&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="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;args&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_gather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;These methods are still around, but the process of starting the client has
changed and we now recommend using the fully public methods even in
asynchronous situations (these used to block).&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Now&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;asynchronous&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&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="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;args&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;gather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# no longer use the underscore&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You can also await futures directly:&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;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You can use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;yield&lt;/span&gt;&lt;/code&gt; instead of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;await&lt;/span&gt;&lt;/code&gt; if you prefer Python 2.&lt;/p&gt;
&lt;p&gt;More information is available at &lt;a class="reference external" href="https://distributed.readthedocs.org/en/latest/asynchronous.html"&gt;https://distributed.readthedocs.org/en/latest/asynchronous.html&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/2017/06/15/dask-0.15.0.md&lt;/span&gt;, line 108)&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="single-threaded-scheduler-moves-from-dask-async-to-dask-local"&gt;
&lt;h1&gt;Single-threaded scheduler moves from dask.async to dask.local&lt;/h1&gt;
&lt;p&gt;The single-machine scheduler used to live in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.async&lt;/span&gt;&lt;/code&gt; module. With
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;async&lt;/span&gt;&lt;/code&gt; becoming a keyword since Python 3.5 we’re forced to rename this. You
can now find the code in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.local&lt;/span&gt;&lt;/code&gt;. This will particularly affect anyone
who was using the single-threaded scheduler, previously known as
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.async.get_sync&lt;/span&gt;&lt;/code&gt;. The term &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.get&lt;/span&gt;&lt;/code&gt; can be used to reliably refer to
the single-threaded base scheduler across versions.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/issues/1834"&gt;dask/dask #1834&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/2318"&gt;dask/dask #2318&lt;/a&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/2017/06/15/dask-0.15.0.md&lt;/span&gt;, line 120)&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="retired-the-distributed-collections-module"&gt;
&lt;h1&gt;Retired the distributed.collections module&lt;/h1&gt;
&lt;p&gt;Early blogposts referred to functions like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;futures_to_dask_array&lt;/span&gt;&lt;/code&gt; which
resided in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed.collections&lt;/span&gt;&lt;/code&gt; module. These have since been
entirely replaced by better interactions between Futures and Delayed objects.
This module has been removed entirely.&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/2017/06/15/dask-0.15.0.md&lt;/span&gt;, line 127)&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="always-create-new-directories-with-the-local-directory-flag"&gt;
&lt;h1&gt;Always create new directories with the –local-directory flag&lt;/h1&gt;
&lt;p&gt;Dask workers create a directory where they can place temporary files.
Typically this goes into your operating system’s temporary directory (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;/tmp&lt;/span&gt;&lt;/code&gt; on
Linux and Mac).&lt;/p&gt;
&lt;p&gt;Some users on network file systems specify this directory explicitly with the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-worker&lt;/span&gt; &lt;span class="pre"&gt;...&lt;/span&gt; &lt;span class="pre"&gt;--local-directory&lt;/span&gt;&lt;/code&gt; option, pointing to some other better place
like a local SSD drive. Previously Dask would dump files into the provided
directory. Now it will create a new subdirectory and place files there. This
tends to be much more convenient for users on network file systems.&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ dask-worker scheduler-address:8786 --local-directory /scratch
$ ls /scratch
worker-1234/
$ ls /scratch/worker-1234/
user-script.py disk-storage/ ...
&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/2017/06/15/dask-0.15.0.md&lt;/span&gt;, line 145)&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-map-no-longer-automatically-expands-tuples"&gt;
&lt;h1&gt;Bag.map no longer automatically expands tuples&lt;/h1&gt;
&lt;p&gt;Previously the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map&lt;/span&gt;&lt;/code&gt; method would inspect functions and automatically expand
tuples to fill arguments:&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;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="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_sequence&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;10&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;20&lt;/span&gt;&lt;span class="p"&gt;),&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;30&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;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;y&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;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;y&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="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;22&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;33&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;While convenient, this behavior gave rise to corner cases and stopped us from
being able to support multi-bag mapping functions. It has since been removed.
As an advantage though, you can now map two co-partitioned bags together.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;a&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;db&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_sequence&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;span class="n"&gt;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_sequence&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="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;30&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;db&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;y&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;y&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;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&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;22&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;33&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/2339"&gt;dask/dask #2339&lt;/a&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/2017/06/15/dask-0.15.0.md&lt;/span&gt;, line 172)&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="styling"&gt;
&lt;h1&gt;Styling&lt;/h1&gt;
&lt;p&gt;Clients and Futures have nicer HTML reprs that show up in the Jupyter notebook.&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/dask-client-future-html-repr.png"
     width="100%"&gt;&lt;/p&gt;
&lt;p&gt;And the dashboard stays a decent width and has a new navigation bar with links
to other dashboard pages. This template is now consistently applied to all
dashboard pages.&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/dask-navbar.png"
     width="100%"&gt;&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/2339"&gt;dask/distributed #1136&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/1126"&gt;dask/distributed #1126&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/1141"&gt;dask/distributed #1141&lt;/a&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/2017/06/15/dask-0.15.0.md&lt;/span&gt;, line 190)&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="multi-client-coordination"&gt;
&lt;h1&gt;Multi-client coordination&lt;/h1&gt;
&lt;p&gt;More primitives to help coordinate between multiple clients on the same cluster
have been added. These include
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/api.html#distributed.Queue"&gt;Queues&lt;/a&gt;
and shared
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/api.html#distributed.Variable"&gt;Variables&lt;/a&gt;
for futures.&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/2017/06/15/dask-0.15.0.md&lt;/span&gt;, line 199)&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="joblib-performance-through-pre-scattering"&gt;
&lt;h1&gt;Joblib performance through pre-scattering&lt;/h1&gt;
&lt;p&gt;When using Dask to power &lt;a class="reference external" href="https://pythonhosted.org/joblib/"&gt;Joblib&lt;/a&gt;
computations (such as occur in Scikit-Learn) with the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;joblib.parallel_backend&lt;/span&gt;&lt;/code&gt;
context manager, you can now pre-scatter select data to all workers. This can
significantly speed up some scikit-learn computations by reducing repeated data
transfer.&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;distributed.joblib&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;sklearn.externals.joblib&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;parallel_backend&lt;/span&gt;

&lt;span class="c1"&gt;# Serialize the training data only once to each worker&lt;/span&gt;
&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;parallel_backend&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;dask.distributed&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scheduler_host&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;localhost:8786&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                      &lt;span class="n"&gt;scatter&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;digits&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;digits&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;target&lt;/span&gt;&lt;span class="p"&gt;]):&lt;/span&gt;
      &lt;span class="n"&gt;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;digits&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;digits&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;target&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/1022"&gt;dask/distributed #1022&lt;/a&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/2017/06/15/dask-0.15.0.md&lt;/span&gt;, line 219)&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="other-array-improvements"&gt;
&lt;h1&gt;Other Array Improvements&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Filled out the dask.array.fft module&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Added a basic dask.array.stats module with functions like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chisquare&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Support the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;&amp;#64;&lt;/span&gt;&lt;/code&gt; matrix multiply operator&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/2017/06/15/dask-0.15.0.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="general-performance-and-stability"&gt;
&lt;h1&gt;General performance and stability&lt;/h1&gt;
&lt;p&gt;As usual, a number of bugs were identified and resolved and a number of
performance optimizations were implemented. Thank you to all users and
developers who continue to help identify and implement areas for improvement.
Users should generally have a smoother experience.&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/2017/06/15/dask-0.15.0.md&lt;/span&gt;, line 232)&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="removed-zmq-networking-backend"&gt;
&lt;h1&gt;Removed ZMQ networking backend&lt;/h1&gt;
&lt;p&gt;We have removed the experimental ZeroMQ networking backend. This was not
particularly useful in practice. However it was very effective in serving as
an example while we were making our network communication layer pluggable with
different protocols.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/1160"&gt;dask/distributed #1160&lt;/a&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/2017/06/15/dask-0.15.0.md&lt;/span&gt;, line 241)&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-releases"&gt;
&lt;h1&gt;Related Releases&lt;/h1&gt;
&lt;p&gt;The following related projects have also been released recently and may be
worth updating:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;NumPy 1.13.0&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Pandas 0.20.2&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bokeh 0.12.6&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Fastparquet 0.1.0&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;S3FS 0.1.1&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Cloudpickle 0.3.1 (pip)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;lz4 0.10.0 (pip)&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/2017/06/15/dask-0.15.0.md&lt;/span&gt;, line 254)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;The following people contributed to the dask/dask repository since the 0.14.3 release
on May 5th:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Elliott Sales de Andrade&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ghislain Antony Vaillant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John A Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joseph Crail&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Juan Nunez-Iglesias&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Julien Lhermitte&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Samantha Hughes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The following people contributed to the dask/distributed repository since the
1.16.2 release on May 5th:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;A. Jesse Jiryu Davis&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brett Naul&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Eugene Van den Bulke&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Fabian Keller&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Krisztián Szűcs&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Simon Perkins&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Thomas Arildsen&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Viacheslav Ostroukh&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/06/15/dask-0.15.0/"/>
    <summary>This work is supported by Continuum Analytics
and the Data Driven Discovery Initiative from the Moore
Foundation.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2017-06-15T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/05/08/dask-0.14.3/</id>
    <title>Dask Release 0.14.3</title>
    <updated>2017-05-08T00: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 Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;I’m pleased to announce the release of Dask version 0.14.3. This release
contains a variety of performance and feature improvements. This blogpost
includes some notable features and changes since the last release on March
22nd.&lt;/p&gt;
&lt;p&gt;As always you can conda install from conda-forge&lt;/p&gt;
&lt;div class="highlight-bash notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;conda&lt;span class="w"&gt; &lt;/span&gt;install&lt;span class="w"&gt; &lt;/span&gt;-c&lt;span class="w"&gt; &lt;/span&gt;conda-forge&lt;span class="w"&gt; &lt;/span&gt;dask&lt;span class="w"&gt; &lt;/span&gt;distributed
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or you can pip install from PyPI&lt;/p&gt;
&lt;div class="highlight-bash notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;pip&lt;span class="w"&gt; &lt;/span&gt;install&lt;span class="w"&gt; &lt;/span&gt;dask&lt;span class="o"&gt;[&lt;/span&gt;complete&lt;span class="o"&gt;]&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;--upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Conda packages should be on the default channel within a few days.&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/2017/05/08/dask-0.14.3.md&lt;/span&gt;, line 31)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="arrays"&gt;

&lt;section id="sparse-arrays"&gt;
&lt;h2&gt;Sparse Arrays&lt;/h2&gt;
&lt;p&gt;Dask.arrays now support sparse arrays and mixed dense/sparse 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;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="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;10000&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;10000&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="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;100&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;100&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;100&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;100&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="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="mf"&gt;0.99&lt;/span&gt;&lt;span class="p"&gt;]&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;sparse&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sparse&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;COO&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# parallel array of sparse arrays&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In order to support sparse arrays we did two things:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Made dask.array support ndarray containers other than NumPy, as long
as they were API compatible&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Made a small &lt;a class="reference external" href="https://github.com/mrocklin/sparse"&gt;sparse&lt;/a&gt; array library
that was API compatible to the numpy.ndarray&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This process was pretty easy and could be extended to other systems.
This also allows for different kinds of ndarrays in the same Dask array, as
long as interactions between the arrays are well defined (using the standard
NumPy protocols like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_priority__&lt;/span&gt;&lt;/code&gt; and so on.)&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Documentation&lt;/strong&gt;: &lt;a class="reference external" href="http://dask.pydata.org/en/latest/array-sparse.html"&gt;http://dask.pydata.org/en/latest/array-sparse.html&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Update: there is already a &lt;a class="reference external" href="https://github.com/dask/dask/pull/2301"&gt;pull
request&lt;/a&gt; for Masked arrays&lt;/em&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="reworked-fft-code"&gt;
&lt;h2&gt;Reworked FFT code&lt;/h2&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;da.fft&lt;/span&gt;&lt;/code&gt; submodule has been extended to include most of the functions in
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.fft&lt;/span&gt;&lt;/code&gt;, with the caveat that multi-dimensional FFTs will only work along
single-chunk dimensions. Still, given that rechunking is decently fast today
this can be very useful for large image stacks.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Documentation&lt;/strong&gt;: &lt;a class="reference external" href="http://dask.pydata.org/en/latest/array-api.html#fast-fourier-transforms"&gt;http://dask.pydata.org/en/latest/array-api.html#fast-fourier-transforms&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="constructor-plugins"&gt;
&lt;h2&gt;Constructor Plugins&lt;/h2&gt;
&lt;p&gt;You can now run arbitrary code whenever a dask array is constructed. This
empowers users to build in their own policies like rechunking, warning users,
or eager evaluation. A dask.array plugin takes in a dask.array and returns
either a new dask array, or returns None, in which case the original will be
returned.&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;f&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="gp"&gt;... &lt;/span&gt;    &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;&lt;/span&gt;&lt;span class="si"&gt;%d&lt;/span&gt;&lt;span class="s1"&gt; bytes&amp;#39;&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;nbytes&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;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_options&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;array_plugins&lt;/span&gt;&lt;span class="o"&gt;=&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="gp"&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;10&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;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="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="n"&gt;y&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;dot&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;T&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;80 bytes&lt;/span&gt;
&lt;span class="go"&gt;80 bytes&lt;/span&gt;
&lt;span class="go"&gt;800 bytes&lt;/span&gt;
&lt;span class="go"&gt;800 bytes&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This can be used, for example, to convert dask.array code into numpy code to
identify bugs quickly:&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;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_options&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;array_plugins&lt;/span&gt;&lt;span class="o"&gt;=&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="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()]):&lt;/span&gt;
&lt;span class="gp"&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;5&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="mi"&gt;2&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="c1"&gt;# this was automatically converted into a numpy array&lt;/span&gt;
&lt;span class="go"&gt;array([0, 1, 2, 3, 4])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Or to warn users if they accidentally produce an array with large chunks:&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;warn_on_large_chunks&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;shapes&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;itertools&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;product&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;chunks&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
    &lt;span class="n"&gt;nbytes&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;itemsize&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;prod&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;shape&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;shape&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;shapes&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="nb"&gt;any&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nb&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mf"&gt;1e9&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;nb&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;nbytes&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="n"&gt;warnings&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;warn&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Array contains very large chunks&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;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;array_plugins&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;warn_on_large_chunks&lt;/span&gt;&lt;span class="p"&gt;]):&lt;/span&gt;
    &lt;span class="o"&gt;...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;These features were heavily requested by the climate science community, which
tends to serve both highly technical computer scientists, and less technical
climate scientists who were running into issues with the nuances of chunking.&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/2017/05/08/dask-0.14.3.md&lt;/span&gt;, line 123)&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="dataframes"&gt;
&lt;h1&gt;DataFrames&lt;/h1&gt;
&lt;p&gt;Dask.dataframe changes are both numerous, and very small, making it difficult
to give a representative accounting of recent changes within a blogpost.
Typically these include small changes to either track new Pandas development,
or to fix slight inconsistencies in corner cases (of which there are many.)&lt;/p&gt;
&lt;p&gt;Still, two highlights follow:&lt;/p&gt;
&lt;section id="rolling-windows-with-time-intervals"&gt;
&lt;h2&gt;Rolling windows with time intervals&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="n"&gt;s&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rolling&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;2s&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;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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;2017-01-01 00:00:00    1.0&lt;/span&gt;
&lt;span class="go"&gt;2017-01-01 00:00:01    2.0&lt;/span&gt;
&lt;span class="go"&gt;2017-01-01 00:00:02    2.0&lt;/span&gt;
&lt;span class="go"&gt;2017-01-01 00:00:03    2.0&lt;/span&gt;
&lt;span class="go"&gt;2017-01-01 00:00:04    2.0&lt;/span&gt;
&lt;span class="go"&gt;2017-01-01 00:00:05    2.0&lt;/span&gt;
&lt;span class="go"&gt;2017-01-01 00:00:06    2.0&lt;/span&gt;
&lt;span class="go"&gt;2017-01-01 00:00:07    2.0&lt;/span&gt;
&lt;span class="go"&gt;2017-01-01 00:00:08    2.0&lt;/span&gt;
&lt;span class="go"&gt;2017-01-01 00:00:09    2.0&lt;/span&gt;
&lt;span class="go"&gt;dtype: float64&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="read-parquet-data-with-arrow"&gt;
&lt;h2&gt;Read Parquet data with Arrow&lt;/h2&gt;
&lt;p&gt;Dask now supports reading Parquet data with both
&lt;a class="reference external" href="http://fastparquet.readthedocs.io/en/latest/"&gt;fastparquet&lt;/a&gt; (a Numpy/Numba
solution) and &lt;a class="reference external" href="https://arrow.apache.org/"&gt;Arrow&lt;/a&gt; and
&lt;a class="reference external" href="https://github.com/apache/parquet-cpp"&gt;Parquet-CPP&lt;/a&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/path/to/mydata.parquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;engine&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;fastparquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/path/to/mydata.parquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;engine&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;arrow&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;Hopefully this capability increases the use of both projects and results in
greater feedback to those libraries so that they can continue to advance
Python’s access to the Parquet format.&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/2017/05/08/dask-0.14.3.md&lt;/span&gt;, line 165)&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="graph-optimizations"&gt;
&lt;h1&gt;Graph Optimizations&lt;/h1&gt;
&lt;p&gt;Dask performs a few passes of simple linear-time graph optimizations before
sending a task graph to the scheduler. These optimizations currently vary by
collection type, for example dask.arrays have different optimizations than
dask.dataframes. These optimizations can greatly improve performance in some
cases, but can also increase overhead, which becomes very important for large
graphs.&lt;/p&gt;
&lt;p&gt;As Dask has grown into more communities, each with strong and differing
performance constraints, we’ve found that we needed to allow each community to
define its own optimization schemes. The defaults have not changed, but now
you can override them with your own. This can be set globally or with a
context manager.&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;my_optimize_function&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;graph&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; Takes a task graph and a list of output keys, returns new graph &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="n"&gt;new_graph&lt;/span&gt; &lt;span class="o"&gt;=&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="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;new_graph&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_options&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;array_optimize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;my_optimize_function&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                      &lt;span class="n"&gt;dataframe_optimize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                      &lt;span class="n"&gt;delayed_optimize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;my_other_optimize_function&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;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&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;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Documentation&lt;/strong&gt;: &lt;a class="reference external" href="http://dask.pydata.org/en/latest/optimize.html#customizing-optimization"&gt;http://dask.pydata.org/en/latest/optimize.html#customizing-optimization&lt;/a&gt;&lt;/p&gt;
&lt;section id="speed-improvements"&gt;
&lt;h2&gt;Speed improvements&lt;/h2&gt;
&lt;p&gt;Additionally, task fusion has been significantly accelerated. This is very
important for large graphs, particularly in dask.array computations.&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/2017/05/08/dask-0.14.3.md&lt;/span&gt;, line 199)&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="web-diagnostics"&gt;
&lt;h1&gt;Web Diagnostics&lt;/h1&gt;
&lt;p&gt;The distributed scheduler’s web diagnostic page is now served from within the
dask scheduler process. This is both good and bad:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Good&lt;/strong&gt;: It is much easier to make new visuals&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Bad&lt;/strong&gt;: Dask and Bokeh now share a single CPU&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Because Bokeh and Dask now share the same Tornado event loop we no longer need
to send messages between them to then send out to a web browser. The Bokeh
server has full access to all of the scheduler state. This lets us build new
diagnostic pages more easily. This has been around for a while but was largely
used for development. In this version we’ve switched the new version to be
default and turned off the old one.&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/bokeh-dask-2017-05-03.png"
     width="50%"
     align="right"&gt;&lt;/p&gt;
&lt;p&gt;The cost here is that the Bokeh scheduler can take 10-20% of the CPU use. If
you are running a computation that heavily taxes the scheduler then you might
want to close your diagnostic pages. Fortunately, this almost never happens.
The dask scheduler is typically fast enough to never get close to this limit.&lt;/p&gt;
&lt;section id="tornado-difficulties"&gt;
&lt;h2&gt;Tornado difficulties&lt;/h2&gt;
&lt;p&gt;Beware that the current versions of Bokeh (0.12.5) and Tornado (4.5) do not
play well together. This has been fixed in development versions, and installing
with conda is fine, but if you naively pip install then you may experience bad behavior.&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/2017/05/08/dask-0.14.3.md&lt;/span&gt;, line 229)&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="joblib"&gt;
&lt;h1&gt;Joblib&lt;/h1&gt;
&lt;p&gt;The Dask.distributed Joblib backend now includes a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scatter=&lt;/span&gt;&lt;/code&gt; keyword, allowing
you to pre-scatter select variables out to all of the Dask workers. This
significantly cuts down on overhead, especially on machine learning workloads
where most of the data doesn’t change very much.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Send the training data only once to each worker&lt;/span&gt;
&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;parallel_backend&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;dask.distributed&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scheduler_host&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;localhost:8786&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                      &lt;span class="n"&gt;scatter&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;digits&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;digits&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;target&lt;/span&gt;&lt;span class="p"&gt;]):&lt;/span&gt;
    &lt;span class="n"&gt;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;digits&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;digits&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;target&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Early trials indicate that computations like scikit-learn’s RandomForest scale
nicely on a cluster without any additional code.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Documentation&lt;/strong&gt;: &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/joblib.html"&gt;http://distributed.readthedocs.io/en/latest/joblib.html&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/2017/05/08/dask-0.14.3.md&lt;/span&gt;, line 248)&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="preload-scripts"&gt;
&lt;h1&gt;Preload scripts&lt;/h1&gt;
&lt;p&gt;When starting a dask.distributed scheduler or worker people often want to
include a bit of custom setup code, for example to configure loggers,
authenticate with some network system, and so on. This has always been possible if
you start scheduler and workers from &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/setup.html#using-the-python-api"&gt;within
Python&lt;/a&gt;
but is tricky if you want to use the command line interface. Now you can write
your custom code as a separate standalone script and ask the command line
interface to run it for you at startup:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# scheduler-setup.py&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.diagnostics.plugin&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;SchedulerPlugin&lt;/span&gt;

&lt;span class="k"&gt;class&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nc"&gt;MyPlugin&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;SchedulerPlugin&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; Prints a message whenever a worker is added to the cluster &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;add_worker&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;worker&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Added a new worker at&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;worker&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;dask_setup&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="n"&gt;plugin&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;MyPlugin&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
        &lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;add_plugin&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;plugin&lt;/span&gt;&lt;span class="p"&gt;)&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;dask-scheduler&lt;span class="w"&gt; &lt;/span&gt;--preload&lt;span class="w"&gt; &lt;/span&gt;scheduler-setup.py
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This makes it easier for people to adapt Dask to their particular institution.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Documentation&lt;/strong&gt;: &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/setup.html#customizing-initialization"&gt;http://distributed.readthedocs.io/en/latest/setup.html#customizing-initialization&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/2017/05/08/dask-0.14.3.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="network-interfaces-for-infiniband"&gt;
&lt;h1&gt;Network Interfaces (for infiniband)&lt;/h1&gt;
&lt;p&gt;Many people use Dask on high performance supercomputers. This hardware
differs from typical commodity clusters or cloud services in several ways,
including very high performance network interconnects like
&lt;a class="reference external" href="https://en.wikipedia.org/wiki/InfiniBand"&gt;InfiniBand&lt;/a&gt;. Typically these
systems also have normal ethernet and other networks. You’re probably familiar
with this on your own laptop when you have both ethernet and wireless:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ ifconfig
lo          Link encap:Local Loopback                       # Localhost
            inet addr:127.0.0.1  Mask:255.0.0.0
            inet6 addr: ::1/128 Scope:Host
eth0        Link encap:Ethernet  HWaddr XX:XX:XX:XX:XX:XX   # Ethernet
            inet addr:192.168.0.101
            ...
ib0         Link encap:Infiniband                           # Fast InfiniBand
            inet addr:172.42.0.101
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The default systems Dask uses to determine network interfaces often choose
ethernet by default. If you are on an HPC system then this is likely not
optimal. You can direct Dask to choose a particular network interface with the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;--interface&lt;/span&gt;&lt;/code&gt; keyword&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ dask-scheduler --interface ib0
distributed.scheduler - INFO -   Scheduler at: tcp://172.42.0.101:8786

$ dask-worker tcp://172.42.0.101:8786 --interface ib0
&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/2017/05/08/dask-0.14.3.md&lt;/span&gt;, line 314)&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="efficient-as-completed"&gt;
&lt;h1&gt;Efficient as_completed&lt;/h1&gt;
&lt;p&gt;The
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/api.html#distributed.client.as_completed"&gt;as_completed&lt;/a&gt;
iterator returns futures in the order in which they complete. It is the base
of many asynchronous applications using Dask.&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;x&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inc&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;span class="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;future&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;as_completed&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;]):&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
&lt;span class="go"&gt;2&lt;/span&gt;
&lt;span class="go"&gt;0&lt;/span&gt;
&lt;span class="go"&gt;1&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;It can now also wait to yield an element only after the result also arrives&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;for&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;as_completed&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;with_results&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;2&lt;/span&gt;
&lt;span class="go"&gt;0&lt;/span&gt;
&lt;span class="go"&gt;1&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And also yield all futures (and results) that have finished up until this
point.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="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;futures&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;as_completed&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;batches&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;   &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;gather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="go"&gt;(2, 0)&lt;/span&gt;
&lt;span class="go"&gt;(1,)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Both of these help to decrease the overhead of tight inner loops within
asynchronous applications.&lt;/p&gt;
&lt;p&gt;Example blogpost here: &lt;a class="reference internal" href="../2017/04/19/dask-glm-2/"&gt;&lt;span class="doc std std-doc"&gt;/2017/04/19/dask-glm-2&lt;/span&gt;&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/2017/05/08/dask-0.14.3.md&lt;/span&gt;, line 355)&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="co-released-libraries"&gt;
&lt;h1&gt;Co-released libraries&lt;/h1&gt;
&lt;p&gt;This release is aligned with a number of other related libraries, notably
Pandas, and several smaller libraries for accessing data, including
&lt;a class="reference external" href="http://s3fs.readthedocs.io/en/latest/"&gt;s3fs&lt;/a&gt;,
&lt;a class="reference external" href="http://hdfs3.readthedocs.io/en/latest/"&gt;hdfs3&lt;/a&gt;,
&lt;a class="reference external" href="http://fastparquet.readthedocs.io/en/latest/"&gt;fastparquet&lt;/a&gt;, and
&lt;a class="reference external" href="https://github.com/andrix/python-snappy"&gt;python-snappy&lt;/a&gt; each of which have
seen numerous updates over the past few months. Much of the work of these
latter libraries is being coordinated by &lt;a class="reference external" href="http://martindurant.github.io/"&gt;Martin
Durant&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/2017/05/08/dask-0.14.3.md&lt;/span&gt;, line 367)&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;The following people contributed to the dask/dask repository since the 0.14.1 release
on March 22nd&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dmitry Shachnev&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Erik Welch&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Eugene Pakhomov&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jeff Reback&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John A Kirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joris Van den Bossche&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Michal Ficek&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Noah D Brenowitz&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stuart Archibald&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tom Augspurger&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Wes McKinney&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;wikiped&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The following people contributed to the dask/distributed repository since the
1.16.1 release on March 22nd&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Bartosz Marcinkowski&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ben Schreck&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jens Nie&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Krisztián Szűcs&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Lezyes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Luke Canavan&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Phil Elson&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/05/08/dask-0.14.3/"/>
    <summary>This work is supported by Continuum Analytics
and the Data Driven Discovery Initiative from the Moore
Foundation.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2017-05-08T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/04/28/dask-dev-8/</id>
    <title>Dask Development Log</title>
    <updated>2017-04-28T00: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
Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;To increase transparency I’m blogging weekly(ish) about the work done on Dask
and related projects during the previous week. This log covers work done
between 2017-04-20 and 2017-04-28. Nothing here is ready for production. This
blogpost is written in haste, so refined polish should not be expected.&lt;/p&gt;
&lt;p&gt;Development in Dask and Dask-related projects during the last week includes the
following notable changes:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Improved &lt;strong&gt;Joblib&lt;/strong&gt; support, accelerating existing Scikit-Learn code&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A &lt;strong&gt;dask-glm&lt;/strong&gt; powered &lt;strong&gt;LogisticRegression&lt;/strong&gt; estimator that is scikit-learn
compatible&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Additional &lt;strong&gt;Parquet&lt;/strong&gt; support by &lt;strong&gt;Arrow&lt;/strong&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Sparse arrays&lt;/strong&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Better spill-to-disk behavior&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;AsyncIO&lt;/strong&gt; compatible Client&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;TLS (SSL)&lt;/strong&gt; support&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;NumPy &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_ufunc__&lt;/span&gt;&lt;/code&gt; protocol&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/2017/04/28/dask-dev-8.md&lt;/span&gt;, line 30)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="joblib"&gt;

&lt;p&gt;Scikit learn parallelizes most of their algorithms with
&lt;a class="reference external" href="https://pythonhosted.org/joblib/"&gt;Joblib&lt;/a&gt;, which provides a simple interface
for embarrassingly parallel computations. Dask has been able to &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/joblib.html"&gt;hijack
joblib&lt;/a&gt; code and
serve as the backend for some time now, but it had some limitations,
particularly because we would repeatedly send data back and forth from a
worker to client for every batch of computations.&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;distributed.joblib&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;joblib&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;Parallel&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;parallel_backend&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;parallel_backend&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;dask.distributed&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scheduler_host&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;HOST:PORT&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="c1"&gt;# normal Joblib code&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now there is a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scatter=&lt;/span&gt;&lt;/code&gt; keyword, which allows you to pre-scatter select
variables out to all of the Dask workers. This significantly cuts down on
overhead, especially on machine learning workloads where most of the data
doesn’t change very much.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Send the training data only once to each worker&lt;/span&gt;
&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;parallel_backend&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;dask.distributed&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scheduler_host&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;localhost:8786&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                      &lt;span class="n"&gt;scatter&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;digits&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;digits&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;target&lt;/span&gt;&lt;span class="p"&gt;]):&lt;/span&gt;
    &lt;span class="n"&gt;search&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;digits&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;digits&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;target&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Early trials indicate that computations like scikit-learn’s RandomForest scale
nicely on a cluster without any additional code.&lt;/p&gt;
&lt;p&gt;This is particularly nice because it allows Dask and Scikit-Learn to play well
together without having to introduce Dask within the Scikit-Learn codebase at
all. From a maintenance perspective this combination is very attractive.&lt;/p&gt;
&lt;p&gt;Work done 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/dask/distributed/pull/1022"&gt;dask/distributed #1022&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/2017/04/28/dask-dev-8.md&lt;/span&gt;, line 69)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="dask-glm-logistic-regression"&gt;
&lt;h1&gt;Dask-GLM Logistic Regression&lt;/h1&gt;
&lt;p&gt;The convex optimization solvers in the
&lt;a class="reference external" href="https://github.com/dask/dask-glm"&gt;dask-glm&lt;/a&gt; project allow us to solve common
machine learning and statistics problems in parallel and at scale.
Historically this young library has contained only optimization solvers and
relatively little in the way of user API.&lt;/p&gt;
&lt;p&gt;This week dask-glm grew new LogisticRegression and LinearRegression estimators
that expose the scalable convex optimization algorithms within dask-glm through
a Scikit-Learn compatible interface. This can both speedup solutions on a
single computer or provide solutions for datasets that were previously too
large to fit 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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_glm.estimators&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;LogisticRegression&lt;/span&gt;

&lt;span class="n"&gt;est&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LogisticRegression&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;est&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;my_dask_array&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;labels&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;a class="reference external" href="http://nbviewer.jupyter.org/gist/anonymous/15742155693794ddd31ea85b654cbc7e"&gt;This notebook&lt;/a&gt;
compares performance to the latest release of scikit-learn on a 5,000,000
dataset running on a single machine. Dask-glm beats scikit-learn by a factor
of four, which is also roughly the number of cores on the development machine.
However in response &lt;a class="reference external" href="http://nbviewer.jupyter.org/gist/ogrisel/5f2d31bc5e7df852b4ca63f5f6049f42"&gt;this
notebook&lt;/a&gt;
by &lt;a class="reference external" href="http://ogrisel.com/"&gt;Olivier Grisel&lt;/a&gt; shows the development version of
scikit-learn (with a new algorithm) beating out dask-glm by a factor of six.
This just goes to show you that being smarter about your algorithms is almost
always a better use of time than adopting parallelism.&lt;/p&gt;
&lt;p&gt;Work done by &lt;a class="reference external" href="https://tomaugspurger.github.io/"&gt;Tom Augspurger&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/moody-marlin/"&gt;Chris
White&lt;/a&gt; in
&lt;a class="reference external" href="https://github.com/dask/dask-glm/pull/40"&gt;dask/dask-glm #40&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/2017/04/28/dask-dev-8.md&lt;/span&gt;, line 105)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="parquet-with-arrow"&gt;
&lt;h1&gt;Parquet with Arrow&lt;/h1&gt;
&lt;p&gt;The Parquet format is quickly becoming a standard for parallel and distributed
dataframes. There are currently two Parquet reader/writers accessible from
Python, &lt;a class="reference external" href="http://fastparquet.readthedocs.io/en/latest/"&gt;fastparquet&lt;/a&gt; a
NumPy/Numba solution, and &lt;a class="reference external" href="https://github.com/apache/parquet-cpp"&gt;Parquet-CPP&lt;/a&gt; a
C++ solution with wrappers provided by &lt;a class="reference external" href="https://arrow.apache.org/"&gt;Arrow&lt;/a&gt;.
Dask.dataframe has supported parquet for a while now with fastparquet.&lt;/p&gt;
&lt;p&gt;However, users will now have an option to use Arrow instead by switching the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;engine=&lt;/span&gt;&lt;/code&gt; keyword in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dd.read_parquet&lt;/span&gt;&lt;/code&gt; function.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/path/to/mydata.parquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;engine&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;fastparquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/path/to/mydata.parquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;engine&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;arrow&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;Hopefully this capability increases the use of both projects and results in
greater feedback to those libraries so that they can continue to advance
Python’s access to the Parquet format. As a gentle reminder, you can typically
get &lt;em&gt;much&lt;/em&gt; faster query times by switching from CSV to Parquet. This is often
much more effective than parallel computing.&lt;/p&gt;
&lt;p&gt;Work by &lt;a class="reference external" href="http://wesmckinney.com/"&gt;Wes McKinney&lt;/a&gt; in &lt;a class="reference external" href="https://github.com/dask/dask/pull/2223"&gt;dask/dask
#2223&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/2017/04/28/dask-dev-8.md&lt;/span&gt;, line 131)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="sparse-arrays"&gt;
&lt;h1&gt;Sparse Arrays&lt;/h1&gt;
&lt;p&gt;There is a small multi-dimensional sparse array library here:
&lt;a class="github reference external" href="https://github.com/mrocklin/sparse"&gt;mrocklin/sparse&lt;/a&gt;. It
allows us to represent arrays compactly in memory when most entries are zero.
This differs from the standard solution in
&lt;a class="reference external" href="https://docs.scipy.org/doc/scipy-0.19.0/reference/sparse.html"&gt;scipy.sparse&lt;/a&gt;,
which can only support arrays of dimension two (matrices) and not greater.&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 sparse
&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;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="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;10&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="mi"&gt;10&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="gp"&gt;&amp;gt;&amp;gt;&amp;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;x&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="mf"&gt;0.9&lt;/span&gt;&lt;span class="p"&gt;]&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;nbytes&lt;/span&gt;
&lt;span class="go"&gt;80000&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;sparse&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;sparse&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;COO&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;COO: shape=(10, 10, 10, 10), dtype=float64, nnz=1074&amp;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;nbytes&lt;/span&gt;
&lt;span class="go"&gt;12888&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;sparse&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tensordot&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;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;axes&lt;/span&gt;&lt;span class="o"&gt;=&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;span class="mi"&gt;3&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;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;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;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="go"&gt;array([ 100.93868073,  128.72312323,  119.12997217,  118.56304153,&lt;/span&gt;
&lt;span class="go"&gt;        133.24522101,   98.33555365,   90.25304866,   98.99823973,&lt;/span&gt;
&lt;span class="go"&gt;        100.57555847,   78.27915528])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Additionally, this &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sparse&lt;/span&gt;&lt;/code&gt; library more faithfully follows the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;numpy.ndarray&lt;/span&gt;&lt;/code&gt;
API, which is exactly what &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt; expects. Because of this close API
matching dask.array is able to parallelize around sparse arrays just as easily
as it parallelizes around dense numpy arrays. This gives us a decent
distributed multidimensional sparse array library relatively cheaply.&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;random&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;10000&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;10000&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="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;100&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;100&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;100&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;100&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="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="mf"&gt;0.9&lt;/span&gt;&lt;span class="p"&gt;]&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;s&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;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sparse&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;COO&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# parallel array of sparse arrays&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Work on the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sparse&lt;/span&gt;&lt;/code&gt; library is so far by &lt;a class="reference external" href="http://matthewrocklin.com/"&gt;myself&lt;/a&gt;
and &lt;a class="reference external" href="https://staff.washington.edu/jakevdp/"&gt;Jake VanderPlas&lt;/a&gt; and is available
&lt;a class="reference external" href="https://github.com/mrocklin/sparse"&gt;here&lt;/a&gt;. Work connecting this up to
Dask.array is in &lt;a class="reference external" href="http://matthewrocklin.com/"&gt;dask/dask #2234&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/2017/04/28/dask-dev-8.md&lt;/span&gt;, line 183)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="better-spill-to-disk-behavior"&gt;
&lt;h1&gt;Better spill to disk behavior&lt;/h1&gt;
&lt;p&gt;I’ve been playing with a 50GB sample of the 1TB &lt;a class="reference external" href="http://labs.criteo.com/2013/12/download-terabyte-click-logs-2/"&gt;Criteo
dataset&lt;/a&gt; on my
laptop (this is where I’m using sparse arrays). To make computations flow a
bit faster I’ve improved the performance of Dask’s spill-to-disk policies.&lt;/p&gt;
&lt;p&gt;Now, rather than depend on (cloud)pickle we use Dask’s network protocol, which
handles data more efficiently, compresses well, and has special handling for
common and important types like NumPy arrays and things built out of NumPy
arrays (like sparse arrays).&lt;/p&gt;
&lt;p&gt;As a result reading and writing excess data to disk is significantly faster.
When performing machine learning computations (which are fairly heavy-weight)
disk access is now fast enough that I don’t notice it in practice and running
out of memory doesn’t significantly impact performance.&lt;/p&gt;
&lt;p&gt;This is only really relevant when using common types (like numpy arrays) and
when your computation to disk access ratio is relatively high (such as is the
case for analytic workloads), but it was a simple fix and yielded a nice boost
to my personal productivity.&lt;/p&gt;
&lt;p&gt;Work by myself in &lt;a class="reference external" href="https://github.com/dask/distributed/pull/946"&gt;dask/distributed #946&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/2017/04/28/dask-dev-8.md&lt;/span&gt;, line 207)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="asyncio-compatible-client"&gt;
&lt;h1&gt;AsyncIO compatible Client&lt;/h1&gt;
&lt;p&gt;The Dask.distributed scheduler maintains a fully asynchronous API for use with
non-blocking systems like Tornado or AsyncIO. Because Dask supports Python 2
all of our internal code is written with Tornado. While Tornado and AsyncIO
can work together, this generally requires a bit of excess book-keeping, like
turning Tornado futures into AsyncIO futures, etc..&lt;/p&gt;
&lt;p&gt;Now there is an AsyncIO specific Client that only includes non-blocking methods
that are AsyncIO native. This allows for more idiomatic asynchronous code in
Python 3.&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;async&lt;/span&gt; &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;AioClient&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scheduler-address:8786&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;future&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;submit&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="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;args&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Work by &lt;a class="reference external" href="https://github.com/kszucs"&gt;Krisztián Szűcs&lt;/a&gt; in &lt;a class="reference external" href="https://github.com/dask/distributed/pull/1029"&gt;dask/distributed
#1029&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/2017/04/28/dask-dev-8.md&lt;/span&gt;, line 228)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="tls-ssl-support"&gt;
&lt;h1&gt;TLS (SSL) support&lt;/h1&gt;
&lt;p&gt;TLS (previously called SSL) is a common and trusted solution to authentication
and encryption. It is a commonly requested feature by companies of
institutions where intra-network security is important. This is currently
being worked on now at &lt;a class="reference external" href="https://github.com/dask/distributed/pull/1034"&gt;dask/distributed
#1034&lt;/a&gt;. I encourage anyone who
this may affect to engage on that pull request.&lt;/p&gt;
&lt;p&gt;Work by &lt;a class="reference external" href="https://github.com/pitrou"&gt;Antoine Pitrou&lt;/a&gt; in &lt;a class="reference external" href="https://github.com/dask/distributed/pull/1034"&gt;dask/distributed
#1034&lt;/a&gt; and previously by &lt;a class="reference external" href="https://github.com/mariusvniekerk"&gt;Marius
van Niekerk&lt;/a&gt; in &lt;a class="reference external" href="https://github.com/dask/distributed/pull/866"&gt;dask/distributed
#866&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/2017/04/28/dask-dev-8.md&lt;/span&gt;, line 242)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="numpy-array-ufunc"&gt;
&lt;h1&gt;NumPy &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array_ufunc__&lt;/span&gt;&lt;/code&gt;&lt;/h1&gt;
&lt;p&gt;This recent change in NumPy (literally merged as I was typing this blogpost)
allows other array libraries to take control of the the existing NumPy ufuncs,
so if you call something like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.exp(my_dask_array)&lt;/span&gt;&lt;/code&gt; this will no longer
convert to a NumPy array, but will rather call the appropriate
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array.exp&lt;/span&gt;&lt;/code&gt; function. This is a big step towards writing generic array
code that works both on NumPy arrays as well as other array projects like
dask.array, xarray, bcolz, sparse, etc..&lt;/p&gt;
&lt;p&gt;As with all large changes in NumPy this was accomplished through a
collaboration of many people. PR in &lt;a class="reference external" href="https://github.com/numpy/numpy/pull/8247"&gt;numpy/numpy #8247&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/04/28/dask-dev-8/"/>
    <summary>This work is supported by Continuum Analytics and the
Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-04-28T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/04/19/dask-glm-2/</id>
    <title>Asynchronous Optimization Algorithms with Dask</title>
    <updated>2017-04-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;,
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;,
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2017/04/19/dask-glm-2.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="summary"&gt;

&lt;p&gt;In a previous post &lt;a class="reference internal" href="../2017/03/22/dask-glm-1/"&gt;&lt;span class="doc std std-doc"&gt;we built convex optimization algorithms with
Dask&lt;/span&gt;&lt;/a&gt; that ran
efficiently on a distributed cluster and were important for a broad class of
statistical and machine learning algorithms.&lt;/p&gt;
&lt;p&gt;We now extend that work by looking at &lt;em&gt;asynchronous algorithms&lt;/em&gt;. We show the
following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;APIs within Dask to build asynchronous computations generally, not just for
machine learning and optimization&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Reasons why asynchronous algorithms are valuable in machine learning&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A concrete asynchronous algorithm (Async ADMM) and its performance on a
toy dataset&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This blogpost is co-authored by &lt;a class="reference external" href="https://github.com/moody-marlin/"&gt;Chris White&lt;/a&gt;
(Capital One) who knows optimization and &lt;a class="reference external" href="http://matthewrocklin.com/"&gt;Matthew
Rocklin&lt;/a&gt; (Continuum Analytics) who knows
distributed computing.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/4fc08482f33d60cc90cc3f8723146de5"&gt;Reproducible notebook available 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/2017/04/19/dask-glm-2.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="asynchronous-vs-blocking-algorithms"&gt;
&lt;h1&gt;Asynchronous vs Blocking Algorithms&lt;/h1&gt;
&lt;p&gt;When we say &lt;em&gt;asynchronous&lt;/em&gt; we contrast it against synchronous or blocking.&lt;/p&gt;
&lt;p&gt;In a blocking algorithm you send out a bunch of work and then wait for the
result. Dask’s normal &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.compute()&lt;/span&gt;&lt;/code&gt; interface is blocking. Consider the
following computation where we score a bunch of inputs in parallel and then
find the best:&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;dask&lt;/span&gt;

&lt;span class="n"&gt;scores&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;score&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;L&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;  &lt;span class="c1"&gt;# many lazy calls to the score function&lt;/span&gt;
&lt;span class="n"&gt;best&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;max&lt;/span&gt;&lt;span class="p"&gt;)(&lt;/span&gt;&lt;span class="n"&gt;scores&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;best&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;best&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;# Trigger all computation and wait until complete&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This &lt;em&gt;blocks&lt;/em&gt;. We can’t do anything while it runs. If we’re in a Jupyter
notebook we’ll see a little asterisk telling us that we have to wait.&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/jupyter-blocking-cell.png"
     width="100%"
     alt="A Jupyter notebook cell blocking on a dask computation"&gt;&lt;/p&gt;
&lt;p&gt;In a non-blocking or asynchronous algorithm we send out work and track results
as they come in. We are still able to run commands locally while our
computations run in the background (or on other computers in the cluster).
Dask has a variety of asynchronous APIs, but the simplest is probably the
&lt;a class="reference external" href="https://docs.python.org/3/library/concurrent.futures.html"&gt;concurrent.futures&lt;/a&gt;
API where we submit functions and then can wait and act on their return.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;as_completed&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scheduler-address:8786&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# Send out several computations&lt;/span&gt;
&lt;span class="n"&gt;futures&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;score&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;L&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

&lt;span class="c1"&gt;# Find max as results arrive&lt;/span&gt;
&lt;span class="n"&gt;best&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;
&lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;as_completed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;score&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;result&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;score&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;best&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;best&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;score&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;These two solutions are computationally equivalent. They do the same work and
run in the same amount of time. The blocking &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.delayed&lt;/span&gt;&lt;/code&gt; solution is
probably simpler to write down but the non-blocking &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;futures&lt;/span&gt; &lt;span class="pre"&gt;+&lt;/span&gt; &lt;span class="pre"&gt;as_completed&lt;/span&gt;&lt;/code&gt;
solution lets us be more &lt;em&gt;flexible&lt;/em&gt;.&lt;/p&gt;
&lt;p&gt;For example, if we get a score that is &lt;em&gt;good enough&lt;/em&gt; then we might stop early.
If we find that certain kinds of values are giving better scores than others
then we might submit more computations around those values while cancelling
others, changing our computation during execution.&lt;/p&gt;
&lt;p&gt;This ability to monitor and adapt a computation during execution is one reason
why people choose asynchronous algorithms. In the case of optimization
algorithms we are doing a search process and frequently updating parameters.
If we are able to update those parameters more frequently then we may be able
to slightly improve every subsequently launched computation. Asynchronous
algorithms enable increased flow of information around the cluster in
comparison to more lock-step batch-iterative 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/2017/04/19/dask-glm-2.md&lt;/span&gt;, line 101)&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="asynchronous-admm"&gt;
&lt;h1&gt;Asynchronous ADMM&lt;/h1&gt;
&lt;p&gt;In our &lt;a class="reference internal" href="../2017/03/22/dask-glm-1/"&gt;&lt;span class="doc std std-doc"&gt;last blogpost&lt;/span&gt;&lt;/a&gt;
we showed a simplified implementation of &lt;a class="reference external" href="http://stanford.edu/~boyd/admm.html"&gt;Alternating Direction Method of
Multipliers&lt;/a&gt; (ADMM) with
&lt;a class="reference external" href="http://dask.pydata.org/en/latest/delayed.html"&gt;dask.delayed&lt;/a&gt;. We saw that in
a distributed context it performed well when compared to a more traditional
distributed gradient descent. This algorithm works by solving a small
optimization problem on every chunk of our data using our current parameter
estimates, bringing these back to the local process, combining them, and then
sending out new computation on updated parameters.&lt;/p&gt;
&lt;p&gt;Now we alter this algorithm to update asynchronously, so that our parameters
change continuously as partial results come in in real-time. Instead of
sending out and waiting on batches of results, we now consume and emit a
constant stream of tasks with slightly improved parameter estimates.&lt;/p&gt;
&lt;p&gt;We show three algorithms in sequence:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Synchronous: The original synchronous algorithm&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Asynchronous-single: updates parameters with every new result&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Asynchronous-batched: updates with all results that have come in since we
last updated.&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/2017/04/19/dask-glm-2.md&lt;/span&gt;, line 125)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="setup"&gt;
&lt;h1&gt;Setup&lt;/h1&gt;
&lt;p&gt;We create fake 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="n"&gt;n&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;k&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;chunksize&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;50000000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;100&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;50000&lt;/span&gt;

&lt;span class="n"&gt;beta&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;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="n"&gt;k&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="c1"&gt;# random beta coefficients, no intercept&lt;/span&gt;
&lt;span class="n"&gt;zero_idx&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;choice&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&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="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;beta&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;zero_idx&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt; &lt;span class="c1"&gt;# set some parameters to 0&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;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="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="n"&gt;n&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;k&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;chunksize&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;k&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;y&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;dot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&lt;/span&gt;&lt;span class="p"&gt;)&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;0&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;size&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;n&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="n"&gt;chunksize&lt;/span&gt;&lt;span class="p"&gt;,))&lt;/span&gt; &lt;span class="c1"&gt;# add noise&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;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# trigger computation in the background&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We define local functions for ADMM. These correspond to solving an l1-regularized Linear
regression problem:&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;local_f&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;rho&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="n"&gt;y&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;dot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;rho&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;np&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;beta&lt;/span&gt; &lt;span class="o"&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;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                                                              &lt;span class="n"&gt;beta&lt;/span&gt; &lt;span class="o"&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;u&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;local_grad&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;rho&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;X&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;T&lt;/span&gt;&lt;span class="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="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;beta&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;rho&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&lt;/span&gt; &lt;span class="o"&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;u&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;shrinkage&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&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="k"&gt;return&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;maximum&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;beta&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;t&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;maximum&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;beta&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;t&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;local_update2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;partial&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;local_update&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;local_f&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;fprime&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;local_grad&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;lamduh&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;7.2&lt;/span&gt; &lt;span class="c1"&gt;# regularization parameter&lt;/span&gt;

&lt;span class="c1"&gt;# algorithm parameters&lt;/span&gt;
&lt;span class="n"&gt;rho&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;1.2&lt;/span&gt;
&lt;span class="n"&gt;abstol&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;1e-4&lt;/span&gt;
&lt;span class="n"&gt;reltol&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;1e-2&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;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;zeros&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;p&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# the initial consensus estimate&lt;/span&gt;

&lt;span class="c1"&gt;# an array of the individual &amp;quot;dual variables&amp;quot; and parameter estimates,&lt;/span&gt;
&lt;span class="c1"&gt;# one for each chunk of data&lt;/span&gt;
&lt;span class="n"&gt;u&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;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;zeros&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;p&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="n"&gt;nchunks&lt;/span&gt;&lt;span class="p"&gt;)])&lt;/span&gt;
&lt;span class="n"&gt;betas&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;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;zeros&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;p&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="n"&gt;nchunks&lt;/span&gt;&lt;span class="p"&gt;)])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Finally because ADMM doesn’t want to work on distributed arrays, but instead
on lists of remote numpy arrays (one numpy array per chunk of the dask.array)
we convert each our Dask.arrays into a list of dask.delayed objects:&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;XD&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;to_delayed&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;flatten&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tolist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="c1"&gt;# a list of numpy arrays, one for each chunk&lt;/span&gt;
&lt;span class="n"&gt;yD&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_delayed&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;flatten&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tolist&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/2017/04/19/dask-glm-2.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="synchronous-admm"&gt;
&lt;h1&gt;Synchronous ADMM&lt;/h1&gt;
&lt;p&gt;In this algorithm we send out many tasks to run, collect their results, update
parameters, and repeat. In this simple implementation we continue for a fixed
amount of time but in practice we would want to check some convergence
criterion.&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;start&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="k"&gt;while&lt;/span&gt; &lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;start&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="n"&gt;MAX_TIME&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="c1"&gt;# process each chunk in parallel, using the black-box &amp;#39;local_update&amp;#39; function&lt;/span&gt;
    &lt;span class="n"&gt;betas&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;local_update2&lt;/span&gt;&lt;span class="p"&gt;)(&lt;/span&gt;&lt;span class="n"&gt;xx&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;yy&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;bb&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;uu&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;rho&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;xx&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;yy&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;bb&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;uu&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="nb"&gt;zip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;XD&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;yD&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;betas&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;
    &lt;span class="n"&gt;betas&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;array&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;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;betas&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;  &lt;span class="c1"&gt;# collect results back&lt;/span&gt;

    &lt;span class="c1"&gt;# Update Parameters&lt;/span&gt;
    &lt;span class="n"&gt;ztilde&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;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;betas&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;array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;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;z&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;shrinkage&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ztilde&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;lamduh&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;rho&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;nchunks&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
    &lt;span class="n"&gt;u&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="n"&gt;betas&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;  &lt;span class="c1"&gt;# update dual variables&lt;/span&gt;

    &lt;span class="c1"&gt;# track convergence metrics&lt;/span&gt;
    &lt;span class="n"&gt;update_metrics&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/2017/04/19/dask-glm-2.md&lt;/span&gt;, line 207)&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;Asynchronous ADMM&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/2017/04/19/dask-glm-2.md&lt;/span&gt;, line 207); &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: “asynchronous admm”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;In the asynchronous version we send out only enough tasks to occupy all of our
workers. We collect results one by one as they finish, update parameters, and
then send out a new task.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Submit enough tasks to occupy our current workers&lt;/span&gt;
&lt;span class="n"&gt;starting_indices&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;choice&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nchunks&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="n"&gt;ncores&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="n"&gt;replace&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;futures&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;local_update&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;XD&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="n"&gt;yD&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="n"&gt;betas&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="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                           &lt;span class="n"&gt;rho&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;local_f&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;fprime&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;local_grad&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="n"&gt;starting_indices&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;index&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;dict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;zip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;starting_indices&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

&lt;span class="c1"&gt;# An iterator that returns results as they come in&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;as_completed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;with_results&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;start&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;count&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;

&lt;span class="k"&gt;while&lt;/span&gt; &lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;start&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="n"&gt;MAX_TIME&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="c1"&gt;# Get next completed result&lt;/span&gt;
    &lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;local_beta&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;next&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pool&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;pop&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;betas&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;local_beta&lt;/span&gt;
    &lt;span class="n"&gt;count&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;

    &lt;span class="c1"&gt;# Update parameters (this could be made more efficient)&lt;/span&gt;
    &lt;span class="n"&gt;ztilde&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;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;betas&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;array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;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="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;count&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="n"&gt;nchunks&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;  &lt;span class="c1"&gt;# artificially inflate beta in the beginning&lt;/span&gt;
        &lt;span class="n"&gt;ztilde&lt;/span&gt; &lt;span class="o"&gt;*=&lt;/span&gt; &lt;span class="n"&gt;nchunks&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;count&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;z&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;shrinkage&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ztilde&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;lamduh&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;rho&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;nchunks&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
    &lt;span class="n"&gt;update_metrics&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

    &lt;span class="c1"&gt;# Submit new task to the cluster&lt;/span&gt;
    &lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;nchunks&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;u&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="n"&gt;betas&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;
    &lt;span class="n"&gt;new_future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;local_update2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;XD&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="n"&gt;yD&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="n"&gt;betas&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="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;rho&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;new_future&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;i&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;add&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;new_future&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/2017/04/19/dask-glm-2.md&lt;/span&gt;, line 250)&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="batched-asynchronous-admm"&gt;
&lt;h1&gt;Batched Asynchronous ADMM&lt;/h1&gt;
&lt;p&gt;With enough distributed workers we find that our parameter-updating loop on the
client can be the limiting factor. After profiling it seems that our client
was bound not by updating parameters, but rather by computing the performance
metrics that we are going to use for the convergence plots below (so not
actually a limitation in practice). However we decided to leave this in
because it is good practice for what is likely to occur in larger clusters,
where the single machine that updates parameters is possibly overwhelmed by a
high volume of updates from the workers. To resolve this, we build in
batching.&lt;/p&gt;
&lt;p&gt;Rather than update our parameters one by one, we update them with however many
results have come in so far. This provides a natural defense against a slow
client. This approach smoothly shifts our algorithm back over to the
synchronous solution when the client becomes overwhelmed. (though again, at
this scale we’re fine).&lt;/p&gt;
&lt;p&gt;Conveniently, the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;as_completed&lt;/span&gt;&lt;/code&gt; iterator has a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.batches()&lt;/span&gt;&lt;/code&gt; method that
iterates over all of the results that have come in so far.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# ... same setup as before&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;as_completed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;new_betas&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;with_results&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;batches&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;batches&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;            &lt;span class="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;--- this is new&lt;/span&gt;

&lt;span class="k"&gt;while&lt;/span&gt; &lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;start&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="n"&gt;MAX_TIME&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;

    &lt;span class="c1"&gt;# Get all tasks that have come in since we checked last time&lt;/span&gt;
    &lt;span class="n"&gt;batch&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;next&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;batches&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;           &lt;span class="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;--- this is new&lt;/span&gt;
    &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;batch&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;pop&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;betas&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;result&lt;/span&gt;
        &lt;span class="n"&gt;count&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;

    &lt;span class="n"&gt;ztilde&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;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;betas&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;array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;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="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;count&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="n"&gt;nchunks&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;ztilde&lt;/span&gt; &lt;span class="o"&gt;*=&lt;/span&gt; &lt;span class="n"&gt;nchunks&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;count&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;z&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;shrinkage&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ztilde&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;lamduh&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;rho&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;nchunks&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
    &lt;span class="n"&gt;update_metrics&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

    &lt;span class="c1"&gt;# Submit as many new tasks as we collected&lt;/span&gt;
    &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;batch&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;                 &lt;span class="c1"&gt;# &amp;lt;&amp;lt;&amp;lt;--- this is new&lt;/span&gt;
        &lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;nchunks&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;u&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="n"&gt;betas&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;
        &lt;span class="n"&gt;new_fut&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;local_update2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;XD&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="n"&gt;yD&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="n"&gt;betas&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="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;rho&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;new_fut&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;i&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;add&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;new_fut&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/2017/04/19/dask-glm-2.md&lt;/span&gt;, line 302)&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="visual-comparison-of-algorithms"&gt;
&lt;h1&gt;Visual Comparison of Algorithms&lt;/h1&gt;
&lt;p&gt;To show the qualitative difference between the algorithms we include profile
plots of each. Note the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Synchronous has blocks of full CPU use followed by blocks of no use&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The Asynchrhonous methods are more smooth&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The Asynchronous single-update method has a lot of whitespace / time when
CPUs are idling. This is artifiical and because our code that tracks
convergence diagnostics for our plots below is wasteful and inside the
client inner-loop&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We intentionally leave in this wasteful code so that we can reduce it by
batching in the third plot, which is more saturated.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;You can zoom in using the tools to the upper right of each plot. You can view
the full profile in a full window by clicking on the “View full page” link.&lt;/p&gt;
&lt;section id="synchronous"&gt;
&lt;h2&gt;Synchronous&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="https://cdn.rawgit.com/mrocklin/2a1dbae5e846dce787bdbdeb7fb13be5/raw/1090bf7698aa72c672b6d490766c2c26b86f9279/task-stream-admm-sync.html"&gt;View full page&lt;/a&gt;&lt;/p&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/2a1dbae5e846dce787bdbdeb7fb13be5/raw/1090bf7698aa72c672b6d490766c2c26b86f9279/task-stream-admm-sync.html"
        width="800" height="300"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="asynchronous-single-update"&gt;
&lt;h2&gt;Asynchronous single-update&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="https://cdn.rawgit.com/mrocklin/2a1dbae5e846dce787bdbdeb7fb13be5/raw/1090bf7698aa72c672b6d490766c2c26b86f9279/task-stream-admm-async.html"&gt;View full page&lt;/a&gt;&lt;/p&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/2a1dbae5e846dce787bdbdeb7fb13be5/raw/1090bf7698aa72c672b6d490766c2c26b86f9279/task-stream-admm-async.html"
        width="800" height="300"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="asynchronous-batched-update"&gt;
&lt;h2&gt;Asynchronous batched-update&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="https://cdn.rawgit.com/mrocklin/2a1dbae5e846dce787bdbdeb7fb13be5/raw/1090bf7698aa72c672b6d490766c2c26b86f9279/task-stream-admm-batched.html"&gt;View full page&lt;/a&gt;&lt;/p&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/2a1dbae5e846dce787bdbdeb7fb13be5/raw/1090bf7698aa72c672b6d490766c2c26b86f9279/task-stream-admm-batched.html"
        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/2017/04/19/dask-glm-2.md&lt;/span&gt;, line 340)&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="plot-convergence-criteria"&gt;
&lt;h1&gt;Plot Convergence Criteria&lt;/h1&gt;
&lt;p&gt;&lt;img src="/images/admm-async-primal-residual.png"
     alt="Primal residual for async-admm"
     width="100%"&gt;
&lt;img src="/images/admm-async-convergence.png"
     alt="Primal residual for async-admm"
     width="100%"&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/2017/04/19/dask-glm-2.md&lt;/span&gt;, line 349)&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="analysis"&gt;
&lt;h1&gt;Analysis&lt;/h1&gt;
&lt;p&gt;To get a better sense of what these plots convey, recall that optimization problems always come in pairs: the &lt;em&gt;primal&lt;/em&gt; problem
is typically the main problem of interest, and the &lt;em&gt;dual&lt;/em&gt; problem is a closely related problem that provides information about
the constraints in the primal problem. Perhaps the most famous example of duality is the &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Max-flow_min-cut_theorem"&gt;Max-flow-min-cut Theorem&lt;/a&gt;
from graph theory. In many cases, solving both of these problems simultaneously leads to gains in performance, which is what ADMM seeks to do.&lt;/p&gt;
&lt;p&gt;In our case, the constraint in the primal problem is that &lt;em&gt;all workers must agree on the optimum parameter estimate.&lt;/em&gt; Consequently, we can think
of the dual variables (one for each chunk of data) as measuring the “cost” of agreement for their respective chunks. Intuitively, they will start
out small and grow incrementally to find the right “cost” for each worker to have consensus. Eventually, they will level out at an optimum cost.&lt;/p&gt;
&lt;p&gt;So:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;the primal residual plot measures the amount of disagreement; “small” values imply agreement&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;the dual residual plot measures the total “cost” of agreement; this increases until the correct cost is found&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The plots then tell us the following:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;the cost of agreement is higher for asynchronous algorithms, which makes sense because each worker is always working with a slightly out-of-date global parameter estimate,
making consensus harder&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;blocked ADMM doesn’t update at all until shortly after 5 seconds have passed, whereas async has already had time to converge.
(In practice with real data, we would probably specify that all workers need to report in every K updates).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;asynchronous algorithms take a little while for the information to properly diffuse, but once that happens they converge quickly.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;both asynchronous and synchronous converge almost immediately; this is most likely due to a high degree of homogeneity in the data (which was generated to fit the model well). Our next experiment should involve real world data.&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/2017/04/19/dask-glm-2.md&lt;/span&gt;, line 374)&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-we-could-have-done-better"&gt;
&lt;h1&gt;What we could have done better&lt;/h1&gt;
&lt;p&gt;Analysis wise we expect richer results by performing this same experiment on a real world data set that isn’t as homogeneous as the current toy dataset.&lt;/p&gt;
&lt;p&gt;Performance wise we can get much better CPU saturation by doing two things:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Not running our convergence diagnostics, or making them much faster&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Not running full &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.mean&lt;/span&gt;&lt;/code&gt; computations over all of beta when we’ve only
updated a few elements. Instead we should maintain a running aggregation
of these results.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;With these two changes (each of which are easy) we’re fairly confident that we
can scale out to decently large clusters while still saturating hardware.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/04/19/dask-glm-2/"/>
    <summary>This work is supported by Continuum Analytics,
the XDATA Program,
and the Data Driven Discovery Initiative from the Moore
Foundation.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-04-19T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/03/28/dask-xgboost/</id>
    <title>Dask and Pandas and XGBoost</title>
    <updated>2017-03-28T00: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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2017/03/28/dask-xgboost.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="summary"&gt;

&lt;p&gt;This post talks about distributing Pandas Dataframes with Dask and then handing
them over to distributed XGBoost for training.&lt;/p&gt;
&lt;p&gt;More generally it discusses the value of launching multiple distributed systems
in the same shared-memory processes and smoothly handing data back and forth
between them.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/3696fe2398dc7152c66bf593a674e4d9"&gt;Notebook&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://youtu.be/Cc4E-PdDSro"&gt;Screencast&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dmlc/xgboost/issues/2032"&gt;Github issue&lt;/a&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/2017/03/28/dask-xgboost.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="introduction"&gt;
&lt;h1&gt;Introduction&lt;/h1&gt;
&lt;p&gt;XGBoost is a well-loved library for a popular class of machine learning
algorithms, gradient boosted trees. It is used widely in business and is one
of the most popular solutions in Kaggle competitions. For larger datasets or
faster training, XGBoost also comes with its own distributed computing system
that lets it scale to multiple machines on a cluster. Fantastic. Distributed
gradient boosted trees are in high demand.&lt;/p&gt;
&lt;p&gt;However before we can use distributed XGBoost we need to do three things:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Prepare and clean our possibly large data, probably with a lot of Pandas wrangling&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Set up XGBoost master and workers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Hand data our cleaned data from a bunch of distributed Pandas dataframes to
XGBoost workers across our cluster&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This ends up being surprisingly easy. This blogpost gives a quick example
using Dask.dataframe to do distributed Pandas data wrangling, then using a new
&lt;a class="reference external" href="https://github.com/dask/dask-xgboost"&gt;dask-xgboost&lt;/a&gt; package to setup an
XGBoost cluster inside the Dask cluster and perform the handoff.&lt;/p&gt;
&lt;p&gt;After this example we’ll talk about the general design and what this means for
other distributed systems.&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/2017/03/28/dask-xgboost.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="example"&gt;
&lt;h1&gt;Example&lt;/h1&gt;
&lt;p&gt;We have a ten-node cluster with eight cores each (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;m4.2xlarges&lt;/span&gt;&lt;/code&gt; on EC2)&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;dask&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;progress&lt;/span&gt;

&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;172.31.33.0:8786&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;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;restart&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;tcp://172.31.33.0:8786&amp;#39;&lt;/span&gt; &lt;span class="n"&gt;processes&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt; &lt;span class="n"&gt;cores&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;80&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We load the Airlines dataset using dask.dataframe (just a bunch of Pandas
dataframes spread across a cluster) and do a bit of preprocessing:&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;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="c1"&gt;# Subset of the columns to use&lt;/span&gt;
&lt;span class="n"&gt;cols&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;Year&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Month&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;DayOfWeek&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Distance&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="s1"&gt;&amp;#39;DepDelay&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;CRSDepTime&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;UniqueCarrier&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Origin&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Dest&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

&lt;span class="c1"&gt;# Create the dataframe&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;dd&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;s3://dask-data/airline-data/20*.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;usecols&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;cols&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                  &lt;span class="n"&gt;storage_options&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;anon&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;})&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sample&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;frac&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;0.2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="c1"&gt;# XGBoost requires a bit of RAM, we need a larger cluster&lt;/span&gt;

&lt;span class="n"&gt;is_delayed&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DepDelay&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fillna&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;16&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;15&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# column of labels&lt;/span&gt;
&lt;span class="k"&gt;del&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;DepDelay&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;  &lt;span class="c1"&gt;# Remove delay information from training dataframe&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;CRSDepTime&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;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;CRSDepTime&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;clip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;upper&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;2399&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;is_delayed&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;is_delayed&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# start work in the background&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This loaded a few hundred pandas dataframes from CSV data on S3. We then had
to downsample because how we are going to use XGBoost in the future seems to
require a lot of RAM. I am not an XGBoost expert. Please forgive my ignorance
here. At the end we have two dataframes:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df&lt;/span&gt;&lt;/code&gt;: Data from which we will learn if flights are delayed&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;is_delayed&lt;/span&gt;&lt;/code&gt;: Whether or not those flights were delayed.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Data scientists familiar with Pandas will probably be familiar with the code
above. Dask.dataframe is &lt;em&gt;very&lt;/em&gt; similar to Pandas, but operates on a cluster.&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;df&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
    &lt;tr style="text-align: right;"&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;Year&lt;/th&gt;
      &lt;th&gt;Month&lt;/th&gt;
      &lt;th&gt;DayOfWeek&lt;/th&gt;
      &lt;th&gt;CRSDepTime&lt;/th&gt;
      &lt;th&gt;UniqueCarrier&lt;/th&gt;
      &lt;th&gt;Origin&lt;/th&gt;
      &lt;th&gt;Dest&lt;/th&gt;
      &lt;th&gt;Distance&lt;/th&gt;
    &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt;182193&lt;/th&gt;
      &lt;td&gt;2000&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;800&lt;/td&gt;
      &lt;td&gt;WN&lt;/td&gt;
      &lt;td&gt;LAX&lt;/td&gt;
      &lt;td&gt;OAK&lt;/td&gt;
      &lt;td&gt;337&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;83424&lt;/th&gt;
      &lt;td&gt;2000&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;6&lt;/td&gt;
      &lt;td&gt;1650&lt;/td&gt;
      &lt;td&gt;DL&lt;/td&gt;
      &lt;td&gt;SJC&lt;/td&gt;
      &lt;td&gt;SLC&lt;/td&gt;
      &lt;td&gt;585&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;346781&lt;/th&gt;
      &lt;td&gt;2000&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;5&lt;/td&gt;
      &lt;td&gt;1140&lt;/td&gt;
      &lt;td&gt;AA&lt;/td&gt;
      &lt;td&gt;ORD&lt;/td&gt;
      &lt;td&gt;LAX&lt;/td&gt;
      &lt;td&gt;1745&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;375935&lt;/th&gt;
      &lt;td&gt;2000&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;1940&lt;/td&gt;
      &lt;td&gt;DL&lt;/td&gt;
      &lt;td&gt;PHL&lt;/td&gt;
      &lt;td&gt;ATL&lt;/td&gt;
      &lt;td&gt;665&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;309373&lt;/th&gt;
      &lt;td&gt;2000&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;4&lt;/td&gt;
      &lt;td&gt;1028&lt;/td&gt;
      &lt;td&gt;CO&lt;/td&gt;
      &lt;td&gt;MCI&lt;/td&gt;
      &lt;td&gt;IAH&lt;/td&gt;
      &lt;td&gt;643&lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&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="n"&gt;is_delayed&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="go"&gt;182193    False&lt;/span&gt;
&lt;span class="go"&gt;83424     False&lt;/span&gt;
&lt;span class="go"&gt;346781    False&lt;/span&gt;
&lt;span class="go"&gt;375935    False&lt;/span&gt;
&lt;span class="go"&gt;309373    False&lt;/span&gt;
&lt;span class="go"&gt;Name: DepDelay, dtype: bool&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;section id="categorize-and-one-hot-encode"&gt;
&lt;h2&gt;Categorize and One Hot Encode&lt;/h2&gt;
&lt;p&gt;XGBoost doesn’t want to work with text data like destination=”LAX”. Instead we
create new indicator columns for each of the known airports and carriers. This
expands our data into many boolean columns. Fortunately Dask.dataframe has
convenience functions for all of this baked in (thank you Pandas!)&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;df2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_dummies&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;categorize&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This expands our data out considerably, but makes it easier to train on.&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;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;685&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="split-and-train"&gt;
&lt;h2&gt;Split and Train&lt;/h2&gt;
&lt;p&gt;Great, now we’re ready to split our distributed dataframes&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;data_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data_test&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random_split&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mf"&gt;0.9&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;random_state&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1234&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;labels_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;labels_test&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;is_delayed&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random_split&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mf"&gt;0.9&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;random_state&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1234&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Start up a distributed XGBoost instance, and train on this 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="o"&gt;%%&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_xgboost&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;dxgb&lt;/span&gt;

&lt;span class="n"&gt;params&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;objective&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;binary:logistic&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;nround&amp;#39;&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="s1"&gt;&amp;#39;max_depth&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;16&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;eta&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mf"&gt;0.01&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;subsample&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mf"&gt;0.5&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
          &lt;span class="s1"&gt;&amp;#39;min_child_weight&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="s1"&gt;&amp;#39;tree_method&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;hist&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
          &lt;span class="s1"&gt;&amp;#39;grow_policy&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;lossguide&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;

&lt;span class="n"&gt;bst&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dxgb&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;train&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;params&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;labels_train&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;355&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;29.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;385&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="mf"&gt;54.5&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Great, so we were able to train an XGBoost model on this data in about a minute
using our ten machines. What we get back is just a plain XGBoost Booster
object.&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;bst&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;xgboost.core.Booster at 0x7fa1c18c4c18&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We could use this on normal Pandas data locally&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;xgboost&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;xgb&lt;/span&gt;
&lt;span class="n"&gt;pandas_df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;data_test&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="n"&gt;dtest&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xgb&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DMatrix&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pandas_df&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;bst&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;predict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dtest&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="mf"&gt;0.464578&lt;/span&gt;  &lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;0.46631625&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;0.47434333&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;0.47245741&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;0.46194169&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="n"&gt;float32&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Of we can use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-xgboost&lt;/span&gt;&lt;/code&gt; again to train on our distributed holdout data,
getting back another Dask series.&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;predictions&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dxgb&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;predict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;bst&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data_test&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;predictions&lt;/span&gt;
&lt;span class="go"&gt;Dask Series Structure:&lt;/span&gt;
&lt;span class="go"&gt;npartitions=93&lt;/span&gt;
&lt;span class="go"&gt;None    float32&lt;/span&gt;
&lt;span class="go"&gt;None        ...&lt;/span&gt;
&lt;span class="go"&gt;         ...&lt;/span&gt;
&lt;span class="go"&gt;None        ...&lt;/span&gt;
&lt;span class="go"&gt;None        ...&lt;/span&gt;
&lt;span class="go"&gt;Name: predictions, dtype: float32&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: _predict_part, 93 tasks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="evaluate"&gt;
&lt;h2&gt;Evaluate&lt;/h2&gt;
&lt;p&gt;We can bring these predictions to the local process and use normal Scikit-learn
operations to evaluate the results.&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;sklearn.metrics&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;roc_auc_score&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;roc_curve&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;roc_auc_score&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;labels_test&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="gp"&gt;... &lt;/span&gt;                    &lt;span class="n"&gt;predictions&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()))&lt;/span&gt;
&lt;span class="go"&gt;0.654800768411&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="n"&gt;fpr&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;tpr&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;roc_curve&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;labels_test&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;predictions&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;# Taken from&lt;/span&gt;
&lt;span class="n"&gt;http&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="o"&gt;//&lt;/span&gt;&lt;span class="n"&gt;scikit&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;learn&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;org&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;stable&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;auto_examples&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;model_selection&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;plot_roc&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;html&lt;/span&gt;&lt;span class="c1"&gt;#sphx-glr-auto-examples-model-selection-plot-roc-py&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;figure&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;figsize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;8&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;lw&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;plot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;fpr&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;tpr&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;color&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;darkorange&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;lw&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;lw&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;label&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ROC curve&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;plot&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;color&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;navy&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;lw&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;lw&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;linestyle&lt;/span&gt;&lt;span class="o"&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="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;xlim&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mf"&gt;0.0&lt;/span&gt;&lt;span class="p"&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;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ylim&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mf"&gt;0.0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;1.05&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;xlabel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;False Positive Rate&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;True Positive Rate&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;title&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;Receiver operating characteristic example&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;legend&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;loc&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;lower right&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;show&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/dask-xgboost-roc-curve.png" width="50%"&gt;
&lt;p&gt;We might want to play with our parameters above or try different data to
improve our solution. The point here isn’t that we predicted airline delays
well, it was that if you are a data scientist who knows Pandas and XGBoost,
everything we did above seemed &lt;em&gt;pretty familiar&lt;/em&gt;. There wasn’t a whole lot of
new material in the example above. We’re using the same tools as before, just
at a larger scale.&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/2017/03/28/dask-xgboost.md&lt;/span&gt;, line 309)&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="analysis"&gt;
&lt;h1&gt;Analysis&lt;/h1&gt;
&lt;p&gt;OK, now that we’ve demonstrated that this works lets talk a bit about what
just happened and what that means generally for cooperation between distributed
services.&lt;/p&gt;
&lt;section id="what-dask-xgboost-does"&gt;
&lt;h2&gt;What dask-xgboost does&lt;/h2&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://github.com/dask/dask-xgboost"&gt;dask-xgboost&lt;/a&gt; project is pretty
small and pretty simple (200 TLOC). Given a Dask cluster of one central scheduler and
several distributed workers it starts up an XGBoost scheduler in the same
process running the Dask scheduler and starts up an XGBoost worker within each
of the Dask workers. They share the same physical processes and memory
spaces. Dask was built to support this kind of situation, so this is
relatively easy.&lt;/p&gt;
&lt;p&gt;Then we ask the Dask.dataframe to fully materialize in RAM and we ask where all
of the constituent Pandas dataframes live. We tell each Dask worker to give
all of the Pandas dataframes that it has to its local XGBoost worker and then
just let XGBoost do its thing. Dask doesn’t power XGBoost, it’s just
sets it up, gives it data, and lets it do it’s work in the background.&lt;/p&gt;
&lt;p&gt;People often ask what machine learning capabilities Dask provides, how they
compare with other distributed machine learning libraries like H2O or Spark’s
MLLib. For gradient boosted trees the 200-line dask-xgboost package is the
answer. Dask has no need to make such an algorithm because XGBoost already
exists, works well and provides Dask users with a fully featured and efficient
solution.&lt;/p&gt;
&lt;p&gt;Because both Dask and XGBoost can live in the same Python process they can
share bytes between each other without cost, can monitor each other, etc..
These two distributed systems co-exist together in multiple processes in the
same way that NumPy and Pandas operate together within a single process.
Sharing distributed processes with multiple systems can be really beneficial if
you want to use multiple specialized services easily and avoid large monolithic
frameworks.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="connecting-to-other-distributed-systems"&gt;
&lt;h2&gt;Connecting to Other distributed systems&lt;/h2&gt;
&lt;p&gt;A while ago I wrote
&lt;a class="reference internal" href="../2017/02/11/dask-tensorflow/"&gt;&lt;span class="doc std std-doc"&gt;a similar blogpost&lt;/span&gt;&lt;/a&gt;
about hosting TensorFlow from Dask in exactly the same way that we’ve done
here. It was similarly easy to setup TensorFlow alongside Dask, feed it data,
and let TensorFlow do its thing.&lt;/p&gt;
&lt;p&gt;Generally speaking this “serve other libraries” approach is how Dask operates
when possible. We’re only able to cover the breadth of functionality that we
do today because we lean heavily on the existing open source ecosystem.
Dask.arrays use Numpy arrays, Dask.dataframes use Pandas, and now the answer to
gradient boosted trees with Dask is just to make it really really easy to use
distributed XGBoost. Ta da! We get a fully featured solution that is
maintained by other devoted developers, and the entire connection process was
done over a weekend (see &lt;a class="reference external" href="https://github.com/dmlc/xgboost/issues/2032"&gt;dmlc/xgboost
#2032&lt;/a&gt; for details).&lt;/p&gt;
&lt;p&gt;Since this has come out we’ve had requests to support other distributed systems
like &lt;a class="reference external" href="http://libelemental.org/"&gt;Elemental&lt;/a&gt; and to do general hand-offs to MPI
computations. If we’re able to start both systems with the same set of
processes then all of this is pretty doable. Many of the challenges of
inter-system collaboration go away when you can hand numpy arrays between the
workers of one system to the workers of the other system within the same
processes.&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/2017/03/28/dask-xgboost.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&gt;
&lt;section id="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;Thanks to &lt;a class="reference external" href="http://homes.cs.washington.edu/~tqchen/"&gt;Tianqi Chen&lt;/a&gt; and &lt;a class="reference external" href="http://ogrisel.com/"&gt;Olivier
Grisel&lt;/a&gt; for their help when &lt;a class="reference external" href="https://github.com/dmlc/xgboost/issues/2032"&gt;building and
testing&lt;/a&gt; &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-xgboost&lt;/span&gt;&lt;/code&gt;. Thanks
to &lt;a class="reference external" href="http://github.com/electronwill"&gt;Will Warner&lt;/a&gt; for his help in editing this
post.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/03/28/dask-xgboost/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-03-28T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/03/23/dask-0.14.1/</id>
    <title>Dask Release 0.14.1</title>
    <updated>2017-03-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;,
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;,
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;I’m pleased to announce the release of Dask version 0.14.1. This release
contains a variety of performance and feature improvements. This blogpost
includes some notable features and changes since the last release on February
27th.&lt;/p&gt;
&lt;p&gt;As always you can conda install from conda-forge&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 -c conda-forge dask distributed
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or you can pip install from PyPI&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[complete] --upgrade
&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/2017/03/23/dask-0.14.1.md&lt;/span&gt;, line 26)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="arrays"&gt;

&lt;p&gt;Recent work in distributed computing and machine learning have motivated new
performance-oriented and usability changes to how we handle arrays.&lt;/p&gt;
&lt;section id="automatic-chunking-and-operation-on-numpy-arrays"&gt;
&lt;h2&gt;Automatic chunking and operation on NumPy arrays&lt;/h2&gt;
&lt;p&gt;Many interactions between Dask arrays and NumPy arrays work smoothly. NumPy
arrays are made lazy and are appropriately chunked to match the operation
and the Dask 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="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;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;                 &lt;span class="c1"&gt;# a numpy array&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;y&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;10&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;# a dask array&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;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;                       &lt;span class="c1"&gt;# combined become a dask.array&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="go"&gt;dask.array&amp;lt;add, shape=(10,), dtype=float64, chunksize=(5,)&amp;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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;array([  1.,   2.,   3.,   4.,   5.,   6.,   7.,   8.,   9.,  10.])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="reshape"&gt;
&lt;h2&gt;Reshape&lt;/h2&gt;
&lt;p&gt;Reshaping distributed arrays is simple in simple cases, and can be quite
complex in complex cases. Reshape now supports a much more broad set of shape
transformations where any dimension is collapsed or merged to other dimensions.&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;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;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;4&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="mi"&gt;6&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;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="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="mi"&gt;2&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;reshape&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;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="mi"&gt;30&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="go"&gt;dask.array&amp;lt;reshape, shape=(6, 2, 2, 30, 1), dtype=float64, chunksize=(3, 1, 2, 6, 1)&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This operation ends up being quite useful in a number of distributed array
cases.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="optimize-slicing-to-minimize-communication"&gt;
&lt;h2&gt;Optimize Slicing to Minimize Communication&lt;/h2&gt;
&lt;p&gt;Dask.array slicing optimizations are now careful to produce graphs that avoid
situations that could cause excess inter-worker communication. The details of
how they do this is a bit out of scope for a short blogpost, but the history
here is interesting.&lt;/p&gt;
&lt;p&gt;Historically dask.arrays were used almost exclusively by researchers with large
on-disk arrays stored as HDF5 or NetCDF files. These users primarily used the
single machine multi-threaded scheduler. We heavily tailored Dask array
optimizations to this situation and made that community pretty happy.
Now as some of that community switches to cluster computing on larger datasets
the optimization goals shift a bit. We have tons of distributed disk bandwidth
but really want to avoid communicating large results between workers.
Supporting both use cases is possible and I think that we’ve achieved that in
this release so far, but it’s starting to require increasing levels of care.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="micro-optimizations"&gt;
&lt;h2&gt;Micro-optimizations&lt;/h2&gt;
&lt;p&gt;With distributed computing also comes larger graphs and a growing importance of
graph-creation overhead. This has been optimized somewhat in this release. We
expect this to be a focus going forward.&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/2017/03/23/dask-0.14.1.md&lt;/span&gt;, line 86)&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="dataframes"&gt;
&lt;h1&gt;DataFrames&lt;/h1&gt;
&lt;section id="set-index"&gt;
&lt;h2&gt;Set_index&lt;/h2&gt;
&lt;p&gt;Set_index is smarter in two ways:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;If you set_index on a column that happens to be sorted then we’ll identify
that and avoid a costly shuffle. This was always possible with the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sorted=&lt;/span&gt;&lt;/code&gt;
keyword but users rarely used this feature. Now this is automatic.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Similarly when setting the index we can look at the size of the data and
determine if there are too many or too few partitions and rechunk the data
while shuffling. This can significantly improve performance if there are too
many partitions (a common case).&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/2025"&gt;dask/dask #2025&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/2091"&gt;dask/dask #2091&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="shuffle-performance"&gt;
&lt;h2&gt;Shuffle performance&lt;/h2&gt;
&lt;p&gt;We’ve micro-optimized some parts of dataframe shuffles. Big thanks to the
Pandas developers for the help here. This accelerates set_index, joins,
groupby-applies, and so on.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/2032"&gt;dask/dask #2032&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="fastparquet"&gt;
&lt;h2&gt;Fastparquet&lt;/h2&gt;
&lt;p&gt;The &lt;a class="reference external" href="http://fastparquet.readthedocs.io/en/latest/"&gt;fastparquet&lt;/a&gt; library has
seen a lot of use lately and has undergone a number of community bugfixes.&lt;/p&gt;
&lt;p&gt;Importantly, Fastparquet now supports Python 2.&lt;/p&gt;
&lt;p&gt;We strongly recommend Parquet as the standard data storage format for Dask
dataframes (and Pandas DataFrames).&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/fastparquet/pull/87"&gt;dask/fastparquet #87&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/2017/03/23/dask-0.14.1.md&lt;/span&gt;, line 123)&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="distributed-scheduler"&gt;
&lt;h1&gt;Distributed Scheduler&lt;/h1&gt;
&lt;section id="replay-remote-exceptions"&gt;
&lt;h2&gt;Replay remote exceptions&lt;/h2&gt;
&lt;p&gt;Debugging is hard in part because exceptions happen on remote machines where
normal debugging tools like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pdb&lt;/span&gt;&lt;/code&gt; can’t reach. Previously we were able to
bring back the traceback and exception, but you couldn’t dive into the stack
trace to investigate what went wrong:&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;div&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;y&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;x&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;

&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;div&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;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt;
&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;Future&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;status&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;error&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;div&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="n"&gt;a34907f5384bcf9161498a635311aeb&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt;

&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# getting result re-raises exception locally&lt;/span&gt;
&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;ipython&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="nb"&gt;input&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="mi"&gt;398&lt;/span&gt;&lt;span class="n"&gt;a43a7781e&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;div&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
      &lt;span class="mi"&gt;1&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;div&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;y&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="o"&gt;----&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;     &lt;span class="k"&gt;return&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;y&lt;/span&gt;

&lt;span class="ne"&gt;ZeroDivisionError&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;division&lt;/span&gt; &lt;span class="n"&gt;by&lt;/span&gt; &lt;span class="n"&gt;zero&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now Dask can bring a failing task and all necessary data back to the local
machine and rerun it so that users can leverage the normal Python debugging
toolchain.&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;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;recreate_error_locally&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;ipython-input-3-398a43a7781e&amp;gt; in div(x, y)&lt;/span&gt;
&lt;span class="go"&gt;      1 def div(x, y):&lt;/span&gt;
&lt;span class="go"&gt;----&amp;gt; 2     return x / y&lt;/span&gt;
&lt;span class="go"&gt;ZeroDivisionError: division by zero&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now if you’re in IPython or a Jupyter notebook you can use the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;%debug&lt;/span&gt;&lt;/code&gt; magic
to jump into the stacktrace, investigate local variables, and so on.&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="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;debug&lt;/span&gt;
&lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;ipython&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="nb"&gt;input&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="mi"&gt;398&lt;/span&gt;&lt;span class="n"&gt;a43a7781e&lt;/span&gt;&lt;span class="o"&gt;&amp;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="n"&gt;div&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
      &lt;span class="mi"&gt;1&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;div&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;y&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="o"&gt;----&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;     &lt;span class="k"&gt;return&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;y&lt;/span&gt;

&lt;span class="n"&gt;ipdb&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;pp&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="n"&gt;ipdb&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;pp&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;
&lt;span class="mi"&gt;0&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/894"&gt;dask/distributed #894&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="async-await-syntax"&gt;
&lt;h2&gt;Async/await syntax&lt;/h2&gt;
&lt;p&gt;Dask.distributed uses Tornado for network communication and Tornado coroutines
for concurrency. Normal users rarely interact with Tornado coroutines; they
aren’t familiar to most people so we opted instead to copy the
concurrent.futures API. However some complex situations are &lt;em&gt;much&lt;/em&gt; easier to
solve if you know a little bit of async programming.&lt;/p&gt;
&lt;p&gt;Fortunately, the Python ecosystem seems to be embracing this change towards
native async code with the async/await syntax in Python 3. In an effort to
motivate people to learn async programming and to gently nudge them towards
Python 3 Dask.distributed we now support async/await in a few cases.&lt;/p&gt;
&lt;p&gt;You can wait on a dask Future&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;async&lt;/span&gt; &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;f&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&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="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;args&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You can put the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;as_completed&lt;/span&gt;&lt;/code&gt; iterator into an async for loop&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;async&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;as_completed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt;
    &lt;span class="o"&gt;...&lt;/span&gt; &lt;span class="n"&gt;do&lt;/span&gt; &lt;span class="n"&gt;stuff&lt;/span&gt; &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And, because Tornado supports the await protocols you can also use the existing
shadow concurrency API (everything prepended with an underscore) with await.
(This was doable before.)&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;results&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;gather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;         &lt;span class="c1"&gt;# synchronous&lt;/span&gt;
&lt;span class="o"&gt;...&lt;/span&gt;
&lt;span class="n"&gt;results&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_gather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# asynchronous&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If you’re in Python 2 you can always do this with normal &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;yield&lt;/span&gt;&lt;/code&gt; and
the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;tornado.gen.coroutine&lt;/span&gt;&lt;/code&gt; decorator.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/952"&gt;dask/distributed #952&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="inproc-transport"&gt;
&lt;h2&gt;Inproc transport&lt;/h2&gt;
&lt;p&gt;In the last release we enabled Dask to communicate over more things than just
TCP. In practice this doesn’t come up (TCP is pretty useful). However in this
release we now support single-machine “clusters” where the clients, scheduler,
and workers are all in the same process and transfer data cost-free over
in-memory queues.&lt;/p&gt;
&lt;p&gt;This allows the in-memory user community to use some of the more advanced
features (asynchronous computation, spill-to-disk support, web-diagnostics)
that are only available in the distributed scheduler.&lt;/p&gt;
&lt;p&gt;This is on by default if you create a cluster with LocalCluster without using
Nanny processes.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nanny&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&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;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;Client: scheduler=&amp;#39;inproc://192.168.1.115/8437/1&amp;#39; processes=1 cores=4&amp;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;threading&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;Lock&lt;/span&gt;         &lt;span class="c1"&gt;# Not serializable&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;lock&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Lock&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;                      &lt;span class="c1"&gt;# Won&amp;#39;t survive going over a socket&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scatter&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;lock&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;  &lt;span class="c1"&gt;# Yet we can send to a worker&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;                    &lt;span class="c1"&gt;# ... and back&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;unlocked _thread.lock object at 0x7fb7f12d08a0&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/919"&gt;dask/distributed #919&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="connection-pooling-for-inter-worker-communications"&gt;
&lt;h2&gt;Connection pooling for inter-worker communications&lt;/h2&gt;
&lt;p&gt;Workers now maintain a pool of sustained connections between each other. This
pool is of a fixed size and removes connections with a least-recently-used
policy. It avoids re-connection delays when transferring data between workers.
In practice this shaves off a millisecond or two from every communication.&lt;/p&gt;
&lt;p&gt;This is actually a revival of an old feature that we had turned off last year
when it became clear that the performance here wasn’t a problem.&lt;/p&gt;
&lt;p&gt;Along with other enhancements, this takes our round-trip latency down to 11ms
on my laptop.&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;10&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="o"&gt;%%&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;
    &lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;     &lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inc&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;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="o"&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="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;4.96&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;348&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;5.31&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;11.1&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;There may be room for improvement here though. For comparison here is the same
test with the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;concurent.futures.ProcessPoolExecutor&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;14&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ProcessPoolExecutor&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;In&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="o"&gt;%%&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;
    &lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;     &lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inc&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;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;future&lt;/span&gt;&lt;span class="o"&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="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;320&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;56&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;376&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;442&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Also, just to be clear, this measures total roundtrip latency, not overhead.
Dask’s distributed scheduler overhead remains in the low hundreds of
microseconds.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/distributed/pull/935"&gt;dask/distributed #935&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/2017/03/23/dask-0.14.1.md&lt;/span&gt;, line 299)&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="related-projects"&gt;
&lt;h1&gt;Related Projects&lt;/h1&gt;
&lt;p&gt;There has been activity around Dask and machine learning:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-learn"&gt;dask-learn&lt;/a&gt; is undergoing some
performance enhancements. It turns out that when you offer distributed grid
search people quickly want to scale up their computations to hundreds of
thousands of trials.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-glm"&gt;dask-glm&lt;/a&gt; now has a few decent algorithms
for convex optimization. The authors of this wrote a blogpost very recently
if you’re interested:
&lt;a class="reference internal" href="../2017/03/22/dask-glm-1/"&gt;&lt;span class="doc std std-doc"&gt;Developing Convex Optimization Algorithms in Dask&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-xgboost"&gt;dask-xgboost&lt;/a&gt; lets you hand off
distributed data in Dask dataframes or arrays and hand it directly to a
distributed XGBoost system (that Dask will nicely set up and tear down for
you). This was a nice example of easy hand-off between two distributed
services running in the same 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/2017/03/23/dask-0.14.1.md&lt;/span&gt;, line 317)&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;The following people contributed to the dask/dask repository since the 0.14.0 release
on February 27th&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brian Martin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Elliott Sales de Andrade&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Erik Welch&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Francisco de la Peña&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;jakirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jitesh Kumar Jha&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Julien Lhermitte&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Markus Gonser&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Talmaj&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The following people contributed to the dask/distributed repository since the
1.16.0 release on February 27th&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ben Schreck&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Elliott Sales de Andrade&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Phil Elson&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/03/23/dask-0.14.1/"/>
    <summary>This work is supported by Continuum Analytics,
the XDATA Program,
and the Data Driven Discovery Initiative from the Moore
Foundation.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2017-03-23T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/03/22/dask-glm-1/</id>
    <title>Developing Convex Optimization Algorithms in Dask</title>
    <updated>2017-03-22T00: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;,
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;,
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2017/03/22/dask-glm-1.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="summary"&gt;

&lt;p&gt;We build distributed optimization algorithms with Dask.  We show both simple
examples and also benchmarks from a nascent
&lt;a class="reference external" href="https://github.com/dask/dask-glm"&gt;dask-glm&lt;/a&gt; library for generalized linear
models.  We also talk about the experience of learning Dask to do this kind
of work.&lt;/p&gt;
&lt;p&gt;This blogpost is co-authored by &lt;a class="reference external" href="https://github.com/moody-marlin/"&gt;Chris White&lt;/a&gt;
(Capital One) who knows optimization and &lt;a class="reference external" href="http://matthewrocklin.com/"&gt;Matthew
Rocklin&lt;/a&gt; (Continuum Analytics) who knows
distributed 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/2017/03/22/dask-glm-1.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="introduction"&gt;
&lt;h1&gt;Introduction&lt;/h1&gt;
&lt;p&gt;Many machine learning and statistics models (such as logistic regression) depend
on convex optimization algorithms like Newton’s method, stochastic gradient
descent, and others.  These optimization algorithms are both pragmatic (they’re
used in many applications) and mathematically interesting.  As a result these
algorithms have been the subject of study by researchers and graduate students
around the world for years both in academia and in industry.&lt;/p&gt;
&lt;p&gt;Things got interesting about five or ten years ago when datasets grew beyond
the size of working memory and “Big Data” became a buzzword.  Parallel and
distributed solutions for these algorithms have become the norm, and a
researcher’s skillset now has to extend beyond linear algebra and optimization
theory to include parallel algorithms and possibly even network programming,
especially if you want to explore and create more interesting algorithms.&lt;/p&gt;
&lt;p&gt;However, relatively few people understand both mathematical optimization theory
and the details of distributed systems. Typically algorithmic researchers
depend on the APIs of distributed computing libraries like Spark or Flink to
implement their algorithms. In this blogpost we explore the extent to which
Dask can be helpful in these applications. We approach this from two
perspectives:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Algorithmic researcher&lt;/strong&gt; (Chris): someone who knows optimization and
iterative algorithms like Conjugate Gradient, Dual Ascent, or GMRES but
isn’t so hot on distributed computing topics like sockets, MPI, load
balancing, and so on&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Distributed systems developer&lt;/strong&gt; (Matt): someone who knows how to move
bytes around and keep machines busy but doesn’t know the right way to do a
line search or handle a poorly conditioned matrix&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/2017/03/22/dask-glm-1.md&lt;/span&gt;, line 58)&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="prototyping-algorithms-in-dask"&gt;
&lt;h1&gt;Prototyping Algorithms in Dask&lt;/h1&gt;
&lt;p&gt;Given knowledge of algorithms and of NumPy array computing it is easy to write parallel algorithms with Dask. For a range of complicated algorithmic structures we have two straightforward choices:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Use parallel multi-dimensional arrays to construct algorithms from common operations like matrix multiplication, SVD, and so on. This mirrors mathematical algorithms well but lacks some flexibility.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Create algorithms by hand that track operations on individual chunks of in-memory data and dependencies between them. This is very flexible but requires a bit more care.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Coding up either of these options from scratch can be a daunting task, but with Dask it can be as simple as writing NumPy code.&lt;/p&gt;
&lt;p&gt;Let’s build up an example of fitting a large linear regression model using both built-in array parallelism and fancier, more customized parallelization features that Dask offers. The &lt;a class="reference external" href="http://dask.pydata.org/en/latest/array.html"&gt;dask.array&lt;/a&gt; module helps us to easily parallelize standard NumPy functionality using the same syntax – we’ll start there.&lt;/p&gt;
&lt;section id="data-creation"&gt;
&lt;h2&gt;Data Creation&lt;/h2&gt;
&lt;p&gt;Dask has &lt;a class="reference external" href="http://dask.pydata.org/en/latest/array-creation.html"&gt;many ways to create dask arrays&lt;/a&gt;; to get us started quickly prototyping let’s create some random data in a way that should look familiar to NumPy users.&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;dask&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;

&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;

&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="c1"&gt;## create inputs with a bunch of independent normals&lt;/span&gt;
&lt;span class="n"&gt;beta&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;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;100&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# random beta coefficients, no intercept&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;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="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;1000000&lt;/span&gt;&lt;span class="p"&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;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;100000&lt;/span&gt;&lt;span class="p"&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;y&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;dot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&lt;/span&gt;&lt;span class="p"&gt;)&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;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;size&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1000000&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;100000&lt;/span&gt;&lt;span class="p"&gt;,))&lt;/span&gt;

&lt;span class="c1"&gt;## make sure all chunks are ~equally sized&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;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rebalance&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;y&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Observe that &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;X&lt;/span&gt;&lt;/code&gt; is a dask array stored in 10 chunks, each of size &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(100000,&lt;/span&gt; &lt;span class="pre"&gt;100)&lt;/span&gt;&lt;/code&gt;. Also note that &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;X.dot(beta)&lt;/span&gt;&lt;/code&gt; runs smoothly for both &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;numpy&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; arrays, so we can write code that basically works in either world.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Caveat:&lt;/strong&gt; If &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;X&lt;/span&gt;&lt;/code&gt; is a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;numpy&lt;/span&gt;&lt;/code&gt; array and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;beta&lt;/span&gt;&lt;/code&gt; is a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; array, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;X.dot(beta)&lt;/span&gt;&lt;/code&gt; will output an &lt;em&gt;in-memory&lt;/em&gt; &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;numpy&lt;/span&gt;&lt;/code&gt; array. This is usually not desirable as you want to carefully choose when to load something into memory. One fix is to use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;multipledispatch&lt;/span&gt;&lt;/code&gt; to handle odd edge cases; for a starting example, check out the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dot&lt;/span&gt;&lt;/code&gt; code &lt;a class="reference external" href="https://github.com/dask/dask-glm/blob/master/dask_glm/utils.py#L65-L84"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Dask also has convenient visualization features built in that we will leverage; below we visualize our data in its 10 independent chunks:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/dask-glm-data-creation-black-on-white.svg"
     width="100%"
     alt="Create data for dask-glm computations"&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="array-programming"&gt;
&lt;h2&gt;Array Programming&lt;/h2&gt;
&lt;p&gt;&lt;em&gt;If you can write iterative array-based algorithms in NumPy, then you can write iterative parallel algorithms in Dask&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;As we’ve already seen, Dask inherits much of the NumPy API that we are familiar with, so we can write simple NumPy-style iterative optimization algorithms that will leverage the parallelism &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt; has built-in already. For example, if we want to naively fit a linear regression model on the data above, we are trying to solve the following convex optimization problem:&lt;/p&gt;
&lt;div class="math notranslate nohighlight"&gt;
\[
\min_{\beta} \|y - X\beta\|_2^2
\]&lt;/div&gt;
&lt;p&gt;Recall that in non-degenerate situations this problem has a closed-form solution that is given by:&lt;/p&gt;
&lt;div class="math notranslate nohighlight"&gt;
\[
\beta^* = \left(X^T X\right)^{-1} X^T y
\]&lt;/div&gt;
&lt;p&gt;We can compute &lt;span class="math notranslate nohighlight"&gt;\(\beta^*\)&lt;/span&gt; using the above formula with Dask:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;## naive solution&lt;/span&gt;
&lt;span class="n"&gt;beta_star&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;solve&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;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;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="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;y&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="nb"&gt;abs&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta_star&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="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;beta&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;max&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="mf"&gt;0.0024817567237768179&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Sometimes a direct solve is too costly, and we want to solve the above problem using only simple matrix-vector multiplications. To this end, let’s take this one step further and actually implement a gradient descent algorithm which exploits parallel matrix operations. Recall that gradient descent iteratively refines an initial estimate of beta via the update:&lt;/p&gt;
&lt;div class="math notranslate nohighlight"&gt;
\[
\beta^+ = \beta - \alpha \nabla f(\beta)
\]&lt;/div&gt;
&lt;p&gt;where &lt;span class="math notranslate nohighlight"&gt;\(\alpha\)&lt;/span&gt; can be chosen based on a number of different “step-size” rules; for the purposes of exposition, we will stick with a constant step-size:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;## quick step-size calculation to guarantee convergence&lt;/span&gt;
&lt;span class="n"&gt;_&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;_&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;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="mi"&gt;2&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;X&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;T&lt;/span&gt;&lt;span class="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="n"&gt;step_size&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mf"&gt;1e-8&lt;/span&gt;

&lt;span class="c1"&gt;## define some parameters&lt;/span&gt;
&lt;span class="n"&gt;max_steps&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;100&lt;/span&gt;
&lt;span class="n"&gt;tol&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;1e-8&lt;/span&gt;
&lt;span class="n"&gt;beta_hat&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;zeros&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;100&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="c1"&gt;# initial guess&lt;/span&gt;

&lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;k&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="n"&gt;max_steps&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;Xbeta&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;dot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta_hat&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;func&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;Xbeta&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="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;gradient&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="n"&gt;X&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;T&lt;/span&gt;&lt;span class="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;Xbeta&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

    &lt;span class="c1"&gt;## Update&lt;/span&gt;
    &lt;span class="n"&gt;obeta&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;beta_hat&lt;/span&gt;
    &lt;span class="n"&gt;beta_hat&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;beta_hat&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;step_size&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;gradient&lt;/span&gt;
    &lt;span class="n"&gt;new_func&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="n"&gt;y&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;dot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta_hat&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;&lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="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;beta_hat&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;new_func&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta_hat&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;new_func&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# &amp;lt;--- Dask code&lt;/span&gt;

    &lt;span class="c1"&gt;## Check for convergence&lt;/span&gt;
    &lt;span class="n"&gt;change&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;absolute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta_hat&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;obeta&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;max&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;change&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="n"&gt;tol&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="k"&gt;break&lt;/span&gt;

&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="nb"&gt;abs&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta_hat&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;beta&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;max&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="mf"&gt;0.0024817567259038942&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;It’s worth noting that almost all of this code is exactly the same as the equivalent NumPy code. Because Dask.array and NumPy share the same API it’s pretty easy for people who are already comfortable with NumPy to get started with distributed algorithms right away. The only thing we had to change was how we produce our original data (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;da.random.normal&lt;/span&gt;&lt;/code&gt; instead of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.random.normal&lt;/span&gt;&lt;/code&gt;) and the call to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.compute&lt;/span&gt;&lt;/code&gt; at the end of the update state. The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.compute&lt;/span&gt;&lt;/code&gt; call tells Dask to go ahead and actually evaluate everything we’ve told it to do so far (Dask is lazy by default). Otherwise, all of the mathematical operations, matrix multiplies, slicing, and so on are exactly the same as with Numpy, except that Dask.array builds up a chunk-wise parallel computation for us and Dask.distributed can execute that computation in parallel.&lt;/p&gt;
&lt;p&gt;To better appreciate all the scheduling that is happening in one update step of the above algorithm, here is a visualization of the computation necessary to compute &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;beta_hat&lt;/span&gt;&lt;/code&gt; and the new function value &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;new_func&lt;/span&gt;&lt;/code&gt;:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/grad-step-black-on-white.svg"
     width="100%"
     alt="Gradient descent step Dask graph"&gt;&lt;/p&gt;
&lt;p&gt;Each rectangle is an in-memory chunk of our distributed array and every circle
is a numpy function call on those in-memory chunks. The Dask scheduler
determines where and when to run all of these computations on our cluster of
machines (or just on the cores of our laptop).&lt;/p&gt;
&lt;section id="array-programming-dask-delayed"&gt;
&lt;h3&gt;Array Programming + dask.delayed&lt;/h3&gt;
&lt;p&gt;Now that we’ve seen how to use the built-in parallel algorithms offered by Dask.array, let’s go one step further and talk about writing more customized parallel algorithms. Many distributed “consensus” based algorithms in machine learning are based on the idea that each chunk of data can be processed independently in parallel, and send their guess for the optimal parameter value to some master node. The master then computes a &lt;em&gt;consensus&lt;/em&gt; estimate for the optimal parameters and reports that back to all of the workers. Each worker then processes their chunk of data given this new information, and the process continues until convergence.&lt;/p&gt;
&lt;p&gt;From a parallel computing perspective this is a pretty simple map-reduce procedure. Any distributed computing framework should be able to handle this easily. We’ll use this as a very simple example for how to use Dask’s more customizable parallel options.&lt;/p&gt;
&lt;p&gt;One such algorithm is the &lt;a class="reference external" href="http://stanford.edu/~boyd/admm.html"&gt;Alternating Direction Method of Multipliers&lt;/a&gt;, or ADMM for short. For the sake of this post, we will consider the work done by each worker to be a black box.&lt;/p&gt;
&lt;p&gt;We will also be considering a &lt;em&gt;regularized&lt;/em&gt; version of the problem above, namely:&lt;/p&gt;
&lt;div class="math notranslate nohighlight"&gt;
\[
\min_{\beta} \|y - X\beta\|_2^2 + \lambda \|\beta\|_1
\]&lt;/div&gt;
&lt;p&gt;At the end of the day, all we will do is:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;create NumPy functions which define how each chunk updates its parameter estimates&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;wrap those functions in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.delayed&lt;/span&gt;&lt;/code&gt;&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;dask.compute&lt;/span&gt;&lt;/code&gt; and process the individual estimates, again using NumPy&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;First we need to define some &lt;em&gt;local&lt;/em&gt; functions that the chunks will use to update their individual parameter estimates, and import the black box &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;local_update&lt;/span&gt;&lt;/code&gt; step from &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask_glm&lt;/span&gt;&lt;/code&gt;; also, we will need the so-called &lt;em&gt;shrinkage&lt;/em&gt; operator (which is the &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Proximal_operator"&gt;proximal operator&lt;/a&gt; for the &lt;span class="math notranslate nohighlight"&gt;\(l1\)&lt;/span&gt;-norm in our problem):&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_glm.algorithms&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;local_update&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;local_f&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;rho&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="n"&gt;y&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;dot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;rho&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;np&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;beta&lt;/span&gt; &lt;span class="o"&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;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                                                                  &lt;span class="n"&gt;beta&lt;/span&gt; &lt;span class="o"&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;u&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;local_grad&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;rho&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;X&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;T&lt;/span&gt;&lt;span class="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="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;beta&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;rho&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&lt;/span&gt; &lt;span class="o"&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;u&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;shrinkage&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&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="k"&gt;return&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;maximum&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;beta&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;t&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;maximum&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;beta&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;t&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;## set some algorithm parameters&lt;/span&gt;
&lt;span class="n"&gt;max_steps&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;10&lt;/span&gt;
&lt;span class="n"&gt;lamduh&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;7.2&lt;/span&gt;
&lt;span class="n"&gt;rho&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;n&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;p&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;shape&lt;/span&gt;
&lt;span class="n"&gt;nchunks&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;npartitions&lt;/span&gt;

&lt;span class="n"&gt;XD&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;to_delayed&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;flatten&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tolist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# A list of pointers to remote numpy arrays&lt;/span&gt;
&lt;span class="n"&gt;yD&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_delayed&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;flatten&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tolist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# ... one for each chunk&lt;/span&gt;

&lt;span class="c1"&gt;# the initial consensus estimate&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;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;zeros&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;p&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# an array of the individual &amp;quot;dual variables&amp;quot; and parameter estimates,&lt;/span&gt;
&lt;span class="c1"&gt;# one for each chunk of data&lt;/span&gt;
&lt;span class="n"&gt;u&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;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;zeros&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;p&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="n"&gt;nchunks&lt;/span&gt;&lt;span class="p"&gt;)])&lt;/span&gt;
&lt;span class="n"&gt;betas&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;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;zeros&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;p&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="n"&gt;nchunks&lt;/span&gt;&lt;span class="p"&gt;)])&lt;/span&gt;

&lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;k&lt;/span&gt; &lt;span class="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="n"&gt;max_steps&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;

    &lt;span class="c1"&gt;# process each chunk in parallel, using the black-box &amp;#39;local_update&amp;#39; magic&lt;/span&gt;
    &lt;span class="n"&gt;new_betas&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;local_update&lt;/span&gt;&lt;span class="p"&gt;)(&lt;/span&gt;&lt;span class="n"&gt;xx&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;yy&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;bb&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;uu&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;rho&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;local_f&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                                            &lt;span class="n"&gt;fprime&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;local_grad&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;xx&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;yy&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;bb&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;uu&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="nb"&gt;zip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;XD&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;yD&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;betas&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;
    &lt;span class="n"&gt;new_betas&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;array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;new_betas&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

    &lt;span class="c1"&gt;# everything else is NumPy code occurring at &amp;quot;master&amp;quot;&lt;/span&gt;
    &lt;span class="n"&gt;beta_hat&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;0.9&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;new_betas&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mf"&gt;0.1&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;

    &lt;span class="c1"&gt;# create consensus estimate&lt;/span&gt;
    &lt;span class="n"&gt;zold&lt;/span&gt; &lt;span class="o"&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;copy&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;ztilde&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;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta_hat&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;array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;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;z&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;shrinkage&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ztilde&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;lamduh&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;rho&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;nchunks&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

    &lt;span class="c1"&gt;# update dual variables&lt;/span&gt;
    &lt;span class="n"&gt;u&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="n"&gt;beta_hat&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;

&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="c1"&gt;# Number of coefficients zeroed out due to L1 regularization&lt;/span&gt;
&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="n"&gt;z&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;sum&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
&lt;span class="mi"&gt;12&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;There is of course a little bit more work occurring in the above algorithm, but it should be clear that the distributed operations are &lt;em&gt;not&lt;/em&gt; one of the difficult pieces. Using dask.delayed we were able to express a simple map-reduce algorithm like ADMM with similarly simple Python for loops and delayed function calls. Dask.delayed is keeping track of all of the function calls we wanted to make and what other function calls they depend on. For example all of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;local_update&lt;/span&gt;&lt;/code&gt; calls can happen independent of each other, but the consensus computation blocks on all of them.&lt;/p&gt;
&lt;p&gt;We hope that both parallel algorithms shown above (gradient descent, ADMM) were
straightforward to someone reading with an optimization background. These
implementations run well on a laptop, a single multi-core workstation, or a
thousand-node cluster if necessary. We’ve been building somewhat more
sophisticated implementations of these algorithms (and others) in
&lt;a class="reference external" href="https://github.com/dask/dask-glm"&gt;dask-glm&lt;/a&gt;. They are more sophisticated from an
optimization perspective (stopping criteria, step size, asynchronicity, and so on)
but remain as simple from a distributed computing perspective.&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/2017/03/22/dask-glm-1.md&lt;/span&gt;, line 271)&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&gt;
&lt;section id="experiment"&gt;
&lt;h1&gt;Experiment&lt;/h1&gt;
&lt;p&gt;&lt;em&gt;We compare dask-glm implementations against Scikit-learn on a laptop, and then
show them running on a cluster.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/a0f0826ea7f4463c8c99880f1893a43f"&gt;Reproducible notebook is available here&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;We’re building more sophisticated versions of the algorithms above in
&lt;a class="reference external" href="https://github.com/dask/dask-glm"&gt;dask-glm&lt;/a&gt;.  This project has convex
optimization algorithms for gradient descent, proximal gradient descent,
Newton’s method, and ADMM.  These implementations extend the implementations
above by also thinking about stopping criteria, step sizes, and other niceties
that we avoided above for simplicity.&lt;/p&gt;
&lt;p&gt;In this section we show off these algorithms by performing a simple numerical
experiment that compares the numerical performance of proximal gradient descent
and ADMM alongside Scikit-Learn’s LogisticRegression and SGD implementations
on a single machine (a personal laptop) and then follows up by scaling the
dask-glm options to a moderate cluster.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Disclaimer: These experiments are crude. We’re using artificial data, we’re not tuning
parameters or even finding parameters at which these algorithms are producing
results of the same accuracy. The goal of this section is just to give a
general feeling of how things compare.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;We create 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="c1"&gt;## size of problem (no. observations)&lt;/span&gt;
&lt;span class="n"&gt;N&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;8e6&lt;/span&gt;
&lt;span class="n"&gt;chunks&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;1e6&lt;/span&gt;
&lt;span class="n"&gt;seed&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;20009&lt;/span&gt;
&lt;span class="n"&gt;beta&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;15&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mf"&gt;0.5&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="mi"&gt;3&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="n"&gt;N&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;&lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&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="n"&gt;chunks&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;make_y&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;beta&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;array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&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="n"&gt;chunks&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;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rebalance&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;y&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And run each of our algorithms 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="c1"&gt;# Dask-GLM Proximal Gradient&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;proximal_grad&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;lamduh&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# Dask-GLM ADMM&lt;/span&gt;
&lt;span class="n"&gt;X2&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;rechunk&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mf"&gt;1e5&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# ADMM prefers smaller chunks&lt;/span&gt;
&lt;span class="n"&gt;y2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;1e5&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;admm&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;lamduh&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# Scikit-Learn LogisticRegression&lt;/span&gt;
&lt;span class="n"&gt;nX&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;ny&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&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;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# sklearn wants numpy arrays&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;LogisticRegression&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;penalty&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;l1&amp;#39;&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="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;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nX&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;ny&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;coef_&lt;/span&gt;

&lt;span class="c1"&gt;# Scikit-Learn Stochastic Gradient Descent&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;SGDClassifier&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;loss&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;log&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                       &lt;span class="n"&gt;penalty&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;l1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                       &lt;span class="n"&gt;l1_ratio&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;n_iter&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="n"&gt;fit_intercept&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nX&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;ny&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;coef_&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We then compare with the &lt;span class="math notranslate nohighlight"&gt;\(L_{\infty}\)&lt;/span&gt; norm (largest different value).&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nb"&gt;abs&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;beta&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;max&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Times and &lt;span class="math notranslate nohighlight"&gt;\(L_\infty\)&lt;/span&gt; distance from the true “generative beta” for these parameters are shown in the table below:&lt;/p&gt;
&lt;table&gt;
&lt;thead&gt;&lt;tr&gt;
  &lt;th&gt;Algorithm&lt;/th&gt;
  &lt;th&gt;Error&lt;/th&gt;
  &lt;th&gt;Duration (s)&lt;/th&gt;
&lt;/tr&gt;&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr&gt;
  &lt;td&gt;Proximal Gradient&lt;/td&gt;
  &lt;td&gt;0.0227&lt;/td&gt;
  &lt;td&gt;128&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
  &lt;td&gt;ADMM&lt;/td&gt;
  &lt;td&gt;0.0125&lt;/td&gt;
  &lt;td&gt;34.7&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
  &lt;td&gt;LogisticRegression&lt;/td&gt;
  &lt;td&gt;0.0132&lt;/td&gt;
  &lt;td&gt;79&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
  &lt;td&gt;SGDClassifier&lt;/td&gt;
  &lt;td&gt;0.0456&lt;/td&gt;
  &lt;td&gt;29.4&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;Again, please don’t take these numbers too seriously: these algorithms all solve
regularized problems, so we don’t expect the results to necessarily be close to the
underlying generative beta (even asymptotically). The numbers above are meant to
demonstrate that they all return results which were roughly the same distance
from the beta above. Also, Dask-glm is using a full four-core laptop while
SKLearn is restricted to use a single core.&lt;/p&gt;
&lt;p&gt;In the sections below we include profile plots for proximal gradient and ADMM.
These show the operations that each of eight threads was doing over time. You
can mouse-over rectangles/tasks and zoom in using the zoom tools in the upper
right. You can see the difference in complexity of the algorithms. ADMM is
much simpler from Dask’s perspective but also saturates hardware better for
this chunksize.&lt;/p&gt;
&lt;section id="profile-plot-for-proximal-gradient-descent"&gt;
&lt;h2&gt;Profile Plot for Proximal Gradient Descent&lt;/h2&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/5dec93966e0daa2edb8fcde8449a5335/raw/b0f096cf72d31df02bda4e5ca3203a7347464808/dask-glm-proximal-grad-small.html"
        width="800" height="300"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="profile-plot-for-admm"&gt;
&lt;h2&gt;Profile Plot for ADMM&lt;/h2&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/5dec93966e0daa2edb8fcde8449a5335/raw/52f81540fd969e3ff4c69f1f44a401d25d9134ff/dask-glm-admm.html"
        width="800" height="300"&gt;&lt;/iframe&gt;
&lt;p&gt;The general takeaway here is that dask-glm performs comparably to Scikit-Learn
on a single machine. If your problem fits in memory on a single machine you
should continue to use Scikit-Learn and Statsmodels. The real benefit to the
dask-glm algorithms is that they &lt;em&gt;scale&lt;/em&gt; and can run efficiently on data that is
larger-than-memory by operating from disk on a single computer or on a
cluster of computers working together.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="cluster-computing"&gt;
&lt;h2&gt;Cluster Computing&lt;/h2&gt;
&lt;p&gt;As a demonstration, we run a larger version of the data above on a cluster of
eight &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;m4.2xlarges&lt;/span&gt;&lt;/code&gt; on EC2 (8 cores and 30GB of RAM each.)&lt;/p&gt;
&lt;p&gt;We create a larger dataset with 800,000,000 rows and 15 columns across eight
processes.&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;N&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;8e8&lt;/span&gt;
&lt;span class="n"&gt;chunks&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;1e7&lt;/span&gt;
&lt;span class="n"&gt;seed&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;20009&lt;/span&gt;
&lt;span class="n"&gt;beta&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;15&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mf"&gt;0.5&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="mi"&gt;3&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="n"&gt;N&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;&lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&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="n"&gt;chunks&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;make_y&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;beta&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;array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;beta&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="n"&gt;chunks&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;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We then run the same &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;proximal_grad&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;admm&lt;/span&gt;&lt;/code&gt; operations from before:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Dask-GLM Proximal Gradient&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;proximal_grad&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;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;lamduh&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# Dask-GLM ADMM&lt;/span&gt;
&lt;span class="n"&gt;X2&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;rechunk&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mf"&gt;1e6&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# ADMM prefers smaller chunks&lt;/span&gt;
&lt;span class="n"&gt;y2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;1e6&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;admm&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;lamduh&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;alpha&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Proximal grad completes in around seventeen minutes while ADMM completes in
around four minutes. Profiles for the two computations are included below:&lt;/p&gt;
&lt;section id="id1"&gt;
&lt;h3&gt;Profile Plot for Proximal Gradient Descent&lt;/h3&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/2017/03/22/dask-glm-1.md&lt;/span&gt;, line 439); &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: “profile plot for proximal gradient descent”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;We include only the first few iterations here. Otherwise this plot is several
megabytes.&lt;/p&gt;
&lt;p&gt;&lt;a href="https://cdn.rawgit.com/mrocklin/c9f1724285af29cfa50fd1430178c5af/raw/0d0edca0fa97a70c0e90699eb41c51e23f503ea6/dask-glm-proximal-grad-large.html"&gt;Link to fullscreen plot&lt;/a&gt;&lt;/p&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/c9f1724285af29cfa50fd1430178c5af/raw/0d0edca0fa97a70c0e90699eb41c51e23f503ea6/dask-glm-proximal-grad-large.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;/section&gt;
&lt;section id="id2"&gt;
&lt;h3&gt;Profile Plot for ADMM&lt;/h3&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/2017/03/22/dask-glm-1.md&lt;/span&gt;, line 449); &lt;em&gt;&lt;a href="#id2"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “profile plot for admm”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;&lt;a href="https://cdn.rawgit.com/mrocklin/c9f1724285af29cfa50fd1430178c5af/raw/0d0edca0fa97a70c0e90699eb41c51e23f503ea6/dask-glm-proximal-grad-large.html"&gt;Link to fullscreen plot&lt;/a&gt;&lt;/p&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/c9f1724285af29cfa50fd1430178c5af/raw/0d0edca0fa97a70c0e90699eb41c51e23f503ea6/dask-glm-admm-large.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;These both obtained similar &lt;span class="math notranslate nohighlight"&gt;\(L_{\infty}\)&lt;/span&gt; errors to what we observed before.&lt;/p&gt;
&lt;table&gt;
&lt;thead&gt;&lt;tr&gt;
  &lt;th&gt;Algorithm&lt;/th&gt;
  &lt;th&gt;Error&lt;/th&gt;
  &lt;th&gt;Duration (s)&lt;/th&gt;
&lt;/tr&gt;&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr&gt;
  &lt;td&gt;Proximal Gradient&lt;/td&gt;
  &lt;td&gt;0.0306&lt;/td&gt;
  &lt;td&gt;1020&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
  &lt;td&gt;ADMM&lt;/td&gt;
  &lt;td&gt;0.00159&lt;/td&gt;
  &lt;td&gt;270&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;Although this time we had to be careful about a couple of things:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;We explicitly deleted the old data after rechunking (ADMM prefers different
chunksizes than proximal_gradient) because our full dataset, 100GB, is
close enough to our total distributed RAM (240GB) that it’s a good idea to
avoid keeping replias around needlessly. Things would have run fine, but
spilling excess data to disk would have negatively affected performance.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We set the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;OMP_NUM_THREADS=1&lt;/span&gt;&lt;/code&gt; environment variable to avoid
over-subscribing our CPUs. Surprisingly not doing so led both to worse
performance and to non-deterministic results. An issue that we’re still
tracking down.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="analysis"&gt;
&lt;h2&gt;Analysis&lt;/h2&gt;
&lt;p&gt;The algorithms in Dask-GLM are new and need development, but are in a usable
state by people comfortable operating at this technical level. Additionally,
we would like to attract other mathematical and algorithmic developers to this
work. We’ve found that Dask provides a nice balance between being flexible
enough to support interesting algorithms, while being managed enough to be
usable by researchers without a strong background in distributed systems. In
this section we’re going to discuss the things that we learned from both
Chris’ (mathematical algorithms) and Matt’s (distributed systems) perspective
and then talk about possible future work. We encourage people to pay attention
to future work; we’re open to collaboration and think that this is a good
opportunity for new researchers to meaningfully engage.&lt;/p&gt;
&lt;section id="chris-s-perspective"&gt;
&lt;h3&gt;Chris’s perspective&lt;/h3&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Creating distributed algorithms with Dask was surprisingly easy; there is
still a small learning curve around when to call things like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;persist&lt;/span&gt;&lt;/code&gt;,
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;compute&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;rebalance&lt;/span&gt;&lt;/code&gt;, and so on, but that can’t be avoided. Using Dask for
algorithm development has been a great learning environment for
understanding the unique challenges associated with distributed algorithms
(including communication costs, among others).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Getting the particulars of algorithms correct is non-trivial; there is still
work to be done in better understanding the tolerance settings vs. accuracy
tradeoffs that are occurring in many of these algorithms, as well as
fine-tuning the convergence criteria for increased precision.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;On the software development side, reliably testing optimization algorithms
is hard. Finding provably correct optimality conditions that should be
satisfied &lt;em&gt;which are also numerically stable&lt;/em&gt; has been a challenge for me.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Working on algorithms in isolation is not nearly as fun as collaborating on
them; please join the conversation and contribute!&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Most importantly from my perspective, I’ve found there is a surprisingly
large amount of misunderstanding in “the community” surrounding what
optimization algorithms do in the world of predictive modeling, what
problems they each individually solve, and whether or not they are
interchangeable for a given problem. For example, Newton’s method can’t be
used to optimize an l1-regularized problem, and the coefficient estimates
from an l1-regularized problem are fundamentally (and numerically) different
from those of an l2-regularized problem (and from those of an unregularized
problem). My own personal goal is that the API for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-glm&lt;/span&gt;&lt;/code&gt; exposes these
subtle distinctions more transparently and leads to more thoughtful modeling
decisions “in the wild”.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
&lt;section id="matt-s-perspective"&gt;
&lt;h3&gt;Matt’s perspective&lt;/h3&gt;
&lt;p&gt;This work triggered a number of concrete changes within the Dask library:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;We can convert Dask.dataframes to Dask.arrays. This is particularly
important because people want to do pre-processing with dataframes but then
switch to efficient multi-dimensional arrays for algorithms.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We had to unify the single-machine scheduler and distributed scheduler APIs
a bit, notably adding a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;persist&lt;/span&gt;&lt;/code&gt; function to the single machine
scheduler. This was particularly important because Chris
generally prototyped on his laptop but we wanted to write code that was
effective on clusters.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scheduler overhead can be a problem for the iterative dask-array algorithms
(gradient descent, proximal gradient descent, BFGS). This is particularly
a problem because NumPy is very fast. Often our tasks take only a few
milliseconds, which makes Dask’s overhead of 200us per task become very
relevant (this is why you see whitespace in the profile plots above).
We’ve started resolving this problem in a few ways like more aggressive
task fusion and lower overheads generally, but this will be a medium-term
challenge. In practice for dask-glm we’ve started handling this just by
choosing chunksizes well. I suspect that for the dask-glm in particular
we’ll just develop auto-chunksize heuristics that will mostly solve this
problem. However we expect this problem to recur in other work with
scientists on HPC systems who have similar situations.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A couple of things can be tricky for algorithmic users:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Placing the calls to asynchronously start computation (persist,
compute). In practice Chris did a good job here and then I came through
and tweaked things afterwards. The web diagnostics ended up being
crucial to identify issues.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Avoiding accidentally calling NumPy functions on dask.arrays and vice
versa. We’ve improved this on the dask.array side, and they now operate
intelligently when given numpy arrays. Changing this on the NumPy side
is harder until NumPy protocols change (which is planned).&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
&lt;section id="future-work"&gt;
&lt;h3&gt;Future work&lt;/h3&gt;
&lt;p&gt;There are a number of things we would like to do, both in terms of measurement
and for the dask-glm project itself. We welcome people to voice their opinions
(and join development) on the following issues:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-glm/issues/5"&gt;Asynchronous Algorithms&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-glm/issues/11"&gt;User APIs&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-glm/issues/35"&gt;Extend GLM families&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Write more extensive rigorous algorithm testing - for &lt;a class="reference external" href="https://github.com/dask/dask-glm/issues/7"&gt;satisfying provable optimality criteria&lt;/a&gt;, and for &lt;a class="reference external" href="https://github.com/dask/dask-glm/issues/9"&gt;robustness to various input data&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask-glm/issues/34"&gt;Begin work on smart initialization routines&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;What is your perspective here, gentle reader? Both Matt and Chris can use help
on this project. We hope that some of the issues above provide seeds for
community engagement. We welcome other questions, comments, and contributions
either as github issues or comments below.&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/2017/03/22/dask-glm-1.md&lt;/span&gt;, line 584)&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&gt;
&lt;section id="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;Thanks also go to &lt;a class="reference external" href="https://github.com/hussainsultan"&gt;Hussain Sultan&lt;/a&gt; (Capital
One) and &lt;a class="reference external" href="https://github.com/TomAugspurger"&gt;Tom Augspurger&lt;/a&gt; for collaboration
on Dask-GLM and to &lt;a class="reference external" href="https://github.com/electronwill"&gt;Will Warner&lt;/a&gt; (Continuum)
for reviewing and editing this post.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/03/22/dask-glm-1/"/>
    <summary>This work is supported by Continuum Analytics,
the XDATA Program,
and the Data Driven Discovery Initiative from the Moore
Foundation.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-03-22T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/02/27/dask-0.14.0/</id>
    <title>Dask Release 0.14.0</title>
    <updated>2017-02-27T00: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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2017/02/27/dask-0.14.0.md&lt;/span&gt;, line 13)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;Dask just released version 0.14.0. This release contains some significant
internal changes as well as the usual set of increased API coverage and bug
fixes. This blogpost outlines some of the major changes since the last release
January, 27th 2017.&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Structural sharing of graphs between collections&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Refactor communications system&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Many small dataframe improvements&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Top-level persist function&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;You can install new versions using Conda or Pip&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 -c conda-forge dask distributed
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or&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[complete] distributed --upgrade
&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/2017/02/27/dask-0.14.0.md&lt;/span&gt;, line 33)&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="share-graphs-between-collections"&gt;
&lt;h1&gt;Share Graphs between Collections&lt;/h1&gt;
&lt;p&gt;Dask collections (arrays, bags, dataframes, delayed) hold onto task graphs that
have all of the tasks necessary to create the desired result. For larger
datasets or complex calculations these graphs may have thousands, or sometimes
even millions of tasks. In some cases the overhead of handling these graphs
can become significant.&lt;/p&gt;
&lt;p&gt;This is especially true because dask collections don’t modify their graphs in
place, they make new graphs with updated computations. Copying graph data
structures with millions of nodes can take seconds and interrupt interactive
workflows.&lt;/p&gt;
&lt;p&gt;To address this dask.arrays and dask.delayed collections now use special graph
data structures with structural sharing. This significantly cuts down on the
amount of overhead when building repetitive computations.&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;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="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;1000000&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="c1"&gt;# 1000 chunks of size 1000&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;section id="version-0-13-0"&gt;
&lt;h2&gt;Version 0.13.0&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="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;time&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;100&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&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;2.69&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;96&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;2.78&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;2.78&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="version-0-14-0"&gt;
&lt;h2&gt;Version 0.14.0&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="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;time&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;100&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&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;756&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;8&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;764&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;763&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The difference in this toy problem is moderate but for real world cases this
can difference can grow fairly large. This was also one of the blockers
identified by the climate science community stopping them from handling
petabyte scale analyses.&lt;/p&gt;
&lt;p&gt;We chose to roll this out for arrays and delayed first just because those are
the two collections that typically produce large task graphs. Dataframes and
bags remain as before for the time being.&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/2017/02/27/dask-0.14.0.md&lt;/span&gt;, line 81)&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="communications-system"&gt;
&lt;h1&gt;Communications System&lt;/h1&gt;
&lt;p&gt;Dask communicates over TCP sockets. It uses Tornado’s IOStreams to handle
non-blocking communication, framing, etc.. We’ve run into some performance
issues with Tornado when moving large amounts of data. Some of this &lt;a class="reference external" href="https://github.com/tornadoweb/tornado/pull/1873"&gt;has been
improved upstream&lt;/a&gt; in Tornado
directly, but we still want the ability to optionally drop Tornado’s
byte-handling communication stack in the future. This is especially important
as dask gets used in institutions with faster and more exotic interconnects
(supercomputers). We’ve been asked a few times to support other transport
mechanisms like MPI.&lt;/p&gt;
&lt;p&gt;The first step (and probably hardest step) was to make Dask’s communication
system is pluggable so that we can use different communication options without
significant source-code changes. We managed this a month ago and now it is
possible to add other transports to Dask relatively easily. TCP remains the
only real choice today though there is also an experimental ZeroMQ option
(which provides little-to-no performance benefit over TCP) as well as a fully
in-memory option in development.&lt;/p&gt;
&lt;p&gt;For users the main difference you’ll see is that &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;tcp://&lt;/span&gt;&lt;/code&gt; is now prepended many
places. For example:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ dask-scheduler
distributed.scheduler - INFO - -----------------------------------------------
distributed.scheduler - INFO -   Scheduler at:  tcp://192.168.1.115:8786
...
&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/2017/02/27/dask-0.14.0.md&lt;/span&gt;, line 111)&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="variety-of-dataframe-changes"&gt;
&lt;h1&gt;Variety of Dataframe Changes&lt;/h1&gt;
&lt;p&gt;As usual the Pandas API has been more fully covered by community contributors.
Some representative changes include the following:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;Support non-uniform categoricals: We no longer need to do a full pass
through the data when categorizing a column. Instead we categorize each
partition independently (even if they have different category values) and
then unify these categories only when necessary&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;df&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="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;astype&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;category&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# this is now fast&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Groupby cumulative reductions&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cumsum&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Support appending to Parquet collections&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/path/to/foo.parquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A new string and HTML representation of dask.dataframes. Typically Pandas
prints dataframes on the screen by rendering the first few rows of data.
However, Because Dask.dataframes are lazy we don’t have this data and so
typically render some metadata about the dataframe&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;df&lt;/span&gt;  &lt;span class="c1"&gt;# version 0.13.0&lt;/span&gt;
&lt;span class="go"&gt;dd.DataFrame&amp;lt;make-ti..., npartitions=366, divisions=(Timestamp(&amp;#39;2000-01-01&lt;/span&gt;
&lt;span class="go"&gt;00:00:00&amp;#39;, freq=&amp;#39;D&amp;#39;), Timestamp(&amp;#39;2000-01-02 00:00:00&amp;#39;, freq=&amp;#39;D&amp;#39;),&lt;/span&gt;
&lt;span class="go"&gt;Timestamp(&amp;#39;2000-01-03 00:00:00&amp;#39;, freq=&amp;#39;D&amp;#39;), ..., Timestamp(&amp;#39;2000-12-31&lt;/span&gt;
&lt;span class="go"&gt;00:00:00&amp;#39;, freq=&amp;#39;D&amp;#39;), Timestamp(&amp;#39;2001-01-01 00:00:00&amp;#39;, freq=&amp;#39;D&amp;#39;))&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This rendering, while informative, can be improved. Now we render
dataframes as a Pandas dataframe, but place metadata in the dataframe
instead of the actual 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;df&lt;/span&gt;  &lt;span class="c1"&gt;# version 0.14.0&lt;/span&gt;
&lt;span class="go"&gt;Dask DataFrame Structure:&lt;/span&gt;
&lt;span class="go"&gt;                       x        y      z&lt;/span&gt;
&lt;span class="go"&gt;npartitions=366&lt;/span&gt;
&lt;span class="go"&gt;2000-01-01       float64  float64  int64&lt;/span&gt;
&lt;span class="go"&gt;2000-01-02           ...      ...    ...&lt;/span&gt;
&lt;span class="go"&gt;...                  ...      ...    ...&lt;/span&gt;
&lt;span class="go"&gt;2000-12-31           ...      ...    ...&lt;/span&gt;
&lt;span class="go"&gt;2001-01-01           ...      ...    ...&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: make-timeseries, 366 tasks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Additionally this renders nicely as an HTML table in a Jupyter notebook&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/2017/02/27/dask-0.14.0.md&lt;/span&gt;, line 169)&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="variety-of-distributed-system-changes"&gt;
&lt;h1&gt;Variety of Distributed System Changes&lt;/h1&gt;
&lt;p&gt;There have also been a wide variety of changes to the distributed system. I’ll
include a representative sample here to give a flavor of what has been
happening:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;Ensure first-come-first-served priorities when dealing with multiple
clients&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Send small amounts of data through Channels. Channels are a way for
multiple clients/users connected to the same scheduler to publish and
exchange data between themselves. Previously they only transmitted Futures
(which could in trun point to larger data living on the cluster). However
we found that it was useful to communicate small bits of metadata as well,
for example to signal progress or stopping critera between clients
collaborating on the same workloads. Now you can publish any msgpack
serializable data on Channels.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Publishing Client&lt;/span&gt;
&lt;span class="n"&gt;scores&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;channel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scores&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;scores&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;123.456&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# Subscribing Client&lt;/span&gt;
&lt;span class="n"&gt;scores&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;channel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scores&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="k"&gt;while&lt;/span&gt; &lt;span class="n"&gt;scores&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="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;&amp;lt;&lt;/span&gt; &lt;span class="n"&gt;THRESHOLD&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="o"&gt;...&lt;/span&gt; &lt;span class="k"&gt;continue&lt;/span&gt; &lt;span class="n"&gt;working&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We’re better at estimating the size in data of SciPy Sparse matrices and
Keras models. This allows Dask to make smarter choices about when it
should and should not move data around for load balancing. Additionally
Dask can now also serialize Keras models.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;To help people deploying on clusters that have a shared network file system
(as is often the case in scientific or academic institutions) the scheduler
and workers can now communicate connection information using the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;--scheduler-file&lt;/span&gt;&lt;/code&gt; keyword&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt; &lt;span class="o"&gt;--&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;file&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;to&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;json&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;worker&lt;/span&gt; &lt;span class="o"&gt;--&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;file&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;to&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;json&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;worker&lt;/span&gt; &lt;span class="o"&gt;--&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;file&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;to&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;json&lt;/span&gt;

&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;scheduler_file&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/path/to/scheudler.json&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;Previously we needed to communicate the address of the scheduler, which
could be challenging when we didn’t know on which node the scheduler would
be run.&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/2017/02/27/dask-0.14.0.md&lt;/span&gt;, line 218)&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="other"&gt;
&lt;h1&gt;Other&lt;/h1&gt;
&lt;p&gt;There are a number of smaller details not mentioned in this blogpost. For more
information visit the changelogs and documentation&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask.pydata.org/en/latest/"&gt;Dask/dask docs&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask.pydata.org/en/latest/changelog.html"&gt;Dask/dask changelog&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://distributed.readthedocs.org/en/latest/"&gt;Dask/distributed docs&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://distributed.readthedocs.org/en/latest/changelog.html"&gt;Dask/distributed changelog&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Additionally a great deal of Dask work over the last month has happened
&lt;em&gt;outside&lt;/em&gt; of these core dask repositories.&lt;/p&gt;
&lt;p&gt;You can install or upgrade using Conda or Pip&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 -c conda-forge dask distributed
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or&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[complete] distributed --upgrade
&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/2017/02/27/dask-0.14.0.md&lt;/span&gt;, line 239)&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;Since the last 0.13.0 release on January 27th the following developers have
contributed to the dask/dask repository:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Chris Barber&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Daniel Davis&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Elmar Ritsch&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Erik Welch&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;jakirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John Crickett&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;jspreston&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Juan Luis Cano Rodríguez&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;kayibal&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kevin Ernst&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Markus Gonser&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Nir&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Sinhrks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Talmaj Marinc&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Vlad Frolov&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Will Warner&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;And the following developers have contributed to the dask/distributed
repository:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Ben Schreck&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;bmaisonn&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Brett Naul&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Demian Wassermann&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Israel Saeta Pérez&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;John Crickett&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joseph Crail&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Malte Gerken&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Min RK&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;strets123&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/02/27/dask-0.14.0/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-02-27T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/02/20/dask-dev-7/</id>
    <title>Dask Development Log</title>
    <updated>2017-02-20T00: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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;To increase transparency I’m blogging weekly(ish) about the work done on Dask
and related projects during the previous week. This log covers work done
between 2017-02-01 and 2017-02-20. Nothing here is ready for production. This
blogpost is written in haste, so refined polish should not be expected.&lt;/p&gt;
&lt;p&gt;Themes of the last couple of weeks:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Profiling experiments with Dask-GLM&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Subsequent graph optimizations, both non-linear fusion and avoiding
repeatedly creating new graphs&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tensorflow and Keras experiments&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;XGBoost experiments&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask tutorial refactor&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Google Cloud Storage support&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Cleanup of Dask + SKLearn project&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/2017/02/20/dask-dev-7.md&lt;/span&gt;, line 29)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="dask-glm-and-iterative-algorithms"&gt;

&lt;p&gt;Dask-GLM is currently just a bunch of solvers like Newton, Gradient Descent,
BFGS, Proximal Gradient Descent, and ADMM. These are useful in solving
problems like logistic regression, but also several others. The mathematical
side of this work is mostly done by &lt;a class="reference external" href="https://github.com/moody-marlin/"&gt;Chris White&lt;/a&gt;
and &lt;a class="reference external" href="https://github.com/hussainsultan"&gt;Hussain Sultan&lt;/a&gt; at Capital One.&lt;/p&gt;
&lt;p&gt;We’ve been using this project also to see how Dask can scale out machine
learning algorithms. To this end we ran a few benchmarks here:
https://github.com/dask/dask-glm/issues/26 . This just generates and solves
some random problems, but at larger scales.&lt;/p&gt;
&lt;p&gt;What we found is that some algorithms, like ADMM perform beautifully, while
for others, like gradient descent, scheduler overhead can become a substantial
bottleneck at scale. This is mostly just because the actual in-memory
NumPy operations are so fast; any sluggishness on Dask’s part becomes very
apparent. Here is a profile of gradient descent:&lt;/p&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/e7bcb979e147102bf9ac428ed9074000/raw/d38234f3e9072816bea98d032f1e4f9e618242c3/task-stream-glm-gradient-descent.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;Notice all the white space. This is Dask figuring out what to do during
different iterations. We’re now working to bring this down to make all of the
colored parts of this graph squeeze together better. This will result in
general overhead improvements throughout the project.&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/2017/02/20/dask-dev-7.md&lt;/span&gt;, line 56)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="graph-optimizations-aggressive-fusion"&gt;
&lt;h1&gt;Graph Optimizations - Aggressive Fusion&lt;/h1&gt;
&lt;p&gt;We’re approaching this in two ways:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;More aggressively fuse tasks together so that there are fewer blocks for
the scheduler to think about&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Avoid repeated work when generating very similar graphs&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In the first case, Dask already does standard task fusion. For example, if you
have the following to 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="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;w&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;g&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;z&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;h&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Dask (along with every other compiler-like project since the 1980’s) already
turns this into 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="n"&gt;z&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;h&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;g&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="n"&gt;w&lt;/span&gt;&lt;span class="p"&gt;)))&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;What’s tricky with a lot of these mathematical or optimization algorithms
though is that they are mostly, but not entirely linear. Consider the
following example:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;exp&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Visualized as a node-link diagram, this graph looks like a diamond like the following:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;         &lt;span class="n"&gt;o&lt;/span&gt;  &lt;span class="n"&gt;exp&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mi"&gt;1&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;exp&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;o&lt;/span&gt;   &lt;span class="n"&gt;o&lt;/span&gt;   &lt;span class="mi"&gt;1&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;o&lt;/span&gt;  &lt;span class="n"&gt;x&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Graphs like this generally don’t get fused together because we &lt;em&gt;could&lt;/em&gt; compute
both &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;exp(x)&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;1/x&lt;/span&gt;&lt;/code&gt; in parallel. However when we’re bound by scheduling
overhead and when we have plenty of parallel work to do, we’d prefer to fuse
these into a single task, even though we lose some potential parallelism.
There is a tradeoff here and we’d like to be able to exchange some parallelism
(of which we have a lot) for less overhead.&lt;/p&gt;
&lt;p&gt;PR here &lt;a class="reference external" href="https://github.com/dask/dask/pull/1979"&gt;dask/dask #1979&lt;/a&gt; by &lt;a class="reference external" href="https://github.com/eriknw"&gt;Erik
Welch&lt;/a&gt; (Erik has written and maintained most of
Dask’s graph optimizations).&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/2017/02/20/dask-dev-7.md&lt;/span&gt;, line 109)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="graph-optimizations-structural-sharing"&gt;
&lt;h1&gt;Graph Optimizations - Structural Sharing&lt;/h1&gt;
&lt;p&gt;Additionally, we no longer make copies of graphs in dask.array. Every
collection like a dask.array or dask.dataframe holds onto a Python dictionary
holding all of the tasks that are needed to construct that array. When we
perform an operation on a dask.array we get a new dask.array with a new
dictionary pointing to a new graph. The new graph generally has all of the
tasks of the old graph, plus a few more. As a result, we frequently make
copies of the underlying task 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="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="k"&gt;assert&lt;/span&gt; &lt;span class="nb"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;issuperset&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;dask&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Normally this doesn’t matter (copying graphs is usually cheap) but it can
become very expensive for large arrays when you’re doing many mathematical
operations.&lt;/p&gt;
&lt;p&gt;Now we keep dask graphs in a custom mapping (dict-like object) that shares
subgraphs with other arrays. As a result, we rarely make unnecessary copies
and some algorithms incur far less overhead. Work done in
&lt;a class="reference external" href="https://github.com/dask/dask/pull/1985"&gt;dask/dask #1985&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/2017/02/20/dask-dev-7.md&lt;/span&gt;, line 133)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="tensorflow-and-keras-experiments"&gt;
&lt;h1&gt;TensorFlow and Keras experiments&lt;/h1&gt;
&lt;p&gt;Two weeks ago I gave a talk with &lt;a class="reference external" href="https://github.com/seibert"&gt;Stan Seibert&lt;/a&gt;
(Numba developer) on Deep Learning (Stan’s bit) and Dask (my bit). As part of
that talk I decided to launch tensorflow from Dask and feed Tensorflow from a
distributed Dask array. See &lt;a class="reference internal" href="../2017/02/11/dask-tensorflow/"&gt;&lt;span class="doc std std-doc"&gt;this
blogpost&lt;/span&gt;&lt;/a&gt; for
more information.&lt;/p&gt;
&lt;p&gt;That experiment was nice in that it showed how easy it is to deploy and
interact with other distributed servies from Dask. However from a deep
learning perspective it was immature. Fortunately, it succeeded in attracting
the attention of other potential developers (the true goal of all blogposts)
and now &lt;a class="reference external" href="https://github.com/bnaul"&gt;Brett Naul&lt;/a&gt; is using Dask to manage his GPU
workloads with Keras. Brett &lt;a class="reference external" href="https://github.com/dask/distributed/pull/878"&gt;contributed
code&lt;/a&gt; to help Dask move around
Keras models. He seems to particularly value &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/resources.html"&gt;Dask’s ability to manage
resources&lt;/a&gt; to help
him fully saturate the GPUs on his workstation.&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/2017/02/20/dask-dev-7.md&lt;/span&gt;, line 153)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="xgboost-experiments"&gt;
&lt;h1&gt;XGBoost experiments&lt;/h1&gt;
&lt;p&gt;After deploying Tensorflow we asked what would it take to do the same for
XGBoost, another very popular (though very different) machine learning library.
The conversation for that is here: &lt;a class="reference external" href="https://github.com/dmlc/xgboost/issues/2032"&gt;dmlc/xgboost
#2032&lt;/a&gt; with prototype code here
&lt;a class="reference external" href="https://github.com/mrocklin/dask-xgboost"&gt;mrocklin/dask-xgboost&lt;/a&gt;. As with
TensorFlow, the integration is relatively straightforward (if perhaps a bit
simpler in this case). The challenge for me is that I have little concrete
experience with the applications that these libraries were designed to solve.
Feedback and collaboration from open source developers who use these libraries
in production is welcome.&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/2017/02/20/dask-dev-7.md&lt;/span&gt;, line 166)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="dask-tutorial-refactor"&gt;
&lt;h1&gt;Dask tutorial refactor&lt;/h1&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://github.com/dask/dask-tutorial"&gt;dask/dask-tutorial&lt;/a&gt; project on
Github was originally written or PyData Seattle in July 2015 (roughly 19 months
ago). Dask has evolved substantially since then but this is still our only
educational material. Fortunately &lt;a class="reference external" href="http://martindurant.github.io/"&gt;Martin
Durant&lt;/a&gt; is doing a &lt;a class="reference external" href="https://github.com/dask/dask-tutorial/pull/29"&gt;pretty
serious rewrite&lt;/a&gt;, both correcting parts that are no longer modern API, and also
adding in new material around distributed computing and debugging.&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/2017/02/20/dask-dev-7.md&lt;/span&gt;, line 176)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="google-cloud-storage"&gt;
&lt;h1&gt;Google Cloud Storage&lt;/h1&gt;
&lt;p&gt;Dask developers (mostly Martin) maintain libraries to help Python users connect
to distributed file systems like HDFS (with
&lt;a class="reference external" href="http://hdfs3.readthedocs.io/en/latest/"&gt;hdfs3&lt;/a&gt;, S3 (with
&lt;a class="reference external" href="http://s3fs.readthedocs.io/en/latest/"&gt;s3fs&lt;/a&gt;, and Azure Data Lake (with
&lt;a class="reference external" href="https://github.com/Azure/azure-data-lake-store-python"&gt;adlfs&lt;/a&gt;), which
subsequently become usable from Dask. Martin has been working on support for
Google Cloud Storage (with &lt;a class="reference external" href="https://github.com/martindurant/gcsfs"&gt;gcsfs&lt;/a&gt;) with
another small project that uses the same API.&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/2017/02/20/dask-dev-7.md&lt;/span&gt;, line 187)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="cleanup-of-dask-sklearn-project"&gt;
&lt;h1&gt;Cleanup of Dask+SKLearn project&lt;/h1&gt;
&lt;p&gt;Last year Jim Crist published
&lt;a class="reference external" href="http://jcrist.github.io/dask-sklearn-part-1.html"&gt;three&lt;/a&gt;
&lt;a class="reference external" href="http://jcrist.github.io/dask-sklearn-part-2.html"&gt;great&lt;/a&gt;
&lt;a class="reference external" href="http://jcrist.github.io/dask-sklearn-part-3.html"&gt;blogposts&lt;/a&gt; about using Dask
with SKLearn. The result was a small library
&lt;a class="reference external" href="https://github.com/dask/dask-learn"&gt;dask-learn&lt;/a&gt; that had a variety of
features, some incredibly useful, like a cluster-ready Pipeline and
GridSearchCV, other less so. Because of the experimental nature of this work
we had labeled the library “not ready for use”, which drew some curious
responses from potential users.&lt;/p&gt;
&lt;p&gt;Jim is now busy dusting off the project, removing less-useful parts and
generally reducing scope to strictly model-parallel algorithms.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/02/20/dask-dev-7/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-02-20T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/02/11/dask-tensorflow/</id>
    <title>Experiment with Dask and TensorFlow</title>
    <updated>2017-02-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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2017/02/11/dask-tensorflow.md&lt;/span&gt;, line 13)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;This post briefly describes potential interactions between Dask and TensorFlow
and then goes through a concrete example using them together for distributed
training with a moderately complex architecture.&lt;/p&gt;
&lt;p&gt;This post was written in haste and the attached experiment is of low quality,
see disclaimers below. A similar and much better example with XGBoost is
included in the comments at the end.&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/2017/02/11/dask-tensorflow.md&lt;/span&gt;, line 23)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="introduction"&gt;
&lt;h1&gt;Introduction&lt;/h1&gt;
&lt;p&gt;Dask and TensorFlow both provide distributed computing in Python.
TensorFlow excels at deep learning applications while Dask is more generic.
We can combine both together in a few applications:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Simple data parallelism:&lt;/strong&gt; hyper-parameter searches during training
and predicting already-trained models against large datasets are both
trivial to distribute with Dask as they would be trivial to distribute with
any distributed computing system (Hadoop/Spark/Flink/etc..) We won’t
discuss this topic much. It should be straightforward.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Deployment:&lt;/strong&gt; A common pain point with TensorFlow is that setup isn’t
well automated. This plagues all distributed systems, especially those
that are run on a wide variety of cluster managers (see &lt;a class="reference internal" href="../2016/09/22/cluster-deployments/"&gt;&lt;span class="doc std std-doc"&gt;cluster deployment
blogpost&lt;/span&gt;&lt;/a&gt;
for more information). Fortunately, if you already have a Dask cluster
running it’s trivial to stand up a distributed TensorFlow network on
top of it running within the same processes.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Pre-processing:&lt;/strong&gt; We pre-process data with dask.dataframe or dask.array,
and then hand that data off to TensorFlow for training. If Dask and
TensorFlow are co-located on the same processes then this movement is
efficient. Working together we can build efficient and general use deep
learning pipelines.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In this blogpost we look &lt;em&gt;very&lt;/em&gt; briefly at the first case of simple
parallelism. Then go into more depth on an experiment that uses Dask and
TensorFlow in a more complex situation. We’ll find we can accomplish a fairly
sophisticated workflow easily, both due to how sensible TensorFlow is to set up
and how flexible Dask can be in advanced situations.&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/2017/02/11/dask-tensorflow.md&lt;/span&gt;, line 53)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="motivation-and-disclaimers"&gt;
&lt;h1&gt;Motivation and Disclaimers&lt;/h1&gt;
&lt;p&gt;Distributed deep learning is fundamentally changing the way humanity solves
some very hard computing problems like natural language translation,
speech-to-text transcription, image recognition, etc.. However, distributed
deep learning also suffers from public excitement, which may distort our image
of its utility. Distributed deep learning is not always the correct choice for
most problems. This is for two reasons:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Focusing on &lt;strong&gt;single machine&lt;/strong&gt; computation is often a better use of time.
Model design, GPU hardware, etc. can have a more dramatic impact than
scaling out. For newcomers to deep learning, watching &lt;a class="reference external" href="https://simons.berkeley.edu/talks/tutorial-deep-learning"&gt;online video lecture
series&lt;/a&gt; may be a
better use of time than reading this blogpost.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Traditional machine learning&lt;/strong&gt; techniques like logistic regression, and
gradient boosted trees can be more effective than deep learning if you have
finite data. They can also sometimes provide valuable interpretability
results.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Regardless, there are some concrete take-aways, even if distributed deep
learning is not relevant to your application:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;TensorFlow is straightforward to set up from Python&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask is sufficiently flexible out of the box to support complex settings
and workflows&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We’ll see an example of a typical distributed learning approach that
generalizes beyond deep learning.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Additionally the author does not claim expertise in deep learning and wrote
this blogpost in haste.&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/2017/02/11/dask-tensorflow.md&lt;/span&gt;, line 84)&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="simple-parallelism"&gt;
&lt;h1&gt;Simple Parallelism&lt;/h1&gt;
&lt;p&gt;Most parallel computing is simple. We easily apply one function to lots of
data, perhaps with slight variation. In the case of deep learning this
can enable a couple of common workflows:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;Build many different models, train each on the same data, choose the best
performing one. Using dask’s concurrent.futures interface, this looks
something 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="c1"&gt;# Hyperparameter search&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;dask-scheduler-address:8786&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;scores&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;train_and_evaluate&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;hyper_param_list&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;best&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;max&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scores&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;best&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Given an already-trained model, use it to predict outcomes on lots of data.
Here we use a big data collection like dask.dataframe:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Distributed prediction&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;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&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="o"&gt;...&lt;/span&gt; &lt;span class="c1"&gt;# do some preprocessing here&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;outcome&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_partitions&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;predict&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;These techniques are relatively straightforward if you have modest exposure to
Dask and TensorFlow (or any other machine learning library like scikit-learn),
so I’m going to ignore them for now and focus on more complex situations.&lt;/p&gt;
&lt;p&gt;Interested readers may find this blogpost on
&lt;a class="reference external" href="https://databricks.com/blog/2016/01/25/deep-learning-with-apache-spark-and-tensorflow.html"&gt;TensorFlow and Spark&lt;/a&gt;
of interest. It is a nice writeup that goes over these two techniques in more
detail.&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/2017/02/11/dask-tensorflow.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="a-distributed-tensorflow-application"&gt;
&lt;h1&gt;A Distributed TensorFlow Application&lt;/h1&gt;
&lt;p&gt;We’re going to replicate &lt;a class="reference external" href="https://github.com/tensorflow/tensorflow/blob/master/tensorflow/tools/dist_test/python/mnist_replica.py"&gt;this TensorFlow example&lt;/a&gt;
which uses multiple machines to train a model that fits in memory using
parameter servers for coordination. Our TensorFlow network will have three
different kinds of servers:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/tensorflow-distributed-network.svg"
     width="50%"
     align="right"
     alt="distributed TensorFlow training graph"&gt;&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Workers&lt;/strong&gt;: which will get updated parameters, consume training data, and
use that data to generate updates to send back to the parameter servers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Parameter Servers:&lt;/strong&gt; which will hold onto model parameters, synchronizing
with the workers as necessary&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Scorer:&lt;/strong&gt; which will periodically test the current parameters against
validation/test data and emit a current cross_entropy score to see how well
the system is running.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This is a fairly typical approach when the model can fit in one machine, but
when we want to use multiple machines to accelerate training or because data
volumes are too large.&lt;/p&gt;
&lt;p&gt;We’ll use TensorFlow to do all of the actual training and scoring. We’ll use
Dask to do everything else. In particular, we’re about to do the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Prepare data with dask.array&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Set up TensorFlow workers as long-running tasks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Feed data from Dask to TensorFlow while scores remain poor&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Let TensorFlow handle training using its own network&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/2017/02/11/dask-tensorflow.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="prepare-data-with-dask-array"&gt;
&lt;h1&gt;Prepare Data with Dask.array&lt;/h1&gt;
&lt;p&gt;For this toy example we’re just going to use the mnist data that comes with
TensorFlow. However, we’ll artificially inflate this data by concatenating
it to itself many times across a cluster:&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;get_mnist&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;tensorflow.examples.tutorials.mnist&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;input_data&lt;/span&gt;
    &lt;span class="n"&gt;mnist&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;input_data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_data_sets&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/tmp/mnist-data&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;one_hot&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;mnist&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;train&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;mnist&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;train&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;labels&lt;/span&gt;

&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&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;delayed&lt;/span&gt;

&lt;span class="n"&gt;datasets&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;get_mnist&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;20&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;  &lt;span class="c1"&gt;# 20 versions of same dataset&lt;/span&gt;
&lt;span class="n"&gt;images&lt;/span&gt; &lt;span class="o"&gt;=&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="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;d&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;datasets&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;labels&lt;/span&gt; &lt;span class="o"&gt;=&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="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;d&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;datasets&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

&lt;span class="n"&gt;images&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_delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;im&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;55000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;784&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;float32&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;im&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;labels&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_delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;la&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;55000&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;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;float32&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;la&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;labels&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

&lt;span class="n"&gt;images&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;images&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;labels&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;labels&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;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;images&lt;/span&gt;
&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;concate&lt;/span&gt;&lt;span class="o"&gt;...&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;1100000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;784&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="n"&gt;float32&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="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;55000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;784&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt;

&lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;labels&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;persist&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;labels&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;  &lt;span class="c1"&gt;# persist data in memory&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This gives us a moderately large distributed array of around a million tiny
images. If we wanted to we could inspect or clean up this data using normal
dask.array constructs:&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;im&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;images&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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reshape&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mi"&gt;28&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;28&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;im&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;gray&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;&lt;img src="/images/tf-images-one.png"
     width="20%"
     alt="mnist number 3"&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;im&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;images&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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reshape&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mi"&gt;28&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;28&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;im&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;gray&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;&lt;img src="/images/tf-images-mean.png"
     width="20%"
     alt="mnist mean"&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;im&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;var&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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reshape&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mi"&gt;28&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;28&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;im&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;gray&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;&lt;img src="/images/tf-images-var.png"
     width="20%"
     alt="mnist var"&gt;&lt;/p&gt;
&lt;p&gt;This shows off how one can use Dask collections to clean up and provide
pre-processing and feature generation on data in parallel before sending it to
TensorFlow. In our simple case we won’t actually do any of this, but it’s
useful in more real-world situations.&lt;/p&gt;
&lt;p&gt;Finally, after doing our preprocessing on the distributed array of all of our
data we’re going to collect images and labels together and batch them into
smaller chunks. Again we use some dask.array constructs and
&lt;a class="reference external" href="http://dask.pydata.org/en/latest/delayed.html"&gt;dask.delayed&lt;/a&gt; when things get
messy.&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;images&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&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;784&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;labels&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;labels&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&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;10&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

&lt;span class="n"&gt;images&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_delayed&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;flatten&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tolist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;labels&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;labels&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_delayed&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;flatten&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tolist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;batches&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;im&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;la&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;im&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;la&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="nb"&gt;zip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;labels&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;

&lt;span class="n"&gt;batches&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;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;batches&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 have a few hundred pairs of NumPy arrays in distributed memory waiting
to be sent to a TensorFlow worker.&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/2017/02/11/dask-tensorflow.md&lt;/span&gt;, line 241)&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="setting-up-tensorflow-workers-alongside-dask-workers"&gt;
&lt;h1&gt;Setting up TensorFlow workers alongside Dask workers&lt;/h1&gt;
&lt;p&gt;Dask workers are just normal Python processes. TensorFlow can launch itself
from a normal Python process. We’ve made a small function
&lt;a class="reference external" href="https://github.com/mrocklin/dask-tensorflow/blob/6fdadb6f52935788d593bdc01d441cfd9ad6a3be/dask_tensorflow/core.py"&gt;here&lt;/a&gt;
that launches TensorFlow servers alongside Dask workers using Dask’s ability to
run long-running tasks and maintain user-defined state. All together, this is
about 80 lines of code (including comments and docstrings) and allows us to
define our TensorFlow network on top of Dask as follows:&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 git+https://github.com/mrocklin/dask-tensorflow
&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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distibuted&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="c1"&gt;# we already had this above&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;dask-scheduler-address:8786&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_tensorflow&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;start_tensorflow&lt;/span&gt;
&lt;span class="n"&gt;tf_spec&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;dask_spec&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;start_tensorflow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;ps&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;worker&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;scorer&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;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;tf_spec&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;as_dict&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;ps&amp;#39;&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;192.168.100.1:2227&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
 &lt;span class="s1"&gt;&amp;#39;scorer&amp;#39;&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;192.168.100.2:2222&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
 &lt;span class="s1"&gt;&amp;#39;worker&amp;#39;&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;192.168.100.3:2223&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="s1"&gt;&amp;#39;192.168.100.4:2224&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="s1"&gt;&amp;#39;192.168.100.5:2225&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="s1"&gt;&amp;#39;192.168.100.6:2226&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;dask_spec&lt;/span&gt;
&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ps&amp;#39;&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;tcp://192.168.100.1:34471&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
 &lt;span class="s1"&gt;&amp;#39;scorer&amp;#39;&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;tcp://192.168.100.2:40623&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
 &lt;span class="s1"&gt;&amp;#39;worker&amp;#39;&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;tcp://192.168.100.3:33075&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="s1"&gt;&amp;#39;tcp://192.168.100.4:37123&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="s1"&gt;&amp;#39;tcp://192.168.100.5:32839&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="s1"&gt;&amp;#39;tcp://192.168.100.6:36822&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;This starts three groups of TensorFlow servers in the Dask worker processes.
TensorFlow will manage its own communication but co-exist right alongside Dask
in the same machines and in the same shared memory spaces (note that in the
specs above the IP addresses match but the ports differ).&lt;/p&gt;
&lt;p&gt;This also sets up a normal Python queue along which Dask can safely send
information to TensorFlow. This is how we’ll send those batches of training
data between the two services.&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/2017/02/11/dask-tensorflow.md&lt;/span&gt;, line 286)&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="define-tensorflow-model-and-distribute-roles"&gt;
&lt;h1&gt;Define TensorFlow Model and Distribute Roles&lt;/h1&gt;
&lt;p&gt;Now is the part of the blogpost where my expertise wanes. I’m just going to
copy-paste-and-modify a canned example from the TensorFlow documentation. This
is a simplistic model for this problem and it’s entirely possible that I’m
making transcription errors. But still, it should get the point across. You
can safely ignore most of this code. Dask stuff gets interesting again
towards the bottom:&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;math&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;tempfile&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;time&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;queue&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;Empty&lt;/span&gt;

&lt;span class="n"&gt;IMAGE_PIXELS&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;28&lt;/span&gt;
&lt;span class="n"&gt;hidden_units&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;100&lt;/span&gt;
&lt;span class="n"&gt;learning_rate&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mf"&gt;0.01&lt;/span&gt;
&lt;span class="n"&gt;sync_replicas&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="kc"&gt;False&lt;/span&gt;
&lt;span class="n"&gt;replicas_to_aggregate&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dask_spec&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;worker&amp;#39;&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;model&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;server&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;worker_device&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;/job:&lt;/span&gt;&lt;span class="si"&gt;%s&lt;/span&gt;&lt;span class="s2"&gt;/task:&lt;/span&gt;&lt;span class="si"&gt;%d&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&lt;/span&gt; &lt;span class="o"&gt;%&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;server&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;server_def&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;job_name&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                                         &lt;span class="n"&gt;server&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;server_def&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;task_index&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;task_index&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;server&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;server_def&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;task_index&lt;/span&gt;
    &lt;span class="n"&gt;is_chief&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;task_index&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;

    &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;device&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;train&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;replica_device_setter&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
                      &lt;span class="n"&gt;worker_device&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;worker_device&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                      &lt;span class="n"&gt;ps_device&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;/job:ps/cpu:0&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                      &lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;tf_spec&lt;/span&gt;&lt;span class="p"&gt;)):&lt;/span&gt;

        &lt;span class="n"&gt;global_step&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Variable&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;name&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;global_step&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;trainable&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="c1"&gt;# Variables of the hidden layer&lt;/span&gt;
        &lt;span class="n"&gt;hid_w&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Variable&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
            &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;truncated_normal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
                &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;IMAGE_PIXELS&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;IMAGE_PIXELS&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;hidden_units&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                &lt;span class="n"&gt;stddev&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;1.0&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="n"&gt;IMAGE_PIXELS&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="s2"&gt;&amp;quot;hid_w&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;hid_b&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Variable&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;zeros&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;hidden_units&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="s2"&gt;&amp;quot;hid_b&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="c1"&gt;# Variables of the softmax layer&lt;/span&gt;
        &lt;span class="n"&gt;sm_w&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Variable&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
            &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;truncated_normal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
                &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;hidden_units&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;stddev&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;1.0&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="n"&gt;math&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sqrt&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;hidden_units&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="s2"&gt;&amp;quot;sm_w&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;sm_b&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Variable&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;zeros&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;name&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;sm_b&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="c1"&gt;# Ops: located on the worker specified with task_index&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;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;placeholder&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;float32&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;IMAGE_PIXELS&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;IMAGE_PIXELS&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
        &lt;span class="n"&gt;y_&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;placeholder&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;float32&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="kc"&gt;None&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;hid_lin&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;nn&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;xw_plus_b&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;hid_w&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;hid_b&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;hid&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;nn&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;relu&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;hid_lin&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;nn&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;softmax&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;nn&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;xw_plus_b&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;hid&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sm_w&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sm_b&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
        &lt;span class="n"&gt;cross_entropy&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reduce_sum&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y_&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;log&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;clip_by_value&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;1e-10&lt;/span&gt;&lt;span class="p"&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;opt&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;train&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;AdamOptimizer&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;learning_rate&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;sync_replicas&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;replicas_to_aggregate&lt;/span&gt; &lt;span class="ow"&gt;is&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                &lt;span class="n"&gt;replicas_to_aggregate&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;num_workers&lt;/span&gt;
            &lt;span class="k"&gt;else&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                &lt;span class="n"&gt;replicas_to_aggregate&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;replicas_to_aggregate&lt;/span&gt;

            &lt;span class="n"&gt;opt&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;train&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;SyncReplicasOptimizer&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
                      &lt;span class="n"&gt;opt&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                      &lt;span class="n"&gt;replicas_to_aggregate&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;replicas_to_aggregate&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                      &lt;span class="n"&gt;total_num_replicas&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;num_workers&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="s2"&gt;&amp;quot;mnist_sync_replicas&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="n"&gt;train_step&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;opt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;minimize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cross_entropy&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;global_step&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;global_step&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;sync_replicas&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
            &lt;span class="n"&gt;local_init_op&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;opt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;local_step_init_op&lt;/span&gt;
            &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;is_chief&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                &lt;span class="n"&gt;local_init_op&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;opt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;chief_init_op&lt;/span&gt;

            &lt;span class="n"&gt;ready_for_local_init_op&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;opt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ready_for_local_init_op&lt;/span&gt;

            &lt;span class="c1"&gt;# Initial token and chief queue runners required by the sync_replicas mode&lt;/span&gt;
            &lt;span class="n"&gt;chief_queue_runner&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;opt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_chief_queue_runner&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
            &lt;span class="n"&gt;sync_init_op&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;opt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_init_tokens_op&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

        &lt;span class="n"&gt;init_op&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;global_variables_initializer&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
        &lt;span class="n"&gt;train_dir&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tempfile&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mkdtemp&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;sync_replicas&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
          &lt;span class="n"&gt;sv&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;train&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Supervisor&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
              &lt;span class="n"&gt;is_chief&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;is_chief&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
              &lt;span class="n"&gt;logdir&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;train_dir&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
              &lt;span class="n"&gt;init_op&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;init_op&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
              &lt;span class="n"&gt;local_init_op&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;local_init_op&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
              &lt;span class="n"&gt;ready_for_local_init_op&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;ready_for_local_init_op&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
              &lt;span class="n"&gt;recovery_wait_secs&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;global_step&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;global_step&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="n"&gt;sv&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;train&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Supervisor&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
              &lt;span class="n"&gt;is_chief&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;is_chief&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
              &lt;span class="n"&gt;logdir&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;train_dir&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
              &lt;span class="n"&gt;init_op&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;init_op&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
              &lt;span class="n"&gt;recovery_wait_secs&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;global_step&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;global_step&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="n"&gt;sess_config&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;tf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ConfigProto&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
            &lt;span class="n"&gt;allow_soft_placement&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="n"&gt;log_device_placement&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="n"&gt;device_filters&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;/job:ps&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;/job:worker/task:&lt;/span&gt;&lt;span class="si"&gt;%d&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&lt;/span&gt; &lt;span class="o"&gt;%&lt;/span&gt; &lt;span class="n"&gt;task_index&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;

        &lt;span class="c1"&gt;# The chief worker (task_index==0) session will prepare the session,&lt;/span&gt;
        &lt;span class="c1"&gt;# while the remaining workers will wait for the preparation to complete.&lt;/span&gt;
        &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;is_chief&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
          &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Worker &lt;/span&gt;&lt;span class="si"&gt;%d&lt;/span&gt;&lt;span class="s2"&gt;: Initializing session...&amp;quot;&lt;/span&gt; &lt;span class="o"&gt;%&lt;/span&gt; &lt;span class="n"&gt;task_index&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="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Worker &lt;/span&gt;&lt;span class="si"&gt;%d&lt;/span&gt;&lt;span class="s2"&gt;: Waiting for session to be initialized...&amp;quot;&lt;/span&gt; &lt;span class="o"&gt;%&lt;/span&gt;
                &lt;span class="n"&gt;task_index&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="n"&gt;sess&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sv&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;prepare_or_wait_for_session&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;server&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;target&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;sess_config&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;sync_replicas&lt;/span&gt; &lt;span class="ow"&gt;and&lt;/span&gt; &lt;span class="n"&gt;is_chief&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
          &lt;span class="c1"&gt;# Chief worker will start the chief queue runner and call the init op.&lt;/span&gt;
          &lt;span class="n"&gt;sess&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;run&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sync_init_op&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
          &lt;span class="n"&gt;sv&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;start_queue_runners&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sess&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;chief_queue_runner&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;sess&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;y_&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;train_step&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;global_step&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cross_entropy&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;ps_task&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;local_client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;c&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;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tensorflow_server&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;join&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;scoring_task&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;local_client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="c1"&gt;# Scores Channel&lt;/span&gt;
        &lt;span class="n"&gt;scores&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;channel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scores&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;maxlen&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;# Make Model&lt;/span&gt;
        &lt;span class="n"&gt;server&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;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tensorflow_server&lt;/span&gt;
        &lt;span class="n"&gt;sess&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cross_entropy&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;model&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;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tensorflow_server&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="c1"&gt;# Testing Data&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;tensorflow.examples.tutorials.mnist&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;input_data&lt;/span&gt;
        &lt;span class="n"&gt;mnist&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;input_data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_data_sets&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/tmp/mnist-data&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;one_hot&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;test_data&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="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;mnist&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;validation&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;images&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                     &lt;span class="n"&gt;y_&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;mnist&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;validation&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;labels&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;

        &lt;span class="c1"&gt;# Main Loop&lt;/span&gt;
        &lt;span class="k"&gt;while&lt;/span&gt; &lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
            &lt;span class="n"&gt;score&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sess&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;run&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cross_entropy&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;feed_dict&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;test_data&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
            &lt;span class="n"&gt;scores&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;float&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;score&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

            &lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sleep&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;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;worker_task&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;local_client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;scores&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;channel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scores&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;num_workers&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;replicas_to_aggregate&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dask_spec&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;worker&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;

        &lt;span class="n"&gt;server&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;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tensorflow_server&lt;/span&gt;
        &lt;span class="n"&gt;queue&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;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tensorflow_queue&lt;/span&gt;

        &lt;span class="c1"&gt;# Make model&lt;/span&gt;
        &lt;span class="n"&gt;sess&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;y_&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;train_step&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;global_step&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;model&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;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tensorflow_server&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="c1"&gt;# Main loop&lt;/span&gt;
        &lt;span class="k"&gt;while&lt;/span&gt; &lt;span class="ow"&gt;not&lt;/span&gt; &lt;span class="n"&gt;scores&lt;/span&gt; &lt;span class="ow"&gt;or&lt;/span&gt; &lt;span class="n"&gt;scores&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="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;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
            &lt;span class="k"&gt;try&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                &lt;span class="n"&gt;batch&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;queue&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;timeout&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;0.5&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
            &lt;span class="k"&gt;except&lt;/span&gt; &lt;span class="n"&gt;Empty&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                &lt;span class="k"&gt;continue&lt;/span&gt;

            &lt;span class="n"&gt;train_data&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="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;batch&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;y_&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;batch&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;sess&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;run&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;train_step&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;global_step&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;feed_dict&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;train_data&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The last three functions defined here, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ps_task&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scorer_task&lt;/span&gt;&lt;/code&gt; and
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;worker_task&lt;/span&gt;&lt;/code&gt; are functions that we want to run on each of our three groups of
TensorFlow server types. The parameter server task just starts a long-running
task and passively joins the TensorFlow network:&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;ps_task&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;local_client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;c&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;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tensorflow_server&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;join&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The scorer task opens up an &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/channels.html"&gt;inter-worker
channel&lt;/a&gt; of
communication named “scores”, creates the TensorFlow model, then every second
scores the current state of the model against validation data. It reports the
score on the inter-worker channel:&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;scoring_task&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;local_client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;scores&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;channel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scores&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;#  inter-worker channel&lt;/span&gt;

        &lt;span class="c1"&gt;# Make Model&lt;/span&gt;
        &lt;span class="n"&gt;sess&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cross_entropy&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;model&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;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tensorflow_server&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="o"&gt;...&lt;/span&gt;

        &lt;span class="k"&gt;while&lt;/span&gt; &lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
            &lt;span class="n"&gt;score&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sess&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;run&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cross_entropy&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;feed_dict&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;test_data&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
            &lt;span class="n"&gt;scores&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;float&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;score&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
            &lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sleep&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The worker task makes the model, listens on the Dask-TensorFlow Queue for new
training data, and continues training until the last reported score is good
enough.&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;worker_task&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;local_client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;scores&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;channel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scores&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="n"&gt;queue&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tensorflow_queue&lt;/span&gt;

        &lt;span class="c1"&gt;# Make model&lt;/span&gt;
        &lt;span class="n"&gt;sess&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;y_&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;train_step&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;global_step&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;model&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;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tensorflow_server&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="k"&gt;while&lt;/span&gt; &lt;span class="n"&gt;scores&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="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;&amp;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;batch&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;queue&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;train_data&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="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;batch&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;y_&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;batch&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;sess&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;run&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;train_step&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;global_step&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;feed_dict&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;train_data&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We launch these tasks on the Dask workers that have the corresponding
TensorFlow servers (see &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;tf_spec&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask_spec&lt;/span&gt;&lt;/code&gt; above):&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;ps_tasks&lt;/span&gt; &lt;span class="o"&gt;=&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;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ps_task&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;worker&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;worker&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;dask_spec&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ps&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]]&lt;/span&gt;

&lt;span class="n"&gt;worker_tasks&lt;/span&gt; &lt;span class="o"&gt;=&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;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;worker_task&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;addr&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;pure&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&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;addr&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;dask_spec&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;worker&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]]&lt;/span&gt;

&lt;span class="n"&gt;scorer_task&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;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;scoring_task&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;dask_spec&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scorer&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This starts long-running tasks that just sit there, waiting for external
stimulation:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/tf-long-running-task.png"
     width="70%"
     alt="long running TensorFlow tasks"&gt;&lt;/p&gt;
&lt;p&gt;Finally we construct a function to dump each of our batches of data
from our Dask.array (from the very beginning of this post) into the
Dask-TensorFlow queues on our workers. We make sure to only run these tasks
where the Dask-worker has a corresponding TensorFlow training 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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;distributed.worker_client&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_worker&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;transfer_dask_to_tensorflow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;batch&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;worker&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;get_worker&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tensorflow_queue&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="n"&gt;batch&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;dump&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;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;transfer_dask_to_tensorflow&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;batches&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
             &lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;dask_spec&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;worker&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;pure&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If we want to we can track progress in our local session by subscribing to the
same inter-worker channel:&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;scores&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;channel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scores&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 can use this to repeatedly dump data into the workers over and over again
until they converge.&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;while&lt;/span&gt; &lt;span class="n"&gt;scores&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="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;&amp;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;dump&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;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;transfer_dask_to_tensorflow&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;batches&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                 &lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;dask_spec&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;worker&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;pure&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dump&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/2017/02/11/dask-tensorflow.md&lt;/span&gt;, line 577)&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;We discussed a non-trivial way to use TensorFlow to accomplish distributed
machine learning. We used Dask to support TensorFlow in a few ways:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Trivially setup the TensorFlow network&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Prepare and clean data&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Coordinate progress and stopping criteria&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We found it convenient that Dask and TensorFlow could play nicely with each
other. Dask supported TensorFlow without getting in the way. The fact that
both libraries play nicely within Python and the greater PyData stack
(NumPy/Pandas) makes it trivial to move data between them without costly or
complex tricks.&lt;/p&gt;
&lt;p&gt;Additionally, we didn’t have to work to integrate these two systems. There is
no need for a separate collaborative effort to integrate Dask and TensorFlow at
a core level. Instead, they are designed in such a way so as to foster this
type of interaction without special attention or effort.&lt;/p&gt;
&lt;p&gt;This is also the first blogpost that I’ve written that, from a Dask
perspective, uses some more complex features like &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/task-launch.html#submit-tasks-from-worker"&gt;long running
tasks&lt;/a&gt;
or publishing state between workers with
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/channels.html"&gt;channels&lt;/a&gt;. These
more advanced features are invaluable when creating more complex/bespoke
parallel computing systems, such as are often found within companies.&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/2017/02/11/dask-tensorflow.md&lt;/span&gt;, line 605)&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-we-could-have-done-better"&gt;
&lt;h1&gt;What we could have done better&lt;/h1&gt;
&lt;p&gt;From a deep learning perspective this example is both elementary and
incomplete. It would have been nice to train on a dataset that was larger and
more complex than MNIST. Also it would be nice to see the effects of training
over time and the performance of using different numbers of workers. In
defense of this blogpost I can only claim that Dask shouldn’t affect any of
these scaling results, because TensorFlow is entirely in control at these
stages and TensorFlow already has plenty of published scaling information.&lt;/p&gt;
&lt;p&gt;Generally speaking though, this experiment was done in a weekend afternoon and
the blogpost was written in a few hours shortly afterwards. If anyone is
interested in performing and publishing about a more serious distributed deep
learning experiment with TensorFlow and Dask I would be happy to support them
on the Dask side. I think that there is plenty to learn here about best
practices.&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/2017/02/11/dask-tensorflow.md&lt;/span&gt;, line 622)&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;The following individuals contributed to the construction of this blogpost:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://stephanhoyer.com/"&gt;Stephan Hoyer&lt;/a&gt; contributed with conversations
about how TensorFlow is used in practice and with concrete experience on
deployment.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/electronwill"&gt;Will Warner&lt;/a&gt; and
&lt;a class="reference external" href="https://github.com/eriknw"&gt;Erik Welch&lt;/a&gt; both provided valuable editing and
language recommendations&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/02/11/dask-tensorflow/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-02-11T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/02/07/dask-sklearn-simple/</id>
    <title>Two Easy Ways to Use Scikit Learn and Dask</title>
    <updated>2017-02-07T00: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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2017/02/07/dask-sklearn-simple.md&lt;/span&gt;, line 13)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;This post describes two simple ways to use Dask to parallelize Scikit-Learn
operations either on a single computer or across a cluster.&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Use the Dask Joblib backend&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Use the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dklearn&lt;/span&gt;&lt;/code&gt; projects drop-in replacements for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Pipeline&lt;/span&gt;&lt;/code&gt;,
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GridSearchCV&lt;/span&gt;&lt;/code&gt;, and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomSearchCV&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;For the impatient, these look 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="c1"&gt;### Joblib&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;joblib&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;parallel_backend&lt;/span&gt;
&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;parallel_backend&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;dask.distributed&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scheduler_host&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scheduler-address:8786&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="c1"&gt;# your now-cluster-ified sklearn code here&lt;/span&gt;


&lt;span class="c1"&gt;### Dask-learn pipeline and GridSearchCV drop-in replacements&lt;/span&gt;

&lt;span class="c1"&gt;# from sklearn.grid_search import GridSearchCV&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;dklearn.grid_search&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;GridSearchCV&lt;/span&gt;
&lt;span class="c1"&gt;# from sklearn.pipeline import Pipeline&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;dklearn.pipeline&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;Pipeline&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;However, neither of these techniques are perfect. These are the easiest things
to try, but not always the best solutions. This blogpost focuses on
low-hanging fruit.&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/2017/02/07/dask-sklearn-simple.md&lt;/span&gt;, line 44)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="joblib"&gt;
&lt;h1&gt;Joblib&lt;/h1&gt;
&lt;p&gt;Scikit-Learn already parallelizes across a multi-core CPU using
&lt;a class="reference external" href="https://pythonhosted.org/joblib/"&gt;Joblib&lt;/a&gt;, a simple but powerful and mature
library that provides an extensible map operation. Here is a simple example of
using Joblib on its own without sklearn:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Sequential code&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;time&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;sleep&lt;/span&gt;
&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;slowinc&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;sleep&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="c1"&gt;# take a bit of time to simulate real work&lt;/span&gt;
    &lt;span class="k"&gt;return&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;1&lt;/span&gt;

&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;slowinc&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;10&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;  &lt;span class="c1"&gt;# this takes 10 seconds&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="mi"&gt;4&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="mi"&gt;6&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="mi"&gt;8&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="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

&lt;span class="c1"&gt;# Parallel code&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;joblib&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;Parallel&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;delayed&lt;/span&gt;
&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;Parallel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_jobs&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;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;slowinc&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;10&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;  &lt;span class="c1"&gt;# this takes 3 seconds&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="mi"&gt;4&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="mi"&gt;6&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="mi"&gt;8&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="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;Dask users will recognize the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;delayed&lt;/span&gt;&lt;/code&gt; function modifier. Dask stole
the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;delayed&lt;/span&gt;&lt;/code&gt; decorator from Joblib.&lt;/p&gt;
&lt;p&gt;Many of Scikit-learn’s parallel algorithms use Joblib internally. If we can
extend Joblib to clusters then we get some added parallelism from
joblib-enabled Scikit-learn functions immediately.&lt;/p&gt;
&lt;section id="distributed-joblib"&gt;
&lt;h2&gt;Distributed Joblib&lt;/h2&gt;
&lt;p&gt;Fortunately Joblib provides an interface for other parallel systems to step in
and act as an execution engine. We can do this with the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;parallel_backend&lt;/span&gt;&lt;/code&gt;
context manager to run with hundreds or thousands of cores in a nearby cluster:&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;distributed.joblib&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;joblib&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;parallel_backend&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;parallel_backend&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;dask.distributed&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scheduler_host&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scheduler-address:8786&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="nb"&gt;print&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Parallel&lt;/span&gt;&lt;span class="p"&gt;()(&lt;/span&gt;&lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;slowinc&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;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;100&lt;/span&gt;&lt;span class="p"&gt;))))&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The main value for Scikit-learn users here is that Scikit-learn already uses
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;joblib.Parallel&lt;/span&gt;&lt;/code&gt; within its code, so this trick works with the Scikit-learn
code that you already have.&lt;/p&gt;
&lt;p&gt;So we can use Joblib to parallelize normally on our multi-core processor:&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;estimator&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;GridSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_jobs&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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# use joblib on local multi-core processor&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;em&gt;or&lt;/em&gt; we can use Joblib together with Dask.distributed to parallelize across a
multi-node cluster:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;parallel_backend&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;dask.distributed&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scheduler_host&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scheduler-address:8786&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;estimator&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;GridSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# use joblib with Dask cluster&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;(There will be a more thorough example towards the end)&lt;/p&gt;
&lt;/section&gt;
&lt;section id="limitations"&gt;
&lt;h2&gt;Limitations&lt;/h2&gt;
&lt;p&gt;Joblib is used throughout many algorithms in Scikit-learn, but not all.
Generally any operation that accepts an &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n_jobs=&lt;/span&gt;&lt;/code&gt; parameter is a possible
choice.&lt;/p&gt;
&lt;p&gt;From Dask’s perspective Joblib’s interface isn’t ideal. For example it will
always collect intermediate results back to the main process, rather than
leaving them on the cluster until necessary. For computationally intense
operations this is fine but does add some unnecessary communication overhead.
Also Joblib doesn’t allow for operations more complex than a parallel map, so
the range of algorithms that this can parallelize is somewhat limited.&lt;/p&gt;
&lt;p&gt;Still though, given the wide use of Joblib-accelerated workflows (particularly
within Scikit-learn) this is a simple thing to try if you have a cluster nearby
with a possible large payoff.&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/2017/02/07/dask-sklearn-simple.md&lt;/span&gt;, line 125)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="dask-learn-pipeline-and-gridsearch"&gt;
&lt;h1&gt;Dask-learn Pipeline and Gridsearch&lt;/h1&gt;
&lt;p&gt;In July 2016, Jim Crist built and &lt;a class="reference external" href="http://jcrist.github.io/blog.html"&gt;wrote
about&lt;/a&gt; a small project,
&lt;a class="reference external" href="https://github.com/dask/dask-learn"&gt;dask-learn&lt;/a&gt;. This project was a
collaboration with SKLearn developers and an attempt to see which parts of
Scikit-learn were trivially and usefully parallelizable. By far the most
productive thing to come out of this work were Dask variants of Scikit-learn’s
Pipeline, GridsearchCV, and RandomSearchCV objects that better handle nested
parallelism. Jim observed significant speedups over SKLearn code by using
these drop-in replacements.&lt;/p&gt;
&lt;p&gt;So if you replace the following imports you may get both better single-threaded
performance &lt;em&gt;and&lt;/em&gt; the ability to scale out to a cluster:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# from sklearn.grid_search import GridSearchCV&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;dklearn.grid_search&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;GridSearchCV&lt;/span&gt;
&lt;span class="c1"&gt;# from sklearn.pipeline import Pipeline&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;dklearn.pipeline&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;Pipeline&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here is a simple example from &lt;a class="reference external" href="http://jcrist.github.io/dask-sklearn-part-1.html"&gt;Jim’s more in-depth blogpost&lt;/a&gt;:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;sklearn.datasets&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;make_classification&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;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;make_classification&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_samples&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;n_features&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;500&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                           &lt;span class="n"&gt;n_classes&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="n"&gt;n_redundant&lt;/span&gt;&lt;span class="o"&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;random_state&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;42&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;sklearn&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;linear_model&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;decomposition&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;sklearn.pipeline&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;Pipeline&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;dklearn.pipeline&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;Pipeline&lt;/span&gt;

&lt;span class="n"&gt;logistic&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;linear_model&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;LogisticRegression&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;pca&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;decomposition&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;PCA&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;pipe&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Pipeline&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;steps&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;pca&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;pca&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;logistic&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;logistic&lt;/span&gt;&lt;span class="p"&gt;)])&lt;/span&gt;


&lt;span class="c1"&gt;#Parameters of pipelines can be set using ‘__’ separated parameter names:&lt;/span&gt;
&lt;span class="n"&gt;grid&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="n"&gt;pca__n_components&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;50&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;100&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;150&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;logistic__C&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mf"&gt;1e-4&lt;/span&gt;&lt;span class="p"&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="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;1e4&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
            &lt;span class="n"&gt;logistic__penalty&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;l1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;l2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;

&lt;span class="c1"&gt;# from sklearn.grid_search import GridSearchCV&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;dklearn.grid_search&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;GridSearchCV&lt;/span&gt;

&lt;span class="n"&gt;estimator&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;GridSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pipe&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;grid&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;estimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;SKLearn performs this computation in around 40 seconds while the dask-learn
drop-in replacements take around 10 seconds. Also, if you add the following
lines to connect to a &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/quickstart.html"&gt;running
cluster&lt;/a&gt; the whole
thing scales out:&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="n"&gt;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;scheduler-address:8786&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;Here is a live &lt;a class="reference external" href="http://bokeh.pydata.org/en/latest/"&gt;Bokeh&lt;/a&gt; plot of the
computation on a tiny eight process “cluster” running on my own laptop. I’m
using processes here to highlight the costs of communication between processes
(red). It’s actually about 30% faster to run this computation within the same
single process.&lt;/p&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/a2a42d71d0dd085753277821e24925a4/raw/e29b24bc656ea619eedfaba9ef176d5f3c19a040/dask-learn-task-stream.html"
        width="800" height="400"&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/2017/02/07/dask-sklearn-simple.md&lt;/span&gt;, line 201)&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;This post showed a couple of simple mechanisms for scikit-learn users to
accelerate their existing workflows with Dask. These aren’t particularly
sophisticated, nor are they performance-optimal, but they are easy to
understand and easy to try out. In a future blogpost I plan to cover more
complex ways in which Dask can accelerate sophisticated machine learning
workflows.&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/2017/02/07/dask-sklearn-simple.md&lt;/span&gt;, line 210)&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-we-could-have-done-better"&gt;
&lt;h1&gt;What we could have done better&lt;/h1&gt;
&lt;p&gt;As always, I include a brief section on what went wrong or what we could have
done better with more time.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;See the bottom of &lt;a class="reference external" href="http://jcrist.github.io/dask-sklearn-part-1.html"&gt;Jim’s post&lt;/a&gt;
for a more thorough explanation of “what we could have done better” for
dask-learn’s pipeline and gridsearch&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joblib + Dask.distributed interaction is convenient, but leaves some
performance on the table. It’s not clear how Dask can help the sklearn
codebase without being too invasive.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It would have been nice to spin up an actual cluster on parallel hardware
for this post. I wrote this quickly (in a few hours) so decided to skip
this. If anyone wants to write a follow-on experiment I would be happy
to publish it.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/02/07/dask-sklearn-simple/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-02-07T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/01/30/dask-dev-6/</id>
    <title>Dask Development Log</title>
    <updated>2017-01-30T00: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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;To increase transparency I’m blogging weekly about the work done on Dask and
related projects during the previous week. This log covers work done between
2017-01-17 and 2016-01-30. Nothing here is ready for production. This
blogpost is written in haste, so refined polish should not be expected.&lt;/p&gt;
&lt;p&gt;Themes of the last couple of weeks:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Micro-release of distributed scheduler&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;as_completed for asynchronous algorithms&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Testing ZeroMQ and communication refactor&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Persist, and Dask-GLM&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/2017/01/30/dask-dev-6.md&lt;/span&gt;, line 25)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="stability-enhancements-and-micro-release"&gt;

&lt;p&gt;We’ve released dask.distributed version 1.15.2, which includes some important
performance improvements for communicating multi-dimensional arrays, cleaner
scheduler shutdown of workers for adaptive deployments, an improved
as_completed iterator that can accept new futures in flight, and a few other
smaller features.&lt;/p&gt;
&lt;p&gt;The feature here that excites me the most is improved communication of
multi-dimensional arrays across the network. In a
&lt;a class="reference internal" href="../2017/01/17/dask-images/"&gt;&lt;span class="doc std std-doc"&gt;recent blogpost about image processing on a cluster&lt;/span&gt;&lt;/a&gt;
we noticed that communication bandwidth was far lower than expected. This led
us to uncover a flaw in our compression heuristics. Dask doesn’t compress all
data, instead it takes a few 10kB samples of the data, compresses them, and if
that goes well, decides to compress the entire thing. Unfortunately, due to
our mishandling of memoryviews we ended up taking &lt;em&gt;much&lt;/em&gt; larger samples than
1kB when dealing with multi-dimensional arrays.&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/2017/01/30/dask-dev-6.md&lt;/span&gt;, line 43)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="xarray-release"&gt;
&lt;h1&gt;XArray release&lt;/h1&gt;
&lt;p&gt;This improvement is particularly timely because a new release of
&lt;a class="reference external" href="http://xarray.pydata.org/en/stable/"&gt;XArray&lt;/a&gt; (a project that wraps around
Dask.array for large labeled arrays) is now available with better data
ingestion support for NetCDF data on distributed clusters. This opens up
distributed array computing to Dask’s first (and possibly largest) scientific
user community, atmospheric scientists.&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/2017/01/30/dask-dev-6.md&lt;/span&gt;, line 52)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="as-completed-accepts-futures"&gt;
&lt;h1&gt;as_completed accepts futures.&lt;/h1&gt;
&lt;p&gt;In addition to arrays, dataframes, and the delayed decorator, Dask.distributed
also implements the &lt;a class="reference external" href="http://xarray.pydata.org/en/stable/"&gt;concurrent.futures&lt;/a&gt;
interface from the standard library (except that Dask’s version parallelizes
across a cluster and has a few other benefits). Part of this interface is the
&lt;a class="reference external" href="http://xarray.pydata.org/en/stable/"&gt;as_completed&lt;/a&gt; function, which takes in a
list of futures and yields those futures in the order in which they finish.
This enables the construction of fairly responsive asynchronous computations.
As soon as some work finishes you can look at the result and submit more work
based on the current state.&lt;/p&gt;
&lt;p&gt;That has been around in Dask for some time.&lt;/p&gt;
&lt;p&gt;What’s &lt;em&gt;new&lt;/em&gt; is that you can now push more futures into as_completed&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;futures&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;my_function&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sequence&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;ac&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;as_completed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&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;future&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;ac&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;future&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# future is already finished, so this is fast&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;condition&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;new_future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;function&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;args&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;ac&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;new_future&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;    &lt;span class="c1"&gt;# &amp;lt;&amp;lt;---- This is the new ability&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;as_completed&lt;/span&gt;&lt;/code&gt; iterator can keep going for quite a while with a set of
futures always in flight. This relatively simple change allows for the easy
expression of a broad set of asynchronous 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/2017/01/30/dask-dev-6.md&lt;/span&gt;, line 83)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="zeromq-and-communication-refactor"&gt;
&lt;h1&gt;ZeroMQ and Communication Refactor&lt;/h1&gt;
&lt;p&gt;As part of a large effort, Antoine Pitrou has been refactoring Dask’s
communication system. One sub-goal of this refactor was to allow us to explore
other transport mechanisms than Tornado IOStreams in a pluggable way.&lt;/p&gt;
&lt;p&gt;One such alternative is ZeroMQ sockets. We’ve gotten both incredibly positive
and incredibly negative praise/warnings about using ZeroMQ. It’s not a great
fit because Dask mostly just does point-to-point communication, so we don’t
benefit from all of ZeroMQ’s patterns, which now become more of a hindrance
than a benefit. However, we were interested in the performance impact of
managing all of our network communication in a completely separately managed
C++ thread unaffected by GIL issues.&lt;/p&gt;
&lt;p&gt;Whether or you hate or love ZeroMQ you can now pick and choose. Antoine’s
branch allows for easy swapping between transport mechanisms and opens the
possibility for more in the future like intra-process communication with
Queues, MPI, etc.. This doesn’t affect most users, but some Dask deployments
are on supercomputers with exotic networks capable of very fast speeds. The
possibility that we might tap into Infiniband someday and have the ability to
manage data locally without copies (Tornado does not achieve this) is very
attractive to some user communities.&lt;/p&gt;
&lt;p&gt;After very preliminary benchmarking we’ve found that ZeroMQ offers a small
speedup, but results in a lack of stability in the cluster under complex
workloads (likely our fault, not ZeroMQs, but still). ZeroMQ support is
strictly experimental and will likely stay that way for a long time. Readers
should not get excited about this.&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/2017/01/30/dask-dev-6.md&lt;/span&gt;, line 112)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="perist-and-dask-glm"&gt;
&lt;h1&gt;Perist and Dask-GLM&lt;/h1&gt;
&lt;p&gt;In collaboration with researchers at Capital One we’ve been working on a &lt;a class="reference external" href="http://github.com/dask/dask-glm"&gt;set
of parallel solvers for first and second order
methods&lt;/a&gt;, such as are commonly used in a broad
class of statistical and machine learning algorithms.&lt;/p&gt;
&lt;p&gt;One challenge in this process has been building algorithms that are
simultaneously optimal for both the single machine and distributed schedulers.
The distributeed scheduler requires that we think about where data is, on the
client or on the cluster, where for the single machine scheudler this is less
of a problem. The distrbuted scheduler appropriately has a new verb,
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;persist&lt;/span&gt;&lt;/code&gt;, which keeps data as a Dask collection, but triggers all of the
internal computation&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt; &lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;numpy&lt;/span&gt; &lt;span class="n"&gt;array&lt;/span&gt;
&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt; &lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt; &lt;span class="n"&gt;array&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We have now mirrored this verb to the single machine scheduler in &lt;a class="reference external" href="https://github.com/dask/dask/pull/1927"&gt;dask/dask
#1927&lt;/a&gt; and we get very nice performance
on dask-glm’s algorithms in both cases now.&lt;/p&gt;
&lt;p&gt;Working with the developers at Capital One has been very rewarding. I would
like to find more machine learning groups that fit the following criteria:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Are focused on performance&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Need parallel computing&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Are committed to built good open source software&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Are sufficiently expert in their field to understand correct algorithms&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;If you know such a person or such a group, either in industry or just a grad
student in university, please encourage them to raise an issue at
http://github.com/dask/dask/issues/new . I will likely write a larger blogpost
on this topic in the near future.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/01/30/dask-dev-6/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-01-30T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/01/24/dask-custom/</id>
    <title>Custom Parallel Algorithms on a Cluster with Dask</title>
    <updated>2017-01-24T00: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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2017/01/24/dask-custom.md&lt;/span&gt;, line 13)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;This post describes Dask as a computational task scheduler that fits somewhere
on a spectrum between big data computing frameworks like Hadoop/Spark and task
schedulers like Airflow/Celery/Luigi. We see how, by combining elements from
both of these types of systems Dask is able to handle complex data science
problems particularly well.&lt;/p&gt;
&lt;p&gt;This post is in contrast to two recent posts on structured parallel
collections:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://mrocklin.github.com/blog/work/2017/01/12/dask-dataframes"&gt;Distributed DataFrames&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://mrocklin.github.com/blog/work/2017/01/17/dask-images"&gt;Distributed Arrays&lt;/a&gt;&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/2017/01/24/dask-custom.md&lt;/span&gt;, line 27)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="big-data-collections"&gt;
&lt;h1&gt;Big Data Collections&lt;/h1&gt;
&lt;p&gt;Most distributed computing systems like Hadoop or Spark or SQL databases
implement a small but powerful set of parallel operations like map, reduce,
groupby, and join. As long as you write your programs using only those
operations then the platforms understand your program and serve you well. Most
of the time this is great because most big data problems are pretty simple.&lt;/p&gt;
&lt;p&gt;However, as we explore new complex algorithms or messier data science problems,
these large parallel operations start to become insufficiently flexible. For
example, consider the following data loading and cleaning problem:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Load data from 100 different files (this is a simple &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map&lt;/span&gt;&lt;/code&gt; operation)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Also load a reference dataset from a SQL database (not parallel at all, but
could run alongside the map above)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Normalize each of the 100 datasets against the reference dataset (sort of
like a map, but with another input)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Consider a sliding window of every three normalized datasets (Might be able
to hack this with a very clever join? Not sure.)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Of all of the 98 outputs of the last stage, consider all pairs. (Join or
cartesian product) However, because we don’t want to compute all ~10000
possibilities, let’s just evaluate a random sample of these pairs&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Find the best of all of these possibilities (reduction)&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In sequential for-loopy code this might look 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="n"&gt;filenames&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;mydata-&lt;/span&gt;&lt;span class="si"&gt;%d&lt;/span&gt;&lt;span class="s1"&gt;.dat&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="nb"&gt;range&lt;/span&gt;&lt;span class="p"&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;data&lt;/span&gt; &lt;span class="o"&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="n"&gt;fn&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="n"&gt;reference&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;load_from_sql&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;sql://mytable&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;processed&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;process&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;reference&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;d&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

&lt;span class="n"&gt;rolled&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[]&lt;/span&gt;
&lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;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="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;processed&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;a&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;processed&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="n"&gt;b&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;processed&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&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;processed&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
    &lt;span class="n"&gt;r&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;roll&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="n"&gt;c&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;rolled&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;r&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;compared&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[]&lt;/span&gt;
&lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;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;200&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;choice&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;rolled&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;choice&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;rolled&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;compare&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="n"&gt;compared&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;best&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;reduction&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;compared&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This code is clearly parallelizeable, but it’s not clear how to write it down
as a MapReduce program, a Spark computation, or a SQL query. These tools
generally fail when asked to express complex or messy problems. We can still
use Hadoop/Spark to solve this problem, but we are often forced to change and
simplify our objectives a bit. (This problem is not particularly complex, and I
suspect that there are clever ways to do it, but it’s not trivial and often
inefficient.)&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/2017/01/24/dask-custom.md&lt;/span&gt;, line 86)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="task-schedulers"&gt;
&lt;h1&gt;Task Schedulers&lt;/h1&gt;
&lt;p&gt;So instead people use task schedulers like Celery, Luigi, or Airflow. These
systems track hundreds of &lt;em&gt;tasks&lt;/em&gt;, each of which is just a normal Python
function that runs on some normal Python data. The task scheduler tracks
dependencies between tasks and so runs as many as it can at once if they don’t
depend on each other.&lt;/p&gt;
&lt;p&gt;This is a far more granular approach than the Big-Bulk-Collection approach of
MapReduce and Spark. However systems like Celery, Luigi, and Airflow are also
generally less efficient. This is both because they know less about their computations (map is much easier to schedule than an arbitrary graph) and because they just don’t have machinery for inter-worker communication, efficient serialization of custom datatypes, etc..&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/2017/01/24/dask-custom.md&lt;/span&gt;, line 98)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="dask-mixes-task-scheduling-with-efficient-computation"&gt;
&lt;h1&gt;Dask Mixes Task Scheduling with Efficient Computation&lt;/h1&gt;
&lt;p&gt;Dask is both a big data system like Hadoop/Spark that is aware of resilience,
inter-worker communication, live state, etc. and also a general task scheduler
like Celery, Luigi, or Airflow, capable of arbitrary task execution.&lt;/p&gt;
&lt;p&gt;Many Dask users use something like Dask dataframe, which generates these graphs
automatically, and so never really observe the task scheduler aspect of Dask
This is, however, the core of what distinguishes Dask from other systems like
Hadoop and Spark. Dask is incredibly &lt;em&gt;flexible&lt;/em&gt; in the kinds of algorithms it
can run. This is because, at its core, it can run &lt;em&gt;any&lt;/em&gt; graph of tasks and not
just map, reduce, groupby, join, etc.. Users can do this natively, without
having to subclass anything or extend Dask to get this extra power.&lt;/p&gt;
&lt;p&gt;There are significant performance advantages to this. For example:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Dask.dataframe can easily represent nearest neighbor computations for
fast time-series algorithms&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask.array can implement complex linear algebra solvers or SVD algorithms
from the latest research&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Complex Machine Learning algorithms are often easier to implement in Dask,
allowing it to be more efficient through smarter algorithms, as well as
through scalable computing.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Complex hierarchies from bespoke data storage solutions can be explicitly
modeled and loaded in to other Dask systems&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This doesn’t come for free. Dask’s scheduler has to be very intelligent to
smoothly schedule arbitrary graphs while still optimizing for data locality,
worker failure, minimal communication, load balancing, scarce resources like
GPUs and more. It’s a tough job.&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/2017/01/24/dask-custom.md&lt;/span&gt;, line 129)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="dask-delayed"&gt;
&lt;h1&gt;Dask.delayed&lt;/h1&gt;
&lt;p&gt;So let’s go ahead and run the data ingestion job described with Dask.&lt;/p&gt;
&lt;p&gt;We craft some fake functions to simulate actual work:&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;random&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;time&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;sleep&lt;/span&gt;

&lt;span class="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;address&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;sleep&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="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="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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;load_from_sql&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;address&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;sleep&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="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="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="mf"&gt;0.5&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;process&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="n"&gt;reference&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;sleep&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="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="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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;roll&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="n"&gt;c&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;sleep&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="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="o"&gt;/&lt;/span&gt; &lt;span class="mi"&gt;5&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;compare&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="n"&gt;sleep&lt;/span&gt;&lt;span class="p"&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="o"&gt;/&lt;/span&gt; &lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;reduction&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;seq&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;sleep&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="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="o"&gt;/&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We annotate these functions with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.delayed&lt;/span&gt;&lt;/code&gt;, which changes a function so
that instead of running immediately it captures its inputs and puts everything
into a task graph for future execution.&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;delayed&lt;/span&gt;

&lt;span class="n"&gt;load&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;delayed&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="n"&gt;load_from_sql&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;load_from_sql&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;process&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;process&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;roll&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;roll&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;compare&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;compare&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;reduction&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;reduction&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 just call our normal Python for-loopy code from before. However now
rather than run immediately our functions capture a computational graph that
can be run elsewhere.&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;filenames&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;mydata-&lt;/span&gt;&lt;span class="si"&gt;%d&lt;/span&gt;&lt;span class="s1"&gt;.dat&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="nb"&gt;range&lt;/span&gt;&lt;span class="p"&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;data&lt;/span&gt; &lt;span class="o"&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="n"&gt;fn&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="n"&gt;reference&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;load_from_sql&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;sql://mytable&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;processed&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;process&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;reference&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;d&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

&lt;span class="n"&gt;rolled&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[]&lt;/span&gt;
&lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;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="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;processed&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;a&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;processed&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="n"&gt;b&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;processed&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&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;processed&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
    &lt;span class="n"&gt;r&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;roll&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="n"&gt;c&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;rolled&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;r&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;compared&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[]&lt;/span&gt;
&lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;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;200&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;choice&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;rolled&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;choice&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;rolled&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;compare&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="n"&gt;compared&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;best&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;reduction&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;compared&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here is an image of that graph for a smaller input of only 10 files and 20
random pairs&lt;/p&gt;
&lt;a href="/images/custom-etl.png"&gt;
    &lt;img src="/images/custom-etl.png"
         alt="Custom ETL Dask Graph"
         width="80%"&gt;&lt;/a&gt;
&lt;p&gt;We can connect to a small cluster with 20 cores&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scheduler-address:8786&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 compute the result and see the trace of the computation running in real
time.&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;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;best&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/task-stream-custom-etl.gif"&gt;
    &lt;img src="/images/task-stream-custom-etl.gif"
         alt="Custom ETL Task Stream"
         width="80%"&gt;&lt;/a&gt;
&lt;p&gt;The completed &lt;a class="reference external" href="https://bokeh.pydata.org"&gt;Bokeh&lt;/a&gt; image below is interactive.
You can pan and zoom by selecting the tools in the upper right. You can see
every task, which worker it ran on and how long it took by hovering over the
rectangles.&lt;/p&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/52e1c411878fcdd64e04574877fe265e/raw/98d9f38c51b250523e9c584779e74156ab14a4fe/task-stream-custom-etl.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;We see that we use all 20 cores well. Intermediate results are transferred
between workers as necessary (these are the red rectangles). We can scale this
up as necessary. Dask scales to thousands of 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/2017/01/24/dask-custom.md&lt;/span&gt;, line 241)&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’s ability to write down arbitrary computational graphs
Celery/Luigi/Airflow-style and yet run them with the scalability promises of
Hadoop/Spark allows for a pleasant freedom to write comfortably and yet still
compute scalably. This ability opens up new possibilities both to support more
sophisticated algorithms and also to handle messy situations that arise in the
real world (enterprise data systems are sometimes messy) while still remaining
within the bounds of “normal and supported” Dask operation.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/01/24/dask-custom/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-01-24T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/01/18/dask-dev-5/</id>
    <title>Dask Development Log</title>
    <updated>2017-01-18T00: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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;To increase transparency I’m blogging weekly about the work done on Dask and
related projects during the previous week. This log covers work done between
2017-01-01 and 2016-01-17. Nothing here is ready for production. This
blogpost is written in haste, so refined polish should not be expected.&lt;/p&gt;
&lt;p&gt;Themes of the last couple of weeks:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Stability enhancements for the distributed scheduler and micro-release&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;NASA Grant writing&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask-EC2 script&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dataframe categorical flexibility (work in progress)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Communication refactor (work in progress)&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/2017/01/18/dask-dev-5.md&lt;/span&gt;, line 26)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="stability-enhancements-and-micro-release"&gt;

&lt;p&gt;We’ve released dask.distributed version 1.15.1, which includes important
bugfixes after the recent 1.15.0 release. There were a number of small issues
that coordinated to remove tasks erroneously. This was generally OK
because the Dask scheduler was able to heal the missing pieces (using the
same machinery that makes Dask resilience) and so we didn’t notice the flaw
until the system was deployed in some of the more serious Dask deployments in
the wild.
PR &lt;a class="reference external" href="https://github.com/dask/distributed/pull/804"&gt;dask/distributed #804&lt;/a&gt;
contains a full writeup in case anyone is interested. The writeup ends with
the following line:&lt;/p&gt;
&lt;p&gt;&lt;em&gt;This was a nice exercise in how coupling mostly-working components can easily
yield a faulty system.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;This also adds other fixes, like a compatibility issue with the new Bokeh
0.12.4 release and others.&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/2017/01/18/dask-dev-5.md&lt;/span&gt;, line 45)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="nasa-grant-writing"&gt;
&lt;h1&gt;NASA Grant Writing&lt;/h1&gt;
&lt;p&gt;I’ve been writing a proposal to NASA to help fund distributed Dask+XArray work
for atmospheric and oceanographic science at the 100TB scale. Many thanks to
our scientific collaborators who are offering support here.&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/2017/01/18/dask-dev-5.md&lt;/span&gt;, line 51)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="dask-ec2-startup"&gt;
&lt;h1&gt;Dask-EC2 startup&lt;/h1&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://github.com/dask/dask-ec2"&gt;Dask-EC2 project&lt;/a&gt; deploys Anaconda, a
Dask cluster, and Jupyter notebooks on Amazon’s Elastic Compute Cloud (EC2)
with a small command line interface:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;ec2&lt;/span&gt; &lt;span class="o"&gt;--&lt;/span&gt;&lt;span class="n"&gt;upgrade&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;ec2&lt;/span&gt; &lt;span class="n"&gt;up&lt;/span&gt; &lt;span class="o"&gt;--&lt;/span&gt;&lt;span class="n"&gt;keyname&lt;/span&gt; &lt;span class="n"&gt;KEYNAME&lt;/span&gt; \
            &lt;span class="o"&gt;--&lt;/span&gt;&lt;span class="n"&gt;keypair&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;to&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;ssh&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;key&lt;/span&gt; \
            &lt;span class="o"&gt;--&lt;/span&gt;&lt;span class="nb"&gt;type&lt;/span&gt; &lt;span class="n"&gt;m4&lt;/span&gt;&lt;span class="mf"&gt;.2&lt;/span&gt;&lt;span class="n"&gt;xlarge&lt;/span&gt;
            &lt;span class="o"&gt;--&lt;/span&gt;&lt;span class="n"&gt;count&lt;/span&gt; &lt;span class="mi"&gt;8&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This project can be either &lt;em&gt;very useful&lt;/em&gt; for people just getting started
and for Dask developers when we run benchmarks, or it can be &lt;em&gt;horribly broken&lt;/em&gt;
if AWS or Dask interfaces change and we don’t keep this project maintained.
Thanks to a great effort from &lt;a class="reference external" href="http://github.com/quasiben/"&gt;Ben Zaitlen&lt;/a&gt;
`dask-ec2 is again in the &lt;em&gt;very useful&lt;/em&gt; state, where I’m hoping it will stay
for some time.&lt;/p&gt;
&lt;p&gt;If you’ve always wanted to try Dask on a real cluster and if you already have
AWS credentials then this is probably the easiest way.&lt;/p&gt;
&lt;p&gt;This already seems to be paying dividends. There have been a few unrelated
pull requests from new developers this week.&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/2017/01/18/dask-dev-5.md&lt;/span&gt;, line 78)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="dataframe-categorical-flexibility"&gt;
&lt;h1&gt;Dataframe Categorical Flexibility&lt;/h1&gt;
&lt;p&gt;Categoricals can &lt;a class="reference internal" href="#/2015/06/18/Categoricals"&gt;&lt;span class="xref myst"&gt;significantly improve
performance&lt;/span&gt;&lt;/a&gt; on
text-based data. Currently Dask’s dataframes support categoricals, but they
expect to know all of the categories up-front. This is easy if this set is
small, like the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;[&amp;quot;Healthy&amp;quot;,&lt;/span&gt; &lt;span class="pre"&gt;&amp;quot;Sick&amp;quot;]&lt;/span&gt;&lt;/code&gt; categories that might arise in medical
research, but requires a full dataset read if the categories are not known
ahead of time, like the names of all of the patients.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="http://jcrist.github.io/"&gt;Jim Crist&lt;/a&gt; is changing this so that Dask can
operates on categorical columns with unknown categories at &lt;a class="reference external" href="https://github.com/dask/dask/pull/1877"&gt;dask/dask
#1877&lt;/a&gt;. The constituent pandas
dataframes all have possibly different categories that are merged as necessary.
This distinction may seem small, but it limits performance in a surprising
number of real-world use cases.&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/2017/01/18/dask-dev-5.md&lt;/span&gt;, line 95)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="communication-refactor"&gt;
&lt;h1&gt;Communication Refactor&lt;/h1&gt;
&lt;p&gt;Since the recent worker refactor and optimizations it has become clear that
inter-worker communication has become a dominant bottleneck in some intensive
applications. &lt;a class="reference external" href="http://github.com/pitrou"&gt;Antoine Pitrou&lt;/a&gt; is currently
&lt;a class="reference external" href="https://github.com/dask/distributed/pull/810"&gt;refactoring Dask’s network communication layer&lt;/a&gt;,
making room for more communication options in the future. This is an ambitious
project. I for one am very happy to have someone like Antoine looking into
this.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/01/18/dask-dev-5/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-01-18T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/01/17/dask-images/</id>
    <title>Distributed NumPy on a Cluster with Dask Arrays</title>
    <updated>2017-01-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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;This page includes embedded large profiles. It may look better on the &lt;a class="reference internal" href="../2017/01/17/dask-images/"&gt;&lt;span class="doc std std-doc"&gt;actual
site&lt;/span&gt;&lt;/a&gt; rather than
through syndicated pages like planet.python and it may take a while to load on
non-broadband connections (total size is around 20MB)&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/2017/01/17/dask-images.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="summary"&gt;

&lt;p&gt;We analyze a stack of images in parallel with NumPy arrays distributed across a
cluster of machines on Amazon’s EC2 with Dask array. This is a model
application shared among many image analysis groups ranging from satellite
imagery to bio-medical applications. We go through a series of common
operations:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Inspect a sample of images locally with Scikit Image&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Construct a distributed Dask.array around all of our images&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Process and re-center images with Numba&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Transpose data to get a time-series for every pixel, compute FFTs&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This last step is quite fun. Even if you skim through the rest of this article
I recommend checking out the last section.&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/2017/01/17/dask-images.md&lt;/span&gt;, line 34)&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="inspect-dataset"&gt;
&lt;h1&gt;Inspect Dataset&lt;/h1&gt;
&lt;p&gt;I asked a colleague at the US National Institutes for Health (NIH) for a
biggish imaging dataset. He came back with the following message:&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Electron microscopy may be generating the biggest ndarray datasets in the field - terabytes regularly. Neuroscience needs EM to see connections between neurons, because the critical features of neural synapses (connections) are below the diffraction limit of light microscopes. This type of research has been called “connectomics”. Many groups are looking at machine vision approaches to follow small neuron parts from one slice to the next.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;This data is from drosophila: &lt;a class="reference external" href="http://emdata.janelia.org/"&gt;http://emdata.janelia.org/&lt;/a&gt;. Here is an example 2d slice of the data &lt;a class="reference external" href="http://emdata.janelia.org/api/node/bf1/grayscale/raw/xy/2000_2000/1800_2300_5000"&gt;http://emdata.janelia.org/api/node/bf1/grayscale/raw/xy/2000_2000/1800_2300_5000&lt;/a&gt;.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;skimage.io&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;matplotlib.pyplot&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;plt&lt;/span&gt;

&lt;span class="n"&gt;sample&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;http://emdata.janelia.org/api/node/bf1/grayscale/raw/xy/2000_2000/1800_2300_5000&amp;#39;&lt;/span&gt;
&lt;span class="n"&gt;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sample&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-imaging-sample.png"&gt;
  &lt;img src="/images/dask-imaging-sample.png"
         alt="Sample electron microscopy image from stack"
                width="60%"&gt;&lt;/a&gt;
&lt;p&gt;The last number in the URL is an index into a large stack of about 10000 images. We can change that number to get different slices through our 3D 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="n"&gt;samples&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;http://emdata.janelia.org/api/node/bf1/grayscale/raw/xy/2000_2000/1800_2300_&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="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="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;2000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3000&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;5000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;6000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;7000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;8000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;9000&lt;/span&gt;&lt;span class="p"&gt;]]&lt;/span&gt;

&lt;span class="n"&gt;fig&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;axarr&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;subplots&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;9&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sharex&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sharey&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;figsize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;24&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;2.5&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;sample&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;samples&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;axarr&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sample&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;gray&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/dask-imaging-row.png"&gt;
  &lt;img src="/images/dask-imaging-row.png"
         alt="Sample electron microscopy images over time"
                width="100%"&gt;&lt;/a&gt;
&lt;p&gt;We see that our field of interest wanders across the frame over time and drops
off in the beginning and at the end.&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/2017/01/17/dask-images.md&lt;/span&gt;, line 75)&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="create-a-distributed-array"&gt;
&lt;h1&gt;Create a Distributed Array&lt;/h1&gt;
&lt;p&gt;Even though our data is spread across many files, we still want to think of it
as a single logical 3D array. We know how to get any particular 2D slice of
that array using Scikit-image. Now we’re going to use Dask.array to stitch
all of those Scikit-image calls into a single distributed 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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&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;delayed&lt;/span&gt;

&lt;span class="n"&gt;imread&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;pure&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Lazy version of imread&lt;/span&gt;

&lt;span class="n"&gt;urls&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;http://emdata.janelia.org/api/node/bf1/grayscale/raw/xy/2000_2000/1800_2300_&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="nb"&gt;range&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="c1"&gt;# A list of our URLs&lt;/span&gt;

&lt;span class="n"&gt;lazy_values&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;imread&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;url&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;url&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;urls&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;     &lt;span class="c1"&gt;# Lazily evaluate imread on each url&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_delayed&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;lazy_value&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;           &lt;span class="c1"&gt;# Construct a small Dask array&lt;/span&gt;
                          &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;sample&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;   &lt;span class="c1"&gt;# for every lazy value&lt;/span&gt;
                          &lt;span class="n"&gt;shape&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;sample&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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;lazy_value&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;lazy_values&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

&lt;span class="n"&gt;stack&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;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="c1"&gt;# Stack all small Dask arrays into one&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="n"&gt;stack&lt;/span&gt;
&lt;span class="go"&gt;dask.array&amp;lt;shape=(10000, 2000, 2000), dtype=uint8, chunksize=(1, 2000, 2000)&amp;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="n"&gt;stack&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;stack&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2000&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;     &lt;span class="c1"&gt;# combine chunks to reduce overhead&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;stack&lt;/span&gt;
&lt;span class="go"&gt;dask.array&amp;lt;shape=(10000, 2000, 2000), dtype=uint8, chunksize=(20, 2000, 2000)&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So here we’ve constructed a lazy Dask.array from 10 000 delayed calls to
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;skimage.io.imread&lt;/span&gt;&lt;/code&gt;. We haven’t done any actual work yet, we’ve just
constructed a parallel array that knows how to get any particular slice of data
by downloading the right image if necessary. This gives us a full NumPy-like
abstraction on top of all of these remote images. For example we can now
download a particular image just by slicing our Dask 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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;stack&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="p"&gt;:,&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="go"&gt;array([[0, 0, 0, ..., 0, 0, 0],&lt;/span&gt;
&lt;span class="go"&gt;       [0, 0, 0, ..., 0, 0, 0],&lt;/span&gt;
&lt;span class="go"&gt;       [0, 0, 0, ..., 0, 0, 0],&lt;/span&gt;
&lt;span class="go"&gt;       ...,&lt;/span&gt;
&lt;span class="go"&gt;       [0, 0, 0, ..., 0, 0, 0],&lt;/span&gt;
&lt;span class="go"&gt;       [0, 0, 0, ..., 0, 0, 0],&lt;/span&gt;
&lt;span class="go"&gt;       [0, 0, 0, ..., 0, 0, 0]], dtype=uint8)&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;stack&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="p"&gt;:,&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;11.49902425&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;However we probably don’t want to operate too much further without connecting
to a cluster. That way we can just download all of the images once into
distributed RAM and start doing some real computations. I happen to have ten
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;m4.2xlarges&lt;/span&gt;&lt;/code&gt; on Amazon’s EC2 (8 cores, 30GB RAM each) running Dask workers.
So we’ll connect to those.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;progress&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;schdeduler-address:8786&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;client&lt;/span&gt;
&lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;scheduler-address:8786&amp;quot;&lt;/span&gt; &lt;span class="n"&gt;processes&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt; &lt;span class="n"&gt;cores&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;80&lt;/span&gt;&lt;span class="o"&gt;&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;I’ve replaced the actual address of my scheduler (something like
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;54.183.180.153&lt;/span&gt;&lt;/code&gt; with `scheduler-address. Let’s go ahead and bring in all of
our images, persisting the array into concrete 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="n"&gt;stack&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;stack&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This starts downloads of our 10 000 images across our 10 workers. When this
completes we have 10 000 NumPy arrays spread around on our cluster, coordinated
by our single logical Dask array. This takes a while, about five minutes.
We’re mostly network bound here (Janelia’s servers are not co-located with our
compute nodes). Here is a parallel profile of the computation as an
interactive &lt;a class="reference external" href="http://bokeh.pydata.org/en/latest/"&gt;Bokeh&lt;/a&gt; plot.&lt;/p&gt;
&lt;p&gt;There will be a few of these profile plots throughout the blogpost, so you
might want to familiarize yoursel with them now. Every horizontal rectangle in
this plot corresponds to a single Python function running somewhere in our
cluster over time. Because we called &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;skimage.io.imread&lt;/span&gt;&lt;/code&gt; 10 000 times there
are 10 000 purple rectangles. Their position along the y-axis denotes which of
the 80 cores in our cluster that they ran on and their position along the
x-axis denotes their start and stop times. You can hover over each rectangle
(function) for more information on what kind of task it was, how long it took,
etc.. In the image below, purple rectangles are &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;skimage.io.imread&lt;/span&gt;&lt;/code&gt; calls and
red rectangles are data transfer between workers in our cluster. Click the
magnifying glass icons in the upper right of the image to enable zooming tools.&lt;/p&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/e09cad939ff7a85a06f3b387f65dc2fc/raw/fa5e20ca674cf5554aa4cab5141019465ef02ce9/task-stream-image-load.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;Now that we have persisted our Dask array in memory our data is based on
hundreds of concrete in-memory NumPy arrays across the cluster, rather than
based on hundreds of lazy scikit-image calls. Now we can do all sorts of fun
distributed array computations more quickly.&lt;/p&gt;
&lt;p&gt;For example we can easily see our field of interest move across the frame by
averaging across time:&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;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;stack&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;compute&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-imaging-time-mean.png"&gt;
  &lt;img src="/images/dask-imaging-time-mean.png"
         alt="Avergage image over time"
                width="100%"&gt;&lt;/a&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/e09cad939ff7a85a06f3b387f65dc2fc/raw/fa5e20ca674cf5554aa4cab5141019465ef02ce9/task-stream-image-mean-time.html"
        width="700" height="300"&gt;&lt;/iframe&gt;
&lt;p&gt;Or we can see when the field of interest is actually present within the frame
by averaging across x and y&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;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;plot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;stack&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="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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/dask-imaging-spatial-mean.png"&gt;
  &lt;img src="/images/dask-imaging-spatial-mean.png"
         alt="Image brightness over time"
                width="100%"&gt;&lt;/a&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/e09cad939ff7a85a06f3b387f65dc2fc/raw/fa5e20ca674cf5554aa4cab5141019465ef02ce9/task-stream-image-mean-spatial.html"
        width="700" height="300"&gt;&lt;/iframe&gt;
&lt;p&gt;By looking at the profile plots for each case we can see that averaging over
time involves much more inter-node communication, which can be quite expensive
in this 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/2017/01/17/dask-images.md&lt;/span&gt;, line 216)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="recenter-images-with-numba"&gt;
&lt;h1&gt;Recenter Images with Numba&lt;/h1&gt;
&lt;p&gt;In order to remove the spatial offset across time we’re going to compute a
centroid for each slice and then crop the image around that center. I looked
up centroids in the Scikit-Image docs and came across a function that did &lt;em&gt;way&lt;/em&gt;
more than what I was looking for, so I just quickly coded up a solution in Pure
Python and then JIT-ed it with &lt;a class="reference external" href="http://numba.pydata.org/"&gt;Numba&lt;/a&gt; (which makes
this run at C-speeds).&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;numba&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;jit&lt;/span&gt;

&lt;span class="nd"&gt;@jit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nogil&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;centroid&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;im&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;n&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;im&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;
    &lt;span class="n"&gt;total_x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;
    &lt;span class="n"&gt;total_y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;
    &lt;span class="n"&gt;total&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&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="n"&gt;n&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;j&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="n"&gt;m&lt;/span&gt;&lt;span class="p"&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;im&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="n"&gt;j&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
            &lt;span class="n"&gt;total_x&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;im&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="n"&gt;j&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
            &lt;span class="n"&gt;total_y&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="n"&gt;j&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;im&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="n"&gt;j&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;total&lt;/span&gt; &lt;span class="o"&gt;&amp;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;total_x&lt;/span&gt; &lt;span class="o"&gt;/=&lt;/span&gt; &lt;span class="n"&gt;total&lt;/span&gt;
        &lt;span class="n"&gt;total_y&lt;/span&gt; &lt;span class="o"&gt;/=&lt;/span&gt; &lt;span class="n"&gt;total&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;total_x&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;total_y&lt;/span&gt;

&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;centroid&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sample&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# this takes around 9ms&lt;/span&gt;
&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;748.7325324581344&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;802.4893005160851&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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;recenter&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;im&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;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;centroid&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;im&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;squeeze&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;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&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;x&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="mi"&gt;500&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;500&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt; &lt;span class="mi"&gt;500&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;500&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1500&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;1500&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1500&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;1500&lt;/span&gt;

    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;im&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;x&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;500&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;500&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;500&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;+&lt;/span&gt;&lt;span class="mi"&gt;500&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;figure&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;figsize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;8&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;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;recenter&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sample&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-imaging-recentered-sample.png"&gt;
  &lt;img src="/images/dask-imaging-recentered-sample.png"
       alt="Recentered image"
       width="40%"&gt;&lt;/a&gt;
&lt;p&gt;Now we map this function across our distributed 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="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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;recenter_block&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;block&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; Recenter a short stack of images &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="k"&gt;return&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;stack&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;recenter&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;block&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="n"&gt;block&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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;recentered&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;stack&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;recenter&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;20&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;# chunk size changes&lt;/span&gt;
                              &lt;span class="n"&gt;dtype&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;dtype&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;recentered&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;recentered&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/e09cad939ff7a85a06f3b387f65dc2fc/raw/fa5e20ca674cf5554aa4cab5141019465ef02ce9/task-stream-image-recentering.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;This profile provides a good opportunity to talk about a scheduling &lt;em&gt;failure&lt;/em&gt;;
things went a bit wrong here. Towards the beginning we quickly recenter
several images (Numba is fast), taking around 300-400ms for each block of
twenty images. However as some workers finish all of their allotted tasks, the
scheduler erroneously starts to load balance, moving images from busy workers
to idle workers. Unfortunately the network at this time appeared to be much
slower than expected and so the move + compute elsewhere strategy ended up
being much slower than just letting the busy workers finish their work. The
scheduler keeps track of expected compute times and transfer times precisely to
avoid mistakes like this one. These sorts of issues are rare, but do occur on
occasion.&lt;/p&gt;
&lt;p&gt;We check our work by averaging our re-centered images across time and displaying
that to the screen. We see that our images are better centered with each other
as expected.&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;skimage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;io&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;recentered&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/dask-imaging-recentered-time-mean.png"&gt;
  &lt;img src="/images/dask-imaging-recentered-time-mean.png"
       alt="Recentered time average"
       width="40%"&gt;&lt;/a&gt;
&lt;p&gt;This shows how easy it is to create fast in-memory code with Numba and then
scale it out with Dask.array. The two projects complement each other nicely,
giving us near-optimal performance with intuitive code across a cluster.&lt;/p&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/e09cad939ff7a85a06f3b387f65dc2fc/raw/fa5e20ca674cf5554aa4cab5141019465ef02ce9/task-stream-image-recenter-mean-time.html"
        width="800" height="400"&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/2017/01/17/dask-images.md&lt;/span&gt;, line 322)&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="rechunk-to-time-series-by-pixel"&gt;
&lt;h1&gt;Rechunk to Time Series by Pixel&lt;/h1&gt;
&lt;p&gt;We’re now going to rearrange our data from being partitioned by time slice, to
being partitioned by pixel. This will allow us to run computations like Fast
Fourier Transforms (FFTs) on each time series efficiently. Switching the chunk
pattern back and forth like this is generally a very difficult operation for
distributed arrays because every slice of the array contributes to every
time-series. We have N-squared communication.&lt;/p&gt;
&lt;p&gt;This analysis may not be appropriate for this data (we won’t learn any useful
science from doing this), but it represents a very frequently asked question,
so I wanted to include it.&lt;/p&gt;
&lt;p&gt;Currently our Dask array has chunkshape (20, 1000, 1000), meaning that our data
is collected into 500 NumPy arrays across the cluster, each of size &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(20,&lt;/span&gt; &lt;span class="pre"&gt;1000,&lt;/span&gt; &lt;span class="pre"&gt;1000)&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="n"&gt;recentered&lt;/span&gt;
&lt;span class="go"&gt;dask.array&amp;lt;shape=(10000, 1000, 1000), dtype=uint8, chunksize=(20, 1000, 1000)&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;But we want to change this shape so that the chunks cover the entire first
axis. We want all data for any particular pixel to be in the same NumPy array,
not spread across hundreds of different NumPy arrays. We could solve this by
rechunking so that each pixel is its own block 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;rechunked&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;recentered&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&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;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;However this would result in one million chunks (there are one million pixels)
which will result in a bit of scheduling overhead. Instead we’ll collect our
time-series into &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;10&lt;/span&gt; &lt;span class="pre"&gt;x&lt;/span&gt; &lt;span class="pre"&gt;10&lt;/span&gt;&lt;/code&gt; groups of one hundred pixels. This will help us to
reduce overhead.&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="c1"&gt;# rechunked = recentered.rechunk((10000, 1, 1))  # Too many chunks&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;rechunked&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;recentered&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&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;10&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="c1"&gt;# Use larger chunks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now we compute the FFT of each pixel, take the absolute value and square to
get the power spectrum. Finally to conserve space we’ll down-grade the dtype
to float32 (our original data is only 8-bit anyway).&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;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;fft&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fft&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;rechunked&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;power&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;abs&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="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;astype&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;float32&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;power&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;power&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;optimize_graph&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is a fun profile to inspect; it includes both the rechunking and the
subsequent FFTs. We’ve included a real-time trace during execution, the full
profile, as well as some diagnostics plots from a single worker. These plots
total up to around 20MB. I sincerely apologize to those without broadband
access.&lt;/p&gt;
&lt;p&gt;Here is a real time plot of the computation finishing over time:&lt;/p&gt;
&lt;a href="/images/task-stream-fft.gif"&gt;
  &lt;img src="/images/task-stream-fft.gif"
         alt="Dask task stream of rechunk + fft"
                width="100%"&gt;&lt;/a&gt;
&lt;p&gt;And here is a single interactive plot of the entire computation after it
completes. Zoom with the tools in the upper right. Hover over rectangles to
get more information. Remember that red is communication.&lt;/p&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/e09cad939ff7a85a06f3b387f65dc2fc/raw/fa5e20ca674cf5554aa4cab5141019465ef02ce9/task-stream-image-fft.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;Screenshots of the diagnostic dashboard of a single worker during this
computation.&lt;/p&gt;
&lt;a href="/images/worker-state-fft.png"&gt;
  &lt;img src="/images/worker-state-fft.png"
         alt="Worker communications during FFT"
                width="45%"&gt;&lt;/a&gt;
&lt;a href="/images/worker-communications-fft.png"&gt;
  &lt;img src="/images/worker-communications-fft.png"
         alt="Worker communications during FFT"
                width="45%"&gt;&lt;/a&gt;
&lt;p&gt;This computation starts with a lot of communication while we rechunk and
realign our data (recent optimizations here by &lt;a class="reference external" href="https://github.com/pitrou"&gt;Antoine
Pitrou&lt;/a&gt; in &lt;a class="reference external" href="https://github.com/dask/dask/pull/1737"&gt;dask #417&lt;/a&gt;).
Then we transition into doing thousands of small FFTs and other arithmetic
operations. All of the plots above show a nice transition from heavy
communication to heavy processing with some overlap each way (once some complex
blocks are available we get to start overlapping communication and
computation). Inter-worker communication was around 100-300 MB/s (typical for
Amazon’s EC2) and CPU load remained high. We’re &lt;em&gt;using&lt;/em&gt; our hardware.&lt;/p&gt;
&lt;p&gt;Finally we can inspect the results. We see that the power spectrum is very
boring in the corner, and has typical activity towards the center of the image.&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;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;semilogy&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;power&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/dask-imaging-fft-0.png"&gt;
  &lt;img src="/images/dask-imaging-fft-0.png"
         alt="Power spectrum near edge"
                width="70%"&gt;&lt;/a&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;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;semilogy&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;power&lt;/span&gt;&lt;span class="p"&gt;[:,&lt;/span&gt; &lt;span class="mi"&gt;500&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;500&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/dask-imaging-fft-center.png"&gt;
  &lt;img src="/images/dask-imaging-fft-center.png"
         alt="Power spectrum at center"
                width="70%"&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/2017/01/17/dask-images.md&lt;/span&gt;, line 436)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="final-thoughts"&gt;
&lt;h1&gt;Final Thoughts&lt;/h1&gt;
&lt;p&gt;This blogpost showed a non-trivial image processing workflow, emphasizing the
following points:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Construct a Dask array from lazy SKImage calls.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Use NumPy syntax with Dask.array to aggregate distributed data across a
cluster.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Build a centroid function with Numba. Use Numba and Dask together to
clean up an image stack.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Rechunk to facilitate time-series operations. Perform FFTs.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Hopefully this example has components that look similar to what you want to do
with your data on your hardware. We would love to see more applications like
this out there in the wild.&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/2017/01/17/dask-images.md&lt;/span&gt;, line 452)&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-we-could-have-done-better"&gt;
&lt;h1&gt;What we could have done better&lt;/h1&gt;
&lt;p&gt;As always with all computationally focused blogposts we’ll include a section on
what went wrong and what we could have done better with more time.&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Communication is too expensive&lt;/strong&gt;: Interworker communications that should
be taking 200ms are taking up to 10 or 20 seconds. We need to take a
closer look at our communications pipeline (which normally performs just
fine on other computations) to see if something is acting up.
Disucssion here &lt;a class="reference external" href="https://github.com/dask/distributed/issues/776"&gt;dask/distributed #776&lt;/a&gt;
and early work here &lt;a class="reference external" href="https://github.com/dask/distributed/pull/810"&gt;dask/distributed #810&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Faulty Load balancing&lt;/strong&gt;: We discovered a case where our load-balancing
heuristics misbehaved, incorrectly moving data between workers when it
would have been better to let everything alone. This is likely due to the
oddly low bandwidth issues observed above.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Loading from disk blocks network I/O&lt;/strong&gt;: While doing this we discovered an
issue where loading large amounts of data from disk can block workers from
responding to network requests (&lt;a class="reference external" href="https://github.com/dask/distributed/issues/774"&gt;dask/distributed
#774&lt;/a&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Larger datasets&lt;/strong&gt;: It would be fun to try this on a much larger dataset
to see how the solutions here scale.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/01/17/dask-images/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-01-17T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/01/12/dask-dataframes/</id>
    <title>Distributed Pandas on a Cluster with Dask DataFrames</title>
    <updated>2017-01-12T00: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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2017/01/12/dask-dataframes.md&lt;/span&gt;, line 13)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;Dask Dataframe extends the popular Pandas library to operate on big data-sets
on a distributed cluster. We show its capabilities by running through common
dataframe operations on a common dataset. We break up these computations into
the following sections:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Introduction: Pandas is intuitive and fast, but needs Dask to scale&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Read CSV and Basic operations&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Read CSV&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Basic Aggregations and Groupbys&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joins and Correlations&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Shuffles and Time Series&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Parquet I/O&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Final thoughts&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;What we could have done better&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/2017/01/12/dask-dataframes.md&lt;/span&gt;, line 30)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="accompanying-plots"&gt;
&lt;h1&gt;Accompanying Plots&lt;/h1&gt;
&lt;p&gt;Throughout this post we accompany computational examples with profiles of
exactly what task ran where on our cluster and when. These profiles are
interactive &lt;a class="reference external" href="https://bokeh.pydata.org"&gt;Bokeh plots&lt;/a&gt; that include every task
that every worker in our cluster runs over time. For example the following
computation &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_csv&lt;/span&gt;&lt;/code&gt; computation produces the following profile:&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://dask-data/nyc-taxi/2015/*.csv&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;&lt;em&gt;If you are reading this through a syndicated website like planet.python.org or
through an RSS reader then these plots will not show up. You may want to visit
&lt;a class="reference internal" href="../2017/01/12/dask-dataframes/"&gt;&lt;span class="doc std std-doc"&gt;/2017/01/12/dask-dataframes&lt;/span&gt;&lt;/a&gt;
directly.&lt;/em&gt;&lt;/p&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/ade9d1e3b0f44b17a84a551e39946e58/raw/1c3345848d5313cc1c0ea827d66089bf200edaac/task-stream-read-csv.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;Dask.dataframe breaks up reading this data into many small tasks of
different types. For example reading bytes and parsing those bytes into
pandas dataframes. Each rectangle corresponds to one task. The y-axis
enumerates each of the worker processes. We have 64 processes spread over
8 machines so there are 64 rows. You can hover over any rectangle to get more
information about that task. You can also use the tools in the upper right
to zoom around and focus on different regions in the computation. In this
computation we can see that workers interleave reading bytes from S3 (light
green) and parsing bytes to dataframes (dark green). The entire computation
took about a minute and most of the workers were busy the entire time (little
white space). Inter-worker communication is always depicted in red (which is
absent in this relatively straightforward 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/2017/01/12/dask-dataframes.md&lt;/span&gt;, line 63)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="introduction"&gt;
&lt;h1&gt;Introduction&lt;/h1&gt;
&lt;p&gt;Pandas provides an intuitive, powerful, and fast data analysis experience on
tabular data. However, because Pandas uses only one thread of execution and
requires all data to be in memory at once, it doesn’t scale well to datasets
much beyond the gigabyte scale. That component is missing. Generally people
move to Spark DataFrames on HDFS or a proper relational database to resolve
this scaling issue. Dask is a Python library for parallel and distributed
computing that aims to fill this need for parallelism among the PyData projects
(NumPy, Pandas, Scikit-Learn, etc.). Dask dataframes combine Dask and Pandas
to deliver a faithful “big data” version of Pandas operating in parallel over a
cluster.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference internal" href="../2016/02/22/dask-distributed-part-2/"&gt;&lt;span class="doc std std-doc"&gt;I’ve written about this topic
before&lt;/span&gt;&lt;/a&gt;.
This blogpost is newer and will focus on performance and newer features like
fast shuffles and the Parquet format.&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/2017/01/12/dask-dataframes.md&lt;/span&gt;, line 81)&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="csv-data-and-basic-operations"&gt;
&lt;h1&gt;CSV Data and Basic Operations&lt;/h1&gt;
&lt;p&gt;I have an eight node cluster on EC2 of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;m4.2xlarges&lt;/span&gt;&lt;/code&gt; (eight cores, 30GB RAM each).
Dask is running on each node with one process per core.&lt;/p&gt;
&lt;p&gt;We have the &lt;a class="reference external" href="http://www.nyc.gov/html/tlc/html/about/trip_record_data.shtml"&gt;2015 Yellow Cab NYC Taxi
data&lt;/a&gt; as 12 CSV
files on S3. We look at that data briefly with
&lt;a class="reference external" href="http://s3fs.readthedocs.io/en/latest/"&gt;s3fs&lt;/a&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;s3fs&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;s3&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;S3FileSystem&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;s3&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ls&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;dask-data/nyc-taxi/2015/&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;[&amp;#39;dask-data/nyc-taxi/2015/yellow_tripdata_2015-01.csv&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;dask-data/nyc-taxi/2015/yellow_tripdata_2015-02.csv&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;dask-data/nyc-taxi/2015/yellow_tripdata_2015-03.csv&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;dask-data/nyc-taxi/2015/yellow_tripdata_2015-04.csv&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;dask-data/nyc-taxi/2015/yellow_tripdata_2015-05.csv&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;dask-data/nyc-taxi/2015/yellow_tripdata_2015-06.csv&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;dask-data/nyc-taxi/2015/yellow_tripdata_2015-07.csv&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;dask-data/nyc-taxi/2015/yellow_tripdata_2015-08.csv&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;dask-data/nyc-taxi/2015/yellow_tripdata_2015-09.csv&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;dask-data/nyc-taxi/2015/yellow_tripdata_2015-10.csv&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;dask-data/nyc-taxi/2015/yellow_tripdata_2015-11.csv&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;dask-data/nyc-taxi/2015/yellow_tripdata_2015-12.csv&amp;#39;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This data is too large to fit into Pandas on a single computer. However, it
can fit in memory if we break it up into many small pieces and load these
pieces onto different computers across a cluster.&lt;/p&gt;
&lt;p&gt;We connect a client to our Dask cluster, composed of one centralized
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-scheduler&lt;/span&gt;&lt;/code&gt; process and several &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-worker&lt;/span&gt;&lt;/code&gt; processes running on each of the
machines in our cluster.&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.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scheduler-address:8786&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;And we load our CSV data using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe&lt;/span&gt;&lt;/code&gt; which looks and feels just
like Pandas, even though it’s actually coordinating hundreds of small Pandas
dataframes. This takes about a minute to load and parse.&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;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://dask-data/nyc-taxi/2015/*.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                 &lt;span class="n"&gt;parse_dates&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;tpep_pickup_datetime&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;tpep_dropoff_datetime&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                 &lt;span class="n"&gt;storage_options&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;anon&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;})&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/ade9d1e3b0f44b17a84a551e39946e58/raw/1c3345848d5313cc1c0ea827d66089bf200edaac/task-stream-read-csv.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;This cuts up our 12 CSV files on S3 into a few hundred blocks of bytes, each
64MB large. On each of these 64MB blocks we then call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pandas.read_csv&lt;/span&gt;&lt;/code&gt; to
create a few hundred Pandas dataframes across our cluster, one for each block
of bytes. Our single Dask Dataframe object, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df&lt;/span&gt;&lt;/code&gt;, coordinates all of those
Pandas dataframes. Because we’re just using Pandas calls it’s very easy for
Dask dataframes to use all of the tricks from Pandas. For example we can use
most of the keyword arguments from &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pd.read_csv&lt;/span&gt;&lt;/code&gt; in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dd.read_csv&lt;/span&gt;&lt;/code&gt; without
having to relearn anything.&lt;/p&gt;
&lt;p&gt;This data is about 20GB on disk or 60GB in RAM. It’s not huge, but is also
larger than we’d like to manage on a laptop, especially if we value
interactivity. The interactive image above is a trace over time of what each
of our 64 cores was doing at any given moment. By hovering your mouse over the
rectangles you can see that cores switched between downloading byte ranges from
S3 and parsing those bytes with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pandas.read_csv&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Our dataset includes every cab ride in the city of New York in the year of
2015, including when and where it started and stopped, a breakdown of the fare,
etc.&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;df&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
    &lt;tr style="text-align: right;"&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;VendorID&lt;/th&gt;
      &lt;th&gt;tpep_pickup_datetime&lt;/th&gt;
      &lt;th&gt;tpep_dropoff_datetime&lt;/th&gt;
      &lt;th&gt;passenger_count&lt;/th&gt;
      &lt;th&gt;trip_distance&lt;/th&gt;
      &lt;th&gt;pickup_longitude&lt;/th&gt;
      &lt;th&gt;pickup_latitude&lt;/th&gt;
      &lt;th&gt;RateCodeID&lt;/th&gt;
      &lt;th&gt;store_and_fwd_flag&lt;/th&gt;
      &lt;th&gt;dropoff_longitude&lt;/th&gt;
      &lt;th&gt;dropoff_latitude&lt;/th&gt;
      &lt;th&gt;payment_type&lt;/th&gt;
      &lt;th&gt;fare_amount&lt;/th&gt;
      &lt;th&gt;extra&lt;/th&gt;
      &lt;th&gt;mta_tax&lt;/th&gt;
      &lt;th&gt;tip_amount&lt;/th&gt;
      &lt;th&gt;tolls_amount&lt;/th&gt;
      &lt;th&gt;improvement_surcharge&lt;/th&gt;
      &lt;th&gt;total_amount&lt;/th&gt;
    &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt;0&lt;/th&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;2015-01-15 19:05:39&lt;/td&gt;
      &lt;td&gt;2015-01-15 19:23:42&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;1.59&lt;/td&gt;
      &lt;td&gt;-73.993896&lt;/td&gt;
      &lt;td&gt;40.750111&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.974785&lt;/td&gt;
      &lt;td&gt;40.750618&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;12.0&lt;/td&gt;
      &lt;td&gt;1.0&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;3.25&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;17.05&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;1&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:33:38&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:53:28&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;3.30&lt;/td&gt;
      &lt;td&gt;-74.001648&lt;/td&gt;
      &lt;td&gt;40.724243&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.994415&lt;/td&gt;
      &lt;td&gt;40.759109&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;14.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;2.00&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;17.80&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;2&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:33:38&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:43:41&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;1.80&lt;/td&gt;
      &lt;td&gt;-73.963341&lt;/td&gt;
      &lt;td&gt;40.802788&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.951820&lt;/td&gt;
      &lt;td&gt;40.824413&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;9.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;10.80&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;3&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:33:39&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:35:31&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;0.50&lt;/td&gt;
      &lt;td&gt;-74.009087&lt;/td&gt;
      &lt;td&gt;40.713818&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-74.004326&lt;/td&gt;
      &lt;td&gt;40.719986&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;3.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;4.80&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;4&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:33:39&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:52:58&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;3.00&lt;/td&gt;
      &lt;td&gt;-73.971176&lt;/td&gt;
      &lt;td&gt;40.762428&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-74.004181&lt;/td&gt;
      &lt;td&gt;40.742653&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;15.0&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;16.30&lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;section id="basic-aggregations-and-groupbys"&gt;
&lt;h2&gt;Basic Aggregations and Groupbys&lt;/h2&gt;
&lt;p&gt;As a quick exercise, we compute the length of the dataframe. When we call
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;len(df)&lt;/span&gt;&lt;/code&gt; Dask.dataframe translates this into many &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;len&lt;/span&gt;&lt;/code&gt; calls on each of the
constituent Pandas dataframes, followed by communication of the intermediate
results to one node, followed by a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sum&lt;/span&gt;&lt;/code&gt; of all of the intermediate lengths.&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;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;146112989&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/ade9d1e3b0f44b17a84a551e39946e58/raw/1c3345848d5313cc1c0ea827d66089bf200edaac/task-stream-len.html"
        width="640" height="630"&gt;&lt;/iframe&gt;
&lt;p&gt;This takes around 400-500ms. You can see that a few hundred length
computations happened quickly on the left, followed by some delay, then a bit
of data transfer (the red bar in the plot), and a final summation call.&lt;/p&gt;
&lt;p&gt;More complex operations like simple groupbys look similar, although sometimes
with more communications. Throughout this post we’re going to do more and more
complex computations and our profiles will similarly become more and more rich
with information. Here we compute the average trip distance, grouped by number
of passengers. We find that single and double person rides go far longer
distances on average. We acheive this one big-data-groupby by performing many
small Pandas groupbys and then cleverly combining their results.&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;passenger_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;trip_distance&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="go"&gt;passenger_count&lt;/span&gt;
&lt;span class="go"&gt;0     2.279183&lt;/span&gt;
&lt;span class="go"&gt;1    15.541413&lt;/span&gt;
&lt;span class="go"&gt;2    11.815871&lt;/span&gt;
&lt;span class="go"&gt;3     1.620052&lt;/span&gt;
&lt;span class="go"&gt;4     7.481066&lt;/span&gt;
&lt;span class="go"&gt;5     3.066019&lt;/span&gt;
&lt;span class="go"&gt;6     2.977158&lt;/span&gt;
&lt;span class="go"&gt;9     5.459763&lt;/span&gt;
&lt;span class="go"&gt;7     3.303054&lt;/span&gt;
&lt;span class="go"&gt;8     3.866298&lt;/span&gt;
&lt;span class="go"&gt;Name: trip_distance, dtype: float64&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/ade9d1e3b0f44b17a84a551e39946e58/raw/1c3345848d5313cc1c0ea827d66089bf200edaac/task-stream-groupby-sum.html"
        width="640" height="630"&gt;&lt;/iframe&gt;
&lt;p&gt;As a more complex operation we see how well New Yorkers tip by hour of day and
by day of week.&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;df2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tip_amount&lt;/span&gt; &lt;span class="o"&gt;&amp;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;&amp;amp;&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fare_amount&lt;/span&gt; &lt;span class="o"&gt;&amp;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;# filter out bad rows&lt;/span&gt;
&lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;tip_fraction&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;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tip_amount&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fare_amount&lt;/span&gt;  &lt;span class="c1"&gt;# make new column&lt;/span&gt;

&lt;span class="n"&gt;dayofweek&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tpep_pickup_datetime&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dayofweek&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
                &lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tip_fraction&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;hour&lt;/span&gt;      &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tpep_pickup_datetime&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;hour&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
                &lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tip_fraction&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/ade9d1e3b0f44b17a84a551e39946e58/raw/1c3345848d5313cc1c0ea827d66089bf200edaac/task-stream-groupby-datetime.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;a href="/images/tip-fraction-hourly.png"&gt;
    &lt;img src="/images/tip-fraction-hourly.png"
         alt="tip fraction by hour"
         width="80%"&gt;&lt;/a&gt;
&lt;p&gt;We see that New Yorkers are generally pretty generous, tipping around 20%-25%
on average. We also notice that they become &lt;em&gt;very generous&lt;/em&gt; at 4am, tipping an
average of 38%.&lt;/p&gt;
&lt;p&gt;This more complex operation uses more of the Dask dataframe API (which mimics
the Pandas API). Pandas users should find the code above fairly familiar. We
remove rows with zero fare or zero tip (not every tip gets recorded), make a
new column which is the ratio of the tip amount to the fare amount, and then
groupby the day of week and hour of day, computing the average tip fraction for
each hour/day.&lt;/p&gt;
&lt;p&gt;Dask evaluates this computation with thousands of small Pandas calls across the
cluster (try clicking the wheel zoom icon in the upper right of the image
above and zooming in). The answer comes back in about 3 seconds.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="joins-and-correlations"&gt;
&lt;h2&gt;Joins and Correlations&lt;/h2&gt;
&lt;p&gt;To show off more basic functionality we’ll join this Dask dataframe against a
smaller Pandas dataframe that includes names of some of the more cryptic
columns. Then we’ll correlate two derived columns to determine if there is a
relationship between paying Cash and the recorded tip.&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;payments&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Series&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="s1"&gt;&amp;#39;Credit Card&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="go"&gt;                          2: &amp;#39;Cash&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;                          3: &amp;#39;No Charge&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;                          4: &amp;#39;Dispute&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;                          5: &amp;#39;Unknown&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;                          6: &amp;#39;Voided trip&amp;#39;})&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;df2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;merge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;payments&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;left_on&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;payment_type&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;right_index&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;payment_name&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tip_amount&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="go"&gt;payment_name&lt;/span&gt;
&lt;span class="go"&gt;Cash           0.000217&lt;/span&gt;
&lt;span class="go"&gt;Credit Card    2.757708&lt;/span&gt;
&lt;span class="go"&gt;Dispute       -0.011553&lt;/span&gt;
&lt;span class="go"&gt;No charge      0.003902&lt;/span&gt;
&lt;span class="go"&gt;Unknown        0.428571&lt;/span&gt;
&lt;span class="go"&gt;Name: tip_amount, dtype: float64&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We see that while the average tip for a credit card transaction is $2.75, the
average tip for a cash transaction is very close to zero. At first glance it
seems like cash tips aren’t being reported. To investigate this a bit further
lets compute the Pearson correlation between paying cash and having zero tip.
Again, this code should look very familiar to Pandas users.&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;zero_tip&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tip_amount&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;
&lt;span class="n"&gt;cash&lt;/span&gt;     &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;payment_name&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Cash&amp;#39;&lt;/span&gt;

&lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;concat&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;zero_tip&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cash&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;corr&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
    &lt;tr style="text-align: right;"&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;tip_amount&lt;/th&gt;
      &lt;th&gt;payment_name&lt;/th&gt;
    &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt;tip_amount&lt;/th&gt;
      &lt;td&gt;1.000000&lt;/td&gt;
      &lt;td&gt;0.943123&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;payment_name&lt;/th&gt;
      &lt;td&gt;0.943123&lt;/td&gt;
      &lt;td&gt;1.000000&lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/ade9d1e3b0f44b17a84a551e39946e58/raw/1c3345848d5313cc1c0ea827d66089bf200edaac/task-stream-join-corr.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;So we see that standard operations like row filtering, column selection,
groupby-aggregations, joining with a Pandas dataframe, correlations, etc. all
look and feel like the Pandas interface. Additionally, we’ve seen through
profile plots that most of the time is spent just running Pandas functions on
our workers, so Dask.dataframe is, in most cases, adding relatively little
overhead. These little functions represented by the rectangles in these plots
are &lt;em&gt;just pandas functions&lt;/em&gt;. For example the plot above has many rectangles
labeled &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;merge&lt;/span&gt;&lt;/code&gt; if you hover over them. This is just the standard
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pandas.merge&lt;/span&gt;&lt;/code&gt; function that we love and know to be very fast in 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/2017/01/12/dask-dataframes.md&lt;/span&gt;, line 459)&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="shuffles-and-time-series"&gt;
&lt;h1&gt;Shuffles and Time Series&lt;/h1&gt;
&lt;p&gt;Distributed dataframe experts will know that none of the operations above
require a &lt;em&gt;shuffle&lt;/em&gt;. That is we can do most of our work with relatively little
inter-node communication. However not all operations can avoid communication
like this and sometimes we need to exchange most of the data between different
workers.&lt;/p&gt;
&lt;p&gt;For example if our dataset is sorted by customer ID but we want to sort it by
time then we need to collect all the rows for January over to one Pandas
dataframe, all the rows for February over to another, etc.. This operation is
called a shuffle and is the base of computations like groupby-apply,
distributed joins on columns that are not the index, etc..&lt;/p&gt;
&lt;p&gt;You can do a lot with dask.dataframe without performing shuffles, but sometimes
it’s necessary. In the following example we sort our data by pickup datetime.
This will allow fast lookups, fast joins, and fast time series operations, all
common cases. We do one shuffle ahead of time to make all future computations
fast.&lt;/p&gt;
&lt;p&gt;We set the index as the pickup datetime column. This takes anywhere from
25-40s and is largely network bound (60GB, some text, eight machines with
eight cores each on AWS non-enhanced network). This also requires running
something like 16000 tiny tasks on the cluster. It’s worth zooming in on the
plot 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;df&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;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_index&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;tpep_pickup_datetime&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/ade9d1e3b0f44b17a84a551e39946e58/raw/1c3345848d5313cc1c0ea827d66089bf200edaac/task-stream-set-index.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;This operation is expensive, far more expensive than it was with Pandas when
all of the data was in the same memory space on the same computer. This is a
good time to point out that you should only use distributed tools like
Dask.datframe and Spark after tools like Pandas break down. We should only
move to distributed systems when absolutely necessary. However, when it does
become necessary, it’s nice knowing that Dask.dataframe can faithfully execute
Pandas operations, even if some of them take a bit longer.&lt;/p&gt;
&lt;p&gt;As a result of this shuffle our data is now nicely sorted by time, which will
keep future operations close to optimal. We can see how the dataset is sorted
by pickup time by quickly looking at the first entries, last entries, and
entries for a particular day.&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;df&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="c1"&gt;# has the first entries of 2015&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
    &lt;tr style="text-align: right;"&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;VendorID&lt;/th&gt;
      &lt;th&gt;tpep_dropoff_datetime&lt;/th&gt;
      &lt;th&gt;passenger_count&lt;/th&gt;
      &lt;th&gt;trip_distance&lt;/th&gt;
      &lt;th&gt;pickup_longitude&lt;/th&gt;
      &lt;th&gt;pickup_latitude&lt;/th&gt;
      &lt;th&gt;RateCodeID&lt;/th&gt;
      &lt;th&gt;store_and_fwd_flag&lt;/th&gt;
      &lt;th&gt;dropoff_longitude&lt;/th&gt;
      &lt;th&gt;dropoff_latitude&lt;/th&gt;
      &lt;th&gt;payment_type&lt;/th&gt;
      &lt;th&gt;fare_amount&lt;/th&gt;
      &lt;th&gt;extra&lt;/th&gt;
      &lt;th&gt;mta_tax&lt;/th&gt;
      &lt;th&gt;tip_amount&lt;/th&gt;
      &lt;th&gt;tolls_amount&lt;/th&gt;
      &lt;th&gt;improvement_surcharge&lt;/th&gt;
      &lt;th&gt;total_amount&lt;/th&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;tpep_pickup_datetime&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
    &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt;2015-01-01 00:00:00&lt;/th&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;2015-01-01 00:00:00&lt;/td&gt;
      &lt;td&gt;3&lt;/td&gt;
      &lt;td&gt;1.56&lt;/td&gt;
      &lt;td&gt;-74.001320&lt;/td&gt;
      &lt;td&gt;40.729057&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-74.010208&lt;/td&gt;
      &lt;td&gt;40.719662&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;7.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;8.8&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;2015-01-01 00:00:00&lt;/th&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;2015-01-01 00:00:00&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;1.68&lt;/td&gt;
      &lt;td&gt;-73.991547&lt;/td&gt;
      &lt;td&gt;40.750069&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;0.000000&lt;/td&gt;
      &lt;td&gt;0.000000&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;10.0&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;10.8&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;2015-01-01 00:00:00&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-01-01 00:11:26&lt;/td&gt;
      &lt;td&gt;5&lt;/td&gt;
      &lt;td&gt;4.00&lt;/td&gt;
      &lt;td&gt;-73.971436&lt;/td&gt;
      &lt;td&gt;40.760201&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.921181&lt;/td&gt;
      &lt;td&gt;40.768269&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;13.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;14.5&lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tail&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# has the last entries of 2015&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
    &lt;tr style="text-align: right;"&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;VendorID&lt;/th&gt;
      &lt;th&gt;tpep_dropoff_datetime&lt;/th&gt;
      &lt;th&gt;passenger_count&lt;/th&gt;
      &lt;th&gt;trip_distance&lt;/th&gt;
      &lt;th&gt;pickup_longitude&lt;/th&gt;
      &lt;th&gt;pickup_latitude&lt;/th&gt;
      &lt;th&gt;RateCodeID&lt;/th&gt;
      &lt;th&gt;store_and_fwd_flag&lt;/th&gt;
      &lt;th&gt;dropoff_longitude&lt;/th&gt;
      &lt;th&gt;dropoff_latitude&lt;/th&gt;
      &lt;th&gt;payment_type&lt;/th&gt;
      &lt;th&gt;fare_amount&lt;/th&gt;
      &lt;th&gt;extra&lt;/th&gt;
      &lt;th&gt;mta_tax&lt;/th&gt;
      &lt;th&gt;tip_amount&lt;/th&gt;
      &lt;th&gt;tolls_amount&lt;/th&gt;
      &lt;th&gt;improvement_surcharge&lt;/th&gt;
      &lt;th&gt;total_amount&lt;/th&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;tpep_pickup_datetime&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
    &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt;2015-12-31 23:59:56&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2016-01-01 00:09:25&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;1.00&lt;/td&gt;
      &lt;td&gt;-73.973900&lt;/td&gt;
      &lt;td&gt;40.742893&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.989571&lt;/td&gt;
      &lt;td&gt;40.750549&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;8.0&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;1.85&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;11.15&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;2015-12-31 23:59:58&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2016-01-01 00:05:19&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;2.00&lt;/td&gt;
      &lt;td&gt;-73.965271&lt;/td&gt;
      &lt;td&gt;40.760281&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.939514&lt;/td&gt;
      &lt;td&gt;40.752388&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;7.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;8.80&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;2015-12-31 23:59:59&lt;/th&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;2016-01-01 00:10:26&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;1.96&lt;/td&gt;
      &lt;td&gt;-73.997559&lt;/td&gt;
      &lt;td&gt;40.725693&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-74.017120&lt;/td&gt;
      &lt;td&gt;40.705322&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;8.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0.0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;9.80&lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;loc&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;2015-05-05&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;head&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# has the entries for just May 5th&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
    &lt;tr style="text-align: right;"&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;VendorID&lt;/th&gt;
      &lt;th&gt;tpep_dropoff_datetime&lt;/th&gt;
      &lt;th&gt;passenger_count&lt;/th&gt;
      &lt;th&gt;trip_distance&lt;/th&gt;
      &lt;th&gt;pickup_longitude&lt;/th&gt;
      &lt;th&gt;pickup_latitude&lt;/th&gt;
      &lt;th&gt;RateCodeID&lt;/th&gt;
      &lt;th&gt;store_and_fwd_flag&lt;/th&gt;
      &lt;th&gt;dropoff_longitude&lt;/th&gt;
      &lt;th&gt;dropoff_latitude&lt;/th&gt;
      &lt;th&gt;payment_type&lt;/th&gt;
      &lt;th&gt;fare_amount&lt;/th&gt;
      &lt;th&gt;extra&lt;/th&gt;
      &lt;th&gt;mta_tax&lt;/th&gt;
      &lt;th&gt;tip_amount&lt;/th&gt;
      &lt;th&gt;tolls_amount&lt;/th&gt;
      &lt;th&gt;improvement_surcharge&lt;/th&gt;
      &lt;th&gt;total_amount&lt;/th&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;tpep_pickup_datetime&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;&lt;/th&gt;
    &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt;2015-05-05&lt;/th&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;2015-05-05 00:00:00&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;1.20&lt;/td&gt;
      &lt;td&gt;-73.981941&lt;/td&gt;
      &lt;td&gt;40.766460&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.972771&lt;/td&gt;
      &lt;td&gt;40.758007&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;6.5&lt;/td&gt;
      &lt;td&gt;1.0&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;8.30&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;2015-05-05&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-05-05 00:10:12&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;1.70&lt;/td&gt;
      &lt;td&gt;-73.994675&lt;/td&gt;
      &lt;td&gt;40.750507&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.980247&lt;/td&gt;
      &lt;td&gt;40.738560&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;9.0&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;2.57&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;12.87&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;2015-05-05&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-05-05 00:07:50&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2.50&lt;/td&gt;
      &lt;td&gt;-74.002930&lt;/td&gt;
      &lt;td&gt;40.733681&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-74.013603&lt;/td&gt;
      &lt;td&gt;40.702362&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;9.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;10.80&lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;Because we know exactly which Pandas dataframe holds which data we can
execute row-local queries like this very quickly. The total round trip from
pressing enter in the interpreter or notebook is about 40ms. For reference,
40ms is the delay between two frames in a movie running at 25 Hz. This means
that it’s fast enough that human users perceive this query to be entirely
fluid.&lt;/p&gt;
&lt;section id="time-series"&gt;
&lt;h2&gt;Time Series&lt;/h2&gt;
&lt;p&gt;Additionally, once we have a nice datetime index all of Pandas’ time series
functionality becomes available to us.&lt;/p&gt;
&lt;p&gt;For example we can resample by day:&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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;passenger_count&lt;/span&gt;
&lt;span class="go"&gt;       .resample(&amp;#39;1d&amp;#39;)&lt;/span&gt;
&lt;span class="go"&gt;       .mean()&lt;/span&gt;
&lt;span class="go"&gt;       .compute()&lt;/span&gt;
&lt;span class="go"&gt;       .plot())&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/resample-day.png"&gt;
    &lt;img src="/images/resample-day.png"
         alt="resample by day"
         width="60%"&gt;&lt;/a&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/ade9d1e3b0f44b17a84a551e39946e58/raw/1c3345848d5313cc1c0ea827d66089bf200edaac/task-stream-resample.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;We observe a strong periodic signal here. The number of passengers is reliably
higher on the weekends.&lt;/p&gt;
&lt;p&gt;We can perform a rolling aggregation in about a second:&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;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;passenger_count&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rolling&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/ade9d1e3b0f44b17a84a551e39946e58/raw/1c3345848d5313cc1c0ea827d66089bf200edaac/task-stream-rolling-mean.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;Because Dask.dataframe inherits the Pandas index all of these operations become
very fast and intuitive.&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/2017/01/12/dask-dataframes.md&lt;/span&gt;, line 898)&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="parquet"&gt;
&lt;h1&gt;Parquet&lt;/h1&gt;
&lt;p&gt;Pandas’ standard “fast” recommended storage solution has generally been the
HDF5 data format. Unfortunately the HDF5 file format is not ideal for
distributed computing, so most Dask dataframe users have had to switch down to
CSV historically. This is unfortunate because CSV is slow, doesn’t support
partial queries (you can’t read in just one column), and also isn’t supported
well by the other standard distributed Dataframe solution, Spark. This makes it
hard to move data back and forth.&lt;/p&gt;
&lt;p&gt;Fortunately there are now two decent Python readers for Parquet, a fast
columnar binary store that shards nicely on distributed data stores like the
Hadoop File System (HDFS, not to be confused with HDF5) and Amazon’s S3. The
already fast &lt;a class="reference external" href="https://github.com/apache/parquet-cpp"&gt;Parquet-cpp project&lt;/a&gt; has
been growing Python and Pandas support through
&lt;a class="reference external" href="http://pyarrow.readthedocs.io/en/latest/"&gt;Arrow&lt;/a&gt;, and the &lt;a class="reference external" href="http://fastparquet.readthedocs.io/"&gt;Fastparquet
project&lt;/a&gt;, which is an offshoot from the
&lt;a class="reference external" href="https://github.com/jcrobak/parquet-python"&gt;pure-python &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;parquet&lt;/span&gt;&lt;/code&gt; library&lt;/a&gt; has
been growing speed through use of
&lt;a class="reference external" href="https://docs.scipy.org/doc/numpy/reference/"&gt;NumPy&lt;/a&gt; and
&lt;a class="reference external" href="http://numba.pydata.org/"&gt;Numba&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Using Fastparquet under the hood, Dask.dataframe users can now happily read and
write to Parquet files. This increases speed, decreases storage costs, and
provides a shared format that both Dask dataframes and Spark dataframes can
understand, improving the ability to use both computational systems in the same
workflow.&lt;/p&gt;
&lt;p&gt;Writing our Dask dataframe to S3 can be as simple as 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="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://dask-data/nyc-taxi/tmp/parquet&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;However there are also a variety of options we can use to store our data more
compactly through compression, encodings, etc.. Expert users will probably
recognize some of the terms 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="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;astype&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;VendorID&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;uint8&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                &lt;span class="s1"&gt;&amp;#39;passenger_count&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;uint8&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                &lt;span class="s1"&gt;&amp;#39;RateCodeID&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;uint8&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                &lt;span class="s1"&gt;&amp;#39;payment_type&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;uint8&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;})&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://dask-data/nyc-taxi/tmp/parquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
              &lt;span class="n"&gt;compression&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;snappy&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
              &lt;span class="n"&gt;has_nulls&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
              &lt;span class="n"&gt;object_encoding&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;utf8&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
              &lt;span class="n"&gt;fixed_text&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;store_and_fwd_flag&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;iframe src="https://cdn.rawgit.com/mrocklin/ade9d1e3b0f44b17a84a551e39946e58/raw/1c3345848d5313cc1c0ea827d66089bf200edaac/task-stream-to-parquet.html"
        width="800" height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;We can then read our nicely indexed dataframe back with the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dd.read_parquet&lt;/span&gt;&lt;/code&gt; function:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;df2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://dask-data/nyc-taxi/tmp/parquet&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;The main benefit here is that we can quickly compute on single columns. The
following computation runs in around 6 seconds, even though we don’t have any
data in memory to start (recall that we started this blogpost with a
minute-long call to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_csv&lt;/span&gt;&lt;/code&gt;.and
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/api.html#distributed.client.Client.persist"&gt;Client.persist&lt;/a&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="n"&gt;df2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;passenger_count&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&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="go"&gt;1    102991045&lt;/span&gt;
&lt;span class="go"&gt;2     20901372&lt;/span&gt;
&lt;span class="go"&gt;5      7939001&lt;/span&gt;
&lt;span class="go"&gt;3      6135107&lt;/span&gt;
&lt;span class="go"&gt;6      5123951&lt;/span&gt;
&lt;span class="go"&gt;4      2981071&lt;/span&gt;
&lt;span class="go"&gt;0        40853&lt;/span&gt;
&lt;span class="go"&gt;7          239&lt;/span&gt;
&lt;span class="go"&gt;8          181&lt;/span&gt;
&lt;span class="go"&gt;9          169&lt;/span&gt;
&lt;span class="go"&gt;Name: passenger_count, dtype: int64&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/2017/01/12/dask-dataframes.md&lt;/span&gt;, line 980)&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;With the recent addition of faster shuffles and Parquet support, Dask
dataframes become significantly more attractive. This blogpost gave a few
categories of common computations, along with precise profiles of their
execution on a small cluster. Hopefully people find this combination of Pandas
syntax and scalable computing useful.&lt;/p&gt;
&lt;p&gt;Now would also be a good time to remind people that Dask dataframe is only one
module among many within the &lt;a class="reference external" href="http://dask.pydata.org/en/latest/"&gt;Dask project&lt;/a&gt;.
Dataframes are nice, certainly, but Dask’s main strength is its flexibility to
move beyond just plain dataframe computations to handle even more complex
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/2017/01/12/dask-dataframes.md&lt;/span&gt;, line 994)&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="learn-more"&gt;
&lt;h1&gt;Learn More&lt;/h1&gt;
&lt;p&gt;If you’d like to learn more about Dask dataframe, the Dask distributed system,
or other components you should look at the following documentation:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask.pydata.org/en/latest/"&gt;http://dask.pydata.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://distributed.readthedocs.io/en/latest/"&gt;http://distributed.readthedocs.io/en/latest/&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The workflows presented here are captured in the following notebooks (among
other examples):&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/ada85ef06d625947f7b34886fd2710f8"&gt;NYC Taxi example, shuffling, others&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/89bccf2f4f37611b40c18967bb182066"&gt;Parquet&lt;/a&gt;&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/2017/01/12/dask-dataframes.md&lt;/span&gt;, line 1008)&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-we-could-have-done-better"&gt;
&lt;h1&gt;What we could have done better&lt;/h1&gt;
&lt;p&gt;As always with computational posts we include a section on what went wrong, or
what could have gone better.&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;The 400ms computation of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;len(df)&lt;/span&gt;&lt;/code&gt; is a regression from previous
versions where this was closer to 100ms. We’re getting bogged down
somewhere in many small inter-worker communications.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It would be nice to repeat this computation at a larger scale. Dask
deployments in the wild are often closer to 1000 cores rather than the 64
core cluster we have here and datasets are often in the terrabyte scale
rather than our 60 GB NYC Taxi dataset. Unfortunately representative large
open datasets are hard to find.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The Parquet timings are nice, but there is still room for improvement. We
seem to be making many small expensive queries of S3 when reading Thrift
headers.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It would be nice to support both Python Parquet readers, both the
&lt;a class="reference external" href="http://numba.pydata.org/"&gt;Numba&lt;/a&gt; solution
&lt;a class="reference external" href="https://fastparquet.readthedocs.io"&gt;fastparquet&lt;/a&gt; and the C++ solution
&lt;a class="reference external" href="https://github.com/apache/parquet-cpp"&gt;parquet-cpp&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/01/12/dask-dataframes/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-01-12T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2017/01/03/dask-0.13.0/</id>
    <title>Dask Release 0.13.0</title>
    <updated>2017-01-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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2017/01/03/dask-0.13.0.md&lt;/span&gt;, line 13)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;Dask just grew to version 0.13.0. This is a signifcant release for arrays,
dataframes, and the distributed scheduler. This blogpost outlines some of the
major changes since the last release November 4th.&lt;/p&gt;
&lt;ol class="arabic simple" start="0"&gt;
&lt;li&gt;&lt;p&gt;Python 3.6 support&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Algorithmic and API improvements for DataFrames&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dataframe to Array conversions for Machine Learning&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Parquet support&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scheduling Performance and Worker Rewrite&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Pervasive Visual Diagnostics with Embedded Bokeh Servers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Windows continuous integration&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Custom serialization&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;You can install new versions using Conda or Pip&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 -c conda-forge dask distributed
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or&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[complete] distributed --upgrade
&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/2017/01/03/dask-0.13.0.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="python-3-6-support"&gt;
&lt;h1&gt;Python 3.6 Support&lt;/h1&gt;
&lt;p&gt;Dask and all necessary dependencies are now available on &lt;a class="reference external" href="https://conda-forge.github.io/"&gt;Conda
Forge&lt;/a&gt; for Python 3.6.&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/2017/01/03/dask-0.13.0.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="algorithmic-and-api-improvements-for-dataframes"&gt;
&lt;h1&gt;Algorithmic and API Improvements for DataFrames&lt;/h1&gt;
&lt;p&gt;Thousand-core Dask deployments have become significantly more common in the
last few months. This has highlighted scaling issues in some of the
Dask.array and Dask.dataframe algorithms, which were originally designed for
single workstations. Algorithmic and API changes can be grouped into the
following two categories:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Filling out the Pandas API&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Algorithms that needed to be changed or added due to scaling issues&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Dask Dataframes now include a fuller set of the Pandas API, including the
following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Inplace operations like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df['x']&lt;/span&gt; &lt;span class="pre"&gt;=&lt;/span&gt; &lt;span class="pre"&gt;df.y&lt;/span&gt; &lt;span class="pre"&gt;+&lt;/span&gt; &lt;span class="pre"&gt;df.z&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The full Groupby-aggregate syntax like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.groupby(...).aggregate({'x':&lt;/span&gt; &lt;span class="pre"&gt;'sum',&lt;/span&gt; &lt;span class="pre"&gt;'y':&lt;/span&gt; &lt;span class="pre"&gt;['min',&lt;/span&gt; &lt;span class="pre"&gt;max']})&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Resample on dataframes as well as series&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Pandas’ new rolling syntax &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.x.rolling(10).mean()&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;And much more&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Additionally, collaboration with some of the larger Dask deployments has
highlighted scaling issues in some algorithms, resulting in the following improvements:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Tree reductions for groupbys, aggregations, etc.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Multi-output-partition aggregations for groupby-aggregations with millions of groups, drop_duplicates, etc..&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Approximate algorithms for nunique&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;etc..&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;These same collaborations have also yielded better handling of open file
descriptors, changes upstream to Tornado, and upstream changes to the
conda-forge CPython recipe itself to increase the default file descriptor limit
on Windows up from 512.&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/2017/01/03/dask-0.13.0.md&lt;/span&gt;, line 74)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="dataframe-to-array-conversions"&gt;
&lt;h1&gt;Dataframe to Array Conversions&lt;/h1&gt;
&lt;p&gt;You can now convert Dask dataframes into Dask arrays. This is mostly to
support efforts of groups building statistics and machine learning
applications, where this conversion is common. For example you can load a
terabyte of CSV or Parquet data, do some basic filtering and manipulation, and
then convert to a Dask array to do more numeric work like SVDs, regressions,
etc..&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;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;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="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://...&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Read raw data&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;values&lt;/span&gt;                 &lt;span class="c1"&gt;# Convert to dask.array&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;span class="c1"&gt;# Perform serious numerics&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This should help machine learning and statistics developers generally, as many
of the more sophisticated algorithms can be more easily implemented with the
Dask array model than can be done with distributed dataframes. This change was
done specifically to support the nascent third-party
&lt;a class="reference external" href="https://github.com/moody-marlin/dask-glm"&gt;dask-glm&lt;/a&gt; project by &lt;a class="reference external" href="https://github.com/moody-marlin/"&gt;Chris
White&lt;/a&gt; at Capital One.&lt;/p&gt;
&lt;p&gt;Previously this was hard because Dask.array wanted to know the size of every
chunk of data, which Dask dataframes can’t provide (because, for example, it is
impossible to lazily tell how many rows are in a CSV file without actually
looking through it). Now that Dask.arrays have relaxed this requirement they
can also support other unknown shape operations, like indexing an array with
another 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;y&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="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;&amp;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;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/2017/01/03/dask-0.13.0.md&lt;/span&gt;, line 112)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="parquet-support"&gt;
&lt;h1&gt;Parquet Support&lt;/h1&gt;
&lt;p&gt;Dask.dataframe now supports &lt;a class="reference external" href="https://parquet.apache.org/"&gt;Parquet&lt;/a&gt;, a columnar
binary store for tabular data commonly used in distributed clusters and the
Hadoop ecosystem.&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;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;myfile.parquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;                 &lt;span class="c1"&gt;# Read from Parquet&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;myfile.parquet&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;compression&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;snappy&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Write to Parquet&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is done through the new
&lt;a class="reference external" href="http://fastparquet.readthedocs.io/en/latest/"&gt;fastparquet&lt;/a&gt; library, a
Numba-accelerated version of the Pure Python
&lt;a class="reference external" href="https://github.com/jcrobak/parquet-python"&gt;parquet-python&lt;/a&gt;. Fastparquet was
built and is maintained by &lt;a class="reference external" href="https://github.com/martindurant"&gt;Martin Durant&lt;/a&gt;.
It’s also exciting to see the
&lt;a class="reference external" href="https://github.com/apache/parquet-cpp"&gt;Parquet-cpp&lt;/a&gt; project gain Python
support through &lt;a class="reference external" href="http://pyarrow.readthedocs.io/en/latest/"&gt;Arrow&lt;/a&gt; and work by
&lt;a class="reference external" href="http://wesmckinney.com/"&gt;Wes McKinney&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/xhochy"&gt;Uwe
Korn&lt;/a&gt;. Parquet has gone from inaccessible in Python
to having multiple competing implementations, which is a wonderful and exciting
change for the “Big Data” Python ecosystem.&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/2017/01/03/dask-0.13.0.md&lt;/span&gt;, line 139)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="scheduling-performance-and-worker-rewrite"&gt;
&lt;h1&gt;Scheduling Performance and Worker Rewrite&lt;/h1&gt;
&lt;p&gt;The internals of the distributed scheduler and workers are significantly
modified. Users shouldn’t experience much change here except for general
performance enhancement, more upcoming features, and much deeper visual
diagnostics through Bokeh servers.&lt;/p&gt;
&lt;p&gt;We’ve pushed some of the scheduling logic from the scheduler onto the workers. This lets us do two things:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;We keep a much larger backlog of tasks on the workers. This allows workers
to optimize and saturate their hardware more effectively. As a result,
complex computations end up being significantly faster.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We can more easily deliver on a rising number of requests for complex
scheduling features. For example, GPU users will be happy to learn that
you can now specify abstract resource constraints like “this task requires
a GPU” and “this worker has four GPUs” and the scheduler and workers will
allocate tasks accordingly. This is just one example of a feature that was
easy to implement after the scheduler/worker redesign and is now available.&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/2017/01/03/dask-0.13.0.md&lt;/span&gt;, line 158)&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="pervasive-visual-diagnostics-with-embedded-bokeh-servers"&gt;
&lt;h1&gt;Pervasive Visual Diagnostics with Embedded Bokeh Servers&lt;/h1&gt;
&lt;p&gt;While optimizing scheduler performance we built several new visual diagnostics
using &lt;a class="reference external" href="http://bokeh.pydata.org/en/latest/"&gt;Bokeh&lt;/a&gt;. There is now a Bokeh Server
running &lt;em&gt;within&lt;/em&gt; the scheduler and within every worker.&lt;/p&gt;
&lt;p&gt;Current Dask.distributed users will be familiar with the current diagnostic
dashboards:&lt;/p&gt;
&lt;a href="https://raw.githubusercontent.com/dask/dask-org/master/images/daskboard.gif"&gt;
    &lt;img src="https://raw.githubusercontent.com/dask/dask-org/master/images/daskboard.gif"
         alt="Dask Bokeh Plots"
         width="60%"&gt;&lt;/a&gt;
&lt;p&gt;These plots provide intuition about the state of the cluster and the
computations currently in flight. These dashboards are generally well loved.&lt;/p&gt;
&lt;p&gt;There are now many more of these, though more focused on internal state and
timings that will be of interest to developers and power users than to a
typical users. Here are a couple of the new pages (of which there are seven)
that show various timings and counters of various parts of the worker and
scheduler internals.&lt;/p&gt;
&lt;a href="/images/bokeh-counters.gif"&gt;
  &lt;img src="/images/bokeh-counters.gif"
         alt="Dask Bokeh counters page"
                width="100%"&gt;&lt;/a&gt;
&lt;p&gt;The previous Bokeh dashboards were served from a separate process that queried
the scheduler periodically (every 100ms). Now there are new Bokeh servers
within every worker and a new Bokeh server &lt;em&gt;within&lt;/em&gt; the scheduler process
itself rather than in a separate process. Because these servers are embedded
they have direct access to the state of the scheduler and workers which
significantly reduces barriers for us to build out new visuals. However, this
also adds some load to the scheduler, which can often be compute bound. These
pages are available at new ports, 8788 for the scheduler and 8789 for the
worker by default.&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/2017/01/03/dask-0.13.0.md&lt;/span&gt;, line 196)&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-serialization"&gt;
&lt;h1&gt;Custom Serialization&lt;/h1&gt;
&lt;p&gt;This is actually a change that occurred in the last release, but I haven’t
written about it and it’s important, so I’m including it here.&lt;/p&gt;
&lt;p&gt;Previously inter-worker communication of data was accomplished with
Pickle/Cloudpickle and optional generic compression like LZ4/Snappy. This was
robust and worked mostly fine, but left out some exotic data types and did not
provide optimal performance.&lt;/p&gt;
&lt;p&gt;Now we can serialize different types with special consideration. This allows
special types, like NumPy arrays, to pass through without unnecessary memory
copies and also allows us to use more exotic data-type specific compression
techniques like &lt;a class="reference external" href="http://www.blosc.org/"&gt;Blosc&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;It also allows Dask to serialize some previously unserializable types. In
particular this was intended to solve the Dask.array climate science
community’s concern about HDF5 and NetCDF files which (correctly) are
unpicklable and so restricted to single-machine use.&lt;/p&gt;
&lt;p&gt;This is also the first step towards two frequently requested features (neither
of these exist yet):&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Better support for GPU-GPU specific serialization options. We are now a
large step closer to generalizing away our assumption of TCP Sockets as
the universal communication mechanism.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Passing data between workers of different runtime languages. By embracing
other protocols than Pickle we begin to allow for the communication of data
between workers of different software environments.&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/2017/01/03/dask-0.13.0.md&lt;/span&gt;, line 226)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="what-s-next"&gt;
&lt;h1&gt;What’s Next&lt;/h1&gt;
&lt;p&gt;So what should we expect to see in the future for Dask?&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Communication&lt;/strong&gt;: Now that workers are more fully saturated we’ve found
that communication issues are arising more frequently as bottlenecks. This
might be because everything else is nearing optimal or it might be
because of the increased contention in the workers now that they are idle
less often. Many of our new diagnostics are intended to measure components
of the communication pipeline.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Third Party Tools&lt;/strong&gt;: We’re seeing a nice growth of utilities like
&lt;a class="reference external" href="https://github.com/dask/dask-drmaa"&gt;dask-drmaa&lt;/a&gt; for launching clusters on
DRMAA job schedulers (SGE, SLURM, LSF) and
&lt;a class="reference external" href="https://github.com/moody-marlin/dask-glm"&gt;dask-glm&lt;/a&gt; for solvers for GLM-like
machine-learning algorithms. I hope that external projects like these
become the main focus of Dask development going forward as Dask penetrates
new domains.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Blogging&lt;/strong&gt;: I’ll be launching a few fun blog posts throughout the next
couple of weeks. Stay tuned.&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/2017/01/03/dask-0.13.0.md&lt;/span&gt;, line 246)&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="learn-more"&gt;
&lt;h1&gt;Learn More&lt;/h1&gt;
&lt;p&gt;You can install or upgrade using Conda or Pip&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 -c conda-forge dask distributed
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or&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[complete] distributed --upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You can learn more about Dask and its distributed scheduler at these websites:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask.pydata.org/en/latest/"&gt;Dask Documentation&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/"&gt;Distributed Scheduler Documentation&lt;/a&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/2017/01/03/dask-0.13.0.md&lt;/span&gt;, line 261)&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;Since the last main release the following developers have contributed to the
core Dask repostiory (parallel algorithms, arrays, dataframes, etc..)&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Alexander C. Booth&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Christopher Prohm&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Frederic Laliberte&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Mike Graham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Rolando (Max) Espinoza&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Sinhrks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stuart Archibald&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;And the following developers have contributed to the Dask/distributed
repository (distributed scheduling, network communication, etc..)&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Antoine Pitrou&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;jakirkham&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jeff Reback&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;rbubley&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Stephan Hoyer&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;strets123&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Travis E. Oliphant&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2017/01/03/dask-0.13.0/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2017-01-03T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/12/24/dask-dev-4/</id>
    <title>Dask Development Log</title>
    <updated>2016-12-24T00: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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;To increase transparency I’m blogging weekly about the work done on Dask and
related projects during the previous week. This log covers work done between
2016-12-11 and 2016-12-18. Nothing here is ready for production. This
blogpost is written in haste, so refined polish should not be expected.&lt;/p&gt;
&lt;p&gt;Themes of last week:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Cleanup of load balancing&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Found cause of worker lag&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Initial Spark/Dask Dataframe comparisons&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Benchmarks with asv&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/2016/12/24/dask-dev-4.md&lt;/span&gt;, line 25)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="load-balancing-cleanup"&gt;

&lt;p&gt;The last two weeks saw several disruptive changes to the scheduler and workers.
This resulted in an overall performance degradation on messy workloads when
compared to the most recent release, which stopped bleeding-edge users from
using recent dev builds. This has been resolved, and bleeding-edge git-master
is back up to the old speed and then some.&lt;/p&gt;
&lt;p&gt;As a visual aid, this is what bad (or in this case random) load balancing looks
like:&lt;/p&gt;
&lt;a href="/images/bad-work-stealing.png"&gt;
    &lt;img src="/images/bad-work-stealing.png"
         alt="bad work stealing"
         width="70%"&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/2016/12/24/dask-dev-4.md&lt;/span&gt;, line 41)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="identified-and-removed-worker-lag"&gt;
&lt;h1&gt;Identified and removed worker lag&lt;/h1&gt;
&lt;p&gt;For a while there have been significant gaps of 100ms or more between successive
tasks in workers, especially when using Pandas. This was particularly odd
because the workers had lots of backed up work to keep them busy (thanks to the
nice load balancing from before). The culprit here was the calculation of the
size of the intermediate on object dtype dataframes.&lt;/p&gt;
&lt;a href="/images/task-stream-pandas-lag.png"&gt;
    &lt;img src="/images/task-stream-pandas-lag.png"
         alt="lag between tasks"
         width="70%"&gt;&lt;/a&gt;
&lt;p&gt;Explaining this in greater depth, recall that to schedule intelligently, the
workers calculate the size in bytes of every intermediate result they produce.
Often this is quite fast, for example for numpy arrays we can just multiply the
number of elements by the dtype itemsize. However for object dtype arrays or
dataframes (which are commonly used for text) it can take a long while to
calculate an accurate result here. Now we no longer calculuate an accurate
result, but instead take a fairly pessimistic guess. The gaps between tasks
shrink considerably.&lt;/p&gt;
&lt;a href="/images/task-stream-pandas-no-lag.png"&gt;
    &lt;img src="/images/task-stream-pandas-no-lag.png"
         alt="no lag between tasks"
         width="40%"&gt;&lt;/a&gt;
&lt;a href="/images/task-stream-pandas-no-lag-zoomed.png"&gt;
    &lt;img src="/images/task-stream-pandas-no-lag-zoomed.png"
         alt="no lag between tasks zoomed"
         width="40%"&gt;&lt;/a&gt;
&lt;p&gt;Although there is still a significant bit of lag around 10ms long between tasks
on these workloads (see zoomed version on the right). On other workloads we’re
able to get inter-task lag down to the tens of microseconds scale. While 10ms
may not sound like a long time, when we perform very many very short tasks this
can quickly become a bottleneck.&lt;/p&gt;
&lt;p&gt;Anyway, this change reduced shuffle overhead by a factor of two. Things are
starting to look pretty snappy for many-small-task workloads.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2016/12/24/dask-dev-4.md&lt;/span&gt;, line 81)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="initial-spark-dask-dataframe-comparisons"&gt;
&lt;h1&gt;Initial Spark/Dask Dataframe Comparisons&lt;/h1&gt;
&lt;p&gt;I would like to run a small benchmark comparing Dask and Spark DataFrames. I
spent a bit of the last couple of days using Spark locally on the NYC Taxi data
and futzing with cluster deployment tools to set up Spark clusters on EC2 for
basic benchmarking. I ran across
&lt;a class="reference external" href="https://github.com/nchammas/flintrock"&gt;flintrock&lt;/a&gt;, which has been highly
recommended to me a few times.&lt;/p&gt;
&lt;p&gt;I’ve been thinking about how to do benchmarks in an unbiased way. Comparative
benchmarks are useful to have around to motivate projects to grow and learn
from each other. However in today’s climate where open source software
developers have a vested interest, benchmarks often focus on a projects’
strengths and hide their deficiencies. Even with the best of intentions and
practices, a developer is likely to correct for deficiencies on the fly.
They’re much more able to do this for their own project than for others’.
Benchmarks end up looking more like sales documents than trustworthy research.&lt;/p&gt;
&lt;p&gt;My tentative plan is to reach out to a few Spark devs and see if we can
collaborate on a problem set and hardware before running computations and
comparing results.&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/2016/12/24/dask-dev-4.md&lt;/span&gt;, line 103)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="benchmarks-with-airspeed-velocity"&gt;
&lt;h1&gt;Benchmarks with airspeed velocity&lt;/h1&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/postelrich"&gt;Rich Postelnik&lt;/a&gt; is building on work from
&lt;a class="reference external" href="https://github.com/TomAugspurger"&gt;Tom Augspurger&lt;/a&gt; to build out benchmarks for
Dask using &lt;a class="reference external" href="https://github.com/spacetelescope/asv"&gt;airspeed velocity&lt;/a&gt; at
&lt;a class="reference external" href="https://github.com/dask/dask-benchmarks"&gt;dask-benchmarks&lt;/a&gt;. Building out
benchmarks is a great way to get involved if anyone is interested.&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/2016/12/24/dask-dev-4.md&lt;/span&gt;, line 111)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="pre-pre-release"&gt;
&lt;h1&gt;Pre-pre-release&lt;/h1&gt;
&lt;p&gt;I intend to publish a pre-release for a 0.X.0 version bump of dask/dask and
dask/distributed sometime next week.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/12/24/dask-dev-4/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2016-12-24T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/12/18/dask-dev-3/</id>
    <title>Dask Development Log</title>
    <updated>2016-12-18T00: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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;To increase transparency I’m blogging weekly about the work done on Dask and
related projects during the previous week. This log covers work done between
2016-12-11 and 2016-12-18. Nothing here is ready for production. This
blogpost is written in haste, so refined polish should not be expected.&lt;/p&gt;
&lt;p&gt;Themes of last week:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Benchmarking new scheduler and worker on larger systems&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kubernetes and Google Container Engine&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Fastparquet on S3&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/2016/12/18/dask-dev-3.md&lt;/span&gt;, line 24)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="rewriting-load-balancing"&gt;

&lt;p&gt;In the last two weeks we rewrote a significant fraction of the worker and
scheduler. This enables future growth, but also resulted in a loss of our load
balancing and work stealing algorithms (the old one no longer made sense in the
context of the new system.) Careful dynamic load balancing is essential to
running atypical workloads (which are surprisingly typical among Dask users) so
rebuilding this has been all-consuming this week for me personally.&lt;/p&gt;
&lt;p&gt;Briefly, Dask initially assigns tasks to workers taking into account the
expected runtime of the task, the size and location of the data that the task
needs, the duration of other tasks on every worker, and where each piece of data
sits on all of the workers. Because the number of tasks can grow into the
millions and the number of workers can grow into the thousands, Dask needs to
figure out a near-optimal placement in near-constant time, which is hard.
Furthermore, after the system runs for a while, uncertainties in our estimates
build, and we need to rebalance work from saturated workers to idle workers
relatively frequently. Load balancing intelligently and responsively is
essential to a satisfying user experience.&lt;/p&gt;
&lt;p&gt;We have a decently strong test suite around these behaviors, but it’s hard to
be comprehensive on performance-based metrics like this, so there has also been
a lot of benchmarking against real systems to identify new failure modes.
We’re doing what we can to create isolated tests for every failure mode that we
find to make future rewrites retain good behavior.&lt;/p&gt;
&lt;p&gt;Generally working on the Dask distributed scheduler has taught me the
brittleness of unit tests. As we have repeatedly rewritten internals while
maintaining the same external API our testing strategy has evolved considerably
away from fine-grained unit tests to a mixture of behavioral integration tests
and a very strict runtime validation system.&lt;/p&gt;
&lt;p&gt;Rebuilding the load balancing algorithms has been high priority for me
personally because these performance issues inhibit current power-users from
using the development version on their problems as effectively as with the
latest release. I’m looking forward to seeing load-balancing humming nicely
again so that users can return to git-master and so that I can return to
handling a broader base of issues. (Sorry to everyone I’ve been ignoring the
last couple of weeks).&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/2016/12/18/dask-dev-3.md&lt;/span&gt;, line 64)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="test-deployments-on-google-container-engine"&gt;
&lt;h1&gt;Test deployments on Google Container Engine&lt;/h1&gt;
&lt;p&gt;I’ve personally started switching over my development cluster from Amazon’s EC2
to Google’s Container Engine. Here are some pro’s and con’s from my particular
perspective. Many of these probably have more to do with how I use each
particular tool rather than intrinsic limitations of the service itself.&lt;/p&gt;
&lt;p&gt;In Google’s Favor&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Native and immediate support for Kubernetes and Docker, the combination of
which allows me to more quickly and dynamically create and scale clusters
for different experiments.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dynamic scaling from a single node to a hundred nodes and back ten minutes
later allows me to more easily run a much larger range of scales.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;I like being charged by the minute rather than by the hour, especially
given the ability to dynamically scale up&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Authentication and billing feel simpler&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In Amazon’s Favor&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;I already have tools to launch Dask on EC2&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;All of my data is on Amazon’s S3&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;I have nice data acquisition tools,
&lt;a class="reference external" href="http://s3fs.readthedocs.io/en/latest/"&gt;s3fs&lt;/a&gt;, for S3 based on boto3.
Google doesn’t seem to have a nice Python 3 library for accessing Google
Cloud Storage :(&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;I’m working from Olivier Grisel’s repository
&lt;a class="reference external" href="https://github.com/ogrisel/docker-distributed"&gt;docker-distributed&lt;/a&gt; although
updating to newer versions and trying to use as few modifications from naive
deployment as possible. My current branch is
&lt;a class="reference external" href="https://github.com/mrocklin/docker-distributed/tree/update"&gt;here&lt;/a&gt;. I hope to
have something more stable for next week.&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/2016/12/18/dask-dev-3.md&lt;/span&gt;, line 98)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="fastparquet-on-s3"&gt;
&lt;h1&gt;Fastparquet on S3&lt;/h1&gt;
&lt;p&gt;We gave fastparquet and Dask.dataframe a spin on some distributed S3 data on
Friday. I was surprised that everything seemed to work out of the box. Martin
Durant, who built both fastparquet and s3fs has done some nice work to make
sure that all of the pieces play nicely together. We ran into some performance
issues pulling bytes from S3 itself. I expect that there will be some tweaking
over the next few weeks.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/12/18/dask-dev-3/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2016-12-18T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/12/12/dask-dev-2/</id>
    <title>Dask Development Log</title>
    <updated>2016-12-12T00: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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;To increase transparency I’m blogging weekly about the work done on Dask and
related projects during the previous week. This log covers work done between
2016-12-05 and 2016-12-12. Nothing here is stable or ready for production.
This blogpost is written in haste, so refined polish should not be expected.&lt;/p&gt;
&lt;p&gt;Themes of last week:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Dask.array without known chunk sizes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Import time&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Fastparquet blogpost and feedback&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scheduler improvements for 1000+ worker clusters&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Channels and inter-client communication&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;New dependencies?&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/2016/12/12/dask-dev-2.md&lt;/span&gt;, line 27)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="dask-array-without-known-chunk-sizes"&gt;

&lt;p&gt;Dask arrays can now work even in situations where we don’t know the exact chunk
size. This is particularly important because it allows us to convert
dask.dataframes to dask.arrays in a standard analysis cycle that includes both
data preparation and statistical or machine learning algorithms.&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;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;values&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_records&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This work was motivated by the work of Christopher White on building scalable
solvers for problems like logistic regression and generalized linear models
over at &lt;a class="reference external" href="https://github.com/moody-marlin/dask-glm"&gt;dask-glm&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;As a pleasant side effect we can now also index dask.arrays with dask.arrays (a
previous limitation)&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;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;&amp;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;and mutate dask.arrays in certain cases with setitem&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;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;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Both of which are frequntly requested.&lt;/p&gt;
&lt;p&gt;However, there are still holes in this implementation and many operations (like
slicing) generally don’t work on arrays without known chunk sizes. We’re
increasing capability here but blurring the lines of what is possible and what
is not possible, which used to be very clear.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="github reference external" href="https://github.com/dask/dask/pull/1838"&gt;dask/dask#1838&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/dask/dask/pull/1840"&gt;dask/dask#1840&lt;/a&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/2016/12/12/dask-dev-2.md&lt;/span&gt;, line 67)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="import-time"&gt;
&lt;h1&gt;Import time&lt;/h1&gt;
&lt;p&gt;Import times had been steadily climbing for a while, rising above one second at
times. These were reduced by Antoine Pitrou down to a more reasonable 300ms.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="github reference external" href="https://github.com/dask/dask/pull/1833"&gt;dask/dask#1833&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/dask/distributed/pull/718"&gt;dask/distributed#718&lt;/a&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/2016/12/12/dask-dev-2.md&lt;/span&gt;, line 75)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="fastparquet-blogpost-and-feedback"&gt;
&lt;h1&gt;FastParquet blogpost and feedback&lt;/h1&gt;
&lt;p&gt;Martin Durant has built a nice Python Parquet library here: &lt;a class="reference external" href="http://fastparquet.readthedocs.io/en/latest/"&gt;http://fastparquet.readthedocs.io/en/latest/&lt;/a&gt;
and released a blogpost about it last week here: &lt;a class="reference external" href="https://www.continuum.io/blog/developer-blog/introducing-fastparquet"&gt;https://www.continuum.io/blog/developer-blog/introducing-fastparquet&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Since then we’ve gotten some good feedback and error reports (non-string column
names etc.) Martin has been optimizing performance and recently adding append
support.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="github reference external" href="https://github.com/dask/fastparquet/pull/39"&gt;dask/fastparquet#39&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/dask/fastparquet/pull/43"&gt;dask/fastparquet#43&lt;/a&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/2016/12/12/dask-dev-2.md&lt;/span&gt;, line 87)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="scheduler-optimizations-for-1000-worker-clusters"&gt;
&lt;h1&gt;Scheduler optimizations for 1000+ worker clusters&lt;/h1&gt;
&lt;p&gt;The recent refactoring of the scheduler and worker exposed new opportunities
for performance and for measurement. One of the 1000+ worker deployments here
in NYC was kind enough to volunteer some compute time to run some experiments.
It was very fun having all of the Dask/Bokeh dashboards up at once (there are
now half a dozen of these things) giving live monitoring information on a
thousand-worker deployment. It’s stunning how clearly performance issues
present themselves when you have the right monitoring system.&lt;/p&gt;
&lt;p&gt;Anyway, this lead to better sequentialization when handling messages, greatly
reduced open file handle requirements, and the use of cytoolz over toolz in a
few critical areas.&lt;/p&gt;
&lt;p&gt;I intend to try this experiment again this week, now with new diagnostics. To
aid in that we’ve made it very easy to turn timings and counters automatically
into live Bokeh plots. It now takes literally one line of code to add a new
plot to these pages (left: scheduler right: worker)&lt;/p&gt;
&lt;a href="/images/bokeh-counters.gif"&gt;
  &lt;img src="/images/bokeh-counters.gif"
       alt="Dask Bokeh counters page"
       width="100%"&gt;&lt;/a&gt;
&lt;p&gt;Already we can see that the time it takes to connect between workers is
absurdly high in the 10ms to 100ms range, highlighting an important performance
flaw.&lt;/p&gt;
&lt;p&gt;This depends on an experimental project,
&lt;a class="reference external" href="https://github.com/jcrist/crick"&gt;crick&lt;/a&gt;, by Jim Crist that provides a fast
T-Digest implemented in C (see also &lt;a class="reference external" href="https://github.com/tdunning/t-digest"&gt;Ted Dunning’s
implementation&lt;/a&gt;.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="github reference external" href="https://github.com/jcrist/crick"&gt;jcrist/crick&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/dask/distributed/pull/738"&gt;dask/distributed#738&lt;/a&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/2016/12/12/dask-dev-2.md&lt;/span&gt;, line 123)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="channels-and-inter-worker-communication"&gt;
&lt;h1&gt;Channels and inter-worker communication&lt;/h1&gt;
&lt;p&gt;I’m starting to experiment with mechanisms for inter-client communication of
futures. This enables both collaborative workflows (two researchers sharing
the same cluster) and also complex workflows in which tasks start other tasks
in a more streaming setting.&lt;/p&gt;
&lt;p&gt;We added a simple mechanism to share a rolling buffer of futures between
clients:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# Client 1&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;scheduler:8786&amp;#39;&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;c&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;channel&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;future&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;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inc&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;x&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="n"&gt;future&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="c1"&gt;# Client 1&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;scheduler:8786&amp;#39;&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;c&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;channel&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;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;next&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;iter&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;p&gt;Additionally, this relatively simple mechanism was built external to the
scheduler and client, establishing a pattern we can repeat in the future for
more complex inter-client communication systems. Generally I’m on the lookout
for other ways to make the system more extensible. This range of extension
requests for the scheduler is somewhat large these days and we’d like to find
ways to keep these expansions maintainable going forward.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="github reference external" href="https://github.com/dask/distributed/pull/729"&gt;dask/distributed#729&lt;/a&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/2016/12/12/dask-dev-2.md&lt;/span&gt;, line 159)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="new-dependency-sorted-collections"&gt;
&lt;h1&gt;New dependency: Sorted collections&lt;/h1&gt;
&lt;p&gt;The scheduler is now using the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sortedcollections&lt;/span&gt;&lt;/code&gt; module, which is based off
of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sortedcontainers&lt;/span&gt;&lt;/code&gt; which is a pure-Python library offering sorted containers
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;SortedList&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;SortedSet&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ValueSortedDict&lt;/span&gt;&lt;/code&gt;, etc. at C-extensions speeds.&lt;/p&gt;
&lt;p&gt;So far I’m pretty sold on these libraries. I encourage other library
maintainers to consider them.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://www.youtube.com/watch?v=7z2Ki44Vs4E"&gt;https://www.youtube.com/watch?v=7z2Ki44Vs4E&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://www.grantjenks.com/docs/sortedcontainers/introduction.html"&gt;http://www.grantjenks.com/docs/sortedcontainers/introduction.html&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://www.grantjenks.com/docs/sortedcollections/"&gt;http://www.grantjenks.com/docs/sortedcollections/&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/12/12/dask-dev-2/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2016-12-12T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/12/05/dask-dev-1/</id>
    <title>Dask Development Log</title>
    <updated>2016-12-05T00: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;
the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
and the Data Driven Discovery Initiative from the &lt;a class="reference external" href="https://www.moore.org/"&gt;Moore
Foundation&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Dask has been active lately due to a combination of increased adoption and
funded feature development by private companies. This increased activity
is great, however an unintended side effect is that I have spent less time
writing about development and engaging with the broader community. To address
this I hope to write one blogpost a week about general development. These will
not be particularly polished, nor will they announce ready-to-use features for
users, however they should increase transparency and hopefully better engage
the developer community.&lt;/p&gt;
&lt;p&gt;So themes of last week&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Embedded Bokeh servers for the Workers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Smarter workers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;An overhauled scheduler that is slightly simpler overall (thanks to the
smarter workers) but with more clever work stealing&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Fastparquet&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/2016/12/05/dask-dev-1.md&lt;/span&gt;, line 30)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="embedded-bokeh-servers-in-dask-workers"&gt;

&lt;p&gt;The distributed scheduler’s &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/web.html"&gt;web diagnostic
page&lt;/a&gt; is one of Dask’s
more flashy features. It shows the passage of every computation on the cluster
in real time. These diagnostics are invaluable for understanding performance
both for users and for core developers.&lt;/p&gt;
&lt;p&gt;I intend to focus on worker performance soon, so I decided to attach a Bokeh
server to every worker to serve web diagnostics about that worker. To make
this easier, I also learned how to &lt;em&gt;embed&lt;/em&gt; Bokeh servers inside of other
Tornado applications. This has reduced the effort to create new visuals and
expose real time information considerably and I can now create a full live
visualization in around 30 minutes. It is now &lt;em&gt;faster&lt;/em&gt; for me to build
a new diagnostic than to grep through logs. It’s pretty useful.&lt;/p&gt;
&lt;p&gt;Here are some screenshots. Nothing too flashy, but this information is highly
valuable to me as I measure bandwidths, delays of various parts of the code,
how workers send data between each other, etc..&lt;/p&gt;
&lt;a href="/images/bokeh-worker-system.png"&gt;
  &lt;img src="/images/bokeh-worker-system.png"
       alt="Dask Bokeh Worker system page"
       width="30%"&gt;&lt;/a&gt;
&lt;a href="/images/bokeh-worker-main.png"&gt;
  &lt;img src="/images/bokeh-worker-main.png"
       alt="Dask Bokeh Worker system page"
       width="30%"&gt;&lt;/a&gt;
&lt;a href="/images/bokeh-worker-crossfilter.png"&gt;
  &lt;img src="/images/bokeh-worker-crossfilter.png"
       alt="Dask Bokeh Worker system page"
       width="30%"&gt;&lt;/a&gt;
&lt;p&gt;To be clear, these diagnostic pages aren’t polished in any way. There’s lots
missing, it’s just what I could get done in a day. Still, everyone running a
Tornado application should have an embedded Bokeh server running. They’re
great for rapidly pushing out visually rich diagnostics.&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/2016/12/05/dask-dev-1.md&lt;/span&gt;, line 68)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="smarter-workers-and-a-simpler-scheduler"&gt;
&lt;h1&gt;Smarter Workers and a Simpler Scheduler&lt;/h1&gt;
&lt;p&gt;Previously the scheduler knew everything and the workers were fairly
simple-minded. Now we’ve moved some of the knowledge and responsibility over
to the workers. Previously the scheduler would give just enough work to the
workers to keep them occupied. This allowed the scheduler to make better
decisions about the state of the entire cluster. By delaying committing a task
to a worker until the last moment we made sure that we were making the right
decision. However, this also means that the worker sometimes has idle
resources, particularly network bandwidth, when it could be speculatively
preparing for future work.&lt;/p&gt;
&lt;p&gt;Now we commit all ready-to-run tasks to a worker immediately and that worker
has the ability to pipeline those tasks as it sees fit. This is better locally
but slightly worse globally. To counter balance this we’re now being much more
aggressive about work stealing and, because the workers have more information,
they can manage some of the administrative costs of works stealing themselves.
Because this isn’t bound to run on just the scheduler we can use more expensive
algorithms than when when did everything on the scheduler.&lt;/p&gt;
&lt;p&gt;There were a few motivations for this change:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Dataframe performance was bound by keeping the worker hardware fully
occupied, which we weren’t doing. I expect that these changes will
eventually yield something like a 30% speedup.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Users on traditional job scheduler machines (SGE, SLURM, TORQUE) and users
who like GPUS, both wanted the ability to tag tasks with specific resource
constraints like “This consumes one GPU” or “This task requires a 5GB of RAM
while running” and ensure that workers would respect those constraints when
running tasks. The old workers weren’t complex enough to reason about these
constraints. With the new workers, adding this feature was trivial.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;By moving logic from the scheduler to the worker we’ve actually made them
both easier to reason about. This should lower barriers for contributors
to get into the core project.&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/2016/12/05/dask-dev-1.md&lt;/span&gt;, line 103)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="dataframe-algorithms"&gt;
&lt;h1&gt;Dataframe algorithms&lt;/h1&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/1807"&gt;Approximate nunique&lt;/a&gt; and
multiple-output-partition groupbys landed in master last week. These arose
because some power-users had very large dataframes that weree running into
scalability limits. Thanks to Mike Graham for the approximate nunique
algorithm. This has also pushed &lt;a class="reference external" href="https://github.com/pandas-dev/pandas/pull/14729"&gt;hashing
changes&lt;/a&gt; upstream to Pandas.&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/2016/12/05/dask-dev-1.md&lt;/span&gt;, line 112)&lt;/p&gt;
&lt;p&gt;Document headings start at H3, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="fast-parquet"&gt;
&lt;h1&gt;Fast Parquet&lt;/h1&gt;
&lt;p&gt;Martin Durant has been working on a Parquet reader/writer for Python using
Numba. It’s pretty slick. He’s been using it on internal Continuum projects
for a little while and has seen both good performance and a very Pythonic
experience for what was previously a format that was pretty inaccessible.&lt;/p&gt;
&lt;p&gt;He’s planning to write about this in the near future so I won’t steal his
thunder. Here is a link to the documentation:
&lt;a class="reference external" href="https://fastparquet.readthedocs.io/en/latest/"&gt;fastparquet.readthedocs.io&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/12/05/dask-dev-1/"/>
    <summary>This work is supported by Continuum Analytics
the XDATA Program
and the Data Driven Discovery Initiative from the Moore
Foundation</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2016-12-05T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/09/22/cluster-deployments/</id>
    <title>Dask Cluster Deployments</title>
    <updated>2016-09-22T00: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;All code in this post is experimental. It should not be relied upon. For
people looking to deploy dask.distributed on a cluster please refer instead to
the &lt;a class="reference external" href="https://distributed.readthedocs.org"&gt;documentation&lt;/a&gt; instead.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Dask is deployed today on the following systems in the wild:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;SGE&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;SLURM,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Torque&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Condor&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;LSF&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Mesos&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Marathon&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Kubernetes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;SSH and custom scripts&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;… there may be more. This is what I know of first-hand.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;These systems provide users access to cluster resources and ensure that
many distributed services / users play nicely together. They’re essential for
any modern cluster deployment.&lt;/p&gt;
&lt;p&gt;The people deploying Dask on these cluster resource managers are power-users;
they know how their resource managers work and they read the &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/setup.html"&gt;documentation on
how to setup Dask
clusters&lt;/a&gt;. Generally
these users are pretty happy; however we should reduce this barrier so that
non-power-users with access to a cluster resource manager can use Dask on their
cluster just as easily.&lt;/p&gt;
&lt;p&gt;Unfortunately, there are a few challenges:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Several cluster resource managers exist, each with significant adoption.
Finite developer time stops us from supporting all of them.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Policies for scaling out vary widely.
For example we might want a fixed number of workers, or we might want
workers that scale out based on current use. Different groups will want
different solutions.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Individual cluster deployments are highly configurable. Dask needs to get
out of the way quickly and let existing technologies configure themselves.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This post talks about some of these issues. It does not contain a definitive
solution.&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/2016/09/22/cluster-deployments.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 id="example-kubernetes"&gt;

&lt;p&gt;For example, both &lt;a class="reference external" href="http://ogrisel.com/"&gt;Olivier Griesl&lt;/a&gt; (INRIA, scikit-learn)
and &lt;a class="reference external" href="https://github.com/timodonnell"&gt;Tim O’Donnell&lt;/a&gt; (Mount Sinai, Hammer lab)
publish instructions on how to deploy Dask.distributed on
&lt;a class="reference external" href="http://kubernetes.io/"&gt;Kubernetes&lt;/a&gt;.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/ogrisel/docker-distributed"&gt;Olivier’s repository&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/hammerlab/dask-distributed-on-kubernetes/"&gt;Tim’s repository&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;These instructions are well organized. They include Dockerfiles, published
images, Kubernetes config files, and instructions on how to interact with cloud
providers’ infrastructure. Olivier and Tim both obviously know what they’re
doing and care about helping others to do the same.&lt;/p&gt;
&lt;p&gt;Tim (who came second) wasn’t aware of Olivier’s solution and wrote up his own.
Tim was capable of doing this but many beginners wouldn’t be.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;One solution&lt;/strong&gt; would be to include a prominent registry of solutions like
these within Dask documentation so that people can find quality references to
use as starting points. I’ve started a list of resources here:
&lt;a class="reference external" href="https://github.com/dask/distributed/pull/547"&gt;dask/distributed #547&lt;/a&gt; comments
pointing to other resources would be most welcome..&lt;/p&gt;
&lt;p&gt;However, even if Tim did find Olivier’s solution I suspect he would still need
to change it. Tim has different software and scalability needs than Olivier.
This raises the question of &lt;em&gt;“What should Dask provide and what should it leave
to administrators?”&lt;/em&gt; It may be that the &lt;em&gt;best&lt;/em&gt; we can do is to support
copy-paste-edit workflows.&lt;/p&gt;
&lt;p&gt;What is Dask-specific, resource-manager specific, and what needs to be
configured by hand each 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/2016/09/22/cluster-deployments.md&lt;/span&gt;, line 88)&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="adaptive-deployments"&gt;
&lt;h1&gt;Adaptive Deployments&lt;/h1&gt;
&lt;p&gt;In order to explore this topic of separable solutions I built a small adaptive
deployment system for Dask.distributed on
&lt;a class="reference external" href="https://mesosphere.github.io/marathon/"&gt;Marathon&lt;/a&gt;, an orchestration platform
on top of Mesos.&lt;/p&gt;
&lt;p&gt;This solution does two things:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;It scales a Dask cluster dynamically based on the current use. If there
are more tasks in the scheduler then it asks for more workers.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It deploys those workers using Marathon.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;To encourage replication, these two different aspects are solved in two different pieces of code with a clean API boundary.&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;A backend-agnostic piece for adaptivity that says when to scale workers up
and how to scale them down safely&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A Marathon-specific piece that deploys or destroys dask-workers using the
Marathon HTTP API&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This combines a policy, &lt;em&gt;adaptive scaling&lt;/em&gt;, with a backend, &lt;em&gt;Marathon&lt;/em&gt; such
that either can be replaced easily. For example we could replace the adaptive
policy with a fixed one to always keep N workers online, or we could replace
Marathon with Kubernetes or Yarn.&lt;/p&gt;
&lt;p&gt;My hope is that this demonstration encourages others to develop third party
packages. The rest of this post will be about diving into this particular
solution.&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/2016/09/22/cluster-deployments.md&lt;/span&gt;, line 117)&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="adaptivity"&gt;
&lt;h1&gt;Adaptivity&lt;/h1&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed.deploy.Adaptive&lt;/span&gt;&lt;/code&gt; class wraps around a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Scheduler&lt;/span&gt;&lt;/code&gt; and
determines when we should scale up and by how many nodes, and when we should
scale down specifying which idle workers to release.&lt;/p&gt;
&lt;p&gt;The current policy is fairly straightforward:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;If there are unassigned tasks or any stealable tasks and no idle workers,
or if the average memory use is over 50%, then increase the number of
workers by a fixed factor (defaults to two).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;If there are idle workers and the average memory use is below 50% then
reclaim the idle workers with the least data on them (after moving data to
nearby workers) until we’re near 50%&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Think this policy could be improved or have other thoughts? Great. It was
easy to implement and entirely separable from the main code so you should be
able to edit it easily or create your own. The current implementation is about
80 lines
(&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/distributed/deploy/adaptive.py"&gt;source&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;However, this &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Adaptive&lt;/span&gt;&lt;/code&gt; class doesn’t actually know how to perform the
scaling. Instead it depends on being handed a separate object, with two
methods, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scale_up&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scale_down&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="k"&gt;class&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nc"&gt;MyCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;object&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;scale_up&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n&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;&lt;/span&gt;
&lt;span class="sd"&gt;        Bring the total count of workers up to ``n``&lt;/span&gt;

&lt;span class="sd"&gt;        This function/coroutine should bring the total number of workers up to&lt;/span&gt;
&lt;span class="sd"&gt;        the number ``n``.&lt;/span&gt;
&lt;span class="sd"&gt;        &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
        &lt;span class="k"&gt;raise&lt;/span&gt; &lt;span class="ne"&gt;NotImplementedError&lt;/span&gt;&lt;span class="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;scale_down&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;workers&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;&lt;/span&gt;
&lt;span class="sd"&gt;        Remove ``workers`` from the cluster&lt;/span&gt;

&lt;span class="sd"&gt;        Given a list of worker addresses this function should remove those&lt;/span&gt;
&lt;span class="sd"&gt;        workers from the cluster.&lt;/span&gt;
&lt;span class="sd"&gt;        &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
        &lt;span class="k"&gt;raise&lt;/span&gt; &lt;span class="ne"&gt;NotImplementedError&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This cluster object contains the backend-specific bits of &lt;em&gt;how&lt;/em&gt; to scale up and
down, but none of the adaptive logic of &lt;em&gt;when&lt;/em&gt; to scale up and down. The
single-machine
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/local-cluster.html"&gt;LocalCluster&lt;/a&gt;
object serves as reference implementation.&lt;/p&gt;
&lt;p&gt;So we combine this adaptive scheme with a deployment scheme. We’ll use a tiny
Dask-Marathon deployment library available
&lt;a class="reference external" href="https://github.com/mrocklin/dask-marathon"&gt;here&lt;/a&gt;&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask_marathon&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;MarathonCluster&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;Scheduler&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;distributed.deploy&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;Adaptive&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;mc&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;MarathonCluster&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;cpus&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;mem&lt;/span&gt;&lt;span class="o"&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;docker_image&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;mrocklin/dask-distributed&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;ac&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Adaptive&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;mc&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This combines a policy, Adaptive, with a deployment scheme, Marathon in a
composable way. The Adaptive cluster watches the scheduler and calls the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scale_up/down&lt;/span&gt;&lt;/code&gt; methods on the MarathonCluster as necessary.&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/2016/09/22/cluster-deployments.md&lt;/span&gt;, line 188)&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="marathon-code"&gt;
&lt;h1&gt;Marathon code&lt;/h1&gt;
&lt;p&gt;Because we’ve isolated all of the “when” logic to the Adaptive code, the
Marathon specific code is blissfully short and specific. We include a slightly
simplified version below. There is a fair amount of Marathon-specific setup in
the constructor and then simple scale_up/down methods below:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;marathon&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;MarathonClient&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;MarathonApp&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;marathon.models.container&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;MarathonContainer&lt;/span&gt;


&lt;span class="k"&gt;class&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nc"&gt;MarathonCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;object&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="fm"&gt;__init__&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                 &lt;span class="n"&gt;executable&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;dask-worker&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                 &lt;span class="n"&gt;docker_image&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;mrocklin/dask-distributed&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                 &lt;span class="n"&gt;marathon_address&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;http://localhost:8080&amp;#39;&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="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cpus&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;mem&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;4000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;scheduler&lt;/span&gt;

        &lt;span class="c1"&gt;# Create Marathon App to run dask-worker&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;executable&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;address&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="s1"&gt;&amp;#39;--nthreads&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="nb"&gt;str&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cpus&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
            &lt;span class="s1"&gt;&amp;#39;--name&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;$MESOS_TASK_ID&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="c1"&gt;# use Mesos task ID as worker name&lt;/span&gt;
            &lt;span class="s1"&gt;&amp;#39;--worker-port&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;$PORT_WORKER&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="s1"&gt;&amp;#39;--nanny-port&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;$PORT_NANNY&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="s1"&gt;&amp;#39;--http-port&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;$PORT_HTTP&amp;#39;&lt;/span&gt;
        &lt;span class="p"&gt;]&lt;/span&gt;

        &lt;span class="n"&gt;ports&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;port&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="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;tcp&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                  &lt;span class="s1"&gt;&amp;#39;name&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;name&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;name&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;worker&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;nanny&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;http&amp;#39;&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;extend&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;--memory-limit&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                     &lt;span class="nb"&gt;str&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;mem&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="mf"&gt;0.6&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="mf"&gt;1e6&lt;/span&gt;&lt;span class="p"&gt;))])&lt;/span&gt;

        &lt;span class="n"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;cmd&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; &amp;#39;&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;join&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;args&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;container&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;MarathonContainer&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;image&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;docker_image&lt;/span&gt;&lt;span class="p"&gt;})&lt;/span&gt;

        &lt;span class="n"&gt;app&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;MarathonApp&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;instances&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;container&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;container&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                          &lt;span class="n"&gt;port_definitions&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;ports&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                          &lt;span class="n"&gt;cpus&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;cpus&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;mem&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;mem&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

        &lt;span class="c1"&gt;# Connect and register app&lt;/span&gt;
        &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;MarathonClient&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;marathon_address&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;app&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;create_app&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;name&lt;/span&gt; &lt;span class="ow"&gt;or&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;dask-&lt;/span&gt;&lt;span class="si"&gt;%s&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;uuid&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;uuid4&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;app&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;scale_up&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;instances&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scale_app&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;app&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;id&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;instances&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;instances&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;scale_down&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;workers&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;w&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
            &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;kill_task&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;app&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;id&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                                  &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;worker_info&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;w&lt;/span&gt;&lt;span class="p"&gt;][&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;name&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                                  &lt;span class="n"&gt;scale&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This isn’t trivial, you need to know about Marathon for this to make sense, but
fortunately you don’t need to know much else. My hope is that people familiar
with other cluster resource managers will be able to write similar objects and
will publish them as third party libraries as I have with this Marathon
solution here:
&lt;a class="github reference external" href="https://github.com/mrocklin/dask-marathon"&gt;mrocklin/dask-marathon&lt;/a&gt;
(thanks goes to Ben Zaitlen for setting up a great testing harness for this and
getting everything started.)&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/2016/09/22/cluster-deployments.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="adaptive-policies"&gt;
&lt;h1&gt;Adaptive Policies&lt;/h1&gt;
&lt;p&gt;Similarly, we can design new policies for deployment. You can read more about
the policies for the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Adaptive&lt;/span&gt;&lt;/code&gt; class in the
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/adaptive.html"&gt;documentation&lt;/a&gt; or
the
&lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/distributed/deploy/adaptive.py"&gt;source&lt;/a&gt;
(about eighty lines long). I encourage people to implement and use other
policies and contribute back those policies that are useful in practice.&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/2016/09/22/cluster-deployments.md&lt;/span&gt;, line 268)&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;We laid out a problem&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;em&gt;How does a distributed system support a variety of cluster resource managers
and a variety of scheduling policies while remaining sensible?&lt;/em&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We proposed two solutions:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;em&gt;Maintain a registry of links to solutions, supporting copy-paste-edit practices&lt;/em&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;em&gt;Develop an API boundary that encourages separable development of third party libraries.&lt;/em&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;It’s not clear that either solution is sufficient, or that the current
implementation of either solution is any good. This is is an important problem
though as Dask.distributed is, today, still mostly used by super-users. I
would like to engage community creativity here as we search for a good
solution.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/09/22/cluster-deployments/"/>
    <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="scipy" label="scipy"/>
    <published>2016-09-22T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/09/13/dask-and-celery/</id>
    <title>Dask and Celery</title>
    <updated>2016-09-13T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;This post compares two Python distributed task processing systems,
Dask.distributed and Celery.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Disclaimer: technical comparisons are hard to do well. I am biased towards
Dask and ignorant of correct Celery practices. Please keep this in mind.
Critical feedback by Celery experts is welcome.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="http://www.celeryproject.org/"&gt;Celery&lt;/a&gt; is a distributed task queue built in
Python and heavily used by the Python community for task-based workloads.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="http://dask.pydata.org/en/latest/"&gt;Dask&lt;/a&gt; is a parallel computing library
popular within the PyData community that has grown a fairly sophisticated
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/"&gt;distributed task scheduler&lt;/a&gt;.
This post explores if Dask.distributed can be useful for Celery-style problems.&lt;/p&gt;
&lt;p&gt;Comparing technical projects is hard both because authors have bias, and also
because the scope of each project can be quite large. This allows authors to
gravitate towards the features that show off our strengths. Fortunately &lt;a class="reference external" href="https://github.com/dask/dask/issues/1537"&gt;a
Celery user asked how Dask compares on
Github&lt;/a&gt; and they listed a few
concrete features:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Handling multiple queues&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Canvas (celery’s workflow)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Rate limiting&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Retrying&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;These provide an opportunity to explore the Dask/Celery comparision from the
bias of a Celery user rather than from the bias of a Dask developer.&lt;/p&gt;
&lt;p&gt;In this post I’ll point out a couple of large differences, then go through the
Celery hello world in both projects, and then address how these requested
features are implemented or not within Dask. This anecdotal comparison over a
few features should give us a general comparison.&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/2016/09/13/dask-and-celery.md&lt;/span&gt;, line 43)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="biggest-difference-worker-state-and-communication"&gt;

&lt;p&gt;First, the biggest difference (from my perspective) is that Dask workers hold
onto intermediate results and communicate data between each other while in
Celery all results flow back to a central authority. This difference was
critical when building out large parallel arrays and dataframes (Dask’s
original purpose) where we needed to engage our worker processes’ memory and
inter-worker communication bandwidths. Computational systems like Dask do
this, more data-engineering systems like Celery/Airflow/Luigi don’t. This is
the main reason why Dask wasn’t built on top of Celery/Airflow/Luigi originally.&lt;/p&gt;
&lt;p&gt;That’s not a knock against Celery/Airflow/Luigi by any means. Typically
they’re used in settings where this doesn’t matter and they’ve focused their
energies on several features that Dask similarly doesn’t care about or do well.
Tasks usually read data from some globally accessible store like a database or
S3 and either return very small results, or place larger results back in the
global store.&lt;/p&gt;
&lt;p&gt;The question on my mind is now is &lt;em&gt;Can Dask be a useful solution in more
traditional loose task scheduling problems where projects like Celery are
typically used? What are the benefits and drawbacks?&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/2016/09/13/dask-and-celery.md&lt;/span&gt;, line 65)&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="hello-world"&gt;
&lt;h1&gt;Hello World&lt;/h1&gt;
&lt;p&gt;To start we do the &lt;a class="reference external" href="http://docs.celeryproject.org/en/latest/getting-started/first-steps-with-celery.html"&gt;First steps with
Celery&lt;/a&gt;
walk-through both in Celery and Dask and compare the two:&lt;/p&gt;
&lt;section id="celery"&gt;
&lt;h2&gt;Celery&lt;/h2&gt;
&lt;p&gt;I follow the Celery quickstart, using Redis instead of RabbitMQ because it’s
what I happen to have handy.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="c1"&gt;# tasks.py&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;celery&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;Celery&lt;/span&gt;

&lt;span class="n"&gt;app&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Celery&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;tasks&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;broker&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;redis://localhost&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;backend&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;redis&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="nd"&gt;@app&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;task&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;x&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&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;y&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;redis-server
celery -A tasks worker --loglevel=info
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;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;tasks&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;add&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="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt; &lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delay&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;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;get&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# submit and retrieve roundtrip&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;60&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;8&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;68&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;567&lt;/span&gt; &lt;span class="n"&gt;ms&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;2&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="mi"&gt;2&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="o"&gt;%%&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;
&lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;futures&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delay&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="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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;results&lt;/span&gt; &lt;span class="o"&gt;=&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;get&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;f&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;futures&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;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;888&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;72&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;960&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="mf"&gt;1.7&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="dask"&gt;
&lt;h2&gt;Dask&lt;/h2&gt;
&lt;p&gt;We do the same workload with dask.distributed’s concurrent.futures interface,
using the default single-machine deployment.&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;distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;

&lt;span class="n"&gt;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;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="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;operator&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;add&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="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;time&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;submit&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="mi"&gt;1&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;result&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;20&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt; &lt;span class="n"&gt;ns&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;20&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="mf"&gt;20.7&lt;/span&gt; &lt;span class="n"&gt;ms&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;4&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="mi"&gt;2&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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;futures&lt;/span&gt; &lt;span class="o"&gt;=&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;submit&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="n"&gt;i&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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;results&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;gather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&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;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;328&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;12&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;340&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;369&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="comparison"&gt;
&lt;h2&gt;Comparison&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Functions&lt;/strong&gt;: In Celery you register computations ahead of time on the
server. This is good if you know what you want to run ahead of time (such
as is often the case in data engineering workloads) and don’t want the
security risk of allowing users to run arbitrary code on your cluster. It’s
less pleasant on users who want to experiment. In Dask we choose the
functions to run on the user side, not on the server side. This ends up
being pretty critical in data exploration but may be a hinderance in more
conservative/secure compute settings.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Setup&lt;/strong&gt;: In Celery we depend on other widely deployed systems like
RabbitMQ or Redis. Dask depends on lower-level Torando TCP IOStreams and
Dask’s own custom routing logic. This makes Dask trivial to set up, but
also probably less durable. Redis and RabbitMQ have both solved lots of
problems that come up in the wild and leaning on them inspires confidence.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Performance&lt;/strong&gt;: They both operate with sub-second latencies and
millisecond-ish overheads. Dask is marginally lower-overhead but for data
engineering workloads differences at this level are rarely significant.
Dask is an order of magnitude lower-latency, which might be a big deal
depending on your application. For example if you’re firing off tasks from
a user clicking a button on a website 20ms is generally within interactive
budget while 500ms feels a bit slower.&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/2016/09/13/dask-and-celery.md&lt;/span&gt;, line 155)&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="simple-dependencies"&gt;
&lt;h1&gt;Simple Dependencies&lt;/h1&gt;
&lt;p&gt;The question asked about
&lt;a class="reference external" href="http://docs.celeryproject.org/en/master/userguide/canvas.html"&gt;Canvas&lt;/a&gt;,
Celery’s dependency management system.&lt;/p&gt;
&lt;p&gt;Often tasks depend on the results of other tasks. Both systems have ways to
help users express these dependencies.&lt;/p&gt;
&lt;section id="id1"&gt;
&lt;h2&gt;Celery&lt;/h2&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/2016/09/13/dask-and-celery.md&lt;/span&gt;, line 164); &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: “celery”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;apply_async&lt;/span&gt;&lt;/code&gt; method has a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;link=&lt;/span&gt;&lt;/code&gt; parameter that can be used to call tasks
after other tasks have run. For example we can compute &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(1&lt;/span&gt; &lt;span class="pre"&gt;+&lt;/span&gt; &lt;span class="pre"&gt;2)&lt;/span&gt; &lt;span class="pre"&gt;+&lt;/span&gt; &lt;span class="pre"&gt;3&lt;/span&gt;&lt;/code&gt; in Celery
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="n"&gt;add&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;apply_async&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="n"&gt;link&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;s&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="dask-distributed"&gt;
&lt;h2&gt;Dask.distributed&lt;/h2&gt;
&lt;p&gt;With the Dask concurrent.futures API, futures can be used within submit calls
and dependencies are implicit.&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;x&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;submit&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="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="n"&gt;y&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;submit&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="n"&gt;x&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We could also use the &lt;a class="reference external" href="http://dask.pydata.org/en/latest/delayed.html"&gt;dask.delayed&lt;/a&gt; decorator to annotate arbitrary functions and then use normal-ish 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="nd"&gt;@dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&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;x&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&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;y&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;add&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="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;add&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;3&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="id2"&gt;
&lt;h2&gt;Comparison&lt;/h2&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/2016/09/13/dask-and-celery.md&lt;/span&gt;, line 196); &lt;em&gt;&lt;a href="#id2"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “comparison”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;I prefer the Dask solution, but that’s subjective.&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/2016/09/13/dask-and-celery.md&lt;/span&gt;, line 200)&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="complex-dependencies"&gt;
&lt;h1&gt;Complex Dependencies&lt;/h1&gt;
&lt;section id="id3"&gt;
&lt;h2&gt;Celery&lt;/h2&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/2016/09/13/dask-and-celery.md&lt;/span&gt;, line 202); &lt;em&gt;&lt;a href="#id3"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “celery”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Celery includes a rich vocabulary of terms to connect tasks in more complex
ways including &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;groups&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chains&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chords&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;maps&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;starmaps&lt;/span&gt;&lt;/code&gt;, etc.. More
detail here in their docs for Canvas, the system they use to construct complex
workflows: &lt;a class="reference external" href="http://docs.celeryproject.org/en/master/userguide/canvas.html"&gt;http://docs.celeryproject.org/en/master/userguide/canvas.html&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;For example here we chord many adds and then follow them with a sum.&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;tasks&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;add&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;tsum&lt;/span&gt;  &lt;span class="c1"&gt;# I had to add a sum method to tasks.py&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;celery&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;chord&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="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt; &lt;span class="n"&gt;chord&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="o"&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;i&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;100&lt;/span&gt;&lt;span class="p"&gt;))(&lt;/span&gt;&lt;span class="n"&gt;tsum&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;())&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;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;172&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;12&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;184&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="mf"&gt;1.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;3&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="mi"&gt;9900&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="id4"&gt;
&lt;h2&gt;Dask&lt;/h2&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/2016/09/13/dask-and-celery.md&lt;/span&gt;, line 222); &lt;em&gt;&lt;a href="#id4"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “dask”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Dask’s trick of allowing futures in submit calls actually goes pretty far.
Dask doesn’t really need any additional primitives. It can do all of the
patterns expressed in Canvas fairly naturally with normal submit calls.&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;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;time&lt;/span&gt;
&lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;futures&lt;/span&gt; &lt;span class="o"&gt;=&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;submit&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="n"&gt;i&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;100&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;total&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;submit&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="n"&gt;futures&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;total&lt;/span&gt;&lt;span class="o"&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="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;52&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt; &lt;span class="n"&gt;ns&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;52&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="mf"&gt;60.8&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Or with &lt;a class="reference external" href="http://dask.pydata.org/en/latest/delayed.html"&gt;Dask.delayed&lt;/a&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;futures&lt;/span&gt; &lt;span class="o"&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="n"&gt;i&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;100&lt;/span&gt;&lt;span class="p"&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;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;delayed&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="n"&gt;futures&lt;/span&gt;&lt;span class="p"&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;result&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/2016/09/13/dask-and-celery.md&lt;/span&gt;, line 246)&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="multiple-queues"&gt;
&lt;h1&gt;Multiple Queues&lt;/h1&gt;
&lt;p&gt;In Celery there is a notion of queues to which tasks can be submitted and that
workers can subscribe. An example use case is having “high priority” workers
that only process “high priority” tasks. Every worker can subscribe to
the high-priority queue but certain workers will subscribe to that queue
exclusively:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;celery&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;A&lt;/span&gt; &lt;span class="n"&gt;my&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;project&lt;/span&gt; &lt;span class="n"&gt;worker&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;Q&lt;/span&gt; &lt;span class="n"&gt;high&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;priority&lt;/span&gt;  &lt;span class="c1"&gt;# only subscribe to high priority&lt;/span&gt;
&lt;span class="n"&gt;celery&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;A&lt;/span&gt; &lt;span class="n"&gt;my&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;project&lt;/span&gt; &lt;span class="n"&gt;worker&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;Q&lt;/span&gt; &lt;span class="n"&gt;celery&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;&lt;span class="n"&gt;high&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;priority&lt;/span&gt;  &lt;span class="c1"&gt;# subscribe to both&lt;/span&gt;
&lt;span class="n"&gt;celery&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;A&lt;/span&gt; &lt;span class="n"&gt;my&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;project&lt;/span&gt; &lt;span class="n"&gt;worker&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;Q&lt;/span&gt; &lt;span class="n"&gt;celery&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;&lt;span class="n"&gt;high&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;priority&lt;/span&gt;
&lt;span class="n"&gt;celery&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;A&lt;/span&gt; &lt;span class="n"&gt;my&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;project&lt;/span&gt; &lt;span class="n"&gt;worker&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;Q&lt;/span&gt; &lt;span class="n"&gt;celery&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;&lt;span class="n"&gt;high&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;priority&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is like the TSA pre-check line or the express lane in the grocery store.&lt;/p&gt;
&lt;p&gt;Dask has a couple of topics that are similar or could fit this need in a pinch, but nothing that is strictly analogous.&lt;/p&gt;
&lt;p&gt;First, for the common case above, tasks have priorities. These are typically
set by the scheduler to minimize memory use but can be overridden directly by
users to give certain tasks precedence over others.&lt;/p&gt;
&lt;p&gt;Second, you can restrict tasks to run on subsets of workers. This was
originally designed for data-local storage systems like the Hadoop FileSystem
(HDFS) or clusters with special hardware like GPUs but can be used in the
queues case as well. It’s not quite the same abstraction but could be used to
achieve the same results in a pinch. For each task you can &lt;em&gt;restrict&lt;/em&gt; the pool
of workers on which it can run.&lt;/p&gt;
&lt;p&gt;The relevant docs for this are here:
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/locality.html#user-control"&gt;http://distributed.readthedocs.io/en/latest/locality.html#user-control&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/2016/09/13/dask-and-celery.md&lt;/span&gt;, line 279)&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="retrying-tasks"&gt;
&lt;h1&gt;Retrying Tasks&lt;/h1&gt;
&lt;p&gt;Celery allows tasks to retry themselves on a failure.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@app&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;task&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;bind&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;send_twitter_status&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;oauth&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;tweet&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;try&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;twitter&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Twitter&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;oauth&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;twitter&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;update_status&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;tweet&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="k"&gt;except&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Twitter&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;FailWhaleError&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Twitter&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;LoginError&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;exc&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="k"&gt;raise&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;retry&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;exc&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;exc&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# Example from http://docs.celeryproject.org/en/latest/userguide/tasks.html#retrying&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Sadly Dask currently has no support for this (see &lt;a class="reference external" href="https://github.com/dask/distributed/issues/391"&gt;open
issue&lt;/a&gt;). All functions are
considered pure and final. If a task errs the exception is considered to be
the true result. This could change though; it has been requested a couple of
times now.&lt;/p&gt;
&lt;p&gt;Until then users need to implement retry logic within the function (which isn’t
a terrible idea regardless).&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@app&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;task&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;bind&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;send_twitter_status&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;oauth&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;tweet&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n_retries&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;5&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="n"&gt;n_retries&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="k"&gt;try&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
            &lt;span class="n"&gt;twitter&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Twitter&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;oauth&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
            &lt;span class="n"&gt;twitter&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;update_status&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;tweet&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
            &lt;span class="k"&gt;return&lt;/span&gt;
        &lt;span class="k"&gt;except&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Twitter&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;FailWhaleError&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Twitter&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;LoginError&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;exc&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
            &lt;span class="k"&gt;pass&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/2016/09/13/dask-and-celery.md&lt;/span&gt;, line 316)&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="rate-limiting"&gt;
&lt;h1&gt;Rate Limiting&lt;/h1&gt;
&lt;p&gt;Celery lets you specify rate limits on tasks, presumably to help you avoid
getting blocked from hammering external APIs&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@app&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;task&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;rate_limit&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;1000/h&amp;#39;&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;query_external_api&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="o"&gt;...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Dask definitely has nothing built in for this, nor is it planned. However,
this could be done externally to Dask fairly easily. For example, Dask
supports mapping functions over arbitrary Python Queues. If you send in a
queue then all current and future elements in that queue will be mapped over.
You could easily handle rate limiting in Pure Python on the client side by
rate limiting your input queues. The low latency and overhead of Dask makes it
fairly easy to manage logic like this on the client-side. It’s not as
convenient, but it’s still straightforward.&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;queue&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;Queue&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;q&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Queue&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;out&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;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;query_external_api&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;q&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;type&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="go"&gt;Queue&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/2016/09/13/dask-and-celery.md&lt;/span&gt;, line 346)&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;Based on this very shallow exploration of Celery, I’ll foolishly claim that
Dask can handle Celery workloads, &lt;em&gt;if you’re not diving into deep API&lt;/em&gt;.
However all of that deep API is actually really important. Celery evolved in
this domain and developed tons of features that solve problems that arise over
and over again. This history saves users an enormous amount of time. Dask
evolved in a very different space and has developed a very different set of
tricks. Many of Dask’s tricks are general enough that they can solve Celery
problems with a small bit of effort, but there’s still that extra step. I’m
seeing people applying that effort to problems now and I think it’ll be
interesting to see what comes out of it.&lt;/p&gt;
&lt;p&gt;Going through the Celery API was a good experience for me personally. I think
that there are some good concepts from Celery that can inform future Dask
development.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/09/13/dask-and-celery/"/>
    <summary>This post compares two Python distributed task processing systems,
Dask.distributed and Celery.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="scipy" label="scipy"/>
    <published>2016-09-13T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/09/12/dask-distributed-release-1.13.0/</id>
    <title>Dask Distributed Release 1.13.0</title>
    <updated>2016-09-12T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;I’m pleased to announce a release of
&lt;a class="reference external" href="http://dask.readthedocs.io/en/latest/"&gt;Dask&lt;/a&gt;’s distributed scheduler,
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/"&gt;dask.distributed&lt;/a&gt;, version
1.13.0.&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 distributed -c conda-forge
or
pip install dask distributed --upgrade
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The last few months have seen a number of important user-facing features:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Executor is renamed to Client&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Workers can spill excess data to disk when they run out of memory&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The Client.compute and Client.persist methods for dealing with dask
collections (like dask.dataframe or dask.delayed) gain the ability to
restrict sub-components of the computation to different parts of the
cluster with a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;workers=&lt;/span&gt;&lt;/code&gt; keyword argument.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;IPython kernels can be deployed on the worker and schedulers for
interactive debugging.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The Bokeh web interface has gained new plots and improve the visual styling
of old ones.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Additionally there are beta features in current development. These features
are available now, but may change without warning in future versions.
Experimentation and feedback by users comfortable with living on the bleeding
edge is most welcome:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Clients can publish named datasets on the scheduler to share between them&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tasks can launch other tasks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Workers can restart themselves in new software environments provided by the
user&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;There have also been significant internal changes. Other than increased
performance these changes should not be directly apparent.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The scheduler was refactored to a more state-machine like architecture.
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/scheduling-state.html"&gt;Doc page&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Short-lived connections are now managed by a connection pool&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Work stealing has changed and grown more responsive:
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/work-stealing.html"&gt;Doc page&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;General resilience improvements&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The rest of this post will contain very brief explanations of the topics above.
Some of these topics may become blogposts of their own at some point. Until
then I encourage people to look at the &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest"&gt;distributed scheduler’s
documentation&lt;/a&gt; which is separate
from &lt;a class="reference external" href="http://dask.readthedocs.io/en/latest/"&gt;dask’s normal documentation&lt;/a&gt; and
so may contain new information for some readers (Google Analytics reports about
5-10x the readership on
&lt;a class="reference external" href="http://dask.readthedocs.org"&gt;http://dask.readthedocs.org&lt;/a&gt; than on
&lt;a class="reference external" href="http://distributed.readthedocs.org"&gt;http://distributed.readthedocs.org&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2016/09/12/dask-distributed-release-1.13.0.md&lt;/span&gt;, line 60)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="major-changes-and-features"&gt;

&lt;section id="rename-executor-to-client"&gt;
&lt;h2&gt;Rename Executor to Client&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/api.html"&gt;http://distributed.readthedocs.io/en/latest/api.html&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;The term &lt;em&gt;Executor&lt;/em&gt; was originally chosen to coincide with the
&lt;a class="reference external" href="https://docs.python.org/3/library/concurrent.futures.html"&gt;concurrent.futures&lt;/a&gt;
Executor interface, which is what defines the behavior for the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.submit&lt;/span&gt;&lt;/code&gt;,
&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;.result&lt;/span&gt;&lt;/code&gt; methods and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Future&lt;/span&gt;&lt;/code&gt; object used as the primary interface.&lt;/p&gt;
&lt;p&gt;Unfortunately, this is the same term used by projects like Spark and Mesos for
“the low-level thing that executes tasks on each of the workers” causing
significant confusion when communicating with other communities or for
transitioning users.&lt;/p&gt;
&lt;p&gt;In response we rename &lt;em&gt;Executor&lt;/em&gt; to a somewhat more generic term, &lt;em&gt;Client&lt;/em&gt; to
designate its role as &lt;em&gt;the thing users interact with to control their
computations&lt;/em&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;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;Executor&lt;/span&gt;  &lt;span class="c1"&gt;# Old&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;e&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Executor&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;                    &lt;span class="c1"&gt;# Old&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;Client&lt;/span&gt;    &lt;span class="c1"&gt;# New&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;Client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;                      &lt;span class="c1"&gt;# New&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Executor remains an alias for Client and will continue to be valid for some
time, but there may be some backwards incompatible changes for internal use of
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;executor=&lt;/span&gt;&lt;/code&gt; keywords within methods. Newer examples and materials will all use
the term &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="workers-spill-excess-data-to-disk"&gt;
&lt;h2&gt;Workers Spill Excess Data to Disk&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/worker.html#spill-excess-data-to-disk"&gt;http://distributed.readthedocs.io/en/latest/worker.html#spill-excess-data-to-disk&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;When workers get close to running out of memory they can send excess data to
disk. This is not on by default and instead requires adding the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;--memory-limit=auto&lt;/span&gt;&lt;/code&gt; option to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-worker&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;worker&lt;/span&gt; &lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;8786&lt;/span&gt;                      &lt;span class="c1"&gt;# Old&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;worker&lt;/span&gt; &lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;8786&lt;/span&gt; &lt;span class="o"&gt;--&lt;/span&gt;&lt;span class="n"&gt;memory&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;limit&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;auto&lt;/span&gt;  &lt;span class="c1"&gt;# New&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This will eventually become the default (and is now when using
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/local-cluster.html"&gt;LocalCluster&lt;/a&gt;)
but we’d like to see how things progress and phase it in slowly.&lt;/p&gt;
&lt;p&gt;Generally this feature should improve robustness and allow the solution of
larger problems on smaller clusters, although with a performance cost. Dask’s
policies to reduce memory use through clever scheduling remain in place, so in
the common case you should never need this feature, but it’s nice to have as a
failsafe.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="enable-restriction-of-valid-workers-for-compute-and-persist-methods"&gt;
&lt;h2&gt;Enable restriction of valid workers for compute and persist methods&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/locality.html#user-control"&gt;http://distributed.readthedocs.io/en/latest/locality.html#user-control&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Expert users of the distributed scheduler will be aware of the ability to
restrict certain tasks to run only on certain computers. This tends to be
useful when dealing with GPUs or with special databases or instruments only
available on some machines.&lt;/p&gt;
&lt;p&gt;Previously this option was available only on the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;submit&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map&lt;/span&gt;&lt;/code&gt;, and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scatter&lt;/span&gt;&lt;/code&gt;
methods, forcing people to use the more immedate interface. Now the dask
collection interface functions &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;compute&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;persist&lt;/span&gt;&lt;/code&gt; support this keyword as
well.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="ipython-integration"&gt;
&lt;h2&gt;IPython Integration&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/ipython.html"&gt;http://distributed.readthedocs.io/en/latest/ipython.html&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;You can start IPython kernels on the workers or scheduler and then access them
directly using either IPython magics or the QTConsole. This tends to be
valuable when things go wrong and you want to interactively debug on the worker
nodes themselves.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Start IPython on the Scheduler&lt;/strong&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="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;start_ipython_scheduler&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# Start IPython kernel on the scheduler&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;scheduler&lt;/span&gt; &lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;processing&lt;/span&gt;   &lt;span class="c1"&gt;# Use IPython magics to inspect scheduler&lt;/span&gt;
&lt;span class="go"&gt;{&amp;#39;127.0.0.1:3595&amp;#39;: [&amp;#39;inc-1&amp;#39;, &amp;#39;inc-2&amp;#39;],&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;127.0.0.1:53589&amp;#39;: [&amp;#39;inc-2&amp;#39;, &amp;#39;add-5&amp;#39;]}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Start IPython on the Workers&lt;/strong&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="n"&gt;info&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;start_ipython_workers&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# Start IPython kernels on all workers&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;info&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;[&amp;#39;127.0.0.1:4595&amp;#39;, &amp;#39;127.0.0.1:53589&amp;#39;]&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;remote&lt;/span&gt; &lt;span class="n"&gt;info&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;127.0.0.1:3595&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="n"&gt;worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;active&lt;/span&gt;  &lt;span class="c1"&gt;# Use IPython magics&lt;/span&gt;
&lt;span class="go"&gt;{&amp;#39;inc-1&amp;#39;, &amp;#39;inc-2&amp;#39;}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="bokeh-interface"&gt;
&lt;h2&gt;Bokeh Interface&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/web.html"&gt;http://distributed.readthedocs.io/en/latest/web.html&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;The Bokeh web interface to the cluster continues to evolve both by improving
existing plots and by adding new plots and new pages.&lt;/p&gt;
&lt;p&gt;&lt;img src="https://raw.githubusercontent.com/dask/dask-org/master/images/bokeh-progress-large.gif"
     alt="dask progress bar"
     width="60%"
     align="right"&gt;&lt;/p&gt;
&lt;p&gt;For example the progress bars have become more compact and shrink down
dynamically to respond to addiional bars.&lt;/p&gt;
&lt;p&gt;And we’ve added in extra tables and plots to monitor workers, such as their
memory use and current backlog of 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/2016/09/12/dask-distributed-release-1.13.0.md&lt;/span&gt;, line 176)&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="experimental-features"&gt;
&lt;h1&gt;Experimental Features&lt;/h1&gt;
&lt;p&gt;The features described below are experimental and may change without warning.
Please do not depend on them in stable code.&lt;/p&gt;
&lt;section id="publish-datasets"&gt;
&lt;h2&gt;Publish Datasets&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/publish.html"&gt;http://distributed.readthedocs.io/en/latest/publish.html&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;You can now save collections on the scheduler, allowing you to come back to the
same computations later or allow collaborators to see and work off of your
results. This can be useful in the following cases:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;There is a dataset from which you frequently base all computations, and you
want that dataset always in memory and easy to access without having to
recompute it each time you start work, even if you disconnect.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You want to send results to a colleague working on the same Dask cluster and
have them get immediate access to your computations without having to send
them a script and without them having to repeat the work on the cluster.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;strong&gt;Example: Client One&lt;/strong&gt;&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scheduler-address:8786&amp;#39;&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.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://my-bucket/*.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;df2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;balance&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&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;df2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df2&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;df2&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="n"&gt;name&lt;/span&gt;  &lt;span class="n"&gt;balance&lt;/span&gt;
&lt;span class="mi"&gt;0&lt;/span&gt;    &lt;span class="n"&gt;Alice&lt;/span&gt;     &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;100&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;      &lt;span class="n"&gt;Bob&lt;/span&gt;     &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;200&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;  &lt;span class="n"&gt;Charlie&lt;/span&gt;     &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;300&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;   &lt;span class="n"&gt;Dennis&lt;/span&gt;     &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;400&lt;/span&gt;
&lt;span class="mi"&gt;4&lt;/span&gt;    &lt;span class="n"&gt;Edith&lt;/span&gt;     &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;500&lt;/span&gt;

&lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;publish_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;accounts&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;df2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Example: Client Two&lt;/strong&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;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;scheduler-address:8786&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;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;list_datasets&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;[&amp;#39;accounts&amp;#39;]&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;accounts&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;df&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="go"&gt;      name  balance&lt;/span&gt;
&lt;span class="go"&gt;0    Alice     -100&lt;/span&gt;
&lt;span class="go"&gt;1      Bob     -200&lt;/span&gt;
&lt;span class="go"&gt;2  Charlie     -300&lt;/span&gt;
&lt;span class="go"&gt;3   Dennis     -400&lt;/span&gt;
&lt;span class="go"&gt;4    Edith     -500&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="launch-tasks-from-tasks"&gt;
&lt;h2&gt;Launch Tasks from tasks&lt;/h2&gt;
&lt;p&gt;&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/task-launch.html"&gt;http://distributed.readthedocs.io/en/latest/task-launch.html&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;You can now submit tasks to the cluster that themselves submit more tasks.
This allows the submission of highly dynamic workloads that can shape
themselves depending on future computed values without ever checking back in
with the original client.&lt;/p&gt;
&lt;p&gt;This is accomplished by starting new local &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Client&lt;/span&gt;&lt;/code&gt;s within the task that can
interact with the 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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;func&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="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;local_client&lt;/span&gt;
    &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;local_client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;c2&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;c2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&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;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="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;future&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;submit&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;There are a few straightforward use cases for this, like iterative algorithms
with stoping criteria, but also many novel use cases including streaming
and monitoring systems.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="restart-workers-in-redeployable-python-environments"&gt;
&lt;h2&gt;Restart Workers in Redeployable Python Environments&lt;/h2&gt;
&lt;p&gt;You can now zip up and distribute full Conda environments, and ask
dask-workers to restart themselves, live, in that environment. This involves
the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Create a conda environment locally (or any redeployable directory including
a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;python&lt;/span&gt;&lt;/code&gt; executable)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Zip up that environment and use the existing dask.distributed network
to copy it to all of the workers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Shut down all of the workers and restart them within the new environment&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This helps users to experiment with different software environments with a much
faster turnaround time (typically tens of seconds) than asking IT to install
libraries or building and deploying Docker containers (which is also a fine
solution). Note that they typical solution of uploading individual python
scripts or egg files has been around for a while, &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/api.html#distributed.client.Client.upload_file"&gt;see API docs for
upload_file&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/2016/09/12/dask-distributed-release-1.13.0.md&lt;/span&gt;, line 282)&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="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;Since version 1.12.0 on August 18th the following people have contributed
commits to the &lt;a class="reference external" href="https://github.com/dask/distributed"&gt;dask/distributed repository&lt;/a&gt;&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Dave Hirschfeld&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;dsidi&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Jim Crist&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Joseph Crail&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Loïc Estève&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Martin Durant&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matthew Rocklin&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Min RK&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scott Sievert&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/09/12/dask-distributed-release-1.13.0/"/>
    <summary>I’m pleased to announce a release of
Dask’s distributed scheduler,
dask.distributed, version
1.13.0.</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2016-09-12T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/08/16/dask-for-institutions/</id>
    <title>Dask for Institutions</title>
    <updated>2016-08-16T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;img src="http://dask.readthedocs.io/en/latest/_images/dask_horizontal.svg"
     align="right"
     width="20%"&gt;&lt;/p&gt;
&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;&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/2016/08/16/dask-for-institutions.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;Institutions use software differently than individuals. Over the last few
months I’ve had dozens of conversations about using Dask within larger
organizations like universities, research labs, private companies, and
non-profit learning systems. This post provides a very coarse summary of those
conversations and extracts common questions. I’ll then try to answer those
questions.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: some of this post will be necessarily vague at points. Some companies
prefer privacy. All details here are either in public Dask issues or have come
up with enough institutions (say at least five) that I’m comfortable listing
the problem here.&lt;/em&gt;&lt;/p&gt;
&lt;section id="common-story"&gt;
&lt;h2&gt;Common story&lt;/h2&gt;
&lt;p&gt;Institution X, a university/research lab/company/… has many
scientists/analysts/modelers who develop models and analyze data with Python,
the PyData stack like NumPy/Pandas/SKLearn, and a large amount of custom code.
These models/data sometimes grow to be large enough to need a moderately large
amount of parallel computing.&lt;/p&gt;
&lt;p&gt;Fortunately, Institution X has an in-house cluster acquired for exactly this
purpose of accelerating modeling and analysis of large computations and
datasets. Users can submit jobs to the cluster using a job scheduler like
SGE/LSF/Mesos/Other.&lt;/p&gt;
&lt;p&gt;However the cluster is still under-utilized and the users are still asking for
help with parallel computing. Either users aren’t comfortable using the
SGE/LSF/Mesos/Other interface, it doesn’t support sufficiently complex/dynamic
workloads, or the interaction times aren’t good enough for the interactive use
that users appreciate.&lt;/p&gt;
&lt;p&gt;There was an internal effort to build a more complex/interactive/Pythonic
system on top of SGE/LSF/Mesos/Other but it’s not particularly mature and
definitely isn’t something that Institution X wants to pursue. It turned out
to be a harder problem than expected to design/build/maintain such a system
in-house. They’d love to find an open source solution that was well featured
and maintained by a community.&lt;/p&gt;
&lt;p&gt;The Dask.distributed scheduler looks like it’s 90% of the system that
Institution X needs. However there are a few open questions:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;How do we integrate dask.distributed with the SGE/LSF/Mesos/Other job
scheduler?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How can we grow and shrink the cluster dynamically based on use?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How do users manage software environments on the workers?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How secure is the distributed scheduler?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask is resilient to worker failure, how about scheduler failure?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;What happens if &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-worker&lt;/span&gt;&lt;/code&gt;s are in two different data centers? Can we
scale in an asymmetric way?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How do we handle multiple concurrent users and priorities?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How does this compare with Spark?&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;So for the rest of this post I’m going to answer these questions. As usual,
few of answers will be of the form “Yes Dask can solve all of your problems.”
These are open questions, not the questions that were easy to answer. We’ll
get into what’s possible today and how we might solve these problems in the
future.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="how-do-we-integrate-dask-distributed-with-sge-lsf-mesos-other"&gt;
&lt;h2&gt;How do we integrate dask.distributed with SGE/LSF/Mesos/Other?&lt;/h2&gt;
&lt;p&gt;It’s not difficult to deploy dask.distributed at scale within an existing
cluster using a tool like SGE/LSF/Mesos/Other. In many cases there is already
a researcher within the institution doing this manually by running
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-scheduler&lt;/span&gt;&lt;/code&gt; on some static node in the cluster and launching &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-worker&lt;/span&gt;&lt;/code&gt;
a few hundred times with their job scheduler and a small job script.&lt;/p&gt;
&lt;p&gt;The goal now is how to formalize this process for the individual version of
SGE/LSF/Mesos/Other used within the institution while also developing and
maintaining a standard Pythonic interface so that all of these tools can be
maintained cheaply by Dask developers into the foreseeable future. In some
cases Institution X is happy to pay for the development of a convenient “start
dask on my job scheduler” tool, but they are less excited about paying to
maintain it forever.&lt;/p&gt;
&lt;p&gt;We want Python users to be able to say something 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="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;Executor&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;SGECluster&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;SGECluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nworkers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;200&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;options&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;e&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Executor&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;… and have this same interface be standardized across different job
schedulers.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="how-can-we-grow-and-shrink-the-cluster-dynamically-based-on-use"&gt;
&lt;h2&gt;How can we grow and shrink the cluster dynamically based on use?&lt;/h2&gt;
&lt;p&gt;Alternatively, we could have a single dask.distributed deployment running 24/7
that scales itself up and down dynamically based on current load. Again, this
is entirely possible today if you want to do it manually (you can add and
remove workers on the fly) but we should add some signals to the scheduler like
the following:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;“I’m under duress, please add workers”&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;“I’ve been idling for a while, please reclaim workers”&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;and connect these signals to a manager that talks to the job scheduler. This
removes an element of control from the users and places it in the hands of a
policy that IT can tune to play more nicely with their other services on the
same network.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="how-do-users-manage-software-environments-on-the-workers"&gt;
&lt;h2&gt;How do users manage software environments on the workers?&lt;/h2&gt;
&lt;p&gt;Today Dask assumes that all users and workers share the exact same software
environment. There are some small tools to send updated &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.py&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.egg&lt;/span&gt;&lt;/code&gt; files
to the workers but that’s it.&lt;/p&gt;
&lt;p&gt;Generally Dask trusts that the full software environment will be handled by
something else. This might be a network file system (NFS) mount on traditional
cluster setups, or it might be handled by moving docker or conda environments
around by some other tool like &lt;a class="reference external" href="http://knit.readthedocs.io/en/latest/"&gt;knit&lt;/a&gt;
for YARN deployments or something more custom. For example Continuum &lt;a class="reference external" href="https://docs.continuum.io/anaconda-cluster/"&gt;sells
proprietary software&lt;/a&gt; that
does this.&lt;/p&gt;
&lt;p&gt;Getting the standard software environment setup generally isn’t such a big deal
for institutions. They typically have some system in place to handle this
already. Where things become interesting is when users want to use
drastically different environments from the system environment, like using Python
2 vs Python 3 or installing a bleeding-edge scikit-learn version. They may
also want to change the software environment many times in a single session.&lt;/p&gt;
&lt;p&gt;The best solution I can think of here is to pass around fully downloaded conda
environments using the dask.distributed network (it’s good at moving large
binary blobs throughout the network) and then teaching the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-worker&lt;/span&gt;&lt;/code&gt;s to
bootstrap themselves within this environment. We should be able to tear
everything down and restart things within a small number of seconds. This
requires some work; first to make relocatable conda binaries (which is usually
fine but is not always fool-proof due to links) and then to help the
dask-workers learn to bootstrap themselves.&lt;/p&gt;
&lt;p&gt;Somewhat related, Hussain Sultan of Capital One recently contributed a
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-submit&lt;/span&gt;&lt;/code&gt; command to run scripts on the cluster:
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/submitting-applications.html"&gt;http://distributed.readthedocs.io/en/latest/submitting-applications.html&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="how-secure-is-the-distributed-scheduler"&gt;
&lt;h2&gt;How secure is the distributed scheduler?&lt;/h2&gt;
&lt;p&gt;Dask.distributed is incredibly insecure. It allows anyone with network access
to the scheduler to execute arbitrary code in an unprotected environment. Data
is sent in the clear. Any malicious actor can both steal your secrets and then
cripple your cluster.&lt;/p&gt;
&lt;p&gt;This is entirely the norm however. Security is usually handled by other
services that manage computational frameworks like Dask.&lt;/p&gt;
&lt;p&gt;For example we might rely on Docker to isolate workers from destroying their
surrounding environment and rely on network access controls to protect data
access.&lt;/p&gt;
&lt;p&gt;Because Dask runs on Tornado, a serious networking library and web framework,
there are some things we can do easily like enabling SSL, authentication, etc..
However I hesitate to jump into providing “just a little bit of security”
without going all the way for fear of providing a false sense of security. In
short, I have no plans to work on this without a lot of encouragement. Even
then I would strongly recommend that institutions couple Dask with tools
intended for security. I believe that is common practice for distributed
computational systems generally.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-is-resilient-to-worker-failure-how-about-scheduler-failure"&gt;
&lt;h2&gt;Dask is resilient to worker failure, how about scheduler failure?&lt;/h2&gt;
&lt;p&gt;Workers can come and go. Clients can come and go. The state in the scheduler
is currently irreplaceable and no attempt is made to back it up. There are a
few things you could imagine here:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Backup state and recent events to some persistent storage so that state can
be recovered in case of catastrophic loss&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Have a hot failover node that gets a copy of every action that the
scheduler takes&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Have multiple peer schedulers operate simultaneously in a way that they can
pick up slack from lost peers&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Have clients remember what they have submitted and resubmit when a
scheduler comes back online&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Currently option 4 is currently the most feasible and gets us most of the way
there. However options 2 or 3 would probably be necessary if Dask were to ever
run as critical infrastructure in a giant institution. We’re not there yet.&lt;/p&gt;
&lt;p&gt;As of &lt;a class="reference external" href="https://github.com/dask/distributed/pull/413"&gt;recent work&lt;/a&gt; spurred on by
Stefan van der Walt at UC Berkeley/BIDS the scheduler can now die and come back
and everyone will reconnect. The state for computations in flight is entirely
lost but the computational infrastructure remains intact so that people can
resubmit jobs without significant loss of service.&lt;/p&gt;
&lt;p&gt;Dask has a bit of a harder time with this topic because it offers a persistent
stateful interface. This problem is much easier for distributed database
projects that run ephemeral queries off of persistent storage, return the
results, and then clear out state.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="what-happens-if-dask-workers-are-in-two-different-data-centers-can-we-scale-in-an-asymmetric-way"&gt;
&lt;h2&gt;What happens if dask-workers are in two different data centers? Can we scale in an asymmetric way?&lt;/h2&gt;
&lt;p&gt;The short answer is no. Other than number of cores and available RAM all
workers are considered equal to each other (except when the user &lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/locality.html#user-control"&gt;explicitly
specifies
otherwise&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;However this problem and problems like it have come up a lot lately. Here are a
few examples of similar cases:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Multiple data centers geographically distributed around the country&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Multiple racks within a single data center&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Multiple workers that have GPUs that can move data between each other easily&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Multiple processes on a single machine&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Having some notion of hierarchical worker group membership or inter-worker
preferred relationships is probably inevitable long term. As with all
distributed scheduling questions the hard part isn’t deciding that this is
useful, or even coming up with a sensible design, but rather figuring out how
to make decisions on the sensible design that are foolproof and operate in
constant time. I don’t personally see a good approach here yet but expect one
to arise as more high priority use cases come in.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="how-do-we-handle-multiple-concurrent-users-and-priorities"&gt;
&lt;h2&gt;How do we handle multiple concurrent users and priorities?&lt;/h2&gt;
&lt;p&gt;There are several sub-questions here:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Can multiple users use Dask on my cluster at the same time?&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Yes, either by spinning up separate scheduler/worker sets or by sharing the same
set.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;If they’re sharing the same workers then won’t they clobber each other’s
data?&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This is very unlikely. Dask is careful about naming tasks, so it’s very
unlikely that the two users will submit conflicting computations that compute to
different values but occupy the same key in memory. However if they both submit
computations that overlap somewhat then the scheduler will nicely avoid
recomputation. This can be very nice when you have many people doing slightly
different computations on the same hardware. This works in the same way that
Git works.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;If they’re sharing the same workers then won’t they clobber each other’s
resources?&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Yes, this is definitely possible. If you’re concerned about this then you
should give everyone their own scheduler/workers (which is easy and standard
practice). There is not currently much user management built into Dask.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="how-does-this-compare-with-spark"&gt;
&lt;h2&gt;How does this compare with Spark?&lt;/h2&gt;
&lt;p&gt;At an institutional level Spark seems to primarily target ETL + Database-like
computations. While Dask modules like Dask.bag and Dask.dataframe can happily
play in this space this doesn’t seem to be the focus of recent conversations.&lt;/p&gt;
&lt;p&gt;Recent conversations are almost entirely around supporting interactive custom
parallelism (lots of small tasks with complex dependencies between them) rather
than the big Map-&amp;gt;Filter-&amp;gt;Groupby-&amp;gt;Join abstractions you often find in a
database or Spark. That’s not to say that these operations aren’t hugely
important; there is a lot of selection bias here. The people I talk to are
people for whom Spark/Databases are clearly not an appropriate fit. They are
tackling problems that are way more complex, more heterogeneous, and with a
broader variety of users.&lt;/p&gt;
&lt;p&gt;I usually describe this situation with an analogy comparing “Big data” systems
to human transportation mechanisms in a city. Here we go:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;em&gt;A Database is like a train&lt;/em&gt;: it goes between a set of well defined points
with great efficiency, speed, and predictability. These are popular and
profitable routes that many people travel between (e.g. business analytics).
You do have to get from home to the train station on your own (ETL), but once
you’re in the database/train you’re quite comfortable.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;em&gt;Spark is like an automobile&lt;/em&gt;: it takes you door-to-door from your home to
your destination with a single tool. While this may not be as fast as the train for
the long-distance portion, it can be extremely convenient to do ETL, Database
work, and some machine learning all from the comfort of a single system.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;em&gt;Dask is like an all-terrain-vehicle&lt;/em&gt;: it takes you out of town on rough
ground that hasn’t been properly explored before. This is a good match for
the Python community, which typically does a lot of exploration into new
approaches. You can also drive your ATV around town and you’ll be just fine,
but if you want to do thousands of SQL queries then you should probably
invest in a proper database or in Spark.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Again, there is a lot of selection bias here, if what you want is a database
then you should probably get a database. Dask is not a database.&lt;/p&gt;
&lt;p&gt;This is also wildly over-simplifying things. Databases like Oracle have lots
of ETL and analytics tools, Spark is known to go off road, etc.. I obviously
have a bias towards Dask. You really should never trust an author of a project
to give a fair and unbiased view of the capabilities of the tools in the
surrounding landscape.&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/2016/08/16/dask-for-institutions.md&lt;/span&gt;, line 298)&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="conclusion"&gt;
&lt;h1&gt;Conclusion&lt;/h1&gt;
&lt;p&gt;That’s a rough sketch of current conversations and open problems for “How Dask
might evolve to support institutional use cases.” It’s really quite surprising
just how prevalent this story is among the full spectrum from universities to
hedge funds.&lt;/p&gt;
&lt;p&gt;The problems listed above are by no means halting adoption. I’m not listing
the 100 or so questions that are answered with “yes, that’s already supported
quite well”. Right now I’m seeing Dask being adopted by individuals and small
groups within various institutions. Those individuals and small groups are
pushing that interest up the stack. It’s still several months before any 1000+
person organization adopts Dask as infrastructure, but the speed at which
momentum is building is quite encouraging.&lt;/p&gt;
&lt;p&gt;I’d also like to thank the several nameless people who exercise Dask on various
infrastructures at various scales on interesting problems and have reported
serious bugs. These people don’t show up on the GitHub issue tracker but their
utility in flushing out bugs is invaluable.&lt;/p&gt;
&lt;p&gt;As interest in Dask grows it’s interesting to see how it will evolve.
Culturally Dask has managed to simultaneously cater to both the open science
crowd as well as the private-sector crowd. The project gets both financial
support and open source contributions from each side. So far there hasn’t been
any conflict of interest (everyone is pushing in roughly the same direction)
which has been a really fruitful experience for all involved I think.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/08/16/dask-for-institutions/"/>
    <summary>&lt;img src="http://dask.readthedocs.io/en/latest/_images/dask_horizontal.svg"
     align="right"
     width="20%"&gt;</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2016-08-16T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/07/12/dask-learn-part-1/</id>
    <title>Dask and Scikit-Learn -- Model Parallelism</title>
    <updated>2016-07-12T00:00:00+00:00</updated>
    <author>
      <name>Jim Crist</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This post was written by Jim Crist. The original post lives at
&lt;a class="reference external" href="http://jcrist.github.io/dask-sklearn-part-1.html"&gt;http://jcrist.github.io/dask-sklearn-part-1.html&lt;/a&gt;
(with better styling)&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;This is the first of a series of posts discussing some recent experiments
combining &lt;a class="reference external" href="http://dask.pydata.org/en/latest/"&gt;dask&lt;/a&gt; and
&lt;a class="reference external" href="http://scikit-learn.org/stable/"&gt;scikit-learn&lt;/a&gt;. A small (and extremely alpha)
library has been built up from these experiments, and can be found
&lt;a class="reference external" href="https://github.com/jcrist/dask-learn"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Before we start, I would like to make the following caveats:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;I am not a machine learning expert. Do not consider this a guide on how to do
machine learning, the usage of scikit-learn below is probably naive.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;All of the code discussed here is in flux, and shouldn’t be considered stable
or robust. That said, if you know something about machine learning and want
to help out, I’d be more than happy to receive issues or pull requests :).&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;There are several ways of parallelizing algorithms in machine learning. Some
algorithms can be made to be data-parallel (either across features or across
samples). In this post we’ll look instead at model-parallelism (use same data
across different models), and dive into a daskified implementation of
&lt;a class="reference external" href="http://scikit-learn.org/stable/modules/generated/sklearn.grid_search.GridSearchCV.html"&gt;GridSearchCV&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/2016/07/12/dask-learn-part-1.md&lt;/span&gt;, line 34)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="what-is-grid-search"&gt;

&lt;p&gt;Many machine learning algorithms have &lt;em&gt;hyperparameters&lt;/em&gt; which can be tuned to
improve the performance of the resulting estimator. A &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Hyperparameter_optimization#Grid_search"&gt;grid
search&lt;/a&gt;
is one way of optimizing these parameters — it works by doing a parameter
sweep across a cartesian product of a subset of these parameters (the “grid”),
and then choosing the best resulting estimator. Since this is fitting many
independent estimators across the same set of data, it can be fairly easily
parallelized.&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/2016/07/12/dask-learn-part-1.md&lt;/span&gt;, line 45)&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="grid-search-with-scikit-learn"&gt;
&lt;h1&gt;Grid search with scikit-learn&lt;/h1&gt;
&lt;p&gt;In scikit-learn, a grid search is performed using the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GridSearchCV&lt;/span&gt;&lt;/code&gt; class, and
can (optionally) be automatically parallelized using
&lt;a class="reference external" href="https://pythonhosted.org/joblib/index.html"&gt;joblib&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;This is best illustrated with an example. First we’ll make an example dataset
for doing classification against:&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;sklearn.datasets&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;make_classification&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;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;make_classification&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_samples&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;n_features&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;500&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                           &lt;span class="n"&gt;n_classes&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="n"&gt;n_redundant&lt;/span&gt;&lt;span class="o"&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;random_state&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;42&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;To solve this classification problem, we’ll create a pipeline of a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;PCA&lt;/span&gt;&lt;/code&gt; and a
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;LogisticRegression&lt;/span&gt;&lt;/code&gt;:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;sklearn&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;linear_model&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;decomposition&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;sklearn.pipeline&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;Pipeline&lt;/span&gt;

&lt;span class="n"&gt;logistic&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;linear_model&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;LogisticRegression&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;pca&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;decomposition&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;PCA&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;pipe&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Pipeline&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;steps&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;pca&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;pca&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;logistic&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;logistic&lt;/span&gt;&lt;span class="p"&gt;)])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Both of these classes take several hyperparameters, we’ll do a grid-search
across only a few of 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="c1"&gt;#Parameters of pipelines can be set using ‘__’ separated parameter names:&lt;/span&gt;
&lt;span class="n"&gt;grid&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="n"&gt;pca__n_components&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;50&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;100&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;logistic__C&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mf"&gt;1e-4&lt;/span&gt;&lt;span class="p"&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="mf"&gt;1e4&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
            &lt;span class="n"&gt;logistic__penalty&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;l1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;l2&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;Finally, we can create an instance of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GridSearchCV&lt;/span&gt;&lt;/code&gt;, and perform the grid
search. The parameter &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n_jobs=-1&lt;/span&gt;&lt;/code&gt; tells joblib to use as many processes as I
have cores (8).&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;sklearn.grid_search&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;GridSearchCV&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;estimator&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;GridSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pipe&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;grid&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n_jobs&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="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;estimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 5.3 s, sys: 243 ms, total: 5.54 s&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 21.6 s&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;What happened here was:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;An estimator was created for each parameter combination and test-train set
(scikit-learn’s grid search also does cross validation across 3-folds by
default).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Each estimator was fit on its corresponding set of training data&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Each estimator was then scored on its corresponding set of testing data&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The best set of parameters was chosen based on these scores&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A new estimator was then fit on &lt;em&gt;all&lt;/em&gt; of the data, using the best parameters&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The corresponding best score, parameters, and estimator can all be found as
attributes on the resulting object:&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;estimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_score_&lt;/span&gt;
&lt;span class="go"&gt;0.89290000000000003&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;estimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_params_&lt;/span&gt;
&lt;span class="go"&gt;{&amp;#39;logistic__C&amp;#39;: 0.0001, &amp;#39;logistic__penalty&amp;#39;: &amp;#39;l2&amp;#39;, &amp;#39;pca__n_components&amp;#39;: 50}&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;estimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_estimator_&lt;/span&gt;
&lt;span class="go"&gt;Pipeline(steps=[(&amp;#39;pca&amp;#39;, PCA(copy=True, n_components=50, whiten=False)), (&amp;#39;logistic&amp;#39;, LogisticRegression(C=0.0001, class_weight=None, dual=False,&lt;/span&gt;
&lt;span class="go"&gt;        fit_intercept=True, intercept_scaling=1, max_iter=100,&lt;/span&gt;
&lt;span class="go"&gt;        multi_class=&amp;#39;ovr&amp;#39;, n_jobs=1, penalty=&amp;#39;l2&amp;#39;, random_state=None,&lt;/span&gt;
&lt;span class="go"&gt;        solver=&amp;#39;liblinear&amp;#39;, tol=0.0001, verbose=0, warm_start=False))])&amp;lt;div class=md_output&amp;gt;&lt;/span&gt;

&lt;span class="go"&gt;    {&amp;#39;logistic__C&amp;#39;: 0.0001, &amp;#39;logistic__penalty&amp;#39;: &amp;#39;l2&amp;#39;, &amp;#39;pca__n_components&amp;#39;: 50}&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/2016/07/12/dask-learn-part-1.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="grid-search-with-dask-learn"&gt;
&lt;h1&gt;Grid search with dask-learn&lt;/h1&gt;
&lt;p&gt;Here we’ll repeat the same fit using dask-learn. I’ve tried to match the
scikit-learn interface as much as possible, although not everything is
implemented. Here the only thing that really changes is the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GridSearchCV&lt;/span&gt;&lt;/code&gt;
import. We don’t need the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n_jobs&lt;/span&gt;&lt;/code&gt; keyword, as this will be parallelized across
all cores by default.&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;dklearn.grid_search&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;GridSearchCV&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;DaskGridSearchCV&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;destimator&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;DaskGridSearchCV&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pipe&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;grid&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;time&lt;/span&gt; &lt;span class="n"&gt;destimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="go"&gt;CPU times: user 16.3 s, sys: 1.89 s, total: 18.2 s&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 5.63 s&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;As before, the best score, parameters, and estimator can all be found as
attributes on the object. Here we’ll just show that they’re equivalent:&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;destimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_score_&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="n"&gt;estimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_score_&lt;/span&gt;
&lt;span class="go"&gt;True&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;destimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_params_&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="n"&gt;estimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_params_&lt;/span&gt;
&lt;span class="go"&gt;True&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;destimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_estimator_&lt;/span&gt;
&lt;span class="go"&gt;Pipeline(steps=[(&amp;#39;pca&amp;#39;, PCA(copy=True, n_components=50, whiten=False)), (&amp;#39;logistic&amp;#39;, LogisticRegression(C=0.0001, class_weight=None, dual=False,&lt;/span&gt;
&lt;span class="go"&gt;        fit_intercept=True, intercept_scaling=1, max_iter=100,&lt;/span&gt;
&lt;span class="go"&gt;        multi_class=&amp;#39;ovr&amp;#39;, n_jobs=1, penalty=&amp;#39;l2&amp;#39;, random_state=None,&lt;/span&gt;
&lt;span class="go"&gt;        solver=&amp;#39;liblinear&amp;#39;, tol=0.0001, verbose=0, warm_start=False))])&amp;lt;div class=md_output&amp;gt;&lt;/span&gt;

&lt;span class="go"&gt;    {&amp;#39;logistic__C&amp;#39;: 0.0001, &amp;#39;logistic__penalty&amp;#39;: &amp;#39;l2&amp;#39;, &amp;#39;pca__n_components&amp;#39;: 50}&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/2016/07/12/dask-learn-part-1.md&lt;/span&gt;, line 164)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="why-is-the-dask-version-faster"&gt;
&lt;h1&gt;Why is the dask version faster?&lt;/h1&gt;
&lt;p&gt;If you look at the times above, you’ll note that the dask version was &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;~4X&lt;/span&gt;&lt;/code&gt;
faster than the scikit-learn version. This is not because we have optimized any
of the pieces of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Pipeline&lt;/span&gt;&lt;/code&gt;, or that there’s a significant amount of
overhead to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;joblib&lt;/span&gt;&lt;/code&gt; (on the contrary, joblib does some pretty amazing things,
and I had to construct a contrived example to beat it this badly). The reason
is simply that the dask version is doing less work.&lt;/p&gt;
&lt;p&gt;This maybe best explained in pseudocode. The scikit-learn version of the above
(in serial) looks something like (pseudocode):&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;for&lt;/span&gt; &lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;X_test&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_test&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;cv&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;n&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;grid&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;pca__n_components&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;C&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;grid&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;logistic__C&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;penalty&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;grid&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;logistic__penalty&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt;
                &lt;span class="c1"&gt;# Create and fit a PCA on the input data&lt;/span&gt;
                &lt;span class="n"&gt;pca&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;PCA&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_components&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;n&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
                &lt;span class="c1"&gt;# Transform both the train and test data&lt;/span&gt;
                &lt;span class="n"&gt;X_train2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pca&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;transform&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
                &lt;span class="n"&gt;X_test2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pca&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;transform&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_test&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
                &lt;span class="c1"&gt;# Create and fit a LogisticRegression on the transformed data&lt;/span&gt;
                &lt;span class="n"&gt;logistic&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LogisticRegression&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;C&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;penalty&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;penalty&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
                &lt;span class="n"&gt;logistic&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
                &lt;span class="c1"&gt;# Score the total pipeline&lt;/span&gt;
                &lt;span class="n"&gt;score&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;logistic&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;score&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_test2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_test&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
                &lt;span class="c1"&gt;# Save the score and parameters&lt;/span&gt;
                &lt;span class="n"&gt;scores_and_params&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="n"&gt;score&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;C&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

&lt;span class="c1"&gt;# Find the best set of parameters (for some definition of best)&lt;/span&gt;
&lt;span class="n"&gt;find_best_parameters&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;scores&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is looping through a cartesian product of the cross-validation sets and
all the parameter combinations, and then creating and fitting a new estimator
for each combination. While embarassingly parallel, this can also result in
repeated work, as earlier stages in the pipeline are refit multiple times on
the same parameter + data combinations.&lt;/p&gt;
&lt;p&gt;In contrast, the dask version hashes all inputs (forming a sort of &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Merkle_tree"&gt;Merkle
DAG&lt;/a&gt;), resulting in the intermediate
results being shared. Keeping with the pseudocode above, the dask version might
look like:&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;for&lt;/span&gt; &lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;X_test&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_test&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;cv&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;n&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;grid&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;pca__n_components&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt;
        &lt;span class="c1"&gt;# Create and fit a PCA on the input data&lt;/span&gt;
        &lt;span class="n"&gt;pca&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;PCA&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_components&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;n&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="c1"&gt;# Transform both the train and test data&lt;/span&gt;
        &lt;span class="n"&gt;X_train2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pca&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;transform&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="n"&gt;X_test2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pca&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;transform&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_test&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;C&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;grid&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;logistic__C&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;penalty&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;grid&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;logistic__penalty&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt;
                &lt;span class="c1"&gt;# Create and fit a LogisticRegression on the transformed data&lt;/span&gt;
                &lt;span class="n"&gt;logistic&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LogisticRegression&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;C&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;penalty&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;penalty&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
                &lt;span class="n"&gt;logistic&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_train2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
                &lt;span class="c1"&gt;# Score the total pipeline&lt;/span&gt;
                &lt;span class="n"&gt;score&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;logistic&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;score&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;X_test2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;y_test&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
                &lt;span class="c1"&gt;# Save the score and parameters&lt;/span&gt;
                &lt;span class="n"&gt;scores_and_params&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="n"&gt;score&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;C&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;penalty&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

&lt;span class="c1"&gt;# Find the best set of parameters (for some definition of best)&lt;/span&gt;
&lt;span class="n"&gt;find_best_parameters&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;scores&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This can still be parallelized, but in a less straightforward manner - the
graph is a bit more complicated than just a simple map-reduce pattern.
Thankfully the &lt;a class="reference external" href="http://dask.pydata.org/en/latest/scheduler-overview.html"&gt;dask
schedulers&lt;/a&gt; are well
equipped to handle arbitrary graph topologies. Below is a GIF showing how the
dask scheduler (the threaded scheduler specifically) executed the grid search
performed above. Each rectangle represents data, and each circle represents a
task. Each is categorized by color:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Red means actively taking up resources. These are tasks executing in a thread,
or intermediate results occupying memory&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Blue means finished or released. These are already finished tasks, or data
that’s been released from memory because it’s no longer needed&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;img src="/images/grid_search_schedule.gif" alt="Dask Graph Execution" style="width:100%"&gt;
&lt;p&gt;Looking at the trace, a few things stand out:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;We do a good job sharing intermediates. Each step in a pipeline is only fit
once given the same parameters/data, resulting in some intermediates having
many dependent tasks.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The scheduler does a decent job of quickly finishing up tasks required to
release data. This doesn’t matter as much here (none of the intermediates
take up much memory), but for other workloads this is very useful. See Matt
Rocklin’s &lt;a class="reference internal" href="../2015/01/06/Towards-OOC-Scheduling/"&gt;&lt;span class="doc std std-doc"&gt;excellent blogpost
here&lt;/span&gt;&lt;/a&gt;
for more discussion on this.&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/2016/07/12/dask-learn-part-1.md&lt;/span&gt;, line 261)&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-grid-search-using-dask-learn"&gt;
&lt;h1&gt;Distributed grid search using dask-learn&lt;/h1&gt;
&lt;p&gt;The &lt;a class="reference external" href="http://dask.pydata.org/en/latest/scheduler-overview.html"&gt;schedulers&lt;/a&gt; used
in dask are configurable. The default (used above) is the threaded scheduler,
but we can just as easily swap it out for the distributed scheduler. Here I’ve
just spun up two local workers to demonstrate, but this works equally well
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="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;Executor&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="c1"&gt;# Create an Executor, and set it as the default scheduler&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;exc&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Executor&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;10.0.0.3:8786&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;set_as_default&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;exc&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;Executor: scheduler=&amp;quot;10.0.0.3:8786&amp;quot; processes=2 cores=8&amp;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;destimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 1.69 s, sys: 433 ms, total: 2.12 s&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 7.66 s&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;destimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&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;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 1.69 s, sys: 433 ms, total: 2.12 s&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 7.66 s&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;destimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_score_&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="n"&gt;estimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_score_&lt;/span&gt; &lt;span class="ow"&gt;and&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt; &lt;span class="n"&gt;destimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_params_&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="n"&gt;estimator&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;best_params_&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;True&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Note that this is slightly slower than the threaded execution, so it doesn’t
make sense for this workload, but for others it might.&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/2016/07/12/dask-learn-part-1.md&lt;/span&gt;, line 293)&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-worked-well"&gt;
&lt;h1&gt;What worked well&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The &lt;a class="reference external" href="https://github.com/jcrist/dask-learn/blob/master/dklearn/grid_search.py"&gt;code for doing
this&lt;/a&gt;
is quite short. There’s also an implementation of
&lt;a class="reference external" href="http://scikit-learn.org/stable/modules/generated/sklearn.grid_search.RandomizedSearchCV.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomizedSearchCV&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;,
which is only a few extra lines (hooray for good class hierarchies!).
Instead of working with dask graphs directly, both implementations use
&lt;a class="reference external" href="http://dask.pydata.org/en/latest/delayed.html"&gt;dask.delayed&lt;/a&gt; wherever
possible, which also makes the code easy to read.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Due to the internal hashing used in dask (which is extensible!), duplicate
computations are avoided.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Since the graphs are separated from the scheduler, this works both locally
and distributed with only a few extra lines.&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/2016/07/12/dask-learn-part-1.md&lt;/span&gt;, line 310)&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="caveats-and-what-could-be-better"&gt;
&lt;h1&gt;Caveats and what could be better&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The scikit-learn api makes use of mutation (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;est.fit(X,&lt;/span&gt; &lt;span class="pre"&gt;y)&lt;/span&gt;&lt;/code&gt; mutates &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;est&lt;/span&gt;&lt;/code&gt;),
while dask collections are mostly immutable. After playing around with a few
different ideas, I settled on dask-learn estimators being immutable (except
for grid-search, more on this in a bit). This made the code easier to reason
about, but does mean that you need to do &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;est&lt;/span&gt; &lt;span class="pre"&gt;=&lt;/span&gt; &lt;span class="pre"&gt;est.fit(X,&lt;/span&gt; &lt;span class="pre"&gt;y)&lt;/span&gt;&lt;/code&gt; when working
with dask-learn estimators.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GridSearchCV&lt;/span&gt;&lt;/code&gt; posed a different problem. Due to the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;refit&lt;/span&gt;&lt;/code&gt; keyword, the
implementation can’t be done in a single pass over the data. This means that
we can’t build a single graph describing both the grid search and the refit,
which prevents it from being done lazily. I debated removing this keyword,
but decided in the end to make &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;fit&lt;/span&gt;&lt;/code&gt; execute immediately. This means that
there’s a bit of a disconnect between &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GridSearchCV&lt;/span&gt;&lt;/code&gt; and the other classes in
the library, which I don’t like. On the other hand, it does mean that this
version of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GridSearchCV&lt;/span&gt;&lt;/code&gt; could be a drop-in for the sckit-learn one.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The approach presented here is nice, but is really &lt;em&gt;only beneficial when
there’s duplicate work to be avoided, and that duplicate work is expensive&lt;/em&gt;.
Repeating the above with only a single estimator (instead of a pipeline)
results in identical (or slightly worse) performance than joblib. Similarly,
if the repeated steps are cheap the difference in performance is much smaller
(try the above using
&lt;a class="reference external" href="http://scikit-learn.org/stable/modules/generated/sklearn.feature_selection.SelectKBest.html"&gt;SelectKBest&lt;/a&gt;
instead of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;PCA&lt;/span&gt;&lt;/code&gt;).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The ability to swap easily from local to distributed execution is nice, but
&lt;a class="reference external" href="http://distributed.readthedocs.io/en/latest/joblib.html"&gt;distributed also contains a joblib
frontend&lt;/a&gt; that can
do this just as easily.&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/2016/07/12/dask-learn-part-1.md&lt;/span&gt;, line 342)&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 am not a machine learning expert. Is any of this useful? Do you have
suggestions for improvements (or better yet PRs for improvements :))? Please
feel free to reach out in the comments below, or &lt;a class="reference external" href="https://github.com/jcrist/dask-learn"&gt;on
github&lt;/a&gt;.&lt;/p&gt;
&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&lt;/a&gt; program 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;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/07/12/dask-learn-part-1/"/>
    <summary>This post was written by Jim Crist. The original post lives at
http://jcrist.github.io/dask-sklearn-part-1.html
(with better styling)</summary>
    <category term="Programming" label="Programming"/>
    <category term="dask" label="dask"/>
    <published>2016-07-12T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/04/20/dask-distributed-part-5/</id>
    <title>Ad Hoc Distributed Random Forests</title>
    <updated>2016-04-20T00: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;A screencast version of this post is available here:
&lt;a class="reference external" href="https://www.youtube.com/watch?v=FkPlEqB8AnE"&gt;https://www.youtube.com/watch?v=FkPlEqB8AnE&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2016/04/20/dask-distributed-part-5.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="tl-dr"&gt;

&lt;p&gt;Dask.distributed lets you submit individual tasks to the cluster. We use this
ability combined with Scikit Learn to train and run a distributed random forest
on distributed tabular NYC Taxi data.&lt;/p&gt;
&lt;p&gt;Our machine learning model does not perform well, but we do learn how to
execute ad-hoc computations easily.&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/2016/04/20/dask-distributed-part-5.md&lt;/span&gt;, line 25)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="motivation"&gt;
&lt;h1&gt;Motivation&lt;/h1&gt;
&lt;p&gt;In the past few posts we analyzed data on a cluster with Dask collections:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../2016/02/17/dask-distributed-part1/"&gt;&lt;span class="doc std std-doc"&gt;Dask.bag on JSON records&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../2016/02/22/dask-distributed-part-2/"&gt;&lt;span class="doc std std-doc"&gt;Dask.dataframe on CSV data&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../2016/02/26/dask-distributed-part-3/"&gt;&lt;span class="doc std std-doc"&gt;Dask.array on HDF5 data&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Often our computations don’t fit neatly into the bag, dataframe, or array
abstractions. In these cases we want the flexibility of normal code with for
loops, but still with the computational power of a cluster. With the
dask.distributed task interface, we achieve something close to this.&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/2016/04/20/dask-distributed-part-5.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="application-naive-distributed-random-forest-algorithm"&gt;
&lt;h1&gt;Application: Naive Distributed Random Forest Algorithm&lt;/h1&gt;
&lt;p&gt;As a motivating application we build a random forest algorithm from the ground
up using the single-machine Scikit Learn library, and dask.distributed’s
ability to quickly submit individual tasks to run on the cluster. Our
algorithm will look like the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Pull data from some external source (S3) into several dataframes on the
cluster&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For each dataframe, create and train one &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomForestClassifier&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Scatter single testing dataframe to all machines&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For each &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomForestClassifier&lt;/span&gt;&lt;/code&gt; predict output on test dataframe&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Aggregate independent predictions from each classifier together by a
majority vote. To avoid bringing too much data to any one machine, perform
this majority vote as a tree reduction.&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/2016/04/20/dask-distributed-part-5.md&lt;/span&gt;, line 54)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="data-nyc-taxi-2015"&gt;
&lt;h1&gt;Data: NYC Taxi 2015&lt;/h1&gt;
&lt;p&gt;As in our &lt;a class="reference internal" href="../2016/02/22/dask-distributed-part-2/"&gt;&lt;span class="doc std std-doc"&gt;blogpost on distributed
dataframes&lt;/span&gt;&lt;/a&gt;
we use the data on all NYC Taxi rides in 2015. This is around 20GB on disk and
60GB in RAM.&lt;/p&gt;
&lt;p&gt;We predict the number of passengers in each cab given the other
numeric columns like pickup and destination location, fare breakdown, distance,
etc..&lt;/p&gt;
&lt;p&gt;We do this first on a small bit of data on a single machine and then on the
entire dataset on the cluster. Our cluster is composed of twelve m4.xlarges (4
cores, 15GB RAM each).&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Disclaimer and Spoiler Alert&lt;/em&gt;: I am not an expert in machine learning. Our
algorithm will perform very poorly. If you’re excited about machine
learning you can stop reading here. However, if you’re interested in how to
&lt;em&gt;build&lt;/em&gt; distributed algorithms with Dask then you may want to read on,
especially if you happen to know enough machine learning to improve upon my
naive solution.&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/2016/04/20/dask-distributed-part-5.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="api-submit-map-gather"&gt;
&lt;h1&gt;API: submit, map, gather&lt;/h1&gt;
&lt;p&gt;We use a small number of &lt;a class="reference external" href="http://distributed.readthedocs.org/en/latest/api.html"&gt;dask.distributed
functions&lt;/a&gt; to build our
computation:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;executor&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scatter&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="c1"&gt;# scatter data&lt;/span&gt;
&lt;span class="n"&gt;future&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;executor&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;function&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;args&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# submit single task&lt;/span&gt;
&lt;span class="n"&gt;futures&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;executor&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;function&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sequence&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;           &lt;span class="c1"&gt;# submit many tasks&lt;/span&gt;
&lt;span class="n"&gt;results&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;executor&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;gather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;                   &lt;span class="c1"&gt;# gather results&lt;/span&gt;
&lt;span class="n"&gt;executor&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;replicate&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;number_of_replications&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In particular, functions like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;executor.submit(function,&lt;/span&gt; &lt;span class="pre"&gt;*args)&lt;/span&gt;&lt;/code&gt; let us send
individual functions out to our cluster thousands of times a second. Because
these functions consume their own results we can create complex workflows that
stay entirely on the cluster and trust the distributed scheduler to move data
around intelligently.&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/2016/04/20/dask-distributed-part-5.md&lt;/span&gt;, line 96)&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="load-pandas-from-s3"&gt;
&lt;h1&gt;Load Pandas from S3&lt;/h1&gt;
&lt;p&gt;First we load data from Amazon S3. We use the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;s3.read_csv(...,&lt;/span&gt; &lt;span class="pre"&gt;collection=False)&lt;/span&gt;&lt;/code&gt;
function to load 178 Pandas DataFrames on our cluster from CSV data on S3. We
get back a list of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Future&lt;/span&gt;&lt;/code&gt; objects that refer to these remote dataframes. The
use of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;collection=False&lt;/span&gt;&lt;/code&gt; gives us this list of futures rather than a single
cohesive Dask.dataframe object.&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;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;Executor&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;s3&lt;/span&gt;
&lt;span class="n"&gt;e&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Executor&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;52.91.1.177:8786&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;dfs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;s3&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;dask-data/nyc-taxi/2015&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                  &lt;span class="n"&gt;parse_dates&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;tpep_pickup_datetime&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                               &lt;span class="s1"&gt;&amp;#39;tpep_dropoff_datetime&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                  &lt;span class="n"&gt;collection&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;dfs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dfs&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Each of these is a lightweight &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Future&lt;/span&gt;&lt;/code&gt; pointing to a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pandas.DataFrame&lt;/span&gt;&lt;/code&gt; on the
cluster.&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;dfs&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="go"&gt;[&amp;lt;Future: status: finished, type: DataFrame, key: finalize-a06c3dd25769f434978fa27d5a4cf24b&amp;gt;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;lt;Future: status: finished, type: DataFrame, key: finalize-7dcb27364a8701f45cb02d2fe034728a&amp;gt;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;lt;Future: status: finished, type: DataFrame, key: finalize-b0dfe075000bd59c3a90bfdf89a990da&amp;gt;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;lt;Future: status: finished, type: DataFrame, key: finalize-1c9bb25cefa1b892fac9b48c0aef7e04&amp;gt;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;lt;Future: status: finished, type: DataFrame, key: finalize-c8254256b09ae287badca3cf6d9e3142&amp;gt;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If we’re willing to wait a bit then we can pull data from any future back to
our local process using the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.result()&lt;/span&gt;&lt;/code&gt; method. We don’t want to do this too
much though, data transfer can be expensive and we can’t hold the entire
dataset in the memory of a single machine. Here we just bring back one of the
dataframes:&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dfs&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;result&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;df&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
    &lt;tr style="text-align: right;"&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;VendorID&lt;/th&gt;
      &lt;th&gt;tpep_pickup_datetime&lt;/th&gt;
      &lt;th&gt;tpep_dropoff_datetime&lt;/th&gt;
      &lt;th&gt;passenger_count&lt;/th&gt;
      &lt;th&gt;trip_distance&lt;/th&gt;
      &lt;th&gt;pickup_longitude&lt;/th&gt;
      &lt;th&gt;pickup_latitude&lt;/th&gt;
      &lt;th&gt;RateCodeID&lt;/th&gt;
      &lt;th&gt;store_and_fwd_flag&lt;/th&gt;
      &lt;th&gt;dropoff_longitude&lt;/th&gt;
      &lt;th&gt;dropoff_latitude&lt;/th&gt;
      &lt;th&gt;payment_type&lt;/th&gt;
      &lt;th&gt;fare_amount&lt;/th&gt;
      &lt;th&gt;extra&lt;/th&gt;
      &lt;th&gt;mta_tax&lt;/th&gt;
      &lt;th&gt;tip_amount&lt;/th&gt;
      &lt;th&gt;tolls_amount&lt;/th&gt;
      &lt;th&gt;improvement_surcharge&lt;/th&gt;
      &lt;th&gt;total_amount&lt;/th&gt;
    &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt;0&lt;/th&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;2015-01-15 19:05:39&lt;/td&gt;
      &lt;td&gt;2015-01-15 19:23:42&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;1.59&lt;/td&gt;
      &lt;td&gt;-73.993896&lt;/td&gt;
      &lt;td&gt;40.750111&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.974785&lt;/td&gt;
      &lt;td&gt;40.750618&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;12.0&lt;/td&gt;
      &lt;td&gt;1.0&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;3.25&lt;/td&gt;
      &lt;td&gt;0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;17.05&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;1&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:33:38&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:53:28&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;3.30&lt;/td&gt;
      &lt;td&gt;-74.001648&lt;/td&gt;
      &lt;td&gt;40.724243&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.994415&lt;/td&gt;
      &lt;td&gt;40.759109&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;14.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;2.00&lt;/td&gt;
      &lt;td&gt;0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;17.80&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;2&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:33:38&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:43:41&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;1.80&lt;/td&gt;
      &lt;td&gt;-73.963341&lt;/td&gt;
      &lt;td&gt;40.802788&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.951820&lt;/td&gt;
      &lt;td&gt;40.824413&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;9.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;10.80&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;3&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:33:39&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:35:31&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;0.50&lt;/td&gt;
      &lt;td&gt;-74.009087&lt;/td&gt;
      &lt;td&gt;40.713818&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-74.004326&lt;/td&gt;
      &lt;td&gt;40.719986&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;3.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;4.80&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;4&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:33:39&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:52:58&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;3.00&lt;/td&gt;
      &lt;td&gt;-73.971176&lt;/td&gt;
      &lt;td&gt;40.762428&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-74.004181&lt;/td&gt;
      &lt;td&gt;40.742653&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;15.0&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;16.30&lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&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/2016/04/20/dask-distributed-part-5.md&lt;/span&gt;, line 277)&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="train-on-a-single-machine"&gt;
&lt;h1&gt;Train on a single machine&lt;/h1&gt;
&lt;p&gt;To start lets go through the standard Scikit Learn fit/predict/score cycle with
this small bit of data on a single machine.&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;sklearn.ensemble&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;RandomForestClassifier&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;sklearn.cross_validation&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;train_test_split&lt;/span&gt;

&lt;span class="n"&gt;df_train&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;df_test&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;train_test_split&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;columns&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;trip_distance&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;pickup_longitude&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;pickup_latitude&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
           &lt;span class="s1"&gt;&amp;#39;dropoff_longitude&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;dropoff_latitude&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;payment_type&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
           &lt;span class="s1"&gt;&amp;#39;fare_amount&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;mta_tax&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;tip_amount&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;tolls_amount&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

&lt;span class="n"&gt;est&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;RandomForestClassifier&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_estimators&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;est&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df_train&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;df_train&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This builds a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;RandomForestClassifer&lt;/span&gt;&lt;/code&gt; with four decision trees and then trains
it against the numeric columns in the data, trying to predict the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;passenger_count&lt;/span&gt;&lt;/code&gt; column. It takes around 10 seconds to train on a single
core. We now see how well we do on the holdout testing 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;est&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;score&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df_test&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;df_test&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="go"&gt;0.65808188654721012&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This 65% accuracy is actually pretty poor. About 70% of the rides in NYC have
a single passenger, so the model of “always guess one” would out-perform our
fancy random forest.&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;sklearn.metrics&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;accuracy_score&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;accuracy_score&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df_test&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="gp"&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_like&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df_test&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="go"&gt;0.70669390028780987&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is where my ignorance in machine learning really
kills us. There is likely a simple way to improve this. However, because I’m
more interested in showing how to build distributed computations with Dask than
in actually doing machine learning I’m going to go ahead with this naive
approach. Spoiler alert: we’re going to do a lot of computation and still not
beat the “always guess one” strategy.&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/2016/04/20/dask-distributed-part-5.md&lt;/span&gt;, line 325)&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="fit-across-the-cluster-with-executor-map"&gt;
&lt;h1&gt;Fit across the cluster with executor.map&lt;/h1&gt;
&lt;p&gt;First we build a function that does just what we did before, builds a random
forest and then trains it on a dataframe.&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;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;est&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;RandomForestClassifier&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_estimators&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;est&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;df&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="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;est&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Second we call this function on all of our training dataframes on the cluster
using the standard &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;e.map(function,&lt;/span&gt; &lt;span class="pre"&gt;sequence)&lt;/span&gt;&lt;/code&gt; function. This sends out many
small tasks for the cluster to run. We use all but the last dataframe for
training data and hold out the last dataframe for testing. There are more
principled ways to do this, but again we’re going to charge ahead here.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;train&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dfs&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="p"&gt;]&lt;/span&gt;
&lt;span class="n"&gt;test&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dfs&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="p"&gt;]&lt;/span&gt;

&lt;span class="n"&gt;estimators&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;fit&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;train&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This takes around two minutes to train on all of the 177 dataframes and now we
have 177 independent estimators, each capable of guessing how many passengers a
particular ride had. There is relatively little overhead in this 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/2016/04/20/dask-distributed-part-5.md&lt;/span&gt;, line 354)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="predict-on-testing-data"&gt;
&lt;h1&gt;Predict on testing data&lt;/h1&gt;
&lt;p&gt;Recall that we kept separate a future, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;test&lt;/span&gt;&lt;/code&gt;, that points to a Pandas dataframe on
the cluster that was not used to train any of our 177 estimators. We’re going
to replicate this dataframe across all workers on the cluster and then ask each
estimator to predict the number of passengers for each ride in this 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="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;replicate&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;test&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;n&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;48&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;predict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;est&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="n"&gt;est&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;predict&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;columns&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;

&lt;span class="n"&gt;predictions&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;predict&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;est&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;test&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;est&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;estimators&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here we used the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;executor.submit(function,&lt;/span&gt; &lt;span class="pre"&gt;*args,&lt;/span&gt; &lt;span class="pre"&gt;**kwrags)&lt;/span&gt;&lt;/code&gt; function in a
list comprehension to individually launch many tasks. The scheduler determines
when and where to run these tasks for optimal computation time and minimal data
transfer. As with all functions, this returns futures that we can use to
collect data if we want in the future.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Developers note: we explicitly replicate here in order to take advantage of
efficient tree-broadcasting algorithms. This is purely a performance
consideration, everything would have worked fine without this, but the explicit
broadcast turns a 30s communication+computation into a 2s
communication+computation.&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/2016/04/20/dask-distributed-part-5.md&lt;/span&gt;, line 382)&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="aggregate-predictions-by-majority-vote"&gt;
&lt;h1&gt;Aggregate predictions by majority vote&lt;/h1&gt;
&lt;p&gt;For each estimator we now have an independent prediction of the passenger
counts for all of the rides in our test data. In other words for each ride we
have 177 different opinions on how many passengers were in the cab. By
averaging these opinions together we hope to achieve a more accurate consensus
opinion.&lt;/p&gt;
&lt;p&gt;For example, consider the first four prediction 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="n"&gt;a_few_predictions&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;gather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;predictions&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="c1"&gt;# remote futures -&amp;gt; local arrays&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;a_few_predictions&lt;/span&gt;
&lt;span class="go"&gt;[array([1, 2, 1, ..., 2, 2, 1]),&lt;/span&gt;
&lt;span class="go"&gt; array([1, 1, 1, ..., 1, 1, 1]),&lt;/span&gt;
&lt;span class="go"&gt; array([2, 1, 1, ..., 1, 1, 1]),&lt;/span&gt;
&lt;span class="go"&gt; array([1, 1, 1, ..., 1, 1, 1])]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;For the first ride/column we see that three of the four predictions are for a
single passenger while one prediction disagrees and is for two passengers. We
create a consensus opinion by taking the mode of the stacked 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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;scipy.stats&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;mode&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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;mymode&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&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;array&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;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="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;mode&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="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="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;mymode&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;a_few_predictions&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="mi"&gt;1&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;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&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;1&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And so when we average these four prediction arrays together we see that the
majority opinion of one passenger dominates for all of the six rides visible
here.&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/2016/04/20/dask-distributed-part-5.md&lt;/span&gt;, line 421)&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="tree-reduction"&gt;
&lt;h1&gt;Tree Reduction&lt;/h1&gt;
&lt;p&gt;We could call our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mymode&lt;/span&gt;&lt;/code&gt; function on all of our predictions like this:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;mode_prediction&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;mymode&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;predictions&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# this doesn&amp;#39;t scale well&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Unfortunately this would move all of our results to a single machine to compute
the mode there. This might swamp that single machine.&lt;/p&gt;
&lt;p&gt;Instead we batch our predictions into groups of size 10, average each group,
and then repeat the process with the smaller set of predictions until we have
only one left. This sort of multi-step reduction is called a tree reduction.
We can write it up with a couple nested loops and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;executor.submit&lt;/span&gt;&lt;/code&gt;. This is
only an approximation of the mode, but it’s a much more scalable computation.
This finishes in about 1.5 seconds.&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;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;partition_all&lt;/span&gt;

&lt;span class="k"&gt;while&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;predictions&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;predictions&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;submit&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;mymode&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;chunk&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
                   &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;chunk&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;partition_all&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;predictions&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;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;gather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;predictions&lt;/span&gt;&lt;span class="p"&gt;)[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;result&lt;/span&gt;
&lt;span class="n"&gt;array&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;1&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="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;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;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/2016/04/20/dask-distributed-part-5.md&lt;/span&gt;, line 452)&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-score"&gt;
&lt;h1&gt;Final Score&lt;/h1&gt;
&lt;p&gt;Finally, after completing all of our work on our cluster we can see how well
our distributed random forest algorithm does.&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;accuracy_score&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="n"&gt;test&lt;/span&gt;&lt;span class="o"&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;passenger_count&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;0.67061974451423045&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Still worse than the naive “always guess one” strategy. This just goes to show
that, no matter how sophisticated your Big Data solution is, there is no
substitute for common sense and a little bit of domain 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/2016/04/20/dask-distributed-part-5.md&lt;/span&gt;, line 466)&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-didn-t-work"&gt;
&lt;h1&gt;What didn’t work&lt;/h1&gt;
&lt;p&gt;As always I’ll have a section like this that honestly says what doesn’t work
well and what I would have done with more time.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Clearly this would have benefited from more machine learning knowledge.
What would have been a good approach for this problem?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;I’ve been thinking a bit about memory management of replicated data on the
cluster. In this exercise we specifically replicated out the test data.
Everything would have worked fine without this step but it would have been
much slower as every worker gathered data from the single worker that
originally had the test dataframe. Replicating data is great until you
start filling up distributed RAM. It will be interesting to think of
policies about when to start cleaning up redundant data and when to keep it
around.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Several people from both open source users and Continuum customers have
asked about a general Dask library for machine learning, something akin to
Spark’s MLlib. Ideally a future Dask.learn module would leverage
Scikit-Learn in the same way that Dask.dataframe leverages Pandas. It’s
not clear how to cleanly break up and parallelize Scikit-Learn algorithms.&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/2016/04/20/dask-distributed-part-5.md&lt;/span&gt;, line 487)&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;This blogpost gives a concrete example using basic task submission with
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;executor.map&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;executor.submit&lt;/span&gt;&lt;/code&gt; to build a non-trivial computation. This
approach is straightforward and not restrictive. Personally this interface
excites me more than collections like Dask.dataframe; there is a lot of freedom
in arbitrary task submission.&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/2016/04/20/dask-distributed-part-5.md&lt;/span&gt;, line 495)&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="links"&gt;
&lt;h1&gt;Links&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/9f5720d8658e5f2f66666815b1f03f00"&gt;Notebook&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://www.youtube.com/watch?v=FkPlEqB8AnE&amp;amp;amp;list=PLRtz5iA93T4PQvWuoMnIyEIz1fXiJ5Pri&amp;amp;amp;index=11"&gt;Video&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://distributed.readthedocs.org/en/latest/"&gt;distributed&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/04/20/dask-distributed-part-5/"/>
    <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>2016-04-20T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/04/14/dask-distributed-optimizing-protocol/</id>
    <title>Fast Message Serialization</title>
    <updated>2016-04-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;em&gt;Very high performance isn’t about doing one thing well, it’s about doing
nothing poorly.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;This week I optimized the inter-node communication protocol used by
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.distributed&lt;/span&gt;&lt;/code&gt;. It was a fun exercise in optimization that involved
several different and unexpected components. I separately had to deal with
Pickle, NumPy, Tornado, MsgPack, and compression libraries.&lt;/p&gt;
&lt;p&gt;This blogpost is not advertising any particular functionality, rather it’s a
story of the problems I ran into when designing and optimizing a protocol to
quickly send both very small and very large numeric data between machines on
the Python stack.&lt;/p&gt;
&lt;p&gt;We care very strongly about both the many small messages case (thousands of
100 byte messages per second) &lt;em&gt;and&lt;/em&gt; the very large messages case (100-1000 MB).
This spans an interesting range of performance space. We end up with a
protocol that costs around 5 microseconds in the small case and operates at
1-1.5 GB/s in the large 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/2016/04/14/dask-distributed-optimizing-protocol.md&lt;/span&gt;, line 31)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="identify-a-problem"&gt;

&lt;p&gt;This came about as I was preparing a demo using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt; on a distributed
cluster for a Continuum webinar. I noticed that my computations were taking
much longer than expected. The
&lt;a class="reference external" href="http://distributed.readthedocs.org/en/latest/web.html"&gt;Web UI&lt;/a&gt; quickly pointed
me to the fact that my machines were spending 10-20 seconds moving 30 MB chunks
of numpy array data between them. This is very strange because I was on
100MB/s network, and so I expected these transfers to happen in more like 0.3s
than 15s.&lt;/p&gt;
&lt;p&gt;The Web UI made this glaringly apparent, so my first lesson was how valuable
visual profiling tools can be when they make performance issues glaringly
obvious. Thanks here goes to the Bokeh developers who helped the development
of the Dask real-time Web UI.&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/2016/04/14/dask-distributed-optimizing-protocol.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="problem-1-tornado-s-sentinels"&gt;
&lt;h1&gt;Problem 1: Tornado’s sentinels&lt;/h1&gt;
&lt;p&gt;Dask’s networking is built off of Tornado’s TCP IOStreams.&lt;/p&gt;
&lt;p&gt;There are two common ways to delineate messages on a socket, sentinel values
that signal the end of a message, and prefixing a length before every message.
Early on we tried both in Dask but found that prefixing a length before every
message was slow. It turns out that this was because TCP sockets try to batch
small messages to increase bandwidth. Turning this optimization off ended up
being an effective and easy solution, see the &lt;a class="reference external" href="http://www.unixguide.net/network/socketfaq/2.16.shtml"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;TCP_NODELAY&lt;/span&gt;&lt;/code&gt; parameter&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;However, before we figured that out we used sentinels for a long time.
Unfortunately Tornado does not handle sentinels well for large messages. At
the receipt of every new message it reads through all buffered data to see if
it can find the sentinel. This makes lots and lots of copies and reads through
lots and lots of bytes. This isn’t a problem if your messages are a few
kilobytes, as is common in web development, but it’s terrible if your messages
are millions or billions of bytes long.&lt;/p&gt;
&lt;p&gt;Switching back to prefixing messages with lengths and turning off the no-delay
optimization moved our bandwidth up from 3MB/s to 20MB/s per node. Thanks goes
to Ben Darnell (main Tornado developer) for helping us to track this down.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2016/04/14/dask-distributed-optimizing-protocol.md&lt;/span&gt;, line 70)&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="problem-2-memory-copies"&gt;
&lt;h1&gt;Problem 2: Memory Copies&lt;/h1&gt;
&lt;p&gt;A nice machine can copy memory at 5 GB/s. If your network is only 100 MB/s
then you can easily suffer several memory copies in your system without caring.
This leads to code that looks like the following:&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;socket.send(header + payload)
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This code concatenates two bytestrings, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;header&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;payload&lt;/span&gt;&lt;/code&gt; before
sending the result down a socket. If we cared deeply about avoiding memory
copies then we might instead send these two separately:&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;socket.send(header)
socket.send(payload)
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;But who cares, right? At 5 GB/s copying memory is cheap!&lt;/p&gt;
&lt;p&gt;Unfortunately this breaks down under either of the following conditions&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;You are sloppy enough to do this multiple times&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You find yourself on a machine with surprisingly low memory bandwidth,
like 10 times slower, as is the case on &lt;a class="reference external" href="http://stackoverflow.com/questions/36523142/why-is-copying-memory-on-ec2-machines-slow"&gt;some EC2 machines.&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Both of these were true for me but fortunately it’s usually straightforward to
reduce the number of copies down to a small number (we got down to three),
with moderate effort.&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/2016/04/14/dask-distributed-optimizing-protocol.md&lt;/span&gt;, line 97)&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="problem-3-unwanted-compression"&gt;
&lt;h1&gt;Problem 3: Unwanted Compression&lt;/h1&gt;
&lt;p&gt;Dask compresses all large messages with LZ4 or Snappy if they’re available.
Unfortunately, if your data isn’t very compressible then this is mostly lost
time. Doubly unforutnate is that you also have to decompress the data on the
recipient side. Decompressing not-very-compressible data was surprisingly
slow.&lt;/p&gt;
&lt;p&gt;Now we compress with the following policy:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;If the message is less than 10kB, don’t bother&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Pick out five 10kB samples of the data and compress those. If the result
isn’t well compressed then don’t bother compressing the full payload.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Compress the full payload, if it doesn’t compress well then just send along
the original to spare the receiver’s side from compressing.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In this case we use cheap checks to guard against unwanted compression. We
also avoid any cost at all for small messages, which we care about deeply.&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/2016/04/14/dask-distributed-optimizing-protocol.md&lt;/span&gt;, line 116)&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="problem-4-cloudpickle-is-not-as-fast-as-pickle"&gt;
&lt;h1&gt;Problem 4: Cloudpickle is not as fast as Pickle&lt;/h1&gt;
&lt;p&gt;This was surprising, because cloudpickle mostly defers to Pickle for the easy
stuff, like 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="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;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="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;data&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;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&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;255&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;u1&amp;#39;&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="mi"&gt;10000000&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;pickle&lt;/span&gt;&lt;span class="o"&gt;,&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cloudpickle&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="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pickle&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dumps&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="n"&gt;protocol&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;8.65&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;8.42&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;17.1&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="mf"&gt;16.9&lt;/span&gt; &lt;span class="n"&gt;ms&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;4&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="mi"&gt;10000161&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="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cloudpickle&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dumps&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="n"&gt;protocol&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;20.6&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;24.5&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;45.1&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="mf"&gt;44.4&lt;/span&gt; &lt;span class="n"&gt;ms&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="mi"&gt;10000161&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;But it turns out that cloudpickle is using the Python implementation, while
pickle itself (or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cPickle&lt;/span&gt;&lt;/code&gt; in Python 2) is using the compiled C implemenation.
Fortunately this is easy to correct, and a quick typecheck on common large
dataformats in Python (NumPy and Pandas) gets us this speed boost.&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/2016/04/14/dask-distributed-optimizing-protocol.md&lt;/span&gt;, line 144)&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="problem-5-pickle-is-still-slower-than-you-d-expect"&gt;
&lt;h1&gt;Problem 5: Pickle is still slower than you’d expect&lt;/h1&gt;
&lt;p&gt;Pickle runs at about half the speed of memcopy, which is what you’d expect from
a protocol that is mostly just “serialize the dtype, strides, then tack on the
data bytes”. There must be an extraneous memory copy in there.&lt;/p&gt;
&lt;p&gt;See &lt;a class="reference external" href="https://github.com/numpy/numpy/issues/7544"&gt;issue 7544&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/2016/04/14/dask-distributed-optimizing-protocol.md&lt;/span&gt;, line 152)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="problem-6-msgpack-is-bad-at-large-bytestrings"&gt;
&lt;h1&gt;Problem 6: MsgPack is bad at large bytestrings&lt;/h1&gt;
&lt;p&gt;Dask serializes most messages with MsgPack, which is ordinarily very fast.
Unfortunately the MsgPack spec doesn’t support bytestrings greater than 4GB
(which do come up for us) and the Python implementations don’t pass through
large bytestrings very efficiently. So we had to handle large bytestrings
separately. Any message that contains bytestrings over 1MB in size will have
them stripped out and sent along in a separate frame. This both avoids the
MsgPack overhead and avoids a memory copy (we can send the bytes directly to
the socket).&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/2016/04/14/dask-distributed-optimizing-protocol.md&lt;/span&gt;, line 163)&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="problem-7-tornado-makes-a-copy"&gt;
&lt;h1&gt;Problem 7: Tornado makes a copy&lt;/h1&gt;
&lt;p&gt;Sockets on Windows don’t accept payloads greater than 128kB in size. As a
result Tornado chops up large messages into many small ones. On linux this
memory copy is extraneous. It can be removed with a bit of logic within
Tornado. I might do this in the moderate 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/2016/04/14/dask-distributed-optimizing-protocol.md&lt;/span&gt;, line 170)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="results"&gt;
&lt;h1&gt;Results&lt;/h1&gt;
&lt;p&gt;We serialize small messages in about 5 microseconds (thanks msgpack!) and move
large bytes around in the cost of three memory copies (about 1-1.5 GB/s) which
is generally faster than most networks in use.&lt;/p&gt;
&lt;p&gt;Here is a profile of sending and receiving a gigabyte-sized NumPy array of
random values through to the same process over localhost (500 MB/s on my
machine.)&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;         &lt;span class="mi"&gt;381360&lt;/span&gt; &lt;span class="n"&gt;function&lt;/span&gt; &lt;span class="n"&gt;calls&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;381323&lt;/span&gt; &lt;span class="n"&gt;primitive&lt;/span&gt; &lt;span class="n"&gt;calls&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="mf"&gt;1.451&lt;/span&gt; &lt;span class="n"&gt;seconds&lt;/span&gt;

   &lt;span class="n"&gt;Ordered&lt;/span&gt; &lt;span class="n"&gt;by&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;internal&lt;/span&gt; &lt;span class="n"&gt;time&lt;/span&gt;

   &lt;span class="n"&gt;ncalls&lt;/span&gt;  &lt;span class="n"&gt;tottime&lt;/span&gt;  &lt;span class="n"&gt;percall&lt;/span&gt;  &lt;span class="n"&gt;cumtime&lt;/span&gt;  &lt;span class="n"&gt;percall&lt;/span&gt; &lt;span class="n"&gt;filename&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="n"&gt;lineno&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;function&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="mi"&gt;1&lt;/span&gt;    &lt;span class="mf"&gt;0.366&lt;/span&gt;    &lt;span class="mf"&gt;0.366&lt;/span&gt;    &lt;span class="mf"&gt;0.366&lt;/span&gt;    &lt;span class="mf"&gt;0.366&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="n"&gt;built&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;method&lt;/span&gt; &lt;span class="n"&gt;dumps&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;
        &lt;span class="mi"&gt;8&lt;/span&gt;    &lt;span class="mf"&gt;0.289&lt;/span&gt;    &lt;span class="mf"&gt;0.036&lt;/span&gt;    &lt;span class="mf"&gt;0.291&lt;/span&gt;    &lt;span class="mf"&gt;0.036&lt;/span&gt; &lt;span class="n"&gt;iostream&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;py&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;360&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;write&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="mi"&gt;15353&lt;/span&gt;    &lt;span class="mf"&gt;0.228&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt;    &lt;span class="mf"&gt;0.228&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="n"&gt;method&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;join&amp;#39;&lt;/span&gt; &lt;span class="n"&gt;of&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;bytes&amp;#39;&lt;/span&gt; &lt;span class="n"&gt;objects&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;
    &lt;span class="mi"&gt;15355&lt;/span&gt;    &lt;span class="mf"&gt;0.166&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt;    &lt;span class="mf"&gt;0.166&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="n"&gt;method&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;recv&amp;#39;&lt;/span&gt; &lt;span class="n"&gt;of&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;_socket.socket&amp;#39;&lt;/span&gt; &lt;span class="n"&gt;objects&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;
    &lt;span class="mi"&gt;15362&lt;/span&gt;    &lt;span class="mf"&gt;0.156&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt;    &lt;span class="mf"&gt;0.398&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt; &lt;span class="n"&gt;iostream&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;py&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;1510&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;_merge_prefix&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
     &lt;span class="mi"&gt;7759&lt;/span&gt;    &lt;span class="mf"&gt;0.101&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt;    &lt;span class="mf"&gt;0.101&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="n"&gt;method&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;send&amp;#39;&lt;/span&gt; &lt;span class="n"&gt;of&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;_socket.socket&amp;#39;&lt;/span&gt; &lt;span class="n"&gt;objects&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;
    &lt;span class="mi"&gt;17&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="mi"&gt;14&lt;/span&gt;    &lt;span class="mf"&gt;0.026&lt;/span&gt;    &lt;span class="mf"&gt;0.002&lt;/span&gt;    &lt;span class="mf"&gt;0.686&lt;/span&gt;    &lt;span class="mf"&gt;0.049&lt;/span&gt; &lt;span class="n"&gt;gen&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;py&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;990&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;run&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="mi"&gt;15355&lt;/span&gt;    &lt;span class="mf"&gt;0.021&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt;    &lt;span class="mf"&gt;0.198&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt; &lt;span class="n"&gt;iostream&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;py&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;721&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;_read_to_buffer&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="mi"&gt;8&lt;/span&gt;    &lt;span class="mf"&gt;0.018&lt;/span&gt;    &lt;span class="mf"&gt;0.002&lt;/span&gt;    &lt;span class="mf"&gt;0.203&lt;/span&gt;    &lt;span class="mf"&gt;0.025&lt;/span&gt; &lt;span class="n"&gt;iostream&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;py&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;876&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;_consume&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
       &lt;span class="mi"&gt;91&lt;/span&gt;    &lt;span class="mf"&gt;0.017&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt;    &lt;span class="mf"&gt;0.335&lt;/span&gt;    &lt;span class="mf"&gt;0.004&lt;/span&gt; &lt;span class="n"&gt;iostream&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;py&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;827&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;_handle_write&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
       &lt;span class="mi"&gt;89&lt;/span&gt;    &lt;span class="mf"&gt;0.015&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt;    &lt;span class="mf"&gt;0.217&lt;/span&gt;    &lt;span class="mf"&gt;0.002&lt;/span&gt; &lt;span class="n"&gt;iostream&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;py&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;585&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;_read_to_buffer_loop&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
   &lt;span class="mi"&gt;122567&lt;/span&gt;    &lt;span class="mf"&gt;0.009&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt;    &lt;span class="mf"&gt;0.009&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="n"&gt;built&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;method&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;
    &lt;span class="mi"&gt;15355&lt;/span&gt;    &lt;span class="mf"&gt;0.008&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt;    &lt;span class="mf"&gt;0.173&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt; &lt;span class="n"&gt;iostream&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;py&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;1010&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;read_from_fd&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="mi"&gt;38369&lt;/span&gt;    &lt;span class="mf"&gt;0.004&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt;    &lt;span class="mf"&gt;0.004&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="n"&gt;method&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;append&amp;#39;&lt;/span&gt; &lt;span class="n"&gt;of&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;list&amp;#39;&lt;/span&gt; &lt;span class="n"&gt;objects&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;
     &lt;span class="mi"&gt;7759&lt;/span&gt;    &lt;span class="mf"&gt;0.004&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt;    &lt;span class="mf"&gt;0.104&lt;/span&gt;    &lt;span class="mf"&gt;0.000&lt;/span&gt; &lt;span class="n"&gt;iostream&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;py&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;1023&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;write_to_fd&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
        &lt;span class="mi"&gt;1&lt;/span&gt;    &lt;span class="mf"&gt;0.003&lt;/span&gt;    &lt;span class="mf"&gt;0.003&lt;/span&gt;    &lt;span class="mf"&gt;1.451&lt;/span&gt;    &lt;span class="mf"&gt;1.451&lt;/span&gt; &lt;span class="n"&gt;ioloop&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;py&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;746&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;start&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Dominant unwanted costs include the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;400ms: Pickling the NumPy array&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;400ms: Bytestring handling within Tornado&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;After this we’re just bound by pushing bytes down a wire.&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/2016/04/14/dask-distributed-optimizing-protocol.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="conclusion"&gt;
&lt;h1&gt;Conclusion&lt;/h1&gt;
&lt;p&gt;Writing fast code isn’t about writing any one thing particularly well, it’s
about mitigating everything that can get in your way. As you approch peak
performance, previously minor flaws suddenly become your dominant bottleneck.
Success here depends on frequent profiling and keeping your mind open to
unexpected and surprising costs.&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/2016/04/14/dask-distributed-optimizing-protocol.md&lt;/span&gt;, line 219)&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="links"&gt;
&lt;h1&gt;Links&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://stackoverflow.com/questions/36523142/why-is-copying-memory-on-ec2-machines-slow"&gt;EC2 slow memory copy StackOverflow question.&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/tornadoweb/tornado/issues/1685"&gt;Tornado issue for sending large messages&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://en.wikipedia.org/wiki/Nagle%27s_algorithm"&gt;Wikipedia page on Nagle’s algorithm for TCP protocol for small packets&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/numpy/numpy/issues/7544"&gt;NumPy issue for double memory copy&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/cloudpipe/cloudpickle/issues/59"&gt;Cloudpickle issue for memoryview support&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://distributed.readthedocs.org/en/latest/"&gt;dask.distributed&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/04/14/dask-distributed-optimizing-protocol/"/>
    <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>2016-04-14T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/02/26/dask-distributed-part-3/</id>
    <title>Distributed Dask Arrays</title>
    <updated>2016-02-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;In this post we analyze weather data across a cluster using NumPy in
parallel with dask.array. We focus on the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;How to set up the distributed scheduler with a job scheduler like Sun
GridEngine.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How to load NetCDF data from a network file system (NFS) into distributed
RAM&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How to manipulate data with dask.arrays&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How to interact with distributed data using IPython widgets&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This blogpost has an accompanying
&lt;a class="reference external" href="https://www.youtube.com/watch?v=ZpMXEVp-iaY"&gt;screencast&lt;/a&gt; which might be a bit
more fun than this text version.&lt;/p&gt;
&lt;p&gt;This is the third in a sequence of blogposts about dask.distributed:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../2016/02/17/dask-distributed-part1/"&gt;&lt;span class="doc std std-doc"&gt;Dask Bags on GitHub Data&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../2016/02/22/dask-distributed-part-2/"&gt;&lt;span class="doc std std-doc"&gt;Dask DataFrames on HDFS&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask Arrays on NetCDF data&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/2016/02/26/dask-distributed-part-3.md&lt;/span&gt;, line 32)&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;We wanted to emulate the typical academic cluster setup using a job scheduler
like SunGridEngine (similar to SLURM, Torque, PBS scripts and other
technologies), a shared network file system, and typical binary stored arrays
in NetCDF files (similar to HDF5).&lt;/p&gt;
&lt;p&gt;To this end we used &lt;a class="reference external" href="http://star.mit.edu/cluster/"&gt;Starcluster&lt;/a&gt;, a quick way to
set up such a cluster on EC2 with SGE and NFS, and we downloaded data from the
&lt;a class="reference external" href="http://www.ecmwf.int/en/research/climate-reanalysis/era-interim"&gt;European Centre for Meteorology and Weather
Forecasting&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;To deploy dask’s distributed scheduler with SGE we made a scheduler on the
master node:&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;sgeadmin@master:~$ dscheduler
distributed.scheduler - INFO - Start Scheduler at:  172.31.7.88:8786
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And then used the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;qsub&lt;/span&gt;&lt;/code&gt; command to start four dask workers, pointing to the
scheduler address:&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;sgeadmin@master:~$ qsub -b y -V dworker 172.31.7.88:8786
Your job 1 (&amp;quot;dworker&amp;quot;) has been submitted
sgeadmin@master:~$ qsub -b y -V dworker 172.31.7.88:8786
Your job 2 (&amp;quot;dworker&amp;quot;) has been submitted
sgeadmin@master:~$ qsub -b y -V dworker 172.31.7.88:8786
Your job 3 (&amp;quot;dworker&amp;quot;) has been submitted
sgeadmin@master:~$ qsub -b y -V dworker 172.31.7.88:8786
Your job 4 (&amp;quot;dworker&amp;quot;) has been submitted
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;After a few seconds these workers start on various nodes in the cluster and
connect to the 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/2016/02/26/dask-distributed-part-3.md&lt;/span&gt;, line 65)&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="load-sample-data-on-a-single-machine"&gt;
&lt;h1&gt;Load sample data on a single machine&lt;/h1&gt;
&lt;p&gt;On the shared NFS drive we’ve downloaded several NetCDF3 files, each holding
the global temperature every six hours for a single day:&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;*.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;filenames&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="go"&gt;[&amp;#39;2014-01-01.nc3&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;2014-01-02.nc3&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;2014-01-03.nc3&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;2014-01-04.nc3&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;2014-01-05.nc3&amp;#39;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We use conda to install the netCDF4 library and make a small function to
read the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;t2m&lt;/span&gt;&lt;/code&gt; variable for “temperature at two meters elevation” from a single
filename:&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 netcdf4
&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="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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;load_temperature&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="k"&gt;with&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="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="k"&gt;return&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;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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This converts a single file into a single numpy array in memory. We could call
this on an individual file locally 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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;load_temperature&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="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;span class="go"&gt;array([[[ 253.96238624,  253.96238624,  253.96238624, ...,  253.96238624,&lt;/span&gt;
&lt;span class="go"&gt;          253.96238624,  253.96238624],&lt;/span&gt;
&lt;span class="go"&gt;        [ 252.80590921,  252.81070124,  252.81389593, ...,  252.79792249,&lt;/span&gt;
&lt;span class="go"&gt;          252.80111718,  252.80271452],&lt;/span&gt;
&lt;span class="go"&gt;          ...&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;load_temperature&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="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;(4, 721, 1440)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Our dataset has dimensions of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(time,&lt;/span&gt; &lt;span class="pre"&gt;latitude,&lt;/span&gt; &lt;span class="pre"&gt;longitude)&lt;/span&gt;&lt;/code&gt;. Note above that
each day has four time entries (measurements every six hours).&lt;/p&gt;
&lt;p&gt;The NFS set up by Starcluster is unfortunately quite small. We were only able
to fit around five months of data (136 days) in shared 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/2016/02/26/dask-distributed-part-3.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="load-data-across-cluster"&gt;
&lt;h1&gt;Load data across cluster&lt;/h1&gt;
&lt;p&gt;We want to call the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;load_temperature&lt;/span&gt;&lt;/code&gt; function on our list &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;filenames&lt;/span&gt;&lt;/code&gt; on each
of our four workers. We connect a dask Executor to our scheduler address and
then map our function on our filenames:&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;Executor&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;progress&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;e&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Executor&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;172.31.7.88:8786&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;e&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;Executor: scheduler=172.31.7.88:8786 workers=4 threads=32&amp;gt;&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;load_temperature&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;progress&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/load-netcdf.gif"&gt;
&lt;p&gt;After this completes we have several numpy arrays scattered about the memory of
each of our four workers.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2016/02/26/dask-distributed-part-3.md&lt;/span&gt;, line 135)&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="coordinate-with-dask-array"&gt;
&lt;h1&gt;Coordinate with dask.array&lt;/h1&gt;
&lt;p&gt;We coordinate these many numpy arrays into a single logical dask array 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="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.collections&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;futures_to_dask_arrays&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;xs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;futures_to_dask_arrays&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;futures&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# many small dask arrays&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;concatenate&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;xs&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="c1"&gt;# one large dask array, joined by time&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="go"&gt;dask.array&amp;lt;concate..., shape=(544, 721, 1440), dtype=float64, chunksize=(4, 721, 1440)&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This single logical dask array is comprised of 136 numpy arrays spread across
our cluster. Operations on the single dask array will trigger many operations
on each of our numpy arrays.&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/2016/02/26/dask-distributed-part-3.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="interact-with-distributed-data"&gt;
&lt;h1&gt;Interact with Distributed Data&lt;/h1&gt;
&lt;p&gt;We can now interact with our dataset using standard NumPy syntax and other
PyData libraries. Below we pull out a single time slice and render it to the
screen with matplotlib.&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;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;pyplot&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;plt&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;100&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;:,&lt;/span&gt; &lt;span class="p"&gt;:]&lt;/span&gt;&lt;span class="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;cmap&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;viridis&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;colorbar&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/temperature-viridis.png"&gt;
&lt;p&gt;In the &lt;a class="reference external" href="https://www.youtube.com/watch?v=ZpMXEVp-iaY"&gt;screencast version of this
post&lt;/a&gt; we hook this up to an
IPython slider widget and scroll around time, which is fun.&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/2016/02/26/dask-distributed-part-3.md&lt;/span&gt;, line 172)&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="speed"&gt;
&lt;h1&gt;Speed&lt;/h1&gt;
&lt;p&gt;We benchmark a few representative operations to look at the strengths and
weaknesses of the distributed system.&lt;/p&gt;
&lt;section id="single-element"&gt;
&lt;h2&gt;Single element&lt;/h2&gt;
&lt;p&gt;This single element computation accesses a single number from a single NumPy
array of our dataset. It is bound by a network roundtrip from client to
scheduler, to worker, and back.&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;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="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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 4 ms, sys: 0 ns, total: 4 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 9.72 ms&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="single-time-slice"&gt;
&lt;h2&gt;Single time slice&lt;/h2&gt;
&lt;p&gt;This time slice computation pulls around 8 MB from a single NumPy array on a
single worker. It is likely bound by network bandwidth.&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;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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 24 ms, sys: 24 ms, total: 48 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 274 ms&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="mean-computation"&gt;
&lt;h2&gt;Mean computation&lt;/h2&gt;
&lt;p&gt;This mean computation touches every number in every NumPy array across all of
our workers. Computing means is quite fast, so this is likely bound by
scheduler overhead.&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;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="go"&gt;CPU times: user 88 ms, sys: 0 ns, total: 88 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 422 ms&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/2016/02/26/dask-distributed-part-3.md&lt;/span&gt;, line 212)&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="interactive-widgets"&gt;
&lt;h1&gt;Interactive Widgets&lt;/h1&gt;
&lt;p&gt;To make these times feel more visceral we hook up these computations to IPython
Widgets.&lt;/p&gt;
&lt;p&gt;This first example looks fairly fluid. This only touches a single worker and
returns a small result. It is cheap because it indexes in a way that is well
aligned with how our NumPy arrays are split up by time.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@interact&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mi"&gt;1&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;f&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;time&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;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;:,&lt;/span&gt; &lt;span class="p"&gt;:]&lt;/span&gt;&lt;span class="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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/mean-time.gif"&gt;
&lt;p&gt;This second example is less fluid because we index across our NumPy chunks.
Each computation touches all of our data. It’s still not bad though and quite
acceptable by today’s standards of interactive distributed data
science.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nd"&gt;@interact&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;lat&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&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="mi"&gt;1&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;f&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;lat&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;x&lt;/span&gt;&lt;span class="p"&gt;[:,&lt;/span&gt; &lt;span class="n"&gt;lat&lt;/span&gt;&lt;span class="p"&gt;,&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/mean-latitude.gif"&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/2016/02/26/dask-distributed-part-3.md&lt;/span&gt;, line 242)&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="normalize-data"&gt;
&lt;h1&gt;Normalize Data&lt;/h1&gt;
&lt;p&gt;Until now we’ve only performed simple calculations on our data, usually grabbing
out means. The image of the temperature above looks unsurprising. The image
is dominated by the facts that land is warmer than oceans and that the equator
is warmer than the poles. No surprises there.&lt;/p&gt;
&lt;p&gt;To make things more interesting we subtract off the mean and divide by the
standard deviation over time. This will tell us how unexpectedly hot or cold a
particular point was, relative to all measurements of that point over time.
This gives us something like a geo-located Z-Score.&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;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="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="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;std&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;z&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;progress&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/normalize.gif"&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;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="nb"&gt;slice&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;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;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;colorbar&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/temperature-denormalized.png"&gt;
&lt;p&gt;We can now see much more fine structure of the currents of the day. In the
&lt;a class="reference external" href="https://www.youtube.com/watch?v=ZpMXEVp-iaY"&gt;screencast version&lt;/a&gt; we hook this
dataset up to a slider as well and inspect various times.&lt;/p&gt;
&lt;p&gt;I’ve avoided displaying GIFs of full images changing in this post to keep the
size down, however we can easily render a plot of average temperature by
latitude changing over time here:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="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="n"&gt;xrange&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;90&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;arange&lt;/span&gt;&lt;span class="p"&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;shape&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="mi"&gt;4&lt;/span&gt;

&lt;span class="nd"&gt;@interact&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="mi"&gt;1&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;f&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;figure&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;figsize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&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;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;plot&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;xrange&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;time&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;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;compute&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
    &lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ylabel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Normalized Temperature&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;xlabel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;Latitude (degrees)&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/latitude-plot.gif"&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/2016/02/26/dask-distributed-part-3.md&lt;/span&gt;, line 291)&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;We showed how to use distributed dask.arrays on a typical academic cluster.
I’ve had several conversations with different groups about this topic; it seems
to be a common case. I hope that the instructions at the beginning of this
post prove to be helpful to others.&lt;/p&gt;
&lt;p&gt;It is really satisfying to me to couple interactive widgets with data on a
cluster in an intuitive way. This sort of fluid interaction on larger datasets
is a core problem in modern data science.&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/2016/02/26/dask-distributed-part-3.md&lt;/span&gt;, line 302)&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-didn-t-work"&gt;
&lt;h1&gt;What didn’t work&lt;/h1&gt;
&lt;p&gt;As always I’ll include a section like this on what didn’t work well or what I
would have done with more time:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;No high-level &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_netcdf&lt;/span&gt;&lt;/code&gt; function: We had to use the mid-level API of
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;executor.map&lt;/span&gt;&lt;/code&gt; to construct our dask array. This is a bit of a pain for
novice users. We should probably adapt existing high-level functions in
dask.array to robustly handle the distributed data case.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Need a larger problem: Our dataset could have fit into a Macbook Pro.
A larger dataset that could not have been efficiently investigated from a
single machine would have really cemented the need for this technology.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Easier deployment: The solution above with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;qsub&lt;/span&gt;&lt;/code&gt; was straightforward but
not always accessible to novice users. Additionally while SGE is common
there are several other systems that are just as common. We need to think
of nice ways to automate this for the user.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;XArray integration: Many people use dask.array on single machines through
&lt;a class="reference external" href="http://xarray.pydata.org/en/stable/"&gt;XArray&lt;/a&gt;, an excellent library for the
analysis of labeled nd-arrays especially common in climate science. It
would be good to integrate this new distributed work into the XArray
project. I suspect that doing this mostly involves handling the data
ingest problem described above.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Reduction speed: The computation of normalized temperature, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;z&lt;/span&gt;&lt;/code&gt;, took a
surprisingly long time. I’d like to look into what is holding up that
computation.&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/2016/02/26/dask-distributed-part-3.md&lt;/span&gt;, line 328)&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="links"&gt;
&lt;h1&gt;Links&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask.pydata.org/en/latest/array.html"&gt;dask.array&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://distributed.readthedocs.org/en/latest/"&gt;dask.distributed&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/02/26/dask-distributed-part-3/"/>
    <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>2016-02-26T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/02/22/dask-distributed-part-2/</id>
    <title>Pandas on HDFS with Dask Dataframes</title>
    <updated>2016-02-22T00: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;In this post we use Pandas in parallel across an HDFS cluster to read CSV data.
We coordinate these computations with dask.dataframe. A screencast version of
this blogpost is available &lt;a class="reference external" href="https://www.youtube.com/watch?v=LioaeHsZDBQ"&gt;here&lt;/a&gt;
and the previous post in this series is available
&lt;a class="reference internal" href="../2016/02/17/dask-distributed-part1/"&gt;&lt;span class="doc std std-doc"&gt;here&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;To start, we connect to our scheduler, import the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;hdfs&lt;/span&gt;&lt;/code&gt; module from the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed&lt;/span&gt;&lt;/code&gt; library, and read our CSV data from HDFS.&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;Executor&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;hdfs&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;progress&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;e&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Executor&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;127.0.0.1:8786&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;e&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;Executor: scheduler=127.0.0.1:8786 workers=64 threads=64&amp;gt;&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;nyc2014&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;hdfs&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;/nyctaxi/2014/*.csv&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;parse_dates&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;pickup_datetime&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;dropoff_datetime&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;skipinitialspace&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;nyc2015&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;hdfs&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;/nyctaxi/2015/*.csv&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;parse_dates&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;tpep_pickup_datetime&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;tpep_dropoff_datetime&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;nyc2014&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;nyc2015&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;nyc2014&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;nyc2015&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;progress&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nyc2014&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;nyc2015&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/distributed-hdfs-read-csv.gif"&gt;
&lt;p&gt;Our data comes from the New York City Taxi and Limousine Commission which
publishes &lt;a class="reference external" href="http://www.nyc.gov/html/tlc/html/about/trip_record_data.shtml"&gt;all yellow cab taxi rides in
NYC&lt;/a&gt; for various
years. This is a nice model dataset for computational tabular data because
it’s large enough to be annoying while also deep enough to be broadly
appealing. Each year is about 25GB on disk and about 60GB in memory as a
Pandas DataFrame.&lt;/p&gt;
&lt;p&gt;HDFS breaks up our CSV files into 128MB chunks on various hard drives spread
throughout the cluster. The dask.distributed workers each read the chunks of
bytes local to them and call the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pandas.read_csv&lt;/span&gt;&lt;/code&gt; function on these bytes,
producing 391 separate Pandas DataFrame objects spread throughout the memory of
our eight worker nodes. The returned objects, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;nyc2014&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;nyc2015&lt;/span&gt;&lt;/code&gt;, are
&lt;a class="reference external" href="http://dask.pydata.org/en/latest/dataframe.html"&gt;dask.dataframe&lt;/a&gt; objects which
present a subset of the Pandas API to the user, but farm out all of the work to
the many Pandas dataframes they control across the network.&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/2016/02/22/dask-distributed-part-2.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 id="play-with-distributed-data"&gt;

&lt;p&gt;If we wait for the data to load fully into memory then we can perform
pandas-style analysis at interactive speeds.&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;nyc2015&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
    &lt;tr style="text-align: right;"&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;VendorID&lt;/th&gt;
      &lt;th&gt;tpep_pickup_datetime&lt;/th&gt;
      &lt;th&gt;tpep_dropoff_datetime&lt;/th&gt;
      &lt;th&gt;passenger_count&lt;/th&gt;
      &lt;th&gt;trip_distance&lt;/th&gt;
      &lt;th&gt;pickup_longitude&lt;/th&gt;
      &lt;th&gt;pickup_latitude&lt;/th&gt;
      &lt;th&gt;RateCodeID&lt;/th&gt;
      &lt;th&gt;store_and_fwd_flag&lt;/th&gt;
      &lt;th&gt;dropoff_longitude&lt;/th&gt;
      &lt;th&gt;dropoff_latitude&lt;/th&gt;
      &lt;th&gt;payment_type&lt;/th&gt;
      &lt;th&gt;fare_amount&lt;/th&gt;
      &lt;th&gt;extra&lt;/th&gt;
      &lt;th&gt;mta_tax&lt;/th&gt;
      &lt;th&gt;tip_amount&lt;/th&gt;
      &lt;th&gt;tolls_amount&lt;/th&gt;
      &lt;th&gt;improvement_surcharge&lt;/th&gt;
      &lt;th&gt;total_amount&lt;/th&gt;
    &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt;0&lt;/th&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;2015-01-15 19:05:39&lt;/td&gt;
      &lt;td&gt;2015-01-15 19:23:42&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;1.59&lt;/td&gt;
      &lt;td&gt;-73.993896&lt;/td&gt;
      &lt;td&gt;40.750111&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.974785&lt;/td&gt;
      &lt;td&gt;40.750618&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;12.0&lt;/td&gt;
      &lt;td&gt;1.0&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;3.25&lt;/td&gt;
      &lt;td&gt;0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;17.05&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;1&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:33:38&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:53:28&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;3.30&lt;/td&gt;
      &lt;td&gt;-74.001648&lt;/td&gt;
      &lt;td&gt;40.724243&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.994415&lt;/td&gt;
      &lt;td&gt;40.759109&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;14.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;2.00&lt;/td&gt;
      &lt;td&gt;0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;17.80&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;2&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:33:38&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:43:41&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;1.80&lt;/td&gt;
      &lt;td&gt;-73.963341&lt;/td&gt;
      &lt;td&gt;40.802788&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-73.951820&lt;/td&gt;
      &lt;td&gt;40.824413&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;9.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;10.80&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;3&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:33:39&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:35:31&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;0.50&lt;/td&gt;
      &lt;td&gt;-74.009087&lt;/td&gt;
      &lt;td&gt;40.713818&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-74.004326&lt;/td&gt;
      &lt;td&gt;40.719986&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;3.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;4.80&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;4&lt;/th&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:33:39&lt;/td&gt;
      &lt;td&gt;2015-01-10 20:52:58&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;3.00&lt;/td&gt;
      &lt;td&gt;-73.971176&lt;/td&gt;
      &lt;td&gt;40.762428&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
      &lt;td&gt;N&lt;/td&gt;
      &lt;td&gt;-74.004181&lt;/td&gt;
      &lt;td&gt;40.742653&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
      &lt;td&gt;15.0&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;0&lt;/td&gt;
      &lt;td&gt;0.3&lt;/td&gt;
      &lt;td&gt;16.30&lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&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;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nyc2014&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;165114373&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nyc2015&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;146112989&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Interestingly it appears that the NYC cab industry has contracted a bit in the
last year. There are &lt;em&gt;fewer&lt;/em&gt; cab rides in 2015 than in 2014.&lt;/p&gt;
&lt;p&gt;When we ask for something like the length of the full dask.dataframe we
actually ask for the length of all of the hundreds of Pandas dataframes and
then sum them up. This process of reaching out to all of the workers completes
in around 200-300 ms, which is generally fast enough to feel snappy in an
interactive session.&lt;/p&gt;
&lt;p&gt;The dask.dataframe API looks just like the Pandas API, except that we call
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.compute()&lt;/span&gt;&lt;/code&gt; when we want an actual result.&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;nyc2014&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;passenger_count&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="go"&gt;279997507.0&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;nyc2015&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;passenger_count&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="go"&gt;245566747&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Dask.dataframes build a plan to get your result and the distributed scheduler
coordinates that plan on all of the little Pandas dataframes on the workers
that make up our dataset.&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/2016/02/22/dask-distributed-part-2.md&lt;/span&gt;, line 237)&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="pandas-for-metadata"&gt;
&lt;h1&gt;Pandas for Metadata&lt;/h1&gt;
&lt;p&gt;Let’s appreciate for a moment all the work we didn’t have to do around CSV
handling because Pandas magically handled it for us.&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;nyc2015&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dtypes&lt;/span&gt;
&lt;span class="go"&gt;VendorID                          int64&lt;/span&gt;
&lt;span class="go"&gt;tpep_pickup_datetime     datetime64[ns]&lt;/span&gt;
&lt;span class="go"&gt;tpep_dropoff_datetime    datetime64[ns]&lt;/span&gt;
&lt;span class="go"&gt;passenger_count                   int64&lt;/span&gt;
&lt;span class="go"&gt;trip_distance                   float64&lt;/span&gt;
&lt;span class="go"&gt;pickup_longitude                float64&lt;/span&gt;
&lt;span class="go"&gt;pickup_latitude                 float64&lt;/span&gt;
&lt;span class="go"&gt;RateCodeID                        int64&lt;/span&gt;
&lt;span class="go"&gt;store_and_fwd_flag               object&lt;/span&gt;
&lt;span class="go"&gt;dropoff_longitude               float64&lt;/span&gt;
&lt;span class="go"&gt;dropoff_latitude                float64&lt;/span&gt;
&lt;span class="go"&gt;payment_type                      int64&lt;/span&gt;
&lt;span class="go"&gt;fare_amount                     float64&lt;/span&gt;
&lt;span class="go"&gt;extra                           float64&lt;/span&gt;
&lt;span class="go"&gt;mta_tax                         float64&lt;/span&gt;
&lt;span class="go"&gt;tip_amount                      float64&lt;/span&gt;
&lt;span class="go"&gt;tolls_amount                    float64&lt;/span&gt;
&lt;span class="go"&gt;improvement_surcharge           float64&lt;/span&gt;
&lt;span class="go"&gt;total_amount\r                  float64&lt;/span&gt;
&lt;span class="go"&gt;dtype: object&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We didn’t have to find columns or specify data-types. We didn’t have to parse
each value with an &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;int&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;float&lt;/span&gt;&lt;/code&gt; function as appropriate. We didn’t have to
parse the datetimes, but instead just specified a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;parse_datetimes=&lt;/span&gt;&lt;/code&gt; keyword.
The CSV parsing happened about as quickly as can be expected for this format,
clocking in at a network total of a bit under 1 GB/s.&lt;/p&gt;
&lt;p&gt;Pandas is well loved because it removes all of these little hurdles from the
life of the analyst. If we tried to reinvent a new
“Big-Data-Frame” we would have to reimplement all of the work already well done
inside of Pandas. Instead, dask.dataframe just coordinates and reuses the code
within the Pandas library. It is successful largely due to work from core
Pandas developers, notably Masaaki Horikoshi
(&lt;a class="reference external" href="https://github.com/sinhrks/"&gt;&amp;#64;sinhrks&lt;/a&gt;), who have done tremendous work to
align the API precisely with the Pandas core library.&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/2016/02/22/dask-distributed-part-2.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="analyze-tips-and-payment-types"&gt;
&lt;h1&gt;Analyze Tips and Payment Types&lt;/h1&gt;
&lt;p&gt;In an effort to demonstrate the abilities of dask.dataframe we ask a simple
question of our data, &lt;em&gt;“how do New Yorkers tip?”&lt;/em&gt;. The 2015 NYCTaxi data is
quite good about breaking down the total cost of each ride into the fare
amount, tip amount, and various taxes and fees. In particular this lets us
measure the percentage that each rider decided to pay in tip.&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;nyc2015&lt;/span&gt;&lt;span class="p"&gt;[[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;fare_amount&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;tip_amount&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;payment_type&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;head&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table border="1" class="dataframe"&gt;
  &lt;thead&gt;
    &lt;tr style="text-align: right;"&gt;
      &lt;th&gt;&lt;/th&gt;
      &lt;th&gt;fare_amount&lt;/th&gt;
      &lt;th&gt;tip_amount&lt;/th&gt;
      &lt;th&gt;payment_type&lt;/th&gt;
    &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt;0&lt;/th&gt;
      &lt;td&gt;12.0&lt;/td&gt;
      &lt;td&gt;3.25&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;1&lt;/th&gt;
      &lt;td&gt;14.5&lt;/td&gt;
      &lt;td&gt;2.00&lt;/td&gt;
      &lt;td&gt;1&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;2&lt;/th&gt;
      &lt;td&gt;9.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;3&lt;/th&gt;
      &lt;td&gt;3.5&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;4&lt;/th&gt;
      &lt;td&gt;15.0&lt;/td&gt;
      &lt;td&gt;0.00&lt;/td&gt;
      &lt;td&gt;2&lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;In the first two lines we see evidence supporting the 15-20% tip standard
common in the US. The following three lines interestingly show zero tip.
Judging only by these first five lines (a very small sample) we see a strong
correlation here with the payment type. We analyze this a bit more by counting
occurrences in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;payment_type&lt;/span&gt;&lt;/code&gt; column both for the full dataset, and
filtered by zero tip:&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;nyc2015&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;payment_type&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&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="go"&gt;CPU times: user 132 ms, sys: 0 ns, total: 132 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 558 ms&lt;/span&gt;

&lt;span class="go"&gt;1    91574644&lt;/span&gt;
&lt;span class="go"&gt;2    53864648&lt;/span&gt;
&lt;span class="go"&gt;3      503070&lt;/span&gt;
&lt;span class="go"&gt;4      170599&lt;/span&gt;
&lt;span class="go"&gt;5          28&lt;/span&gt;
&lt;span class="go"&gt;Name: payment_type, dtype: int64&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;nyc2015&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;nyc2015&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tip_amount&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;payment_type&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value_counts&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="go"&gt;CPU times: user 212 ms, sys: 4 ms, total: 216 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 1.69 s&lt;/span&gt;

&lt;span class="go"&gt;2    53862557&lt;/span&gt;
&lt;span class="go"&gt;1     3365668&lt;/span&gt;
&lt;span class="go"&gt;3      502025&lt;/span&gt;
&lt;span class="go"&gt;4      170234&lt;/span&gt;
&lt;span class="go"&gt;5          26&lt;/span&gt;
&lt;span class="go"&gt;Name: payment_type, dtype: int64&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We find that almost all zero-tip rides correspond to payment type 2, and that
almost all payment type 2 rides don’t tip. My un-scientific hypothesis here is
payment type 2 corresponds to cash fares and that we’re observing a tendancy of
drivers not to record cash tips. However we would need more domain knowledge
about our data to actually make this claim with any degree of authority.&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/2016/02/22/dask-distributed-part-2.md&lt;/span&gt;, line 373)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="analyze-tips-fractions"&gt;
&lt;h1&gt;Analyze Tips Fractions&lt;/h1&gt;
&lt;p&gt;Lets make a new column, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;tip_fraction&lt;/span&gt;&lt;/code&gt;, and then look at the average of this
column grouped by day of week and grouped by hour of day.&lt;/p&gt;
&lt;p&gt;First, we need to filter out bad rows, both rows with this odd payment type,
and rows with zero fare (there are a surprising number of free cab rides in
NYC.) Second we create a new column equal to the ratio of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;tip_amount&lt;/span&gt; &lt;span class="pre"&gt;/&lt;/span&gt; &lt;span class="pre"&gt;fare_amount&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="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;nyc2015&lt;/span&gt;&lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="n"&gt;nyc2015&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fare_amount&lt;/span&gt; &lt;span class="o"&gt;&amp;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;&amp;amp;&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nyc2015&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;payment_type&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;assign&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;tip_fraction&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tip_amount&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;fare_amount&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Next we choose to groupby the pickup datetime column in order to see how the
average tip fraction changes by day of week and by hour. The groupby and
datetime handling of Pandas makes these operations 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="n"&gt;dayofweek&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tpep_pickup_datetime&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dayofweek&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tip_fraction&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;hour&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tpep_pickup_datetime&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;hour&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tip_fraction&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;dayofweek&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;hour&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;dayofweek&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;hour&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;progress&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dayofweek&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;hour&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/distributed-hdfs-groupby-tip-fraction.gif"&gt;
&lt;p&gt;Grouping by day-of-week doesn’t show anything too striking to my eye. However
I would like to note at how generous NYC cab riders seem to be. A 23-25% tip
can be quite nice:&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;dayofweek&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;tpep_pickup_datetime&lt;/span&gt;
&lt;span class="go"&gt;0    0.237510&lt;/span&gt;
&lt;span class="go"&gt;1    0.236494&lt;/span&gt;
&lt;span class="go"&gt;2    0.236073&lt;/span&gt;
&lt;span class="go"&gt;3    0.246007&lt;/span&gt;
&lt;span class="go"&gt;4    0.242081&lt;/span&gt;
&lt;span class="go"&gt;5    0.232415&lt;/span&gt;
&lt;span class="go"&gt;6    0.259974&lt;/span&gt;
&lt;span class="go"&gt;Name: tip_fraction, dtype: float64&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;But grouping by hour shows that late night and early morning riders are more
likely to tip extravagantly:&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;hour&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;tpep_pickup_datetime&lt;/span&gt;
&lt;span class="go"&gt;0     0.263602&lt;/span&gt;
&lt;span class="go"&gt;1     0.278828&lt;/span&gt;
&lt;span class="go"&gt;2     0.293536&lt;/span&gt;
&lt;span class="go"&gt;3     0.276784&lt;/span&gt;
&lt;span class="go"&gt;4     0.348649&lt;/span&gt;
&lt;span class="go"&gt;5     0.248618&lt;/span&gt;
&lt;span class="go"&gt;6     0.233257&lt;/span&gt;
&lt;span class="go"&gt;7     0.216003&lt;/span&gt;
&lt;span class="go"&gt;8     0.221508&lt;/span&gt;
&lt;span class="go"&gt;9     0.217018&lt;/span&gt;
&lt;span class="go"&gt;10    0.225618&lt;/span&gt;
&lt;span class="go"&gt;11    0.231396&lt;/span&gt;
&lt;span class="go"&gt;12    0.225186&lt;/span&gt;
&lt;span class="go"&gt;13    0.235662&lt;/span&gt;
&lt;span class="go"&gt;14    0.237636&lt;/span&gt;
&lt;span class="go"&gt;15    0.228832&lt;/span&gt;
&lt;span class="go"&gt;16    0.234086&lt;/span&gt;
&lt;span class="go"&gt;17    0.240635&lt;/span&gt;
&lt;span class="go"&gt;18    0.237488&lt;/span&gt;
&lt;span class="go"&gt;19    0.272792&lt;/span&gt;
&lt;span class="go"&gt;20    0.235866&lt;/span&gt;
&lt;span class="go"&gt;21    0.242157&lt;/span&gt;
&lt;span class="go"&gt;22    0.243244&lt;/span&gt;
&lt;span class="go"&gt;23    0.244586&lt;/span&gt;
&lt;span class="go"&gt;Name: tip_fraction, dtype: float64&lt;/span&gt;
&lt;span class="go"&gt;In [24]:&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We plot this with matplotlib and see a nice trough during business hours with a
surge in the early morning with an astonishing peak of 34% at 4am:&lt;/p&gt;
&lt;img src="/images/nyctaxi-2015-hourly-tips.png"&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/2016/02/22/dask-distributed-part-2.md&lt;/span&gt;, line 457)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="performance"&gt;
&lt;h1&gt;Performance&lt;/h1&gt;
&lt;p&gt;Lets dive into a few operations that run at different time scales. This gives
a good understanding of the strengths and limits of the 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="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;nyc2015&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="go"&gt;CPU times: user 4 ms, sys: 0 ns, total: 4 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 20.9 ms&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This head computation is about as fast as a film projector. You could perform
this roundtrip computation between every consecutive frame of a movie; to a
human eye this appears fluid. In the &lt;a class="reference internal" href="../2016/02/17/dask-distributed-part1/"&gt;&lt;span class="doc std std-doc"&gt;last post&lt;/span&gt;&lt;/a&gt;
we asked about how low we could bring latency. In that post we were running
computations from my laptop in California and so were bound by transcontinental
latencies of 200ms. This time, because we’re operating from the cluster, we
can get down to 20ms. We’re only able to be this fast because we touch only a
single data element, the first partition. Things change when we need to touch
the entire 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="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;nyc2015&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 48 ms, sys: 0 ns, total: 48 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 271 ms&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The length computation takes 200-300 ms. This computation takes longer because we
touch every individual partition of the data, of which there are 178. The
scheduler incurs about 1ms of overhead per task, add a bit of latency
and you get the ~200ms total. This means that the scheduler will likely be the
bottleneck whenever computations are very fast, such as is the case for
computing &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;len&lt;/span&gt;&lt;/code&gt;. Really, this is good news; it means that by improving the
scheduler we can reduce these durations even further.&lt;/p&gt;
&lt;p&gt;If you look at the groupby computations above you can add the numbers in the
progress bars to show that we computed around 3000 tasks in around 7s. It
looks like this computation is about half scheduler overhead and about half
bound by actual 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/2016/02/22/dask-distributed-part-2.md&lt;/span&gt;, line 497)&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;We used dask+distributed on a cluster to read CSV data from HDFS
into a dask dataframe. We then used dask.dataframe, which looks identical to
the Pandas dataframe, to manipulate our distributed dataset intuitively and
efficiently.&lt;/p&gt;
&lt;p&gt;We looked a bit at the performance characteristics of simple computations.&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/2016/02/22/dask-distributed-part-2.md&lt;/span&gt;, line 506)&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-doesn-t-work"&gt;
&lt;h1&gt;What doesn’t work&lt;/h1&gt;
&lt;p&gt;As always I’ll have a section like this that honestly says what doesn’t work
well and what I would have done with more time.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Dask dataframe implements a commonly used &lt;em&gt;subset&lt;/em&gt; of Pandas functionality,
not all of it. It’s surprisingly hard to communicate the exact bounds of
this subset to users. Notably, in the distributed setting we don’t have a
shuffle algorithm, so &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;groupby(...).apply(...)&lt;/span&gt;&lt;/code&gt; and some joins are not
yet possible.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;If you want to use threads, you’ll need Pandas 0.18.0 which, at the time of
this writing, was still in release candidate stage. This Pandas release
fixes some important GIL related issues.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The 1ms overhead per task limit is significant. While we can still scale
out to clusters far larger than what we have here, we probably won’t be
able to strongly accelerate very quick operations until we reduce this
number.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We use the &lt;a class="reference external" href="http://hdfs3.readthedocs.org/en/latest/"&gt;hdfs3 library&lt;/a&gt; to read
data from HDFS. This library seems to work great but is new and could use
more active users to flush out bug reports.&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/2016/02/22/dask-distributed-part-2.md&lt;/span&gt;, line 530)&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="links"&gt;
&lt;h1&gt;Links&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask.pydata.org/en/latest/"&gt;dask&lt;/a&gt;, the original project&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://distributed.readthedocs.org/en/latest/"&gt;dask.distributed&lt;/a&gt;, the
distributed memory scheduler powering the cluster computing&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask.pydata.org/en/latest/dataframe.html"&gt;dask.dataframe&lt;/a&gt;, the user
API we’ve used in this post.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://www.nyc.gov/html/tlc/html/about/trip_record_data.shtml"&gt;NYC Taxi Data Downloads&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://hdfs3.readthedocs.org/en/latest"&gt;hdfs3&lt;/a&gt;: Python library we use for
HDFS interations.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The &lt;a class="reference internal" href="../2016/02/17/dask-distributed-part1/"&gt;&lt;span class="doc std std-doc"&gt;previous post&lt;/span&gt;&lt;/a&gt; in this blog series.&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/2016/02/22/dask-distributed-part-2.md&lt;/span&gt;, line 542)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="setup-and-data"&gt;
&lt;h1&gt;Setup and Data&lt;/h1&gt;
&lt;p&gt;You can obtain public data from the New York City Taxi and Limousine Commission
&lt;a class="reference external" href="http://www.nyc.gov/html/tlc/html/about/trip_record_data.shtml"&gt;here&lt;/a&gt;. I
downloaded this onto the head node and dumped it into HDFS with commands like
the following:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;wget&lt;/span&gt; &lt;span class="n"&gt;https&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="o"&gt;//&lt;/span&gt;&lt;span class="n"&gt;storage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;googleapis&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;com&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;tlc&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;trip&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="mi"&gt;2015&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;yellow_tripdata_2015&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="mf"&gt;01..12&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;csv&lt;/span&gt;
&lt;span class="n"&gt;hdfs&lt;/span&gt; &lt;span class="n"&gt;dfs&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;mkdir&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;nyctaxi&lt;/span&gt;
&lt;span class="n"&gt;hdfs&lt;/span&gt; &lt;span class="n"&gt;dfs&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;mkdir&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;nyctaxi&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="mi"&gt;2015&lt;/span&gt;
&lt;span class="n"&gt;hdfs&lt;/span&gt; &lt;span class="n"&gt;dfs&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;put&lt;/span&gt; &lt;span class="n"&gt;yellow&lt;/span&gt;&lt;span class="o"&gt;*.&lt;/span&gt;&lt;span class="n"&gt;csv&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;nyctaxi&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="mi"&gt;2015&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The cluster was hosted on EC2 and was comprised of nine &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;m3.2xlarges&lt;/span&gt;&lt;/code&gt; with 8
cores and 30GB of RAM each. Eight of these nodes were used as workers; they
used processes for parallelism, not threads.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/02/22/dask-distributed-part-2/"/>
    <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>2016-02-22T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2016/02/17/dask-distributed-part1/</id>
    <title>Introducing Dask distributed</title>
    <updated>2016-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 analyze JSON data on a cluster using pure Python projects.&lt;/p&gt;
&lt;p&gt;Dask, a Python library for parallel computing, now works on clusters. During
the past few months I and others have extended dask with a new distributed
memory scheduler. This enables dask’s existing parallel algorithms to scale
across 10s to 100s of nodes, and extends a subset of PyData to distributed
computing. Over the next few weeks I and others will write about this system.
Please note that dask+distributed is developing quickly and so the API is
likely to shift around a bit.&lt;/p&gt;
&lt;p&gt;Today we start simple with the typical cluster computing problem, parsing JSON
records, filtering, and counting events using dask.bag and the new distributed
scheduler. We’ll dive into more advanced problems in future posts.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;A video version of this blogpost is available
&lt;a class="reference external" href="https://www.youtube.com/watch?v=W0Q0uwmYD6o"&gt;here&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2016/02/17/dask-distributed-part1.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 id="github-archive-data-on-s3"&gt;

&lt;p&gt;GitHub releases data dumps of their public event stream as gzipped compressed,
line-delimited, JSON. This data is too large to fit comfortably into memory,
even on a sizable workstation. We could stream it from disk but, due to the
compression and JSON encoding this takes a while and so slogs down interactive
use. For an interactive experience with data like this we need a distributed
cluster.&lt;/p&gt;
&lt;section id="setup-and-data"&gt;
&lt;h2&gt;Setup and Data&lt;/h2&gt;
&lt;p&gt;We provision nine &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;m3.2xlarge&lt;/span&gt;&lt;/code&gt; nodes on EC2. These have eight cores and 30GB
of RAM each. On this cluster we provision one scheduler and nine workers (see
&lt;a class="reference external" href="http://distributed.readthedocs.org/en/latest/setup.html"&gt;setup docs&lt;/a&gt;). (More
on launching in later posts.) We have five months of data, from 2015-01-01 to
2015-05-31 on the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;githubarchive-data&lt;/span&gt;&lt;/code&gt; bucket in S3. This data is publicly
avaialble if you want to play with it on EC2. You can download the full
dataset at https://www.githubarchive.org/ .&lt;/p&gt;
&lt;p&gt;The first record 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="p"&gt;{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;actor&amp;#39;&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;avatar_url&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;https://avatars.githubusercontent.com/u/9152315?&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
   &lt;span class="s1"&gt;&amp;#39;gravatar_id&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;id&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;9152315&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
   &lt;span class="s1"&gt;&amp;#39;login&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;davidjhulse&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
   &lt;span class="s1"&gt;&amp;#39;url&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;https://api.github.com/users/davidjhulse&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;created_at&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;2015-01-01T00:00:00Z&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;id&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;2489368070&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;payload&amp;#39;&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;before&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;86ffa724b4d70fce46e760f8cc080f5ec3d7d85f&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
   &lt;span class="s1"&gt;&amp;#39;commits&amp;#39;&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;author&amp;#39;&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;email&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;david.hulse@live.com&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
      &lt;span class="s1"&gt;&amp;#39;name&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;davidjhulse&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt;
     &lt;span class="s1"&gt;&amp;#39;distinct&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
     &lt;span class="s1"&gt;&amp;#39;message&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;Altered BingBot.jar&lt;/span&gt;&lt;span class="se"&gt;\n\n&lt;/span&gt;&lt;span class="s1"&gt;Fixed issue with multiple account support&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
     &lt;span class="s1"&gt;&amp;#39;sha&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;a9b22a6d80c1e0bb49c1cf75a3c075b642c28f81&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
     &lt;span class="s1"&gt;&amp;#39;url&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;https://api.github.com/repos/davidjhulse/davesbingrewardsbot/commits/a9b22a6d80c1e0bb49c1cf75a3c075b642c28f81&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;}],&lt;/span&gt;
   &lt;span class="s1"&gt;&amp;#39;distinct_size&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="s1"&gt;&amp;#39;head&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;a9b22a6d80c1e0bb49c1cf75a3c075b642c28f81&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
   &lt;span class="s1"&gt;&amp;#39;push_id&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;536740396&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
   &lt;span class="s1"&gt;&amp;#39;ref&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;refs/heads/master&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
   &lt;span class="s1"&gt;&amp;#39;size&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="s1"&gt;&amp;#39;public&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;repo&amp;#39;&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;id&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;28635890&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
   &lt;span class="s1"&gt;&amp;#39;name&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;davidjhulse/davesbingrewardsbot&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
   &lt;span class="s1"&gt;&amp;#39;url&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;https://api.github.com/repos/davidjhulse/davesbingrewardsbot&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;},&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;type&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;PushEvent&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;So we have a large dataset on S3 and a moderate sized play cluster on EC2,
which has access to S3 data at about 100MB/s per node. We’re ready to play.&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/2016/02/17/dask-distributed-part1.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&gt;
&lt;section id="play"&gt;
&lt;h1&gt;Play&lt;/h1&gt;
&lt;p&gt;We start an &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ipython&lt;/span&gt;&lt;/code&gt; interpreter on our local laptop and connect to the
dask scheduler running on the cluster. For the purposes of timing, the cluster
is on the East Coast while the local machine is in California on commercial
broadband internet.&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;Executor&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;s3&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;e&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Executor&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;54.173.84.107:8786&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;e&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;Executor: scheduler=54.173.84.107:8786 workers=72 threads=72&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Our seventy-two worker processes come from nine workers with eight processes
each. We chose processes rather than threads for this task because
computations will be bound by the GIL. We will change this to threads in later
examples.&lt;/p&gt;
&lt;p&gt;We start by loading a single month of data into distributed 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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;json&lt;/span&gt;
&lt;span class="n"&gt;text&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;s3&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_text&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&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;compression&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;gzip&amp;#39;&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;text&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;json&lt;/span&gt;&lt;span class="o"&gt;.&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;records&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;records&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The data lives in S3 in hourly files as gzipped encoded, line delimited JSON.
The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;s3.read_text&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;text.map&lt;/span&gt;&lt;/code&gt; functions produce
&lt;a class="reference external" href="http://dask.pydata.org/en/latest/bag.html"&gt;dask.bag&lt;/a&gt; objects which track our
operations in a lazily built task graph. When we ask the executor to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;persist&lt;/span&gt;&lt;/code&gt;
this collection we ship those tasks off to the scheduler to run on all of the
workers in parallel. The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;persist&lt;/span&gt;&lt;/code&gt; function gives us back another &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.bag&lt;/span&gt;&lt;/code&gt;
pointing to these remotely running results. This persist function returns
immediately, and the computation happens on the cluster in the background
asynchronously. We gain control of our interpreter immediately while the
cluster hums along.&lt;/p&gt;
&lt;p&gt;The cluster takes around 40 seconds to download, decompress, and parse this
data. If you watch the video embedded above you’ll see fancy progress-bars.&lt;/p&gt;
&lt;p&gt;We ask for a single record. This returns in around 200ms, which is fast enough
that it feels instantaneous to a human.&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;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="go"&gt;({&amp;#39;actor&amp;#39;: {&amp;#39;avatar_url&amp;#39;: &amp;#39;https://avatars.githubusercontent.com/u/9152315?&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;gravatar_id&amp;#39;: &amp;#39;&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;id&amp;#39;: 9152315,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;login&amp;#39;: &amp;#39;davidjhulse&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;url&amp;#39;: &amp;#39;https://api.github.com/users/davidjhulse&amp;#39;},&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;created_at&amp;#39;: &amp;#39;2015-01-01T00:00:00Z&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;id&amp;#39;: &amp;#39;2489368070&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;payload&amp;#39;: {&amp;#39;before&amp;#39;: &amp;#39;86ffa724b4d70fce46e760f8cc080f5ec3d7d85f&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;commits&amp;#39;: [{&amp;#39;author&amp;#39;: {&amp;#39;email&amp;#39;: &amp;#39;david.hulse@live.com&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;      &amp;#39;name&amp;#39;: &amp;#39;davidjhulse&amp;#39;},&lt;/span&gt;
&lt;span class="go"&gt;     &amp;#39;distinct&amp;#39;: True,&lt;/span&gt;
&lt;span class="go"&gt;     &amp;#39;message&amp;#39;: &amp;#39;Altered BingBot.jar\n\nFixed issue with multiple account support&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;     &amp;#39;sha&amp;#39;: &amp;#39;a9b22a6d80c1e0bb49c1cf75a3c075b642c28f81&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;     &amp;#39;url&amp;#39;: &amp;#39;https://api.github.com/repos/davidjhulse/davesbingrewardsbot/commits/a9b22a6d80c1e0bb49c1cf75a3c075b642c28f81&amp;#39;}],&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;distinct_size&amp;#39;: 1,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;head&amp;#39;: &amp;#39;a9b22a6d80c1e0bb49c1cf75a3c075b642c28f81&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;push_id&amp;#39;: 536740396,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;ref&amp;#39;: &amp;#39;refs/heads/master&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;size&amp;#39;: 1},&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;public&amp;#39;: True,&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;repo&amp;#39;: {&amp;#39;id&amp;#39;: 28635890,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;name&amp;#39;: &amp;#39;davidjhulse/davesbingrewardsbot&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;url&amp;#39;: &amp;#39;https://api.github.com/repos/davidjhulse/davesbingrewardsbot&amp;#39;},&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;type&amp;#39;: &amp;#39;PushEvent&amp;#39;},)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This particular event is a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;'PushEvent'&lt;/span&gt;&lt;/code&gt;. Let’s quickly see all the kinds of
events. For fun, we’ll also time the interaction:&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;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;type&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 112 ms, sys: 0 ns, total: 112 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 2.41 s&lt;/span&gt;

&lt;span class="go"&gt;[(&amp;#39;ReleaseEvent&amp;#39;, 44312),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;MemberEvent&amp;#39;, 69757),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;IssuesEvent&amp;#39;, 693363),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;PublicEvent&amp;#39;, 14614),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;CreateEvent&amp;#39;, 1651300),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;PullRequestReviewCommentEvent&amp;#39;, 214288),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;PullRequestEvent&amp;#39;, 680879),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;ForkEvent&amp;#39;, 491256),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;DeleteEvent&amp;#39;, 256987),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;PushEvent&amp;#39;, 7028566),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;IssueCommentEvent&amp;#39;, 1322509),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;GollumEvent&amp;#39;, 150861),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;CommitCommentEvent&amp;#39;, 96468),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;WatchEvent&amp;#39;, 1321546)]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And we compute the total count of all commits for this month.&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;records&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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 134 ms, sys: 133 µs, total: 134 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 1.49 s&lt;/span&gt;

&lt;span class="go"&gt;14036706&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We see that it takes a few seconds to walk through the data (and perform all
scheduling overhead.) The scheduler adds about a millisecond overhead per
task, and there are about 1000 partitions/files here (the GitHub data is split
by hour and there are 730 hours in a month) so most of the cost here is
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/2016/02/17/dask-distributed-part1.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="investigate-jupyter"&gt;
&lt;h1&gt;Investigate Jupyter&lt;/h1&gt;
&lt;p&gt;We investigate the activities of &lt;a class="reference external" href="http://jupyter.org/"&gt;Project Jupyter&lt;/a&gt;. We
chose this project because it’s sizable and because we understand the players
involved and so can check our accuracy. This will require us to filter our
data to a much smaller subset, then find popular repositories and members.&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;jupyter&lt;/span&gt; &lt;span class="o"&gt;=&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;filter&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;d&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;repo&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;][&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;name&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;startswith&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;jupyter/&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="go"&gt;                      .repartition(10))&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;jupyter&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;jupyter&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;All records, regardless of event type, have a repository which has a name like
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;'organization/repository'&lt;/span&gt;&lt;/code&gt; in typical GitHub fashion. We filter all records
that start with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;'jupyter/'&lt;/span&gt;&lt;/code&gt;. Additionally, because this dataset is likely
much smaller, we push all of these records into just ten partitions. This
dramatically reduces scheduling overhead. The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;persist&lt;/span&gt;&lt;/code&gt; call hands this
computation off to the scheduler and then gives us back our collection that
points to that computing result. Filtering this month for Jupyter events takes
about 7.5 seconds. Afterwards computations on this subset feel snappy.&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;jupyter&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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 5.19 ms, sys: 97 µs, total: 5.28 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 199 ms&lt;/span&gt;

&lt;span class="go"&gt;747&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;jupyter&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="go"&gt;CPU times: user 7.01 ms, sys: 259 µs, total: 7.27 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 182 ms&lt;/span&gt;

&lt;span class="go"&gt;({&amp;#39;actor&amp;#39;: {&amp;#39;avatar_url&amp;#39;: &amp;#39;https://avatars.githubusercontent.com/u/26679?&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;gravatar_id&amp;#39;: &amp;#39;&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;id&amp;#39;: 26679,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;login&amp;#39;: &amp;#39;marksteve&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;url&amp;#39;: &amp;#39;https://api.github.com/users/marksteve&amp;#39;},&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;created_at&amp;#39;: &amp;#39;2015-01-01T13:25:44Z&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;id&amp;#39;: &amp;#39;2489612400&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;org&amp;#39;: {&amp;#39;avatar_url&amp;#39;: &amp;#39;https://avatars.githubusercontent.com/u/7388996?&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;gravatar_id&amp;#39;: &amp;#39;&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;id&amp;#39;: 7388996,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;login&amp;#39;: &amp;#39;jupyter&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;url&amp;#39;: &amp;#39;https://api.github.com/orgs/jupyter&amp;#39;},&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;payload&amp;#39;: {&amp;#39;action&amp;#39;: &amp;#39;started&amp;#39;},&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;public&amp;#39;: True,&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;repo&amp;#39;: {&amp;#39;id&amp;#39;: 5303123,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;name&amp;#39;: &amp;#39;jupyter/nbviewer&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt;   &amp;#39;url&amp;#39;: &amp;#39;https://api.github.com/repos/jupyter/nbviewer&amp;#39;},&lt;/span&gt;
&lt;span class="go"&gt;  &amp;#39;type&amp;#39;: &amp;#39;WatchEvent&amp;#39;},)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So the first event of the year was by &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;'marksteve'&lt;/span&gt;&lt;/code&gt; who decided to watch the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;'nbviewer'&lt;/span&gt;&lt;/code&gt; repository on new year’s day.&lt;/p&gt;
&lt;p&gt;Notice that these computations take around 200ms. I can’t get below this from
my local machine, so we’re likely bound by communicating to such a remote
location. A 200ms latency is not great if you’re playing a video game, but
it’s decent for interactive computing.&lt;/p&gt;
&lt;p&gt;Here are all of the Jupyter repositories touched in the month of January,&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;jupyter&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;repo&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;pluck&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;name&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;distinct&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="go"&gt;CPU times: user 2.84 ms, sys: 4.03 ms, total: 6.86 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 204 ms&lt;/span&gt;

&lt;span class="go"&gt;[&amp;#39;jupyter/dockerspawner&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/design&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/docker-demo-images&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/jupyterhub&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/configurable-http-proxy&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/nbshot&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/sudospawner&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/colaboratory&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/strata-sv-2015-tutorial&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/tmpnb-deploy&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/nature-demo&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/nbcache&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/jupyter.github.io&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/try.jupyter.org&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/jupyter-drive&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/tmpnb&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/tmpnb-redirector&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/nbgrader&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/nbindex&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/nbviewer&amp;#39;,&lt;/span&gt;
&lt;span class="go"&gt; &amp;#39;jupyter/oauthenticator&amp;#39;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And the top ten most active people on GitHub.&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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;jupyter&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;actor&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;                  .pluck(&amp;#39;login&amp;#39;)&lt;/span&gt;
&lt;span class="go"&gt;                  .frequencies()&lt;/span&gt;
&lt;span class="go"&gt;                  .topk(10, lambda kv: kv[1])&lt;/span&gt;
&lt;span class="go"&gt;                  .compute())&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 8.03 ms, sys: 90 µs, total: 8.12 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 226 ms&lt;/span&gt;

&lt;span class="go"&gt;[(&amp;#39;rgbkrk&amp;#39;, 156),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;minrk&amp;#39;, 87),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;Carreau&amp;#39;, 87),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;KesterTong&amp;#39;, 74),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jhamrick&amp;#39;, 70),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;bollwyvl&amp;#39;, 25),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;pkt&amp;#39;, 18),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;ssanderson&amp;#39;, 13),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;smashwilson&amp;#39;, 13),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;ellisonbg&amp;#39;, 13)]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Nothing too surprising here if you know these folks.&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/2016/02/17/dask-distributed-part1.md&lt;/span&gt;, line 309)&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="full-dataset"&gt;
&lt;h1&gt;Full Dataset&lt;/h1&gt;
&lt;p&gt;The full five months of data is too large to fit in memory, even for this
cluster. When we represent semi-structured data like this with dynamic data
structures like lists and dictionaries there is quite a bit of memory bloat.
Some careful attention to efficient semi-structured storage here could save us
from having to switch to such a large cluster, but that will have to be
the topic of another post.&lt;/p&gt;
&lt;p&gt;Instead, we operate efficiently on this dataset by flowing it through
memory, persisting only the records we care about. The distributed dask
scheduler descends from the single-machine dask scheduler, which was quite good
at flowing through a computation and intelligently removing intermediate
results.&lt;/p&gt;
&lt;p&gt;From a user API perspective, we call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;persist&lt;/span&gt;&lt;/code&gt; only on the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;jupyter&lt;/span&gt;&lt;/code&gt; dataset,
and not the full &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;records&lt;/span&gt;&lt;/code&gt; 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="n"&gt;full&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;s3&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_text&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&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;compression&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;gzip&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;jupyter&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;full&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="k"&gt;lambda&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;d&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;repo&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;][&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;name&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;startswith&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;jupyter/&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="go"&gt;                   .repartition(10))&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;jupyter&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;e&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;jupyter&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;It takes 2m36s to download, decompress, and parse the five months of publicly
available GitHub events for all Jupyter events on nine &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;m3.2xlarges&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;There were seven thousand such events.&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;jupyter&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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;7065&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We find which repositories saw the most activity during that time:&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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;jupyter&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;repo&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;                  .pluck(&amp;#39;name&amp;#39;)&lt;/span&gt;
&lt;span class="go"&gt;                  .frequencies()&lt;/span&gt;
&lt;span class="go"&gt;                  .topk(20, lambda kv: kv[1])&lt;/span&gt;
&lt;span class="go"&gt;                  .compute())&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 6.98 ms, sys: 474 µs, total: 7.46 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 219 ms&lt;/span&gt;

&lt;span class="go"&gt;[(&amp;#39;jupyter/jupyterhub&amp;#39;, 1262),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/nbgrader&amp;#39;, 1235),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/nbviewer&amp;#39;, 846),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter_notebook&amp;#39;, 507),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter-drive&amp;#39;, 505),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/notebook&amp;#39;, 451),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/docker-demo-images&amp;#39;, 363),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/tmpnb&amp;#39;, 284),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter_client&amp;#39;, 162),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/dockerspawner&amp;#39;, 149),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/colaboratory&amp;#39;, 134),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter_core&amp;#39;, 127),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/strata-sv-2015-tutorial&amp;#39;, 108),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter_nbconvert&amp;#39;, 103),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/configurable-http-proxy&amp;#39;, 89),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/hubpress.io&amp;#39;, 85),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter.github.io&amp;#39;, 84),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/tmpnb-deploy&amp;#39;, 76),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/nbconvert&amp;#39;, 66),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter_qtconsole&amp;#39;, 59)]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We see that projects like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;jupyterhub&lt;/span&gt;&lt;/code&gt; were quite active during that time
while, surprisingly, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;nbconvert&lt;/span&gt;&lt;/code&gt; saw relatively little action.&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/2016/02/17/dask-distributed-part1.md&lt;/span&gt;, line 383)&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="local-data"&gt;
&lt;h1&gt;Local Data&lt;/h1&gt;
&lt;p&gt;The Jupyter data is quite small and easily fits in a single machine. Let’s
bring the data to our local machine so that we can compare times:&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;L&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;jupyter&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 4.74 s, sys: 10.9 s, total: 15.7 s&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 30.2 s&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;It takes surprisingly long to download the data, but once its here, we can
iterate far more quickly with basic 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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;toolz.curried&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;pluck&lt;/span&gt;&lt;span class="p"&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;topk&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;pipe&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;pipe&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;L&lt;/span&gt;&lt;span class="p"&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;repo&amp;#39;&lt;/span&gt;&lt;span class="p"&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;name&amp;#39;&lt;/span&gt;&lt;span class="p"&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;               dict.items, topk(20, key=lambda kv: kv[1]), list)&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 11.8 ms, sys: 0 ns, total: 11.8 ms&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 11.5 ms&lt;/span&gt;

&lt;span class="go"&gt;[(&amp;#39;jupyter/jupyterhub&amp;#39;, 1262),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/nbgrader&amp;#39;, 1235),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/nbviewer&amp;#39;, 846),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter_notebook&amp;#39;, 507),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter-drive&amp;#39;, 505),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/notebook&amp;#39;, 451),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/docker-demo-images&amp;#39;, 363),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/tmpnb&amp;#39;, 284),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter_client&amp;#39;, 162),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/dockerspawner&amp;#39;, 149),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/colaboratory&amp;#39;, 134),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter_core&amp;#39;, 127),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/strata-sv-2015-tutorial&amp;#39;, 108),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter_nbconvert&amp;#39;, 103),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/configurable-http-proxy&amp;#39;, 89),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/hubpress.io&amp;#39;, 85),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter.github.io&amp;#39;, 84),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/tmpnb-deploy&amp;#39;, 76),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/nbconvert&amp;#39;, 66),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;jupyter/jupyter_qtconsole&amp;#39;, 59)]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The difference here is 20x, which is a good reminder that, once you no longer
have a large problem you should probably eschew distributed systems and act
locally.&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/2016/02/17/dask-distributed-part1.md&lt;/span&gt;, line 430)&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;Downloading, decompressing, parsing, filtering, and counting JSON records
is the new wordcount. It’s the first problem anyone sees. Fortunately it’s
both easy to solve and the common case. Woo hoo!&lt;/p&gt;
&lt;p&gt;Here we saw that dask+distributed handle the common case decently well and with
a Pure Python stack. Typically Python users rely on a JVM technology like
Hadoop/Spark/Storm to distribute their computations. Here we have Python
distributing Python; there are some usability gains to be had here like nice
stack traces, a bit less serialization overhead, and attention to other
Pythonic style choices.&lt;/p&gt;
&lt;p&gt;Over the next few posts I intend to deviate from this common case. Most “Big
Data” technologies were designed to solve typical data munging problems found
in web companies or with simple database operations in mind. Python users care
about these things too, but they also reach out to a wide variety of fields.
In dask+distributed development we care about the common case, but also support
less traditional workflows that are commonly found in the life, physical, and
algorithmic sciences.&lt;/p&gt;
&lt;p&gt;By designing to support these more extreme cases we’ve nailed some common pain
points in current distributed systems. Today we’ve seen low latency and remote
control; in the future we’ll see others.&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/2016/02/17/dask-distributed-part1.md&lt;/span&gt;, line 455)&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-doesn-t-work"&gt;
&lt;h1&gt;What doesn’t work&lt;/h1&gt;
&lt;p&gt;I’ll have an honest section like this at the end of each upcoming post
describing what doesn’t work, what still feels broken, or what I would have
done differently with more time.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The imports for dask and distributed are still strange. They’re two
separate codebases that play very nicely together. Unfortunately the
functionality you need is sometimes in one or in the other and it’s not
immediately clear to the novice user where to go. For example dask.bag, the
collection we’re using for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;records&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;jupyter&lt;/span&gt;&lt;/code&gt;, etc. is in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; but the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;s3&lt;/span&gt;&lt;/code&gt; module is within the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed&lt;/span&gt;&lt;/code&gt; library. We’ll have to merge things
at some point in the near-to-moderate future. Ditto for the API: there are
compute methods both on the dask collections (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;records.compute()&lt;/span&gt;&lt;/code&gt;) and on
the distributed executor (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;e.compute(records)&lt;/span&gt;&lt;/code&gt;) that behave slightly
differently.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We lack an efficient distributed shuffle algorithm. This is very important
if you want to use operations like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.groupby&lt;/span&gt;&lt;/code&gt; (which you should avoid
anyway). The user API here doesn’t even cleanly warn users that this is
missing in the distributed case which is kind of a mess. (It works fine on a
single machine.) Efficient alternatives like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;foldby&lt;/span&gt;&lt;/code&gt; &lt;em&gt;are&lt;/em&gt; available.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;I would have liked to run this experiment directly on the cluster to see
how low we could have gone below the 200ms barrier we ran into here.&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/2016/02/17/dask-distributed-part1.md&lt;/span&gt;, line 481)&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="links"&gt;
&lt;h1&gt;Links&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://dask.pydata.org/en/latest/"&gt;dask&lt;/a&gt;, the original project&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://distributed.readthedocs.org/en/latest/"&gt;dask.distributed&lt;/a&gt;, the
distributed memory scheduler powering the cluster computing&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask.pydata.org/en/latest/bag.html"&gt;dask.bag&lt;/a&gt;, the user API we’ve
used in this post.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;This post largely repeats work by &lt;a class="reference external" href="https://github.com/cowlicks"&gt;Blake Griffith&lt;/a&gt; in a
&lt;a class="reference external" href="https://www.continuum.io/content/dask-distributed-and-anaconda-cluster"&gt;similar post&lt;/a&gt;
last year with an older iteration of the dask distributed scheduler&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2016/02/17/dask-distributed-part1/"/>
    <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>2016-02-17T00:00:00+00:00</published>
  </entry>
  <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>
  <entry>
    <id>https://blog.dask.org/2014/12/30/Towards-OOC-Frontend/</id>
    <title>Towards Out-of-core ND-Arrays -- Frontend</title>
    <updated>2014-12-30T00: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; Blaze adds usability to our last post on out-of-core ND-Arrays&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;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/2014/12/30/Towards-OOC-Frontend.md&lt;/span&gt;, line 17)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="setup"&gt;

&lt;p&gt;This follows my &lt;a class="reference internal" href="../2014/12/27/Towards-OOC/"&gt;&lt;span class="doc std std-doc"&gt;last
post&lt;/span&gt;&lt;/a&gt; designing a simple
task scheduler for use with out-of-core (or distributed) nd-arrays. We
encoded tasks-with-data-dependencies as simple dictionaries. We then
built functions to create dictionaries that describe blocked array operations.
We found that this was an effective-but-unfriendly way to solve some
important-but-cumbersome problems.&lt;/p&gt;
&lt;p&gt;This post sugars the programming experience with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;blaze&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;into&lt;/span&gt;&lt;/code&gt; to give a
numpy-like experience out-of-core.&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/2014/12/30/Towards-OOC-Frontend.md&lt;/span&gt;, line 30)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="old-low-level-code"&gt;
&lt;h1&gt;Old low-level code&lt;/h1&gt;
&lt;p&gt;Here is the code we wrote for an
out-of-core transpose/dot-product (actually a symmetric rank-k update).&lt;/p&gt;
&lt;section id="create-random-array-on-disk"&gt;
&lt;h2&gt;Create random array on disk&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;bcolz&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="n"&gt;b&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;bcolz&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;carray&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;empty&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;0&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="n"&gt;rootdir&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;A.bcolz&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;chunklen&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="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="n"&gt;b&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;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;rand&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;b&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;flush&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="define-computation-a-t-a"&gt;
&lt;h2&gt;Define computation &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;A.T&lt;/span&gt; &lt;span class="pre"&gt;*&lt;/span&gt; &lt;span class="pre"&gt;A&lt;/span&gt;&lt;/code&gt;&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="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;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="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;update&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;getem&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;blocksize&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;shape&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;shape&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

&lt;span class="c1"&gt;# Add A.T into the mix&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;update&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;top&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="s1"&gt;&amp;#39;At&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;ij&amp;#39;&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="s1"&gt;&amp;#39;ji&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;numblocks&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;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;1&lt;/span&gt;&lt;span class="p"&gt;)}))&lt;/span&gt;

&lt;span class="c1"&gt;# Dot product A.T * A&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;update&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;top&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="s1"&gt;&amp;#39;AtA&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;ik&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;At&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;ij&amp;#39;&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="s1"&gt;&amp;#39;jk&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
         &lt;span class="n"&gt;numblocks&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;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;1&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;At&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;1&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/2014/12/30/Towards-OOC-Frontend.md&lt;/span&gt;, line 61)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="new-pleasant-feeling-code-with-blaze"&gt;
&lt;h1&gt;New pleasant feeling code with Blaze&lt;/h1&gt;
&lt;section id="targetting-users"&gt;
&lt;h2&gt;Targetting users&lt;/h2&gt;
&lt;p&gt;The last section “Define computation” is written in a style that is great for
library writers and automated systems but is challenging to users
accustomed to Matlab/NumPy or R/Pandas style.&lt;/p&gt;
&lt;p&gt;We wrap this process with Blaze, an extensible front-end for analytic
computations&lt;/p&gt;
&lt;/section&gt;
&lt;section id="redefine-computation-a-t-a-with-blaze"&gt;
&lt;h2&gt;Redefine computation &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;A.T&lt;/span&gt; &lt;span class="pre"&gt;*&lt;/span&gt; &lt;span class="pre"&gt;A&lt;/span&gt;&lt;/code&gt; with Blaze&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="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="c1"&gt;# a proxy object holding on to a dask dict&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="o"&gt;*&lt;/span&gt;

&lt;span class="c1"&gt;# Load data into dask dictionaries&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;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;A.bcolz&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;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;dA&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Wrap with blaze.Data&lt;/span&gt;

&lt;span class="c1"&gt;# Describe computation in friendly numpy style&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;A&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="c1"&gt;# Compute results&lt;/span&gt;
&lt;span class="o"&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;compute&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;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;2&lt;/span&gt;&lt;span class="nb"&gt;min&lt;/span&gt; &lt;span class="mi"&gt;57&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;6.4&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;3&lt;/span&gt;&lt;span class="nb"&gt;min&lt;/span&gt; &lt;span class="mi"&gt;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="mi"&gt;2&lt;/span&gt;&lt;span class="nb"&gt;min&lt;/span&gt; &lt;span class="mi"&gt;50&lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;
&lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="p"&gt;([[&lt;/span&gt; &lt;span class="mf"&gt;334071.93541158&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250297.16968262&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250404.87729587&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="mf"&gt;250436.85274716&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250330.64262904&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250590.98832611&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
       &lt;span class="p"&gt;[&lt;/span&gt; &lt;span class="mf"&gt;250297.16968262&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;333451.72293343&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;249978.2751824&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="mf"&gt;250103.20601281&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250014.96660956&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250251.0146828&lt;/span&gt; &lt;span class="p"&gt;],&lt;/span&gt;
       &lt;span class="p"&gt;[&lt;/span&gt; &lt;span class="mf"&gt;250404.87729587&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;249978.2751824&lt;/span&gt; &lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;333279.76376277&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="mf"&gt;249961.44796719&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250061.8068036&lt;/span&gt; &lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250125.80971858&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="p"&gt;[&lt;/span&gt; &lt;span class="mf"&gt;250436.85274716&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250103.20601281&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;249961.44796719&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="mf"&gt;333444.797894&lt;/span&gt;  &lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250021.78528189&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250147.12015207&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
       &lt;span class="p"&gt;[&lt;/span&gt; &lt;span class="mf"&gt;250330.64262904&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250014.96660956&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250061.8068036&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="mf"&gt;250021.78528189&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;333240.10323875&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250307.86236815&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
       &lt;span class="p"&gt;[&lt;/span&gt; &lt;span class="mf"&gt;250590.98832611&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250251.0146828&lt;/span&gt; &lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250125.80971858&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="mf"&gt;250147.12015207&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;250307.86236815&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="mf"&gt;333467.87105673&lt;/span&gt;&lt;span class="p"&gt;]])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="under-the-hood"&gt;
&lt;h2&gt;Under the hood&lt;/h2&gt;
&lt;p&gt;Under the hood, Blaze creates the same dask dicts we created by hand last time.
I’ve doctored the result rendered here to include suggestive names.&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;compute&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;post_compute&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;False&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;A&amp;#39;: carray((10000000, 1000), float64), ...&lt;/span&gt;
&lt;span class="go"&gt; ...&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;A&amp;#39;, 0, 0): (ndget, &amp;#39;A&amp;#39;, (1000, 1000), 0, 0),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;A&amp;#39;, 1, 0): (ndget, &amp;#39;A&amp;#39;, (1000, 1000), 1, 0),&lt;/span&gt;
&lt;span class="go"&gt; ...&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;At&amp;#39;, 0, 0): (np.transpose, (&amp;#39;A&amp;#39;, 0, 0)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;At&amp;#39;, 0, 1): (np.transpose, (&amp;#39;A&amp;#39;, 1, 0)),&lt;/span&gt;
&lt;span class="go"&gt; ...&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;AtA&amp;#39;, 0, 0): (dotmany, [(&amp;#39;At&amp;#39;, 0, 0), (&amp;#39;At&amp;#39;, 0, 1), (&amp;#39;At&amp;#39;, 0, 2), ...],&lt;/span&gt;
&lt;span class="go"&gt;                          [(&amp;#39;A&amp;#39;, 0, 0),  (&amp;#39;A&amp;#39;, 1, 0),  (&amp;#39;A&amp;#39;, 2, 0), ...])&lt;/span&gt;
&lt;span class="go"&gt;}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We then compute this sequentially on a single core. However we could have
passed this on to a distributed system. This result contains all necessary
information to go from on-disk arrays to computed result in whatever manner you
choose.&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/2014/12/30/Towards-OOC-Frontend.md&lt;/span&gt;, line 129)&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="separating-backend-from-frontend"&gt;
&lt;h1&gt;Separating Backend from Frontend&lt;/h1&gt;
&lt;p&gt;Recall that Blaze is an extensible front-end to data analytics technologies.
It lets us wrap messy computational APIs with a pleasant and familiar
user-centric API. Extending Blaze to dask dicts was the straightforward work
of an afternoon. This separation allows us to continue to build out
dask-oriented solutions without worrying about user-interface. By separating
backend work from frontend work we allow both sides to be cleaner and to
progress more swiftly.&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/2014/12/30/Towards-OOC-Frontend.md&lt;/span&gt;, line 139)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="future-work"&gt;
&lt;h1&gt;Future work&lt;/h1&gt;
&lt;p&gt;I’m on vacation right now. Work for recent posts has been done in evenings
while watching TV with the family. It isn’t particularly robust. Still, it’s
exciting how effective this approach has been with relatively little effort.&lt;/p&gt;
&lt;p&gt;Perhaps now would be a good time to mention that Continuum has ample grant
funding. We’re looking for people who want to create usable large-scale data
analytics tools. For what it’s worth, I quit my academic postdoc to work on
this and couldn’t be happier with the switch.&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/2014/12/30/Towards-OOC-Frontend.md&lt;/span&gt;, line 150)&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="source"&gt;
&lt;h1&gt;Source&lt;/h1&gt;
&lt;p&gt;This code is experimental and buggy. I don’t expect it to stay around for
forever in it’s current form (it’ll improve). Still, if you’re reading this
when it comes out then you might want to check out the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/mrocklin/dask"&gt;master branch on dask&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/mrocklin/blaze/tree/array-expr"&gt;array-expr branch on my blaze fork&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2014/12/30/Towards-OOC-Frontend/"/>
    <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>2014-12-30T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2014/12/27/Towards-OOC/</id>
    <title>Towards Out-of-core ND-Arrays</title>
    <updated>2014-12-27T00: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 propose a system for task-centered computation, show an example
with out-of-core nd-arrays, and ask for comments.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: This post is not user-focused. It is intended for library writers.&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/2014/12/27/Towards-OOC.md&lt;/span&gt;, line 17)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="motivation"&gt;

&lt;p&gt;Recent notebooks (links
&lt;a class="reference external" href="http://nbviewer.ipython.org/url/blaze.pydata.org/en/latest/_static/notebooks/timings-csv.ipynb"&gt;1&lt;/a&gt;,
&lt;a class="reference external" href="http://nbviewer.ipython.org/url/blaze.pydata.org/en/latest/_static/notebooks/timings-bcolz.ipynb"&gt;2&lt;/a&gt;)
describe how Blaze handles out-of-core single-dataset tabular computations in
the following stages.&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Partition the dataset into chunks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Apply some computation on each chunk&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Concatenate the results (hopefully smaller)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Apply another computation into the concatenated result&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Steps 2 and 4 require symbolic analysis of &lt;em&gt;what&lt;/em&gt; work should be done; Blaze
does this well. Steps 1 and 3 are more about coordinating &lt;em&gt;where&lt;/em&gt; data goes
and &lt;em&gt;when&lt;/em&gt; computation executes.&lt;/p&gt;
&lt;p&gt;This setup is effective for a broad class of single-dataset tabular
computations. It fails for more complex cases. Blaze doesn’t currently have a
good target for describing complex inter-block data dependencies. The model
for breaking apart data and arranging computations (1 and 3) is too simple.&lt;/p&gt;
&lt;p&gt;A good example of a complex case is an nd-array matrix-matrix multiply / dot
product / tensor contraction. In this case a blocked approach has a more
complex communication pattern. This post is about finding a simple framework
that allows us to express these patterns. It’s about finding a replacement for
steps 1 and 3 above.&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/2014/12/27/Towards-OOC.md&lt;/span&gt;, line 45)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="task-scheduling"&gt;
&lt;h1&gt;Task Scheduling&lt;/h1&gt;
&lt;p&gt;The common solution to this problem is to describe the computation as a
bipartite directed acyclic graph where nodes are computations and data and
edges indicate which pieces of data a computation takes as input and delivers
as output.&lt;/p&gt;
&lt;p&gt;Many solutions to this problem exist, both theoretical algorithms and
implemented software. Forgive me for describing yet-another system.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2014/12/27/Towards-OOC.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="dask"&gt;
&lt;h1&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt;&lt;/h1&gt;
&lt;p&gt;&lt;img src="/images/dask-simple.png"
     align="right"&gt;&lt;/p&gt;
&lt;p&gt;We use a low-tech representation of a task dependency graph.
We use a dictionary of key-value pairs where keys are any hashable identifier
and values are one of the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;A value, like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;1&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A tuple containing a function and arguments, like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(inc,&lt;/span&gt; &lt;span class="pre"&gt;1)&lt;/span&gt;&lt;/code&gt;. This is like
an s-expression and should be interpreted as an unevaluated &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;inc(1)&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A tuple containing a function and arguments. Arguments may include other
keys, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(inc,&lt;/span&gt; &lt;span class="pre"&gt;'my_key')&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This is more clear with an example. We show this example on the right.&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;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;1&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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inc&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;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;y&amp;#39;&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The &lt;a class="reference external" href="http://github.com/mrocklin/dask"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; library&lt;/a&gt; contains a small
reference implementation to get values associated to keys in this task 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&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;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="s1"&gt;&amp;#39;x&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;1&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;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="s1"&gt;&amp;#39;y&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Triggers computation&lt;/span&gt;
&lt;span class="go"&gt;2&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;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="s1"&gt;&amp;#39;z&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Triggers computation&lt;/span&gt;
&lt;span class="go"&gt;12&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In principle this could be executed by a variety of different implementations
each with different solutions for distributed computing, caching, etc..&lt;/p&gt;
&lt;p&gt;Dask also includes convenience functions to help build this 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="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;d&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;add&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="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;Although this is mainly to help those who feel uncomfortable putting the
parenthesis on the left side of a function call to avoid immediate execution&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="c1"&gt;# d[&amp;#39;a&amp;#39;] =  add( &amp;#39;x&amp;#39;, &amp;#39;y&amp;#39;)  # intend this&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;a&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="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="c1"&gt;# but write this to avoid immediate execution&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/2014/12/27/Towards-OOC.md&lt;/span&gt;, line 108)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="why-low-tech"&gt;
&lt;h1&gt;Why low tech?&lt;/h1&gt;
&lt;p&gt;These “graphs” are just dictionaries of tuples. Notably, we imported &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt;
&lt;em&gt;after&lt;/em&gt; we built our graph. The framework investment is very light.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Q&lt;/strong&gt;: Why don’t we build &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Task&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Data&lt;/span&gt;&lt;/code&gt; classes and construct a Python
framework to represent these things formally?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;A&lt;/strong&gt;: Because people have to learn and buy in to that framework and that’s
hard to sell. Dictionaries are easier to sell. They’re also easy to translate
into other systems. Additionally, I was able to write a reference
implementation in &lt;a class="reference external" href="https://github.com/mrocklin/dask/blob/master/dask/core.py#L36-L68"&gt;a couple dozen lines&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;It’s easy to build functions that create &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dict&lt;/span&gt;&lt;/code&gt;s like this for various
applications. There is a decent chance that, if you’ve made it this far in
this blogpost, you already understand the spec.&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/2014/12/27/Towards-OOC.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="nd-arrays"&gt;
&lt;h1&gt;ND-Arrays&lt;/h1&gt;
&lt;p&gt;I want to encode out-of-core ND-Array algorithms as data.
I’ve written a few functions that create dask-style dictionaries to help me
describe a decent class of blocked nd-array computations.&lt;/p&gt;
&lt;p&gt;The following section is a specific example applying these ideas to the domain
of array computing. This is just one application and not core to the idea
of task scheduling. The core ideas to task scheduling and the dask
implementation have already been covered above.&lt;/p&gt;
&lt;section id="getting-blocks-from-an-array"&gt;
&lt;h2&gt;Getting blocks from an array&lt;/h2&gt;
&lt;p&gt;First, we break apart a large possibly out-of-core array into blocks.
For convenience in these examples we work in in-memory numpy arrays rather than
on-disk arrays. Jump to the end if you’d like to see a real OOC dot product
on on-disk data.&lt;/p&gt;
&lt;p&gt;We make a function &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ndget&lt;/span&gt;&lt;/code&gt; to pull out a single block&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;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;arange&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;24&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;reshape&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;6&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="go"&gt;array([[ 0,  1,  2,  3,  4,  5],&lt;/span&gt;
&lt;span class="go"&gt;       [ 6,  7,  8,  9, 10, 11],&lt;/span&gt;
&lt;span class="go"&gt;       [12, 13, 14, 15, 16, 17],&lt;/span&gt;
&lt;span class="go"&gt;       [18, 19, 20, 21, 22, 23]])&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="c1"&gt;# Cutting into (2, 3) shaped blocks, get the (0, 0)th block&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ndget&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="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;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="go"&gt;array([[0, 1, 2],&lt;/span&gt;
&lt;span class="go"&gt;       [6, 7, 8]])&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="c1"&gt;# Cutting into (2, 3) shaped blocks, get the (1, 0)th block&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ndget&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="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;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;span class="go"&gt;array([[12, 13, 14],&lt;/span&gt;
&lt;span class="go"&gt;       [18, 19, 20]])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We now make a function &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;getem&lt;/span&gt;&lt;/code&gt; that makes a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dict&lt;/span&gt;&lt;/code&gt; that uses this &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ndget&lt;/span&gt;&lt;/code&gt;
function to pull out all of the blocks. This creates more &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;keys&lt;/span&gt;&lt;/code&gt; in our
dictionary, one for each block. We name each key by the key of the array
followed by a block-index.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;getem&lt;/span&gt;&lt;/code&gt;: Given a large possibly out-of-core array and a blocksize, pull
apart that array into many small blocks&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="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="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;  &lt;span class="c1"&gt;# map the key &amp;#39;X&amp;#39; to the data x&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;getem&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;blocksize&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;3&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;4&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="go"&gt;{(&amp;#39;X&amp;#39;, 0, 0): (ndget, &amp;#39;X&amp;#39;, (2, 3), 0, 0),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;X&amp;#39;, 1, 0): (ndget, &amp;#39;X&amp;#39;, (2, 3), 1, 0),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;X&amp;#39;, 1, 1): (ndget, &amp;#39;X&amp;#39;, (2, 3), 1, 1),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;X&amp;#39;, 0, 1): (ndget, &amp;#39;X&amp;#39;, (2, 3), 0, 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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;update&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;_&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# dump in getem dict&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So we have a single original array, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x&lt;/span&gt;&lt;/code&gt; and using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;getem&lt;/span&gt;&lt;/code&gt; we describe how to
get many blocks out of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x&lt;/span&gt;&lt;/code&gt; using the function &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ndget&lt;/span&gt;&lt;/code&gt; for on each block.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ndget&lt;/span&gt;&lt;/code&gt; actually does work on data&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;getem&lt;/span&gt;&lt;/code&gt; creates dask dict that describes on what ndget should operate&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We haven’t done work yet. We only do work when we finally call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.get&lt;/span&gt;&lt;/code&gt; on
the appropriate key for one of the blocks.&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;dask&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;d&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="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;span class="c1"&gt;# Get block corresponding to key (&amp;#39;X&amp;#39; ,1, 0)&lt;/span&gt;
&lt;span class="go"&gt;array([[12, 13, 14],&lt;/span&gt;
&lt;span class="go"&gt;       [18, 19, 20]])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;numpy.ndarrays&lt;/span&gt;&lt;/code&gt; for convenience. This would have worked with anything
that supports numpy-style indexing, including out-of-core structures like
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;h5py.Dataset&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;tables.Array&lt;/span&gt;&lt;/code&gt;, or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;bcolz.carray&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="example-embarrassingly-parallel-computation"&gt;
&lt;h2&gt;Example: Embarrassingly Parallel Computation&lt;/h2&gt;
&lt;p&gt;If we have a simple function&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;inc&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="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;That we want to apply to all blocks of the dataset we could, in principle,
add the following to our dictionary.&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;update&lt;/span&gt;&lt;span class="p"&gt;({(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;X-plus-1&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="n"&gt;j&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;inc&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="n"&gt;j&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;2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;                                                     for j in range(2)})&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;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="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;X-plus-1&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="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="go"&gt;array([[1, 2, 3],&lt;/span&gt;
&lt;span class="go"&gt;       [7, 8, 9]])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Our use of keys like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;('name',&lt;/span&gt; &lt;span class="pre"&gt;i,&lt;/span&gt; &lt;span class="pre"&gt;j)&lt;/span&gt;&lt;/code&gt; to refer to the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;i,jth&lt;/span&gt;&lt;/code&gt; block of an array is
an incidental convention and not intrinsic to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; itself. This use of
tuples as keys should not be confused with the use of tuples in values to
encode unevaluated functions.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="index-expressions"&gt;
&lt;h2&gt;Index expressions&lt;/h2&gt;
&lt;p&gt;A broad class of array computations can be written with index expressions&lt;/p&gt;
&lt;p&gt;$&lt;span class="math notranslate nohighlight"&gt;\( Z*{ij} = X*{ji} \;\;\)&lt;/span&gt;$ – Matrix transpose&lt;/p&gt;
&lt;p&gt;$&lt;span class="math notranslate nohighlight"&gt;\( Z*{ik} = \sum_j X*{ij} Y\_{jk} \;\; \)&lt;/span&gt;$ – Matrix-matrix multiply&lt;/p&gt;
&lt;p&gt;Fortunately, the blocked versions of these algorithms look pretty much the
same. To leverage this structure we made the function &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;top&lt;/span&gt;&lt;/code&gt; for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;t&lt;/span&gt;&lt;/code&gt;ensor
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;op&lt;/span&gt;&lt;/code&gt;erations (ideas for a better name welcome). This writes index operations
like the following for blocked transpose:&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;top&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="s1"&gt;&amp;#39;Z&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;ji&amp;#39;&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;ij&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;numblocks&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="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="go"&gt;{(&amp;#39;Z&amp;#39;, 0, 0): (numpy.transpose, (&amp;#39;X&amp;#39;, 0, 0)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;Z&amp;#39;, 0, 1): (numpy.transpose, (&amp;#39;X&amp;#39;, 1, 0)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;Z&amp;#39;, 1, 0): (numpy.transpose, (&amp;#39;X&amp;#39;, 0, 1)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;Z&amp;#39;, 1, 1): (numpy.transpose, (&amp;#39;X&amp;#39;, 1, 1))}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The first argument &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.transpose&lt;/span&gt;&lt;/code&gt; is the function to apply to each block.
The second and third arguments are the name and index pattern of the output.
The succeeding arguments are the key and index pattern of the inputs. In this
case the index pattern is the reverse. We map the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ij&lt;/span&gt;&lt;/code&gt;th block to the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ji&lt;/span&gt;&lt;/code&gt;th
block of the output after we call the function &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.transpose&lt;/span&gt;&lt;/code&gt;.
Finally we have the numblocks keyword arguments that give the block structure
of the inputs. Index structure can be any iterable.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="matrix-multiply"&gt;
&lt;h2&gt;Matrix Multiply&lt;/h2&gt;
&lt;p&gt;We represent tensor contractions like matrix-matrix multiply with indices that
are repeated in the inputs and missing in the output like the following. In
the following example the index &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;'j'&lt;/span&gt;&lt;/code&gt; is a contracted dummy index.&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;top&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;Z&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;ik&amp;#39;&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="s1"&gt;&amp;#39;ij&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Y&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;jk&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;numblocks&lt;/span&gt;&lt;span class="o"&gt;=...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In this case the function receives an iterator of blocks of data that iterate
over the dummy index, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;j&lt;/span&gt;&lt;/code&gt;. We make such a function to take iterators of square
array blocks, dot product the pairs, and then sum the results. This is the
inner-most loop of a conventional blocked-matrix-matrix multiply algorithm.&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;dotmany&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="k"&gt;return&lt;/span&gt; &lt;span class="nb"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;np&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;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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;By combining this per-block function with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;top&lt;/span&gt;&lt;/code&gt; we get an out-of-core 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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;top&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="s1"&gt;&amp;#39;Z&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;ik&amp;#39;&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;ij&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="s1"&gt;&amp;#39;jk&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="n"&gt;numblocks&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="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="go"&gt;                                                              &amp;#39;Y&amp;#39;: (2, 2)})&lt;/span&gt;
&lt;span class="go"&gt;{(&amp;#39;Z&amp;#39;, 0, 0): (dotmany, [(&amp;#39;X&amp;#39;, 0, 0), (&amp;#39;X&amp;#39;, 0, 1)],&lt;/span&gt;
&lt;span class="go"&gt;                        [(&amp;#39;Y&amp;#39;, 0, 0), (&amp;#39;Y&amp;#39;, 1, 0)]),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;Z&amp;#39;, 0, 1): (dotmany, [(&amp;#39;X&amp;#39;, 0, 0), (&amp;#39;X&amp;#39;, 0, 1)],&lt;/span&gt;
&lt;span class="go"&gt;                        [(&amp;#39;Y&amp;#39;, 0, 1), (&amp;#39;Y&amp;#39;, 1, 1)]),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;Z&amp;#39;, 1, 0): (dotmany, [(&amp;#39;X&amp;#39;, 1, 0), (&amp;#39;X&amp;#39;, 1, 1)],&lt;/span&gt;
&lt;span class="go"&gt;                        [(&amp;#39;Y&amp;#39;, 0, 0), (&amp;#39;Y&amp;#39;, 1, 0)]),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;Z&amp;#39;, 1, 1): (dotmany, [(&amp;#39;X&amp;#39;, 1, 0), (&amp;#39;X&amp;#39;, 1, 1)],&lt;/span&gt;
&lt;span class="go"&gt;                        [(&amp;#39;Y&amp;#39;, 0, 1), (&amp;#39;Y&amp;#39;, 1, 1)])}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;top&lt;/span&gt;&lt;/code&gt; function inspects the index structure of the inputs and outputs and
constructs dictionaries that reflect this structure, matching indices between
keys and creating lists of keys over dummy indices like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;j&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;And that was it, we have an out-of-core dot product. Calling dask.get on these
keys results in out-of-core execution.&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/2014/12/27/Towards-OOC.md&lt;/span&gt;, line 298)&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="full-example"&gt;
&lt;h1&gt;Full example&lt;/h1&gt;
&lt;p&gt;Here is a tiny proof of concept for an out-of-core dot product. I wouldn’t
expect users to write this. I would expect libraries like Blaze to write this.&lt;/p&gt;
&lt;section id="create-random-array-on-disk"&gt;
&lt;h2&gt;Create random array on disk&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;bcolz&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="n"&gt;b&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;bcolz&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;carray&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;empty&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;0&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="n"&gt;rootdir&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;A.bcolz&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;chunklen&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="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="n"&gt;b&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;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;rand&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;b&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;flush&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="define-computation-a-t-a"&gt;
&lt;h2&gt;Define computation &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;A.T&lt;/span&gt; &lt;span class="pre"&gt;*&lt;/span&gt; &lt;span class="pre"&gt;A&lt;/span&gt;&lt;/code&gt;&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="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;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="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;update&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;getem&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;blocksize&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;shape&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;shape&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;

&lt;span class="c1"&gt;# Add A.T into the mix&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;update&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;top&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="s1"&gt;&amp;#39;At&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;ij&amp;#39;&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="s1"&gt;&amp;#39;ji&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;numblocks&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;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;1&lt;/span&gt;&lt;span class="p"&gt;)}))&lt;/span&gt;

&lt;span class="c1"&gt;# Dot product A.T * A&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;update&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;top&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="s1"&gt;&amp;#39;AtA&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;ik&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;At&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;ij&amp;#39;&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="s1"&gt;&amp;#39;jk&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
         &lt;span class="n"&gt;numblocks&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;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;1&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;At&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;1&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;/section&gt;
&lt;section id="do-work"&gt;
&lt;h2&gt;Do work&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="n"&gt;dask&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;d&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;AtA&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="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="go"&gt;CPU times: user 2min 57s, sys: 6.59 s, total: 3min 4s&lt;/span&gt;
&lt;span class="go"&gt;Wall time: 2min 49s&lt;/span&gt;
&lt;span class="go"&gt;array([[ 334071.93541158,  250297.16968262,  250404.87729587, ...,&lt;/span&gt;
&lt;span class="go"&gt;         250436.85274716,  250330.64262904,  250590.98832611],&lt;/span&gt;
&lt;span class="go"&gt;       [ 250297.16968262,  333451.72293343,  249978.2751824 , ...,&lt;/span&gt;
&lt;span class="go"&gt;         250103.20601281,  250014.96660956,  250251.0146828 ],&lt;/span&gt;
&lt;span class="go"&gt;       [ 250404.87729587,  249978.2751824 ,  333279.76376277, ...,&lt;/span&gt;
&lt;span class="go"&gt;         249961.44796719,  250061.8068036 ,  250125.80971858],&lt;/span&gt;
&lt;span class="go"&gt;       ...,&lt;/span&gt;
&lt;span class="go"&gt;       [ 250436.85274716,  250103.20601281,  249961.44796719, ...,&lt;/span&gt;
&lt;span class="go"&gt;         333444.797894  ,  250021.78528189,  250147.12015207],&lt;/span&gt;
&lt;span class="go"&gt;       [ 250330.64262904,  250014.96660956,  250061.8068036 , ...,&lt;/span&gt;
&lt;span class="go"&gt;         250021.78528189,  333240.10323875,  250307.86236815],&lt;/span&gt;
&lt;span class="go"&gt;       [ 250590.98832611,  250251.0146828 ,  250125.80971858, ...,&lt;/span&gt;
&lt;span class="go"&gt;         250147.12015207,  250307.86236815,  333467.87105673]])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Three minutes for a 7GB dot product. This runs at about half the FLOPS of a
normal in-memory matmul. I’m not sure yet why the discrepancy. Also, this
isn’t using an optimized BLAS; we have yet to leverage multiple cores.&lt;/p&gt;
&lt;p&gt;This isn’t trivial to write, but it’s not bad either.&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/2014/12/27/Towards-OOC.md&lt;/span&gt;, line 356)&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="complexity-and-usability"&gt;
&lt;h1&gt;Complexity and Usability&lt;/h1&gt;
&lt;p&gt;This system is not appropriate for users; it’s unPythonic, low-level, and
LISP-y. However I believe that something like this would be an appropriate
standard for infrastructural libraries. It’s a simple and easy standard for
code to target.&lt;/p&gt;
&lt;p&gt;Using projects like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;into&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;blaze&lt;/span&gt;&lt;/code&gt; we can build a usable high-level
front-end onto &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; for the subproblems of arrays and tables. Blaze could
generate these dictionaries and then hand them off to other systems to execute.&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/2014/12/27/Towards-OOC.md&lt;/span&gt;, line 367)&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;Using the reference implementation, multithreading, HDF5/BColz, and out-of-core
caching systems like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chest&lt;/span&gt;&lt;/code&gt; I think that we can build a decent out-of-core
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ndarray&lt;/span&gt;&lt;/code&gt; solution that fully leverages a large workstation.&lt;/p&gt;
&lt;p&gt;Ideally other people come along and build better execution engines / task
schedulers. This might be an appropriate application for IPython 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/2014/12/27/Towards-OOC.md&lt;/span&gt;, line 376)&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;This could use design and technical feedback.
What would encourage community buy-in to a system like this?&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2014/12/27/Towards-OOC/"/>
    <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>2014-12-27T00:00:00+00:00</published>
  </entry>
</feed>
