-
-
Notifications
You must be signed in to change notification settings - Fork 719
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
1 parent
e727d19
commit 7d31b76
Showing
6 changed files
with
285 additions
and
28 deletions.
There are no files selected for viewing
This file contains 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 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 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 |
---|---|---|
@@ -0,0 +1,107 @@ | ||
from __future__ import annotations | ||
|
||
from collections import defaultdict | ||
from collections.abc import Iterator | ||
from contextlib import contextmanager | ||
from typing import TYPE_CHECKING | ||
|
||
import dask.config | ||
|
||
if TYPE_CHECKING: | ||
from distributed import Scheduler | ||
|
||
|
||
@contextmanager | ||
def span(*tags: str) -> Iterator[None]: | ||
"""Tag group of tasks to be part of a certain group, called a span. | ||
This context manager can be nested, thus creating sub-spans. | ||
Each dask.distributed Client automatically defines a root span, which is its own | ||
random client ID. | ||
Examples | ||
-------- | ||
>>> import dask.array as da | ||
>>> import distributed | ||
>>> client = distributed.Client() | ||
>>> with span("my_workflow"): | ||
... with span("phase 1"): | ||
... a = da.random.random(10) | ||
... b = a + 1 | ||
... with span("phase 2"): | ||
... c = b * 2 | ||
>>> c.compute() | ||
In the above example, | ||
- Tasks of collections a and b will be annotated on the scheduler and workers with | ||
``{'span': ('Client-6e31a38d-fbe3-11ed-83dd-b42e99c1ab7d', 'my_workflow', 'phase 1')}`` | ||
- Tasks of collection c (that aren't already part of a or b) will be annotated with | ||
``{'span': ('Client-6e31a38d-fbe3-11ed-83dd-b42e99c1ab7d', 'my_workflow', 'phase 2')}`` | ||
The client ID will change randomly every time the client is reinitialized. | ||
You may also set more than one tag at once; e.g. | ||
>>> with span("workflow1", "version1"): | ||
... ... | ||
Note | ||
---- | ||
Spans are based on annotations, and just like annotations they can be lost during | ||
optimization. Set config ``optimize.fuse.active: false`` to prevent this issue. | ||
""" | ||
prev_id = dask.config.get("annotations.span", ()) | ||
with dask.config.set({"annotations.span": prev_id + tags}): | ||
yield | ||
|
||
|
||
class Span: | ||
id: tuple[str, ...] | ||
children: set[Span] | ||
|
||
__slots__ = tuple(__annotations__) | ||
|
||
def __init__(self, span_id: tuple[str, ...]): | ||
self.id = span_id | ||
self.children = set() | ||
|
||
def __repr__(self) -> str: | ||
return f"Span{self.id}" | ||
|
||
|
||
class SpansExtension: | ||
"""Scheduler extension for spans support""" | ||
|
||
#: All Span objects by span_id | ||
spans: dict[tuple[str, ...], Span] | ||
|
||
#: Only the spans that don't have any parents {client_id: Span}. | ||
#: This is a convenience helper structure to speed up searches. | ||
root_spans: dict[str, Span] | ||
|
||
#: All spans, keyed by the individual tags that make up their span_id. | ||
#: This is a convenience helper structure to speed up searches. | ||
spans_search_by_tag: defaultdict[str, set[Span]] | ||
|
||
def __init__(self, scheduler: Scheduler): | ||
self.spans = {} | ||
self.root_spans = {} | ||
self.spans_search_by_tag = defaultdict(set) | ||
|
||
def ensure_span(self, span_id: tuple[str, ...]) -> Span: | ||
"""Create Span if it doesn't exist and return it""" | ||
try: | ||
return self.spans[span_id] | ||
except KeyError: | ||
pass | ||
|
||
span = self.spans[span_id] = Span(span_id) | ||
for tag in span_id: | ||
self.spans_search_by_tag[tag].add(span) | ||
if len(span_id) > 1: | ||
parent = self.ensure_span(span_id[:-1]) | ||
parent.children.add(span) | ||
else: | ||
self.root_spans[span_id[0]] = span | ||
|
||
return span |
This file contains 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.