Skip to content
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

Use IO task marker in scheduling #8950

Merged
merged 21 commits into from
Jan 16, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 3 additions & 0 deletions distributed/scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -3085,6 +3085,9 @@ def is_rootish(self, ts: TaskState) -> bool:
"""
if ts.resource_restrictions or ts.worker_restrictions or ts.host_restrictions:
return False
# Check explicitly marked data producer tasks
if ts.run_spec and ts.run_spec.data_producer:
return True
tg = ts.group
# TODO short-circuit to True if `not ts.dependencies`?
return (
Expand Down
32 changes: 32 additions & 0 deletions distributed/tests/test_scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@

import dask
from dask import bag, delayed
from dask.base import DaskMethodsMixin
from dask.core import flatten
from dask.highlevelgraph import HighLevelGraph, MaterializedLayer
from dask.utils import parse_timedelta, tmpfile, typename
Expand Down Expand Up @@ -5315,3 +5316,34 @@ async def test_alias_resolving_break_queuing(c, s, a):
while not s.tasks:
await asyncio.sleep(0.01)
assert sum([s.is_rootish(v) for v in s.tasks.values()]) == 18


@gen_cluster(client=True, nthreads=[("", 1)])
async def test_data_producers(c, s, a):
from dask._task_spec import DataNode, Task, TaskRef

def func(*args):
return 100

class MyArray(DaskMethodsMixin):
def __dask_graph__(self):
return {
"a": DataNode("a", 10),
"b": Task("b", func, TaskRef("a"), _data_producer=True),
"c": Task("c", func, TaskRef("b")),
"d": Task("d", func, TaskRef("c")),
}

def __dask_keys__(self):
return ["d"]

def __dask_postcompute__(self):
return func, ()

arr = MyArray()
x = c.compute(arr)
await async_poll_for(lambda: s.tasks, 5)
assert (
sum([s.is_rootish(v) and v.run_spec.data_producer for v in s.tasks.values()])
== 2
)
Loading