def partial_reduce(func, x, split_every, keepdims=False, dtype=None, name=None, reduced_meta=None): """Partial reduction across multiple axes. Parameters ---------- func : function x : Array split_every : dict Maximum reduction block sizes in each dimension. Examples -------- Reduce across axis 0 and 2, merging a maximum of 1 block in the 0th dimension, and 3 blocks in the 2nd dimension: >>> partial_reduce(np.min, x, {0: 1, 2: 3}) # doctest: +SKIP """ name = ((name or funcname(func)) + "-" + tokenize(func, x, split_every, keepdims, dtype)) parts = [ list(partition_all(split_every.get(i, 1), range(n))) for (i, n) in enumerate(x.numblocks) ] keys = product(*map(range, map(len, parts))) out_chunks = [ tuple(1 for p in partition_all(split_every[i], c)) if i in split_every else c for (i, c) in enumerate(x.chunks) ] if not keepdims: out_axis = [i for i in range(x.ndim) if i not in split_every] getter = lambda k: get(out_axis, k) keys = map(getter, keys) out_chunks = list(getter(out_chunks)) dsk = {} for k, p in zip(keys, product(*parts)): decided = dict((i, j[0]) for (i, j) in enumerate(p) if len(j) == 1) dummy = dict(i for i in enumerate(p) if i[0] not in decided) g = lol_tuples((x.name, ), range(x.ndim), decided, dummy) dsk[(name, ) + k] = (func, g) graph = HighLevelGraph.from_collections(name, dsk, dependencies=[x]) meta = x._meta if reduced_meta is not None: try: meta = func(reduced_meta, computing_meta=True) # no meta keyword argument exists for func, and it isn't required except TypeError: meta = func(reduced_meta) # when no work can be computed on the empty array (e.g., func is a ufunc) except ValueError: pass # some functions can't compute empty arrays (those for which reduced_meta # fall into the ValueError exception) and we have to rely on reshaping # the array according to len(out_chunks) if is_arraylike(meta) and meta.ndim != len(out_chunks): if len(out_chunks) == 0: meta = meta.sum() else: meta = meta.reshape((0, ) * len(out_chunks)) if np.isscalar(meta): return Array(graph, name, out_chunks, dtype=dtype) else: with ignoring(AttributeError): meta = meta.astype(dtype) return Array(graph, name, out_chunks, meta=meta)
def map_overlap(func, *args, depth=None, boundary=None, trim=True, align_arrays=True, **kwargs): """Map a function over blocks of arrays with some overlap We share neighboring zones between blocks of the array, map a function, and then trim away the neighboring strips. Parameters ---------- func: function The function to apply to each extended block. If multiple arrays are provided, then the function should expect to receive chunks of each array in the same order. args : dask arrays depth: int, tuple, dict or list The number of elements that each block should share with its neighbors If a tuple or dict then this can be different per axis. If a list then each element of that list must be an int, tuple or dict defining depth for the corresponding array in `args`. Asymmetric depths may be specified using a dict value of (-/+) tuples. Note that asymmetric depths are currently only supported when ``boundary`` is 'none'. The default value is 0. boundary: str, tuple, dict or list How to handle the boundaries. Values include 'reflect', 'periodic', 'nearest', 'none', or any constant value like 0 or np.nan. If a list then each element must be a str, tuple or dict defining the boundary for the corresponding array in `args`. The default value is 'reflect'. trim: bool Whether or not to trim ``depth`` elements from each block after calling the map function. Set this to False if your mapping function already does this for you align_arrays: bool Whether or not to align chunks along equally sized dimensions when multiple arrays are provided. This allows for larger chunks in some arrays to be broken into smaller ones that match chunk sizes in other arrays such that they are compatible for block function mapping. If this is false, then an error will be thrown if arrays do not already have the same number of blocks in each dimension. **kwargs: Other keyword arguments valid in ``map_blocks`` Examples -------- >>> import numpy as np >>> import dask.array as da >>> x = np.array([1, 1, 2, 3, 3, 3, 2, 1, 1]) >>> x = da.from_array(x, chunks=5) >>> def derivative(x): ... return x - np.roll(x, 1) >>> y = x.map_overlap(derivative, depth=1, boundary=0) >>> y.compute() array([ 1, 0, 1, 1, 0, 0, -1, -1, 0]) >>> x = np.arange(16).reshape((4, 4)) >>> d = da.from_array(x, chunks=(2, 2)) >>> d.map_overlap(lambda x: x + x.size, depth=1).compute() array([[16, 17, 18, 19], [20, 21, 22, 23], [24, 25, 26, 27], [28, 29, 30, 31]]) >>> func = lambda x: x + x.size >>> depth = {0: 1, 1: 1} >>> boundary = {0: 'reflect', 1: 'none'} >>> d.map_overlap(func, depth, boundary).compute() # doctest: +NORMALIZE_WHITESPACE array([[12, 13, 14, 15], [16, 17, 18, 19], [20, 21, 22, 23], [24, 25, 26, 27]]) The ``da.map_overlap`` function can also accept multiple arrays. >>> func = lambda x, y: x + y >>> x = da.arange(8).reshape(2, 4).rechunk((1, 2)) >>> y = da.arange(4).rechunk(2) >>> da.map_overlap(func, x, y, depth=1).compute() # doctest: +NORMALIZE_WHITESPACE array([[ 0, 2, 4, 6], [ 4, 6, 8, 10]]) When multiple arrays are given, they do not need to have the same number of dimensions but they must broadcast together. Arrays are aligned block by block (just as in ``da.map_blocks``) so the blocks must have a common chunk size. This common chunking is determined automatically as long as ``align_arrays`` is True. >>> x = da.arange(8, chunks=4) >>> y = da.arange(8, chunks=2) >>> r = da.map_overlap(func, x, y, depth=1, align_arrays=True) >>> len(r.to_delayed()) 4 >>> da.map_overlap(func, x, y, depth=1, align_arrays=False).compute() Traceback (most recent call last): ... ValueError: Shapes do not align {'.0': {2, 4}} Note also that this function is equivalent to ``map_blocks`` by default. A non-zero ``depth`` must be defined for any overlap to appear in the arrays provided to ``func``. >>> func = lambda x: x.sum() >>> x = da.ones(10, dtype='int') >>> block_args = dict(chunks=(), drop_axis=0) >>> da.map_blocks(func, x, **block_args).compute() 10 >>> da.map_overlap(func, x, **block_args).compute() 10 >>> da.map_overlap(func, x, **block_args, depth=1).compute() 12 """ # Look for invocation using deprecated single-array signature # map_overlap(x, func, depth, boundary=None, trim=True, **kwargs) if isinstance(func, Array) and callable(args[0]): warnings.warn( "The use of map_overlap(array, func, **kwargs) is deprecated since dask 2.17.0 " "and will be an error in a future release. To silence this warning, use the syntax " "map_overlap(func, array0,[ array1, ...,] **kwargs) instead.", FutureWarning, ) sig = ["func", "depth", "boundary", "trim"] depth = get(sig.index("depth"), args, depth) boundary = get(sig.index("boundary"), args, boundary) trim = get(sig.index("trim"), args, trim) func, args = args[0], [func] if not callable(func): raise TypeError("First argument must be callable function, not {}\n" "Usage: da.map_overlap(function, x)\n" " or: da.map_overlap(function, x, y, z)".format( type(func).__name__)) if not all(isinstance(x, Array) for x in args): raise TypeError("All variadic arguments must be arrays, not {}\n" "Usage: da.map_overlap(function, x)\n" " or: da.map_overlap(function, x, y, z)".format( [type(x).__name__ for x in args])) # Coerce depth and boundary arguments to lists of individual # specifications for each array argument def coerce(xs, arg, fn): if not isinstance(arg, list): arg = [arg] * len(xs) return [fn(x.ndim, a) for x, a in zip(xs, arg)] depth = coerce(args, depth, coerce_depth) boundary = coerce(args, boundary, coerce_boundary) # Align chunks in each array to a common size if align_arrays: # Reverse unification order to allow block broadcasting inds = [list(reversed(range(x.ndim))) for x in args] _, args = unify_chunks(*list(concat(zip(args, inds))), warn=False) for i, x in enumerate(args): for j in range(x.ndim): if isinstance(depth[i][j], tuple) and boundary[i][j] != "none": raise NotImplementedError( "Asymmetric overlap is currently only implemented " "for boundary='none', however boundary for dimension " "{} in array argument {} is {}".format( j, i, boundary[i][j])) def assert_int_chunksize(xs): assert all(type(c) is int for x in xs for cc in x.chunks for c in cc) assert_int_chunksize(args) if not trim and "chunks" not in kwargs: kwargs["chunks"] = args[0].chunks args = [ overlap(x, depth=d, boundary=b) for x, d, b in zip(args, depth, boundary) ] assert_int_chunksize(args) x = map_blocks(func, *args, **kwargs) assert_int_chunksize([x]) if trim: # Find index of array argument with maximum rank and break ties by choosing first provided i = sorted(enumerate(args), key=lambda v: (v[1].ndim, -v[0]))[-1][0] # Trim using depth/boundary setting for array of highest rank return trim_internal(x, depth[i], boundary[i]) else: return x
name = f'lsf_nw-{num_workers}_tpw-{tpw}_chunk-locking-{chunk_locking}' levels = list(range(1,6)) crop = (slice(8192),) * 3 def reducer(v, **kwargs): return np.mean(v, dtype='float32', **kwargs) source_path = '/nrs/flyem/bench/Z0720_07m_BR.n5/render/Sec30/v1_acquire_trimmed_align___20210413_194018/s0' target_path = '/nrs/flyem/bench/Z0720_07m_BR.n5/test_dask_down/' store_chunks = read(source_path, storage_options={'normalize_keys': False}).chunks read_chunks=(1024,) * 3 data = read_xarray(source_path, storage_options={'normalize_keys': False}, chunks=read_chunks, name='test_data')[crop] multi = get(levels, multiscale(data, reducer, (2,2,2))) if not chunk_locking: for m in multi: m.data = ensure_minimum_chunksize(m.data, store_chunks) multi_store = Multiscales(name, {f's{l}' : m for l,m in zip(levels, multi)}) if __name__ == '__main__': with get_cluster(threads_per_worker=tpw) as cluster, Client(cluster) as cl: print(cl.cluster.dashboard_link) cl.cluster.scale(num_workers) cl.wait_for_workers(num_workers) with performance_report(filename=os.path.join(target_path, f'{name}_report.html')): store_group, store_arrays, storage_op = multi_store.store(target_path, locking=chunk_locking, client=cl, mode='w')