dask.array.from_array
dask.array.from_array¶
- dask.array.from_array(x, chunks='auto', name=None, lock=False, asarray=None, fancy=True, getitem=None, meta=None, inline_array=False)[source]¶
Create dask array from something that looks like an array.
Input must have a
.shape
,.ndim
,.dtype
and support numpy-style slicing.- Parameters
- xarray_like
- chunksint, tuple
How to chunk the array. Must be one of the following forms:
A blocksize like 1000.
A blockshape like (1000, 1000).
Explicit sizes of all blocks along all dimensions like ((1000, 1000, 500), (400, 400)).
A size in bytes, like “100 MiB” which will choose a uniform block-like shape
The word “auto” which acts like the above, but uses a configuration value
array.chunk-size
for the chunk size
-1 or None as a blocksize indicate the size of the corresponding dimension.
- namestr or bool, optional
The key name to use for the array. Defaults to a hash of
x
.Hashing is useful if the same value of
x
is used to create multiple arrays, as Dask can then recognise that they’re the same and avoid duplicate computations. However, it can also be slow, and if the array is not contiguous it is copied for hashing. If the array uses stride tricks (such asnumpy.broadcast_to()
orskimage.util.view_as_windows()
) to have a larger logical than physical size, this copy can cause excessive memory usage.If you don’t need the deduplication provided by hashing, use
name=False
to generate a random name instead of hashing, which avoids the pitfalls described above. Usingname=True
is equivalent to the default.By default, hashing uses python’s standard sha1. This behaviour can be changed by installing cityhash, xxhash or murmurhash. If installed, a large-factor speedup can be obtained in the tokenisation step.
Note
Because this
name
is used as the key in task graphs, you should ensure that it uniquely identifies the data contained within. If you’d like to provide a descriptive name that is still unique, combine the descriptive name withdask.base.tokenize()
of thearray_like
. See Task Graphs for more.- lockbool or Lock, optional
If
x
doesn’t support concurrent reads then provide a lock here, or pass in True to have dask.array create one for you.- asarraybool, optional
If True then call np.asarray on chunks to convert them to numpy arrays. If False then chunks are passed through unchanged. If None (default) then we use True if the
__array_function__
method is undefined.Note
Dask does not preserve the memory layout of the original array when the array is created using Fortran rather than C ordering.
- fancybool, optional
If
x
doesn’t support fancy indexing (e.g. indexing with lists or arrays) then set to False. Default is True.- metaArray-like, optional
The metadata for the resulting dask array. This is the kind of array that will result from slicing the input array. Defaults to the input array.
- inline_arraybool, default False
How to include the array in the task graph. By default (
inline_array=False
) the array is included in a task by itself, and each chunk refers to that task by its key.>>> x = h5py.File("data.h5")["/x"] >>> a = da.from_array(x, chunks=500) >>> dict(a.dask) { 'array-original-<name>': <HDF5 dataset ...>, ('array-<name>', 0): (getitem, "array-original-<name>", ...), ('array-<name>', 1): (getitem, "array-original-<name>", ...) }
With
inline_array=True
, Dask will instead inline the array directly in the values of the task graph.>>> a = da.from_array(x, chunks=500, inline_array=True) >>> dict(a.dask) { ('array-<name>', 0): (getitem, <HDF5 dataset ...>, ...), ('array-<name>', 1): (getitem, <HDF5 dataset ...>, ...) }
Note that there’s no key in the task graph with just the array x anymore. Instead it’s placed directly in the values.
The right choice for
inline_array
depends on several factors, including the size ofx
, how expensive it is to create, which scheduler you’re using, and the pattern of downstream computations. As a heuristic,inline_array=True
may be the right choice when the arrayx
is cheap to serialize and deserialize (since it’s included in the graph many times) and if you’re experiencing ordering issues (see Ordering for more).This has no effect when
x
is a NumPy array.
Examples
>>> x = h5py.File('...')['/data/path'] >>> a = da.from_array(x, chunks=(1000, 1000))
If your underlying datastore does not support concurrent reads then include the
lock=True
keyword argument orlock=mylock
if you want multiple arrays to coordinate around the same lock.>>> a = da.from_array(x, chunks=(1000, 1000), lock=True)
If your underlying datastore has a
.chunks
attribute (as h5py and zarr datasets do) then a multiple of that chunk shape will be used if you do not provide a chunk shape.>>> a = da.from_array(x, chunks='auto') >>> a = da.from_array(x, chunks='100 MiB') >>> a = da.from_array(x)
If providing a name, ensure that it is unique
>>> import dask.base >>> token = dask.base.tokenize(x) >>> a = da.from_array('myarray-' + token)
NumPy ndarrays are eagerly sliced and then embedded in the graph.
>>> import dask.array >>> a = dask.array.from_array(np.array([[1, 2], [3, 4]]), chunks=(1,1)) >>> a.dask[a.name, 0, 0][0] array([1])
Chunks with exactly-specified, different sizes can be created.
>>> import numpy as np >>> import dask.array as da >>> rng = np.random.default_rng() >>> x = rng.random((100, 6)) >>> a = da.from_array(x, chunks=((67, 33), (6,)))