def test_annotation_pack_unpack(): layer = MaterializedLayer({"n": 42}, annotations={"workers": ("alice", )}) packed_anno = layer.__dask_distributed_annotations_pack__() annotations = {} Layer.__dask_distributed_annotations_unpack__(annotations, packed_anno, layer.keys()) assert annotations == {"workers": {"n": ("alice", )}}
def test_materializedlayer_cull_preserves_annotations(): layer = MaterializedLayer( { "a": 42, "b": 3.14 }, annotations={"foo": "bar"}, ) culled_layer, _ = layer.cull({"a"}, []) assert len(culled_layer) == 1 assert culled_layer.annotations == {"foo": "bar"}
def _rename_layer(layer, keymap, salt): """Rename a single layer in a :class:`dask.highlevelgraph.HighLevelGraph`.""" if type(layer) is Blockwise: new_indices = tuple( (_rename_key(name, salt) if ind is not None else name, ind) for name, ind in layer.indices) sub_keymap = {key: _rename_key(key, salt) for key in layer.dsk} kwargs = {} # The available arguments depend on the Dask version. sig = inspect.signature(Blockwise) for arg_name in ['output_blocks', 'annotations']: if arg_name in sig.parameters: kwargs[arg_name] = getattr(layer, arg_name) if 'io_deps' in sig.parameters: kwargs['io_deps'] = { _rename_key(key, salt): value for key, value in layer.io_deps.items() } return Blockwise( _rename_key(layer.output, salt), layer.output_indices, _rename_layer(layer.dsk, sub_keymap, salt), new_indices, { _rename_key(name, salt): value for name, value in layer.numblocks.items() }, layer.concatenate, layer.new_axes, **kwargs) elif type(layer) is MaterializedLayer: mapping = { keymap[key]: _rename(value, keymap) for (key, value) in layer.mapping.items() } return MaterializedLayer(mapping, layer.annotations) else: return { keymap[key]: _rename(value, keymap) for (key, value) in layer.items() }
async def test_annotation_pack_unpack(c, s, a, b): hlg = HighLevelGraph({"l1": MaterializedLayer({"n": 42})}, {"l1": set()}) packed_hlg = hlg.__dask_distributed_pack__(c, ["n"]) annotations = {"workers": ("alice",)} unpacked_hlg = HighLevelGraph.__dask_distributed_unpack__(packed_hlg, annotations) annotations = unpacked_hlg["annotations"] assert annotations == {"workers": {"n": ("alice",)}}
def _build_map_layer( func: Callable, prev_name: str, new_name: str, collection, dependencies: tuple[Delayed, ...] = (), ) -> Layer: """Apply func to all keys of collection. Create a Blockwise layer whenever possible; fall back to MaterializedLayer otherwise. Parameters ---------- func Callable to be invoked on the graph node prev_name : str name of the layer to map from; in case of dask base collections, this is the collection name. Note how third-party collections, e.g. xarray.Dataset, can have multiple names. new_name : str name of the layer to map to collection Arbitrary dask collection dependencies Zero or more Delayed objects, which will be passed as arbitrary variadic args to func after the collection's chunk """ if _can_apply_blockwise(collection): # Use a Blockwise layer try: numblocks = collection.numblocks except AttributeError: numblocks = (collection.npartitions, ) indices = tuple(i for i, _ in enumerate(numblocks)) kwargs = { "_deps": [d.key for d in dependencies] } if dependencies else {} return blockwise( func, new_name, indices, prev_name, indices, numblocks={prev_name: numblocks}, dependencies=dependencies, **kwargs, ) else: # Delayed, bag.Item, dataframe.core.Scalar, or third-party collection; # fall back to MaterializedLayer dep_keys = tuple(d.key for d in dependencies) return MaterializedLayer({ replace_name_in_key(k, {prev_name: new_name}): (func, k) + dep_keys for k in flatten(collection.__dask_keys__()) if get_name_from_key(k) == prev_name })
async def test_pack_MaterializedLayer_handles_futures_in_graph_properly( c, s, a, b): fut = c.submit(inc, 1) hlg = HighLevelGraph( { "l1": MaterializedLayer({ "x": fut, "y": (inc, "x"), "z": (inc, "y") }) }, {"l1": set()}, ) # fill hlg.key_dependencies cache. This excludes known futures, so only # includes a subset of all dependencies. Previously if the cache was present # the future dependencies would be missing when packed. hlg.get_all_dependencies() packed = hlg.__dask_distributed_pack__(c, ["z"], {}) unpacked = HighLevelGraph.__dask_distributed_unpack__(packed) assert unpacked["deps"] == {"x": {fut.key}, "y": {fut.key}, "z": {"y"}}
def test_highlevelgraph_dicts_deprecation(): with pytest.warns(FutureWarning): layers = {"a": MaterializedLayer({"x": 1, "y": (inc, "x")})} hg = HighLevelGraph(layers, {"a": set()}) assert hg.dicts == layers