Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
103 commits
Select commit Hold shift + click to select a range
f909281
Dask Array: add _meta attribute
pentschev Mar 5, 2019
0bf17c3
Use _meta attribute for diag() construction
pentschev Mar 5, 2019
2bfec0d
Dask Array: avoid using empty_like() for _meta
pentschev Mar 5, 2019
c8b61af
Revert "Use _meta attribute for diag() construction"
pentschev Mar 5, 2019
f54856b
Add sparse meta test
mrocklin Mar 5, 2019
70d2a01
Assign to _meta rather than dtype
mrocklin Mar 5, 2019
db5e021
More support for Array._meta
pentschev Mar 6, 2019
1323ee7
Remove redundant return value from blockwise.py
pentschev Mar 6, 2019
04f70f8
Consider more _meta conditions in blockwise()
pentschev Mar 7, 2019
95856d7
Pass missing _meta to Array constructor calls
pentschev Mar 14, 2019
3e66595
Handle _meta slicing in __getitem__
pentschev Mar 14, 2019
2e0ecf0
Handle reduction of _meta
pentschev Mar 14, 2019
1eddd26
Test for proper _meta.ndim in assert_eq
pentschev Mar 14, 2019
30c98c0
Do not use np.empty() for blockwise's meta construction
pentschev Mar 19, 2019
7f0ebe8
Fix a couple of _meta indexing bugs
pentschev Mar 20, 2019
f584872
Compensate _meta for NaN dims in compress()
pentschev Mar 20, 2019
c9fad68
Merge remote-tracking branch 'upstream/master' into dask-array-meta
pentschev Mar 20, 2019
ce7e5dd
Simplify handling of _meta for reductions
pentschev Mar 22, 2019
646ca13
Avoid bare exceptions when calling meta = func() on blockwise
pentschev Mar 22, 2019
fd5406f
Cleanup of some _meta handling in core
pentschev Mar 22, 2019
3dec891
Handle _meta in concatenate
pentschev Mar 26, 2019
2c05399
Handle _meta in rechunk
pentschev Mar 26, 2019
b703153
Handle blockwise multi-dim _meta on types that do not contain _meta
pentschev Mar 26, 2019
8c57904
Add _meta test for elemwise, rechunk, concatenate
pentschev Mar 26, 2019
ef4f970
Add wrap for *_like() functions, uses them for overlap
pentschev Apr 1, 2019
2cb0e3b
Schedule computation with simple scheduler
mrocklin Apr 16, 2019
49f0b8e
Merge remote-tracking branch 'upstream/master' into dask-array-meta
pentschev Apr 17, 2019
8d4de71
Fix for reduction resulting on scalar _meta
pentschev May 2, 2019
b8f63db
Add _meta support for frexp and modf functions
pentschev May 7, 2019
185765d
Fix for count_nonzero errors with _meta
pentschev May 8, 2019
0bab189
Fix call np.full_like with _meta in constant()
pentschev May 8, 2019
0f63efb
Add missing dtype, ndim attributes to ARange test classes
pentschev May 9, 2019
3698661
Further adjustment in _meta dimension during __getitem__
pentschev May 9, 2019
1cf6d72
Fix _meta shape on reduction functions that can't compute empty arrays
pentschev May 10, 2019
096e4c1
Add _meta support to diag()
pentschev May 13, 2019
6f95928
Add CuPy diag() test
pentschev May 13, 2019
7aad860
Add more _meta checks in assert_eq
pentschev May 13, 2019
48add7d
Add _meta support to reshape()
pentschev May 13, 2019
af69515
Add more reshape() test with CuPy
pentschev May 13, 2019
fe96d2b
Add check_meta argument to assert_eq function
pentschev May 13, 2019
0acb35d
Disable meta checking for mixed arrays in test_masked
pentschev May 13, 2019
fb49b15
Add _meta support to diagonal()
pentschev May 13, 2019
6808517
Add CuPy test for diagonal()
pentschev May 13, 2019
f5281ee
Remove concatenate() dtype logic in favor of _meta
pentschev May 14, 2019
2a62263
Add back concatenate() dtype handling of input data
pentschev May 14, 2019
0bb0dea
Remove use of bare except in blockwise
pentschev May 14, 2019
0281f06
Use assert_eq to compare metas
pentschev May 14, 2019
757e7ba
Add _meta support for tril/triu functions
pentschev May 14, 2019
6812e68
Simplify numpy_like_safe
pentschev May 14, 2019
3885f97
Use numpy_like_safe() for frexp/modf functions
pentschev May 14, 2019
87eed6b
Fix _meta computation for multidimensional frexp/modf
pentschev May 14, 2019
3d2f396
Better blockwise handling
pentschev May 14, 2019
abcddcb
Add normalize_meta utility function
pentschev May 14, 2019
cf59c47
Add gufunc _meta support
pentschev May 14, 2019
ac44219
Pass *args to numpy_like_safe, allowing full_like to function
pentschev May 16, 2019
2135cb7
Use numpy_like_safe for numel() in reductions
pentschev May 16, 2019
4e0ea2a
Add more CuPy reduction tests
pentschev May 16, 2019
1abb56d
Split numpy_like_safe into multiple variants (empty, full, ones, zeros)
pentschev May 16, 2019
9a2d333
Allow CuPy tests requiring shape in *_like() to xfail
pentschev May 16, 2019
e3370f2
Add accidentally removed condition back to numel()
pentschev May 16, 2019
99e8a79
Add new meta_from_array utility function
pentschev May 20, 2019
62f5a42
Add _meta support to overlap_internal()
pentschev May 20, 2019
edc7650
Add CuPy tests for overlap-related functions
pentschev May 20, 2019
30bc10d
Add safe wrapper for *_like functions
pentschev May 20, 2019
3f659ea
Add dtype keyword argument to meta_from_array()
pentschev May 20, 2019
d78540a
Add _meta support for random functions
pentschev May 21, 2019
0cada3e
Add tests for random functions with CuPy
pentschev May 21, 2019
80cf6d2
Ensure slicing of _meta is 0-D when array is 0-D
pentschev May 22, 2019
37f9e9c
Fix _meta on squeeze()
pentschev May 22, 2019
80e11c6
Add _meta support to tsqr and sfqr functions
pentschev May 22, 2019
c22dea5
Check for NumPy version in special _tensordot() condition
pentschev May 22, 2019
a52806a
Add CuPy tests for tsqr/sfqr functions
pentschev May 22, 2019
d6912ec
Normalize meta within meta_from_array()
pentschev May 22, 2019
39f43e7
Split _meta computation from blockwise, fix some reduction errors
pentschev May 23, 2019
1640271
Merge remote-tracking branch 'upstream/master' into dask-array-meta
pentschev May 23, 2019
02d9fb4
Fix _meta dimensions in __getitem__
pentschev May 23, 2019
b76f1c2
Skip CuPy tests if __array_function__ is not available
pentschev May 23, 2019
2f2cf72
Fix dataframe ufunc issue related to new Array._meta format
pentschev May 24, 2019
3c34b56
Skip Sparse concatenate test when __array_function__ is disabled
pentschev May 24, 2019
66a6d00
Allow Sparse tensordot() test to xfail
pentschev May 24, 2019
3a27b7a
Ignore assert_eq() check for matching flexible dtype lengths
pentschev May 24, 2019
c419a07
Fix flake8 errors
pentschev May 24, 2019
d2abc3c
Add workaround for _meta with TileDB in from_array()
pentschev May 24, 2019
342b8ac
Fix exception in overlap call to wrap.empty_like
pentschev May 24, 2019
b7bfbb0
Removed xfail from test_tensordot, fixed in Sparse upstream
pentschev May 29, 2019
834c806
Add _meta support to remaining linalg functions
pentschev May 29, 2019
113d6e4
Use zeros_like_safe for NEP-18 support in bincount()
pentschev May 29, 2019
90fdfb2
Add bincount CuPy test
pentschev May 29, 2019
136e0de
Merge remote-tracking branch 'upstream/master' into dask-array-meta
pentschev May 29, 2019
426ed92
Skip test_tensordot if sparse < 0.7.0+10
pentschev Jun 1, 2019
dc5b7e7
Raise TypeError if both dtype and meta are passed to Array()
pentschev Jun 3, 2019
9c41a54
Improve meta_from_array with list support
pentschev Jun 3, 2019
bbc5910
Increase usage of meta_from_array
pentschev Jun 3, 2019
45a852f
Only use _meta in meta_from_array for Dask arrays
pentschev Jun 3, 2019
1395b20
Allow passing Dask Array as meta in Array constructor
pentschev Jun 3, 2019
c84276a
Safer handling of tiledb in from_array
pentschev Jun 3, 2019
509c4f2
Fix flake8 errors
pentschev Jun 3, 2019
1e778a8
Handle _meta automatically by slicing within squeeze()
pentschev Jun 3, 2019
c7ec24d
Inline condition/reason for NEP-18 skipping within test_cupy
pentschev Jun 3, 2019
7149c78
Remove some compute() calls from test_cupy
pentschev Jun 3, 2019
6c4ea85
Use zeros_like_safe in diag
pentschev Jun 3, 2019
eb83eb6
Update CuPy moment tests
pentschev Jun 3, 2019
9879a77
Remove unused normalize_meta import
pentschev Jun 3, 2019
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
36 changes: 35 additions & 1 deletion dask/array/blockwise.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,12 +3,42 @@

