Skip to content

cleanup_lingering_clusters atexit fails to stop cluster: RuntimeError in DNS resolution after executor shutdown #930

@gabchx

Description

@gabchx

Describe the issue:

GatewayCluster's atexit handler cleanup_lingering_clusters fails to send the HTTP DELETE to the gateway server during interpreter shutdown. The cluster keeps running and billing continues until it times out or is manually stopped.

The root cause: CPython's concurrent.futures.thread._python_exit() atexit handler runs before dask-gateway's cleanup_lingering_clusters and kills all ThreadPoolExecutor instances while also setting a module-level _shutdown = True flag that blocks even new executors. When _stop_internalGateway._request()aiohttp.ClientSession.request() tries to resolve DNS via aiohttp's default ThreadedResolver, it calls loop.run_in_executor(None, socket.getaddrinfo, ...) on a dead executor → RuntimeError → DELETE never sent.

Minimal Complete Verifiable Example:

# Requires a running dask-gateway server — cannot be reproduced with LocalCluster.
from dask_gateway import GatewayCluster

if __name__ == "__main__":
    cluster = GatewayCluster(address="https://your-gateway.example.com")
    client = cluster.get_client()

    future = client.submit(lambda: 42)
    print(future.result())
    # Exit without cluster.close() — Ctrl+C or natural script end.
    # Expected: cleanup_lingering_clusters atexit sends DELETE, cluster stops.
    # Actual: RuntimeError in DNS resolution, DELETE never sent, cluster keeps running.

Anything else we need to know?:

The call chain is: cleanup_lingering_clusters (atexit) → GatewayCluster.close()_stop_internalGateway._stop_cluster(name)Gateway._request("DELETE", url)aiohttp.ClientSession.request()ThreadedResolver.resolve()loop.getaddrinfo()loop.run_in_executor(None, ...)RuntimeError.

Creating a new ThreadPoolExecutor does not help — CPython's concurrent.futures.thread.submit() checks both self._shutdown (instance flag) and the module-level _shutdown flag set by _python_exit().

Suggested fix: when _stop_cluster raises RuntimeError during shutdown, retry with a throwaway aiohttp.ClientSession using a synchronous DNS resolver that calls socket.getaddrinfo() directly (no executor needed). Blocking the event loop is acceptable during shutdown.

import socket

class _SyncResolver:
    """Blocking DNS resolver — fallback for shutdown when executors are dead."""
    async def resolve(self, host, port=0, family=socket.AF_INET):
        infos = socket.getaddrinfo(host, port, family, socket.SOCK_STREAM)
        if not infos:
            raise OSError(f"DNS lookup failed for {host}")
        return [
            {"hostname": host, "host": a[-1][0], "port": a[-1][1],
             "family": a[0], "proto": a[2],
             "flags": socket.AI_NUMERICHOST | socket.AI_NUMERICSERV}
            for a in infos
        ]

    async def close(self):
        pass

In _stop_internal:

try:
    await self.gateway._stop_cluster(self.name)
except RuntimeError:
    # Executors dead at shutdown — retry with blocking DNS
    connector = aiohttp.TCPConnector(resolver=_SyncResolver())
    async with aiohttp.ClientSession(
        connector=connector, cookie_jar=aiohttp.CookieJar(unsafe=True)
    ) as session:
        url = f"{self.gateway.address}/api/v1/clusters/{self.name}"
        resp = await session.request("DELETE", url, **self.gateway._request_kwargs)
        if resp.status == 401:
            hdrs, _ctx = self.gateway.auth.pre_request(resp)
            await session.request("DELETE", url, headers=hdrs, **self.gateway._request_kwargs)

Related: dask/distributed issue — done_callback raises the same RuntimeError during shutdown (separate bug, same root cause).

Environment:

  • Dask version: 2025.9.1
  • dask-gateway version: 2025.4.0
  • aiohttp version: 3.13.3
  • Python version: 3.11.14
  • Operating System: macOS 15.7.4 (arm64)
  • Install method: uv (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