Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

test_stress_creation_and_deletion flaky #5388

Closed
jrbourbeau opened this issue Oct 4, 2021 · 1 comment · Fixed by #7215
Closed

test_stress_creation_and_deletion flaky #5388

jrbourbeau opened this issue Oct 4, 2021 · 1 comment · Fixed by #7215
Assignees
Labels
flaky test Intermittent failures on CI.

Comments

@jrbourbeau
Copy link
Member

distributed/tests/test_stress.py::test_stress_creation_and_deletion has been sporadically failing recently (it's particularly bad on Windows, though it does occationally fail on other OS). See this CI build for an example. Full traceback is pasted below.

Full traceback:
================================== FAILURES ===================================
______________________ test_stress_creation_and_deletion ______________________

addr = 'tcp://127.0.0.1:51347', timeout = 5, deserialize = True
handshake_overrides = None
connection_args = {'extra_conn_args': {}, 'require_encryption': False, 'ssl_context': None}
scheme = 'tcp', loc = '127.0.0.1:51347'
backend = <distributed.comm.tcp.TCPBackend object at 0x000001C58A28C130>
connector = <distributed.comm.tcp.TCPConnector object at 0x000001C5A9350DF0>
comm = None
time_left = <function connect.<locals>.time_left at 0x000001C5B056F940>
backoff_base = 0.01

    async def connect(
        addr, timeout=None, deserialize=True, handshake_overrides=None, **connection_args
    ):
        """
        Connect to the given address (a URI such as ``tcp://127.0.0.1:1234``)
        and yield a ``Comm`` object.  If the connection attempt fails, it is
        retried until the *timeout* is expired.
        """
        if timeout is None:
            timeout = dask.config.get("distributed.comm.timeouts.connect")
        timeout = parse_timedelta(timeout, default="seconds")
    
        scheme, loc = parse_address(addr)
        backend = registry.get_backend(scheme)
        connector = backend.get_connector()
        comm = None
    
        start = time()
    
        def time_left():
            deadline = start + timeout
            return max(0, deadline - time())
    
        backoff_base = 0.01
        attempt = 0
    
        # Prefer multiple small attempts than one long attempt. This should protect
        # primarily from DNS race conditions
        # gh3104, gh4176, gh4167
        intermediate_cap = timeout / 5
        active_exception = None
        while time_left() > 0:
            try:
>               comm = await asyncio.wait_for(
                    connector.connect(loc, deserialize=deserialize, **connection_args),
                    timeout=min(intermediate_cap, time_left()),
                )

distributed\comm\core.py:284: 
_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _

fut = <Task cancelled name='Task-19081' coro=<BaseTCPConnector.connect() done, defined at D:\a\distributed\distributed\distributed\comm\tcp.py:392>>
timeout = 1.0

    async def wait_for(fut, timeout, *, loop=None):
        """Wait for the single Future or coroutine to complete, with timeout.
    
        Coroutine will be wrapped in Task.
    
        Returns result of the Future or coroutine.  When a timeout occurs,
        it cancels the task and raises TimeoutError.  To avoid the task
        cancellation, wrap it in shield().
    
        If the wait is cancelled, the task is also cancelled.
    
        This function is a coroutine.
        """
        if loop is None:
            loop = events.get_running_loop()
        else:
            warnings.warn("The loop argument is deprecated since Python 3.8, "
                          "and scheduled for removal in Python 3.10.",
                          DeprecationWarning, stacklevel=2)
    
        if timeout is None:
            return await fut
    
        if timeout <= 0:
            fut = ensure_future(fut, loop=loop)
    
            if fut.done():
                return fut.result()
    
            await _cancel_and_wait(fut, loop=loop)
            try:
                fut.result()
            except exceptions.CancelledError as exc:
                raise exceptions.TimeoutError() from exc
            else:
                raise exceptions.TimeoutError()
    
        waiter = loop.create_future()
        timeout_handle = loop.call_later(timeout, _release_waiter, waiter)
        cb = functools.partial(_release_waiter, waiter)
    
        fut = ensure_future(fut, loop=loop)
        fut.add_done_callback(cb)
    
        try:
            # wait until the future completes or the timeout
            try:
                await waiter
            except exceptions.CancelledError:
                if fut.done():
                    return fut.result()
                else:
                    fut.remove_done_callback(cb)
                    # We must ensure that the task is not running
                    # after wait_for() returns.
                    # See https://bugs.python.org/issue32751
                    await _cancel_and_wait(fut, loop=loop)
                    raise
    
            if fut.done():
                return fut.result()
            else:
                fut.remove_done_callback(cb)
                # We must ensure that the task is not running
                # after wait_for() returns.
                # See https://bugs.python.org/issue32751
                await _cancel_and_wait(fut, loop=loop)
>               raise exceptions.TimeoutError()
E               asyncio.exceptions.TimeoutError

C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py:501: TimeoutError

The above exception was the direct cause of the following exception:

c = <Client: 'tcp://127.0.0.1:51347' processes=9 threads=18, memory=63.00 GiB>
s = <Scheduler: "tcp://127.0.0.1:51347" workers: 0 cores: 0, tasks: 0>

    @pytest.mark.slow
    @gen_cluster(
        nthreads=[],
        client=True,
        timeout=180,
        scheduler_kwargs={"allowed_failures": 100_000},
    )
    async def test_stress_creation_and_deletion(c, s):
        # Assertions are handled by the validate mechanism in the scheduler
        da = pytest.importorskip("dask.array")
    
        def _disable_suspicious_counter(dask_worker):
            dask_worker._suspicious_count_limit = None
    
        rng = da.random.RandomState(0)
        x = rng.random(size=(2000, 2000), chunks=(100, 100))
        y = ((x + 1).T + (x * 2) - x.mean(axis=1)).sum().round(2)
        z = c.persist(y)
    
        async def create_and_destroy_worker(delay):
            start = time()
            while time() < start + 5:
                async with Nanny(s.address, nthreads=2) as n:
                    await c.run(_disable_suspicious_counter, workers=[n.worker_address])
                    await asyncio.sleep(delay)
                print("Killed nanny")
    
>       await asyncio.gather(*(create_and_destroy_worker(0.1 * i) for i in range(20)))

distributed\tests\test_stress.py:115: 
_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py:349: in __wakeup
    future.result()
C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py:280: in __step
    result = coro.send(None)
distributed\tests\test_stress.py:111: in create_and_destroy_worker
    await c.run(_disable_suspicious_counter, workers=[n.worker_address])
distributed\client.py:2414: in _run
    responses = await self.scheduler.broadcast(
distributed\core.py:860: in send_recv_from_rpc
    comm = await self.pool.connect(self.addr)
distributed\core.py:1048: in connect
    raise exc
distributed\core.py:1032: in connect
    comm = await fut
C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py:260: in __await__
    yield self  # This tells Task to wait for completion.
C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py:349: in __wakeup
    future.result()
C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py:178: in result
    raise self._exception
C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py:280: in __step
    result = coro.send(None)
_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _

addr = 'tcp://127.0.0.1:51347', timeout = 5, deserialize = True
handshake_overrides = None
connection_args = {'extra_conn_args': {}, 'require_encryption': False, 'ssl_context': None}
scheme = 'tcp', loc = '127.0.0.1:51347'
backend = <distributed.comm.tcp.TCPBackend object at 0x000001C58A28C130>
connector = <distributed.comm.tcp.TCPConnector object at 0x000001C5A9350DF0>
comm = None
time_left = <function connect.<locals>.time_left at 0x000001C5B056F940>
backoff_base = 0.01

    async def connect(
        addr, timeout=None, deserialize=True, handshake_overrides=None, **connection_args
    ):
        """
        Connect to the given address (a URI such as ``tcp://127.0.0.1:1234``)
        and yield a ``Comm`` object.  If the connection attempt fails, it is
        retried until the *timeout* is expired.
        """
        if timeout is None:
            timeout = dask.config.get("distributed.comm.timeouts.connect")
        timeout = parse_timedelta(timeout, default="seconds")
    
        scheme, loc = parse_address(addr)
        backend = registry.get_backend(scheme)
        connector = backend.get_connector()
        comm = None
    
        start = time()
    
        def time_left():
            deadline = start + timeout
            return max(0, deadline - time())
    
        backoff_base = 0.01
        attempt = 0
    
        # Prefer multiple small attempts than one long attempt. This should protect
        # primarily from DNS race conditions
        # gh3104, gh4176, gh4167
        intermediate_cap = timeout / 5
        active_exception = None
        while time_left() > 0:
            try:
                comm = await asyncio.wait_for(
                    connector.connect(loc, deserialize=deserialize, **connection_args),
                    timeout=min(intermediate_cap, time_left()),
                )
                break
            except FatalCommClosedError:
                raise
            # CommClosed, EnvironmentError inherit from OSError
            except (TimeoutError, OSError) as exc:
                active_exception = exc
    
                # The intermediate capping is mostly relevant for the initial
                # connect. Afterwards we should be more forgiving
                intermediate_cap = intermediate_cap * 1.5
                # FullJitter see https://aws.amazon.com/blogs/architecture/exponential-backoff-and-jitter/
    
                upper_cap = min(time_left(), backoff_base * (2 ** attempt))
                backoff = random.uniform(0, upper_cap)
                attempt += 1
                logger.debug(
                    "Could not connect to %s, waiting for %s before retrying", loc, backoff
                )
                await asyncio.sleep(backoff)
        else:
>           raise OSError(
                f"Timed out trying to connect to {addr} after {timeout} s"
            ) from active_exception
E           OSError: Timed out trying to connect to tcp://127.0.0.1:51347 after 5 s

distributed\comm\core.py:308: OSError
---------------------------- Captured stdout call -----------------------------
Killed nanny
---------------------------- Captured stderr call -----------------------------
distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51536

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51536

distributed.worker - INFO -          dashboard at:            127.0.0.1:51537

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-ssk3dfa_

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51538

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51538

distributed.worker - INFO -          dashboard at:            127.0.0.1:51539

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-vy9neoql

distributed.worker - INFO - -------------------------------------------------

C:\Miniconda3\envs\dask-distributed\lib\contextlib.py:120: UserWarning: Creating scratch directories is taking a surprisingly long time. This is often due to running workers on a network file system. Consider specifying a local-directory to point workers to write scratch data to a local disk.

  next(self.gen)

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51542

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51542

distributed.worker - INFO -          dashboard at:            127.0.0.1:51543

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-_js_y4zd

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51552

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51552

distributed.worker - INFO -          dashboard at:            127.0.0.1:51553

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-0wqpr8in

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51565

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51565

distributed.worker - INFO -          dashboard at:            127.0.0.1:51566

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-mf814dbx

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51576

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51576

distributed.worker - INFO -          dashboard at:            127.0.0.1:51578

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-84kke66f

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51577

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51577

distributed.worker - INFO -          dashboard at:            127.0.0.1:51579

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-wnm4m5ky

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51580

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51580

distributed.worker - INFO -          dashboard at:            127.0.0.1:51581

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-zm6cwj3n

distributed.worker - INFO - -------------------------------------------------

C:\Miniconda3\envs\dask-distributed\lib\contextlib.py:120: UserWarning: Creating scratch directories is taking a surprisingly long time. This is often due to running workers on a network file system. Consider specifying a local-directory to point workers to write scratch data to a local disk.

  next(self.gen)

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51588

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51588

distributed.worker - INFO -          dashboard at:            127.0.0.1:51589

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-4j7qz94r

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51590

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51590

distributed.worker - INFO -          dashboard at:            127.0.0.1:51593

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-wd0yc8og

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51595

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51595

distributed.worker - INFO -          dashboard at:            127.0.0.1:51596

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-807bi0zg

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51609

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51609

distributed.worker - INFO -          dashboard at:            127.0.0.1:51610

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-kxzy0p2v

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Starting established connection

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51622

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51622

distributed.worker - INFO -          dashboard at:            127.0.0.1:51623

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-_apa_bwh

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51626

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51626

distributed.worker - INFO -          dashboard at:            127.0.0.1:51630

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-n5ae4o7v

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51629

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51629

distributed.worker - INFO -          dashboard at:            127.0.0.1:51631

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-x06vpoaa

distributed.worker - INFO - -------------------------------------------------

C:\Miniconda3\envs\dask-distributed\lib\contextlib.py:120: UserWarning: Creating scratch directories is taking a surprisingly long time. This is often due to running workers on a network file system. Consider specifying a local-directory to point workers to write scratch data to a local disk.

  next(self.gen)

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51650

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51650

distributed.worker - INFO -          dashboard at:            127.0.0.1:51651

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-2x477les

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Event loop was unresponsive in Worker for 3.53s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.worker - INFO - Run out-of-band function '_disable_suspicious_counter'

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51658

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51658

distributed.worker - INFO -          dashboard at:            127.0.0.1:51660

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-jgsj_81t

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51662

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51662

distributed.worker - INFO -          dashboard at:            127.0.0.1:51663

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-rnyqz7ry

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51664

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51664

distributed.worker - INFO -          dashboard at:            127.0.0.1:51665

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-iu6e7l4c

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51536

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Starting established connection

distributed.core - INFO - Starting established connection

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Starting established connection

distributed.core - INFO - Starting established connection

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.nanny - WARNING - Worker process still alive after 4 seconds, killing
distributed.core - INFO - Starting established connection

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Starting established connection

distributed.core - INFO - Starting established connection

C:\Miniconda3\envs\dask-distributed\lib\contextlib.py:120: UserWarning: Creating scratch directories is taking a surprisingly long time. This is often due to running workers on a network file system. Consider specifying a local-directory to point workers to write scratch data to a local disk.

  next(self.gen)

distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:51699

distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:51699

distributed.worker - INFO -          dashboard at:            127.0.0.1:51700

distributed.worker - INFO - Waiting to connect to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -               Threads:                          2

distributed.worker - INFO -                Memory:                   7.00 GiB

distributed.worker - INFO -       Local Directory: D:\a\distributed\distributed\dask-worker-space\worker-bqew1i_n

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Starting established connection

distributed.core - INFO - Starting established connection

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Starting established connection

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Starting established connection

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Starting established connection

distributed.core - INFO - Event loop was unresponsive in Worker for 4.82s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

tornado.application - ERROR - Exception in callback functools.partial(<bound method IOLoop._discard_future_result of <tornado.platform.asyncio.AsyncIOLoop object at 0x000001A7FD452CD0>>, <Task finished name='Task-9' coro=<Worker.heartbeat() done, defined at D:\a\distributed\distributed\distributed\worker.py:1056> exception=OSError('Timed out during handshake while connecting to tcp://127.0.0.1:51347 after 5 s')>)

Traceback (most recent call last):

  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 465, in wait_for

    fut.result()

asyncio.exceptions.CancelledError



The above exception was the direct cause of the following exception:



Traceback (most recent call last):

  File "D:\a\distributed\distributed\distributed\comm\core.py", line 320, in connect

    await asyncio.wait_for(comm.write(local_info), time_left())

  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 467, in wait_for

    raise exceptions.TimeoutError() from exc

asyncio.exceptions.TimeoutError



The above exception was the direct cause of the following exception:



Traceback (most recent call last):

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 741, in _run_callback

    ret = callback()

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 765, in _discard_future_result

    future.result()

  File "D:\a\distributed\distributed\distributed\worker.py", line 1104, in heartbeat

    raise e

  File "D:\a\distributed\distributed\distributed\worker.py", line 1064, in heartbeat

    response = await retry_operation(

  File "D:\a\distributed\distributed\distributed\utils_comm.py", line 385, in retry_operation

    return await retry(

  File "D:\a\distributed\distributed\distributed\utils_comm.py", line 370, in retry

    return await coro()

  File "D:\a\distributed\distributed\distributed\core.py", line 860, in send_recv_from_rpc

    comm = await self.pool.connect(self.addr)

  File "D:\a\distributed\distributed\distributed\core.py", line 1048, in connect

    raise exc

  File "D:\a\distributed\distributed\distributed\core.py", line 1032, in connect

    comm = await fut

  File "D:\a\distributed\distributed\distributed\comm\core.py", line 324, in connect

    raise OSError(

OSError: Timed out during handshake while connecting to tcp://127.0.0.1:51347 after 5 s

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Event loop was unresponsive in Worker for 4.90s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Event loop was unresponsive in Worker for 5.67s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

tornado.application - ERROR - Exception in callback functools.partial(<bound method IOLoop._discard_future_result of <tornado.platform.asyncio.AsyncIOLoop object at 0x000001BDD4E72CA0>>, <Task finished name='Task-8' coro=<Worker.heartbeat() done, defined at D:\a\distributed\distributed\distributed\worker.py:1056> exception=OSError('Timed out during handshake while connecting to tcp://127.0.0.1:51347 after 5 s')>)

Traceback (most recent call last):

  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 465, in wait_for

    fut.result()

asyncio.exceptions.CancelledError



The above exception was the direct cause of the following exception:



Traceback (most recent call last):

  File "D:\a\distributed\distributed\distributed\comm\core.py", line 320, in connect

    await asyncio.wait_for(comm.write(local_info), time_left())

  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 467, in wait_for

    raise exceptions.TimeoutError() from exc

asyncio.exceptions.TimeoutError



The above exception was the direct cause of the following exception:



Traceback (most recent call last):

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 741, in _run_callback

    ret = callback()

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 765, in _discard_future_result

    future.result()

  File "D:\a\distributed\distributed\distributed\worker.py", line 1104, in heartbeat

    raise e

  File "D:\a\distributed\distributed\distributed\worker.py", line 1064, in heartbeat

    response = await retry_operation(

  File "D:\a\distributed\distributed\distributed\utils_comm.py", line 385, in retry_operation

    return await retry(

  File "D:\a\distributed\distributed\distributed\utils_comm.py", line 370, in retry

    return await coro()

  File "D:\a\distributed\distributed\distributed\core.py", line 860, in send_recv_from_rpc

    comm = await self.pool.connect(self.addr)

  File "D:\a\distributed\distributed\distributed\core.py", line 1048, in connect

    raise exc

  File "D:\a\distributed\distributed\distributed\core.py", line 1032, in connect

    comm = await fut

  File "D:\a\distributed\distributed\distributed\comm\core.py", line 324, in connect

    raise OSError(

OSError: Timed out during handshake while connecting to tcp://127.0.0.1:51347 after 5 s

distributed.core - INFO - Event loop was unresponsive in Worker for 3.97s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51576

distributed.core - INFO - Starting established connection

distributed.core - INFO - Event loop was unresponsive in Worker for 5.72s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.core - INFO - Starting established connection

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

tornado.application - ERROR - Exception in callback functools.partial(<bound method IOLoop._discard_future_result of <tornado.platform.asyncio.AsyncIOLoop object at 0x000001306B602CA0>>, <Task finished name='Task-8' coro=<Worker.heartbeat() done, defined at D:\a\distributed\distributed\distributed\worker.py:1056> exception=OSError('Timed out during handshake while connecting to tcp://127.0.0.1:51347 after 5 s')>)

Traceback (most recent call last):

  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 465, in wait_for

    fut.result()

asyncio.exceptions.CancelledError



The above exception was the direct cause of the following exception:



Traceback (most recent call last):

  File "D:\a\distributed\distributed\distributed\comm\core.py", line 320, in connect

    await asyncio.wait_for(comm.write(local_info), time_left())

  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 467, in wait_for

    raise exceptions.TimeoutError() from exc

asyncio.exceptions.TimeoutError



The above exception was the direct cause of the following exception:



Traceback (most recent call last):

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 741, in _run_callback

    ret = callback()

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 765, in _discard_future_result

    future.result()

  File "D:\a\distributed\distributed\distributed\worker.py", line 1104, in heartbeat

    raise e

  File "D:\a\distributed\distributed\distributed\worker.py", line 1064, in heartbeat

    response = await retry_operation(

  File "D:\a\distributed\distributed\distributed\utils_comm.py", line 385, in retry_operation

    return await retry(

  File "D:\a\distributed\distributed\distributed\utils_comm.py", line 370, in retry

    return await coro()

  File "D:\a\distributed\distributed\distributed\core.py", line 860, in send_recv_from_rpc

    comm = await self.pool.connect(self.addr)

  File "D:\a\distributed\distributed\distributed\core.py", line 1048, in connect

    raise exc

  File "D:\a\distributed\distributed\distributed\core.py", line 1032, in connect

    comm = await fut

  File "D:\a\distributed\distributed\distributed\comm\core.py", line 324, in connect

    raise OSError(

OSError: Timed out during handshake while connecting to tcp://127.0.0.1:51347 after 5 s

distributed.core - INFO - Event loop was unresponsive in Worker for 5.48s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.comm.tcp - INFO - Connection from tcp://127.0.0.1:51705 closed before handshake completed

distributed.core - INFO - Starting established connection

distributed.core - INFO - Event loop was unresponsive in Worker for 4.98s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51565

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51588

distributed.comm.tcp - INFO - Connection from tcp://127.0.0.1:51737 closed before handshake completed

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51580

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51542

distributed.core - ERROR - Exception while handling op broadcast
Traceback (most recent call last):
  File "D:\a\distributed\distributed\distributed\comm\core.py", line 284, in connect
    comm = await asyncio.wait_for(
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 501, in wait_for
    raise exceptions.TimeoutError()
asyncio.exceptions.TimeoutError

The above exception was the direct cause of the following exception:

Traceback (most recent call last):
  File "D:\a\distributed\distributed\distributed\core.py", line 498, in handle_comm
    result = await result
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line 260, in __await__
    yield self  # This tells Task to wait for completion.
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 349, in __wakeup
    future.result()
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line 178, in result
    raise self._exception
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 282, in __step
    result = coro.throw(exc)
  File "D:\a\distributed\distributed\distributed\scheduler.py", line 5836, in broadcast
    results = await All(
  File "D:\a\distributed\distributed\distributed\utils.py", line 211, in All
    result = await tasks.next()
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 349, in __wakeup
    future.result()
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 282, in __step
    result = coro.throw(exc)
  File "D:\a\distributed\distributed\distributed\scheduler.py", line 5828, in send_message
    comm = await self.rpc.connect(addr)
  File "D:\a\distributed\distributed\distributed\core.py", line 1048, in connect
    raise exc
  File "D:\a\distributed\distributed\distributed\core.py", line 1032, in connect
    comm = await fut
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line 260, in __await__
    yield self  # This tells Task to wait for completion.
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 349, in __wakeup
    future.result()
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line 178, in result
    raise self._exception
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 280, in __step
    result = coro.send(None)
  File "D:\a\distributed\distributed\distributed\comm\core.py", line 308, in connect
    raise OSError(
OSError: Timed out trying to connect to tcp://127.0.0.1:51538 after 5 s
distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Starting established connection

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51552

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Starting established connection

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Starting established connection

distributed.nanny - WARNING - Worker process still alive after 4 seconds, killing
distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51538

distributed.worker - INFO -         Registered to:      tcp://127.0.0.1:51347

distributed.worker - INFO - -------------------------------------------------

distributed.core - INFO - Starting established connection

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51590

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51609

distributed.core - ERROR - Exception while handling op broadcast
Traceback (most recent call last):
  File "D:\a\distributed\distributed\distributed\comm\core.py", line 284, in connect
    comm = await asyncio.wait_for(
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 501, in wait_for
    raise exceptions.TimeoutError()
asyncio.exceptions.TimeoutError

The above exception was the direct cause of the following exception:

Traceback (most recent call last):
  File "D:\a\distributed\distributed\distributed\core.py", line 498, in handle_comm
    result = await result
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line 260, in __await__
    yield self  # This tells Task to wait for completion.
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 349, in __wakeup
    future.result()
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line 178, in result
    raise self._exception
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 282, in __step
    result = coro.throw(exc)
  File "D:\a\distributed\distributed\distributed\scheduler.py", line 5836, in broadcast
    results = await All(
  File "D:\a\distributed\distributed\distributed\utils.py", line 211, in All
    result = await tasks.next()
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 349, in __wakeup
    future.result()
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 282, in __step
    result = coro.throw(exc)
  File "D:\a\distributed\distributed\distributed\scheduler.py", line 5828, in send_message
    comm = await self.rpc.connect(addr)
  File "D:\a\distributed\distributed\distributed\core.py", line 1048, in connect
    raise exc
  File "D:\a\distributed\distributed\distributed\core.py", line 1032, in connect
    comm = await fut
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line 260, in __await__
    yield self  # This tells Task to wait for completion.
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 349, in __wakeup
    future.result()
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line 178, in result
    raise self._exception
  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 280, in __step
    result = coro.send(None)
  File "D:\a\distributed\distributed\distributed\comm\core.py", line 308, in connect
    raise OSError(
OSError: Timed out trying to connect to tcp://127.0.0.1:51577 after 5 s
distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51577

distributed.nanny - WARNING - Worker process still alive after 4 seconds, killing
distributed.nanny - WARNING - Worker process still alive after 4 seconds, killing
distributed.nanny - WARNING - Worker process still alive after 4 seconds, killing
distributed.nanny - WARNING - Worker process still alive after 4 seconds, killing
distributed.core - INFO - Event loop was unresponsive in Worker for 5.45s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.core - INFO - Event loop was unresponsive in Worker for 5.46s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.core - INFO - Event loop was unresponsive in Worker for 5.53s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

tornado.application - ERROR - Exception in callback functools.partial(<bound method IOLoop._discard_future_result of <tornado.platform.asyncio.AsyncIOLoop object at 0x000002CC182C2CA0>>, <Task finished name='Task-11' coro=<Worker.heartbeat() done, defined at D:\a\distributed\distributed\distributed\worker.py:1056> exception=OSError('Timed out during handshake while connecting to tcp://127.0.0.1:51347 after 5 s')>)

Traceback (most recent call last):

  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 465, in wait_for

    fut.result()

asyncio.exceptions.CancelledError



The above exception was the direct cause of the following exception:



Traceback (most recent call last):

  File "D:\a\distributed\distributed\distributed\comm\core.py", line 320, in connect

    await asyncio.wait_for(comm.write(local_info), time_left())

  File "C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line 467, in wait_for

    raise exceptions.TimeoutError() from exc

asyncio.exceptions.TimeoutError



The above exception was the direct cause of the following exception:



Traceback (most recent call last):

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 741, in _run_callback

    ret = callback()

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 765, in _discard_future_result

    future.result()

  File "D:\a\distributed\distributed\distributed\worker.py", line 1104, in heartbeat

    raise e

  File "D:\a\distributed\distributed\distributed\worker.py", line 1064, in heartbeat

    response = await retry_operation(

  File "D:\a\distributed\distributed\distributed\utils_comm.py", line 385, in retry_operation

    return await retry(

  File "D:\a\distributed\distributed\distributed\utils_comm.py", line 370, in retry

    return await coro()

  File "D:\a\distributed\distributed\distributed\core.py", line 860, in send_recv_from_rpc

    comm = await self.pool.connect(self.addr)

  File "D:\a\distributed\distributed\distributed\core.py", line 1048, in connect

    raise exc

  File "D:\a\distributed\distributed\distributed\core.py", line 1032, in connect

    comm = await fut

  File "D:\a\distributed\distributed\distributed\comm\core.py", line 324, in connect

    raise OSError(

OSError: Timed out during handshake while connecting to tcp://127.0.0.1:51347 after 5 s

distributed.core - INFO - Event loop was unresponsive in Worker for 5.59s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51658

distributed.core - INFO - Event loop was unresponsive in Worker for 5.59s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51664

distributed.core - INFO - Event loop was unresponsive in Worker for 5.60s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.core - INFO - Event loop was unresponsive in Worker for 5.62s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.core - INFO - Event loop was unresponsive in Worker for 5.63s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51626

distributed.comm.tcp - INFO - Connection from tcp://127.0.0.1:51802 closed before handshake completed

distributed.core - INFO - Event loop was unresponsive in Worker for 5.64s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51622

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51650

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51662

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51629

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51595

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51699

distributed.worker - INFO - Run out-of-band function '_disable_suspicious_counter'

distributed.worker - INFO - Run out-of-band function '_disable_suspicious_counter'

distributed.worker - INFO - Run out-of-band function '_disable_suspicious_counter'

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51595

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51622

distributed.batched - INFO - Batched Comm Closed <TCP (closed) Worker->Scheduler local=tcp://127.0.0.1:51600 remote=tcp://127.0.0.1:51347>

Traceback (most recent call last):

  File "D:\a\distributed\distributed\distributed\batched.py", line 93, in _background_send

    nbytes = yield self.comm.write(

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\gen.py", line 762, in run

    value = future.result()

  File "D:\a\distributed\distributed\distributed\comm\tcp.py", line 248, in write

    raise CommClosedError()

distributed.comm.core.CommClosedError

distributed.batched - INFO - Batched Comm Closed <TCP (closed) Worker->Scheduler local=tcp://127.0.0.1:51627 remote=tcp://127.0.0.1:51347>

Traceback (most recent call last):

  File "D:\a\distributed\distributed\distributed\batched.py", line 93, in _background_send

    nbytes = yield self.comm.write(

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\gen.py", line 762, in run

    value = future.result()

  File "D:\a\distributed\distributed\distributed\comm\tcp.py", line 248, in write

    raise CommClosedError()

distributed.comm.core.CommClosedError

distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:51626

distributed.batched - INFO - Batched Comm Closed <TCP (closed) Worker->Scheduler local=tcp://127.0.0.1:51648 remote=tcp://127.0.0.1:51347>

Traceback (most recent call last):

  File "D:\a\distributed\distributed\distributed\batched.py", line 93, in _background_send

    nbytes = yield self.comm.write(

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\gen.py", line 762, in run

    value = future.result()

  File "D:\a\distributed\distributed\distributed\comm\tcp.py", line 248, in write

    raise CommClosedError()

distributed.comm.core.CommClosedError

distributed.worker - INFO - Run out-of-band function '_disable_suspicious_counter'

tornado.application - ERROR - Exception in callback functools.partial(<bound method IOLoop._discard_future_result of <tornado.platform.asyncio.AsyncIOLoop object at 0x000002B03E122CA0>>, <Task finished name='Task-38' coro=<Worker.close() done, defined at D:\a\distributed\distributed\distributed\worker.py:1314> exception=CommClosedError('ConnectionPool not running. Status: Status.closed')>)

Traceback (most recent call last):

  File "D:\a\distributed\distributed\distributed\core.py", line 1032, in connect

    comm = await fut

asyncio.exceptions.CancelledError



The above exception was the direct cause of the following exception:



Traceback (most recent call last):

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 741, in _run_callback

    ret = callback()

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 765, in _discard_future_result

    future.result()

  File "D:\a\distributed\distributed\distributed\worker.py", line 1342, in close

    await r.close_gracefully()

  File "D:\a\distributed\distributed\distributed\core.py", line 860, in send_recv_from_rpc

    comm = await self.pool.connect(self.addr)

  File "D:\a\distributed\distributed\distributed\core.py", line 1043, in connect

    raise CommClosedError(

distributed.comm.core.CommClosedError: ConnectionPool not running. Status: Status.closed

tornado.application - ERROR - Exception in callback functools.partial(<bound method IOLoop._discard_future_result of <tornado.platform.asyncio.AsyncIOLoop object at 0x000001607C9E2CA0>>, <Task finished name='Task-60' coro=<Worker.close() done, defined at D:\a\distributed\distributed\distributed\worker.py:1314> exception=CommClosedError('ConnectionPool not running. Status: Status.closed')>)

Traceback (most recent call last):

  File "D:\a\distributed\distributed\distributed\core.py", line 1032, in connect

    comm = await fut

asyncio.exceptions.CancelledError



The above exception was the direct cause of the following exception:



Traceback (most recent call last):

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 741, in _run_callback

    ret = callback()

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 765, in _discard_future_result

    future.result()

  File "D:\a\distributed\distributed\distributed\worker.py", line 1342, in close

    await r.close_gracefully()

  File "D:\a\distributed\distributed\distributed\core.py", line 860, in send_recv_from_rpc

    comm = await self.pool.connect(self.addr)

  File "D:\a\distributed\distributed\distributed\core.py", line 1043, in connect

    raise CommClosedError(

distributed.comm.core.CommClosedError: ConnectionPool not running. Status: Status.closed

tornado.application - ERROR - Exception in callback functools.partial(<bound method IOLoop._discard_future_result of <tornado.platform.asyncio.AsyncIOLoop object at 0x000001BE71AA2C70>>, <Task finished name='Task-69' coro=<Worker.close() done, defined at D:\a\distributed\distributed\distributed\worker.py:1314> exception=CommClosedError('ConnectionPool not running. Status: Status.closed')>)

Traceback (most recent call last):

  File "D:\a\distributed\distributed\distributed\core.py", line 1032, in connect

    comm = await fut

asyncio.exceptions.CancelledError



The above exception was the direct cause of the following exception:



Traceback (most recent call last):

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 741, in _run_callback

    ret = callback()

  File "C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line 765, in _discard_future_result

    future.result()

  File "D:\a\distributed\distributed\distributed\worker.py", line 1342, in close

    await r.close_gracefully()

  File "D:\a\distributed\distributed\distributed\core.py", line 860, in send_recv_from_rpc

    comm = await self.pool.connect(self.addr)

  File "D:\a\distributed\distributed\distributed\core.py", line 1043, in connect

    raise CommClosedError(

distributed.comm.core.CommClosedError: ConnectionPool not running. Status: Status.closed
@crusaderky
Copy link
Collaborator

The test spawns 20 nannies (20 processes, 60 threads). This is well known to upset CI (#5662).
It's also affected by #7063.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
flaky test Intermittent failures on CI.
Projects
None yet
Development

Successfully merging a pull request may close this issue.

2 participants