Skip to content

Check that computed chunks have right size and dtype#7277

Merged
jsignell merged 10 commits intodask:masterfrom
bmerry:check-chunk-sizes
Mar 4, 2021
Merged

Check that computed chunks have right size and dtype#7277
jsignell merged 10 commits intodask:masterfrom
bmerry:check-chunk-sizes

Conversation

@bmerry
Copy link
Contributor

@bmerry bmerry commented Feb 26, 2021

This was inspired by a question in #7234. At present assert_eq checks
that the computed array has the right size and dtype, but it's still
possible that the individual chunks have the wrong size (as long as they
add up to the right total size) and maybe the wrong dtype (as long as
they coerce to produce the right dtype - although I haven't actually
checked that this would slip through).

Now persist is used to access the individual computed chunks and
checked for shape and dtype.

This is a draft because there are a couple of failures. One is in
test_zarr_return_stored[False] (which I'll need to investigate
further, but possibly persist just doesn't make sense for this case)
and the other is in test_blockwise_concatenate which looks like it
might just be a test bug.

  • Tests added / passed
  • Passes black dask / flake8 dask

This was inspired by a question in dask#7234. At present `assert_eq` checks
that the computed array has the right size and dtype, but it's still
possible that the individual chunks have the wrong size (as long as they
add up to the right total size) and maybe the wrong dtype (as long as
they coerce to produce the right dtype - although I haven't actually
checked that this would slip through).

Now `persist` is used to access the individual computed chunks and
checked for shape and dtype.

This is a draft because there are a couple of failures. One is in
`test_zarr_return_stored[False]` (which I'll need to investigate
further, but possibly `persist` just doesn't make sense for this case)
and the other is in `test_blockwise_concatenate` which looks like it
might just be a test bug.
@bmerry
Copy link
Contributor Author

bmerry commented Feb 26, 2021

It might also be worth making _check_chunks a public function (named assert_something to make it clear that it's for testing) so that it can be used independently of assert_eq.


def _get_dt_meta_computed(x, check_shape=True, check_graph=True):
def _check_chunks(x):
x = x.persist()
Copy link
Member

Choose a reason for hiding this comment

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

In testing we typically use the synchronous scheduler by default. Happy either way though.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Oh yes, I'd meant to do that and forgot. Done now.

if check_shape:
assert_eq_shape(x_original.shape, x.shape, check_nan=False)
if check_chunks:
_check_chunks(x_original)
Copy link
Member

Choose a reason for hiding this comment

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

Would it be possible to compute only once? I'm concerned about doubling the compute time of every dask array test

We might call persist at the beginning of the assert_eq function, and then call .compute() on that result just afterwards so that we have the original, persisted, and computed versions.

Or, more broadly, how does this change affect the runtime of py.test dask/array ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good question - I wasn't sure which way was better. It goes from 51s to 69s (with the synchronous scheduler for persist). Not double, but not trivial either. I'll try it the other way and see what the difference is.

Is there much risk that replacing x.compute() with x.persist().compute() will affect test coverage? I'm not really up to speed on the postprocessing protocols used in each case.

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've changed it to your suggestion, and now it only adds about 2-3s. It also made the zarr failure go away; my guess is that since the test was to do with writing to a zarr array, it wasn't idempotent and so computing twice failed.

@bmerry bmerry marked this pull request as draft February 26, 2021 14:42
@bmerry
Copy link
Contributor Author

bmerry commented Feb 26, 2021

Oops, had meant it to be a draft PR. Changed now.

@bmerry bmerry mentioned this pull request Feb 26, 2021
6 tasks
It had a `blockwise` function that returned a shape inconsistent with
that inferred from the input block sizes.
The addition of chunk checking to assert_eq broke this test, which
fiddles with array_optimize in a way that makes x.compute() different to
x.persist().compute(). Fix by just disabling check_chunks for this test.
@bmerry
Copy link
Contributor Author

bmerry commented Feb 27, 2021

I've mostly fixed up the tests, but the mindeps testing is showing up a real bug: tril and triu use zeros_like_safe, and on older versions of numpy those functions don't match the dtype, so one ends up with chunks of the wrong dtype.

I'm happy to fix the _like_safe functions, but I don't know their history so I'm not sure what the semantics are supposed to be.

@Illviljan
Copy link
Contributor

I've mostly fixed up the tests, but the mindeps testing is showing up a real bug: tril and triu use zeros_like_safe, and on older versions of numpy those functions don't match the dtype, so one ends up with chunks of the wrong dtype.

I'm happy to fix the _like_safe functions, but I don't know their history so I'm not sure what the semantics are supposed to be.

There is a PR that refactors tril and triu to follow the numpy implementation more closely: #6997
I haven't checked if it actually solves your issue though, but it doesn't use zeros_like_safe at least.

@bmerry
Copy link
Contributor Author

bmerry commented Feb 27, 2021

There is a PR that refactors tril and triu to follow the numpy implementation more closely: #6997

Interesting, but assuming it fixes the test failure it's just hiding the actual bug (in zeros_like_safe) which may well affect other code.

@bmerry
Copy link
Contributor Author

bmerry commented Feb 27, 2021

I think I've finally gotten the tests passing again (just waiting for MacOS CI), so ready for review. The history has being a bit of a mess so let me know if you'd like a rebase (I think it should be 3 commits: one to fix a test, one to fix #7287 and one for the headline change).

Copy link
Member

@jsignell jsignell left a comment

Choose a reason for hiding this comment

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

Thanks for doing this. I think the history is fine, we'll squash-merge anyways.

@bmerry
Copy link
Contributor Author

bmerry commented Mar 2, 2021

Thanks for doing this. I think the history is fine, we'll squash-merge anyways.

Ok, as long as the commit message gets cleaned up. The default of just concatenating all the individual commits into a message for the squashed commit would look awful.

@jsignell
Copy link
Member

jsignell commented Mar 4, 2021

Is this good to merge then?

@bmerry
Copy link
Contributor Author

bmerry commented Mar 4, 2021

Is this good to merge then?

The code is ready from my point of view. If you like I can rebase to get a cleaner commit message when you squash-merge.

@jsignell jsignell merged commit 7a12889 into dask:master Mar 4, 2021
dcherian added a commit to dcherian/dask that referenced this pull request Mar 8, 2021
* upstream/master: (43 commits)
  bump version to 2021.03.0
  Bump minimum version of distributed (dask#7328)
  Fix `percentiles_summary` with `dask_cudf` (dask#7325)
  Temporarily revert recent Array.__setitem__ updates (dask#7326)
  Blockwise.clone (dask#7312)
  NEP-35 duck array update (dask#7321)
  Don't allow setting `.name` for array (dask#7222)
  Use nearest interpolation for creating percentiles of integer input (dask#7305)
  Test `exp` with CuPy arrays (dask#7322)
  Check that computed chunks have right size and dtype (dask#7277)
  pytest.mark.flaky (dask#7319)
  Contributing docs: add note to pull the latest git tags before pip installing Dask (dask#7308)
  Support for Python 3.9 (dask#7289)
  Add broadcast-based merge implementation (dask#7143)
  Add split_every to graph_manipulation (dask#7282)
  Typo in optimize docs (dask#7306)
  dask.graph_manipulation support for xarray.Dataset (dask#7276)
  Add plot width and height support for Bokeh 2.3.0 (dask#7297)
  Add numpy functions tri, triu_indices, triu_indices_from, tril_indices, tril_indices_from (dask#6997)
  Remove "cleanup" task in dataframe on-disk shuffle. The partd directory (dask#7260)
  ...
@bmerry bmerry deleted the check-chunk-sizes branch March 8, 2021 16:18
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants