Skip to content

Conversation

@bmerry
Copy link
Contributor

@bmerry bmerry commented Jun 10, 2019

Currently the cached boundaries are used only for slicing, and not in
all cases. This makes slicing of a small piece of a very large array (as
measured in chunks) significantly faster: for an array with 10^6 chunks,
a slice returning a single chunk is about 10x faster to construct after
the cache is initialized.

  • Tests added / passed
  • Passes flake8 dask

Currently the cached boundaries are used only for slicing, and not in
all cases. This makes slicing of a small piece of a very large array (as
measured in chunks) significantly faster: for an array with 10^6 chunks,
a slice returning a single chunk is about 10x faster to construct after
the cache is initialized.
@bmerry
Copy link
Contributor Author

bmerry commented Jun 10, 2019

I can probably simplify _slice_1d a bit to take advantage of having the initial 0 in the chunk boundaries, rather than throwing it away and then having lots of special cases for chunk 0. But I'd like to get some feedback on the general approach first.

I've got a few questions:

  • Do 'block' and 'chunk' mean different things (in dask), or are they interchangeable? I wasn't sure where to use what.
  • What is the thread-safety model in dask? The getter isn't thread-safe if two threads read from the same array at the same time, but that's also true of __dask_keys__ which has a similar cache.
  • Should I add something to the changelog? The development guidelines say one should, but looking at the history of changelog.rst it seems the changelog gets written at release time.

@bmerry
Copy link
Contributor Author

bmerry commented Jun 10, 2019

Forgot to mention: this is to implement #4867.

@mrocklin
Copy link
Member

Thanks for PR @bmerry . A few answers to your questions, and then some larger comments below:

Do 'block' and 'chunk' mean different things (in dask), or are they interchangeable? I wasn't sure where to use what.

Interchangable I guess. chunks very specifically also refers to the tuple-of-tuples, but we refer to a single numpy array interchangably as both a block and a chunk

What is the thread-safety model in dask? The getter isn't thread-safe if two threads read from the same array at the same time, but that's also true of dask_keys which has a similar cache.

This hasn't really come up before. I don't think that there are many multithreaded applications that modify dask array state.

Should I add something to the changelog? The development guidelines say one should, but looking at the history of changelog.rst it seems the changelog gets written at release time.

Nope, thanks for pointing this out. I've changed the docs in #4915

@mrocklin
Copy link
Member

So, in terms of the actual content here, I'm quite glad to see the speedup, but the amount of code change is a little disheartening, particularly the modification of the core dask array state and having to thread through the cached value through all of the slicing code. The combination of these two makes me wonder if the added complexity is worth the speed boost. I wonder if there is another way.

At first I thought, "oh, let's just cache the result of cumsum based on the chunks value, maybe something like this"

import toolz, weakref
cache = weakref.WeakKeyDictionary

cumsum = toolz.memoize(cache=cache)(np.cumsum)

"that would be great, we get the same caching, but we don't have to hold onto things manually and route things through ourselves". Unfortunately, it looks like we can't make weak references of tuples.

As an alternative, I wonder if we might do the following:

cumsum = functools.lru_cache()(np.cumsum)
In [1]: import time, functools

In [2]: def slow_sum(c):
   ...:     time.sleep(1)
   ...:     return sum(c)
   ...:

In [3]: memo_sum = functools.lru_cache()(slow_sum)

In [4]: chunks = (1, 2, 3, 4, 5)

In [5]: %time memo_sum((0,) + chunks)
CPU times: user 847 µs, sys: 1.05 ms, total: 1.9 ms
Wall time: 1 s
Out[5]: 15

In [6]: %time memo_sum((0,) + chunks)
CPU times: user 4 µs, sys: 0 ns, total: 4 µs
Wall time: 7.87 µs
Out[6]: 15

Thoughts? This probably isn't as optimalm, but it might be a lighter-weight code change, and add less maintenance costs going forward.

@bmerry
Copy link
Contributor Author

bmerry commented Jun 12, 2019

What is the thread-safety model in dask? The getter isn't thread-safe if two threads read from the same array at the same time, but that's also true of dask_keys which has a similar cache.

This hasn't really come up before. I don't think that there are many multithreaded applications that modify dask array state.

The issue is that whenever you have a cache (as you do for __dask_keys__), you have an operation that has read-only semantics but a read-write implementation. So doing "read" operations on an array from two different threads can have a race condition.

bmerry added a commit to bmerry/dask that referenced this pull request Jun 12, 2019
This is an alternative to dask#4909, to implement dask#4867.

Instead of caching in the class as in dask#4909, use functools.lru_cache.
This unfortunately has a fixed cache size rather than a cache entry
stored with each array, but simplifies the code as it is not necessary
to pass the cached value from the Array class down through the call tree
to the point of use.

A quick benchmark shows that the result for indexing a single value from
a large array is similar to that from dask#4909, i.e., around 10x faster for
constructing the graph.

This only applies the cache in `_slice_1d`, so should be considered a
proof-of-concept.
@mrocklin
Copy link
Member

The issue is that whenever you have a cache (as you do for dask_keys), you have an operation that has read-only semantics but a read-write implementation. So doing "read" operations on an array from two different threads can have a race condition.

Yup. No disagreement there.

Any thoughts on using something like lru_cache? This would sidestep this issue (among others).

@mrocklin
Copy link
Member

mrocklin commented Jun 12, 2019 via email

bmerry added a commit to bmerry/dask that referenced this pull request Jun 12, 2019
This is an alternative to dask#4909, to implement dask#4867.

Instead of caching in the class as in dask#4909, use functools.lru_cache.
This unfortunately has a fixed cache size rather than a cache entry
stored with each array, but simplifies the code as it is not necessary
to pass the cached value from the Array class down through the call tree
to the point of use.

A quick benchmark shows that the result for indexing a single value from
a large array is similar to that from dask#4909, i.e., around 10x faster for
constructing the graph.

This only applies the cache in `_slice_1d`, so should be considered a
proof-of-concept.
@bmerry
Copy link
Contributor Author

bmerry commented Jun 13, 2019

Closing in favour of #4923.

@bmerry bmerry closed this Jun 13, 2019
mrocklin pushed a commit that referenced this pull request Jun 14, 2019
This is an alternative to #4909, to implement #4867.

Instead of caching in the class as in #4909, use functools.lru_cache.
This unfortunately has a fixed cache size rather than a cache entry
stored with each array, but simplifies the code as it is not necessary
to pass the cached value from the Array class down through the call tree
to the point of use.

A quick benchmark shows that the result for indexing a single value from
a large array is similar to that from #4909, i.e., around 10x faster for
constructing the graph.

This only applies the cache in `_slice_1d`, so should be considered a
proof-of-concept.

* Move cached_cumsum to dask/array/slicing.py

It can't go in dask/utils.py because the top level is not supposed to
depend on numpy.

* cached_cumsum: index cache by both id and hash

The underlying _cumsum is first called with _HashIdWrapper, which will
hit (very cheaply) if we've seen this tuple object before. If not, it
will call itself again without the wrapper, which will hit (but at a
higher cost for tuple.__hash__) if we've seen the same value before but
in a different tuple object.

* Apply cached_cumsum in more places
TomAugspurger added a commit to TomAugspurger/dask that referenced this pull request Jun 17, 2019
commit 255cc5b
Author: Justin Waugh <[email protected]>
Date:   Mon Jun 17 08:18:26 2019 -0600

    Map Dask Series to Dask Series (dask#4872)

    * index-test needed fix

    * single-parititon-error

    * added code to make it work

    * add tests

    * delete some comments

    * remove seed set

    * updated tests

    * remove sort_index and add tests

commit f7d73f8
Author: Matthew Rocklin <[email protected]>
Date:   Mon Jun 17 15:22:35 2019 +0200

    Further relax Array meta checks for Xarray (dask#4944)

    Our checks in slicing were causing issues for Xarray, which has some
    unslicable array types.  Additionally, this centralizes a bit of logic
    from blockwise into meta_from_array

    * simplify slicing meta code with meta_from_array

commit 4f97be6
Author: Peter Andreas Entschev <[email protected]>
Date:   Mon Jun 17 15:21:15 2019 +0200

    Expand *_like_safe usage (dask#4946)

commit abe9e28
Author: Peter Andreas Entschev <[email protected]>
Date:   Mon Jun 17 15:19:24 2019 +0200

    Defer order/casting einsum parameters to NumPy implementation (dask#4914)

commit 76f55fd
Author: Matthew Rocklin <[email protected]>
Date:   Mon Jun 17 09:28:07 2019 +0200

    Remove numpy warning in moment calculation (dask#4921)

    Previously we would divide by 0 in meta calculations for dask array
    moments, which would raise a Numpy RuntimeWarning to users.

    Now we avoid that situation, though we may also want to investigate a
    more thorough solution.

commit c437e63
Author: Matthew Rocklin <[email protected]>
Date:   Sun Jun 16 10:42:16 2019 +0200

    Fix meta_from_array to support Xarray test suite (dask#4938)

    Fixes pydata/xarray#3009

commit d8ff4c4
Author: jakirkham <[email protected]>
Date:   Fri Jun 14 10:35:00 2019 -0400

    Add a diagnostics extra (includes bokeh) (dask#4924)

    * Add a diagnostics extra (includes bokeh)

    * Bump bokeh minimum to 0.13.0

    * Add to `test_imports`

commit 773f775
Author: btw08 <[email protected]>
Date:   Fri Jun 14 14:34:34 2019 +0000

    4809 fix extra cr (dask#4935)

    * added test that fails to demonstrate the issue in 4809

    * modfied open_files/OpenFile to accept a newline parameter, similar to io.TextIOWrapper or the builtin open on py3. Pass newline='' to open_files when preparing to write csv files.

    Fixed dask#4809

    * modified newline documentation to follow convention

    * added blank line to make test_csv.py flake8-compliant

commit 419d27e
Author: Peter Andreas Entschev <[email protected]>
Date:   Fri Jun 14 15:18:42 2019 +0200

    Minor meta construction cleanup in concatenate (dask#4937)

commit 1f821f4
Author: Bruce Merry <[email protected]>
Date:   Fri Jun 14 12:49:59 2019 +0200

    Cache chunk boundaries for integer slicing (dask#4923)

    This is an alternative to dask#4909, to implement dask#4867.

    Instead of caching in the class as in dask#4909, use functools.lru_cache.
    This unfortunately has a fixed cache size rather than a cache entry
    stored with each array, but simplifies the code as it is not necessary
    to pass the cached value from the Array class down through the call tree
    to the point of use.

    A quick benchmark shows that the result for indexing a single value from
    a large array is similar to that from dask#4909, i.e., around 10x faster for
    constructing the graph.

    This only applies the cache in `_slice_1d`, so should be considered a
    proof-of-concept.

    * Move cached_cumsum to dask/array/slicing.py

    It can't go in dask/utils.py because the top level is not supposed to
    depend on numpy.

    * cached_cumsum: index cache by both id and hash

    The underlying _cumsum is first called with _HashIdWrapper, which will
    hit (very cheaply) if we've seen this tuple object before. If not, it
    will call itself again without the wrapper, which will hit (but at a
    higher cost for tuple.__hash__) if we've seen the same value before but
    in a different tuple object.

    * Apply cached_cumsum in more places

commit 66531ba
Author: jakirkham <[email protected]>
Date:   Thu Jun 13 12:13:55 2019 -0400

    Drop size 0 arrays in concatenate (dask#4167)

    * Test `da.concatenate` with size 0 array

    Make sure that `da.concatenate` does not include empty arrays in the
    result as they don't contribute any data.

    * Drop size 0 arrays from `da.concatenate`

    If any of the arrays passed to `da.concatenate` has a size of 0, then it
    won't contribute anything to the array created by concatenation. As such
    make sure to drop any size 0 arrays from the sequence of arrays to
    concatenate before proceeding.

    * Handle dtype and all 0 size case

    * Cast inputs with asarray

    * Coerce all arrays to concatenate to the same type

    * Drop obsoleted type handling code

    * Comment on why arrays are being dropped

    * Use `np.promote_types` for parity w/old behavior

    * Handle endianness during type promotion

    * Construct empty array of right type

    Avoids the need to cast later and the addition of another node to the
    graph.

    * Promote types in `concatenate` using `_meta`

    There was some left over type promotion code for the arrays to
    concatenate using their `dtype`s. However this should now use the
    `_meta` information instead since that is available.

    * Ensure `concatenate` is working on Dask Arrays

    * Raise `ValueError` if `concatenate` gets no arrays

    NumPy will raise if no arrays are provided to concatenate as it is
    unclear what to do. This adds a similar exception for Dask Arrays. Also
    this short circuits handling unusual cases later. Plus raises a clearer
    exception than one might see if this weren't raised.

    * Test `concatenate` raises when no arrays are given

    * Determine the concatenated array's shape

    Needed to handle the case where all arrays have trivial shapes.

    * Handle special sequence cases together

    * Update dask/array/core.py

    Co-Authored-By: James Bourbeau <[email protected]>

    * Drop outdated comment

    * Assume valid `_meta` in `concatenate`

    Simplifies the `_meta` handling logic in `concatenate` to assume that
    `_meta` is valid. As all arguments have been coerced to Dask Arrays,
    this is a reasonable assumption to make.

commit 46aef58
Author: James Bourbeau <[email protected]>
Date:   Thu Jun 13 11:04:47 2019 -0500

    Overload HLG values method (dask#4918)

    * Overload HLG values method

    * Return lists for keys, values, and items

    * Add tests for keys and items

commit f9cd802
Author: mcsoini <[email protected]>
Date:   Thu Jun 13 18:03:55 2019 +0200

    Merge dtype warning (dask#4917)

    * add test covering the merge column dtype mismatch warning

    * for various merge types: checks that the resulting dataframe
      has either no nans or that a UserWarning has been thrown

    * Add warning for mismatches between column data types

    * fixes issue dask#4574
    * Warning is thrown if the on-columns of left and right have
      different dtypes

    * flake8 fixes

    * fixes

    * use asciitable for warning string

commit c400691
Author: Hugo <[email protected]>
Date:   Thu Jun 13 17:38:37 2019 +0300

    Docs: Drop support for Python 2.7 (dask#4932)

commit 985cdf2
Author: Benjamin Zaitlen <[email protected]>
Date:   Thu Jun 13 10:38:15 2019 -0400

    Groupby Covariance/Correlation (dask#4889)

commit 6e8c1b7
Author: Jim Crist <[email protected]>
Date:   Wed Jun 12 15:55:11 2019 -0500

    Drop Python 2.7 (dask#4919)

    * Drop Python 2.7

    Drops Python 2.7 from our `setup.py`, and from our test matrix. We don't
    drop any of the compatability fixes (yet), but won't be adding new ones.

    * fixup

commit 7a9cfaf
Author: Ian Bolliger <[email protected]>
Date:   Wed Jun 12 11:44:26 2019 -0700

    keep index name with to_datetime (dask#4905)

    * keep index name with to_datetime

    * allow users to pass meta

    * Update dask/dataframe/core.py

    put meta as explicit kwarg

    Co-Authored-By: Matthew Rocklin <[email protected]>

    * Update dask/dataframe/core.py

    remove meta kwargs.pop

    Co-Authored-By: Matthew Rocklin <[email protected]>

    * remove test for index

    * allow index

commit abc86d3
Author: jakirkham <[email protected]>
Date:   Wed Jun 12 14:20:59 2019 -0400

    Raise ValueError if concatenate is given no arrays (dask#4927)

    * Raise `ValueError` if `concatenate` gets no arrays

    NumPy will raise if no arrays are provided to concatenate as it is
    unclear what to do. This adds a similar exception for Dask Arrays. Also
    this short circuits handling unusual cases later. Plus raises a clearer
    exception than one might see if this weren't raised.

    * Test `concatenate` raises when no arrays are given

commit ce2f866
Author: jakirkham <[email protected]>
Date:   Wed Jun 12 14:09:35 2019 -0400

    Promote types in `concatenate` using `_meta` (dask#4925)

    * Promote types in `concatenate` using `_meta`

    There was some left over type promotion code for the arrays to
    concatenate using their `dtype`s. However this should now use the
    `_meta` information instead since that is available.

    * Ensure `concatenate` is working on Dask Arrays
Merge remote-tracking branch 'upstream/master' into dataframe-warnings
@bmerry bmerry deleted the cache-chunk-boundaries branch February 17, 2020 10:56
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants