Thanks to visit codestin.com
Credit goes to github.com

Skip to content

spans and annotations lost across copute() boundaries #9138

@Fogapod

Description

@Fogapod

Describe the issue:

Minimal Complete Verifiable Example:

This is broken for both sync and async delayed functions. Calling functions directly (removing delayed decorator) preserves spans and annotations.
This is an issue for both local and distributed cluster.

import asyncio
import time

from dask import annotate, delayed, get_annotations
from dask.distributed import LocalCluster
from distributed import span

cluster = LocalCluster(processes=False)
client = cluster.get_client()


def outer(i: int):
    with span(str(i)), annotate(also_i=i):
        print("annotations before compute", get_annotations())

        # inner_async(i).compute(optimize_graph=False)
        inner_sync(i).compute(optimize_graph=False)


@delayed
def inner_sync(i: int):
    for _ in range(5):
        time.sleep(1)

        print("task", i, "annotations:", get_annotations())


@delayed
async def inner_async(i: int):
    for _ in range(5):
        await asyncio.sleep(1)

        print("task", i, "annotations:", get_annotations())


client.submit(outer, 1)

time.sleep(2)

client.submit(outer, 2).result()

Results in:

annotations before compute {'span': {'name': ('1',), 'ids': ('e6f9a463-dbea-42d9-86e9-74348f017677',)}, 'also_i': 1}
task 1 annotations: {'span': {'name': ('1',), 'ids': ('e6f9a463-dbea-42d9-86e9-74348f017677',)}}
annotations before compute {'span': {'name': ('2',), 'ids': ('e975dde3-d903-4643-b7ad-ee26f239be78',)}, 'also_i': 2}
task 1 annotations: {'span': {'name': ('1',), 'ids': ('e6f9a463-dbea-42d9-86e9-74348f017677',)}}
task 2 annotations: {'span': {'name': ('1',), 'ids': ('e6f9a463-dbea-42d9-86e9-74348f017677',)}}
task 1 annotations: {'span': {'name': ('1',), 'ids': ('e6f9a463-dbea-42d9-86e9-74348f017677',)}}
task 2 annotations: {'span': {'name': ('1',), 'ids': ('e6f9a463-dbea-42d9-86e9-74348f017677',)}}
task 1 annotations: {'span': {'name': ('1',), 'ids': ('e6f9a463-dbea-42d9-86e9-74348f017677',)}}
task 2 annotations: {'span': {'name': ('1',), 'ids': ('e6f9a463-dbea-42d9-86e9-74348f017677',)}}
task 1 annotations: {'span': {'name': ('1',), 'ids': ('e6f9a463-dbea-42d9-86e9-74348f017677',)}}
task 2 annotations: {'span': {'name': ('1',), 'ids': ('e6f9a463-dbea-42d9-86e9-74348f017677',)}}
task 2 annotations: {'span': {'name': ('1',), 'ids': ('e6f9a463-dbea-42d9-86e9-74348f017677',)}}

2 things here:

  • annotations got lost after entering delayed
  • span 2 did not overwrite span 1 for second task

Anything else we need to know?:

There is an issue about lost annotations from 2021: dask/dask#7036
It mentions graph optimization but optimize_graph=False does not help.

Environment:

  • Dask version: 2025.11.0
  • Python version: 3.12
  • Operating System: Linux
  • Install method (conda, pip, source): uv

Metadata

Metadata

Assignees

No one assigned

    Labels

    bugSomething is broken

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions