-
-
Notifications
You must be signed in to change notification settings - Fork 1.8k
[WIP] Pass non HLG objects wout materialization #10369
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
fjetter
wants to merge
3
commits into
dask:main
from
fjetter:dask_graph_protocol_wout_materialization
Closed
Changes from all commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -32,7 +32,7 @@ | |
| from dask.core import literal, quote | ||
| from dask.hashing import hash_buffer_hex | ||
| from dask.system import CPU_COUNT | ||
| from dask.typing import Key, SchedulerGetCallable | ||
| from dask.typing import DaskCollection2, Key, SchedulerGetCallable, TaskGraphFactory | ||
| from dask.utils import ( | ||
| Dispatch, | ||
| apply, | ||
|
|
@@ -231,6 +231,8 @@ def is_dask_collection(x) -> bool: | |
| implementation of the protocol. | ||
|
|
||
| """ | ||
| if isinstance(x, DaskCollection2): | ||
| return True | ||
| if ( | ||
| isinstance(x, type) | ||
| or not hasattr(x, "__dask_graph__") | ||
|
|
@@ -414,36 +416,69 @@ def optimization_function(x): | |
| return getattr(x, "__dask_optimize__", dont_optimize) | ||
|
|
||
|
|
||
| def collections_to_dsk(collections, optimize_graph=True, optimizations=(), **kwargs): | ||
| """ | ||
| Convert many collections into a single dask graph, after optimization | ||
| """ | ||
| from dask.highlevelgraph import HighLevelGraph | ||
|
|
||
| optimizations = tuple(optimizations) + tuple(config.get("optimizations", ())) | ||
| def newstyle_collections(collections): | ||
| from dask.delayed import Delayed | ||
|
|
||
| if optimize_graph: | ||
| groups = groupby(optimization_function, collections) | ||
| is_newstyle = [ | ||
| isinstance(c, DaskCollection2) and not isinstance(c, Delayed) | ||
| for c in collections | ||
| ] | ||
| if any(is_newstyle): | ||
| if not all(is_newstyle): | ||
| raise RuntimeError("Provided new- and old-style collections.") | ||
| if not all( | ||
| isinstance(c.__dask_graph_factory__(), TaskGraphFactory) | ||
| for c in collections | ||
| ): | ||
| raise TypeError("Newstyle collections must have a TaskGraphFactory graph.") | ||
|
|
||
| graphs = [] | ||
| for opt, val in groups.items(): | ||
| dsk, keys = _extract_graph_and_keys(val) | ||
| dsk = opt(dsk, keys, **kwargs) | ||
| return True | ||
| return False | ||
|
|
||
| for opt_inner in optimizations: | ||
| dsk = opt_inner(dsk, keys, **kwargs) | ||
|
|
||
| graphs.append(dsk) | ||
| def collections_to_dsk( | ||
| collections, optimize_graph=True, optimizations=(), **kwargs | ||
| ) -> TaskGraphFactory: | ||
| """ | ||
| Convert many collections into a single dask graph, after optimization | ||
| """ | ||
| from dask.highlevelgraph import HighLevelGraph, TaskFactoryHLGWrapper | ||
|
|
||
| # Merge all graphs | ||
| if any(isinstance(graph, HighLevelGraph) for graph in graphs): | ||
| dsk = HighLevelGraph.merge(*graphs) | ||
| if newstyle_collections(collections): | ||
| graph_factories = [c.__dask_graph_factory__() for c in collections] | ||
| if len(graph_factories) > 1: | ||
| expr = type(graph_factories[0]).combine_factories(*graph_factories) | ||
| else: | ||
| dsk = merge(*map(ensure_dict, graphs)) | ||
| expr = collections[0] | ||
| return expr.optimize() | ||
| else: | ||
| dsk, _ = _extract_graph_and_keys(collections) | ||
|
|
||
| return dsk | ||
| optimizations = tuple(optimizations) + tuple(config.get("optimizations", ())) | ||
| if optimize_graph: | ||
| ext_keys = [] | ||
| groups = groupby(optimization_function, collections) | ||
|
|
||
| graphs = [] | ||
| for opt, val in groups.items(): | ||
| dsk, keys = _extract_graph_and_keys(val) | ||
| ext_keys.extend(keys) | ||
| dsk = opt(dsk, keys, **kwargs) | ||
|
|
||
| for opt_inner in optimizations: | ||
| dsk = opt_inner(dsk, keys, **kwargs) | ||
|
|
||
| graphs.append(dsk) | ||
|
|
||
| # Merge all graphs | ||
| if any(isinstance(graph, HighLevelGraph) for graph in graphs): | ||
| dsk = HighLevelGraph.merge(*graphs) | ||
| return TaskFactoryHLGWrapper(dsk, ext_keys) | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I defined this wrapper just to streamline stuff in distributed. As pointed out in dask/distributed#7942 (comment) we need more than just the graph, we also need the output keys to actually know what to compute |
||
| else: | ||
| dsk = merge(*map(ensure_dict, graphs)) | ||
| return TaskFactoryHLGWrapper.from_low_level(dsk, ext_keys) | ||
| else: | ||
| return TaskFactoryHLGWrapper.from_low_level( | ||
| *_extract_graph_and_keys(collections) | ||
| ) | ||
|
|
||
|
|
||
| def _extract_graph_and_keys(vals): | ||
|
|
@@ -654,17 +689,27 @@ def compute( | |
| collections=collections, | ||
| get=get, | ||
| ) | ||
| if newstyle_collections(collections): | ||
| collections = [c.finalize_compute() for c in collections] | ||
| graph_factory = collections_to_dsk(collections, optimize_graph, **kwargs) | ||
| dsk = graph_factory.materialize() | ||
| keys = list(graph_factory.__dask_output_keys__()) | ||
| return schedule(dsk, keys, **kwargs) | ||
| else: | ||
| graph_factory = collections_to_dsk(collections, optimize_graph, **kwargs) | ||
| from dask.highlevelgraph import TaskFactoryHLGWrapper | ||
|
|
||
| dsk = collections_to_dsk(collections, optimize_graph, **kwargs) | ||
| keys, postcomputes = [], [] | ||
| for x in collections: | ||
| keys.append(x.__dask_keys__()) | ||
| postcomputes.append(x.__dask_postcompute__()) | ||
| assert isinstance(graph_factory, TaskFactoryHLGWrapper) | ||
| dsk = graph_factory.materialize() | ||
| keys, postcomputes = [], [] | ||
| for x in collections: | ||
| keys.append(x.__dask_keys__()) | ||
| postcomputes.append(x.__dask_postcompute__()) | ||
|
|
||
| with shorten_traceback(): | ||
| results = schedule(dsk, keys, **kwargs) | ||
| with shorten_traceback(): | ||
| results = schedule(dsk, keys, **kwargs) | ||
|
|
||
| return repack([f(r, *a) for r, (f, a) in zip(results, postcomputes)]) | ||
| return repack([f(r, *a) for r, (f, a) in zip(results, postcomputes)]) | ||
|
|
||
|
|
||
| def visualize( | ||
|
|
@@ -757,7 +802,8 @@ def visualize( | |
| """ | ||
| args, _ = unpack_collections(*args, traverse=traverse) | ||
|
|
||
| dsk = dict(collections_to_dsk(args, optimize_graph=optimize_graph)) | ||
| graph_fact = collections_to_dsk(args, optimize_graph=optimize_graph) | ||
| dsk = graph_fact.materialize() | ||
|
|
||
| return visualize_dsk( | ||
| dsk=dsk, | ||
|
|
@@ -989,21 +1035,32 @@ def persist(*args, traverse=True, optimize_graph=True, scheduler=None, **kwargs) | |
| collections, optimize_graph=optimize_graph, **kwargs | ||
| ) | ||
| return repack(results) | ||
| # FIXME: Both paths needed | ||
| if newstyle_collections(collections): | ||
| graph_fact = collections_to_dsk(collections, optimize_graph, **kwargs) | ||
| dsk = graph_fact.materialize() | ||
| keys = graph_fact.__dask_output_keys__() | ||
| with shorten_traceback(): | ||
| results = schedule(dsk, keys, **kwargs) | ||
|
|
||
| d = dict(zip(keys, results)) | ||
| return [c.postpersist(d) for c in collections] | ||
| else: | ||
| dsk = collections_to_dsk(collections, optimize_graph, **kwargs) | ||
| dsk = dsk._hlg | ||
| keys, postpersists = [], [] | ||
| for a in collections: | ||
| a_keys = list(flatten(a.__dask_keys__())) | ||
| rebuild, state = a.__dask_postpersist__() | ||
| keys.extend(a_keys) | ||
| postpersists.append((rebuild, a_keys, state)) | ||
|
|
||
| dsk = collections_to_dsk(collections, optimize_graph, **kwargs) | ||
| keys, postpersists = [], [] | ||
| for a in collections: | ||
| a_keys = list(flatten(a.__dask_keys__())) | ||
| rebuild, state = a.__dask_postpersist__() | ||
| keys.extend(a_keys) | ||
| postpersists.append((rebuild, a_keys, state)) | ||
|
|
||
| with shorten_traceback(): | ||
| results = schedule(dsk, keys, **kwargs) | ||
| with shorten_traceback(): | ||
| results = schedule(dsk, keys, **kwargs) | ||
|
|
||
| d = dict(zip(keys, results)) | ||
| results2 = [r({k: d[k] for k in ks}, *s) for r, ks, s in postpersists] | ||
| return repack(results2) | ||
| d = dict(zip(keys, results)) | ||
| results2 = [r({k: d[k] for k in ks}, *s) for r, ks, s in postpersists] | ||
| return repack(results2) | ||
|
|
||
|
|
||
| ############ | ||
|
|
@@ -1283,6 +1340,8 @@ def register_numpy(): | |
| def normalize_array(x): | ||
| if not x.shape: | ||
| return (x.item(), x.dtype) | ||
| if x.size == 0: | ||
| return (None, x.dtype, x.shape) | ||
| if hasattr(x, "mode") and getattr(x, "filename", None): | ||
| if hasattr(x.base, "ctypes"): | ||
| offset = ( | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The code here is still pretty ugly since it requires boilerplate all over for compat purposes.
It probably doesn't make sense to have a switch inside of this function. I'd likely define a function that only works for newstyle and oldstyle collections since we'll need another switch outside as well.