import toolz

import numpy as np

from .. import base, utils
from ..delayed import unpack_collections
from ..highlevelgraph import HighLevelGraph
from ..blockwise import blockwise as core_blockwise


def blockwise_meta(func, dtype, *args, **kwargs):
arrays = args[::2]
ndims = [a.ndim if hasattr(a, 'ndim') else 0 for a in arrays]
args_meta = [arg._meta if hasattr(arg, '_meta') else
arg[tuple(slice(0, 0, None) for _ in range(nd))] if nd > 0 else arg
for arg, nd in zip(arrays, ndims)]
kwargs_meta = {k: v._meta if hasattr(v, '_meta') else v for k, v in kwargs.items()}

# TODO: look for alternative to this, causes issues when using map_blocks()
# with np.vectorize, such as dask.array.routines._isnonzero_vec().
if isinstance(func, np.vectorize):
meta = func(*args_meta)
return meta.astype(dtype)

try:
meta = func(*args_meta, **kwargs_meta)
except TypeError:
# The concatenate argument is an argument introduced by this
# function and may not be support by some external functions,
# such as in NumPy
kwargs_meta.pop('concatenate', None)
meta = func(*args_meta, **kwargs_meta)
except ValueError:
return None

return meta.astype(dtype)


def blockwise(func, out_ind, *args, **kwargs):
""" Tensor operation: Generalized inner and outer products

Expand Down Expand Up @@ -203,7 +233,11 @@ def blockwise(func, out_ind, *args, **kwargs):
"adjust_chunks values must be callable, int, or tuple")
chunks = tuple(chunks)

return Array(graph, out, chunks, dtype=dtype)
try:
meta = blockwise_meta(func, dtype, *args, **kwargs)
return Array(graph, out, chunks, meta=meta)
except Exception:
return Array(graph, out, chunks, dtype=dtype)


def atop(*args, **kwargs):
Expand Down
88 changes: 69 additions & 19 deletions dask/array/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -731,7 +731,7 @@ def store(sources, targets, lock=True, regions=None, compute=True,
sources_dsk,
list(core.flatten([e.__dask_keys__() for e in sources]))
)
sources2 = [Array(sources_dsk, e.name, e.chunks, e.dtype) for e in sources]
sources2 = [Array(sources_dsk, e.name, e.chunks, meta=e) for e in sources]

# Optimize all targets together
targets2 = []
Expand Down Expand Up @@ -774,7 +774,7 @@ def store(sources, targets, lock=True, regions=None, compute=True,
)

result = tuple(
Array(load_store_dsk, 'load-store-%s' % t, s.chunks, s.dtype)
Array(load_store_dsk, 'load-store-%s' % t, s.chunks, meta=s)
for s, t in zip(sources, toks)
)

Expand Down Expand Up @@ -857,28 +857,40 @@ class Array(DaskMethodsMixin):
Shape of the entire array
chunks: iterable of tuples
block sizes along each dimension
dtype : str or dtype
Typecode or data-type for the new Dask Array
meta : empty ndarray
empty ndarray created with same NumPy backend, ndim and dtype as the
Dask Array being created (overrides dtype)

See Also
--------
dask.array.from_array
"""
__slots__ = 'dask', '_name', '_cached_keys', '_chunks', 'dtype'
__slots__ = 'dask', '_name', '_cached_keys', '_chunks', '_meta'

