Skip to content

Input objects to client.submit are serialized multiple times after bump to 2025.4.0 #9054

@fcharras

Description

@fcharras

Describe the issue:

After bump to 2025.4.0 releases, some inputs to client.submit are serialized several times, where only one serialization used to occur with 2025.3.0.

More context: this caused a joblib unit test to fail. The test checks that task inputs are serialized an expected number of time when using dask backend. For reference, here is the relevant test and here's an example of failing pipeline. But instead the following snippet reproduce the issue in a simpler way.

Minimal Complete Verifiable Example:

from dask.distributed import Client, LocalCluster

if __name__ == "__main__":
    cluster = LocalCluster()
    client = Client(cluster)

    class CountSerialized:
        """Simple wrapper on an object that counts the number of times it's
        serialized."""

        def __init__(self, x):
            self.x = x
            self.count = 0

        def __reduce__(self):
            self.count += 1
            return (CountSerialized, (self.x,))

    def task(foo, bar):
        """Some dummy task on CountSerialized objects."""
        return foo.x == bar.x == 1

    def func_(task_and_args):
        f, args = task_and_args
        return f(*args)

    x = CountSerialized(1)
    task_and_args = task, [x, x]
    task = lambda: func_(task_and_args)
    result = client.submit(task).result()

    # output with 2025.3.0: 4
    # output with 2025.4.0: 6
    print(x.count)

Anything else we need to know?:

I did some superficial inspection, and it seems the related changes are the addition of the LLGExpr in the body of client.submit . There are several serializations occuring in LLGExpr, and additional ones in the _graph_to_futures ones.

The issue seems to only appear in callables where the x is hidden in several layers of sub functions and variables, like shown in the reproducer. In joblib, the issue appears because of the commonly used delayed helper function.

(edit: I am also a bit surprised that the print with 2025.3.0 shows "4" rather than "1" there ?)

Environment:

  • Dask version: 2025.4.0
  • Python version: tested in 3.12.10 and 3.10.17
  • Operating System: linux
  • Install method (conda, pip, source): tested with conda and pip

Metadata

Metadata

Assignees

No one assigned

    Labels

    No labels
    No labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions