diff --git a/distributed/tests/test_spans.py b/distributed/tests/test_spans.py index cf01e11ec7..c345a7dfcd 100644 --- a/distributed/tests/test_spans.py +++ b/distributed/tests/test_spans.py @@ -26,25 +26,22 @@ async def test_spans(c, s, a): x = delayed(inc)(1) # Default span - @span("p2") - def f(i): - return i * 2 - with span("my workflow") as mywf_id: with span("p1") as p1_id: y = x + 1 - z = f(y) + with span("p2") as p2_id: + z = y * 2 zp = c.persist(z) assert await c.compute(zp) == 6 ext = s.extensions["spans"] - p2_id = s.tasks[z.key].group.span_id assert mywf_id assert p1_id assert p2_id assert s.tasks[y.key].group.span_id == p1_id + assert s.tasks[z.key].group.span_id == p2_id assert mywf_id != p1_id != p2_id expect_annotations = { @@ -135,15 +132,12 @@ async def test_repeat_span(c, s, a, b): """Opening and closing the same span will result in multiple spans with different ids and same name """ - - @span("foo") - def f(x, key): - return c.submit(inc, x, key=key) - with span("foo"): x = c.submit(inc, 1, key="x") - y = f(x, key="y") - z = f(y, key="z") + with span("foo"): + y = c.submit(inc, x, key="y") + with span("foo"): + z = c.submit(inc, y, key="z") assert await z == 4 sbn = s.extensions["spans"].spans_search_by_name["foo",] diff --git a/docs/source/spans.rst b/docs/source/spans.rst index dcb5339cb8..235cc4fb91 100644 --- a/docs/source/spans.rst +++ b/docs/source/spans.rst @@ -32,27 +32,19 @@ For example: dask.config.set({"optimization.fuse.active": False}) client = Client() - @span("ML preprocessing") - def preprocess(df): - ... - return df - - @span("Model training") - def train(df): - ... - return model - with span("Alice's workflow"): with span("data load"): df = dd.read_parquet(...) - df = preprocess(df) - model = train(df) + with span("ML preprocessing"): + df = preprocess(df) + + with span("Model training"): + model = train(df) model = model.compute() -In the above example, note how the :func:`span` context manager is used as a decorator -too, and how it can be nested. +Note how the :func:`span` context manager can be nested. The example will create the following spans on the scheduler: - ``("Alice's workflow", )``