-
-
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
Priorities are ignored when stealing tasks #8992
Comments
Are you able to put together some kind of reproducible code example so that we can explore this behaviour? |
The code I use is quite convoluted. Give me some time to distil simplified code that reproduces the problem. |
If this is a bug in the scheduling/stealing behaviour you should be able to reproduce it with |
Hey, it took me a while, but I've got the code - it uses from dask.distributed import Client, LocalCluster
from dask.distributed import as_completed
import time
def my_func(t):
time.sleep(t)
return t
def run_cluster():
cluster = LocalCluster(n_workers=1, resources={'WIN10':1, 'CLOUD':1, 'DB':1})
client: Client = cluster.get_client()
try:
futures = []
for i in range(20):
futures.append(
client.submit(my_func, i, key=f"my_func{i}", priority=i, pure=False, resources={'WIN10':1, 'CLOUD':1, 'DB':1}
))
time.sleep(1)
client.cluster.scale(2)
for future, result in as_completed(futures, with_results=True):
print(f'Result {result}')
except:
pass
cluster.close()
if __name__ == '__main__':
run_cluster() The output:
|
I reproduced it using CLI/python (it's close to what I actually do in my project) too. Failing sequnce:
import asyncio
import time
from dask.distributed import Client
from dask.distributed import as_completed
def my_func(t):
print(f'sleep {t}')
time.sleep(t)
return t
async def main():
client = await Client(address='tcp://10.249.224.32:8786', name="client", asynchronous=True, timeout=180)
futures = []
for i in range(1,21):
futures.append(
client.submit(my_func, i, key=f"my_func{i}", priority=i, pure=False, resources={'WIN10':1, 'CLOUD':1, 'DB':1}
))
print('Ready for workers to register')
async for _, result in as_completed(futures, with_results=True):
print(f'Result {result}')
client.close()
result = asyncio.run(main())
First worker output:
Second worker output:
However, it worked correctly when I removed the
import asyncio
import time
from dask.distributed import Client
from dask.distributed import as_completed
def my_func(t):
print(f'sleep {t}')
time.sleep(t)
return t
async def main():
client = await Client(address='tcp://10.249.224.32:8786', name="client", asynchronous=True, timeout=180)
futures = []
for i in range(1,21):
futures.append(
client.submit(my_func, i, key=f"my_func{i}", priority=i, pure=False#, resources={'WIN10':1, 'CLOUD':1, 'DB':1}
))
print('Ready for workers to register')
async for _, result in as_completed(futures, with_results=True):
print(f'Result {result}')
client.close()
result = asyncio.run(main())
Output:
Second worker:
|
Describe the issue:
There are a few issues actually (two of them related to stealing), so let's go through the whole process.
I use dask and distributed packages to run automation in my project.
To speed up test execution (the tests cannot be run on a single environment in parallel), I spin up multiple VMs (dask workers) and then create a queue of tests (tasks) and assign them to workers until the tasks are depleted. Each task is unique and run once unless it failed (then it’s rerun).
This works perfectly but I noticed that using priorities may speed up execution significantly.
I tried to use dask priorities but ultimately failed. Here are my findings (the first issue is not related to the topic but is a problem nevertheless):
I would prefer assigning only one task per VM instead of stealing, but if I were really to use stealing, than it should grab the highest priority tasks that are not being processed (now it grabs random tasks)
To resolve the first problem, I set
distributed.scheduler.worker-saturation
to 1.0 but it didn’t help, all of the tasks are consumed at once.I tried using
fifo-timeout
to deal with the second one, but it wasn’t helpful (I thought that fifo-timeout is taken into consideration when stealing).The whole process looks as follows:
1.1. Register set up VMs as workers with cmd (nthreads=1).
2.1. Create a list of futures
2.2. Append (client.submit(test)) each test to the list of futures
2.3. Run as_completed to gather futures; if there are failed tests, readd them to the futures list
With the current behaviour I haven’t encountered a single situation where a lowest priority task was run last. My task lengths vary from between 20 seconds to 3 minutes; the whole runtime can increase significantly if the tasks are ordered improperly.
Example:
I tested it on a small scale (8 tasks, 2 VMs).
The priorities are equal to an estimated task runtime in seconds (actual runtime is about
priority
+ 40 seconds). That ensures that the slowest tasks have the highest priority.The first worker consumed all of the tasks immediately.
As soon as the second VM was created, it stole several tasks from the first worker.
Then the workers kept stealing work from each other.
Tasks on the second VM were run in random order.
After the first task was executed on the first VM, the other tasks (which were re-stolen from the second VM) were run in order they were stolen.
Environment:
Conclusion
What I only want to achieve here is to make prioritization work.
I am not sure if all of the work consumed by the first VM I create is due to a flaw in my setup process; if this can be fixed, then I could skip stealing altogether.
Still, priorities seem to be ignored when stealing tasks which is a real issue.
The text was updated successfully, but these errors were encountered: