Allow tasks with restrictions to be stolen#2740
Allow tasks with restrictions to be stolen#2740calebho wants to merge 30 commits intodask:masterfrom
Conversation
Addresses stealing tasks with resource restrictions, as mentioned in dask#1851. If a task has hard restrictions, do not just give up on stealing. Instead, use the restrictions to determine which workers can steal it before attempting to execute a steal operation. A follow up PR will be needed to address the issue of long-running tasks not being stolen because the scheduler has no information about their runtime.
|
@TomAugspurger, would you have a chance to look at this PR? |
…dask#2733) Convert resource key toples to a string representation before they are submitted to the scheduler. The commit is intended to fix dask#2716. The test case persists the result of a tiny DataFrame operation and checks the resource restrictions.
TomAugspurger
left a comment
There was a problem hiding this comment.
Looks good, but I'm not very familiar with this. May want a second opinion from @mrocklin when he has time.
| .pytest_cache/ | ||
| dask-worker-space/ | ||
| .vscode/ | ||
| *.swp |
There was a problem hiding this comment.
This seems fine since we include it for other files, but I typically put these files under a global gitignore.
distributed/stealing.py
Outdated
| ``from_``. | ||
| """ | ||
| if what.loose_restrictions: | ||
| logger.debug("Task {} has loose restrictions".format(what.key)) |
There was a problem hiding this comment.
Typically logging uses old style string formatting, and the values are passed to logger.debug, .e.g
logger.debug("Task %s has loose restrictions", what.key)
distributed/stealing.py
Outdated
| return out | ||
|
|
||
|
|
||
| def _can_steal(ws, what, from_): |
There was a problem hiding this comment.
So this is going to be called relatively often. AFAICT, it'll be called once per task, thief-worker pair for every task we're considering stealing. Given that, we should ensure that it's fast. How confident are you that there aren't any surprising performance issues here? Most things look OK, but may want to look into get_address_host.
get_address_host
There was a problem hiding this comment.
In general, get_address_host is linear in the length of the input address, however I think it's safe to assume that len(ws.address) is small and bounded as a function of ws.
The four collections used here are
what.host_restrictions,what.worker_restrictions,ws.resources, andfrom_.resources.
Key containment is checked using (1) and (2), and since they are sets these operations are constant with respect to the size of the containers. (3) and (4) are used in _has_resources. The work in the loop only uses (3). Because (3) is a dict, value retrieval is constant with respect to the size of (3). The loop iterates over (4) once, so the total work in this function is linear in the size of (4). So overall, _can_steal is O(len(from_.resources)).
There was a problem hiding this comment.
Yeah, I'm also concerned about the performance here.
This code gets called often. Previously we didn't iterate through idle, we just indexed into it (indeed we made it a SortedSet object just to get this constant-time access).
The work stealing code accounts for a non-trivial amount of the scheduler performance costs. I would not be surprised if adding this check significantly increases those costs. Given that resources aren't used in the common case I'm very hesitant to go this route without strong demonstration that it doesn't affect performance.
However, I would be much more comfortable with this if it was only engaged in the uncommon case that the task had resource constraints. It could be that a simple test like the following resolves the issue.
if ts.resources:
theives = [...]
theif = ...
else:
thief = self.idle[i * len(self.idle)]Then this PR is essentially a no-op in the common case, and we can be a lot more relaxed about performance.
There was a problem hiding this comment.
def _can_steal(ws, what, from_):I like the use of the pronoun ws for a worker state. I think that if what is a task state then we should probably use the ts pronoun instead. It's not easy to see what what means otherwise without reading the docstring carefully. If ws and from_ are both worker states then I recommend the names thief and victim instead.
There was a problem hiding this comment.
It seems like we're going to call this function a lot and it also seems like a lot of it might be pretty repetitive. Do you have any thoughts on how to reduce repetition here?
As an example, lets imagine that we have 1000 workers and 10,000,000 tasks. I think that we call balance every 100ms and we'd like stealing to take up less than 10% of our CPU scheduler overhead. My guess is that we don't currently stay under budget when there are restrictions, and my guess is that a lot of it has to do with this function. I think that it would be good to take a hard look at this function and reduce costs. Some of this might be through things like caching, but I'm not sure of the best approach here.
|
@calebho, any thoughts on Tom and Matt's suggestions above? 🙂 |
|
So I was investigating another issue related to work stealing where all tasks were running on single worker leaving all other workers idle. Here's a demonstrative example: import time
from distributed import Client
from dask_jobqueue import SLURMCluster
from tempfile import TemporaryDirectory
import os
def long_identity(i):
time.sleep(15)
return i
def main():
world_size = 5
ntasks = 5
with TemporaryDirectory(dir=os.getcwd()) as local_dir:
print(local_dir)
cluster = SLURMCluster(
name='testing',
queue='dev',
cores=10, # set to at most `ntasks` to observe work stealing
memory='4GB',
local_directory=local_dir,
log_directory=local_dir,
extra=['--resources "TASKS=1"'],
silence_logs=False,
)
cluster.adapt(minimum=world_size, maximum=world_size)
cl = Client(cluster)
futs = []
for i in range(ntasks):
futs.append(cl.submit(long_identity, i, resources={'TASKS': 1}))
for f in futs:
f.result()
if __name__ == '__main__':
main()The interesting thing about this example is if you use more cores than there are tasks, then no work stealing occurs. If you set cores to at most This gave me another idea for how to solve the current issue which would also handle the problem above; what if |
* bokeh -> dashboard in template
* Add doc to ProfileTimePlot
* Add test for bokeh worker routes
* Remove info route from worker
To do this we ...
1. Remove the baked in "info" link in the base template
2. Add that to the scheduler's list of links
3. Add a redirect from "info" to the actual page
4. Create a generic redirect route
5. Move that and the RequestHandler to utils to avoid code duplication
between scheduler and worker
Fixes dask#2722
* Add worker name
Fixes dask/dask#4878
* Add type name to LocalCluster.__repr__ * Add SpecCluster.new_worker_spec method This is helpful for subclassing
* Move some of the adaptive logic into the scheduler * don't close closed clusters * require pytest >= 4 in CI * use worker_spec if it exists * Don't scale a closed cluster * handle intermittent failures
This test generated rapidsai/ucx-py#120
This also fixes an ImportError in prometheus-client=0.7
mrocklin
left a comment
There was a problem hiding this comment.
Ah, I had a review sitting here for a few days. I apologize for the delay in response.
distributed/stealing.py
Outdated
| if not _has_restrictions(what): | ||
| logger.debug( | ||
| "Task %s has loose restrictions or no restrictions at all", what.key | ||
| ) |
There was a problem hiding this comment.
I think that this causes a logging event for every non-matching worker for every task at every stealing balance call, is that correct? If so, we can't do this, both because it will flood logs, and because the call to logger.debug can be non-trivially expensive.
distributed/stealing.py
Outdated
| return out | ||
|
|
||
|
|
||
| def _can_steal(ws, what, from_): |
There was a problem hiding this comment.
def _can_steal(ws, what, from_):I like the use of the pronoun ws for a worker state. I think that if what is a task state then we should probably use the ts pronoun instead. It's not easy to see what what means otherwise without reading the docstring carefully. If ws and from_ are both worker states then I recommend the names thief and victim instead.
distributed/stealing.py
Outdated
| return out | ||
|
|
||
|
|
||
| def _can_steal(ws, what, from_): |
There was a problem hiding this comment.
It seems like we're going to call this function a lot and it also seems like a lot of it might be pretty repetitive. Do you have any thoughts on how to reduce repetition here?
As an example, lets imagine that we have 1000 workers and 10,000,000 tasks. I think that we call balance every 100ms and we'd like stealing to take up less than 10% of our CPU scheduler overhead. My guess is that we don't currently stay under budget when there are restrictions, and my guess is that a lot of it has to do with this function. I think that it would be good to take a hard look at this function and reduce costs. Some of this might be through things like caching, but I'm not sure of the best approach here.
This allows for setting the config after importing distributed xref dask/dask-examples#75 (comment)
Co-Authored-By: Matthew Rocklin <mrocklin@gmail.com>
If this is the concern, then I don't think Looking at this if-else block, one way to get around this cost might be to sample a constant number of workers from idle instead of using the entire population. The issue then becomes how to construct an appropriate distribution over the idle workers such that
|
This changes Worker / Nanny startup to raise when they timeout. This bubbles up to the `dask-worker` CLI. Closes dask#2781
* Add version option to scheduler and worker CLI * Add version to other cli commands * Add tests
|
@mrocklin , @TomAugspurger , I am not following this discussion, but the OP seems to have replied to at least some concerns - did you want more specific information or work? |
|
ah , @calebho , there seems to be a conflict now - sorry about that. Can you please merge from master? |
Addresses stealing tasks with resource restrictions, as mentioned in dask#1851. If a task has hard restrictions, do not just give up on stealing. Instead, use the restrictions to determine which workers can steal it before attempting to execute a steal operation. A follow up PR will be needed to address the issue of long-running tasks not being stolen because the scheduler has no information about their runtime.
Co-Authored-By: Matthew Rocklin <mrocklin@gmail.com>
…tributed into resource_restricted_stealing
|
Oops I might have inadvertently polluted this PR with a bunch of commits from upstream because I rebased. I guess I can remake the branch on my fork? |
|
Sure, start a new PR, if more convenient. |
Addresses stealing tasks with resource restrictions, as mentioned in #1851. If a task has hard restrictions, do not just give up on stealing. Instead, use the restrictions to determine which workers can steal it before attempting to execute a steal operation. A follow up PR will be needed to address the issue of long-running tasks not being stolen because the scheduler has no information about their runtime. Supercedes #2740
Addresses stealing tasks with resource restrictions, as mentioned in #1851.
If a task has hard restrictions, do not just give up on stealing.
Instead, use the restrictions to determine which workers can steal it
before attempting to execute a steal operation.
A follow up PR will be needed to address the issue of long-running tasks
not being stolen because the scheduler has no information about their
runtime.