Skip to content

Conversation

@bmerry
Copy link
Contributor

@bmerry bmerry commented Jun 12, 2019

This is an alternative to #4909. Fixes #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.

  • Tests added / passed
  • Passes flake8 dask

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 12, 2019

@mrocklin this is based on your idea for using an lru_cache to cache chunk boundaries - thanks for the suggestion! If you're happy with it I can go looking for other places it can be used.

I'm not sure which module cached_cumsum belongs in. It felt a little wrong to have to add numpy to the imports for dask.utils.

@mrocklin mrocklin mentioned this pull request Jun 12, 2019
2 tasks
@bmerry bmerry force-pushed the cache-chunk-boundaries2 branch from 41e8c95 to 9cfeb81 Compare June 12, 2019 14:44
@bmerry
Copy link
Contributor Author

bmerry commented Jun 12, 2019

Oops, looks like I pushed a slightly older version of this. I've just force-pushed my corrected version, which hopefully will actually pass CI.

dask/utils.py Outdated
return 1 + ndimlist(seq[0])


class _HashIdWrapper(object):
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm curious, why do we need this?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hashing a large tuple takes time proportional to the length of the tuple (about 2ms for 10^6 elements on my machine), which would somewhat negate the benefits of caching (although it is still faster than np.cumsum, mostly because converting a tuple to an ndarray is surprisingly slow). So I'm trading off poorer cache usage (because tuples with the same value but different identity require separate cache slots) to get higher performance lookups.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we didn't add the HashIdWrapper, and hashed tuples directly, how much slower would that make your workloads?

When I look at the difference I see about a 20x difference

In [1]: t = tuple(range(1000000))

In [2]: d = {}

In [3]: %time d[t] = 1
CPU times: user 2.54 ms, sys: 0 ns, total: 2.54 ms
Wall time: 2.55 ms

In [4]: %timeit d[t]
2.31 ms ± 151 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)

In [5]: import numpy as n

In [6]: import numpy as np

In [7]: %timeit np.cumsum(t)
43.1 ms ± 477 µs per loop (mean ± std. dev. of 7 runs, 10 loops each)

I'm hoping that this is fast enough, and that something else becomes the bottleneck instead, especially given your statement here:

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.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll take a look, but my goal is to make slicing M chunks out of an N-chunk array take O(M + log N) rather than O(N). Are you objecting to the wrapper because it reduces cache re-use or just because it adds complexity?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just because of complexity. It's not a big deal, we can keep it (it's well isolated), I just thought I'd ask in case it wasn't necessary in practice.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here's some benchmarking. It's somewhat pathological (an enormous number of very small chunks) so I don't expect the differences to be as significant in real use, but it shows that the wrapper does improve performance.

#!/usr/bin/env python

import time

from katdal.lazy_indexer import dask_getitem
import dask.array as da
import numpy as np


a = da.ones((1000000, 100), chunks=(1, 100))
step = 100
start = time.monotonic()
for i in range(0, a.shape[0], step):
    dask_getitem(a, np.s_[i : i + step]).compute()
stop = time.monotonic()
print(stop - start)
  • master: 1301 seconds
  • branch: 278 seconds
  • branch without _HashIdWrapper: 339 seconds

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It improves things, but not by a large amount. My guess is that if we didn't use _HashIdWrapper that you wouldn't notice enough to add it.

I'm happy either way, with a slight preference for simpler/slower code. It sounds like you prefer HashIdWrapper though, given that you seem to be making a few changes these days I'm happy to defer to your preference here.

@bmerry
Copy link
Contributor Author

bmerry commented Jun 13, 2019

I'm not sure which module cached_cumsum belongs in. It felt a little wrong to have to add numpy to the imports for dask.utils.

In fact I see the tests fail because they don't like import dask sucking in numpy. Any preference on where it goes? My first thought is array/core.py, but that requires slicing to import core and vice versa (which could be solved by having _slice_1d import core at the function level).

@mrocklin
Copy link
Member

In fact I see the tests fail because they don't like import dask sucking in numpy. Any preference on where it goes? My first thought is array/core.py, but that requires slicing to import core and vice versa (which could be solved by having _slice_1d import core at the function level).

Right, the core dask library doesn't have any dependencies. Depending on NumPy would definitely be unwelcome. Perhaps a dask/array/utils.py file?

@bmerry
Copy link
Contributor Author

bmerry commented Jun 13, 2019

Right, the core dask library doesn't have any dependencies. Depending on NumPy would definitely be unwelcome. Perhaps a dask/array/utils.py file?

There is one already, but it depends on array/core, so if I add it there then you end up with a cycle of dependencies array/slicing -> array/utils -> array/core -> array/slicing.

I could also add it to array/slicing (which doesn't depend on anything else from array), but it's not really specific to slicing.

bmerry added 2 commits June 13, 2019 10:29
It can't go in dask/utils.py because the top level is not supposed to
depend on numpy.
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.
@bmerry
Copy link
Contributor Author

bmerry commented Jun 13, 2019

I've put it in array/slicing for now. I also tweaked the cache lookup so that it looks up by value if there is a miss in the lookup by name, which should give the benefits of both without noticeably increasing storage (because each entry is cached twice but the key and value objects are shared rather than duplicated).

@bmerry
Copy link
Contributor Author

bmerry commented Jun 13, 2019

@mrocklin It sounds like you're happy with this overall approach (modulo the concerns about _HashIdWrapper). The next step would be to go looking for other uses of np.cumsum to see which make sense to cache. Would you like me to do that in this PR, or would you rather merge this and leave further optimisations for other PRs?

@mrocklin
Copy link
Member

If it's easy (<1hr ?) then I recommend that we do it here. If we're going to add things then we might as well add them consistently.

@bmerry
Copy link
Contributor Author

bmerry commented Jun 13, 2019

Ok, I'll look for low-hanging fruit tomorrow.

There are also a number of places that use toolz.accumulate rather than np.cumsum. Those might need more work to check that it's safe to switch out one for the other.

@bmerry
Copy link
Contributor Author

bmerry commented Jun 14, 2019

I've picked off the low-hanging fruit.

@mrocklin
Copy link
Member

OK. Merging this in. Thanks for all the work @bmerry !

@mrocklin mrocklin merged commit 1f821f4 into dask:master Jun 14, 2019
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-boundaries2 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.

Performance enhancement: pre-compute chunk boundaries

3 participants