import abc
import collections.abc
import contextlib
import copy
import html
from typing import (
    AbstractSet,
    Any,
    Dict,
    Hashable,
    Iterable,
    Mapping,
    MutableMapping,
    Optional,
    Tuple,
)

import tlz as toolz

from . import config
from .base import clone_key, flatten, is_dask_collection
from .core import keys_in_tasks, reverse_dict
from .utils import _deprecated, ensure_dict, key_split, stringify
from .utils_test import add, inc  # noqa: F401


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 = set(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


class Layer(collections.abc.Mapping):
    """High level graph layer

    This abstract class establish a protocol for high level graph layers.

    The main motivation of a layer is to represent a collection of tasks
    symbolically in order to speedup a series of operations significantly.
    Ideally, a layer should stay in this symbolic state until execution
    but in practice some operations will force the layer to generate all
    its internal tasks. We say that the layer has been materialized.

    Most of the default implementations in this class will materialize the
    layer. It is up to derived classes to implement non-materializing
    implementations.
    """

    annotations: Optional[Mapping[str, Any]]
    collection_annotations: Optional[Mapping[str, Any]]

    def __init__(
        self,
        annotations: Mapping[str, Any] = None,
        collection_annotations: Mapping[str, Any] = None,
    ):
        """Initialize Layer object.

        Parameters
        ----------
        annotations : Mapping[str, Any], optional
            By default, None.
            Annotations are metadata or soft constraints associated with tasks
            that dask schedulers may choose to respect:
            They signal intent without enforcing hard constraints.
            As such, they are primarily designed for use with the distributed
            scheduler. See the dask.annotate function for more information.
        collection_annotations : Mapping[str, Any], optional. By default, None.
            Experimental, intended to assist with visualizing the performance
            characteristics of Dask computations.
            These annotations are *not* passed to the distributed scheduler.
        """
        self.annotations = annotations or copy.copy(config.get("annotations", None))
        self.collection_annotations = collection_annotations or copy.copy(
            config.get("collection_annotations", None)
        )

    @abc.abstractmethod
    def is_materialized(self) -> bool:
        """Return whether the layer is materialized or not"""
        return True

    @abc.abstractmethod
    def get_output_keys(self) -> AbstractSet:
        """Return a set of all output keys

        Output keys are all keys in the layer that might be referenced by
        other layers.

        Classes overriding this implementation should not cause the layer
        to be materialized.

        Returns
        -------
        keys: AbstractSet
            All output keys
        """
        return self.keys()  # this implementation will materialize the graph

    def cull(
        self, keys: set, all_hlg_keys: Iterable
    ) -> Tuple["Layer", Mapping[Hashable, set]]:
        """Remove unnecessary tasks from the layer

        In other words, return a new Layer with only the tasks required to
        calculate `keys` and a map of external key dependencies.

        Examples
        --------
        >>> d = MaterializedLayer({'x': 1, 'y': (inc, 'x'), 'out': (add, 'x', 10)})
        >>> _, deps = d.cull({'out'}, d.keys())
        >>> deps
        {'out': {'x'}, 'x': set()}

        Returns
        -------
        layer: Layer
            Culled layer
        deps: Map
            Map of external key dependencies
        """

        if len(keys) == len(self):
            # Nothing to cull if preserving all existing keys
            return (
                self,
                {k: self.get_dependencies(k, all_hlg_keys) for k in self.keys()},
            )

        ret_deps = {}
        seen = set()
        out = {}
        work = keys.copy()
        while work:
            k = work.pop()
            out[k] = self[k]
            ret_deps[k] = self.get_dependencies(k, all_hlg_keys)
            for d in ret_deps[k]:
                if d not in seen:
                    if d in self:
                        seen.add(d)
                        work.add(d)

        return MaterializedLayer(out), ret_deps

    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_annotations_pack__(
        self, annotations: Mapping[str, Any] = None
    ) -> Optional[Mapping[str, Any]]:
        """Packs Layer annotations for transmission to scheduler

        Callables annotations are fully expanded over Layer keys, while
        other values are simply transmitted as is

        Parameters
        ----------
        annotations : Mapping[str, Any], optional
            A top-level annotations.

        Returns
        -------
        packed_annotations : dict
            Packed annotations.
        """
        annotations = toolz.merge(self.annotations or {}, annotations or {})
        packed = {}
        for a, v in annotations.items():
            if callable(v):
                packed[a] = {stringify(k): v(k) for k in self}
                packed[a]["__expanded_annotations__"] = True
            else:
                packed[a] = v
        return packed

    @staticmethod
    def __dask_distributed_annotations_unpack__(
        annotations: MutableMapping[str, Any],
        new_annotations: Optional[Mapping[str, Any]],
        keys: Iterable[Hashable],
    ) -> None:
        """
        Unpack a set of layer annotations across a set of keys, then merge those
        expanded annotations for the layer into an existing annotations mapping.

        This is not a simple shallow merge because some annotations like retries,
        priority, workers, etc need to be able to retain keys from different layers.

        Parameters
        ----------
        annotations: MutableMapping[str, Any], input/output
            Already unpacked annotations, which are to be updated with the new
            unpacked annotations
        new_annotations: Mapping[str, Any], optional
            New annotations to be unpacked into `annotations`
        keys: Iterable
            All keys in the layer.
        """
        if new_annotations is None:
            return

        expanded = {}
        keys_stringified = False

        # Expand the new annotations across the keyset
        for a, v in new_annotations.items():
            if type(v) is dict and "__expanded_annotations__" in v:
                # Maybe do a destructive update for efficiency?
                v = v.copy()
                del v["__expanded_annotations__"]
                expanded[a] = v
            else:
                if not keys_stringified:
                    keys = [stringify(k) for k in keys]
                    keys_stringified = True

                expanded[a] = dict.fromkeys(keys, v)

        # Merge the expanded annotations with the existing annotations mapping
        for k, v in expanded.items():
            v.update(annotations.get(k, {}))
        annotations.update(expanded)

    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 .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 __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
        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 dsk.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 = dsk.keys() - known_key_dependencies.keys()
        dependencies = {
            k: keys_in_tasks(all_hlg_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
        dependencies.update(known_key_dependencies)

        # 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}

    @classmethod
    def __dask_distributed_unpack__(
        cls,
        state: Any,
        dsk: Mapping[str, Any],
        dependencies: Mapping[str, set],
    ) -> Dict:
        """Unpack the state of a layer previously packed by __dask_distributed_pack__()

        This method is called by the scheduler in Distributed in order to unpack
        the state of a layer and merge it into its global task graph. The method
        can use `dsk` and `dependencies`, which are the already materialized
        state of the preceding layers in the high level graph. The layers of the
        high level graph are unpacked in topological order.

        See Layer.__dask_distributed_pack__() for packing detail.

        Parameters
        ----------
        state: Any
            The state returned by Layer.__dask_distributed_pack__()
        dsk: Mapping, read-only
            The materialized low level graph of the already unpacked layers
        dependencies: Mapping, read-only
            The dependencies of each key in `dsk`

        Returns
        -------
        unpacked-layer: dict
            layer_dsk: Mapping[str, Any]
                Materialized (stringified) graph of the layer
            layer_deps: Mapping[str, set]
                Dependencies of each key in `layer_dsk`
        """
        return {"dsk": state["dsk"], "deps": state["dependencies"]}

    def __reduce__(self):
        """Default serialization implementation, which materializes the Layer"""
        return (MaterializedLayer, (dict(self),))

    def __copy__(self):
        """Default shallow copy implementation"""
        obj = type(self).__new__(self.__class__)
        obj.__dict__.update(self.__dict__)
        return obj

    def _repr_html_(self, layer_index="", highlevelgraph_key=""):
        unmaterialized_layer_icon = """
            <svg width="24" height="24" viewBox="0 0 32 32" fill="none"
                xmlns="http://www.w3.org/2000/svg" style="position: absolute;">
                <circle cx="16" cy="16" r="14"
                    style="stroke: var(--jp-ui-font-color2, #1D1D1D); fill: var(--jp-layout-color1, #F2F2F2);"
                    stroke-width="2" />
            </svg>
        """
        materialized_layer_icon = """
            <svg width="24" height="24" viewBox="0 0 32 32" fill="none"
                xmlns="http://www.w3.org/2000/svg" style="position: absolute;">
                <circle cx="16" cy="16" r="14" fill="#8F8F8F"
                    style="stroke: var(--jp-ui-font-color2, #1D1D1D);" stroke-width="2"/>
            </svg>
        """
        if self.is_materialized():
            layer_icon = materialized_layer_icon
        else:
            layer_icon = unmaterialized_layer_icon

        if highlevelgraph_key != "":
            shortname = key_split(highlevelgraph_key)
        elif hasattr(self, "name"):
            shortname = key_split(self.name)
        else:
            shortname = self.__class__.__name__

        svg_repr = ""
        if (
            self.collection_annotations
            and self.collection_annotations.get("type") == "dask.array.core.Array"
        ):
            chunks = self.collection_annotations.get("chunks")
            if chunks:
                from .array.svg import svg

                svg_repr = "<br />" + svg(chunks)

        info = self.layer_info_dict()
        layer_info_table = html_from_dict(info)
        html = f"""
            <div style="">
                {layer_icon}
                <details style="margin-left: 32px;">
                    <summary style="margin-bottom: 10px; margin-top: 10px;">
                        <h4 style="display: inline;">Layer{layer_index}: {shortname}</h4>
                    </summary>
                    <p style="color: var(--jp-ui-font-color2, #5D5851); margin: -0.25em 0px 0px 0px;">
                        {highlevelgraph_key}
                    </p>
                    {svg_repr}
                    {layer_info_table}
                </details>
            </div>
            """
        return html

    def layer_info_dict(self):
        info = {
            "layer_type": type(self).__name__,
            "is_materialized": self.is_materialized(),
        }
        if self.annotations is not None:
            for key, val in self.annotations.items():
                info[key] = html.escape(str(val))
        if self.collection_annotations is not None:
            for key, val in self.collection_annotations.items():
                # Hide verbose chunk details from the HTML table
                if key != "chunks":
                    info[key] = html.escape(str(val))
        return info


class MaterializedLayer(Layer):
    """Fully materialized layer of `Layer`

    Parameters
    ----------
    mapping: Mapping
        The mapping between keys and tasks, typically a dask graph.
    """

    def __init__(self, mapping: Mapping, annotations=None):
        super().__init__(annotations=annotations)
        self.mapping = mapping

    def __contains__(self, k):
        return k in self.mapping

    def __getitem__(self, k):
        return self.mapping[k]

    def __iter__(self):
        return iter(self.mapping)

    def __len__(self):
        return len(self.mapping)

    def is_materialized(self):
        return True

    def get_output_keys(self):
        return self.keys()


class HighLevelGraph(Mapping):
    """Task graph composed of layers of dependent subgraphs

    This object encodes a Dask task graph that is composed of layers of
    dependent subgraphs, such as commonly occurs when building task graphs
    using high level collections like Dask array, bag, or dataframe.

    Typically each high level array, bag, or dataframe operation takes the task
    graphs of the input collections, merges them, and then adds one or more new
    layers of tasks for the new operation.  These layers typically have at
    least as many tasks as there are partitions or chunks in the collection.
    The HighLevelGraph object stores the subgraphs for each operation
    separately in sub-graphs, and also stores the dependency structure between
    them.

    Parameters
    ----------
    layers : Mapping[str, Mapping]
        The subgraph layers, keyed by a unique name
    dependencies : Mapping[str, set[str]]
        The set of layers on which each layer depends
    key_dependencies : Mapping[Hashable, set], optional
        Mapping (some) keys in the high level graph to their dependencies. If
        a key is missing, its dependencies will be calculated on-the-fly.

    Examples
    --------
    Here is an idealized example that shows the internal state of a
    HighLevelGraph

    >>> import dask.dataframe as dd

    >>> df = dd.read_csv('myfile.*.csv')  # doctest: +SKIP
    >>> df = df + 100  # doctest: +SKIP
    >>> df = df[df.name == 'Alice']  # doctest: +SKIP

    >>> graph = df.__dask_graph__()  # doctest: +SKIP
    >>> graph.layers  # doctest: +SKIP
    {
     'read-csv': {('read-csv', 0): (pandas.read_csv, 'myfile.0.csv'),
                  ('read-csv', 1): (pandas.read_csv, 'myfile.1.csv'),
                  ('read-csv', 2): (pandas.read_csv, 'myfile.2.csv'),
                  ('read-csv', 3): (pandas.read_csv, 'myfile.3.csv')},
     'add': {('add', 0): (operator.add, ('read-csv', 0), 100),
             ('add', 1): (operator.add, ('read-csv', 1), 100),
             ('add', 2): (operator.add, ('read-csv', 2), 100),
             ('add', 3): (operator.add, ('read-csv', 3), 100)}
     'filter': {('filter', 0): (lambda part: part[part.name == 'Alice'], ('add', 0)),
                ('filter', 1): (lambda part: part[part.name == 'Alice'], ('add', 1)),
                ('filter', 2): (lambda part: part[part.name == 'Alice'], ('add', 2)),
                ('filter', 3): (lambda part: part[part.name == 'Alice'], ('add', 3))}
    }

    >>> graph.dependencies  # doctest: +SKIP
    {
     'read-csv': set(),
     'add': {'read-csv'},
     'filter': {'add'}
    }

    See Also
    --------
    HighLevelGraph.from_collections :
        typically used by developers to make new HighLevelGraphs
    """

    layers: Mapping[str, Layer]
    dependencies: Mapping[str, AbstractSet]
    key_dependencies: Dict[Hashable, AbstractSet]
    _to_dict: dict
    _all_external_keys: set

    def __init__(
        self,
        layers: Mapping[str, Mapping],
        dependencies: Mapping[str, AbstractSet],
        key_dependencies: Optional[Dict[Hashable, AbstractSet]] = None,
    ):
        self.dependencies = dependencies
        self.key_dependencies = key_dependencies or {}
        # Makes sure that all layers are `Layer`
        self.layers = {
            k: v if isinstance(v, Layer) else MaterializedLayer(v)
            for k, v in layers.items()
        }

    @classmethod
    def _from_collection(cls, name, layer, collection):
        """`from_collections` optimized for a single collection"""
        if is_dask_collection(collection):
            graph = collection.__dask_graph__()
            if isinstance(graph, HighLevelGraph):
                layers = ensure_dict(graph.layers, copy=True)
                layers.update({name: layer})
                deps = ensure_dict(graph.dependencies, copy=True)
                with contextlib.suppress(AttributeError):
                    deps.update({name: set(collection.__dask_layers__())})
            else:
                key = _get_some_layer_name(collection)
                layers = {name: layer, key: graph}
                deps = {name: {key}, key: set()}
        else:
            raise TypeError(type(collection))

        return cls(layers, deps)

    @classmethod
    def from_collections(cls, name, layer, dependencies=()):
        """Construct a HighLevelGraph from a new layer and a set of collections

        This constructs a HighLevelGraph in the common case where we have a single
        new layer and a set of old collections on which we want to depend.

        This pulls out the ``__dask_layers__()`` method of the collections if
        they exist, and adds them to the dependencies for this new layer.  It
        also merges all of the layers from all of the dependent collections
        together into the new layers for this graph.

        Parameters
        ----------
        name : str
            The name of the new layer
        layer : Mapping
            The graph layer itself
        dependencies : List of Dask collections
            A list of other dask collections (like arrays or dataframes) that
            have graphs themselves

        Examples
        --------

        In typical usage we make a new task layer, and then pass that layer
        along with all dependent collections to this method.

        >>> def add(self, other):
        ...     name = 'add-' + tokenize(self, other)
        ...     layer = {(name, i): (add, input_key, other)
        ...              for i, input_key in enumerate(self.__dask_keys__())}
        ...     graph = HighLevelGraph.from_collections(name, layer, dependencies=[self])
        ...     return new_collection(name, graph)
        """
        if len(dependencies) == 1:
            return cls._from_collection(name, layer, dependencies[0])
        layers = {name: layer}
        deps = {name: set()}
        for collection in toolz.unique(dependencies, key=id):
            if is_dask_collection(collection):
                graph = collection.__dask_graph__()
                if isinstance(graph, HighLevelGraph):
                    layers.update(graph.layers)
                    deps.update(graph.dependencies)
                    with contextlib.suppress(AttributeError):
                        deps[name] |= set(collection.__dask_layers__())
                else:
                    key = _get_some_layer_name(collection)
                    layers[key] = graph
                    deps[name].add(key)
                    deps[key] = set()
            else:
                raise TypeError(type(collection))

        return cls(layers, deps)

    def __getitem__(self, key):
        # Attempt O(1) direct access first, under the assumption that layer names match
        # either the keys (Scalar, Item, Delayed) or the first element of the key tuples
        # (Array, Bag, DataFrame, Series). This assumption is not always true.
        try:
            return self.layers[key][key]
        except KeyError:
            pass
        try:
            return self.layers[key[0]][key]
        except (KeyError, IndexError, TypeError):
            pass

        # Fall back to O(n) access
        for d in self.layers.values():
            try:
                return d[key]
            except KeyError:
                pass

        raise KeyError(key)

    def __len__(self) -> int:
        # NOTE: this will double-count keys that are duplicated between layers, so it's
        # possible that `len(hlg) > len(hlg.to_dict())`. However, duplicate keys should
        # not occur through normal use, and their existence would usually be a bug.
        # So we ignore this case in favor of better performance.
        # https://github.com/dask/dask/issues/7271
        return sum(len(layer) for layer in self.layers.values())

    def __iter__(self):
        return iter(self.to_dict())

    def to_dict(self) -> dict:
        """Efficiently convert to plain dict. This method is faster than dict(self)."""
        try:
            return self._to_dict
        except AttributeError:
            out = self._to_dict = ensure_dict(self)
            return out

    def keys(self) -> AbstractSet:
        """Get all keys of all the layers.

        This will in many cases materialize layers, which makes it a relatively
        expensive operation. See :meth:`get_all_external_keys` for a faster alternative.
        """
        return self.to_dict().keys()

    @_deprecated(use_instead="HighLevelGraph.keys")
    def keyset(self) -> AbstractSet:
        # Backwards compatibility for now
        return self.keys()

    def get_all_external_keys(self) -> set:
        """Get all output keys of all layers

        This will in most cases _not_ materialize any layers, which makes
        it a relative cheap operation.

        Returns
        -------
        keys: set
            A set of all external keys
        """
        try:
            return self._all_external_keys
        except AttributeError:
            keys: set = set()
            for layer in self.layers.values():
                # Note: don't use `keys |= ...`, because the RHS is a
                # collections.abc.Set rather than a real set, and this will
                # cause a whole new set to be constructed.
                keys.update(layer.get_output_keys())
            self._all_external_keys = keys
            return keys

    def items(self):
        return self.to_dict().items()

    def values(self):
        return self.to_dict().values()

    def get_all_dependencies(self) -> Dict[Hashable, AbstractSet]:
        """Get dependencies of all keys

        This will in most cases materialize all layers, which makes
        it an expensive operation.

        Returns
        -------
        map: Mapping
            A map that maps each key to its dependencies
        """
        all_keys = self.keys()
        missing_keys = all_keys - self.key_dependencies.keys()
        if missing_keys:
            for layer in self.layers.values():
                for k in missing_keys & layer.keys():
                    self.key_dependencies[k] = layer.get_dependencies(k, all_keys)
        return self.key_dependencies

    @property
    def dependents(self):
        return reverse_dict(self.dependencies)

    @property
    @_deprecated(use_instead="HighLevelGraph.layers")
    def dicts(self):
        # Backwards compatibility for now
        return self.layers

    def copy(self):
        return HighLevelGraph(
            ensure_dict(self.layers, copy=True),
            ensure_dict(self.dependencies, copy=True),
            self.key_dependencies.copy(),
        )

    @classmethod
    def merge(cls, *graphs):
        layers = {}
        dependencies = {}
        for g in graphs:
            if isinstance(g, HighLevelGraph):
                layers.update(g.layers)
                dependencies.update(g.dependencies)
            elif isinstance(g, Mapping):
                layers[id(g)] = g
                dependencies[id(g)] = set()
            else:
                raise TypeError(g)
        return cls(layers, dependencies)

    def visualize(self, filename="dask.pdf", format=None, **kwargs):
        from .dot import graphviz_to_file

        g = to_graphviz(self, **kwargs)
        graphviz_to_file(g, filename, format)
        return g

    def _toposort_layers(self):
        """Sort the layers in a high level graph topologically

        Parameters
        ----------
        hlg : HighLevelGraph
            The high level graph's layers to sort

        Returns
        -------
        sorted: list
            List of layer names sorted topologically
        """
        degree = {k: len(v) for k, v in self.dependencies.items()}
        reverse_deps = {k: [] for k in self.dependencies}
        ready = []
        for k, v in self.dependencies.items():
            for dep in v:
                reverse_deps[dep].append(k)
            if not v:
                ready.append(k)
        ret = []
        while len(ready) > 0:
            layer = ready.pop()
            ret.append(layer)
            for rdep in reverse_deps[layer]:
                degree[rdep] -= 1
                if degree[rdep] == 0:
                    ready.append(rdep)
        return ret

    def cull(self, keys: Iterable) -> "HighLevelGraph":
        """Return new HighLevelGraph with only the tasks required to calculate keys.

        In other words, remove unnecessary tasks from dask.

        Parameters
        ----------
        keys
            iterable of keys or nested list of keys such as the output of
            ``__dask_keys__()``

        Returns
        -------
        hlg: HighLevelGraph
            Culled high level graph
        """
        keys_set = set(flatten(keys))

        all_ext_keys = self.get_all_external_keys()
        ret_layers = {}
        ret_key_deps = {}
        for layer_name in reversed(self._toposort_layers()):
            layer = self.layers[layer_name]
            # Let's cull the layer to produce its part of `keys`.
            # Note: use .intersection rather than & because the RHS is
            # a collections.abc.Set rather than a real set, and using &
            # would take time proportional to the size of the LHS, which
            # if there is no culling can be much bigger than the RHS.
            output_keys = keys_set.intersection(layer.get_output_keys())
            if output_keys:
                culled_layer, culled_deps = layer.cull(output_keys, all_ext_keys)
                # Update `keys` with all layer's external key dependencies, which
                # are all the layer's dependencies (`culled_deps`) excluding
                # the layer's output keys.
                external_deps = set()
                for d in culled_deps.values():
                    external_deps |= d
                external_deps -= culled_layer.get_output_keys()
                keys_set |= external_deps

                # Save the culled layer and its key dependencies
                ret_layers[layer_name] = culled_layer
                ret_key_deps.update(culled_deps)

        # Converting dict_keys to a real set lets Python optimise the set
        # intersection to iterate over the smaller of the two sets.
        ret_layers_keys = set(ret_layers.keys())
        ret_dependencies = {
            layer_name: self.dependencies[layer_name] & ret_layers_keys
            for layer_name in ret_layers
        }

        return HighLevelGraph(ret_layers, ret_dependencies, ret_key_deps)

    def cull_layers(self, layers: Iterable[str]) -> "HighLevelGraph":
        """Return a new HighLevelGraph with only the given layers and their
        dependencies. Internally, layers are not modified.

        This is a variant of :meth:`HighLevelGraph.cull` which is much faster and does
        not risk creating a collision between two layers with the same name and
        different content when two culled graphs are merged later on.

        Returns
        -------
        hlg: HighLevelGraph
            Culled high level graph
        """
        to_visit = set(layers)
        ret_layers = {}
        ret_dependencies = {}
        while to_visit:
            k = to_visit.pop()
            ret_layers[k] = self.layers[k]
            ret_dependencies[k] = self.dependencies[k]
            to_visit |= ret_dependencies[k] - ret_dependencies.keys()

        return HighLevelGraph(ret_layers, ret_dependencies)

    def validate(self):
        # Check dependencies
        for layer_name, deps in self.dependencies.items():
            if layer_name not in self.layers:
                raise ValueError(
                    f"dependencies[{repr(layer_name)}] not found in layers"
                )
            for dep in deps:
                if dep not in self.dependencies:
                    raise ValueError(f"{repr(dep)} not found in dependencies")

        for layer in self.layers.values():
            assert hasattr(layer, "annotations")

        # Re-calculate all layer dependencies
        dependencies = compute_layer_dependencies(self.layers)

        # Check keys
        dep_key1 = self.dependencies.keys()
        dep_key2 = dependencies.keys()
        if dep_key1 != dep_key2:
            raise ValueError(
                f"incorrect dependencies keys {set(dep_key1)!r} "
                f"expected {set(dep_key2)!r}"
            )

        # Check values
        for k in dep_key1:
            if self.dependencies[k] != dependencies[k]:
                raise ValueError(
                    f"incorrect dependencies[{repr(k)}]: {repr(self.dependencies[k])} "
                    f"expected {repr(dependencies[k])}"
                )

    def __dask_distributed_pack__(
        self,
        client,
        client_keys: Iterable[Hashable],
        annotations: Mapping[str, Any] = None,
    ) -> dict:
        """Pack the high level graph for Scheduler -> Worker communication

        The approach is to delegate the packaging to each layer in the high level graph
        by calling .__dask_distributed_pack__() and .__dask_distributed_annotations_pack__()
        on each layer.

        Parameters
        ----------
        client : distributed.Client
            The client calling this function.
        client_keys : Iterable[Hashable]
            List of keys requested by the client.
        annotations : Mapping[str, Any], optional
            A top-level annotations.

        Returns
        -------
        data: dict
            Packed high level graph layers
        """
        # Dump each layer (in topological order)
        layers = []
        for layer in (self.layers[name] for name in self._toposort_layers()):
            layers.append(
                {
                    "__module__": layer.__module__,
                    "__name__": type(layer).__name__,
                    "state": layer.__dask_distributed_pack__(
                        self.get_all_external_keys(),
                        self.key_dependencies,
                        client,
                        client_keys,
                    ),
                    "annotations": layer.__dask_distributed_annotations_pack__(
                        annotations
                    ),
                }
            )
        return {"layers": layers}

    @staticmethod
    def __dask_distributed_unpack__(hlg: dict) -> dict:
        """Unpack the high level graph for Scheduler -> Worker communication

        The approach is to delegate the unpackaging to each layer in the high level graph
        by calling ..._unpack__() and ..._annotations_unpack__()
        on each layer.

        Parameters
        ----------
        hlg: dict
            Packed high level graph layers

        Returns
        -------
        unpacked-graph: dict
            dsk: Dict[str, Any]
                Materialized (stringified) graph of all nodes in the high level graph
            deps: Dict[str, set]
                Dependencies of each key in `dsk`
            annotations: Dict[str, Any]
                Annotations for `dsk`
        """
        from distributed.protocol.serialize import import_allowed_module

        dsk = {}
        deps = {}
        anno = {}

        # Unpack each layer (in topological order)
        for layer in hlg["layers"]:
            # Find the unpack functions
            if layer["__module__"] is None:  # Default implementation
                unpack_state = Layer.__dask_distributed_unpack__
                unpack_anno = Layer.__dask_distributed_annotations_unpack__
            else:
                mod = import_allowed_module(layer["__module__"])
                cls = getattr(mod, layer["__name__"])
                unpack_state = cls.__dask_distributed_unpack__
                unpack_anno = cls.__dask_distributed_annotations_unpack__

            # Unpack state into a graph and key dependencies
            unpacked_layer = unpack_state(layer["state"], dsk, deps)
            dsk.update(unpacked_layer["dsk"])
            for k, v in unpacked_layer["deps"].items():
                deps[k] = deps.get(k, set()) | v

            # Unpack the annotations
            unpack_anno(anno, layer["annotations"], unpacked_layer["dsk"].keys())

        return {"dsk": dsk, "deps": deps, "annotations": anno}

    def __repr__(self) -> str:
        representation = f"{type(self).__name__} with {len(self.layers)} layers.\n"
        representation += f"<{self.__class__.__module__}.{self.__class__.__name__} object at {hex(id(self))}>\n"
        for i, layerkey in enumerate(self._toposort_layers()):
            representation += f" {i}. {layerkey}\n"
        return representation

    def _repr_html_(self):
        highlevelgraph_info = f"{type(self).__name__} with {len(self.layers)} layers."
        highlevelgraph_icon = """
            <svg width="76" height="71" viewBox="0 0 76 71" fill="none" xmlns="http://www.w3.org/2000/svg">
                <circle cx="61.5" cy="36.5" r="13.5"
                    style="stroke: var(--jp-ui-font-color2, #1D1D1D); fill: var(--jp-layout-color1, #F2F2F2);"
                    stroke-width="2"/>
                <circle cx="14.5" cy="14.5" r="13.5"
                    style="stroke: var(--jp-ui-font-color2, #1D1D1D); fill: var(--jp-layout-color1, #F2F2F2);"
                    stroke-width="2"/>
                <circle cx="14.5" cy="56.5" r="13.5"
                    style="stroke: var(--jp-ui-font-color2, #1D1D1D); fill: var(--jp-layout-color1, #F2F2F2);"
                    stroke-width="2"/>
                <path d="M28 16L30.5 16C33.2614 16 35.5 18.2386 35.5 21L35.5 32.0001C35.5 34.7615 37.7386
                    37.0001 40.5 37.0001L43 37.0001" style="stroke: var(--jp-ui-font-color2, #1D1D1D);"
                    stroke-width="1.5"/>
                <path d="M40.5 37L40.5 37.75L40.5 37.75L40.5 37ZM35.5 42L36.25 42L35.5 42ZM35.5 52L34.75
                    52L35.5 52ZM30.5 57L30.5 57.75L30.5 57ZM41.5001 36.25L40.5 36.25L40.5 37.75L41.5001
                    37.75L41.5001 36.25ZM34.75 42L34.75 52L36.25 52L36.25 42L34.75 42ZM30.5 56.25L28.0001
                    56.25L28.0001 57.75L30.5 57.75L30.5 56.25ZM34.75 52C34.75 54.3472 32.8472 56.25 30.5
                    56.25L30.5 57.75C33.6756 57.75 36.25 55.1756 36.25 52L34.75 52ZM40.5 36.25C37.3244 36.25
                    34.75 38.8243 34.75 42L36.25 42C36.25 39.6528 38.1528 37.75 40.5 37.75L40.5 36.25Z"
                    style="fill: var(--jp-ui-font-color2, #1D1D1D);"/>
                <circle cx="28" cy="16" r="2.25" fill="#E5E5E5"
                    style="stroke: var(--jp-ui-font-color2, #1D1D1D);" stroke-width="1.5"/>
                <circle cx="28" cy="57" r="2.25" fill="#E5E5E5"
                    style="stroke: var(--jp-ui-font-color2, #1D1D1D);" stroke-width="1.5"/>
                <path d="M45.25 36.567C45.5833 36.7594 45.5833 37.2406 45.25 37.433L42.25 39.1651C41.9167
                    39.3575 41.5 39.117 41.5 38.7321V35.2679C41.5 34.883 41.9167 34.6425 42.25 34.8349L45.25
                    36.567Z" style="fill: var(--jp-ui-font-color2, #1D1D1D);"/>
            </svg>
        """
        layers_html = ""
        for i, key in enumerate(self._toposort_layers()):
            layer = self.layers[key]
            layers_html += layer._repr_html_(
                layer_index=f" {i}", highlevelgraph_key=key
            )

        html = f"""
            <div>
                <div>
                    <div style="width: 52px; height: 52px; position: absolute;">
                        {highlevelgraph_icon}
                    </div>
                    <div style="margin-left: 64px;">
                        <h3 style="margin-bottom: 0px;">HighLevelGraph</h3>
                        <p style="color: var(--jp-ui-font-color2, #5D5851); margin-bottom:0px;">
                            {highlevelgraph_info}
                        </p>
                        {layers_html}
                    </div>
                </div>
            </div>
        """
        return html


def html_from_dict(info):
    html = """<table style="width: 100%;">"""
    suffix = """</table>"""
    for key, val in info.items():
        table_row = f"""
          <tr>
            <th style="text-align: left; width: 150px;">{key}</th>
            <td style="text-align: left;">{val}</td>
          </tr>
        """
        html += table_row
    html += suffix
    return html


def to_graphviz(
    hg,
    data_attributes=None,
    function_attributes=None,
    rankdir="BT",
    graph_attr=None,
    node_attr=None,
    edge_attr=None,
    **kwargs,
):
    from .dot import graphviz, label, name

    data_attributes = data_attributes or {}
    function_attributes = function_attributes or {}
    graph_attr = graph_attr or {}
    node_attr = node_attr or {}
    edge_attr = edge_attr or {}

    graph_attr["rankdir"] = rankdir
    node_attr["shape"] = "box"
    node_attr["fontname"] = "helvetica"

    graph_attr.update(kwargs)
    g = graphviz.Digraph(
        graph_attr=graph_attr, node_attr=node_attr, edge_attr=edge_attr
    )

    n_tasks = {}
    for layer in hg.dependencies:
        n_tasks[layer] = len(hg.layers[layer])

    mn = min(n_tasks.values())
    mx = max(n_tasks.values())

    cache = {}

    for layer in hg.dependencies:
        layer_name = name(layer)
        attrs = data_attributes.get(layer, {})

        xlabel = label(layer, cache=cache)
        xfontsize = (
            20 if mx == mn else int(20 + ((n_tasks[layer] - mn) / (mx - mn)) * 20)
        )

        attrs.setdefault("label", str(xlabel))
        attrs.setdefault("fontsize", str(xfontsize))

        g.node(layer_name, **attrs)

    for layer, deps in hg.dependencies.items():
        layer_name = name(layer)
        for dep in deps:
            dep_name = name(dep)
            g.edge(dep_name, layer_name)
    return g


def _get_some_layer_name(collection) -> str:
    """Somehow get a unique name for a Layer from a non-HighLevelGraph dask mapping"""
    try:
        (name,) = collection.__dask_layers__()
        return name
    except (AttributeError, ValueError):
        # collection does not define the optional __dask_layers__ method
        # or it spuriously returns more than one layer
        return str(id(collection))
