def test_bind(layers): dsk1 = {("a-1", h1): 1, ("a-1", h2): 2} dsk2 = {"b-1": (add, ("a-1", h1), ("a-1", h2))} dsk3 = {"c-1": "b-1"} cnt = NodeCounter() dsk4 = {("d-1", h1): (cnt.f, 1), ("d-1", h2): (cnt.f, 2)} dsk4b = {"e": (cnt.f, 3)} if layers: dsk1 = HighLevelGraph.from_collections("a-1", dsk1) dsk2 = HighLevelGraph({ "a-1": dsk1, "b-1": dsk2 }, dependencies={ "a-1": set(), "b-1": {"a-1"} }) dsk3 = HighLevelGraph( { "a-1": dsk1, "b-1": dsk2, "c-1": dsk3 }, dependencies={ "a-1": set(), "b-1": {"a-1"}, "c-1": {"b-1"} }, ) dsk4 = HighLevelGraph({"d-1": dsk4, "e": dsk4b}, {}) else: dsk2.update(dsk1) dsk3.update(dsk2) dsk4.update(dsk4b) # t1 = Tuple(dsk1, [("a", h1), ("a", h2)]) t2 = Tuple(dsk2, ["b-1"]) t3 = Tuple(dsk3, ["c-1"]) t4 = Tuple(dsk4, [("d-1", h1), ("d-1", h2), "e"]) # Multiple names bound1 = bind(t3, t4, seed=1, assume_layers=layers) cloned_a_name = clone_key("a-1", seed=1) assert bound1.__dask_graph__()[cloned_a_name, h1][0] is chunks.bind assert bound1.__dask_graph__()[cloned_a_name, h2][0] is chunks.bind assert bound1.compute() == (3, ) assert cnt.n == 3 bound2 = bind(t3, t4, omit=t2, seed=1, assume_layers=layers) cloned_c_name = clone_key("c-1", seed=1) assert bound2.__dask_graph__()[cloned_c_name][0] is chunks.bind assert bound2.compute() == (3, ) assert cnt.n == 6
def assert_did_not_materialize(cloned, orig): """Test that all layers of the original collection exist in the cloned collection too and that Blockwise layers have not been materialized """ olayers = orig.__dask_graph__().layers clayers = cloned.__dask_graph__().layers for k, v in olayers.items(): try: cv = clayers[k] except KeyError: cv = clayers[clone_key(k, 0)] if isinstance(v, Blockwise): assert not v.is_materialized() assert not cv.is_materialized()
def test_clone_key(): h = object() # arbitrary hashable assert clone_key("inc-1-2-3", 123) == "inc-4dfeea2f9300e67a75f30bf7d6182ea4" assert clone_key("x", 123) == "x-dc2b8d1c184c72c19faa81c797f8c6b0" assert clone_key("x", 456) == "x-b76f061b547b00d18b9c7a18ccc47e2d" assert clone_key(("x", 1), 456) == ("x-b76f061b547b00d18b9c7a18ccc47e2d", 1) assert clone_key(("sum-1-2-3", h, 1), 123) == ( "sum-1efd41f02035dc802f4ebb9995d07e9d", h, 1, ) with pytest.raises(TypeError): clone_key(1, 2)
def test_clone_key(): h = object() # arbitrary hashable assert clone_key("inc-1-2-3", 123) == "inc-27b6e15b795fcaff169e0e0df14af97a" assert clone_key("x", 123) == "dc2b8d1c184c72c19faa81c797f8c6b0" assert clone_key("x", 456) == "b76f061b547b00d18b9c7a18ccc47e2d" assert clone_key(("sum-1-2-3", h, 1), 123) == ( "sum-27b6e15b795fcaff169e0e0df14af97a", h, 1, ) with pytest.raises(TypeError): clone_key(1, 2)
def clone_value(o): """Variant of distributed.utils_comm.subs_multiple, which allows injecting bind_to """ nonlocal is_leaf typ = type(o) if typ is tuple and o and callable(o[0]): return (o[0], ) + tuple(clone_value(i) for i in o[1:]) elif typ is list: return [clone_value(i) for i in o] elif typ is dict: return {k: clone_value(v) for k, v in o.items()} else: try: if o not in keys: return o except TypeError: return o is_leaf = False return clone_key(o, seed)
def clone( self, keys: set, seed: Hashable, bind_to: Hashable = None, ) -> tuple[Layer, bool]: """Clone selected keys in the layer, as well as references to keys in other layers Parameters ---------- keys Keys to be replaced. This never includes keys not listed by :meth:`get_output_keys`. It must also include any keys that are outside of this layer that may be referenced by it. seed Common hashable used to alter the keys; see :func:`dask.base.clone_key` bind_to Optional key to bind the leaf nodes to. A leaf node here is one that does not reference any replaced keys; in other words it's a node where the replacement graph traversal stops; it may still have dependencies on non-replaced nodes. A bound node will not be computed until after ``bind_to`` has been computed. Returns ------- - New layer - True if the ``bind_to`` key was injected anywhere; False otherwise Notes ----- This method should be overridden by subclasses to avoid materializing the layer. """ from dask.graph_manipulation import chunks is_leaf: bool def clone_value(o): """Variant of distributed.utils_comm.subs_multiple, which allows injecting bind_to """ nonlocal is_leaf typ = type(o) if typ is tuple and o and callable(o[0]): return (o[0], ) + tuple(clone_value(i) for i in o[1:]) elif typ is list: return [clone_value(i) for i in o] elif typ is dict: return {k: clone_value(v) for k, v in o.items()} else: try: if o not in keys: return o except TypeError: return o is_leaf = False return clone_key(o, seed) dsk_new = {} bound = False for key, value in self.items(): if key in keys: key = clone_key(key, seed) is_leaf = True value = clone_value(value) if bind_to is not None and is_leaf: value = (chunks.bind, value, bind_to) bound = True dsk_new[key] = value return MaterializedLayer(dsk_new), bound
def _bind_one( child: T, blocker: Delayed | None, omit_layers: set[str], omit_keys: set[Hashable], seed: Hashable, ) -> T: prev_coll_names = get_collection_names(child) if not prev_coll_names: # Collection with no keys; this is a legitimate use case but, at the moment of # writing, can only happen with third-party collections return child dsk = child.__dask_graph__() # type: ignore new_layers: dict[str, Layer] = {} new_deps: dict[str, Set[Any]] = {} if isinstance(dsk, HighLevelGraph): try: layers_to_clone = set(child.__dask_layers__()) # type: ignore except AttributeError: layers_to_clone = prev_coll_names.copy() else: if len(prev_coll_names) == 1: hlg_name = next(iter(prev_coll_names)) else: hlg_name = tokenize(*prev_coll_names) dsk = HighLevelGraph.from_collections(hlg_name, dsk) layers_to_clone = {hlg_name} clone_keys = dsk.get_all_external_keys() - omit_keys for layer_name in omit_layers: try: layer = dsk.layers[layer_name] except KeyError: continue clone_keys -= layer.get_output_keys() # Note: when assume_layers=True, clone_keys can contain keys of the omit collections # that are not top-level. This is OK, as they will never be encountered inside the # values of their dependent layers. if blocker is not None: blocker_key = blocker.key blocker_dsk = blocker.__dask_graph__() assert isinstance(blocker_dsk, HighLevelGraph) new_layers.update(blocker_dsk.layers) new_deps.update(blocker_dsk.dependencies) else: blocker_key = None layers_to_copy_verbatim = set() while layers_to_clone: prev_layer_name = layers_to_clone.pop() new_layer_name = clone_key(prev_layer_name, seed=seed) if new_layer_name in new_layers: continue layer = dsk.layers[prev_layer_name] layer_deps = dsk.dependencies[prev_layer_name] layer_deps_to_clone = layer_deps - omit_layers layer_deps_to_omit = layer_deps & omit_layers layers_to_clone |= layer_deps_to_clone layers_to_copy_verbatim |= layer_deps_to_omit new_layers[new_layer_name], is_bound = layer.clone(keys=clone_keys, seed=seed, bind_to=blocker_key) new_dep = {clone_key(dep, seed=seed) for dep in layer_deps_to_clone} | layer_deps_to_omit if is_bound: new_dep.add(blocker_key) new_deps[new_layer_name] = new_dep # Add the layers of the collections from omit from child.dsk. Note that, when # assume_layers=False, it would be unsafe to simply do HighLevelGraph.merge(dsk, # omit[i].dsk). Also, collections in omit may or may not be parents of this specific # child, or of any children at all. while layers_to_copy_verbatim: layer_name = layers_to_copy_verbatim.pop() if layer_name in new_layers: continue layer_deps = dsk.dependencies[layer_name] layers_to_copy_verbatim |= layer_deps new_deps[layer_name] = layer_deps new_layers[layer_name] = dsk.layers[layer_name] rebuild, args = child.__dask_postpersist__() # type: ignore return rebuild( HighLevelGraph(new_layers, new_deps), *args, rename={ prev_name: clone_key(prev_name, seed) for prev_name in prev_coll_names }, )