Skip to content

test_computation_object_code_dask_compute fails when using --reruns #9053

Open
@penguinpee

Description

@penguinpee

Describe the issue:

Trying to tackle flaky tests, I added --reruns 3 to pytest. That causes test_computation_object_code_dask_compute to fail consistently, while it happily succeeds without --reruns/

Minimal Complete Verifiable Example:

python3 -m venv .venv
. .venv/bin/activate
pip install --prefer-binary .
pip install --prefer-binary pytest pytest-rerunfailures pytest-repeat pytest-anyio pytest-timeout numpy
python3 -m pytest -k 'test_computation_object_code_dask_compute' --reruns 3 --timeout_method=signal distributed/tests/test_client.py

Result:

=================================================================================================================================== FAILURES ===================================================================================================================================
__________________________________________________________________________________________________________________ test_computation_object_code_dask_compute ___________________________________________________________________________________________________________________

client = <Client: 'tcp://127.0.0.1:36961' processes=2 threads=2, memory=62.50 GiB>

    def test_computation_object_code_dask_compute(client):
        pytest.importorskip("numpy")
        da = pytest.importorskip("dask.array")
    
        with dask.config.set({"distributed.diagnostics.computations.nframes": 2}):
            x = da.ones((10, 10), chunks=(3, 3))
            x.sum().compute()
    
        test_function_code = inspect.getsource(test_computation_object_code_dask_compute)
    
        def fetch_comp_code(dask_scheduler):
            computations = list(dask_scheduler.computations)
            assert len(computations) == 1
            comp = computations[0]
            assert len(comp.code) == 1
            return comp.code[0]
    
        code = client.run_on_scheduler(fetch_comp_code)
>       assert len(code) == 1
E       assert 2 == 1
E        +  where 2 = len((SourceCode(code='def pytest_runtest_protocol(item, nextitem):\n    """\n    Run the test protocol.\n\n    Note: when teardown fails, two reports are generated for the case, one for\n    the test case and the other for the teardown error.\n    """\n    reruns = get_reruns_count(item)\n    if reruns is None:\n        # global setting is not specified, and this test is not marked with\n        # flaky\n        return\n\n    # while this doesn\'t need to be run with every item, it will fail on the\n    # first item if necessary\n    check_options(item.session.config)\n    delay = get_reruns_delay(item)\n    parallel = not is_master(item.config)\n    db = item.session.config.failures_db\n    item.execution_count = db.get_test_failures(item.nodeid)\n    db.set_test_reruns(item.nodeid, reruns)\n\n    if item.execution_count > reruns:\n        return True\n\n    need_to_run = True\n    while need_to_run:\n        item.execution_count += 1\n        item.ihook.pytest_runtest_logstart(nodeid=item.nodeid, location=item.location)\n        reports = runtestprotocol(item, nextitem=nextitem, log=False)\n\n        for report in reports:  # 3 reports: setup, call, teardown\n            report.rer...trigger rerun\n        else:\n            need_to_run = False\n\n        item.ihook.pytest_runtest_logfinish(nodeid=item.nodeid, location=item.location)\n\n    return True\n', lineno_frame=566, lineno_relative=29, filename='/home/sandro/devel/distributed/.py313/lib64/python3.13/site-packages/pytest_rerunfailures.py'), SourceCode(code='def test_computation_object_code_dask_compute(client):\n    pytest.importorskip("numpy")\n    da = pytest.importorskip("dask.array")\n\n    with dask.config.set({"distributed.diagnostics.computations.nframes": 2}):\n        x = da.ones((10, 10), chunks=(3, 3))\n        x.sum().compute()\n\n    test_function_code = inspect.getsource(test_computation_object_code_dask_compute)\n\n    def fetch_comp_code(dask_scheduler):\n        computations = list(dask_scheduler.computations)\n        assert len(computations) == 1\n        comp = computations[0]\n        assert len(comp.code) == 1\n        return comp.code[0]\n\n    code = client.run_on_scheduler(fetch_comp_code)\n    assert len(code) == 1\n    assert code[0].code == test_function_code\n', lineno_frame=7097, lineno_relative=6, filename='/home/sandro/devel/distributed/distributed/tests/test_client.py')))

distributed/tests/test_client.py:7109: AssertionError
Captured stderr ``` ---------------------------------------------------------------------------------------------------------------------------- Captured stderr setup ----------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:43,121 - distributed.http.proxy - INFO - To route to workers diagnostics web server please install jupyter-server-proxy: python -m pip install jupyter-server-proxy 2025-04-24 10:26:43,123 - distributed.scheduler - INFO - State start 2025-04-24 10:26:43,124 - distributed.diskutils - INFO - Found stale lock file and directory '/tmp/dask-scratch-space/worker-tzsmibgz', purging 2025-04-24 10:26:43,124 - distributed.diskutils - INFO - Found stale lock file and directory '/tmp/dask-scratch-space/worker-lfyt4erv', purging 2025-04-24 10:26:43,124 - distributed.diskutils - INFO - Found stale lock file and directory '/tmp/dask-scratch-space/scheduler-f9_4ejpi', purging 2025-04-24 10:26:43,127 - distributed.scheduler - INFO - Scheduler at: tcp://127.0.0.1:42365 2025-04-24 10:26:43,127 - distributed.scheduler - INFO - dashboard at: http://127.0.0.1:8787/status 2025-04-24 10:26:43,128 - distributed.scheduler - INFO - Registering Worker plugin shuffle 2025-04-24 10:26:43,136 - distributed.worker - INFO - Start worker at: tcp://127.0.0.1:46343 2025-04-24 10:26:43,136 - distributed.worker - INFO - Start worker at: tcp://127.0.0.1:43109 2025-04-24 10:26:43,136 - distributed.worker - INFO - Listening to: tcp://127.0.0.1:46343 2025-04-24 10:26:43,136 - distributed.worker - INFO - Listening to: tcp://127.0.0.1:43109 2025-04-24 10:26:43,136 - distributed.worker - INFO - dashboard at: 127.0.0.1:35993 2025-04-24 10:26:43,136 - distributed.worker - INFO - dashboard at: 127.0.0.1:44437 2025-04-24 10:26:43,136 - distributed.worker - INFO - Waiting to connect to: tcp://127.0.0.1:42365 2025-04-24 10:26:43,136 - distributed.worker - INFO - Waiting to connect to: tcp://127.0.0.1:42365 2025-04-24 10:26:43,136 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:43,136 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:43,136 - distributed.worker - INFO - Threads: 1 2025-04-24 10:26:43,136 - distributed.worker - INFO - Memory: 31.25 GiB 2025-04-24 10:26:43,137 - distributed.worker - INFO - Local Directory: /tmp/dask-scratch-space/worker-4j9bvzpq 2025-04-24 10:26:43,136 - distributed.worker - INFO - Threads: 1 2025-04-24 10:26:43,137 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:43,137 - distributed.worker - INFO - Memory: 31.25 GiB 2025-04-24 10:26:43,137 - distributed.worker - INFO - Local Directory: /tmp/dask-scratch-space/worker-jxc_37zx 2025-04-24 10:26:43,137 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:43,148 - distributed.scheduler - INFO - Register worker addr: tcp://127.0.0.1:46343 name: tcp://127.0.0.1:46343 2025-04-24 10:26:43,158 - distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:46343 2025-04-24 10:26:43,158 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:52356 2025-04-24 10:26:43,158 - distributed.worker - INFO - Starting Worker plugin shuffle 2025-04-24 10:26:43,158 - distributed.scheduler - INFO - Register worker addr: tcp://127.0.0.1:43109 name: tcp://127.0.0.1:43109 2025-04-24 10:26:43,159 - distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:43109 2025-04-24 10:26:43,159 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:52344 2025-04-24 10:26:43,159 - distributed.worker - INFO - Registered to: tcp://127.0.0.1:42365 2025-04-24 10:26:43,159 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:43,159 - distributed.worker - INFO - Starting Worker plugin shuffle 2025-04-24 10:26:43,159 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:42365 2025-04-24 10:26:43,160 - distributed.worker - INFO - Registered to: tcp://127.0.0.1:42365 2025-04-24 10:26:43,160 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:43,160 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:42365 2025-04-24 10:26:43,174 - distributed.scheduler - INFO - Receive client connection: Client-da5d4b8b-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:43,175 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:52364 ----------------------------------------------------------------------------------------------------------------------------- Captured stderr call ----------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:43,284 - distributed.worker - INFO - Run out-of-band function 'fetch_comp_code' --------------------------------------------------------------------------------------------------------------------------- Captured stderr teardown --------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:43,473 - distributed.scheduler - INFO - Remove client Client-da5d4b8b-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:43,473 - distributed.core - INFO - Received 'close-stream' from tcp://127.0.0.1:52364; closing. 2025-04-24 10:26:43,473 - distributed.scheduler - INFO - Remove client Client-da5d4b8b-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:43,474 - distributed.scheduler - INFO - Close client connection: Client-da5d4b8b-20e5-11f0-aa73-001b21b55f90 ---------------------------------------------------------------------------------------------------------------------------- Captured stderr setup ----------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:44,344 - distributed.http.proxy - INFO - To route to workers diagnostics web server please install jupyter-server-proxy: python -m pip install jupyter-server-proxy 2025-04-24 10:26:44,346 - distributed.scheduler - INFO - State start 2025-04-24 10:26:44,347 - distributed.diskutils - INFO - Found stale lock file and directory '/tmp/dask-scratch-space/worker-jxc_37zx', purging 2025-04-24 10:26:44,347 - distributed.diskutils - INFO - Found stale lock file and directory '/tmp/dask-scratch-space/worker-4j9bvzpq', purging 2025-04-24 10:26:44,347 - distributed.diskutils - INFO - Found stale lock file and directory '/tmp/dask-scratch-space/scheduler-xwuxja5u', purging 2025-04-24 10:26:44,350 - distributed.scheduler - INFO - Scheduler at: tcp://127.0.0.1:46339 2025-04-24 10:26:44,350 - distributed.scheduler - INFO - dashboard at: http://127.0.0.1:8787/status 2025-04-24 10:26:44,350 - distributed.scheduler - INFO - Registering Worker plugin shuffle 2025-04-24 10:26:44,359 - distributed.worker - INFO - Start worker at: tcp://127.0.0.1:44907 2025-04-24 10:26:44,359 - distributed.worker - INFO - Start worker at: tcp://127.0.0.1:33333 2025-04-24 10:26:44,359 - distributed.worker - INFO - Listening to: tcp://127.0.0.1:44907 2025-04-24 10:26:44,359 - distributed.worker - INFO - Listening to: tcp://127.0.0.1:33333 2025-04-24 10:26:44,359 - distributed.worker - INFO - dashboard at: 127.0.0.1:43623 2025-04-24 10:26:44,359 - distributed.worker - INFO - dashboard at: 127.0.0.1:38429 2025-04-24 10:26:44,359 - distributed.worker - INFO - Waiting to connect to: tcp://127.0.0.1:46339 2025-04-24 10:26:44,359 - distributed.worker - INFO - Waiting to connect to: tcp://127.0.0.1:46339 2025-04-24 10:26:44,359 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:44,359 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:44,359 - distributed.worker - INFO - Threads: 1 2025-04-24 10:26:44,359 - distributed.worker - INFO - Threads: 1 2025-04-24 10:26:44,359 - distributed.worker - INFO - Memory: 31.25 GiB 2025-04-24 10:26:44,359 - distributed.worker - INFO - Memory: 31.25 GiB 2025-04-24 10:26:44,359 - distributed.worker - INFO - Local Directory: /tmp/dask-scratch-space/worker-eibbq7dx 2025-04-24 10:26:44,359 - distributed.worker - INFO - Local Directory: /tmp/dask-scratch-space/worker-kbvedq5a 2025-04-24 10:26:44,359 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:44,359 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:44,377 - distributed.scheduler - INFO - Register worker addr: tcp://127.0.0.1:33333 name: tcp://127.0.0.1:33333 2025-04-24 10:26:44,388 - distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:33333 2025-04-24 10:26:44,388 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:32978 2025-04-24 10:26:44,388 - distributed.worker - INFO - Starting Worker plugin shuffle 2025-04-24 10:26:44,388 - distributed.scheduler - INFO - Register worker addr: tcp://127.0.0.1:44907 name: tcp://127.0.0.1:44907 2025-04-24 10:26:44,388 - distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:44907 2025-04-24 10:26:44,388 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:32962 2025-04-24 10:26:44,389 - distributed.worker - INFO - Registered to: tcp://127.0.0.1:46339 2025-04-24 10:26:44,389 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:44,389 - distributed.worker - INFO - Starting Worker plugin shuffle 2025-04-24 10:26:44,389 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:46339 2025-04-24 10:26:44,390 - distributed.worker - INFO - Registered to: tcp://127.0.0.1:46339 2025-04-24 10:26:44,390 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:44,390 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:46339 2025-04-24 10:26:44,396 - distributed.scheduler - INFO - Receive client connection: Client-db190959-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:44,396 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:33000 ----------------------------------------------------------------------------------------------------------------------------- Captured stderr call ----------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:44,490 - distributed.worker - INFO - Run out-of-band function 'fetch_comp_code' --------------------------------------------------------------------------------------------------------------------------- Captured stderr teardown --------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:44,722 - distributed.scheduler - INFO - Remove client Client-db190959-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:44,722 - distributed.core - INFO - Received 'close-stream' from tcp://127.0.0.1:33000; closing. 2025-04-24 10:26:44,722 - distributed.scheduler - INFO - Remove client Client-db190959-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:44,723 - distributed.scheduler - INFO - Close client connection: Client-db190959-20e5-11f0-aa73-001b21b55f90 ---------------------------------------------------------------------------------------------------------------------------- Captured stderr setup ----------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:45,476 - distributed.http.proxy - INFO - To route to workers diagnostics web server please install jupyter-server-proxy: python -m pip install jupyter-server-proxy 2025-04-24 10:26:45,478 - distributed.scheduler - INFO - State start 2025-04-24 10:26:45,478 - distributed.diskutils - INFO - Found stale lock file and directory '/tmp/dask-scratch-space/worker-kbvedq5a', purging 2025-04-24 10:26:45,479 - distributed.diskutils - INFO - Found stale lock file and directory '/tmp/dask-scratch-space/worker-eibbq7dx', purging 2025-04-24 10:26:45,479 - distributed.diskutils - INFO - Found stale lock file and directory '/tmp/dask-scratch-space/scheduler-dur89vvy', purging 2025-04-24 10:26:45,482 - distributed.scheduler - INFO - Scheduler at: tcp://127.0.0.1:33855 2025-04-24 10:26:45,482 - distributed.scheduler - INFO - dashboard at: http://127.0.0.1:8787/status 2025-04-24 10:26:45,482 - distributed.scheduler - INFO - Registering Worker plugin shuffle 2025-04-24 10:26:45,490 - distributed.worker - INFO - Start worker at: tcp://127.0.0.1:35251 2025-04-24 10:26:45,490 - distributed.worker - INFO - Listening to: tcp://127.0.0.1:35251 2025-04-24 10:26:45,491 - distributed.worker - INFO - dashboard at: 127.0.0.1:42763 2025-04-24 10:26:45,491 - distributed.worker - INFO - Waiting to connect to: tcp://127.0.0.1:33855 2025-04-24 10:26:45,491 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:45,491 - distributed.worker - INFO - Start worker at: tcp://127.0.0.1:45177 2025-04-24 10:26:45,491 - distributed.worker - INFO - Listening to: tcp://127.0.0.1:45177 2025-04-24 10:26:45,491 - distributed.worker - INFO - Threads: 1 2025-04-24 10:26:45,491 - distributed.worker - INFO - dashboard at: 127.0.0.1:42995 2025-04-24 10:26:45,491 - distributed.worker - INFO - Memory: 31.25 GiB 2025-04-24 10:26:45,491 - distributed.worker - INFO - Waiting to connect to: tcp://127.0.0.1:33855 2025-04-24 10:26:45,491 - distributed.worker - INFO - Local Directory: /tmp/dask-scratch-space/worker-z0nwhcy9 2025-04-24 10:26:45,491 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:45,491 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:45,491 - distributed.worker - INFO - Threads: 1 2025-04-24 10:26:45,491 - distributed.worker - INFO - Memory: 31.25 GiB 2025-04-24 10:26:45,491 - distributed.worker - INFO - Local Directory: /tmp/dask-scratch-space/worker-di_bf6gq 2025-04-24 10:26:45,491 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:45,502 - distributed.scheduler - INFO - Register worker addr: tcp://127.0.0.1:35251 name: tcp://127.0.0.1:35251 2025-04-24 10:26:45,512 - distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:35251 2025-04-24 10:26:45,512 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:60796 2025-04-24 10:26:45,512 - distributed.scheduler - INFO - Register worker addr: tcp://127.0.0.1:45177 name: tcp://127.0.0.1:45177 2025-04-24 10:26:45,512 - distributed.worker - INFO - Starting Worker plugin shuffle 2025-04-24 10:26:45,513 - distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:45177 2025-04-24 10:26:45,513 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:60800 2025-04-24 10:26:45,513 - distributed.worker - INFO - Registered to: tcp://127.0.0.1:33855 2025-04-24 10:26:45,513 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:45,513 - distributed.worker - INFO - Starting Worker plugin shuffle 2025-04-24 10:26:45,514 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:33855 2025-04-24 10:26:45,514 - distributed.worker - INFO - Registered to: tcp://127.0.0.1:33855 2025-04-24 10:26:45,514 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:45,514 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:33855 2025-04-24 10:26:45,519 - distributed.scheduler - INFO - Receive client connection: Client-dbc47271-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:45,520 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:60814 ----------------------------------------------------------------------------------------------------------------------------- Captured stderr call ----------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:45,610 - distributed.worker - INFO - Run out-of-band function 'fetch_comp_code' --------------------------------------------------------------------------------------------------------------------------- Captured stderr teardown --------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:45,798 - distributed.scheduler - INFO - Remove client Client-dbc47271-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:45,798 - distributed.core - INFO - Received 'close-stream' from tcp://127.0.0.1:60814; closing. 2025-04-24 10:26:45,798 - distributed.scheduler - INFO - Remove client Client-dbc47271-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:45,799 - distributed.scheduler - INFO - Close client connection: Client-dbc47271-20e5-11f0-aa73-001b21b55f90 ---------------------------------------------------------------------------------------------------------------------------- Captured stderr setup ----------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:46,582 - distributed.http.proxy - INFO - To route to workers diagnostics web server please install jupyter-server-proxy: python -m pip install jupyter-server-proxy 2025-04-24 10:26:46,585 - distributed.scheduler - INFO - State start 2025-04-24 10:26:46,586 - distributed.diskutils - INFO - Found stale lock file and directory '/tmp/dask-scratch-space/worker-di_bf6gq', purging 2025-04-24 10:26:46,586 - distributed.diskutils - INFO - Found stale lock file and directory '/tmp/dask-scratch-space/worker-z0nwhcy9', purging 2025-04-24 10:26:46,587 - distributed.diskutils - INFO - Found stale lock file and directory '/tmp/dask-scratch-space/scheduler-57jkg1po', purging 2025-04-24 10:26:46,590 - distributed.scheduler - INFO - Scheduler at: tcp://127.0.0.1:36961 2025-04-24 10:26:46,590 - distributed.scheduler - INFO - dashboard at: http://127.0.0.1:8787/status 2025-04-24 10:26:46,590 - distributed.scheduler - INFO - Registering Worker plugin shuffle 2025-04-24 10:26:46,598 - distributed.worker - INFO - Start worker at: tcp://127.0.0.1:33097 2025-04-24 10:26:46,599 - distributed.worker - INFO - Listening to: tcp://127.0.0.1:33097 2025-04-24 10:26:46,599 - distributed.worker - INFO - dashboard at: 127.0.0.1:40115 2025-04-24 10:26:46,599 - distributed.worker - INFO - Waiting to connect to: tcp://127.0.0.1:36961 2025-04-24 10:26:46,599 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:46,599 - distributed.worker - INFO - Threads: 1 2025-04-24 10:26:46,599 - distributed.worker - INFO - Memory: 31.25 GiB 2025-04-24 10:26:46,599 - distributed.worker - INFO - Local Directory: /tmp/dask-scratch-space/worker-4l1nzkog 2025-04-24 10:26:46,599 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:46,611 - distributed.scheduler - INFO - Register worker addr: tcp://127.0.0.1:33097 name: tcp://127.0.0.1:33097 2025-04-24 10:26:46,622 - distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:33097 2025-04-24 10:26:46,622 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:52864 2025-04-24 10:26:46,623 - distributed.worker - INFO - Starting Worker plugin shuffle 2025-04-24 10:26:46,623 - distributed.worker - INFO - Registered to: tcp://127.0.0.1:36961 2025-04-24 10:26:46,623 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:46,624 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:36961 2025-04-24 10:26:46,631 - distributed.worker - INFO - Start worker at: tcp://127.0.0.1:42741 2025-04-24 10:26:46,631 - distributed.worker - INFO - Listening to: tcp://127.0.0.1:42741 2025-04-24 10:26:46,631 - distributed.worker - INFO - dashboard at: 127.0.0.1:36897 2025-04-24 10:26:46,631 - distributed.worker - INFO - Waiting to connect to: tcp://127.0.0.1:36961 2025-04-24 10:26:46,631 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:46,631 - distributed.worker - INFO - Threads: 1 2025-04-24 10:26:46,631 - distributed.worker - INFO - Memory: 31.25 GiB 2025-04-24 10:26:46,631 - distributed.worker - INFO - Local Directory: /tmp/dask-scratch-space/worker-ccybxm06 2025-04-24 10:26:46,631 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:46,643 - distributed.scheduler - INFO - Register worker addr: tcp://127.0.0.1:42741 name: tcp://127.0.0.1:42741 2025-04-24 10:26:46,643 - distributed.scheduler - INFO - Starting worker compute stream, tcp://127.0.0.1:42741 2025-04-24 10:26:46,643 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:52870 2025-04-24 10:26:46,644 - distributed.worker - INFO - Starting Worker plugin shuffle 2025-04-24 10:26:46,644 - distributed.worker - INFO - Registered to: tcp://127.0.0.1:36961 2025-04-24 10:26:46,644 - distributed.worker - INFO - ------------------------------------------------- 2025-04-24 10:26:46,645 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:36961 2025-04-24 10:26:46,649 - distributed.scheduler - INFO - Receive client connection: Client-dc70ee17-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:46,650 - distributed.core - INFO - Starting established connection to tcp://127.0.0.1:52888 ----------------------------------------------------------------------------------------------------------------------------- Captured stderr call ----------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:46,726 - distributed.worker - INFO - Run out-of-band function 'fetch_comp_code' --------------------------------------------------------------------------------------------------------------------------- Captured stderr teardown --------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:43,473 - distributed.scheduler - INFO - Remove client Client-da5d4b8b-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:43,473 - distributed.core - INFO - Received 'close-stream' from tcp://127.0.0.1:52364; closing. 2025-04-24 10:26:43,473 - distributed.scheduler - INFO - Remove client Client-da5d4b8b-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:43,474 - distributed.scheduler - INFO - Close client connection: Client-da5d4b8b-20e5-11f0-aa73-001b21b55f90 --------------------------------------------------------------------------------------------------------------------------- Captured stderr teardown --------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:44,722 - distributed.scheduler - INFO - Remove client Client-db190959-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:44,722 - distributed.core - INFO - Received 'close-stream' from tcp://127.0.0.1:33000; closing. 2025-04-24 10:26:44,722 - distributed.scheduler - INFO - Remove client Client-db190959-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:44,723 - distributed.scheduler - INFO - Close client connection: Client-db190959-20e5-11f0-aa73-001b21b55f90 --------------------------------------------------------------------------------------------------------------------------- Captured stderr teardown --------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:45,798 - distributed.scheduler - INFO - Remove client Client-dbc47271-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:45,798 - distributed.core - INFO - Received 'close-stream' from tcp://127.0.0.1:60814; closing. 2025-04-24 10:26:45,798 - distributed.scheduler - INFO - Remove client Client-dbc47271-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:45,799 - distributed.scheduler - INFO - Close client connection: Client-dbc47271-20e5-11f0-aa73-001b21b55f90 --------------------------------------------------------------------------------------------------------------------------- Captured stderr teardown --------------------------------------------------------------------------------------------------------------------------- 2025-04-24 10:26:46,924 - distributed.scheduler - INFO - Remove client Client-dc70ee17-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:46,925 - distributed.core - INFO - Received 'close-stream' from tcp://127.0.0.1:52888; closing. 2025-04-24 10:26:46,925 - distributed.scheduler - INFO - Remove client Client-dc70ee17-20e5-11f0-aa73-001b21b55f90 2025-04-24 10:26:46,925 - distributed.scheduler - INFO - Close client connection: Client-dc70ee17-20e5-11f0-aa73-001b21b55f90 ```

Anything else we need to know?:

Rerunning tackles the flaky tests we've seen.

Environment:

  • Dask version: 2025.4.0
  • Python version: 3.13.2
  • Operating System: Fedora 41
  • Install method (conda, pip, source): source

Metadata

Metadata

Assignees

No one assigned

    Labels

    flaky testIntermittent failures on CI.

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions