-
-
Notifications
You must be signed in to change notification settings - Fork 719
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
Worker state machine refactor #5046
Conversation
distributed/worker.py
Outdated
def transition_released_fetch__recs(self, ts): | ||
if self.validate: | ||
assert ts.state == "released" | ||
assert ts.runspec is None | ||
|
||
for w in ts.who_has: | ||
self.pending_data_per_worker[w].append(ts.key) | ||
ts.state = "fetch" | ||
heapq.heappush(self.data_needed, (ts.priority, ts.key)) | ||
return {}, [] |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we want to keep track on scheduler side of "who is fetching what" we could emit a message in here
distributed/worker.py
Outdated
skipped_worker_in_flight = list() | ||
while self.data_needed and ( | ||
len(self.in_flight_workers) < self.total_out_connections | ||
or self.comm_nbytes < self.total_comm_nbytes | ||
): |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The logic is now (subjectively) a bit simpler. self.data_needed
no longer holds the dependents for which we are fetching dependencies but now it includes the actual dependencies / replicas to fetch
else: | ||
worker = random.choice(list(workers)) | ||
|
||
to_gather, total_nbytes = self.select_keys_for_gather(worker, ts.key) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Even replicas will now benefit from batched fetching, just like ordinary dependencies
distributed/worker.py
Outdated
# FIXME: With the changed system this no longer makes any sense | ||
# It didn't make much sense before since not all of the tasks | ||
# fetched in this coro are even dependencies of `cause` but are | ||
# simply co-located | ||
if cause: | ||
cause.startstops.append( | ||
{ | ||
"action": "transfer", | ||
"start": start + self.scheduler_delay, | ||
"stop": stop + self.scheduler_delay, | ||
"source": worker, | ||
} | ||
) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a bit awkward. The task stream is built such that the startstops are attached to the cause
. The cause
/ dependent
is actually something I would like to remove entirely from gather_dep
but with this work around everything works almost like before
not blocked. working on my TODO list and will update shortly. I'll ping once I require a proper review |
0980736
to
d416520
Compare
|
for w in ts.who_has: | ||
self.pending_data_per_worker[w].append(ts.key) | ||
ts.state = "fetch" | ||
heapq.heappush(self.data_needed, (ts.priority, ts.key)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is the important change. Every task in state fetch
is now part of data_needed
I look forward to reviewing this (although warning, this week is busy for me). In the meantime, two comments:
|
Yes. The logic is probably not 100%, yet, but it should be good enough to get started with |
There is a failure in OSX errors are
|
The |
02fd235
to
da9659f
Compare
759f31d
to
51755d3
Compare
005b17c
to
1ab1327
Compare
@fjetter can you say a little bit about the current status here? Is this close to done, or is there still an indeterminate amount of work left to do? |
I think it's good enough for an increment. There are still things I would like to do but considering the scope I will defer this to a later PR. I will give this another shake down and see if I encounter any more instabilities |
cbb1b28
to
c3ceffd
Compare
distributed/scheduler.py
Outdated
@@ -1969,6 +1969,8 @@ def __init__( | |||
("processing", "erred"): self.transition_processing_erred, | |||
("no-worker", "released"): self.transition_no_worker_released, | |||
("no-worker", "waiting"): self.transition_no_worker_waiting, | |||
# TODO: Write a test. Worker disconnects -> no-worker -> reconnect with task to memory. Triggered every few hundred times by test_handle_superfluous_data |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
- Add a test for the no-worker -> memory scheduler transition
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think this is stable now 🤞
Even though I claimed this to be almost done, I was facing stability issues and went all the way implementing #4413, i.e. there are a few new states but I'll leave explanatory comments once I cleaned up
I will still need to cleanup the code base so I discourage reviews until this is through. I will ping once cleaned up and leave section specific comments about new/changed behaviour to guide reviews.
bee8e03
to
df8e7df
Compare
distributed/cfexecutor.py
Outdated
if isinstance(fs, list): | ||
# Below iterator relies on this being a generator to cancel | ||
# remaining futures | ||
fs = (val for val in fs) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This chunk should disappear after merging from main
Edit: I already reverted 1. and 2.
diff --git a/distributed/stealing.py b/distributed/stealing.py
index 0297691f..24f7ce74 100644
--- a/distributed/stealing.py
+++ b/distributed/stealing.py
@@ -264,7 +264,7 @@ class WorkStealing(SchedulerPlugin):
await self.scheduler.remove_worker(thief.address)
self.log(("confirm", key, victim.address, thief.address))
else:
- raise ValueError(f"Unexpected task state: {state}")
+ raise ValueError(f"Unexpected task state: {ts}")
except Exception as e:
logger.exception(e)
if LOG_PDB:
diff --git a/distributed/tests/test_client_executor.py b/distributed/tests/test_client_executor.py
index f4a90297..fe23f8c7 100644
--- a/distributed/tests/test_client_executor.py
+++ b/distributed/tests/test_client_executor.py
@@ -155,7 +155,7 @@ def test_map(client):
assert number_of_processing_tasks(client) > 0
# Garbage collect the iterator => remaining tasks are cancelled
del it
- time.sleep(0.5)
+ time.sleep(0.1)
assert number_of_processing_tasks(client) == 0
diff --git a/distributed/tests/test_steal.py b/distributed/tests/test_steal.py
index be7f1bc5..82fe97be 100644
--- a/distributed/tests/test_steal.py
+++ b/distributed/tests/test_steal.py
@@ -703,9 +703,10 @@ async def test_dont_steal_already_released(c, s, a, b):
with captured_logger(
logging.getLogger("distributed.stealing"), level=logging.DEBUG
) as stealing_logs:
- msg = f"Key released between request and confirm: {key}"
- while msg not in stealing_logs.getvalue():
+ logs = stealing_logs.getvalue()
+ while f"Key released between request and confirm: {key}" not in logs:
await asyncio.sleep(0.05)
+ logs = stealing_logs.getvalue()
@gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2)
diff --git a/distributed/worker.py b/distributed/worker.py
index c0e48f26..54152d4a 100644
--- a/distributed/worker.py
+++ b/distributed/worker.py
@@ -1693,46 +1693,6 @@ class Worker(ServerNode):
return recommendations, scheduler_msgs
- def transition_table_to_dot(self, filename="worker-transitions", format=None):
- import graphviz
-
- from dask.dot import graphviz_to_file
-
- g = graphviz.Digraph(
- graph_attr={
- "concentrate": "True",
- },
- # node_attr=node_attr,
- # edge_attr=edge_attr
- )
- all_states = set()
- for edge in self._transitions_table.keys():
- all_states.update(set(edge))
-
- seen = set()
- with g.subgraph(name="cluster_0") as c:
- c.attr(style="filled", color="lightgrey")
- c.node_attr.update(style="filled", color="white")
- c.attr(label="executable")
- for state in (
- "waiting",
- "ready",
- "executing",
- "constrained",
- "long-running",
- ):
- c.node(state, label=state)
- seen.add(state)
-
- with g.subgraph(name="cluster_1") as c:
- for state in ["fetch", "flight", "missing"]:
- c.attr(label="dependency")
- c.node(state, label=state)
- seen.add(state)
-
- g.edges(self._transitions_table.keys())
- return graphviz_to_file(g, filename=filename, format=format)
-
def handle_compute_task(
self,
*,
|
You are doing
You set that timeout to 0.5s in main in another PR, which looks like a partial backport of this one. It looks stable there, so I would advise changing it further.
The two algorithms are functionally equivalent
OK |
You are right, of course. Reverted everything except of the removal of the table_to_dot thing 👍 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's Green Seal of Approval from me 🎊
Since there is no more feedback incoming, I intend to merge this PR by tomorrow. If anybody intends to leave another review, please let me know and I can postpone the merge. If nobody raises another issue, I will merge tomorrow around 1:00 pm UTC |
I don't plan to raise any more issues. Thank you for owning this.
…On Mon, Sep 13, 2021 at 1:44 PM Florian Jetter ***@***.***> wrote:
Since there is no more feedback incoming, I intend to merge this PR by
tomorrow. If anybody intends to leave another review, please let me know
and I can postpone the merge. If nobody raises another issue, I will merge
tomorrow around 1:00 pm UTC
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#5046 (comment)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACKZTDK4ZCYRJH2ZFUDLKTUBZA7FANCNFSM5ADF3LHQ>
.
Triage notifications on the go with GitHub Mobile for iOS
<https://apps.apple.com/app/apple-store/id1477376905?ct=notification-email&mt=8&pt=524675>
or Android
<https://play.google.com/store/apps/details?id=com.github.android&referrer=utm_campaign%3Dnotification-email%26utm_medium%3Demail%26utm_source%3Dgithub>.
|
We discussed whether or not we want to merge this in todays dask maintainer meeting. Since this Friday is an upcoming ordinary release scheduled, we settled on delaying merging this PR until after the release. The delay will give us time to address follow up issues and should help us maintain stable releases. |
Merge conflicts from main resolved at crusaderky/worker_state_machine_refactor: |
Test failures appear to be unrelated. Thank you @crusaderky and @gforsyth for the review, I know this was a tough one. |
Whoa.
…On Mon, Sep 27, 2021 at 7:05 AM Florian Jetter ***@***.***> wrote:
Merged #5046 <#5046> into main.
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#5046 (comment)>, or
unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACKZTEFWYMAG3DFS32WEGDUEBMY5ANCNFSM5ADF3LHQ>
.
Triage notifications on the go with GitHub Mobile for iOS
<https://apps.apple.com/app/apple-store/id1477376905?ct=notification-email&mt=8&pt=524675>
or Android
<https://play.google.com/store/apps/details?id=com.github.android&referrer=utm_campaign%3Dnotification-email%26utm_medium%3Demail%26utm_source%3Dgithub>.
|
Wow. Nice work! This seems to be a bit more than a refactor, but it's a lot for me to take in. Are there some expected gains, as well? (I regularly need to go in and to kill GKE workloads that get stuck, and wonder if this may help me.) |
That's our hope. We hope this will add to stability, in paticular with deadlocks. There are a few other benefits in here
I've been debugging such stuck workloads for the past >6 months and patched many deadlocks / race conditions. I ended up writing a script for extracting the cluster state once it is stuck, see #5068 If you are facing this situation again you might want to run that script and open an issue with the resulting dump. Best case, this helps me track down your deadlock and makes dask better for everyone. |
Thanks. I filed #5366 . My tests are showing several stalled graphs. I think it is worse than 2021.9.1, but these things are hard to pin down for certain as the cluster environments can be different (scaling up/down at different rates, or even different # of total nodes available). |
This refactors the worker state machine and copies the scheduler transition machine to a significant amount. There are a few changes in mechanics I want to highlight
acquire-replica
andremove-replica
which can be used to collect any kind of task, not merely tasks which are required as a dependency. this required some significant remodelling of the gather_dependency machinery, most notably since it can no longer be guaranteed that everygather_dep
call has acause
assigned to it. Therefore, for diagnostics, we'll apply some heuristics to infer one.data_needed
is no longer a simple list of keys of tasks which require data but rather a heap of keys to be fetched. They are sorted by their compute priority as a proxy to how urgently a worker needs a given taskfind_tasks
. For this a dedicated statemissing
has been introduced with appropriate transitionsflight
andexecuting
tasks. Releasing these tasks historically left either a thread or a async task dangling we lost track of. This has frequently been causing race conditions. Now, these tasks are no longer forgotten, released or removed but rather transitioned to an intermediary statecancelled
. This cancelled state will block any further transitions until the thread/async task finishes and transitions the task then to the state, the most recent stimulus requested.Worker.release_key
I highly encourage the immediate deprecation of theWorkerPlugin.release_key
hook. Very similar events can be intercepted using theWorkerPlugin.transition
hook with more context information. I added the relevant deprecation warnings alreadyCloses #5323
Closes #4413
Old Description
This is another attempt to refactor the worker state machine as drafted out in https://github.com//issues/4413The transition mechanism is 95% copied from the scheduler side with a few refinements. The state machine, for instance, allows to concatenate states with kwargs and it adds nicer exceptions. This is not necessarily a functional enhancement but should rather help out with tracing if anything goes wrong
More importantly, as we discussed over in #4982 (comment) , our gather_dep mechanism did not work for "tasks without dependents" since every fetched task expects there to be a dependents. On main this is implemented by putting every task-to-be-executed into
data_needed
such that we'll fetch the dependencies for those tasks.This PR now puts all tasks in "fetch" state into this
data_needed
explicitly instead of working indirectly via the dependents. Thedata_needed
is kept sorted using the ordinary priorities which should resemble the previous "fetch dependencies for tasks we want to compute now first". I ran some performance tests on shuffle workloads and couldn't detect any difference. Will try to test a few other workloads but so far I don't see any regressions.I will leave a few dedicated comment in the code with the relevant logic.
For the
remove_replica
, I would suggest to refactor the free_keys, superfluous_data, release_key methods to be less complicated in a follow up PRcc @mrocklin @crusaderky
TODOs (ignore them; these are mental notes for myself):