Skip to content

Commit 53a5679

Browse files
authored
Remove alias resolving to fix queuing (#8933)
1 parent e2f3f96 commit 53a5679

File tree

2 files changed

+18
-18
lines changed

2 files changed

+18
-18
lines changed

distributed/scheduler.py

+4-18
Original file line numberDiff line numberDiff line change
@@ -54,14 +54,9 @@
5454

5555
import dask
5656
import dask.utils
57-
from dask._task_spec import (
58-
DependenciesMapping,
59-
GraphNode,
60-
convert_legacy_graph,
61-
resolve_aliases,
62-
)
57+
from dask._task_spec import DependenciesMapping, GraphNode, convert_legacy_graph
6358
from dask.base import TokenizationError, normalize_token, tokenize
64-
from dask.core import istask, reverse_dict, validate_key
59+
from dask.core import istask, validate_key
6560
from dask.typing import Key, no_default
6661
from dask.utils import (
6762
_deprecated,
@@ -9411,19 +9406,10 @@ def _materialize_graph(
94119406
)
94129407

94139408
dsk2 = convert_legacy_graph(dsk)
9414-
dependents = reverse_dict(DependenciesMapping(dsk2))
9415-
# This is removing weird references like "x-foo": "foo" which often make up
9416-
# a substantial part of the graph
9417-
# This also performs culling!
9418-
dsk3 = resolve_aliases(dsk2, keys, dependents)
9419-
9420-
logger.debug(
9421-
"Removing aliases. Started with %i and got %i left", len(dsk2), len(dsk3)
9422-
)
94239409
# FIXME: There should be no need to fully materialize and copy this but some
94249410
# sections in the scheduler are mutating it.
9425-
dependencies = {k: set(v) for k, v in DependenciesMapping(dsk3).items()}
9426-
return dsk3, dependencies, annotations_by_type
9411+
dependencies = {k: set(v) for k, v in DependenciesMapping(dsk2).items()}
9412+
return dsk2, dependencies, annotations_by_type
94279413

94289414

94299415
def _cull(dsk: dict[Key, GraphNode], keys: set[Key]) -> dict[Key, GraphNode]:

distributed/tests/test_scheduler.py

+14
Original file line numberDiff line numberDiff line change
@@ -5304,3 +5304,17 @@ async def before_close(self):
53045304
async def test_rootish_taskgroup_configuration(c, s, *workers):
53055305
assert s.rootish_tg_threshold == 10
53065306
assert s.rootish_tg_dependencies_threshold == 15
5307+
5308+
5309+
@gen_cluster(client=True, nthreads=[("", 1)])
5310+
async def test_alias_resolving_break_queuing(c, s, a):
5311+
pytest.importorskip("numpy")
5312+
import dask.array as da
5313+
5314+
arr = da.random.random((90, 100), chunks=(10, 50))
5315+
result = arr.rechunk(((10, 7, 7, 6) * 3, (50, 50)))
5316+
result = result.sum(split_every=1000)
5317+
x = result.persist()
5318+
while not s.tasks:
5319+
await asyncio.sleep(0.01)
5320+
assert sum([s.is_rootish(v) for v in s.tasks.values()]) == 18

0 commit comments

Comments
 (0)