-
-
Notifications
You must be signed in to change notification settings - Fork 727
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 resources is deleted after call to publish_dataset on the task future since 2023.4.0 #7859
Comments
I can reproduce and I think I triaged the issue down to #7564 . I don't have a lot of time to figure out what's going on . I'm not very familiar with this area of the code but shouldn't the tasks on scheduler have the annotations not the worker ? |
Indeed. The relevant annotations are on scheduler side. However, afaik we're just forwarding the scheduler annotations so I assume something broke |
Would using In [6]: client.run(lambda dask_worker: dask_worker.state.total_resources)
Out[6]: {'tcp://127.0.0.1:57326': {'GPU': 2.0}}
In [7]: client.run(lambda dask_worker: dask_worker.state.available_resources)
Out[7]: {'tcp://127.0.0.1:57326': {'GPU': 2.0}} |
Thanks for your answer @quasiben . So I tried with what you propose, and the exact same issue arises. |
But then this is still an issue and for us that means we cannot upgrade dask anymore since we do run computations in the background. |
I was now testing with distributed==2023.7.0 and it got worse. Now even the first test fails when the fixture of the dask_client is tearing down. I have a feeling this might have to do with #6527, was this expected? |
Sorry for the slow response time. I'm struggling a bit to understand what your problems is about. Is this about |
@fjetter sorry was off for some time and I missed your answer. yes the problem is that the task annotations disappear from the worker as soon as I use async def test_submit_future_with_resources(dask_client: Client):
future = dask_client.submit(_retrieve_annotations, resources=RESOURCES)
assert future
coro = future.result()
assert isinstance(coro, Coroutine)
assert await coro == {"resources": RESOURCES} # THIS WORKS
async def test_submit_future_with_resources_and_published_dataset(dask_client: Client):
future = dask_client.submit(_retrieve_annotations, resources=RESOURCES)
assert future
await dask_client.publish_dataset(future, name="myfuture") # CALLING THIS MAKES THE ANNOTATIONS IN WORKER DISAPPEAR
coro = future.result()
assert isinstance(coro, Coroutine)
assert await coro == {"resources": RESOURCES} # THIS DOES NOT WORK Yes After some thinking I guess I can also manually pass the resources as an argument of the function instead. then I do not rely on the Worker code anymore. But I guess that is not the intention of that code. |
Hey @fjetter, sorry to bother again but is there some way I can help in solving that issue? I and my company cannot upgrade dask anymore because of it since 2023.3. |
@fjetter @quasiben I created a test inside of the distributed repository for you to demonstrate the issue: # distributed/tests/test_publish_dataset_issue.py
from distributed.client import Client
from distributed.scheduler import Scheduler
from distributed.utils_test import gen_cluster
from distributed.worker import get_worker
def _retrieve_annotations() -> None:
print("starting task")
worker = get_worker()
task = worker.state.tasks.get(worker.get_current_task())
print("finished task")
return task.annotations
RESOURCES = {"CPU": 1.0, "RAM": 123423}
@gen_cluster(client=True, worker_kwargs={"resources":{"CPU": 2, "RAM": 48e9}})
async def test_submit(c: Client, s: Scheduler, a, b):
future = c.submit(_retrieve_annotations, resources=RESOURCES, pure=False)
await c.publish_dataset(future, name="thefailing")
assert await c.list_datasets() == ("thefailing",)
result = await future.result()
assert result == {"resources": RESOURCES}
|
Sorry for the silence. We've been busy with the latest release since that changed a lot. The reproducer was quite helpful and I was able to track the issue down and fix it, see #8577 It's a subtle race condition. As you already noticed yourself, if you sleep briefly before the publish, everything works as expected |
@fjetter thank you! that is great and I'm looking forward to seeing this in the next release and test it! |
Describe the issue:
After upgrading dask[distributed] from 2023.3.x to the latest 2023.5.0 I detected in my tests that the behavior of task resources changed.
In my system one can start a computational task, logout and check later the status of the computation. To this end I heavily rely on the dask client
submit
andpublish_dataset
methods. From version 2023.4.0 it seems that publishing the dataset actually has the side effect to also completely remove the task defined resources (e.g. CPU, RAM, etc) and therefore breaking my code.Am I using a feature that was not supposed to work? I see inthe changelog of 2023.4 that there were it seems big changes with the dask-scheduler needing similar "software and hardware". is there a link with this sentence?
Thanks a lot for the great work with dask, it's an awesome platform!
Minimal Complete Verifiable Example:
run using the following code
Anything else we need to know?:
using
pip install dask[distributed]==2023.3
will make the test run correctly, wherepip install dask[distributed]==2023.4
will fail the second test (same with 2023.5)Environment:
Dask version: 2023.3.x vs 2023.4.x,2023.5.x
Python version: 3.10.10
Operating System: ubuntu 22.04, 20.04, windows 11
Install method (conda, pip, source): pip
The text was updated successfully, but these errors were encountered: