def _materialized_layer_pack( layer: Layer, all_keys, known_key_dependencies, client, client_keys, ): from ..client import Future dsk = dict(layer) # Find aliases not in `client_keys` and substitute all matching keys # with its Future values = { k: v for k, v in dsk.items() if isinstance(v, Future) and k not in client_keys } if values: dsk = subs_multiple(dsk, values) # Unpack remote data and record its dependencies dsk = {k: unpack_remotedata(v, byte_keys=True) for k, v in layer.items()} unpacked_futures = set.union(*[v[1] for v in dsk.values()]) if dsk else set() for future in unpacked_futures: if future.client is not client: raise ValueError( "Inputs contain futures that were created by another client.") if stringify(future.key) not in client.futures: raise CancelledError(stringify(future.key)) unpacked_futures_deps = {} for k, v in dsk.items(): if len(v[1]): unpacked_futures_deps[k] = {f.key for f in v[1]} dsk = {k: v[0] for k, v in dsk.items()} # Calculate dependencies without re-calculating already known dependencies missing_keys = set(dsk.keys()).difference(known_key_dependencies.keys()) dependencies = { k: keys_in_tasks(all_keys, [dsk[k]], as_list=False) for k in missing_keys } for k, v in unpacked_futures_deps.items(): dependencies[k] = set(dependencies.get(k, ())) | v # The scheduler expect all keys to be strings dependencies = { stringify(k): [stringify(dep) for dep in deps] for k, deps in dependencies.items() } all_keys = all_keys.union(dsk) dsk = { stringify(k): stringify(v, exclusive=all_keys) for k, v in dsk.items() } dsk = valmap(dumps_task, dsk) return {"dsk": dsk, "dependencies": dependencies}
def compute_layer_dependencies(layers): """Returns the dependencies between layers""" def _find_layer_containing_key(key): for k, v in layers.items(): if key in v: return k raise RuntimeError(f"{repr(key)} not found") all_keys = {key for layer in layers.values() for key in layer} ret = {k: set() for k in layers} for k, v in layers.items(): for key in keys_in_tasks(all_keys - v.keys(), v.values()): ret[k].add(_find_layer_containing_key(key)) return ret
def get_dependencies(self, key: Hashable, all_hlg_keys: Iterable) -> set: """Get dependencies of `key` in the layer Parameters ---------- key: Hashable The key to find dependencies of all_hlg_keys: Iterable All keys in the high level graph. Returns ------- deps: set A set of dependencies """ return keys_in_tasks(all_hlg_keys, [self[key]])
def __dask_distributed_pack__( self, all_hlg_keys: Iterable[Hashable], known_key_dependencies: Mapping[Hashable, Set], client, client_keys: Iterable[Hashable], ) -> Any: """Pack the layer for scheduler communication in Distributed This method should pack its current state and is called by the Client when communicating with the Scheduler. The Scheduler will then use .__dask_distributed_unpack__(data, ...) to unpack the state, materialize the layer, and merge it into the global task graph. The returned state must be compatible with Distributed's scheduler, which means it must obey the following: - Serializable by msgpack (notice, msgpack converts lists to tuples) - All remote data must be unpacked (see unpack_remotedata()) - All keys must be converted to strings now or when unpacking - All tasks must be serialized (see dumps_task()) The default implementation materialize the layer thus layers such as Blockwise and ShuffleLayer should implement a specialized pack and unpack function in order to avoid materialization. Parameters ---------- all_hlg_keys: Iterable[Hashable] All keys in the high level graph known_key_dependencies: Mapping[Hashable, Set] Already known dependencies client: distributed.Client The client calling this function. client_keys : Iterable[Hashable] List of keys requested by the client. Returns ------- state: Object serializable by msgpack Scheduler compatible state of the layer """ from distributed.client import Future from distributed.utils import CancelledError from distributed.utils_comm import subs_multiple, unpack_remotedata from distributed.worker import dumps_task dsk = dict(self) # Find aliases not in `client_keys` and substitute all matching keys # with its Future future_aliases = { k: v for k, v in dsk.items() if isinstance(v, Future) and k not in client_keys } if future_aliases: dsk = subs_multiple(dsk, future_aliases) # Remove `Future` objects from graph and note any future dependencies dsk2 = {} fut_deps = {} for k, v in dsk.items(): dsk2[k], futs = unpack_remotedata(v, byte_keys=True) if futs: fut_deps[k] = futs dsk = dsk2 # Check that any collected futures are valid unpacked_futures = set.union(*fut_deps.values()) if fut_deps else set() for future in unpacked_futures: if future.client is not client: raise ValueError( "Inputs contain futures that were created by another client." ) if stringify(future.key) not in client.futures: raise CancelledError(stringify(future.key)) # Calculate dependencies without re-calculating already known dependencies # - Start with known dependencies dependencies = ensure_dict(known_key_dependencies, copy=True) # - Remove aliases for any tasks that depend on both an alias and a future. # These can only be found in the known_key_dependencies cache, since # any dependencies computed in this method would have already had the # aliases removed. if future_aliases: alias_keys = set(future_aliases) dependencies = {k: v - alias_keys for k, v in dependencies.items()} # - Add in deps for any missing keys missing_keys = dsk.keys() - dependencies.keys() dependencies.update( (k, keys_in_tasks(all_hlg_keys, [dsk[k]], as_list=False)) for k in missing_keys) # - Add in deps for any tasks that depend on futures for k, futures in fut_deps.items(): if futures: d = ensure_set(dependencies[k], copy=True) d.update(f.key for f in futures) dependencies[k] = d # The scheduler expect all keys to be strings dependencies = { stringify(k): {stringify(dep) for dep in deps} for k, deps in dependencies.items() } merged_hlg_keys = all_hlg_keys | dsk.keys() dsk = { stringify(k): stringify(v, exclusive=merged_hlg_keys) for k, v in dsk.items() } dsk = toolz.valmap(dumps_task, dsk) return {"dsk": dsk, "dependencies": dependencies}