-
-
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
Task stealing regression in 2021-11-0+ (preventing task load balancing) #5564
Comments
Thanks @arnaudsj! I think this example is a little bit too minimized actually, so here's a different one:
import time
import distributed
if __name__ == "__main__":
def slow_function(x):
time.sleep(10)
with distributed.Client(n_workers=1, threads_per_worker=1) as client:
input(f"Open dashboard at {client.dashboard_link}, then press enter to start")
fs = client.map(slow_function, range(30))
time.sleep(1)
client.cluster.scale(10)
client.wait_for_workers(10)
print("Cluster has 10 workers now, tasks _should_ get balanced to the new workers, but don't")
distributed.wait(fs) If you watch the dashboard, you'll see that at first, all 30 slow tasks get assigned to the one worker in the cluster. It's not until one of the tasks finish that the tasks get load-balanced: Now, the tasks run on all workers: According to @arnaudsj, the regression here is that tasks used to get load-balanced even when none had completed yet. In his case, these slow tasks could take 5-10min, so having to wait that long with a cluster that's 99% idle before other workers pick up the work is costly. cc @fjetter, @crusaderky |
Thank you @gjoseph92 for making such a more detailed use case and better capturing what I was trying to report. Let me know if you need anything else! |
I don't think this is a regression. It has been a long known problem that we need to wait for a task to finish at least once before we get any measurements in. See #3516 There is a config option to give the scheduler a hint on how long an unknown task execution might take with There is also a config option I started a patch to get measurements in for not-yet-finished tasks here #4459 a while ago but didn't have the time to finish it up. That would likely allow us to get rid of all of this. @arnaudsj what makes you think this is a regression? Have you seen a different behaviour on another version? |
This was resolved at some point when we started sending the currently
executing tasks from the workers to the scheduler in the heartbeat. If any
of the currently executing tasks have an unknown task duration then we
tried to use the time that they had been executing in order to make
stealing decisions.
Or at least, I think that we did that. I did a quick search in github to
find an old PR that did that. I recommend using git blame to look at the
executing key in the heartbeat and see if there is an associated commit/PR.
…On Mon, Dec 6, 2021 at 4:35 AM Florian Jetter ***@***.***> wrote:
I don't think this is a regression. It has been a long known problem that
we need to wait for a task to finish at least once before we get any
measurements in. See #3516
<#3516>
There is a config option to give the scheduler a hint on how long an
unknown task execution might take, see #3642
<#3642>
I started a patch to get measurements in for not-yet-finished tasks here
#4459 <#4459> a while ago but
didn't have the time to finish it up
@arnaudsj <https://github.com/arnaudsj> what makes you think this is a
regression? Have you seen a different behaviour on another version?
—
You are receiving this because you are subscribed to this thread.
Reply to this email directly, view it on GitHub
<#5564 (comment)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACKZTEEY24TUUVXP4HQJHLUPSGW3ANCNFSM5JKY525A>
.
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>.
|
Yeah, prior to 2021-10-0 (worker state machine refactor) he saw tasks get stolen before any in the group had completed.
I suggested Interestingly, setting
I think this is the behavior we'd want, and why the current behavior feels unreasonable. If a task has been running for 2s, even if it's not done yet, we should guess that others will probably also take 2s, and balance accordingly. Based on title alone, it seems very likely to me that #5392 is where this behavior changed. |
Yeah, this fixes my reproducer: diff --git a/distributed/stealing.py b/distributed/stealing.py
index e5fa43d7..0acb1d60 100644
--- a/distributed/stealing.py
+++ b/distributed/stealing.py
@@ -177,7 +177,7 @@ class WorkStealing(SchedulerPlugin):
level: The location within a stealable list to place this value
"""
split = ts.prefix.name
- if split in fast_tasks or split in self.scheduler.unknown_durations:
+ if split in fast_tasks:
return None, None
if not ts.dependencies: # no dependencies fast path As a more reasonable change, perhaps once |
@gjoseph92 - I tested the workaround by setting @fjetter, yes, I confirmed that this behavior was not occurring with 2021.9.2 and prior (did not test to verify if 2021.10.x were still good or not). Our team switched to 2021.11.2 at the beginning of our Sprint last week, and that's when we noticed the change (which for our particular section of our ETL is quite noticeable as we have tasks loading lots of data into the worker's worker_data space before it can start running). |
Something doesn't add up. With the default |
@crusaderky I believe the problem is distributed/distributed/stealing.py Lines 180 to 181 in ababd21
The task is in |
I agree with @gjoseph92 that #5392 is likely the culprit. The interesting thing is that I classified as a regression since there were tests available asserting that tasks should not be allowed to be stolen. I traced back the test |
What happened: There seems to be an issue with the task scheduling. When the cluster starts, and the first worker grabs the first task, it appears to block the rest of the workers until it completes the first task. This is particularly an issue when large clusters are spun in the cloud, as the scheduler start to send tasks when not all the workers are ready. This minimum working example below also demonstrate how one of the workers appears to keep all the tasks assigned and only distribute them as needed throughout the length of the computation.
What you expected to happen: Workers should start processing tasks immediatly and be able to steal tasks from the 1st worker. This is appears to be a regresssion from pre dask-2021-11-0 (I confirmed for sure it works as expected in 2021-9-2)
Minimal Complete Verifiable Example:
Anything else we need to know?:
This is a minimized example of an issue our team ran into at scale on Coiled (cc @gjoseph92)
Environment:
The text was updated successfully, but these errors were encountered: