def diag(v, k=0): if not isinstance(v, np.ndarray) and not isinstance(v, Array): raise TypeError( f"v must be a dask array or numpy array, got {type(v)}") name = "diag-" + tokenize(v, k) 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: m = abs(k) chunks = ((v.shape[0] + m, ), (v.shape[0] + m, )) dsk = {(name, 0, 0): (np.diag, v, k)} elif v.ndim == 2: kdiag_row_start = max(0, -k) kdiag_row_stop = min(v.shape[0], v.shape[1] - k) len_kdiag = kdiag_row_stop - kdiag_row_start chunks = ((0, ), ) if len_kdiag <= 0 else ((len_kdiag, ), ) dsk = {(name, 0): (np.diag, v, k)} else: raise ValueError("Array must be 1d or 2d only") return Array(dsk, name, chunks, meta=meta) if v.ndim != 1: if v.ndim != 2: raise ValueError("Array must be 1d or 2d only") if k == 0 and v.chunks[0] == v.chunks[1]: 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], ), meta=meta) else: return diagonal(v, k) if k == 0: chunks_1d = v.chunks[0] blocks = v.__dask_keys__() dsk = {} for i, m in enumerate(chunks_1d): for j, n in enumerate(chunks_1d): key = (name, i, j) if i == j: dsk[key] = (np.diag, blocks[i]) else: dsk[key] = (np.zeros, (m, n)) dsk[key] = (partial(np.zeros_like, shape=(m, n)), meta) graph = HighLevelGraph.from_collections(name, dsk, dependencies=[v]) return Array(graph, name, (chunks_1d, chunks_1d), meta=meta) elif k > 0: return pad(diag(v), [[0, k], [k, 0]], mode="constant") elif k < 0: return pad(diag(v), [[-k, 0], [0, -k]], mode="constant")
def stream_reduction(time_index, antenna1, antenna2, dde1_jones, source_coh, dde2_jones, predict_check_tup, out_dtype, streams): """ Reduces source coherencies + ddes over the source dimension in ``N`` parallel streams. This is accomplished by calling predict_vis on on ddes and source coherencies to produce visibilities which are passed into the `base_vis` argument of ``predict_vis`` for the next chunk. """ # Unique name and token for this operation token = tokenize(time_index, antenna1, antenna2, dde1_jones, source_coh, dde2_jones, streams) name = 'stream-coherency-reduction-' + token # Number of dim blocks blocks = _extract_blocks(time_index, dde1_jones, source_coh, dde2_jones) (src_blocks, row_blocks, _, chan_blocks), corr_blocks = blocks[:4], blocks[4:] # Total number of other dimension blocks nblocks = reduce(mul, (row_blocks, chan_blocks) + corr_blocks, 1) # Create the compressed mapping layers = CoherencyStreamReduction(time_index, antenna1, antenna2, dde1_jones, source_coh, dde2_jones, name, streams) # Create the graph extra_deps = [a for a in (dde1_jones, source_coh, dde2_jones) if a is not None] deps = [time_index, antenna1, antenna2] + extra_deps graph = HighLevelGraph.from_collections(name, layers, deps) chunks = ((1,) * src_blocks, (1,)*nblocks) # This should never be directly computed, reported chunks # and dtype don't match the actual data. We create it # because it makes chaining HighLevelGraphs easier stream_reduction = da.Array(graph, name, chunks, dtype=np.int8) name = "coherency-reduction-" + tokenize(stream_reduction) layers = CoherencyFinalReduction(name, layers) graph = HighLevelGraph.from_collections(name, layers, [stream_reduction]) chunks = _extract_chunks(time_index, dde1_jones, source_coh, dde2_jones) return da.Array(graph, name, chunks[1:], dtype=out_dtype)
def rearrange_by_column_disk(df, column, npartitions=None, compute=False): """Shuffle using local disk See Also -------- rearrange_by_column_tasks: Same function, but using tasks rather than partd Has a more informative docstring """ if npartitions is None: npartitions = df.npartitions token = tokenize(df, column, npartitions) always_new_token = uuid.uuid1().hex p = ("zpartd-" + always_new_token, ) dsk1 = {p: (maybe_buffered_partd(), )} # Partition data on disk name = "shuffle-partition-" + always_new_token dsk2 = {(name, i): (shuffle_group_3, key, column, npartitions, p) for i, key in enumerate(df.__dask_keys__())} dependencies = [] if compute: graph = HighLevelGraph.merge(df.dask, dsk1, dsk2) graph = HighLevelGraph.from_collections(name, graph, dependencies=[df]) keys = [p, sorted(dsk2)] pp, values = compute_as_if_collection(DataFrame, graph, keys) dsk1 = {p: pp} dsk2 = dict(zip(sorted(dsk2), values)) else: dependencies.append(df) # Barrier barrier_token = "barrier-" + always_new_token dsk3 = {barrier_token: (barrier, list(dsk2))} # Collect groups name = "shuffle-collect-" + token dsk4 = {(name, i): (collect, p, i, df._meta, barrier_token) for i in range(npartitions)} divisions = (None, ) * (npartitions + 1) layer = toolz.merge(dsk1, dsk2, dsk3, dsk4) graph = HighLevelGraph.from_collections(name, layer, dependencies=dependencies) return new_dd_object(graph, name, df._meta, divisions)
def _da_from_mem( token: Delayed, shape: ShapeLike, dtype: DtypeLike, chunks: Tuple[int, ...], name: str = "from_mem", ) -> da.Array: """ Construct dask view of some yet to be computed in RAM store. :param token: Should evaluate to either Token or string key in to the Cache, which is expected to contain ``numpy`` array of supplied ``shape`` and ``dtype`` :param shape: Expected shape of the future array :param dtype: Expected dtype of the future array :param chunks: Tuple of integers describing chunk partitioning for output array :param name: Dask name Gotchas ======= - Output array can not be moved from one worker to another. - Works with in-process Client - Works with single worker cluster - Can work if scheduler is told to schedule this on a single worker - Cache life cycle management can be tough. If token evaluates to a ``Token`` object then automatic cache cleanup should happen when output array is destroyed. If it is just a string, then it's up to caller to ensure that there is cleanup and no use after free. Returns ======= Dask Array """ if not isinstance(shape, tuple): shape = (shape, ) assert dask.is_dask_collection(token) assert len(shape) == len(chunks) _chunks = unpack_chunks(chunks, shape) _rois = [tuple(_roi_from_chunks(ch)) for ch in _chunks] _roi = lambda idx: tuple(_rois[i][k] for i, k in enumerate(idx)) shape_in_chunks = tuple(len(ch) for ch in _chunks) dsk = {} name = randomize(name) for idx in np.ndindex(shape_in_chunks): dsk[(name, *idx)] = (_chunk_extractor, token.key, _roi(idx)) dsk = HighLevelGraph.from_collections(name, dsk, dependencies=[token]) return da.Array(dsk, name, shape=shape, dtype=dtype, chunks=_chunks)
def test_write_dict_data(tmp_path, chunks, dtype): rs = np.random.RandomState(42) row_sum = 0 def _vis_factory(chan, corr): # Variably sized-channels per row, as in BDA data nchan = rs.randint(chan) return (rs.normal(size=(1, nchan, corr)) + rs.normal(size=(1, nchan, corr))*1j) shapes = {k: sum(c) for k, c in chunks.items()} row_sum += shapes['row'] # assert len(chunks['chan']) == 1 assert len(chunks['corr']) == 1 # Make some visibilities dims = ("row", "chan", "corr") row, chan, corr = (shapes[d] for d in dims) name = "vis-data-" + uuid.uuid4().hex nchunks = (len(chunks[d]) for d in dims) keys = product((name,), *(range(c) for c in nchunks)) chunk_sizes = product(*(chunks[d] for d in dims)) layer = {k: {'r%d' % (i + 1): _vis_factory(chan, corr) for i in range(r)} for k, (r, _, _) in zip(keys, chunk_sizes)} hlg = HighLevelGraph.from_collections(name, layer, []) chunks = tuple(chunks[d] for d in dims) meta = np.empty((0,)*len(chunks), dtype=np.complex128) vis = da.Array(hlg, name, chunks, meta=meta) ds = Dataset({"DATA": (dims, vis)}) table_name = os.path.join(str(tmp_path), 'test.table') writes, table_proxy = write_datasets(table_name, ds, ["DATA"], table_proxy=True, # No fixed shape columns descriptor="ms(False)") dask.compute(writes) data = table_proxy.getvarcol("DATA").result() # First row chunk assert_array_almost_equal(layer[(name, 0, 0, 0)]['r1'], data['r1']) assert_array_almost_equal(layer[(name, 0, 0, 0)]['r2'], data['r2']) assert_array_almost_equal(layer[(name, 0, 0, 0)]['r3'], data['r3']) assert_array_almost_equal(layer[(name, 0, 0, 0)]['r4'], data['r4']) assert_array_almost_equal(layer[(name, 0, 0, 0)]['r5'], data['r5']) # Second row chunk assert_array_almost_equal(layer[(name, 1, 0, 0)]['r1'], data['r6']) assert_array_almost_equal(layer[(name, 1, 0, 0)]['r2'], data['r7']) assert_array_almost_equal(layer[(name, 1, 0, 0)]['r3'], data['r8']) # Third row chunk assert_array_almost_equal(layer[(name, 2, 0, 0)]['r1'], data['r9']) assert_array_almost_equal(layer[(name, 2, 0, 0)]['r2'], data['r10'])
def chan_metadata(row_chan_arrays, chan_arrays, chan_bin_size): """ Create dask array with channel metadata for each chunk channel """ chan_chunks = None for array in row_chan_arrays: if array is not None: chan_chunks = array.chunks[1] break if chan_chunks is None: for array in chan_arrays: if array is not None: chan_chunks = array.chunks[0] break if chan_chunks is None: return None # Create a dask channel mapping structure name = "channel-mapper-" + tokenize(chan_chunks, chan_bin_size) layers = {(name, i): (np_channel_mapper, c, chan_bin_size) for i, c in enumerate(chan_chunks)} graph = HighLevelGraph.from_collections(name, layers, ()) chunks = (chan_chunks, ) chan_mapper = da.Array(graph, name, chunks, dtype=np.object) return chan_mapper
def call_function(func, func_token, args, kwargs, pure=None, nout=None): dask_key_name = kwargs.pop("dask_key_name", None) pure = kwargs.pop("pure", pure) if dask_key_name is None: name = "{}-{}".format( funcname(func), tokenize(func_token, *args, pure=pure, **kwargs), ) else: name = dask_key_name args2, collections = unzip(map(unpack_collections, args), 2) collections = list(concat(collections)) if kwargs: dask_kwargs, collections2 = unpack_collections(kwargs) collections.extend(collections2) task = (apply, func, list(args2), dask_kwargs) else: task = (func, ) + args2 graph = HighLevelGraph.from_collections(name, {name: task}, dependencies=collections) nout = nout if nout is not None else None return Delayed(name, graph, length=nout)
def optimize(dsk, keys, **kwargs): if not isinstance(keys, (list, set)): keys = [keys] if not isinstance(dsk, HighLevelGraph): dsk = HighLevelGraph.from_collections(id(dsk), dsk, dependencies=()) dsk = dsk.cull(set(flatten(keys))) return dsk
def dask_to_tfrecords(df, folder, compression_type="GZIP", compression_level=9): """Store Dask.dataframe to TFRecord files.""" makedirs(folder, exist_ok=True) compression_ext = get_compression_ext(compression_type) filenames = [ get_part_filename(i, compression_ext) for i in range(df.npartitions) ] # Also write a meta data file write_meta(df, folder, compression_type) dsk = {} name = "to-tfrecord-" + tokenize(df, folder) part_tasks = [] kwargs = {} for d, filename in enumerate(filenames): dsk[(name, d)] = (apply, pandas_df_to_tfrecords, [ (df._name, d), os.path.join(folder, filename), compression_type, compression_level ], kwargs) part_tasks.append((name, d)) dsk[name] = (lambda x: None, part_tasks) graph = HighLevelGraph.from_collections(name, dsk, dependencies=[df]) out = Delayed(name, graph) out = out.compute() return out
def overlap_internal(x, axes): """Share boundaries between neighboring blocks Parameters ---------- x: da.Array A dask array axes: dict The size of the shared boundary per axis The axes input informs how many cells to overlap between neighboring blocks {0: 2, 2: 5} means share two cells in 0 axis, 5 cells in 2 axis """ token = tokenize(x, axes) name = "overlap-" + token graph = ArrayOverlapLayer( name=x.name, axes=axes, chunks=x.chunks, numblocks=x.numblocks, token=token, ) graph = HighLevelGraph.from_collections(name, graph, dependencies=[x]) chunks = _overlap_internal_chunks(x.chunks, axes) return Array(graph, name, chunks, meta=x)
def cached_array(array): """ Return a new array that functionally has the same values as array, but flattens the underlying graph and introduces a cache lookup when the individual array chunks are accessed. Useful for caching data that can fit in-memory for the duration of the graph's execution. """ dsk = dict(array.__dask_graph__()) keys = set(flatten(array.__dask_keys__())) # Inline + cull everything except the current array inline_keys = set(dsk.keys() - keys) dsk2 = inline(dsk, inline_keys, inline_constants=True) dsk3, _ = cull(dsk2, keys) # Create a cache used to store array values cache = ArrayCache(uuid.uuid4().hex) for k in keys: dsk3[k] = (cache_entry, cache, Key(k), dsk3.pop(k)) graph = HighLevelGraph.from_collections(array.name, dsk3, []) return da.Array(graph, array.name, array.chunks, array.dtype)
def da_yxbt_sink( bands: Tuple[da.Array, ...], chunks: Tuple[int, ...], name="yxbt" ) -> da.Array: """ each band is in <t,y,x> output is <y,x,b,t> eval(bands) |> transpose(YXBT) |> Store(RAM) |> DaskArray(RAM, chunks) """ tk = tokenize(*bands) b = bands[0] dtype = b.dtype nt, ny, nx = b.shape nb = len(bands) shape = (ny, nx, nb, nt) token = Cache.dask_new(shape, dtype, f"{name}_alloc") sinks = [dask.delayed(_YXBTSink)(token, idx) for idx in range(nb)] fut = da.store(bands, sinks, lock=False, compute=False) sink_name = f"{name}_collect-{tk}" dsk = dict(fut.dask) dsk[sink_name] = (lambda *x: x[0], token.key, *fut.dask[fut.key]) dsk = HighLevelGraph.from_collections(sink_name, dsk, dependencies=sinks) token_done = Delayed(sink_name, dsk) return _da_from_mem(token_done, shape=shape, dtype=dtype, chunks=chunks, name=name)
def _compute_rechunk(x, chunks): """Compute the rechunk of *x* to the given *chunks*.""" if x.size == 0: # Special case for empty array, as the algorithm below does not behave correctly return empty(x.shape, chunks=chunks, dtype=x.dtype) ndim = x.ndim crossed = intersect_chunks(x.chunks, chunks) x2 = dict() intermediates = dict() token = tokenize(x, chunks) merge_name = "rechunk-merge-" + token split_name = "rechunk-split-" + token split_name_suffixes = count() # Pre-allocate old block references, to allow re-use and reduce the # graph's memory footprint a bit. old_blocks = np.empty([len(c) for c in x.chunks], dtype="O") for index in np.ndindex(old_blocks.shape): old_blocks[index] = (x.name, ) + index # Iterate over all new blocks new_index = product(*(range(len(c)) for c in chunks)) for new_idx, cross1 in zip(new_index, crossed): key = (merge_name, ) + new_idx old_block_indices = [[cr[i][0] for cr in cross1] for i in range(ndim)] subdims1 = [len(set(old_block_indices[i])) for i in range(ndim)] rec_cat_arg = np.empty(subdims1, dtype="O") rec_cat_arg_flat = rec_cat_arg.flat # Iterate over the old blocks required to build the new block for rec_cat_index, ind_slices in enumerate(cross1): old_block_index, slices = zip(*ind_slices) name = (split_name, next(split_name_suffixes)) old_index = old_blocks[old_block_index][1:] if all(slc.start == 0 and slc.stop == x.chunks[i][ind] for i, (slc, ind) in enumerate(zip(slices, old_index))): rec_cat_arg_flat[rec_cat_index] = old_blocks[old_block_index] else: intermediates[name] = (getitem, old_blocks[old_block_index], slices) rec_cat_arg_flat[rec_cat_index] = name assert rec_cat_index == rec_cat_arg.size - 1 # New block is formed by concatenation of sliced old blocks if all(d == 1 for d in rec_cat_arg.shape): x2[key] = rec_cat_arg.flat[0] else: x2[key] = (concatenate3, rec_cat_arg.tolist()) del old_blocks, new_index layer = toolz.merge(x2, intermediates) graph = HighLevelGraph.from_collections(merge_name, layer, dependencies=[x]) return Array(graph, merge_name, chunks, meta=x)
def optimize(dsk, keys, **kwargs): if not isinstance(keys, (list, set)): keys = [keys] keys = list(core.flatten(keys)) if not isinstance(dsk, HighLevelGraph): dsk = HighLevelGraph.from_collections(id(dsk), dsk, dependencies=()) else: # Perform Blockwise optimizations for HLG input dsk = optimize_dataframe_getitem(dsk, keys=keys) dsk = optimize_blockwise(dsk, keys=keys) dsk = fuse_roots(dsk, keys=keys) dsk = dsk.cull(set(keys)) # Do not perform low-level fusion unless the user has # specified True explicitly. The configuration will # be None by default. if not config.get("optimization.fuse.active"): return dsk dependencies = dsk.get_all_dependencies() dsk = ensure_dict(dsk) fuse_subgraphs = config.get("optimization.fuse.subgraphs") if fuse_subgraphs is None: fuse_subgraphs = True dsk, _ = fuse( dsk, keys, dependencies=dependencies, fuse_subgraphs=fuse_subgraphs, ) dsk, _ = cull(dsk, keys) return dsk
def to_textfiles_binned(b, path, bin_size=64, nbins=8, compression="infer", encoding=system_encoding, compute=True, storage_options=None, last_endline=False, **kwargs): mode = "wb" if encoding is None else "wt" files = open_files(path, compression=compression, mode=mode, encoding=encoding, name_function=file_namer(bin_size, nbins).name_function, num=b.npartitions * nbins, **(storage_options or {})) name = "to-textfiles-binned-" + uuid.uuid4().hex dsk = {(name, i): (_to_textfiles_chunk_binned, (b.name, i), files[k:k + nbins], last_endline, bin_size) for i, k in enumerate(range(0, len(files), nbins))} graph = HighLevelGraph.from_collections(name, dsk, dependencies=[b]) out = type(b)(graph, name, b.npartitions) if compute: out.compute(**kwargs) return [f.path for f in files] else: return out.to_delayed()
def to_dask_array( agghist: AggHistogram, flow: bool = False, dd: bool = False, ) -> tuple[DaskArray, ...] | tuple[DaskArray, list[DaskArray]]: """Convert `agghist` to a `dask.array` return style. Parameters ---------- agghist : AggHistogram The aggregated histogram collection to convert. flow : bool If ``True``, include under- and over-flow bins dd : bool If True, use ``histogramdd`` style return. See Also -------- dask_histogram.AggHistogram.to_dask_array Returns ------- Union[Tuple[DaskCollection, List[DaskCollection]], Tuple[DaskCollection, ...]] The first return is always the bin counts. If `dd` is ``True`` the second return is a list where each element is an array of bin edges for each axis. If `dd` is ``False``, the bin edge arrays will not be stored in a list (`histogram2d` style return). """ name = f"to-dask-array-{tokenize(agghist)}" zeros = (0, ) * agghist.histref.ndim dsk = { (name, *zeros): (lambda x, f: x.to_numpy(flow=f)[0], agghist.name, flow) } graph = HighLevelGraph.from_collections(name, dsk, dependencies=(agghist, )) shape = agghist.histref.shape if flow: shape = tuple(i + 2 for i in shape) int_storage = agghist.histref._storage_type in ( bh.storage.Int64, bh.storage.AtomicInt64, ) dt = int if int_storage else float c = DaskArray(graph, name=name, shape=shape, chunks=shape, dtype=dt) axes = agghist.histref.axes if flow: edges = [ asarray(np.concatenate([[-np.inf], ax.edges, [np.inf]])) for ax in axes ] else: edges = [asarray(ax.edges) for ax in axes] if dd: return c, edges return (c, *tuple(edges))
def dask_hist2d(x: da.Array, y: da.Array, bins: int, range, density=False): if x.shape != y.shape: raise ValueError( f"Mismatch in argument shaoes: x.shape == {x.shape}; y.shape == {y.shape}" ) token = tokenize(x, y, bins, range, density) name = "histogram2d-sum-" + token x_keys = flatten(x.__dask_keys__()) y_keys = flatten(y.__dask_keys__()) dsk = { (name, i, 0, 0): (_block_fast_hist2d, xi, yi, bins, range) for i, (xi, yi) in enumerate(zip(x_keys, y_keys)) } dtype = np.histogram2d([], [])[0].dtype graph = HighLevelGraph.from_collections(name, dsk, dependencies=(x, y)) # turn graph into a 3D array of shape (nchunks, nbins, nbins) nchunks = len(list(flatten(x.__dask_keys__()))) chunks = ((1,) * nchunks, (bins,), (bins,)) mapped = Array(graph, name, chunks, dtype=dtype) n = mapped.sum(axis=0) return n
def random_frame(self, seed: int, dc: DataContainer, **kwargs) -> dd.Series: """This function - in contrast to others in this module - will only ever be called on data frames""" random_state = np.random.RandomState(seed=seed) # Idea taken from dask.DataFrame.sample: # initialize a random state for each of the partitions # separately and then create a random series # for each partition df = dc.df name = "sample-" + tokenize(df, random_state) state_data = random_state_data(df.npartitions, random_state) dsk = {(name, i): ( self.random_function, (df._name, i), np.random.RandomState(state), kwargs, ) for i, state in enumerate(state_data)} graph = HighLevelGraph.from_collections(name, dsk, dependencies=[df]) random_series = Series(graph, name, ("random", "float64"), df.divisions) # This part seems to be stupid, but helps us do a very simple # task without going into the (private) internals of Dask: # copy all meta information from the original input dataframe # This is important so that the returned series looks # exactly like coming from the input dataframe return_df = df.assign(random=random_series)["random"] return return_df
def wavg_full_f(data, flags, weights, chanav, threshold=0.8): """Perform weighted average of data, flags and weights, over axis -3. This applies flags and uses the specified number of channels. Parameters ---------- data : array of complex flags : array of boolean weights : array of floats chanav : number of channels over which to average, integer Returns ------- av_data : weighted average of data av_flags : weighted average of flags av_weights : weighted average of weights """ # We rechunk (if needed) to get blocks that are multiples of chanav # long, then apply the non-dask version of wavg_full_f per block. # This would be simple with da.core.map_blocks, but it doesn't # support multiple outputs, so we need to do manual construction # of the dask graphs. chunks = _align_chunks(data.chunks, {1: chanav}) out_chunks = list(chunks) # Divide by chanav, rounding up # use axis -3 for freq, to support cases where time axis has been averaged away out_chunks[-3] = tuple((x + chanav - 1) // chanav for x in chunks[-3]) out_chunks = tuple(out_chunks) data = data.rechunk(chunks) flags = flags.rechunk(chunks) weights = weights.rechunk(chunks) token = da.core.tokenize(data, flags, weights, chanav, threshold) base_name = 'wavg_full_f-' + token keys = list(dask.core.flatten(data.__dask_keys__())) base_layer = {(base_name, ) + key[1:]: (calprocs.wavg_full_f, key, (flags.name, ) + key[1:], (weights.name, ) + key[1:], chanav, threshold) for key in keys} base_graph = HighLevelGraph.from_collections(base_name, base_layer, [data, flags, weights]) def sub_array(name, idx, dtype): layer = {(name, ) + key[1:]: (operator.getitem, (base_name, ) + key[1:], idx) for key in keys} layers = dict(base_graph.layers) layers[name] = layer dependencies = dict(base_graph.dependencies) dependencies[name] = {base_name} dsk = HighLevelGraph(layers, dependencies) return da.Array(dsk, name, out_chunks, dtype) av_data = sub_array('wavg_full_f-data-' + token, 0, data.dtype) av_flags = sub_array('wavg_full_f-flags-' + token, 1, flags.dtype) av_weights = sub_array('wavg_full_f-weights-' + token, 2, weights.dtype) return av_data, av_flags, av_weights
def _loc_element(self, iindexer, cindexer): name = "loc-%s" % tokenize(iindexer, self.obj) part = self._get_partitions(iindexer) if iindexer < self.obj.divisions[0] or iindexer > self.obj.divisions[ -1]: raise KeyError("the label [%s] is not in the index" % str(iindexer)) dsk = { (name, 0): ( methods.loc, (self._name, part), slice(iindexer, iindexer), cindexer, ) } meta = self._make_meta(iindexer, cindexer) graph = HighLevelGraph.from_collections(name, dsk, dependencies=[self.obj]) return new_dd_object(graph, name, meta=meta, divisions=[iindexer, iindexer])
def _run_fornav_single(self, data, out_chunks, target_geo_def, fill_value, **kwargs): ll2cr_result = self.cache['ll2cr_result'] ll2cr_blocks = self.cache['ll2cr_blocks'].items() ll2cr_numblocks = ll2cr_result.shape if isinstance( ll2cr_result, np.ndarray) else ll2cr_result.numblocks fornav_task_name = f"fornav-{data.name}-{ll2cr_result.name}" maximum_weight_mode = kwargs.setdefault('maximum_weight_mode', False) weight_sum_min = kwargs.setdefault('weight_sum_min', -1.0) output_stack = self._generate_fornav_dask_tasks( out_chunks, ll2cr_blocks, fornav_task_name, data.name, target_geo_def, fill_value, kwargs) dsk_graph = HighLevelGraph.from_collections( fornav_task_name, output_stack, dependencies=[data, ll2cr_result]) stack_chunks = ( (1, ) * (ll2cr_numblocks[0] * ll2cr_numblocks[1]), ) + out_chunks out_stack = da.Array(dsk_graph, fornav_task_name, stack_chunks, data.dtype) combine_fornav_with_kwargs = partial( _combine_fornav, maximum_weight_mode=maximum_weight_mode) average_fornav_with_kwargs = partial( _average_fornav, maximum_weight_mode=maximum_weight_mode, weight_sum_min=weight_sum_min, dtype=data.dtype, fill_value=fill_value) out = da.reduction(out_stack, _chunk_callable, average_fornav_with_kwargs, combine=combine_fornav_with_kwargs, axis=(0, ), dtype=data.dtype, concatenate=False) return out
def _create_window_dask(name, ntime, nchan, nbl, ncorr, token, dtype, default=0, backend="numpy", path=None): if backend == "zarr-disk" and path is None: path = mkdtemp(prefix='-'.join(('tricolour', name, 'windows', ''))) # Include name and token in new token token = dask.base.tokenize(name, ntime, nchan, nbl, ncorr, token, dtype, default, backend, path) collection_name = '-'.join(("create", name, "windows", token)) layers = { (collection_name, 0): (_create_window, name, ntime, nchan, nbl, ncorr, dtype, default, token, backend, path) } graph = HighLevelGraph.from_collections(collection_name, layers, ()) chunks = ((0, ), ) # One chunk containing single zarr array object return da.Array(graph, collection_name, chunks, dtype=np.object)
def _rechunk_2x2(xx, name="2x2"): """ this is for testing only, ignore it, it's not robust """ assert xx.ndim == 2 name = randomize(name) ny, nx = (len(ch) // 2 for ch in xx.chunks[:2]) dsk = {} chunks = _chunk_getter(xx) for r, c in np.ndindex((ny, nx)): r2 = r * 2 c2 = c * 2 ch_idx = np.s_[r2:r2 + 2, c2:c2 + 2] _xx = chunks(ch_idx) dsk[(name, r, c)] = (_stack_2d_np, (2, 2), *_xx) chy = tuple(xx.chunks[0][i * 2] + xx.chunks[0][i * 2 + 1] for i in range(ny)) chx = tuple(xx.chunks[1][i * 2] + xx.chunks[1][i * 2 + 1] for i in range(nx)) chunks = (chy, chx) dsk = HighLevelGraph.from_collections(name, dsk, dependencies=(xx, )) return da.Array(dsk, name, chunks=chunks, dtype=xx.dtype, shape=xx.shape)
def test_annotations_survive_optimization(): with dask.annotate(foo="bar"): graph = HighLevelGraph.from_collections( "b", { "a": 1, "b": (inc, "a"), "c": (inc, "b") }, [], ) d = Delayed("b", graph) assert type(d.dask) is HighLevelGraph assert len(d.dask.layers) == 1 assert len(d.dask.layers["b"]) == 3 assert d.dask.layers["b"].annotations == {"foo": "bar"} # Ensure optimizing a Delayed object returns a HighLevelGraph # and doesn't loose annotations (d_opt, ) = dask.optimize(d) assert type(d_opt.dask) is HighLevelGraph assert len(d_opt.dask.layers) == 1 assert len(d_opt.dask.layers["b"]) == 2 # c is culled assert d_opt.dask.layers["b"].annotations == {"foo": "bar"}
def clip(gdf, mask, keep_geom_type=False): if isinstance(mask, (GeoDataFrame, GeoSeries)): raise NotImplementedError( "Mask cannot be a Dask GeoDataFrame or GeoSeries.") if gdf.spatial_partitions is None: return gdf.map_partitions( lambda partition: geopandas.clip( gdf=partition, mask=mask, keep_geom_type=keep_geom_type), token="clip", meta=gdf._meta, ) new_spatial_partitions = geopandas.clip( gdf=gdf.spatial_partitions, mask=mask, # keep_geom_type is always false for clipping the spatial partitions # otherwise we'd be falsely creating new partition(s) keep_geom_type=False, ) intersecting_partitions = np.asarray(new_spatial_partitions.index) name = f"clip-{tokenize(gdf, mask, keep_geom_type)}" dsk = {(name, i): (geopandas.clip, (gdf._name, l), mask, keep_geom_type) for i, l in enumerate(intersecting_partitions)} divisions = [None] * (len(dsk) + 1) graph = HighLevelGraph.from_collections(name, dsk, dependencies=[gdf]) if isinstance(gdf, GeoDataFrame): result = GeoDataFrame(graph, name, gdf._meta, tuple(divisions)) elif isinstance(gdf, GeoSeries): result = GeoSeries(graph, name, gdf._meta, tuple(divisions)) result.spatial_partitions = new_spatial_partitions return result
def downscale_dask( array: Any, reduction: Callable[[NDArray[Any], Tuple[int, ...]], NDArray[Any]], scale_factors: Union[int, Sequence[int], Dict[int, int]], **kwargs: Any, ) -> Any: if not np.all((np.array(array.shape) % np.array(scale_factors)) == 0): raise ValueError( f"Coarsening factors {scale_factors} do not align with array shape {array.shape}." ) array = align_chunks(array, scale_factors) name = "downscale-" + tokenize(reduction, array, scale_factors) dsk = { (name,) + key[1:]: (apply, reduction, [key, scale_factors], kwargs) for key in flatten(array.__dask_keys__()) } chunks = tuple( tuple(int(size // scale_factors[axis]) for size in sizes) for axis, sizes in enumerate(array.chunks) ) meta = reduction( np.empty(scale_factors, dtype=array.dtype), scale_factors, **kwargs ) graph = HighLevelGraph.from_collections(name, dsk, dependencies=[array]) return Array(graph, name, chunks, meta=meta)
def read(filename, shape, chunks): from dask.highlevelgraph import HighLevelGraph from dask.array.core import normalize_chunks, Array from itertools import product from ...tunable import delayed from numpy import prod, dtype import xmltodict records = scan_file(filename) records = {r["lime_type"]: r for r in records} data_record = records["ildg-binary-data"] data_offset = data_record["pos"] info = xmltodict.parse(records["ildg-format"]["data"])["ildgFormat"] dtype = dtype("complex%d" % (int(info["precision"]) * 2)) assert data_record["data_length"] == prod(shape) * dtype.itemsize normal_chunks = normalize_chunks(chunks, shape=shape) chunks_id = list(product(*[range(len(bd)) for bd in normal_chunks])) reads = [ delayed(read_chunk)(filename, shape, dtype, data_offset, chunks, chunk_id) for chunk_id in chunks_id ] keys = [(filename, *chunk_id) for chunk_id in chunks_id] vals = [read.key for read in reads] dsk = dict(zip(keys, vals)) graph = HighLevelGraph.from_collections(filename, dsk, dependencies=reads) return Array(graph, filename, normal_chunks, dtype=dtype)
def _checkpoint_one(collection, split_every) -> Delayed: tok = tokenize(collection) name = "checkpoint-" + tok keys_iter = flatten(collection.__dask_keys__()) try: next(keys_iter) next(keys_iter) except StopIteration: # Collection has 0 or 1 keys; no need for a map step layer = {name: (chunks.checkpoint, collection.__dask_keys__())} dsk = HighLevelGraph.from_collections(name, layer, dependencies=(collection, )) return Delayed(name, dsk) # Collection has 2+ keys; apply a two-step map->reduce algorithm so that we # transfer over the network and store in RAM only a handful of None's instead of # the full computed collection's contents dsks = [] map_names = set() map_keys = [] for prev_name in get_collection_names(collection): map_name = "checkpoint_map-" + tokenize(prev_name, tok) map_names.add(map_name) map_layer = _build_map_layer(chunks.checkpoint, prev_name, map_name, collection) map_keys += list(map_layer.get_output_keys()) dsks.append( HighLevelGraph.from_collections(map_name, map_layer, dependencies=(collection, ))) # recursive aggregation reduce_layer: dict = {} while split_every and len(map_keys) > split_every: k = (name, len(reduce_layer)) reduce_layer[k] = (chunks.checkpoint, map_keys[:split_every]) map_keys = map_keys[split_every:] + [k] reduce_layer[name] = (chunks.checkpoint, map_keys) dsks.append( HighLevelGraph({name: reduce_layer}, dependencies={name: map_names})) dsk = HighLevelGraph.merge(*dsks) return Delayed(name, dsk)
def test_clone(layers): dsk1 = {("a", h1): 1, ("a", h2): 2} dsk2 = {"b": (add, ("a", h1), ("a", h2))} dsk3 = {"c": 1, "d": 1} # Multiple names if layers: dsk1 = HighLevelGraph.from_collections("a", dsk1) dsk2 = HighLevelGraph({ "a": dsk1, "b": dsk2 }, dependencies={ "a": set(), "b": {"a"} }) dsk3 = HighLevelGraph.from_collections("c", dsk3) else: dsk2.update(dsk1) t1 = Tuple(dsk1, [("a", h1), ("a", h2)]) t2 = Tuple(dsk2, ["b"]) t3 = Tuple(dsk3, ["c"]) c1 = clone(t2, seed=1, assume_layers=layers) c2 = clone(t2, seed=1, assume_layers=layers) c3 = clone(t2, seed=2, assume_layers=layers) c4 = clone(c1, seed=1, assume_layers=layers) # Clone of a clone has different keys c5 = clone(t2, assume_layers=layers) # Random seed c6 = clone(t2, assume_layers=layers) # Random seed c7 = clone(t2, omit=t1, seed=1, assume_layers=layers) assert c1.__dask_graph__() == c2.__dask_graph__() assert_no_common_keys(c1, t2, layers=layers) assert_no_common_keys(c1, c3, layers=layers) assert_no_common_keys(c1, c4, layers=layers) assert_no_common_keys(c1, c5, layers=layers) assert_no_common_keys(c5, c6, layers=layers) assert_no_common_keys(c7, t2, omit=t1, layers=layers) assert dask.compute(t2, c1, c2, c3, c4, c5, c6, c7) == ((3, ), ) * 8 # Clone nested; some of the collections in omit are unrelated out = clone({"x": [t2]}, omit={"y": [t1, t3]}, assume_layers=layers) assert dask.compute(out) == ({"x": [(3, )]}, ) c8 = out["x"][0] assert_no_common_keys(c8, t2, omit=t1, layers=layers) assert_no_common_keys(c8, t3, layers=layers)
def _simple_shuffle(df, columns, npartitions, ignore_index=True): token = tokenize(df, columns) simple_shuffle_group_token = "simple-shuffle-group-" + token simple_shuffle_split_token = "simple-shuffle-split-" + token simple_shuffle_combine_token = "simple-shuffle-combine-" + token # Pre-Materialize tuples with max number of values # to be iterated upon in this function and # loop using slicing later. iter_tuples = tuple(range(max(df.npartitions, npartitions))) group = {} split = {} combine = {} for i in iter_tuples[:df.npartitions]: # Convert partition into dict of dataframe pieces group[(simple_shuffle_group_token, i)] = ( _shuffle_group, (df._name, i), columns, 0, npartitions, npartitions, ignore_index, npartitions, ) for j in iter_tuples[:npartitions]: _concat_list = [] for i in iter_tuples[:df.npartitions]: # Get out each individual dataframe piece from the dicts split[(simple_shuffle_split_token, i, j)] = ( getitem, (simple_shuffle_group_token, i), j, ) _concat_list.append((simple_shuffle_split_token, i, j)) # concatenate those pieces together, with their friends combine[(simple_shuffle_combine_token, j)] = ( _concat, _concat_list, ignore_index, ) dsk = toolz.merge(group, split, combine) graph = HighLevelGraph.from_collections(simple_shuffle_combine_token, dsk, dependencies=[df]) if df.npartitions == npartitions: divisions = df.divisions else: divisions = (None, ) * (npartitions + 1) return df.__class__(graph, simple_shuffle_combine_token, df, divisions)
def test_ensure_dict(): d = {'x': 1} assert ensure_dict(d) is d hlg = HighLevelGraph.from_collections('x', d) assert type(ensure_dict(hlg)) is dict assert ensure_dict(hlg) == d class mydict(dict): pass md = mydict() md['x'] = 1 assert type(ensure_dict(md)) is dict assert ensure_dict(md) == d