def __new__(cls, dask, name, chunks, dtype, shape=None):
def __new__(cls, dask, name, chunks, dtype=None, meta=None, shape=None):
self = super(Array, cls).__new__(cls)
assert isinstance(dask, Mapping)
if not isinstance(dask, HighLevelGraph):
dask = HighLevelGraph.from_collections(name, dask, dependencies=())
self.dask = dask
self.name = name
if dtype is None:
raise ValueError("You must specify the dtype of the array")
self.dtype = np.dtype(dtype)
if dtype is not None and meta is not None:
raise TypeError("You must not specify both meta and dtype")
if dtype is None and meta is None:
raise ValueError("You must specify the meta or dtype of the array")

self._chunks = normalize_chunks(chunks, shape, dtype=self.dtype)
self._chunks = normalize_chunks(chunks, shape, dtype=dtype or meta.dtype)
if self._chunks is None:
raise ValueError(CHUNKS_NONE_ERROR_MESSAGE)

if dtype:
self._meta = np.empty((0,) * self.ndim, dtype=dtype)
else:
from .utils import meta_from_array
self._meta = meta_from_array(meta, meta.ndim)

for plugin in config.get('array_plugins', ()):
result = plugin(self)
if result is not None:
Expand Down Expand Up @@ -944,8 +956,8 @@ def chunksize(self):
return tuple(max(c) for c in self.chunks)

@property
def _meta(self):
return np.empty(shape=(), dtype=self.dtype)
def dtype(self):
return self._meta.dtype

def _get_chunks(self):
return self._chunks
Expand Down Expand Up @@ -1217,7 +1229,7 @@ def __setitem__(self, key, value):
if isinstance(value, Array) and value.ndim > 1:
raise ValueError('boolean index array should have 1 dimension')
y = where(key, value, self)
self.dtype = y.dtype
self._meta = y._meta
self.dask = y.dask
self.name = y.name
self._chunks = y.chunks
Expand Down Expand Up @@ -1267,7 +1279,37 @@ def __getitem__(self, index):
dsk, chunks = slice_array(out, self.name, self.chunks, index2)

graph = HighLevelGraph.from_collections(out, dsk, dependencies=[self])
return Array(graph, out, chunks, dtype=self.dtype)

if isinstance(index2, tuple):
new_index = []
for i in range(len(index2)):
if not isinstance(index2[i], tuple):
types = [Integral, list, np.ndarray]
cond = any([isinstance(index2[i], t) for t in types])
new_index.append(slice(0, 0) if cond else index2[i])
else:
new_index.append(tuple([Ellipsis if j is not None else
None for j in index2[i]]))
new_index = tuple(new_index)
meta = self._meta[new_index].astype(self.dtype)
else:
meta = self._meta[index2].astype(self.dtype)
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 going to try to defer to @jakirkham on this :)

Copy link
Member

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?

Copy link
Member Author

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:

  1. Check all elements of the index2 tuple, if it's not a tuple, make it a slice(0, 0);
  2. For elements that are not tuples (e.g., a single index), make it an Ellipsis if the index is valid, otherwise make it None.

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), the 1s that arise from None should probably still be fixed, even though it may not be a problem generally

Copy link
Member

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?

Copy link
Member Author

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) and Ellipsis on _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 None would 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.


# Exception for object dtype and ndim == 1, which results in primitive types
if not (meta.dtype == object and meta.ndim == 1):

# If meta still has more dimensions than actual data
if meta.ndim > len(chunks):
meta = np.sum(meta, axis=tuple([i for i in range(meta.ndim - len(chunks))]))

# Ensure all dimensions are 0
if not np.isscalar(meta):
meta = meta[tuple([slice(0, 0) for i in range(meta.ndim)])]
# If return array is 0-D, ensure _meta is 0-D
if len(chunks) == 0:
meta = meta.sum()

return Array(graph, out, chunks, meta=meta)

def _vindex(self, key):
if not isinstance(key, tuple):
Expand Down Expand Up @@ -1336,7 +1378,7 @@ def _blocks(self, index):
layer = {(name,) + key: tuple(new_keys[key].tolist()) for key in keys}

graph = HighLevelGraph.from_collections(name, layer, dependencies=[self])
return Array(graph, name, chunks, self.dtype)
return Array(graph, name, chunks, meta=self)

@property
def blocks(self):
Expand Down Expand Up @@ -1848,7 +1890,7 @@ def copy(self):
if self.npartitions == 1:
return self.map_blocks(M.copy)
else:
return Array(self.dask, self.name, self.chunks, self.dtype)
return Array(self.dask, self.name, self.chunks, meta=self)

def __deepcopy__(self, memo):
c = self.copy()
Expand Down Expand Up @@ -2292,7 +2334,12 @@ def from_array(x, chunks='auto', name=None, lock=False, asarray=True, fancy=True
dtype=x.dtype)
dsk[original_name] = x

return Array(dsk, name, chunks, dtype=x.dtype)
# Workaround for TileDB, its indexing is 1-based,
# and doesn't seems to support 0-length slicing
if x.__class__.__module__.split('.')[0] == 'tiledb' and hasattr(x, '_ctx_'):
return Array(dsk, name, chunks, dtype=x.dtype)

return Array(dsk, name, chunks, meta=x)


def from_zarr(url, component=None, storage_options=None, chunks=None,name=None, **kwargs):
Expand Down Expand Up @@ -2908,6 +2955,11 @@ def concatenate(seq, axis=0, allow_unknown_chunksizes=False):
for i, ind in enumerate(inds):
ind[axis] = -(i + 1)

from .utils import meta_from_array
metas = [getattr(s, '_meta', s) for s in seq]
metas = [meta_from_array(m, getattr(m, 'ndim', 1)) for m in metas]
meta = np.concatenate(metas)

uc_args = list(concat(zip(seq, inds)))
_, seq = unify_chunks(*uc_args, warn=False)

Expand All @@ -2922,8 +2974,6 @@ def concatenate(seq, axis=0, allow_unknown_chunksizes=False):
if len(set(seq_dtypes)) > 1:
dt = reduce(np.promote_types, seq_dtypes)
seq = [x.astype(dt) for x in seq]
Copy link
Member

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 ).

Copy link
Member Author

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! :)

else:
dt = seq_dtypes[0]

names = [a.name for a in seq]

Expand All @@ -2937,7 +2987,7 @@ def concatenate(seq, axis=0, allow_unknown_chunksizes=False):
dsk = dict(zip(keys, values))
graph = HighLevelGraph.from_collections(name, dsk, dependencies=seq)

return Array(graph, name, chunks, dtype=dt)
return Array(graph, name, chunks, meta=meta)


def load_store_chunk(x, out, index, lock, return_stored, load_stored):
Expand Down Expand Up @@ -3318,7 +3368,7 @@ def handle_out(out, result):
"out=%s, result=%s" % (str(out.shape), str(result.shape)))
out._chunks = result.chunks
out.dask = result.dask
out.dtype = result.dtype
out._meta = result._meta
out.name = result.name
elif out is not None:
msg = ("The out parameter is not fully supported."
Expand Down
32 changes: 21 additions & 11 deletions dask/array/creation.py
Original file line number Diff line number Diff line change
Expand Up @@ -11,13 +11,13 @@
from ..highlevelgraph import HighLevelGraph
from ..base import tokenize
from ..compatibility import Sequence
from ..utils import derived_from
from . import chunk
from .core import (Array, asarray, normalize_chunks,
stack, concatenate, block,
broadcast_to, broadcast_arrays)
from .wrap import empty, ones, zeros, full
from .utils import AxisError
from ..utils import derived_from
from .utils import AxisError, meta_from_array, zeros_like_safe


def empty_like(a, dtype=None, chunks=None):
Expand Down Expand Up @@ -473,7 +473,11 @@ def eye(N, chunks='auto', M=None, k=0, dtype=float):
@derived_from(np)
def diag(v):
name = 'diag-' + tokenize(v)
if isinstance(v, np.ndarray):

meta = meta_from_array(v, 2 if v.ndim == 1 else 1)

if (isinstance(v, np.ndarray) or
(hasattr(v, '__array_function__') and not isinstance(v, Array))):
if v.ndim == 1:
chunks = ((v.shape[0],), (v.shape[0],))
dsk = {(name, 0, 0): (np.diag, v)}
Expand All @@ -482,7 +486,7 @@ def diag(v):
dsk = {(name, 0): (np.diag, v)}
else:
raise ValueError("Array must be 1d or 2d only")
return Array(dsk, name, chunks, dtype=v.dtype)
return Array(dsk, name, chunks, meta=meta)
if not isinstance(v, Array):
raise TypeError("v must be a dask array or numpy array, "
"got {0}".format(type(v)))
Expand All @@ -491,7 +495,7 @@ def diag(v):
dsk = {(name, i): (np.diag, row[i])
for i, row in enumerate(v.__dask_keys__())}
graph = HighLevelGraph.from_collections(name, dsk, dependencies=[v])
return Array(graph, name, (v.chunks[0],), dtype=v.dtype)
return Array(graph, name, (v.chunks[0],), meta=meta)
else:
raise NotImplementedError("Extracting diagonals from non-square "
"chunked arrays")
Expand All @@ -505,9 +509,10 @@ def diag(v):
dsk[key] = (np.diag, blocks[i])
else:
dsk[key] = (np.zeros, (m, n))
dsk[key] = (partial(zeros_like_safe, shape=(m, n)), meta)

graph = HighLevelGraph.from_collections(name, dsk, dependencies=[v])
return Array(graph, name, (chunks_1d, chunks_1d), dtype=v.dtype)
return Array(graph, name, (chunks_1d, chunks_1d), meta=meta)


@derived_from(np)
Expand Down Expand Up @@ -574,7 +579,8 @@ def _diag_len(dim1, dim2, offset):
chunks = left_chunks + right_shape

graph = HighLevelGraph.from_collections(name, dsk, dependencies=[a])
return Array(graph, name, shape=shape, chunks=chunks, dtype=a.dtype)
meta = meta_from_array(a, len(shape))
return Array(graph, name, shape=shape, chunks=chunks, meta=meta)


def triu(m, k=0):
Expand Down Expand Up @@ -616,13 +622,15 @@ def triu(m, k=0):
for i in range(rdim):
for j in range(hdim):
if chunk * (j - i + 1) < k:
dsk[(name, i, j)] = (np.zeros, (m.chunks[0][i], m.chunks[1][j]))
dsk[(name, i, j)] = (partial(zeros_like_safe,
shape=(m.chunks[0][i], m.chunks[1][j])),
m._meta)
elif chunk * (j - i - 1) < k <= chunk * (j - i + 1):
dsk[(name, i, j)] = (np.triu, (m.name, i, j), k - (chunk * (j - i)))
else:
dsk[(name, i, j)] = (m.name, i, j)
graph = HighLevelGraph.from_collections(name, dsk, dependencies=[m])
return Array(graph, name, shape=m.shape, chunks=m.chunks, dtype=m.dtype)
return Array(graph, name, shape=m.shape, chunks=m.chunks, meta=m)


def tril(m, k=0):
Expand Down Expand Up @@ -668,9 +676,11 @@ def tril(m, k=0):
elif chunk * (j - i - 1) < k <= chunk * (j - i + 1):
dsk[(name, i, j)] = (np.tril, (m.name, i, j), k - (chunk * (j - i)))
else:
dsk[(name, i, j)] = (np.zeros, (m.chunks[0][i], m.chunks[1][j]))
dsk[(name, i, j)] = (partial(zeros_like_safe,
shape=(m.chunks[0][i], m.chunks[1][j])),
m._meta)
graph = HighLevelGraph.from_collections(name, dsk, dependencies=[m])
return Array(graph, name, shape=m.shape, chunks=m.chunks, dtype=m.dtype)
return Array(graph, name, shape=m.shape, chunks=m.chunks, meta=m)


def _np_fromfunction(func, shape, dtype, offset, func_kwargs):
Expand Down
4 changes: 3 additions & 1 deletion dask/array/gufunc.py
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@
from toolz import concat, merge, unique

from .core import Array, asarray, blockwise, getitem, apply_infer_dtype
from .utils import normalize_meta
from ..highlevelgraph import HighLevelGraph
from ..core import flatten

Expand Down Expand Up @@ -397,11 +398,12 @@ def apply_gufunc(func, signature, *args, **kwargs):
leaf_name = "%s_%d-%s" % (name, i, token)
leaf_dsk = {(leaf_name,) + key[1:] + core_chunkinds: ((getitem, key, i) if nout else key) for key in keys}
graph = HighLevelGraph.from_collections(leaf_name, leaf_dsk, dependencies=[tmp])
meta = normalize_meta(tmp._meta, len(output_shape), dtype=odt)
leaf_arr = Array(graph,
leaf_name,
chunks=output_chunks,
shape=output_shape,
dtype=odt)
meta=meta)

### Axes:
if keepdims:
Expand Down
Loading