-
-
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 stuck in closing_gracefully state #6223
Comments
I hope that this is easy to resolve (we should be able to manage state in the In the meantime, a plugin like this might be helpful. class CloseUngracefully(WorkerPlugin):
def setup(self, worker):
self.worker = worker
self.count = 0
pc = PeriodicCallback(self.check, 1000)
pc.start()
def check(self):
if self.worker.status == Status.closing_gracefully:
self.count += 1
if self.count >= 10:
sys.exit(0) |
To be clear, the code above is not actually tested. It is likely wrong (but hopefully right directionally) |
I don't think this is the worker's fault. Having a sanity timeout so that Anyway, my guess is that there's a discrepancy between distributed/distributed/active_memory_manager.py Lines 662 to 667 in 198522b
and the cases in which the RetireWorker AMM policy removes itself from the AMM policies.
Because @crusaderky what do you think? It still feels odd to me that the criteria for diff --git a/distributed/active_memory_manager.py b/distributed/active_memory_manager.py
index 448a7581..b549d300 100644
--- a/distributed/active_memory_manager.py
+++ b/distributed/active_memory_manager.py
@@ -661,6 +661,8 @@ class RetireWorker(ActiveMemoryManagerPolicy):
def done(self) -> bool:
"""Return True if it is safe to close the worker down; False otherwise"""
+ if self not in self.manager.policies:
+ return True
ws = self.manager.scheduler.workers.get(self.address)
if ws is None:
return True Also, the obvious place to add a timeout would be in this
This might be a nice optimization (steal tasks immediately from workers that are closing gracefully, instead of waiting for them to close, and letting the normal remove_worker handling reassign the tasks). But if the tasks on the worker depend on data that's on the worker, having work-stealing replicating data around at the same time as a |
Should the scheduler immediately reschedule processing tasks on other workers? |
@gjoseph92 do you think it is doable to get a fix that would resolve this issue in by the release on Friday? |
Probably not. I discussed this in the last paragraph of my previous comment. This would only help for tasks that are queued on the worker that don't have any dependencies on the worker, which is rare enough it doesn't seem worth optimizing for right now. For tasks besides that, it adds more complexity and chaos.
@bnaul can you try with 2fa193a If it works, and @crusaderky is okay with this approach, I would be too. However, I've been trying to write a test to trigger this behavior and can't quite make it happen. If we can poke at a cluster that's doing this, it will be much faster. But if we can't figure out what the problem is or how to test it, then I'm not sure we could get a fix in by Friday. I think we can probably figure that out though, if we devote the time to it. To be clear: the fact that we see Adding a timeout to |
@bnaul it sounds like @gjoseph92 would benefit from access to a live cluster. When something next fails can I ask you to ping him? Gabe, for reference Brett pings me from time to time saying "I've got a failure if you want to come play" and I drop things and we screenshare. It's been effective so far if you're interruptable. |
Well, this could be pretty common if, for example, the worker had more ready tasks than threads.
I would be ok getting a fix in now without a dedicated test for it, and delaying the test for after the release if we have confirmation that it resolves the live issue. I think that there is significant value here. |
It sounds like this bandaid would stop the bleeding though, which I suspect would be highly appreciated by the folks that are currently bleeding out (like @bnaul). Bandaids aren't bad if they're good to have anyway. They buy us more time to think without stress. |
If those ready tasks depend on data that's on the worker, that data has to be transferred to a different worker before they can execute elsewhere. And that transferring is what
Let's see if the patch helps. From reading the logs and the code, I think there's a very limited place where this bug can be happening. If that's not actually true, then adding the timeout is pretty easy. |
Sounds good to me. In general I I like the approach of "Try to solve the deeper problem quickly. If we can't, see if there is a bandaid that we like, get that out quickly, and then refocus on the deeper problem for a longer period of time". I think that that approach balances short-term pain and long-term design. If you agree with that approach generally then I'm happy. Mostly I want to make sure that we don't spend a couple of weeks on the deeper problem leaving users feeling bad when we could stop the bleeding with an hour or two of work. I think I'm hearing you say something similar above. Let me know if you disagree in principle. |
This is a known issue: #3761
This is already implemented: distributed/distributed/active_memory_manager.py Lines 638 to 651 in 198522b
and distributed/distributed/scheduler.py Lines 6075 to 6082 in 198522b
The timeout here is implicit: if, while you're waiting indefinitely to copy your worker's data to apparently healthy recipients, you keep failing because they're unresponsive, then eventually the scheduler will notice (timeout) and remove them; at which point the above code will be triggered. This won't save you from deadlocked workers that are still sending the heartbeat. I don't think a recipient's deadlock should be dealt with a timeout though.
Good point, however not the right bit of code. distributed/distributed/active_memory_manager.py Lines 638 to 651 in 198522b
as it will trigger this response in the scheduler: distributed/distributed/scheduler.py Lines 6075 to 6082 in 198522b
This one however is vulnerable to a race condition:
What I think is happening: In theory, there could also be tasks in flight state (and before #6195, they could also be in fetch state). Before the next polling for done() happens, they reach the worker and become in memory AND the worker they received the data from loses them - so done() will return False. This last bit is not impossible, just very very unlikely. It should never be the AMM's fault - maybe the other worker was closed ungracefully? or maybe it had been unresponsive to the scheduler for several tens of seconds but for some weird reason managed to send one last message to its peer worker, and then the scheduler declared it dead. In theory, you could just remove that line. The policy will be uninstalled by
There is one caveat, which is if you submit this: futs = [c.submit(slowinc, 1, delay=0.5 + 2 * i, workers=[a.address]) for i in range(16)]
await c.retire_workers(a.address) and a has 16 threads, it will take 32.5s to shut down. In other words, if at least one task, which was already running when the worker transitioned from running to closing_gracefully, transitions to memory between each iteration of the AMM, then you'll have to wait until the next iteration. To fix it,
The problem is that all transfers could finish successfully in 0.1s, but the policy won't rerun before 2s, and the user may have set a much slower period for the AMM - say, 60s. That's the same reason why there's a polling with a sleep in retire_workers() and not just an asyncio.Event that is set when removing the policy.
What would a timeout protect you from? I do not agree that we should implement a timeout in
|
Thanks for the great explanation @crusaderky.
Just to note—this wasn't the case by default until very recently #6200. Another reason why having a TTL is important, but agreed it's not AMM's problem.
This totally makes sense. These are the cases I was looking for. It also makes sense with @bnaul's workload (which only has root tasks).
What about the fetch scenario? We could also traverse all the processing tasks and add their dependencies to this set? Something like gjoseph92@a176149? How much benefit do we get from leaving the Versus something simple like eae1bd4? |
As a stopgap: #6230 |
I have to admit that I didn't understand the AMM race condition wizardry above so maybe this is not helpful. Since this is about a worker closing, I would like to point to two related issues that can prohibit the closing of a worker. The latter would even prohibit us from introducing an asyncio timeout on worker side as proposed in (#6230)
I've been debugging these issues as part of |
Hrm, I'm not seeing the connection here yet. In practice what I saw with @bnaul 's setup was that there was nothing running in the ThreadPoolExecutor. The worker was quiet, and was just waiting on the scheduler to give it the "time to close" signal. The diff in that PR is bit wonky though, so I'll wait until the rebase and look again to see if I see what's going on. Also, to be clear, do you think that #6230 is a bad idea? |
Looking at the commit in #6091, yes, I think that that would be useful. It seems somewhat orthogonal to this topic though (but still a good and easy thing to fix). |
Not claiming for this to be the problem. I just wanted to raise awareness that there is something ongoing in this space. I haven't had enough time to familiarize myself with this specific problem to make a call.
I am reviewing. If there is actually a problem in the worker closing method, I am wondering how much this is helping. In this case only a timeout on scheduler side would help. I believe this is discussed in the above thread but I am struggling to follow the entire conversation right now |
FWIW I 100% agree we should have a reliable timeout for graceful shutdown |
Unnecessary. When an in-flight task lands and becomes in-memory, by definition there are now two replicas of it. So you won't waste any AMM iterations. In the weird edge case where the sender worker dies in the meantime, you are covered by not removing the policy - so it will just take an extra iteration of the AMM to deal with it. |
It lets you stop worrying about race conditions. To clarify, the ignorelist is a nice-to-have that makes the retirement faster in the edge case of a multitude of tasks that transition from running to memory at ~2 seconds interval.
This also works. |
Yes I think it's a bad idea - it adds to the complexity of the whole system, and it does not work when the retirement is initiated from the client or the scheduler, at all. |
This is my vote then. It's simple and easy to understand. And it should work just as well for the "multitude of tasks that transition from running to memory at ~2 seconds interval" scenario, right? (The worker will ignore them and retire immediately.) |
Yes, it does. +1 from me. |
I'll open a PR. |
(Note: @mrocklin heavily edited this issue, the words are his (sorry Brett to impersonate you) but the logs are coming from Brett)
We've found that workers can be stuck in a
closing_gracefully
state indefinitely. Here is some context:Cluster
We're running an adaptively scaled cluster of roughly 4000 workers.
Computation
Our workload is pretty simple. We're just reading a bunch of data, and then calling a map_partitions call and that's it. We see that the cluster scales up pretty fast, and then scales down pretty fast.
Some findings
Both the scheduler and the worker agree that it is in a closing gracefully state (this is after a long while)
Interestingly, they both also agree that there are three tasks that are ready to go. An artificial call to
Worker._ensure_computing
at this point doesn't do anything because the first check in that method isif self.status != Status.Running: return
.Here are the logs on the scheduler that refer to this worker.
And the events from the scheduler's perspective
And the logs on the worker side
Thoughts
My guess is that when entering a
closing_gracefully
state on the worker side we don't immediately clear out ready work, when maybe we should.Also, we should have some sanity checks so that
close_gracefully
can't go on for forever.cc @crusaderky @fjetter
The text was updated successfully, but these errors were encountered: