-
-
Notifications
You must be signed in to change notification settings - Fork 748
Description
We've recently been refactoring the worker state machine (#5736)
The new system works great for single-task-based transitions but feels awkward as soon as we're trying to treat groups or batches of tasks similarly which is particularly important for fetching remote data. We are trying to not perform a network request per key but rather try to perform a request per worker and try to get as many keys of it as reasonable.
This mechanism worked so far by running Worker.(_)ensure_communicating frequently which iterates over all tasks and schedules a fetch/gather_dep (pseudo code below)
def ensure_communicating(...):
skip = []
while self.data_needed and free_slots_for_fetch:
ts = self.data_needed.pop() # highest priority task
worker = pick_good_worker(ts) # not busy, maybe localhost, etc.
if not worker:
skip.add(ts)
more_keys_on_worker = select_keys_for_gather(worker)
schedule_gather_dep(ts, more_keys_on_worker, worker) # This schedules a coroutine and actually fetches the data
self.data_needed.update(skip) # data needed is a heap so this is non-linearThis has a couple of problems
- It's unclear when to run this function. Past iterations just ran this frequently (on every incoming request and in various code places where we guess there may be something that needs transitioning
- We're iterating over all keys, all the time. This is bad for performance although not critical on the worker Remove EnsureCommunicatingAfterTransitions #6462 (comment)
- If we're calling this too soon, e.g. within a released->fetch transition, we may get poor scheduling decisions by fetching tasks prematurely since we didn't work off all fetch recommendations before calling this. For this reason, the
EnsureCommunicatingAfterTransitionsinstruction was introduced (see also Remove EnsureCommunicatingAfterTransitions #6462) to mimic earlier behavior
Proposal
I think we can get rid of Worker.data_needed entirely. Since we're storing everything in heaps, by now, Worker.data_needed_per_worker carries everything we need if we change how ensure_communicating works which might have other interesting side effects.
To illustrate this, consider the following two helper functions
def get_most_important_worker_to_fetch_from(self):
"""
Get the remote worker that holds the _most important_ data we should fetch. This is typically inferred by looking at task priorities.
This could also be used to modify what we consider most important worker to fetch from, e.g. due to load/business, memory pressure, etc.
"""
# Linear in num_workers
res = None
prio = None
for w, heap in self.data_needed_per_worker.items():
nprio = heap.peek().priority
if w not in self.busy_workers and (res is None or nprio > prio):
res = w
prio = nprio
return res
def get_tasks_for_worker(self, w, limit=42):
"""Get a list of tasks on worker `w` such that combined they have at most `limit` estimated bytes"""
# This is more or less the current `Worker._select_keys_for_gather`
# Linear in number of tasks in local state fetch on remote worker
res = []
size = 0
for ts in self.data_needed_per_worker[w]:
size += ts.get_nbytes()
res.append(ts)
if size > limit:
break
return resWith these two helpers we should be able to construct a new, simplified version if we switch our loop from tasks to workers
def _new_ensure_communicate_with_tasks_by_worker():
if network_limits_not_reached and worker := self.get_most_important_worker_to_fetch_from():
tasks_to_fetch = self.get_tasks_for_worker(worker)
instruction = GatherDep(
worker,
tasks_to_fetch,
)or even change the GatherDep instruction to worker-only and delay the decision of what tasks to fetch until this instruction is acted on, which typically only happens after all transitions in Worker.transitions are worked off
def _new_ensure_communicate_with_tasks_by_worker():
if network_limits_not_reached and worker := self.get_most_important_worker_to_fetch_from():
instruction = GatherDep(worker)
def gather_dep(self, worker):
tasks_to_fetch = self.get_tasks_for_worker(worker)
...cc @crusaderky