-
-
Notifications
You must be signed in to change notification settings - Fork 722
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
Behavior change in Future / task resolution for Client.submit with complicated arguments #8998
Comments
Oh, the title at #8920 (Remove recursion in task spec) looks pretty darn relevant. |
Two probably important points I missed earlier:
is necessary to reproduce the failure. If you just have a non-perisisted dask Array there's no issue. The difference is that the task graph of the persisted array contains a
This gives a slightly simpler reproducer: import dask
from distributed import LocalCluster
import dask.array as da
import numpy as np
def main():
with LocalCluster(n_workers=1) as cluster:
with cluster.get_client() as client:
arr1 = da.random.uniform(size=(10,), chunks=(10,))
arr2 = arr1.persist()
a = dask.delayed([arr1.to_delayed().flatten().tolist()]).compute()
b = dask.delayed([arr2.to_delayed().flatten().tolist()]).compute()
assert isinstance(a[0][0], np.ndarray), a # OK
assert isinstance(b[0][0], np.ndarray), b # error
if __name__ == "__main__":
main() which fails with Traceback (most recent call last):
File "/home/nfs/toaugspurger/gh/dask/distributed/bug2.py", line 22, in <module>
main()
File "/home/nfs/toaugspurger/gh/dask/distributed/bug2.py", line 17, in main
assert isinstance(b[0][0], np.ndarray), b # error
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
AssertionError: [[('uniform-f325b9030be3fffa906d1695ac839cc1', 0)]] |
This comment has been minimized.
This comment has been minimized.
I've traced things to where we call Going into that call, we have a dictionary with the
That's because we reach see that the (converted) task is an I think the problem, though, is that we don't ever have the target of the alias in the task graph, since no one else is putting it there. At this point, I might need some help from @fjetter or someone else familiar with this. |
Describe the issue:
I'm debugging a behavior change somewhere between distributed 2024.11.2 and 2024.12.1 that's affecting xgboost.
In https://github.com/dmlc/xgboost/blob/a46585a36c4bf30bfd58a2653fe8ae40beea25ce/python-package/xgboost/dask/__init__.py#L358-L488, there's some pretty complicated logic for laying splitting references to data.
I'm still trying to understand that code, but in the meantime here's a minimal(?) reproducer:
Minimal Complete Verifiable Example:
That completes fine with
dask[complete]==2024.11.2
. Withdask[complete]==2024.12.1
that raises with an assertion error on the worker insidefunc_boxed
:Previously, the worker got the data that's wrapped in the future (after unboxing several levels), so that
func_boxed
got an ndarray. Now it gets a task key.Anything else we need to know?:
I'm still working to understand that xgboost code. Seeing all the Delayed in Delayed objects makes me think there might be room for simplification. In the meantime, I wanted to check if this was expected. The fact that we're getting a task key like
(uniform-4fe..., 0, 0)
makes me think it's not deliberate, but I know that theTask
stuff has been under some flux.The snippet includes a commented-out block that calls
client.submit(func_plain, arg)
where the argument is just aFuture[ndarray]
. I'm not sure, but the change in behavior might be in the handling ofclient.compute
on this complicated data, rather thanclient.submit
.Environment:
The text was updated successfully, but these errors were encountered: