Skip to content

Commit 07974fd

Browse files
committed
Catch case where only one run_spec is non-deterministic
1 parent 2f42db3 commit 07974fd

File tree

2 files changed

+31
-14
lines changed

2 files changed

+31
-14
lines changed

distributed/scheduler.py

+7-4
Original file line numberDiff line numberDiff line change
@@ -4774,12 +4774,15 @@ def _generate_taskstates(
47744774
# run_spec in the submitted graph may be None. This happens
47754775
# when an already persisted future is part of the graph
47764776
elif k in dsk:
4777+
# If both tokens are non-deterministic, skip comparison
47774778
try:
4778-
tok_lhs: Any = tokenize(ts.run_spec, ensure_deterministic=True)
4779-
tok_rhs: Any = tokenize(dsk[k], ensure_deterministic=True)
4779+
tok_lhs = tokenize(ts.run_spec, ensure_deterministic=True)
47804780
except TokenizationError:
4781-
# Non-deterministic tokens; skip comparison
4782-
tok_lhs = tok_rhs = None
4781+
tok_lhs = ""
4782+
try:
4783+
tok_rhs = tokenize(dsk[k], ensure_deterministic=True)
4784+
except TokenizationError:
4785+
tok_rhs = ""
47834786

47844787
# Additionally check dependency names. This should only be necessary
47854788
# if run_specs can't be tokenized deterministically.

distributed/tests/test_scheduler.py

+24-10
Original file line numberDiff line numberDiff line change
@@ -4757,21 +4757,35 @@ async def test_resubmit_different_task_same_key_many_clients(c, s):
47574757
assert await x2 == 2 # kept old run_spec
47584758

47594759

4760+
@pytest.mark.parametrize(
4761+
"before,after,expect_msg",
4762+
[
4763+
(object(), 123, True),
4764+
(123, object(), True),
4765+
(o := object(), o, False),
4766+
],
4767+
)
47604768
@gen_cluster(client=True, nthreads=[])
4761-
async def test_resubmit_nondeterministic_task_same_deps(c, s):
4769+
async def test_resubmit_nondeterministic_task_same_deps(
4770+
c, s, before, after, expect_msg
4771+
):
47624772
"""Some run_specs can't be tokenized deterministically. Silently skip comparison on
4763-
the run_spec in those cases. Dependencies must be the same.
4773+
the run_spec when both lhs and rhs are nondeterministic.
4774+
Dependencies must be the same.
47644775
"""
4765-
o = object()
4766-
# Round-tripping `o` through two separate cloudpickle.dumps() calls generates two
4767-
# different object instances, which yield different tokens.
4768-
x1 = c.submit(lambda x: x, o, key="x")
4769-
x2 = delayed(lambda x: x)(o, dask_key_name="x")
4776+
x1 = c.submit(lambda x: x, before, key="x")
4777+
x2 = delayed(lambda x: x)(after, dask_key_name="x")
47704778
y = delayed(lambda x: x)(x2, dask_key_name="y")
4771-
fut = c.compute(y)
4772-
await async_poll_for(lambda: "y" in s.tasks, timeout=5)
4779+
4780+
with captured_logger("distributed.scheduler", level=logging.WARNING) as log:
4781+
fut = c.compute(y)
4782+
await async_poll_for(lambda: "y" in s.tasks, timeout=5)
4783+
4784+
has_msg = "Detected different `run_spec` for key 'x'" in log.getvalue()
4785+
assert has_msg == expect_msg
4786+
47734787
async with Worker(s.address):
4774-
assert type(await fut) is object
4788+
assert type(await fut) is type(before)
47754789

47764790

47774791
@pytest.mark.parametrize("add_deps", [False, True])

0 commit comments

Comments
 (0)