Skip to content

RuntimeError: cannot schedule new futures after shutdown when using external Kubernetes cluster #707

Open
@Artimi

Description

@Artimi

Describe the issue:

Hi,

I'm encountering the following error when trying to run Dask jobs on an external Kubernetes cluster:

RuntimeError: cannot schedule new futures after shutdown

However, when I use a local Kubernetes cluster like rancher-desktop, everything works as expected without any RuntimeError.

I suspect that the issue might be related to the getaddrinfo call, which utilizes ThreadPoolExecutor for asynchronous operation. It seems that the default ThreadPoolExecutor might already be closed by the time it's called, leading to the error.

Minimal Complete Verifiable Example:

from dask_kubernetes.operator import KubeCluster
import time
import random
import joblib


def square(x):
    time.sleep(random.expovariate(1.5))
    return x**2


def main():
    cluster = KubeCluster(
        name="my-dask-cluster",
        image="ghcr.io/dask/dask:2023.3.2-py3.11",
        env={"EXTRA_PIP_PACKAGES": "joblib"},
    )
    print("Cluster created")
    cluster.scale(1)
    client = cluster.get_client()
    print("Client", client)
    joblib.parallel_backend(
        "dask", client=client, pure=False, wait_for_workers_timeout=60
    )

    results = joblib.Parallel(n_jobs=2)(
        joblib.delayed(square)(arg) for arg in range(10)
    )
    print(results)


if __name__ == "__main__":
    main()

This results in

$ python k8s_cluster.py
╭─────────────────── Creating KubeCluster 'my-dask-cluster' ───────────────────╮
│                                                                              │
│   DaskCluster                                                      Running   │
│   Scheduler Pod                                                    Running   │
│   Scheduler Service                                                Created   │
│   Default Worker Group                                             Created   │
│                                                                              │
│ ⠏ Getting dashboard URL                                                      │
╰──────────────────────────────────────────────────────────────────────────────╯
Cluster created
/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/distributed/client.py:1381: VersionMismatchWarning: Mismatched versions found

+-------------+----------------+----------------+---------+
| Package     | Client         | Scheduler      | Workers |
+-------------+----------------+----------------+---------+
| distributed | 2023.3.2.1     | 2023.3.2       | None    |
| lz4         | None           | 4.3.2          | None    |
| python      | 3.11.3.final.0 | 3.11.0.final.0 | None    |
+-------------+----------------+----------------+---------+
  warnings.warn(version_module.VersionMismatchWarning(msg[0]["warning"]))
Client <Client: 'tcp://10.18.148.16:8786' processes=0 threads=0, memory=0 B>
[0, 1, 4, 9, 16, 25, 36, 49, 64, 81]
Exception ignored in atexit callback: <function reap_clusters at 0x10cc33ba0>
Traceback (most recent call last):
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/dask_kubernetes/operator/kubecluster/kubecluster.py", line 1056, in reap_clusters
    asyncio.run(_reap_clusters())
  File "/opt/homebrew/Cellar/[email protected]/3.11.3/Frameworks/Python.framework/Versions/3.11/lib/python3.11/asyncio/runners.py", line 190, in run
    return runner.run(main)
           ^^^^^^^^^^^^^^^^
  File "/opt/homebrew/Cellar/[email protected]/3.11.3/Frameworks/Python.framework/Versions/3.11/lib/python3.11/asyncio/runners.py", line 118, in run
    return self._loop.run_until_complete(task)
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/opt/homebrew/Cellar/[email protected]/3.11.3/Frameworks/Python.framework/Versions/3.11/lib/python3.11/asyncio/base_events.py", line 653, in run_until_complete
    return future.result()
           ^^^^^^^^^^^^^^^
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/dask_kubernetes/operator/kubecluster/kubecluster.py", line 1054, in _reap_clusters
    cluster.close(timeout=10)
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/dask_kubernetes/operator/kubecluster/kubecluster.py", line 702, in close
    return self.sync(self._close, timeout=timeout)
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/distributed/utils.py", line 349, in sync
    return sync(
           ^^^^^
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/distributed/utils.py", line 416, in sync
    raise exc.with_traceback(tb)
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/distributed/utils.py", line 389, in f
    result = yield future
             ^^^^^^^^^^^^
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/tornado/gen.py", line 769, in run
    value = future.result()
            ^^^^^^^^^^^^^^^
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/dask_kubernetes/operator/kubecluster/kubecluster.py", line 710, in _close
    await custom_objects_api.delete_namespaced_custom_object(
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/kubernetes_asyncio/client/api_client.py", line 185, in __call_api
    response_data = await self.request(
                    ^^^^^^^^^^^^^^^^^^^
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/kubernetes_asyncio/client/rest.py", line 220, in DELETE
    return (await self.request("DELETE", url,
            ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/kubernetes_asyncio/client/rest.py", line 177, in request
    r = await self.pool_manager.request(**args)
        ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/aiohttp/client.py", line 536, in _request
    conn = await self._connector.connect(
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/aiohttp/connector.py", line 540, in connect
    proto = await self._create_connection(req, traces, timeout)
            ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/aiohttp/connector.py", line 901, in _create_connection
    _, proto = await self._create_direct_connection(req, traces, timeout)
               ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/aiohttp/connector.py", line 1152, in _create_direct_connection
    hosts = await asyncio.shield(host_resolved)
            ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/aiohttp/connector.py", line 874, in _resolve_host
    addrs = await self._resolver.resolve(host, port, family=self._family)
            ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/Artimi/playground/dask/.venv/lib/python3.11/site-packages/aiohttp/resolver.py", line 33, in resolve
    infos = await self._loop.getaddrinfo(
            ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/opt/homebrew/Cellar/[email protected]/3.11.3/Frameworks/Python.framework/Versions/3.11/lib/python3.11/asyncio/base_events.py", line 867, in getaddrinfo
    return await self.run_in_executor(
                 ^^^^^^^^^^^^^^^^^^^^^
  File "/opt/homebrew/Cellar/[email protected]/3.11.3/Frameworks/Python.framework/Versions/3.11/lib/python3.11/asyncio/base_events.py", line 829, in run_in_executor
    executor.submit(func, *args), loop=self)
    ^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/opt/homebrew/Cellar/[email protected]/3.11.3/Frameworks/Python.framework/Versions/3.11/lib/python3.11/concurrent/futures/thread.py", line 167, in submit
    raise RuntimeError('cannot schedule new futures after shutdown')
RuntimeError: cannot schedule new futures after shutdown

Anything else we need to know?:

Environment:

  • Dask version: 2023.3.2
  • Python version: 3.11
  • Operating System: MacOS - host, ghcr.io/dask/dask:2023.3.2-py3.11 - image
  • Install method (conda, pip, source): pip

Metadata

Metadata

Assignees

No one assigned

    Labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions