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

Do not filter tasks before gathering data #6371

Merged
merged 7 commits into from
May 20, 2022
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
4 changes: 2 additions & 2 deletions distributed/tests/test_cancelled_state.py
Original file line number Diff line number Diff line change
Expand Up @@ -321,7 +321,7 @@ async def get_data(self, comm, *args, **kwargs):
# The initial free-keys is rejected
("free-keys", (fut1.key,)),
(fut1.key, "resumed", "released", "cancelled", {}),
# After gather_dep receives the data, it tries to transition to memory but the task will release instead
# After gather_dep receives the data, the task is forgotten
(fut1.key, "cancelled", "memory", "released", {fut1.key: "forgotten"}),
],
)
Expand Down Expand Up @@ -366,7 +366,7 @@ def block_execution(event, lock):


@gen_cluster(client=True, nthreads=[("", 1, {"resources": {"A": 1}})])
async def test_cancelled_error_with_ressources(c, s, a):
async def test_cancelled_error_with_resources(c, s, a):
executing = Event()
lock_executing = Lock()
await lock_executing.acquire()
Expand Down
48 changes: 47 additions & 1 deletion distributed/tests/test_worker_state_machine.py
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@

from distributed.protocol.serialize import Serialize
from distributed.utils import recursive_to_dict
from distributed.utils_test import assert_story, gen_cluster, inc
from distributed.utils_test import _LockedCommPool, assert_story, gen_cluster, inc
from distributed.worker_state_machine import (
ExecuteFailureEvent,
ExecuteSuccessEvent,
Expand Down Expand Up @@ -302,3 +302,49 @@ async def test_fetch_via_amm_to_compute(c, s, a, b):
b.comm_threshold_bytes = old_comm_threshold

await f1


@gen_cluster(client=True)
async def test_cancelled_while_in_flight(c, s, a, b):
event = asyncio.Event()
a.rpc = _LockedCommPool(a.rpc, write_event=event)

x = c.submit(inc, 1, key="x", workers=[b.address])
y = c.submit(inc, x, key="y", workers=[a.address])
await wait_for_state("x", "flight", a)
y.release()
await wait_for_state("x", "cancelled", a)

# Let the comm from b to a return the result
event.set()
# upon reception, x transitions cancelled->forgotten
while a.tasks:
await asyncio.sleep(0.01)


@gen_cluster(client=True)
async def test_in_memory_while_in_flight(c, s, a, b):
"""
1. A client scatters x to a
2. The scheduler does not know about scattered keys until the three-way round-trip
between client, worker, and scheduler has been completed (see Scheduler.scatter)
3. In the middle of that handshake, a client (not necessarily the same client) calls
``{op: compute-task, key: x}`` on b and then
``{op: compute-task, key: y, who_has: {x: [b]}`` on a, which triggers a
gather_dep call to copy x key from b to a.
4. while x is in flight from b to a, the scatter finishes, which triggers
update_data, which in turn transitions x from flight to memory.
5. later on, gather_dep finishes, but the key is already in memory.
"""
event = asyncio.Event()
a.rpc = _LockedCommPool(a.rpc, write_event=event)

x = c.submit(inc, 1, key="x", workers=[b.address])
y = c.submit(inc, x, key="y", workers=[a.address])
await wait_for_state("x", "flight", a)
a.update_data({"x": 3})
Comment on lines +344 to +345
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if x is already in flight, how could the data end up in memory without us doing it explicitly like in this unit test?
acquire_replica and "fetch_dependency" should not fetch this key a second time.

From me reading the code, the only way this could happen is via Client.scatter. I would argue a user should not be allowed to scatter a key that is already known to the cluster to be computed.

I don't want to block this PR for this test but if the above outline is how we end up in this situation, I think we should prohibit scattering such keys and shrink the space of possible/allowed transitions.

Specifically, I'm inclined to say a.update({"x": 3}) should raise an exception if x is in flight.

Thoughts?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Specifically, I'm inclined to say a.update({"x": 3}) should raise an exception if x is in flight.

would might translate to something like

def transition_flight_memory(...):
    if not ts.done:
        raise ImpossibleTransition("A nice exception that tells us that we cannot move data to memory while in flight but coro/task still running")

(Where the exception is supposed to be raised is not the point of my argument. It may not be feasible to raise in the transition itself, idk)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The more I think about this the stronger I feel about it because these kind of race conditions are part of why I introduced cancelled/resumed to avoid us needing to deal with these transitions.
If the fetch task would finish successfully, this would cause a memory->memory transition. Since this is not allowed/possible this would cause a

memory->released
(possibly the released transition would cancel some follow up tasks)
released->memory

or as a concrete story

[
    (ts.key, "flight", "memory", "memory", {dependent: "executing"})
    (dependent.key, "waiting", "executing", "executing", {}),
    # A bit later after gather_dep returns
    (ts.key, "memory", "memory", "released", {dependent: "released"}),
    (dependent.key, "executing", "released", "cancelled", {}),
    (ts.key, "released", "memory", "memory", {dependent: "waiting"}),
    (dependent.key, "cancelled", "waiting", "executing", {}),
]

writing down the expected story made me realize that our transition flow should heal us here but we'd be performing a lot of unnecessary transitions that could expose us to problems.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that 80% of the problem is caused by the non-sequentiality of RPC calls vs. bulk comms.

  1. client scatters to a
  2. the scheduler does not know about scattered keys until the three-way round-trip between client, workers, and scheduler has been completed:
    keys, who_has, nbytes = await scatter_to_workers(
    nthreads, data, rpc=self.rpc, report=False
    )
    self.update_data(who_has=who_has, nbytes=nbytes, client=client)
  3. in the middle of that handshake, a client (not necessarily the same client) calls compute on b and then gather_dep to copy the key from b to a
  4. while the flight from b to a is in progress, the scatter finishes, which triggers update_data as shown in the test.

The only way to avoid this would be to fundamentally rewrite the scatter implementation. Which, for the record, I think is long overdue.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll explain the above in a comment in the test

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My point is only partially about technical correctness of race conditions but also about whether this is even a sane operation. How can a user know the value of x if x is supposed to be computed on the cluster?

await wait_for_state("x", "memory", a)

# Let the comm from b to a return the result
event.set()
assert await y == 4 # Data in flight from b has been discarded
2 changes: 1 addition & 1 deletion distributed/utils_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -1923,7 +1923,7 @@ class _LockedCommPool(ConnectionPool):
)
# It might be necessary to remove all existing comms
# if the wrapped pool has been used before
>>> w.remove(remote_address)
>>> w.rpc.remove(remote_address)

>>> async def ping_pong():
return await w.rpc(remote_address).ping()
Expand Down
109 changes: 25 additions & 84 deletions distributed/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -3150,61 +3150,32 @@ def total_comm_bytes(self):
)
return self.comm_threshold_bytes

def _filter_deps_for_fetch(
self, to_gather_keys: Iterable[str]
) -> tuple[set[str], set[str], TaskState | None]:
"""Filter a list of keys before scheduling coroutines to fetch data from workers.
def _get_cause(self, keys: Iterable[str]) -> TaskState:
"""For diagnostics, we want to attach a transfer to a single task. This task is
typically the next to be executed but since we're fetching tasks for potentially
many dependents, an exact match is not possible. Additionally, if a key was
fetched through acquire-replicas, dependents may not be known at all.

Returns
-------
in_flight_keys:
The subset of keys in to_gather_keys in state `flight` or `resumed`
cancelled_keys:
The subset of tasks in to_gather_keys in state `cancelled` or `memory`
cause:
The task to attach startstops of this transfer to
The task to attach startstops of this transfer to
"""
in_flight_tasks: set[TaskState] = set()
cancelled_keys: set[str] = set()
for key in to_gather_keys:
ts = self.tasks.get(key)
if ts is None:
continue
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should never happen. The finally clause of gather_dep strongly states this by using an unguarded access self.tasks[key].

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, we worked very hard to ensure tasks are not accidentally forgotten. I encourage being as strict as possible with this. A KeyError is often a sign of a messed up transition somewhere else


# At this point, a task has been transitioned fetch->flight
# flight is only allowed to be transitioned into
# {memory, resumed, cancelled}
# resumed and cancelled will block any further transition until this
# coro has been finished

if ts.state in ("flight", "resumed"):
in_flight_tasks.add(ts)
# If the key is already in memory, the fetch should not happen which
# is signalled by the cancelled_keys
elif ts.state in {"cancelled", "memory"}:
cancelled_keys.add(key)
else:
raise RuntimeError(
f"Task {ts.key} found in illegal state {ts.state}. "
"Only states `flight`, `resumed` and `cancelled` possible."
)

# For diagnostics we want to attach the transfer to a single task. this
# task is typically the next to be executed but since we're fetching
# tasks for potentially many dependents, an exact match is not possible.
# If there are no dependents, this is a pure replica fetch
cause = None
for ts in in_flight_tasks:
for key in keys:
ts = self.tasks[key]
if ts.dependents:
cause = next(iter(ts.dependents))
break
else:
cause = ts
in_flight_keys = {ts.key for ts in in_flight_tasks}
return in_flight_keys, cancelled_keys, cause
return next(iter(ts.dependents))
cause = ts
assert cause # Always at least one key
return cause

def _update_metrics_received_data(
self, start: float, stop: float, data: dict, cause: TaskState, worker: str
self,
start: float,
stop: float,
data: dict[str, Any],
cause: TaskState,
worker: str,
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

off topic: post refactor, should this method move to the state machine class, or stay in Worker proper?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's network and diagnostics related so I'm inclined to say this does not belong to the state machine class

) -> None:

total_bytes = sum(self.tasks[key].get_nbytes() for key in data)
Expand Down Expand Up @@ -3253,7 +3224,7 @@ def _update_metrics_received_data(
async def gather_dep(
self,
worker: str,
to_gather: Iterable[str],
to_gather: Collection[str],
total_nbytes: int,
*,
stimulus_id: str,
Expand All @@ -3277,46 +3248,23 @@ async def gather_dep(
recommendations: Recs = {}
instructions: Instructions = []
response = {}
to_gather_keys: set[str] = set()
cancelled_keys: set[str] = set()

def done_event():
return GatherDepDoneEvent(stimulus_id=f"gather-dep-done-{time()}")

try:
to_gather_keys, cancelled_keys, cause = self._filter_deps_for_fetch(
to_gather
)

if not to_gather_keys:
self.log.append(
("nothing-to-gather", worker, to_gather, stimulus_id, time())
)
return done_event()

assert cause
# Keep namespace clean since this func is long and has many
# dep*, *ts* variables
del to_gather

self.log.append(
("request-dep", worker, to_gather_keys, stimulus_id, time())
)
logger.debug(
"Request %d keys for task %s from %s",
len(to_gather_keys),
cause,
worker,
)
self.log.append(("request-dep", worker, to_gather, stimulus_id, time()))
logger.debug("Request %d keys from %s", len(to_gather), worker)

start = time()
response = await get_data_from_worker(
self.rpc, to_gather_keys, worker, who=self.address
self.rpc, to_gather, worker, who=self.address
)
stop = time()
if response["status"] == "busy":
return done_event()

cause = self._get_cause(to_gather)
self._update_metrics_received_data(
start=start,
stop=stop,
Expand Down Expand Up @@ -3369,9 +3317,7 @@ def done_event():
data = response.get("data", {})

if busy:
self.log.append(
("busy-gather", worker, to_gather_keys, stimulus_id, time())
)
self.log.append(("busy-gather", worker, to_gather, stimulus_id, time()))
# Avoid hammering the worker. If there are multiple replicas
# available, immediately try fetching from a different worker.
self.busy_workers.add(worker)
Expand All @@ -3382,12 +3328,7 @@ def done_event():
for d in self.in_flight_workers.pop(worker):
ts = self.tasks[d]
ts.done = True
if d in cancelled_keys:
if ts.state == "cancelled":
recommendations[ts] = "released"
else:
recommendations[ts] = "fetch"
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ts.state == "memory"

elif d in data:
if d in data:
recommendations[ts] = ("memory", data[d])
elif busy:
recommendations[ts] = "fetch"
Expand Down