-
-
Notifications
You must be signed in to change notification settings - Fork 1.8k
Add Dask Array._meta attribute #4543
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
|
@mrocklin before diving in too deep, could you look at this first change and tell me if that's more or less what you had in mind? Note: build will not succeed since the |
mrocklin
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for sending the WIP PR @pentschev !
Yes, in principle this seems like it's going in the right direction to me. I added a few comments below.
|
To be honest I'm surprised at how much actually works out of the box here. I suppose that that's because we don't try to propagate the meta type yet (we're only passing around dtypes in the various routines) so almost everything just passes through. cc @jcrist (masked arrays) @shoyer (NEP-18) @TomAugspurger (sparse matrices) who have all expressed an interest in this before Also cc @jakirkham , just because it's an array thing :) |
|
I hope you don't mind I pushed a test for It fails currently because most functions just supply a I think that the first place to start on this is probably the Lines 232 to 234 in 2732793
|
|
@mrocklin No worries, thanks for adding the test, will definitely make things easier to verify. |
|
The test should be the same for CuPy by the way (the code paths are identical at this point). It's just easier to include sparse than CuPy in CI :) |
|
Understood, nevertheless, I don't want #4538 to be forgotten. :) |
|
I was checking |
|
Or could we just overwrite the dtype of |
|
I recommend not worrying too much about the dtype errors just yet. I think that we should first try to change a few internal functions like The dtype errors on string and unicode types seem smaller to me. I recommend that we avoid getting too bogged down in them early on. |
|
@mrocklin that's what I am doing for now, but I usually try to fix issues I can see from failing tests as early as possible to avoid preventing other errors from showing up (e.g., errors in further steps of those tests). I will for now postpone that dtype issue, but would be great if someone has any ideas. :) |
|
In my latest commit I've managed to get the test cases introduced by @mrocklin, with the exception of the one using I'm sure this is not the cleanest solution, but it gives us an initial idea of what conditions we have to consider to fix this issue. To make sure this works, I needed to import numpy in Improvement ideas are very much welcome. :) |
|
It's ok to |
mrocklin
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In general this looks good. I raised a few points below where I think that we will eventually need to be more careful. I should probably come up with more tests though.
dask/array/core.py
Outdated
|
|
||
| graph = HighLevelGraph.from_collections(out, dsk, dependencies=[self]) | ||
| return Array(graph, out, chunks, dtype=self.dtype) | ||
| return Array(graph, out, chunks, meta=self._meta.astype(self.dtype)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hrm, what if slicing changes the type of the array?
For example, I could imagine that slicing some types of sparse matrices might result in other types of sparse matrices, or possibly even numpy arrays. Ideally we transform meta as we perform the operation, rather than just changing around the dtype. In this way our dask array's meta type will track what happens in the lazy chunks.
So for an operation like __getitem__ we might make a new index that was just like the input, but had a bunch of elements that were safe to apply against an empty array.
In [16]: index = (slice(0, 5), [1, 2, 3], None, slice(0, 2))
In [17]: x = np.empty((0, 0, 0))
In [18]: index2 = (slice(0, 0), [], None, slice(0, 0)) # like index, but empty
In [19]: x[index2]
Out[19]: array([], shape=(0, 0, 1, 0), dtype=float64)However I'm not sure how best to handle integer elements in the index. I'm not sure that there is an empty-safe way to handle this.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, I think that will need some more careful thinking. We were aware of this kind of situation, unless I misunderstood, this case is analogous to the case of a Dask array containing multiple arrays of different types, which was mentioned by you in #2977:
The plan I've proposed above doesn't support the mixed-array case (where we have a mix of dense and sparse arrays in the same dask array for example), but I suggest that we start there. I think that it will be a good and productive first step.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think that this concern is independent of mixed chunks.
Lets say that some CSR matrix type exists, and I have a dask array full of them.
When I do something like x[0, :] I pull out a row. It could be that the return type of that object is no longer a CSR matrix, but some other form (maybe COO). The output still has a uniform chunk type, it's just a new type.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think I get it now. I'll try to think how to handle it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The best I could do so far was:
meta = self._meta[index].astype(self.dtype)
meta.shape = (0,) * len(chunks)That second line probably suffices to keep the shape as we need it, unless the result should be a 0d array (for such case, I don't have a solution yet). But if the index could change the type of the matrix, isn't there a case that the actual array size will matter and not change it if the matrix is empty, i.e., after slicing, data and _meta be different anyway?
To be honest, at this point I'm starting to ask myself if this is really a good design. There are so many ifs and buts that I think it may immediately render the code unmaintainable.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oh, I forgot to mention that the code above works only for slicing, indexing would need to be a separate case.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
To be honest, at this point I'm starting to ask myself if this is really a good design.
It's a good question to ask. And thank you for going through the experiment that helps us to understand whether or not this is the right approach.
There are so many ifs and buts that I think it may immediately render the code unmaintainable.
We've gone through this process before both with dtype and with shape. It's definitely a challenge.
My guess is that we'll be able to find a solution for most cases. For the more difficult cases there are people watching this PR that have deeper NumPy experience than you or I do (people like @shoyer @jakirkham @hameerabbasi and @TomAugspurger ) and they can help once we identify the difficult issues.
At this point we've identified that slicing is going to be difficult. Lets move on to other operations before diving too deeply here, and maybe one of the people above can offer a suggestion based on the description above.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We've gone through this process before both with dtype and with shape. It's definitely a challenge.
I understand this, and I'm not even worried about NumPy, but answering the question of what could 3rd-party libraries do to their own types in every possible case may be untenable. I'm thinking that waiting for empty_like() to be available may be the right choice, since we could then just recreate _meta with the right size every time something may have changed, rather than spending so much time in handling all possible cases.
At this point we've identified that slicing is going to be difficult. Lets move on to other operations before diving too deeply here, and maybe one of the people above can offer a suggestion based on the description above.
I ended up on slicing, because fixing the issue below (in partial_reduce()) depends on slicing. At some point things fail because the _meta array has the wrong shape. I suspect this will be a requirement for other pieces of the code as well, since it's one of the most basic operations.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@mrocklin I pushed some new commits, one of them attempts to handle slicing of _meta. This has been capable of handling all the cases that previously failed in existing tests, but there's always a chance that I've missed cases that I couldn't test for.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
However I'm not sure how best to handle integer elements in the index. I'm not sure that there is an empty-safe way to handle this.
There is, just prepend Ellipsis to the list of indexes.
>>> import numpy as np
>>> np.float64(0)[..., None, None]
array([[0.]])And if you want to guarantee that the output is an array, even if 0-D, postpend with Ellipsis:
>>> np.array([[0]])[0, 0, ...]
array(0)
dask/array/reductions.py
Outdated
| dsk[(name,) + k] = (func, g) | ||
| graph = HighLevelGraph.from_collections(name, dsk, dependencies=[x]) | ||
| return Array(graph, name, out_chunks, dtype=dtype) | ||
| return Array(graph, name, out_chunks, meta=x._meta.astype(dtype)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think that ideally we would call
func(x._meta, keepdims=keepdims)
That way if the shape or type of the array changes we'll learn about it. Changing the dtype alone probably isn't enough.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good catch, I'll try to do that.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It ended up being much more complicated to handle reductions, but just like slicing, the commit that I pushed to handle this case, seems to handle all cases I've seen in existing tests.
dask/array/blockwise.py
Outdated
| arrays = args[::2] | ||
| arrays_empty = [arr._meta if hasattr(arr, '_meta') else np.empty((0,)) | ||
| for arr in arrays] | ||
| meta = func(*arrays_empty) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This function probably needs all of the inputs, not just the arrays. Maybe something like the following:
args_meta = [arg._meta if hasattr(arg, '_meta') else arg for arg in args]
kwargs_meta = {k: v._meta if hasattr(v, '_meta') else v for k, v in kwargs.items()}
func = func(*args_meta, **kwargs_meta)This happens because sometimes people put non-arrays in map_blocks or other blockwise operations like the following:
x.map_blocks(func, y, 123, option='foo')
In these cases it's important to keep the extra args/keyword args
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I was aware my handling wasn't complete, but I wasn't sure what else was missing there, thanks for pointing that out. And to be honest, I wouldn't be surprised if your changes don't make it complete either, there are really many things that can happen in this blockwise() function.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, it's at the core of many calls and is the frequent target of many many feature requests. Keeping it as simple as it is simple has been a challenge.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
(not that I think it's simple)
Regardless, thanks for being willing to dive into it. I acknowledge that it's not trivial.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I did the changes you suggested, it mostly works but breaks test_argwhere_str and test_count_nonzero_str. The issue is with passing **kwargs, after a lot of unsuccessful digging, I think what happens is the 'concatenate' argument is passed to numpy.vectorize() that doesn't expect it. I will come back to check this later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'll try to take a look at this later today as well
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's still difficult to handle all cases when we pass kwargs, I'm still working on fixing this.
|
I've added a checklist up top of all missing cases that we know about. If any missing cases are found or current ones get resolve, please update that. |
|
I think @mrocklin is correct that it's impossible to know the dtype without performing a calculation (think of |
dask/array/blockwise.py
Outdated
| # a basic type | ||
| try: | ||
| arrays = args[::2] | ||
| args_meta = [arg._meta if hasattr(arg, '_meta') else np.empty((0,)) for arg in arrays] |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would expect this to be else arg instead of else np.empty((0,)). What if the user provides an argument that is not an array, like a string?
def f(a, b):
if b == 'add':
return a + 1
elif b == 'sub':
return a - 1
else:
raise ValueError(b)
blockwise(f, 'i', x, 'i', 'add', None)There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
At first, I'm trying to get current tests to work, but yes, this is a valid concern. In fact, I've already seen some issues with that part of the code, I'll try to fix what I can before moving on to more complicated cases.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In my latest commit I fixed the condition to else arg, as @mrocklin suggested.
I've added a second try/except to attempt passing **kwargs and if there's an exception, don't. I noticed the only case this happens is in the test that @mrocklin added here: https://github.com/dask/dask/pull/4543/files#diff-8f3045290928cb33f56d0b894e765b88R141
This happens because blockwise() is always called passing a concatenate=True in map_blocks(), as seen here https://github.com/dask/dask/blob/master/dask/array/core.py#L484, and sparse.COO.from_numpy() doesn't have this argument. What would be the correct solution here?
dask/array/reductions.py
Outdated
| reduced_meta = None | ||
| if hasattr(x, '_meta'): | ||
| reduced_meta = blockwise(chunk, inds, x._meta, inds, axis=axis, | ||
| keepdims=True, dtype=x.dtype) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm confused about the objective of reduced_meta here. Can you help me understand this?
Also, do we need to call blockwise? Or are you just calling blockwise in order to get the meta object?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm trying to ensure that _meta goes through all the same steps as the main array, since I assume it's not possible to predict how every single step may change the arrays and their data type.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we use tmp._meta instead here perhaps?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In general calling compute can be quite complex to do. We engage the full optimization system. If we're using the distributed scheudler then we trigger network communications. If we're using Dask in an asynchronous mode then things will probably block in bad ways. If we need to call compute then we can turn all of this stuff off during that compute call, but I'd prefer to avoid it if we can.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I totally get your point now @mrocklin. However, I don't immediately see a different way here. What I could do is not "reduce" _meta, but I presume that could lead to different results for complex types. Do you have any suggestions on how to better handle it here?
dask/array/reductions.py
Outdated
| try: | ||
| # must try multiple argument combinations, some functions may not | ||
| # support all of them, or even none of them | ||
| meta = try_combinations(meta_func, TypeError, reduced_meta.compute(), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Calling .compute() anywhere is somewhat concerning within dask array code. Ideally things would stay lazy.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
reduced_meta is _meta wrapped in a Dask array, the one returned by blockwise(). So far, this is how I managed to ensure we go through all steps.
dask/array/reductions.py
Outdated
| try: | ||
| return func(x, **combinations[0]) | ||
| except e: | ||
| return try_combinations(func, e, x, combinations[1:]) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can I ask for information about this? Why did we need this?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Or, maybe more directly, the fact that we need this is somewhat concerning. I'm curious about what the problem is so that maybe someone can find another approach.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't like this either, but what happens here is that in partial_reduce() I'm calling func() (or meta_func() which doesn't _concatenate2()), and depending on the function in question, it may accept different arguments. In particular the meta argument had to be added because functions chunk/combine/aggregate functions may return dicts, which aren't valid arrays to reduce.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah I agree with both of you, we should find another approach here if possible. Do you have a couple of examples where this is needed so we can formulate and try out some alternatives?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Alright, I pushed commit ce7e5dd to get rid of (part of) that. We still need some exception handling for some of the reduction functions (e.g., mean_chunk, mean_combine, etc.) for which I added a meta= keyword argument to prevent them from returning dicts that we don't want for _meta construction, but most of them won't have such keyword. The second case is for ValueError exception, like ufuncs that can't be executed on empty arrays.
I now honestly don't know if I had a legitimate case where I've failure due to dtype= and keepdims=. The two possibilities I see now for that are:
- I had some bug in another part of the code during my development process; or
- I'm really going insane.
Please let me know if the solution on the aforementioned commit seems acceptable. But IMHO, it seems acceptable.
| new_index = tuple(new_index) | ||
| meta = self._meta[new_index].astype(self.dtype) | ||
| else: | ||
| meta = self._meta[index2].astype(self.dtype) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm going to try to defer to @jakirkham on this :)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry I'm not totally following what is happening in this block. Could you please provide a summary, @pentschev?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, what happens here is very simple, basically creating a new_index in this order:
- Check all elements of the
index2tuple, if it's not a tuple, make it aslice(0, 0); - For elements that are not tuples (e.g., a single index), make it an
Ellipsisif the index is valid, otherwise make itNone.
This way, I convert all cases to 0-sized slices. For example:
[1, slice(1:2), :]becomes shape(0, 0, ...)[None, 1, None]becomes shape(1, 0, 1), the1s that arise from None should probably still be fixed, even though it may not be a problem generally
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for providing more info. I'm still confused unfortunately. Have a couple of follow-up questions.
What is the difference between using slice(0, 0) and Ellipsis on _meta?
Also, I thought None would add singleton dimensions. Why do we want that for this use case?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What is the difference between using
slice(0, 0)andEllipsison_meta?
I think they're equivalent, I probably started using Ellipsis and forgot to fix that. Plus, having two consecutive Ellipsis isn't allowed, so that's wrong for sure, I'll fix it.
Also, I thought
Nonewould add singleton dimensions. Why do we want that for this use case?
It does add a singleton dimension. We would want one if the user requests so, as per @mrocklin's previous example here.
Just to be clear here, I was attempting to cover all cases I could think of. Maybe not all of them would make sense in a realistic context, but are technically allowed, so we should cover them.
mrocklin
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I took another pass through this. Most of my comments are pretty minor. I would be pretty comfortable merging this if there are no other objections.
dask/array/tests/test_cupy.py
Outdated
| lambda x: np.sinc(x), | ||
| lambda x: np.nan_to_num(x), | ||
| pytest.param(lambda x: np.isreal(x), | ||
| marks=pytest.mark.skipif(missing_arrfunc_cond, reason=missing_arrfunc_reason)), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I have a slight preference to inline not NEP_18_ACTIVE here (and maybe the reason too). It's shorter and doesn't requrie the reader to go find the definition of missing_arrfunc_cond.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just a style preference though. Not a big deal.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It definitely isn't shorter, due to the long reason string. But I have no strong preference, so I'm just gonna go ahead and inline things.
dask/array/tests/test_cupy.py
Outdated
| def rnd_test(func, *args, **kwargs): | ||
| a = func(*args, **kwargs) | ||
| assert type(a._meta) == cupy.core.core.ndarray | ||
| assert type(a.compute()) == cupy.core.core.ndarray |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Assuming that assert_eq checks that type(x.compute()) == type(x._meta) then would it work to use assert_eq(a, a) here instead?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, I feared that it would look ridiculous and someone would end up removing that at one point. I'm going to do that and add a comment that it's done on purpose.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We use assert_eq(a, a) in other places as well. I recommend just doing that. I think that most of the maintainers know that this is common, just because of all of the other checks that assert_eq runs.
| ss = func(s) | ||
|
|
||
| assert_eq(dd, ss) | ||
| assert_eq(dd, ss, check_meta=False) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If masked arrays aren't handling meta well can I ask you to raise an issue? I can ping some people that I think are motivated to keep it up to date, and maybe they can help.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's not the case they're not handled well, but that we're indeed comparing a masked array to a regular one, which means types really don't match, so we have to purposely ignore comparing _meta. All other tests (where we compared two masked arrays) work well. Do you have a different suggestion here? Otherwise, I think these two cases are handled properly as they are now.
|
I plan to merge this tomorrow if there are no further comments. |
|
I hope this doesn't break too many things that we didn't foresee (or didn't have tests to verify). :) |
|
Merging this in. Thanks @pentschev ! |
|
Just a heads up -- this seems to have bunch of tests in xarray: pydata/xarray#3009 |
|
Thanks for the heads up @shoyer, I will take a look at it soon. |
| seq_dtypes = [a.dtype for a in seq] | ||
| if len(set(seq_dtypes)) > 1: | ||
| dt = reduce(np.promote_types, seq_dtypes) | ||
| seq = [x.astype(dt) for x in seq] |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Leaving a note here for reference, we should be able to use _meta to perform this type promotion thus dropping this now outdated code. Handling this in PR ( #4925 ).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agreed, I totally missed that. For the record, there's still quite some work to be done in that file, as well as a few others, as tracked on #4888. Thanks a lot for fixing this @jakirkham! :)
meta was introduced [1] to replace a wwidely misuse of dtype. Turns out we were the few that uses dtype as it should be. [1] dask/dask#4543
This PR intends to add the _meta attribute, as per discussion in #2977.
Incomplete work will continue to be tracked in #4888.
Completed work:
blockwise()casestest_count_nonzero_strandtest_count_nonzero_str: broken after passing**kwargstofunc()inblockwise().__getitem__metawithfuncinpartial_reduce()metato allArray()constructors wheredtypedoes not sufficedask/array/blockwise.pydask/array/core.py(not complete yet)dask/array/creation.py(must include fixes for Dask Array's pad should handle 0-dim constants on the edge #4841, Dask Array's constant pad coerces array to NumPy ndarray #4842)dask/array/gufunc.pydask/array/image.pydask/array/linalg.pydask/array/overlap.pydask/array/percentile.pydask/array/random.pydask/array/rechunk.pydask/array/reductions.pydask/array/reshape.pydask/array/routines.pydask/array/slicing.pydask/array/ufunc.pydask/array/wrap.pydask/array/tests/test_array_core.pydask/array/tests/test_atop.pydask/array/tests/test_percentiles.pydask/array/tests/test_slicing.pydask/dataframe/core.pydask/dataframe/io/tests/test_io.pydocs/source/array-design.rst(update docs)_metaconstruction, e.g., several places currently containmeta = x[tuple(slice(0, 0, None) for _ in range(x.ndim))]and similar cases.Array(..., meta=dask_array)instead ofArray(..., meta=dask_array._meta).Known Issues:
test_from_array_scalarfails,dtypedoesn't include the length for flexible types, like|SNand<UN(whereNrepresents the string length), since that information isn't included in_metatest_nan_objectfails, differentnp.sum()behavior on arrays of type object:CI:
flake8 dask