Skip to content
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

Race condition in scatter->dereference->scatter #8576

Open
rikardn opened this issue Mar 13, 2024 · 11 comments
Open

Race condition in scatter->dereference->scatter #8576

rikardn opened this issue Mar 13, 2024 · 11 comments
Labels
bug Something is broken p2 Affects more than a few users but doesn't prevent core functions

Comments

@rikardn
Copy link

rikardn commented Mar 13, 2024

Triage summary

If you scatter a key, dereference the returned future, and then scatter again the same key (with the same value), the release of the first future is likely to reach the scheduler after the transition to memory of the second one and you'll end up holding a Future to a forgotten task. This in turn will cause your computation to fail.

Reproducer below: #8576 (comment)

Original post

I am using dask/distributed to run custom workflows. A workflow that used to work with versions 2024.2.0 stopped working with 2024.2.1 (and still does not work with 2024.3.0). I have been unable to create a minimal reproducible example, but I have some information that could give clues to what could be the problem.

The setup is that a static workflow, run on a LocalCluster with threads, is calling two dynamic workflows (see the graphs below) and many of the objects are scattered. I think that the problem is that the scattered object with id ModelEntry-d3c014c28f9af6108cba2a6c960688ce is the same for both dynamic workflows. Using distributed 2024.2.0 (or earlier) have different ids for this object even though it is the same object. Also if telling scatter to not use hash the workflow will run, I guess because now the ids will be different. Not scattering the object will also work. Given the log messages (see below) it seems as if we are losing the ModelEntry-d3c014c28f9af6108cba2a6c960688ce in the second dynamic workflow and it seems to only happen when the scatter-id is the same in the two workflows.

The stacktrace:

tests/integration/test_ruvsearch.py:47: in test_ruvsearch_plain
    res = run_tool('ruvsearch', model=model, results=results, groups=4, p_value=0.05, skip=[])
.tox/py311-integration/lib/python3.11/site-packages/pharmpy/tools/run.py:169: in run_tool
    return run_tool_with_name(name, tool, args, kwargs)
.tox/py311-integration/lib/python3.11/site-packages/pharmpy/tools/run.py:207: in run_tool_with_name
    res = execute_workflow(wf, dispatcher=dispatcher, database=tool_database)
.tox/py311-integration/lib/python3.11/site-packages/pharmpy/workflows/execute.py:73: in execute_workflow
    res: T = dispatcher.run(workflow)
.tox/py311-integration/lib/python3.11/site-packages/pharmpy/workflows/dispatchers/local_dask.py:84: in run
    res = client.get(dsk_optimized, 'results')
.tox/py311-integration/lib/python3.11/site-packages/distributed/client.py:3290: in get
    results = self.gather(packed, asynchronous=asynchronous, direct=direct)
.tox/py311-integration/lib/python3.11/site-packages/distributed/client.py:2384: in gather
    return self.sync(
.tox/py311-integration/lib/python3.11/site-packages/pharmpy/tools/ruvsearch/tool.py:207: in start
    res, best_model_entry, selected_model_name = call_workflow(
.tox/py311-integration/lib/python3.11/site-packages/pharmpy/workflows/call.py:42: in call_workflow
    res: T = client.gather(futures)  # pyright: ignore [reportGeneralTypeIssues]
.tox/py311-integration/lib/python3.11/site-packages/distributed/client.py:2245: in _gather
    raise exc
E   concurrent.futures._base.CancelledError: results1

The log:

INFO     distributed.http.proxy:proxy.py:71 To route to workers diagnostics web server please install jupyter-server-proxy: python -m pip install jupyter-server-proxy
INFO     distributed.scheduler:scheduler.py:1709 State start
INFO     distributed.scheduler:scheduler.py:4060   Scheduler at: inproc://11.22.33.44/15219/1
INFO     distributed.scheduler:scheduler.py:4075   dashboard at:  http://11.22.33.44:31058/status
INFO     distributed.scheduler:scheduler.py:7682 Registering Worker plugin shuffle
INFO     distributed.worker:worker.py:1442       Start worker at: inproc://11.22.33.44/15219/4
INFO     distributed.worker:worker.py:1443          Listening to:       inproc11.22.33.44
INFO     distributed.worker:worker.py:1446           Worker name:                          0
INFO     distributed.worker:worker.py:1448          dashboard at:       11.22.33.44:32973
INFO     distributed.worker:worker.py:1449 Waiting to connect to: inproc://11.22.33.44/15219/1
INFO     distributed.worker:worker.py:1450 -------------------------------------------------
INFO     distributed.worker:worker.py:1451               Threads:                          8
INFO     distributed.worker:worker.py:1453                Memory:                  31.19 GiB
INFO     distributed.worker:worker.py:1457       Local Directory: /tmp/tmpblhs_4s0/dask-scratch-space/worker-tpk9rqpo
INFO     distributed.worker:worker.py:1170 -------------------------------------------------
INFO     distributed.scheduler:scheduler.py:4412 Register worker <WorkerState 'inproc://11.22.33.44/15219/4', name: 0, status: init, memory: 0, processing: 0>
INFO     distributed.scheduler:scheduler.py:5898 Starting worker compute stream, inproc://11.22.33.44/15219/4
INFO     distributed.core:core.py:1019 Starting established connection to inproc://11.22.33.44/15219/5
INFO     distributed.worker:worker.py:1873 Starting Worker plugin shuffle
INFO     distributed.worker:worker.py:1237         Registered to: inproc://11.22.33.44/15219/1
INFO     distributed.worker:worker.py:1238 -------------------------------------------------
INFO     distributed.core:core.py:1019 Starting established connection to inproc://11.22.33.44/15219/1
INFO     distributed.scheduler:scheduler.py:5656 Receive client connection: Client-4a2d1674-e11a-11ee-bb73-9319feeedaca
INFO     distributed.core:core.py:1019 Starting established connection to inproc://11.22.33.44/15219/6
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, start_iteration-3f5dcf5b-57b7-4d18-ab6b-dce3697aa7d4
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, create_base_model-2e21b800-0274-4d54-abaa-d01bdd02bbae
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, run0-44d21a45-7334-4544-b681-88ca358e5b1c
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, results1
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, create_power_model-run2-bd5cdfb5-2b19-44f7-a6fb-370a9f31cc01
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, create_iiv_on_ruv_model-run1-7b47d7cb-9eb0-4b75-9e96-143b57a37937
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, create_combined_error_model-run3-1492ffa5-700b-4cbf-8a7a-6671f3633915
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, create_time_varying_model1-run4-13c1f90e-b67f-47ec-8f1b-d1fcb952b64f
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, create_time_varying_model2-run5-0e29c005-1db4-4154-8113-ed73ef4f787f
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, create_time_varying_model3-run6-515a0923-a5e6-4841-a408-7cf8133e4328
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, run1-7b47d7cb-9eb0-4b75-9e96-143b57a37937
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, run2-bd5cdfb5-2b19-44f7-a6fb-370a9f31cc01
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, run3-1492ffa5-700b-4cbf-8a7a-6671f3633915
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, run4-13c1f90e-b67f-47ec-8f1b-d1fcb952b64f
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, run5-0e29c005-1db4-4154-8113-ed73ef4f787f
INFO     distributed.scheduler:scheduler.py:4469 User asked for computation on lost data, run6-515a0923-a5e6-4841-a408-7cf8133e4328
WARNING  distributed.worker:worker.py:2343 Compute Failed
Key:       start_ruvsearch-results
Function:  execute_task
args:      ((<function _results at 0x7f28ae662ca0>, (<function start at 0x7f28ae662b60>, <pharmpy.workflows.tool_database.local_directory.LocalDirectoryToolDatabase objec
t at 0x7f28af874fd0>, <Pharmpy model object mox3>, <Pharmpy modelfit results object mox3>, 4, 0.05, [], 3, None, 'minimization_successful or (rounding_errors and sigdigs>
=0.1)')))
kwargs:    {}
Exception: "CancelledError('results1')"

INFO     distributed.scheduler:scheduler.py:5700 Remove client Client-4a2d1674-e11a-11ee-bb73-9319feeedaca
INFO     distributed.core:core.py:1044 Received 'close-stream' from inproc://11.22.33.44/15219/6; closing.
INFO     distributed.scheduler:scheduler.py:5700 Remove client Client-4a2d1674-e11a-11ee-bb73-9319feeedaca
INFO     distributed.scheduler:scheduler.py:5692 Close client connection: Client-4a2d1674-e11a-11ee-bb73-9319feeedaca
INFO     distributed.worker:worker.py:1546 Stopping worker at inproc://11.22.33.44/15219/4. Reason: worker-close
INFO     distributed.core:core.py:1044 Received 'close-stream' from inproc://11.22.33.44/15219/5; closing.
INFO     distributed.scheduler:scheduler.py:5199 Remove worker <WorkerState 'inproc://11.22.33.44/15219/4', name: 0, status: closing, memory: 0, processing: 0> (stimul
us_id='handle-worker-cleanup-1710321352.7051728')
INFO     distributed.scheduler:scheduler.py:5297 Lost all workers
INFO     distributed.core:core.py:1029 Connection to inproc://11.22.33.44/15219/1 has been closed.
INFO     distributed.scheduler:scheduler.py:4134 Scheduler closing due to unknown reason...
INFO     distributed.scheduler:scheduler.py:4152 Scheduler closing all comms

First dynamic workflow (works):

{'Convert_error_model': (<function _start_iteration at 0x7f28ae662a20>, ('ModelEntry-d3c014c28f9af6108cba2a6c960688ce',), {})}

Failing dynamic workflow (see first function call for the suspect scattered object):

{'start_iteration-3f5dcf5b-57b7-4d18-ab6b-dce3697aa7d4': (<function _start_iteration at 0x7f28ae662a20>, ('ModelEntry-d3c014c28f9af6108cba2a6c960688ce',), {}), 'create_ba
se_model-2e21b800-0274-4d54-abaa-d01bdd02bbae': (functools.partial(<function _create_base_model at 0x7f28ae662160>, current_iteration=1, dv=None), ('start_iteration-3f5dc
f5b-57b7-4d18-ab6b-dce3697aa7d4',), {}), 'run0-44d21a45-7334-4544-b681-88ca358e5b1c': (<function retrieve_from_database_or_execute_model_with_tool.<locals>.task at 0x7f28
a94ec720>, ('LocalDirectoryToolDatabase-679f1737d27257bd5df0f03d71eb8535', 'create_base_model-2e21b800-0274-4d54-abaa-d01bdd02bbae'), {}), 'run1-7b47d7cb-9eb0-4b75-9e96-1
43b57a37937': (<function execute_task at 0x7f28aba20680>, ('create_iiv_on_ruv_model-run1-7b47d7cb-9eb0-4b75-9e96-143b57a37937',), {}), 'run2-bd5cdfb5-2b19-44f7-a6fb-370a9
f31cc01': (<function execute_task at 0x7f28aba20680>, ('create_power_model-run2-bd5cdfb5-2b19-44f7-a6fb-370a9f31cc01',), {}), 'run3-1492ffa5-700b-4cbf-8a7a-6671f3633915':
 (<function execute_task at 0x7f28aba20680>, ('create_combined_error_model-run3-1492ffa5-700b-4cbf-8a7a-6671f3633915',), {}), 'run4-13c1f90e-b67f-47ec-8f1b-d1fcb952b64f':
 (<function execute_task at 0x7f28aba20680>, ('create_time_varying_model1-run4-13c1f90e-b67f-47ec-8f1b-d1fcb952b64f',), {}), 'run5-0e29c005-1db4-4154-8113-ed73ef4f787f': 
(<function execute_task at 0x7f28aba20680>, ('create_time_varying_model2-run5-0e29c005-1db4-4154-8113-ed73ef4f787f',), {}), 'run6-515a0923-a5e6-4841-a408-7cf8133e4328': (
<function execute_task at 0x7f28aba20680>, ('create_time_varying_model3-run6-515a0923-a5e6-4841-a408-7cf8133e4328',), {}), 'results1': (<function post_process at 0x7f28ae
662d40>, ('LocalDirectoryToolDatabase-679f1737d27257bd5df0f03d71eb8535', 3.8414588206941285, 1, None, 'start_iteration-3f5dcf5b-57b7-4d18-ab6b-dce3697aa7d4', 'run0-44d21a
45-7334-4544-b681-88ca358e5b1c', 'run1-7b47d7cb-9eb0-4b75-9e96-143b57a37937', 'run2-bd5cdfb5-2b19-44f7-a6fb-370a9f31cc01', 'run3-1492ffa5-700b-4cbf-8a7a-6671f3633915', 'r
un4-13c1f90e-b67f-47ec-8f1b-d1fcb952b64f', 'run5-0e29c005-1db4-4154-8113-ed73ef4f787f', 'run6-515a0923-a5e6-4841-a408-7cf8133e4328'), {}), 'create_power_model-run2-bd5cdf
b5-2b19-44f7-a6fb-370a9f31cc01': (<function execute_task at 0x7f28aba20680>, ((<function retrieve_from_database_or_execute_model_with_tool.<locals>.task at 0x7f28a94ec720
>, 'LocalDirectoryToolDatabase-679f1737d27257bd5df0f03d71eb8535', (functools.partial(<function _create_power_model at 0x7f28ae660fe0>, current_iteration=1, dv=None), 'cre
ate_base_model-2e21b800-0274-4d54-abaa-d01bdd02bbae')),), {}), 'create_iiv_on_ruv_model-run1-7b47d7cb-9eb0-4b75-9e96-143b57a37937': (<function execute_task at 0x7f28aba20
680>, ((<function retrieve_from_database_or_execute_model_with_tool.<locals>.task at 0x7f28a94ec720>, 'LocalDirectoryToolDatabase-679f1737d27257bd5df0f03d71eb8535', (func
tools.partial(<function _create_iiv_on_ruv_model at 0x7f28ae662c00>, current_iteration=1, dv=None), 'create_base_model-2e21b800-0274-4d54-abaa-d01bdd02bbae')),), {}), 'cr
eate_combined_error_model-run3-1492ffa5-700b-4cbf-8a7a-6671f3633915': (<function execute_task at 0x7f28aba20680>, ((<function retrieve_from_database_or_execute_model_with
_tool.<locals>.task at 0x7f28a94ec720>, 'LocalDirectoryToolDatabase-679f1737d27257bd5df0f03d71eb8535', (functools.partial(<function _create_combined_model at 0x7f28ae6619
40>, current_iteration=1), 'create_base_model-2e21b800-0274-4d54-abaa-d01bdd02bbae')),), {}), 'create_time_varying_model1-run4-13c1f90e-b67f-47ec-8f1b-d1fcb952b64f': (<fu
nction execute_task at 0x7f28aba20680>, ((<function retrieve_from_database_or_execute_model_with_tool.<locals>.task at 0x7f28a94ec720>, 'LocalDirectoryToolDatabase-679f17
37d27257bd5df0f03d71eb8535', (functools.partial(<function _create_time_varying_model at 0x7f28ae660d60>, groups=4, i=1, current_iteration=1, dv=None), 'create_base_model-
2e21b800-0274-4d54-abaa-d01bdd02bbae')),), {}), 'create_time_varying_model2-run5-0e29c005-1db4-4154-8113-ed73ef4f787f': (<function execute_task at 0x7f28aba20680>, ((<fun
ction retrieve_from_database_or_execute_model_with_tool.<locals>.task at 0x7f28a94ec720>, 'LocalDirectoryToolDatabase-679f1737d27257bd5df0f03d71eb8535', (functools.partia
l(<function _create_time_varying_model at 0x7f28ae660d60>, groups=4, i=2, current_iteration=1, dv=None), 'create_base_model-2e21b800-0274-4d54-abaa-d01bdd02bbae')),), {})
, 'create_time_varying_model3-run6-515a0923-a5e6-4841-a408-7cf8133e4328': (<function execute_task at 0x7f28aba20680>, ((<function retrieve_from_database_or_execute_model_
with_tool.<locals>.task at 0x7f28a94ec720>, 'LocalDirectoryToolDatabase-679f1737d27257bd5df0f03d71eb8535', (functools.partial(<function _create_time_varying_model at 0x7f
28ae660d60>, groups=4, i=3, current_iteration=1, dv=None), 'create_base_model-2e21b800-0274-4d54-abaa-d01bdd02bbae')),), {})}```
@fjetter
Copy link
Member

fjetter commented Mar 13, 2024

We've done some nontrivial refactoring in how we're hashing/tokenizing objects (dask/dask#10905)

I see this came up in an OSS project. Is it possible for you to link to the code this is running?

cc @crusaderky

@rikardn
Copy link
Author

rikardn commented Mar 13, 2024

Certainly. The problem is that the code is quite complex and the objects we use are also quite complex. Actually running this particular workflow currently requires a proprietary program. I was hoping to be able to create a smaller example, but wasn't able to. I could potentially do some debugging in dask/distributed myself if I knew where to start looking. Could it be that scattered objects gets removed after they are deemed not needed any longer?

The main github page: https://github.com/pharmpy/pharmpy
The call to scatter: https://github.com/pharmpy/pharmpy/blob/main/src/pharmpy/workflows/optimize.py
The call to gather: https://github.com/pharmpy/pharmpy/blob/main/src/pharmpy/workflows/call.py
Function that creates and calls the dynamic workflows: https://github.com/pharmpy/pharmpy/blob/b1323be4ae18652c9ca0efce40cde1b8237e5399/src/pharmpy/tools/ruvsearch/tool.py#L187
Creating the local cluster: https://github.com/pharmpy/pharmpy/blob/main/src/pharmpy/workflows/dispatchers/local_dask.py
Executing a workflow: https://github.com/pharmpy/pharmpy/blob/main/src/pharmpy/workflows/execute.py
The failed test: https://github.com/pharmpy/pharmpy/actions/runs/8260071687/job/22595069029

@fjetter
Copy link
Member

fjetter commented Mar 13, 2024

Could it be that scattered objects gets removed after they are deemed not needed any longer?

The lifetime of a scattered object is coupled to the Future instances. As soon as the Future is fully dereferenced, the cluster will release the data.
If at some point in your code you are switching from the Future instance to it's key and only retain the key, this would explain the situation.

@crusaderky
Copy link
Collaborator

Hi @rikardn,

First dynamic workflow (works):
Failing dynamic workflow (see first function call for the suspect scattered object):

What does "first" and "second" workflow mean? are they two keys inside the same dsk that end up summarized and retrieved at once through the results key? Or are they two iterative calls to run, each starting and destroying its own cluster?

In other words - could your algorithm be simplified down to this?

import distributed

with distributed.Client(processes=False) as client:
    dsk = {
        "results": (sum, ["first", "second"]), 
        "first": client.scatter(123), 
        "second": client.scatter(123),  # produces a future with the same key as first
    }
    print(client.get(dsk, "results"))

or to this?

with distributed.Client(processes=False) as client:
    dsk = {"results": client.scatter(123)}
    print("first", client.get(dsk, "results"))

with distributed.Client(processes=False) as client:
    dsk = {"results": client.scatter(123)}
    print("second", client.get(dsk, "results"))

@rikardn
Copy link
Author

rikardn commented Mar 13, 2024

The full workflow is something like:

def func():
    obj = create_object()
    dsk1 = create_workflow1(client.scatter(obj))
    res1 = run_dynamic_workflow(dsk1)
    dsk2 = create_workflow2(client.scatter(obj), res1)   # Note same object as before. New call to scatter.
    res2 = run_dynamic_workflow2(dsk2)
    return res2


def run_dynamic_workflow(dsk):
    client = get_client()
    futures = client.get(dsk, "result", sync=False)
    secede()
    T = client.gather(futures)
    rejoin()
    return res


with distributed.Client(processes=False) as client:
    dsk = {
        "results": (func), 
    }
    print(client.get(dsk, "results"))

@crusaderky
Copy link
Collaborator

Thanks @rikardn, this helps a lot.

An important nitpick though: did you accidentally omit

def func():
    client = get_client()

? In other words, do func (which scatters) and the two calls to run_dynamic_workflow call get_client() three times sequentially from the same thread?

Second important nit:

dsk2 = create_workflow2(client.scatter(obj), res1)

is res1 a computed result, as your pseudocode lets intend, or a Future to the output of the first workflow?

@rikardn
Copy link
Author

rikardn commented Mar 13, 2024

First:
No, that was not accidental. run_dynamic_workflow calls get_client for each dynamically created workflow to launch. So So in this case get_client is run twice.

Second:
Sorry, yes it is also scattered, I did't think it was important. It is only obj that has a problem being scattered since it is the only object being scattered twice to different workflows (or at least that's what I believe is the cause).

Third:
I added a third potentially important thing. So actually we are using LocalCluster and gets the client from it like this:

with LocalCluster(processes=False) as cluster, Client(cluster) as client:

Fourth:
An other thing that I omitted is that obj is actually passed through the first dynamic workflow without change.

An updated example for reference:

def func():
    obj = create_object()
    dsk1 = create_workflow1(client.scatter(obj))
    res1, new_obj = run_dynamic_workflow(dsk1)    # Note that new_obj and obj is the same object, i.e. having the same hash
    dsk2 = create_workflow2(client.scatter(new_obj), client.scatter(res1))  
    res2 = run_dynamic_workflow2(dsk2)
    return res2


def run_dynamic_workflow(dsk):
    client = get_client()
    futures = client.get(dsk, "result", sync=False)
    secede()
    res = client.gather(futures)
    rejoin()
    return res


with distributed.with LocalCluster(processes=False) as cluster, Client(cluster) as client:
    dsk = {
        "results": (func), 
    }
    print(client.get(dsk, "results"))

@rikardn
Copy link
Author

rikardn commented Mar 13, 2024

I realize my first point doesn't make sense. It is acutally the run_dynamic_workflow that goes through the dsk dict and scatters objects. But as my example stands there should be a call to get_client in func

@crusaderky
Copy link
Collaborator

Sorry, yes it is also scattered, I did't think it was important.

It isn't. What I meant was if res is actually just the raw output of client.get, e.g. distributed.Future objects.
If you're going through a gather -> scatter it's not important.

@crusaderky
Copy link
Collaborator

crusaderky commented Mar 13, 2024

Reproduced.

I'm willing to bet that your pseudocode is missing a detail: before you're calling create_workflow2, you're dereferencing dsk1.
My reproducer falls apart almost immediately:

from time import sleep
import distributed

client = distributed.Client(processes=False)

while True:
    print(".", end="")
    x = client.scatter(123)
    assert client.cluster.scheduler.tasks[x.key].state == "memory"
    del x
    # while client.cluster.scheduler.tasks:
    #     sleep(0.01)

output:

..........
---------------------------------------------------------------------------
KeyError                                  Traceback (most recent call last)
Cell In[37], line 9
      7 print(".", end="")
      8 x = client.scatter(123)
----> 9 assert client.cluster.scheduler.tasks[x.key].state == "memory"
     10 del x
     11 # while client.cluster.scheduler.tasks:
     12 #     sleep(0.01)

KeyError: 'int-4951b9977632a52fcd6f0cc65c57bb33'

if I uncomment the sleep after the deletion, it goes on indefinitely.

@crusaderky crusaderky changed the title Problem with scatter in distributed 2024.2.1 Race condition in scatter->release->scatter Mar 13, 2024
@crusaderky crusaderky added bug Something is broken p2 Affects more than a few users but doesn't prevent core functions and removed needs triage labels Mar 13, 2024
@crusaderky crusaderky changed the title Race condition in scatter->release->scatter Race condition in scatter->dereference->scatter Mar 13, 2024
@rikardn
Copy link
Author

rikardn commented Mar 13, 2024

@crusaderky Just wow! I am amazed that you from my messy information could figure this one out. Thanks!

I can confirm that this issue is also in dask/distributed 2024.2.0 and the reason it was triggered for me starting with 2024.2.1 was that the hash function in dask was changed so that it now gives the same hash for the objects that are the same (which is a good change). One potential workaround is to use client.scatter(123, hash=False) to get unique keys and avoid the race condition.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something is broken p2 Affects more than a few users but doesn't prevent core functions
Projects
None yet
Development

No branches or pull requests

3 participants