Skip to content

Behavior change in Future / task resolution for Client.submit with complicated arguments #8998

Closed
@TomAugspurger

Description

@TomAugspurger

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:

import dask
from dask.delayed import Delayed
from distributed import Client, LocalCluster, wait
import dask.array as da
import numpy as np
import dask.array.core


def func_plain(x: np.ndarray) -> str:
    assert isinstance(x, np.ndarray), x
    return type(x).__name__


def func_boxed(x: list[np.ndarray]) -> str:
    assert isinstance(x[0], np.ndarray), x
    return type(x).__name__


def main():
    with LocalCluster(n_workers=1) as cluster:
        with Client(cluster) as client:
            arr = da.random.uniform(size=(10, 10), chunks=(10, 10))
            arr = arr.persist()
            delayed = arr.to_delayed().flatten().tolist()

            # # client.submit(func, arg) works fine for a `Delayed` arg.
            # delayed_plain = [dask.delayed(x) for x in delayed]
            # fut_plain = client.compute(delayed_plain)
            # wait(fut_plain)
            # result = client.submit(func_plain, fut_plain[0]).result()
            # print(f"{result=}")

            # Boxing the Delayed object in a list or a dict causes the error
            boxed_parts: list[dict[str, Delayed]] = [[o] for o in delayed]

            # list[Delayed[list[Delayed[np.ndarray]]]]
            delayed_boxed_parts = [dask.delayed(x) for x in boxed_parts]

            fut_boxed_parts = client.compute(delayed_boxed_parts)
            wait(fut_boxed_parts)
            result = client.submit(func_boxed, fut_boxed_parts[0]).result()
            print(f"{result=}")


if __name__ == "__main__":
    main()

That completes fine with dask[complete]==2024.11.2. With dask[complete]==2024.12.1 that raises with an assertion error on the worker inside func_boxed:

2025-01-31 07:16:20,171 - distributed.worker - ERROR - Compute Failed
Key:       func_boxed-b2f2ed77eb007708d94c86cdd1e0640c
State:     executing
Task:  <Task 'func_boxed-b2f2ed77eb007708d94c86cdd1e0640c' func_boxed(...)>
Exception: "AssertionError([('uniform-4fe800e9f8fbe6f6234e25b7c6f797e7', 0, 0)])"
Traceback: '  File "/home/nfs/toaugspurger/gh/dmlc/xgboost/debug.py", line 15, in func_boxed\n    assert isinstance(x[0], np.ndarray), x\n           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^\n'

Traceback (most recent call last):
  File "/home/nfs/toaugspurger/gh/dmlc/xgboost/debug.py", line 45, in <module>
    main()
  File "/home/nfs/toaugspurger/gh/dmlc/xgboost/debug.py", line 40, in main
    result = client.submit(func_boxed, fut_boxed_parts[0]).result()
             ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/raid/toaugspurger/envs/xgboost/lib/python3.12/site-packages/distributed/client.py", line 402, in result
    return self.client.sync(self._result, callback_timeout=timeout)
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/home/nfs/toaugspurger/gh/dmlc/xgboost/debug.py", line 15, in func_boxed
    assert isinstance(x[0], np.ndarray), x
      ^^^^^^^^^^^^^^^^^
AssertionError: [('uniform-4fe800e9f8fbe6f6234e25b7c6f797e7', 0, 0)]

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 the Task 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 a Future[ndarray]. I'm not sure, but the change in behavior might be in the handling of client.compute on this complicated data, rather than client.submit.

(Pdb) pp fut_plain[0]
<Future: finished, type: numpy.ndarray, key: ('uniform-48875bc65c093d89f0e4c416d5d471e5', 0, 0)>
(Pdb) pp fut_plain[0].result()[0, :2]
array([0.92037022, 0.86310934])
(Pdb) pp fut_boxed_parts[0]
<Future: finished, type: list, key: list-eaeba21a-5cce-4fbb-8b20-c3e4447988e2>
(Pdb) pp fut_boxed_parts[0].result()
[('uniform-48875bc65c093d89f0e4c416d5d471e5', 0, 0)]

Environment:

  • Dask version: the issue is reproducible with Dask 2024.12.1 through 2025.1.0. I haven't tested with main
  • Python version: 3.12
  • Operating System: linux
  • Install method (conda, pip, source): pip

Metadata

Metadata

Assignees

No one assigned

    Labels

    bugSomething is brokengood expert issueClearly described but requires someone extremely familiar with the project to implement successfullyhelp wantedregression

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions