Skip to content

Commit dbb13ec

Browse files
authored
No longer hold dependencies of erred tasks in memory #4918
This is a follow up to #4784 and reduces complexity of Worker.release_key significantly. There is one non-trivial behavioural change regarding erred tasks. Current main branch holds on to dependencies of an erred task on a worker and implements a release mechanism once that erred task is released. I implemented this recently trying to capture status quo but I'm not convinced any longer that this is the correct behaviour. It treats the erred case specially which introduces a lot of complexity. The only place where this might be of interest is if an erred task wants to be recomputed locally. Not forgetting the data keys until the erred task was released would speed up this process. However, we'd still need to potentially compute some keys and I'm inclined to strike this feature in favour of reduced complexity.
1 parent b9d2e3b commit dbb13ec

File tree

2 files changed

+41
-76
lines changed

2 files changed

+41
-76
lines changed

distributed/tests/test_worker.py

Lines changed: 30 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -48,6 +48,7 @@
4848
mul,
4949
nodebug,
5050
slowinc,
51+
slowsum,
5152
)
5253
from distributed.worker import Worker, error_message, logger, parse_memory_limit
5354

@@ -2087,11 +2088,6 @@ def raise_exc(*args):
20872088
await asyncio.sleep(0.01)
20882089

20892090
expected_states = {
2090-
# We currently don't have a good way to actually release this memory as
2091-
# long as the tasks still have a dependent. We'll need to live with this
2092-
# memory for now
2093-
f.key: "memory",
2094-
g.key: "memory",
20952091
res.key: "error",
20962092
}
20972093

@@ -2159,14 +2155,14 @@ def raise_exc(*args):
21592155

21602156
expected_states = {
21612157
f.key: "memory",
2158+
g.key: "memory",
21622159
}
21632160

21642161
assert_task_states_on_worker(expected_states, a)
21652162

21662163
f.release()
21672164
g.release()
21682165

2169-
# This is not happening
21702166
for server in [s, a, b]:
21712167
while server.tasks:
21722168
await asyncio.sleep(0.01)
@@ -2220,13 +2216,14 @@ def raise_exc(*args):
22202216
res.release()
22212217
# We no longer hold any refs to f or g and B didn't have any erros. It
22222218
# releases everything as expected
2223-
while a.tasks:
2219+
while len(a.tasks) > 1:
22242220
await asyncio.sleep(0.01)
22252221

22262222
expected_states = {
22272223
g.key: "memory",
22282224
}
22292225

2226+
assert_task_states_on_worker(expected_states, a)
22302227
assert_task_states_on_worker(expected_states, b)
22312228

22322229
g.release()
@@ -2283,7 +2280,6 @@ def raise_exc(*args):
22832280
assert_task_states_on_worker(expected_states_A, a)
22842281

22852282
expected_states_B = {
2286-
f.key: "memory",
22872283
g.key: "memory",
22882284
h.key: "memory",
22892285
res.key: "error",
@@ -2301,15 +2297,6 @@ def raise_exc(*args):
23012297

23022298
# B must not forget a task since all have a still valid dependent
23032299
expected_states_B = {
2304-
f.key: "memory",
2305-
# We actually cannot hold on to G even though the graph would suggest
2306-
# otherwise. This is because H was only introduced as a dependency and
2307-
# the scheduler never told the worker how H fits into the big picture.
2308-
# Therefore, it thinks that G does not have any dependents anymore and
2309-
# releases it. Too bad. Once we have speculative task assignments this
2310-
# should be more exact since we should always tell the worker what's
2311-
# going on
2312-
# g.key: released,
23132300
h.key: "memory",
23142301
res.key: "error",
23152302
}
@@ -2320,10 +2307,6 @@ def raise_exc(*args):
23202307
expected_states_A = {}
23212308
assert_task_states_on_worker(expected_states_A, a)
23222309
expected_states_B = {
2323-
f.key: "memory",
2324-
# See above
2325-
# g.key: released,
2326-
h.key: "memory",
23272310
res.key: "error",
23282311
}
23292312

@@ -2334,3 +2317,29 @@ def raise_exc(*args):
23342317
for server in [s, a, b]:
23352318
while server.tasks:
23362319
await asyncio.sleep(0.01)
2320+
2321+
2322+
@gen_cluster(client=True, nthreads=[("127.0.0.1", x) for x in range(4)], timeout=None)
2323+
async def test_hold_on_to_replicas(c, s, *workers):
2324+
f1 = c.submit(inc, 1, workers=[workers[0].address], key="f1")
2325+
f2 = c.submit(inc, 2, workers=[workers[1].address], key="f2")
2326+
2327+
sum_1 = c.submit(
2328+
slowsum, [f1, f2], delay=0.1, workers=[workers[2].address], key="sum"
2329+
)
2330+
sum_2 = c.submit(
2331+
slowsum, [f1, sum_1], delay=0.2, workers=[workers[3].address], key="sum_2"
2332+
)
2333+
f1.release()
2334+
f2.release()
2335+
2336+
while sum_2.key not in workers[3].tasks:
2337+
await asyncio.sleep(0.01)
2338+
2339+
while not workers[3].tasks[sum_2.key].state == "memory":
2340+
assert len(s.tasks[f1.key].who_has) >= 2
2341+
assert s.tasks[f2.key].state == "released"
2342+
await asyncio.sleep(0.01)
2343+
2344+
while len(workers[2].tasks) > 1:
2345+
await asyncio.sleep(0.01)

distributed/worker.py

Lines changed: 11 additions & 55 deletions
Original file line numberDiff line numberDiff line change
@@ -1658,8 +1658,7 @@ def add_task(
16581658
ts.dependencies.add(dep_ts)
16591659
dep_ts.dependents.add(ts)
16601660

1661-
if dep_ts.state in ("fetch", "flight"):
1662-
# if we _need_ to grab data or are in the process
1661+
if dep_ts.state not in ("memory",):
16631662
ts.waiting_for_data.add(dep_ts.key)
16641663

16651664
self.update_who_has(who_has=who_has)
@@ -1762,9 +1761,6 @@ def transition_fetch_waiting(self, ts, runspec):
17621761
# clear `who_has` of stale info
17631762
ts.who_has.clear()
17641763

1765-
# remove entry from dependents to avoid a spurious `gather_dep` call``
1766-
for dependent in ts.dependents:
1767-
dependent.waiting_for_data.discard(ts.key)
17681764
except Exception as e:
17691765
logger.exception(e)
17701766
if LOG_PDB:
@@ -1794,9 +1790,6 @@ def transition_flight_waiting(self, ts, runspec):
17941790
# clear `who_has` of stale info
17951791
ts.who_has.clear()
17961792

1797-
# remove entry from dependents to avoid a spurious `gather_dep` call``
1798-
for dependent in ts.dependents:
1799-
dependent.waiting_for_data.discard(ts.key)
18001793
except Exception as e:
18011794
logger.exception(e)
18021795
if LOG_PDB:
@@ -1991,6 +1984,8 @@ def transition_executing_done(self, ts, value=no_value, report=True):
19911984
ts.traceback = msg["traceback"]
19921985
ts.state = "error"
19931986
out = "error"
1987+
for d in ts.dependents:
1988+
d.waiting_for_data.add(ts.key)
19941989

19951990
# Don't release the dependency keys, but do remove them from `dependents`
19961991
for dependency in ts.dependencies:
@@ -2621,12 +2616,12 @@ def release_key(
26212616

26222617
if self.validate:
26232618
assert isinstance(key, str)
2624-
ts = self.tasks.get(key, TaskState(key=key))
2619+
ts = self.tasks.get(key, None)
26252620
# If the scheduler holds a reference which is usually the
26262621
# case when it instructed the task to be computed here or if
26272622
# data was scattered we must not release it unless the
26282623
# scheduler allow us to. See also handle_delete_data and
2629-
if ts and ts.scheduler_holds_ref:
2624+
if ts is None or ts.scheduler_holds_ref:
26302625
return
26312626
logger.debug(
26322627
"Release key %s",
@@ -2640,28 +2635,14 @@ def release_key(
26402635
self.log.append((key, "release-key", {"cause": cause}, reason))
26412636
else:
26422637
self.log.append((key, "release-key", reason))
2643-
if key in self.data and not ts.dependents:
2638+
if key in self.data:
26442639
try:
26452640
del self.data[key]
26462641
except FileNotFoundError:
26472642
logger.error("Tried to delete %s but no file found", exc_info=True)
2648-
if key in self.actors and not ts.dependents:
2643+
if key in self.actors:
26492644
del self.actors[key]
26502645

2651-
# for any dependencies of key we are releasing remove task as dependent
2652-
for dependency in ts.dependencies:
2653-
dependency.dependents.discard(ts)
2654-
2655-
if not dependency.dependents and dependency.state not in (
2656-
# don't boot keys that are in flight
2657-
# we don't know if they're already queued up for transit
2658-
# in a gather_dep callback
2659-
"flight",
2660-
# The same is true for already executing keys.
2661-
"executing",
2662-
):
2663-
self.release_key(dependency.key, reason=f"Dependent {ts} released")
2664-
26652646
for worker in ts.who_has:
26662647
self.has_what[worker].discard(ts.key)
26672648
ts.who_has.clear()
@@ -2681,8 +2662,10 @@ def release_key(
26812662
# Inform the scheduler of keys which will have gone missing
26822663
# We are releasing them before they have completed
26832664
if ts.state in PROCESSING:
2665+
# This path is only hit with work stealing
26842666
msg = {"op": "release", "key": key, "cause": cause}
26852667
else:
2668+
# This path is only hit when calling release_key manually
26862669
msg = {
26872670
"op": "release-worker-data",
26882671
"keys": [key],
@@ -2691,9 +2674,8 @@ def release_key(
26912674
self.batched_stream.send(msg)
26922675

26932676
self._notify_plugins("release_key", key, ts.state, cause, reason, report)
2694-
if key in self.tasks and not ts.dependents:
2695-
self.tasks.pop(key)
2696-
del ts
2677+
del self.tasks[key]
2678+
26972679
except CommClosedError:
26982680
pass
26992681
except Exception as e:
@@ -2704,32 +2686,6 @@ def release_key(
27042686
pdb.set_trace()
27052687
raise
27062688

2707-
def rescind_key(self, key):
2708-
try:
2709-
if self.tasks[key].state not in PENDING:
2710-
return
2711-
2712-
ts = self.tasks.pop(key)
2713-
2714-
# Task has been rescinded
2715-
# For every task that it required
2716-
for dependency in ts.dependencies:
2717-
# Remove it as a dependent
2718-
dependency.dependents.remove(key)
2719-
# If the dependent is now without purpose (no dependencies), remove it
2720-
if not dependency.dependents:
2721-
self.release_key(
2722-
dependency.key, reason="All dependent keys rescinded"
2723-
)
2724-
2725-
except Exception as e:
2726-
logger.exception(e)
2727-
if LOG_PDB:
2728-
import pdb
2729-
2730-
pdb.set_trace()
2731-
raise
2732-
27332689
################
27342690
# Execute Task #
27352691
################

0 commit comments

Comments
 (0)