You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
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=Truehandshake_overrides=Noneconnection_args= {'extra_conn_args': {}, 'require_encryption': False, 'ssl_context': None}
scheme='tcp', loc='127.0.0.1:51347'backend=<distributed.comm.tcp.TCPBackendobjectat0x000001C58A28C130>connector=<distributed.comm.tcp.TCPConnectorobjectat0x000001C5A9350DF0>comm=Nonetime_left=<functionconnect.<locals>.time_leftat0x000001C5B056F940>backoff_base=0.01asyncdefconnect(
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. """iftimeoutisNone:
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=Nonestart=time()
deftime_left():
deadline=start+timeoutreturnmax(0, deadline-time())
backoff_base=0.01attempt=0# Prefer multiple small attempts than one long attempt. This should protect# primarily from DNS race conditions# gh3104, gh4176, gh4167intermediate_cap=timeout/5active_exception=Nonewhiletime_left() >0:
try:
>comm=awaitasyncio.wait_for(
connector.connect(loc, deserialize=deserialize, **connection_args),
timeout=min(intermediate_cap, time_left()),
)
distributed\comm\core.py:284:
__ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
fut=<Taskcancelledname='Task-19081'coro=<BaseTCPConnector.connect() done, definedatD:\a\distributed\distributed\distributed\comm\tcp.py:392>>timeout=1.0asyncdefwait_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. """ifloopisNone:
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)
iftimeoutisNone:
returnawaitfutiftimeout<=0:
fut=ensure_future(fut, loop=loop)
iffut.done():
returnfut.result()
await_cancel_and_wait(fut, loop=loop)
try:
fut.result()
exceptexceptions.CancelledErrorasexc:
raiseexceptions.TimeoutError() fromexcelse:
raiseexceptions.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 timeouttry:
awaitwaiterexceptexceptions.CancelledError:
iffut.done():
returnfut.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/issue32751await_cancel_and_wait(fut, loop=loop)
raiseiffut.done():
returnfut.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/issue32751await_cancel_and_wait(fut, loop=loop)
>raiseexceptions.TimeoutError()
Easyncio.exceptions.TimeoutErrorC:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py:501: TimeoutErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
c=<Client: 'tcp://127.0.0.1:51347'processes=9threads=18, memory=63.00GiB>s=<Scheduler: "tcp://127.0.0.1:51347"workers: 0cores: 0, tasks: 0>@pytest.mark.slow@gen_cluster(nthreads=[],client=True,timeout=180,scheduler_kwargs={"allowed_failures": 100_000}, )asyncdeftest_stress_creation_and_deletion(c, s):
# Assertions are handled by the validate mechanism in the schedulerda=pytest.importorskip("dask.array")
def_disable_suspicious_counter(dask_worker):
dask_worker._suspicious_count_limit=Nonerng=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)
asyncdefcreate_and_destroy_worker(delay):
start=time()
whiletime() <start+5:
asyncwithNanny(s.address, nthreads=2) asn:
awaitc.run(_disable_suspicious_counter, workers=[n.worker_address])
awaitasyncio.sleep(delay)
print("Killed nanny")
>awaitasyncio.gather(*(create_and_destroy_worker(0.1*i) foriinrange(20)))
distributed\tests\test_stress.py:115:
__ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py:349: in__wakeupfuture.result()
C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py:280: in__stepresult=coro.send(None)
distributed\tests\test_stress.py:111: increate_and_destroy_workerawaitc.run(_disable_suspicious_counter, workers=[n.worker_address])
distributed\client.py:2414: in_runresponses=awaitself.scheduler.broadcast(
distributed\core.py:860: insend_recv_from_rpccomm=awaitself.pool.connect(self.addr)
distributed\core.py:1048: inconnectraiseexcdistributed\core.py:1032: inconnectcomm=awaitfutC:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py:260: in__await__yieldself# This tells Task to wait for completion.C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py:349: in__wakeupfuture.result()
C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py:178: inresultraiseself._exceptionC:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py:280: in__stepresult=coro.send(None)
_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
addr='tcp://127.0.0.1:51347', timeout=5, deserialize=Truehandshake_overrides=Noneconnection_args= {'extra_conn_args': {}, 'require_encryption': False, 'ssl_context': None}
scheme='tcp', loc='127.0.0.1:51347'backend=<distributed.comm.tcp.TCPBackendobjectat0x000001C58A28C130>connector=<distributed.comm.tcp.TCPConnectorobjectat0x000001C5A9350DF0>comm=Nonetime_left=<functionconnect.<locals>.time_leftat0x000001C5B056F940>backoff_base=0.01asyncdefconnect(
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. """iftimeoutisNone:
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=Nonestart=time()
deftime_left():
deadline=start+timeoutreturnmax(0, deadline-time())
backoff_base=0.01attempt=0# Prefer multiple small attempts than one long attempt. This should protect# primarily from DNS race conditions# gh3104, gh4176, gh4167intermediate_cap=timeout/5active_exception=Nonewhiletime_left() >0:
try:
comm=awaitasyncio.wait_for(
connector.connect(loc, deserialize=deserialize, **connection_args),
timeout=min(intermediate_cap, time_left()),
)
breakexceptFatalCommClosedError:
raise# CommClosed, EnvironmentError inherit from OSErrorexcept (TimeoutError, OSError) asexc:
active_exception=exc# The intermediate capping is mostly relevant for the initial# connect. Afterwards we should be more forgivingintermediate_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+=1logger.debug(
"Could not connect to %s, waiting for %s before retrying", loc, backoff
)
awaitasyncio.sleep(backoff)
else:
>raiseOSError(
f"Timedouttryingtoconnectto {addr} after {timeout} s"
) fromactive_exceptionEOSError: Timedouttryingtoconnecttotcp://127.0.0.1:51347after5sdistributed\comm\core.py:308: OSError----------------------------Capturedstdoutcall-----------------------------Killednanny----------------------------Capturedstderrcall-----------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51536distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51536distributed.worker-INFO-dashboardat: 127.0.0.1:51537distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-ssk3dfa_distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51538distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51538distributed.worker-INFO-dashboardat: 127.0.0.1:51539distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-vy9neoqldistributed.worker-INFO--------------------------------------------------C:\Miniconda3\envs\dask-distributed\lib\contextlib.py:120: UserWarning: Creatingscratchdirectoriesistakingasurprisinglylongtime. Thisisoftenduetorunningworkersonanetworkfilesystem. Considerspecifyingalocal-directorytopointworkerstowritescratchdatatoalocaldisk.
next(self.gen)
distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51542distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51542distributed.worker-INFO-dashboardat: 127.0.0.1:51543distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-_js_y4zddistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51552distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51552distributed.worker-INFO-dashboardat: 127.0.0.1:51553distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-0wqpr8indistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51565distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51565distributed.worker-INFO-dashboardat: 127.0.0.1:51566distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-mf814dbxdistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51576distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51576distributed.worker-INFO-dashboardat: 127.0.0.1:51578distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-84kke66fdistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51577distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51577distributed.worker-INFO-dashboardat: 127.0.0.1:51579distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-wnm4m5kydistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51580distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51580distributed.worker-INFO-dashboardat: 127.0.0.1:51581distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-zm6cwj3ndistributed.worker-INFO--------------------------------------------------C:\Miniconda3\envs\dask-distributed\lib\contextlib.py:120: UserWarning: Creatingscratchdirectoriesistakingasurprisinglylongtime. Thisisoftenduetorunningworkersonanetworkfilesystem. Considerspecifyingalocal-directorytopointworkerstowritescratchdatatoalocaldisk.
next(self.gen)
distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51588distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51588distributed.worker-INFO-dashboardat: 127.0.0.1:51589distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-4j7qz94rdistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51590distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51590distributed.worker-INFO-dashboardat: 127.0.0.1:51593distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-wd0yc8ogdistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51595distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51595distributed.worker-INFO-dashboardat: 127.0.0.1:51596distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-807bi0zgdistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51609distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51609distributed.worker-INFO-dashboardat: 127.0.0.1:51610distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-kxzy0p2vdistributed.worker-INFO--------------------------------------------------distributed.core-INFO-Startingestablishedconnectiondistributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51622distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51622distributed.worker-INFO-dashboardat: 127.0.0.1:51623distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-_apa_bwhdistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51626distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51626distributed.worker-INFO-dashboardat: 127.0.0.1:51630distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-n5ae4o7vdistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51629distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51629distributed.worker-INFO-dashboardat: 127.0.0.1:51631distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-x06vpoaadistributed.worker-INFO--------------------------------------------------C:\Miniconda3\envs\dask-distributed\lib\contextlib.py:120: UserWarning: Creatingscratchdirectoriesistakingasurprisinglylongtime. Thisisoftenduetorunningworkersonanetworkfilesystem. Considerspecifyingalocal-directorytopointworkerstowritescratchdatatoalocaldisk.
next(self.gen)
distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51650distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51650distributed.worker-INFO-dashboardat: 127.0.0.1:51651distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-2x477lesdistributed.worker-INFO--------------------------------------------------distributed.core-INFO-EventloopwasunresponsiveinWorkerfor3.53s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.worker-INFO-Runout-of-bandfunction'_disable_suspicious_counter'distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51658distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51658distributed.worker-INFO-dashboardat: 127.0.0.1:51660distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-jgsj_81tdistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51662distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51662distributed.worker-INFO-dashboardat: 127.0.0.1:51663distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-rnyqz7rydistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51664distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51664distributed.worker-INFO-dashboardat: 127.0.0.1:51665distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-iu6e7l4cdistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51536distributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.core-INFO-Startingestablishedconnectiondistributed.core-INFO-Startingestablishedconnectiondistributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.core-INFO-Startingestablishedconnectiondistributed.core-INFO-Startingestablishedconnectiondistributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.nanny-WARNING-Workerprocessstillaliveafter4seconds, killingdistributed.core-INFO-Startingestablishedconnectiondistributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.core-INFO-Startingestablishedconnectiondistributed.core-INFO-StartingestablishedconnectionC:\Miniconda3\envs\dask-distributed\lib\contextlib.py:120: UserWarning: Creatingscratchdirectoriesistakingasurprisinglylongtime. Thisisoftenduetorunningworkersonanetworkfilesystem. Considerspecifyingalocal-directorytopointworkerstowritescratchdatatoalocaldisk.
next(self.gen)
distributed.worker-INFO-Startworkerat: tcp://127.0.0.1:51699distributed.worker-INFO-Listeningto: tcp://127.0.0.1:51699distributed.worker-INFO-dashboardat: 127.0.0.1:51700distributed.worker-INFO-Waitingtoconnectto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Threads: 2distributed.worker-INFO-Memory: 7.00GiBdistributed.worker-INFO-LocalDirectory: D:\a\distributed\distributed\dask-worker-space\worker-bqew1i_ndistributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.core-INFO-Startingestablishedconnectiondistributed.core-INFO-Startingestablishedconnectiondistributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.core-INFO-Startingestablishedconnectiondistributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.core-INFO-Startingestablishedconnectiondistributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.core-INFO-Startingestablishedconnectiondistributed.core-INFO-EventloopwasunresponsiveinWorkerfor4.82s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
tornado.application-ERROR-Exceptionincallbackfunctools.partial(<boundmethodIOLoop._discard_future_resultof<tornado.platform.asyncio.AsyncIOLoopobjectat0x000001A7FD452CD0>>, <Taskfinishedname='Task-9'coro=<Worker.heartbeat() done, definedatD:\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 (mostrecentcalllast):
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line465, inwait_forfut.result()
asyncio.exceptions.CancelledErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
Traceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\comm\core.py", line320, inconnectawaitasyncio.wait_for(comm.write(local_info), time_left())
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line467, inwait_forraiseexceptions.TimeoutError() fromexcasyncio.exceptions.TimeoutErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
Traceback (mostrecentcalllast):
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line741, in_run_callbackret=callback()
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line765, in_discard_future_resultfuture.result()
File"D:\a\distributed\distributed\distributed\worker.py", line1104, inheartbeatraiseeFile"D:\a\distributed\distributed\distributed\worker.py", line1064, inheartbeatresponse=awaitretry_operation(
File"D:\a\distributed\distributed\distributed\utils_comm.py", line385, inretry_operationreturnawaitretry(
File"D:\a\distributed\distributed\distributed\utils_comm.py", line370, inretryreturnawaitcoro()
File"D:\a\distributed\distributed\distributed\core.py", line860, insend_recv_from_rpccomm=awaitself.pool.connect(self.addr)
File"D:\a\distributed\distributed\distributed\core.py", line1048, inconnectraiseexcFile"D:\a\distributed\distributed\distributed\core.py", line1032, inconnectcomm=awaitfutFile"D:\a\distributed\distributed\distributed\comm\core.py", line324, inconnectraiseOSError(
OSError: Timedoutduringhandshakewhileconnectingtotcp://127.0.0.1:51347after5sdistributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.core-INFO-EventloopwasunresponsiveinWorkerfor4.90s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.core-INFO-EventloopwasunresponsiveinWorkerfor5.67s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
tornado.application-ERROR-Exceptionincallbackfunctools.partial(<boundmethodIOLoop._discard_future_resultof<tornado.platform.asyncio.AsyncIOLoopobjectat0x000001BDD4E72CA0>>, <Taskfinishedname='Task-8'coro=<Worker.heartbeat() done, definedatD:\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 (mostrecentcalllast):
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line465, inwait_forfut.result()
asyncio.exceptions.CancelledErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
Traceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\comm\core.py", line320, inconnectawaitasyncio.wait_for(comm.write(local_info), time_left())
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line467, inwait_forraiseexceptions.TimeoutError() fromexcasyncio.exceptions.TimeoutErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
Traceback (mostrecentcalllast):
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line741, in_run_callbackret=callback()
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line765, in_discard_future_resultfuture.result()
File"D:\a\distributed\distributed\distributed\worker.py", line1104, inheartbeatraiseeFile"D:\a\distributed\distributed\distributed\worker.py", line1064, inheartbeatresponse=awaitretry_operation(
File"D:\a\distributed\distributed\distributed\utils_comm.py", line385, inretry_operationreturnawaitretry(
File"D:\a\distributed\distributed\distributed\utils_comm.py", line370, inretryreturnawaitcoro()
File"D:\a\distributed\distributed\distributed\core.py", line860, insend_recv_from_rpccomm=awaitself.pool.connect(self.addr)
File"D:\a\distributed\distributed\distributed\core.py", line1048, inconnectraiseexcFile"D:\a\distributed\distributed\distributed\core.py", line1032, inconnectcomm=awaitfutFile"D:\a\distributed\distributed\distributed\comm\core.py", line324, inconnectraiseOSError(
OSError: Timedoutduringhandshakewhileconnectingtotcp://127.0.0.1:51347after5sdistributed.core-INFO-EventloopwasunresponsiveinWorkerfor3.97s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51576distributed.core-INFO-Startingestablishedconnectiondistributed.core-INFO-EventloopwasunresponsiveinWorkerfor5.72s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.core-INFO-Startingestablishedconnectiondistributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------tornado.application-ERROR-Exceptionincallbackfunctools.partial(<boundmethodIOLoop._discard_future_resultof<tornado.platform.asyncio.AsyncIOLoopobjectat0x000001306B602CA0>>, <Taskfinishedname='Task-8'coro=<Worker.heartbeat() done, definedatD:\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 (mostrecentcalllast):
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line465, inwait_forfut.result()
asyncio.exceptions.CancelledErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
Traceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\comm\core.py", line320, inconnectawaitasyncio.wait_for(comm.write(local_info), time_left())
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line467, inwait_forraiseexceptions.TimeoutError() fromexcasyncio.exceptions.TimeoutErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
Traceback (mostrecentcalllast):
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line741, in_run_callbackret=callback()
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line765, in_discard_future_resultfuture.result()
File"D:\a\distributed\distributed\distributed\worker.py", line1104, inheartbeatraiseeFile"D:\a\distributed\distributed\distributed\worker.py", line1064, inheartbeatresponse=awaitretry_operation(
File"D:\a\distributed\distributed\distributed\utils_comm.py", line385, inretry_operationreturnawaitretry(
File"D:\a\distributed\distributed\distributed\utils_comm.py", line370, inretryreturnawaitcoro()
File"D:\a\distributed\distributed\distributed\core.py", line860, insend_recv_from_rpccomm=awaitself.pool.connect(self.addr)
File"D:\a\distributed\distributed\distributed\core.py", line1048, inconnectraiseexcFile"D:\a\distributed\distributed\distributed\core.py", line1032, inconnectcomm=awaitfutFile"D:\a\distributed\distributed\distributed\comm\core.py", line324, inconnectraiseOSError(
OSError: Timedoutduringhandshakewhileconnectingtotcp://127.0.0.1:51347after5sdistributed.core-INFO-EventloopwasunresponsiveinWorkerfor5.48s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.comm.tcp-INFO-Connectionfromtcp://127.0.0.1:51705closedbeforehandshakecompleteddistributed.core-INFO-Startingestablishedconnectiondistributed.core-INFO-EventloopwasunresponsiveinWorkerfor4.98s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51565distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51588distributed.comm.tcp-INFO-Connectionfromtcp://127.0.0.1:51737closedbeforehandshakecompleteddistributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51580distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51542distributed.core-ERROR-ExceptionwhilehandlingopbroadcastTraceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\comm\core.py", line284, inconnectcomm=awaitasyncio.wait_for(
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line501, inwait_forraiseexceptions.TimeoutError()
asyncio.exceptions.TimeoutErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
Traceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\core.py", line498, inhandle_commresult=awaitresultFile"C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line260, in__await__yieldself# This tells Task to wait for completion.File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line349, in__wakeupfuture.result()
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line178, inresultraiseself._exceptionFile"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line282, in__stepresult=coro.throw(exc)
File"D:\a\distributed\distributed\distributed\scheduler.py", line5836, inbroadcastresults=awaitAll(
File"D:\a\distributed\distributed\distributed\utils.py", line211, inAllresult=awaittasks.next()
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line349, in__wakeupfuture.result()
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line282, in__stepresult=coro.throw(exc)
File"D:\a\distributed\distributed\distributed\scheduler.py", line5828, insend_messagecomm=awaitself.rpc.connect(addr)
File"D:\a\distributed\distributed\distributed\core.py", line1048, inconnectraiseexcFile"D:\a\distributed\distributed\distributed\core.py", line1032, inconnectcomm=awaitfutFile"C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line260, in__await__yieldself# This tells Task to wait for completion.File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line349, in__wakeupfuture.result()
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line178, inresultraiseself._exceptionFile"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line280, in__stepresult=coro.send(None)
File"D:\a\distributed\distributed\distributed\comm\core.py", line308, inconnectraiseOSError(
OSError: Timedouttryingtoconnecttotcp://127.0.0.1:51538after5sdistributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.core-INFO-Startingestablishedconnectiondistributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51552distributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.core-INFO-Startingestablishedconnectiondistributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.core-INFO-Startingestablishedconnectiondistributed.nanny-WARNING-Workerprocessstillaliveafter4seconds, killingdistributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51538distributed.worker-INFO-Registeredto: tcp://127.0.0.1:51347distributed.worker-INFO--------------------------------------------------distributed.core-INFO-Startingestablishedconnectiondistributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51590distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51609distributed.core-ERROR-ExceptionwhilehandlingopbroadcastTraceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\comm\core.py", line284, inconnectcomm=awaitasyncio.wait_for(
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line501, inwait_forraiseexceptions.TimeoutError()
asyncio.exceptions.TimeoutErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
Traceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\core.py", line498, inhandle_commresult=awaitresultFile"C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line260, in__await__yieldself# This tells Task to wait for completion.File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line349, in__wakeupfuture.result()
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line178, inresultraiseself._exceptionFile"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line282, in__stepresult=coro.throw(exc)
File"D:\a\distributed\distributed\distributed\scheduler.py", line5836, inbroadcastresults=awaitAll(
File"D:\a\distributed\distributed\distributed\utils.py", line211, inAllresult=awaittasks.next()
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line349, in__wakeupfuture.result()
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line282, in__stepresult=coro.throw(exc)
File"D:\a\distributed\distributed\distributed\scheduler.py", line5828, insend_messagecomm=awaitself.rpc.connect(addr)
File"D:\a\distributed\distributed\distributed\core.py", line1048, inconnectraiseexcFile"D:\a\distributed\distributed\distributed\core.py", line1032, inconnectcomm=awaitfutFile"C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line260, in__await__yieldself# This tells Task to wait for completion.File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line349, in__wakeupfuture.result()
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\futures.py", line178, inresultraiseself._exceptionFile"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line280, in__stepresult=coro.send(None)
File"D:\a\distributed\distributed\distributed\comm\core.py", line308, inconnectraiseOSError(
OSError: Timedouttryingtoconnecttotcp://127.0.0.1:51577after5sdistributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51577distributed.nanny-WARNING-Workerprocessstillaliveafter4seconds, killingdistributed.nanny-WARNING-Workerprocessstillaliveafter4seconds, killingdistributed.nanny-WARNING-Workerprocessstillaliveafter4seconds, killingdistributed.nanny-WARNING-Workerprocessstillaliveafter4seconds, killingdistributed.core-INFO-EventloopwasunresponsiveinWorkerfor5.45s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.core-INFO-EventloopwasunresponsiveinWorkerfor5.46s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.core-INFO-EventloopwasunresponsiveinWorkerfor5.53s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
tornado.application-ERROR-Exceptionincallbackfunctools.partial(<boundmethodIOLoop._discard_future_resultof<tornado.platform.asyncio.AsyncIOLoopobjectat0x000002CC182C2CA0>>, <Taskfinishedname='Task-11'coro=<Worker.heartbeat() done, definedatD:\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 (mostrecentcalllast):
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line465, inwait_forfut.result()
asyncio.exceptions.CancelledErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
Traceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\comm\core.py", line320, inconnectawaitasyncio.wait_for(comm.write(local_info), time_left())
File"C:\Miniconda3\envs\dask-distributed\lib\asyncio\tasks.py", line467, inwait_forraiseexceptions.TimeoutError() fromexcasyncio.exceptions.TimeoutErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
Traceback (mostrecentcalllast):
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line741, in_run_callbackret=callback()
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line765, in_discard_future_resultfuture.result()
File"D:\a\distributed\distributed\distributed\worker.py", line1104, inheartbeatraiseeFile"D:\a\distributed\distributed\distributed\worker.py", line1064, inheartbeatresponse=awaitretry_operation(
File"D:\a\distributed\distributed\distributed\utils_comm.py", line385, inretry_operationreturnawaitretry(
File"D:\a\distributed\distributed\distributed\utils_comm.py", line370, inretryreturnawaitcoro()
File"D:\a\distributed\distributed\distributed\core.py", line860, insend_recv_from_rpccomm=awaitself.pool.connect(self.addr)
File"D:\a\distributed\distributed\distributed\core.py", line1048, inconnectraiseexcFile"D:\a\distributed\distributed\distributed\core.py", line1032, inconnectcomm=awaitfutFile"D:\a\distributed\distributed\distributed\comm\core.py", line324, inconnectraiseOSError(
OSError: Timedoutduringhandshakewhileconnectingtotcp://127.0.0.1:51347after5sdistributed.core-INFO-EventloopwasunresponsiveinWorkerfor5.59s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51658distributed.core-INFO-EventloopwasunresponsiveinWorkerfor5.59s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51664distributed.core-INFO-EventloopwasunresponsiveinWorkerfor5.60s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.core-INFO-EventloopwasunresponsiveinWorkerfor5.62s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.core-INFO-EventloopwasunresponsiveinWorkerfor5.63s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51626distributed.comm.tcp-INFO-Connectionfromtcp://127.0.0.1:51802closedbeforehandshakecompleteddistributed.core-INFO-EventloopwasunresponsiveinWorkerfor5.64s. Thisisoftencausedbylong-runningGIL-holdingfunctionsormovinglargechunksofdata. Thiscancausetimeoutsandinstability.
distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51622distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51650distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51662distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51629distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51595distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51699distributed.worker-INFO-Runout-of-bandfunction'_disable_suspicious_counter'distributed.worker-INFO-Runout-of-bandfunction'_disable_suspicious_counter'distributed.worker-INFO-Runout-of-bandfunction'_disable_suspicious_counter'distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51595distributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51622distributed.batched-INFO-BatchedCommClosed<TCP (closed) Worker->Schedulerlocal=tcp://127.0.0.1:51600remote=tcp://127.0.0.1:51347>Traceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\batched.py", line93, in_background_sendnbytes=yieldself.comm.write(
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\gen.py", line762, inrunvalue=future.result()
File"D:\a\distributed\distributed\distributed\comm\tcp.py", line248, inwriteraiseCommClosedError()
distributed.comm.core.CommClosedErrordistributed.batched-INFO-BatchedCommClosed<TCP (closed) Worker->Schedulerlocal=tcp://127.0.0.1:51627remote=tcp://127.0.0.1:51347>Traceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\batched.py", line93, in_background_sendnbytes=yieldself.comm.write(
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\gen.py", line762, inrunvalue=future.result()
File"D:\a\distributed\distributed\distributed\comm\tcp.py", line248, inwriteraiseCommClosedError()
distributed.comm.core.CommClosedErrordistributed.worker-INFO-Stoppingworkerattcp://127.0.0.1:51626distributed.batched-INFO-BatchedCommClosed<TCP (closed) Worker->Schedulerlocal=tcp://127.0.0.1:51648remote=tcp://127.0.0.1:51347>Traceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\batched.py", line93, in_background_sendnbytes=yieldself.comm.write(
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\gen.py", line762, inrunvalue=future.result()
File"D:\a\distributed\distributed\distributed\comm\tcp.py", line248, inwriteraiseCommClosedError()
distributed.comm.core.CommClosedErrordistributed.worker-INFO-Runout-of-bandfunction'_disable_suspicious_counter'tornado.application-ERROR-Exceptionincallbackfunctools.partial(<boundmethodIOLoop._discard_future_resultof<tornado.platform.asyncio.AsyncIOLoopobjectat0x000002B03E122CA0>>, <Taskfinishedname='Task-38'coro=<Worker.close() done, definedatD:\a\distributed\distributed\distributed\worker.py:1314>exception=CommClosedError('ConnectionPool not running. Status: Status.closed')>)
Traceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\core.py", line1032, inconnectcomm=awaitfutasyncio.exceptions.CancelledErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
Traceback (mostrecentcalllast):
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line741, in_run_callbackret=callback()
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line765, in_discard_future_resultfuture.result()
File"D:\a\distributed\distributed\distributed\worker.py", line1342, incloseawaitr.close_gracefully()
File"D:\a\distributed\distributed\distributed\core.py", line860, insend_recv_from_rpccomm=awaitself.pool.connect(self.addr)
File"D:\a\distributed\distributed\distributed\core.py", line1043, inconnectraiseCommClosedError(
distributed.comm.core.CommClosedError: ConnectionPoolnotrunning. Status: Status.closedtornado.application-ERROR-Exceptionincallbackfunctools.partial(<boundmethodIOLoop._discard_future_resultof<tornado.platform.asyncio.AsyncIOLoopobjectat0x000001607C9E2CA0>>, <Taskfinishedname='Task-60'coro=<Worker.close() done, definedatD:\a\distributed\distributed\distributed\worker.py:1314>exception=CommClosedError('ConnectionPool not running. Status: Status.closed')>)
Traceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\core.py", line1032, inconnectcomm=awaitfutasyncio.exceptions.CancelledErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
Traceback (mostrecentcalllast):
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line741, in_run_callbackret=callback()
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line765, in_discard_future_resultfuture.result()
File"D:\a\distributed\distributed\distributed\worker.py", line1342, incloseawaitr.close_gracefully()
File"D:\a\distributed\distributed\distributed\core.py", line860, insend_recv_from_rpccomm=awaitself.pool.connect(self.addr)
File"D:\a\distributed\distributed\distributed\core.py", line1043, inconnectraiseCommClosedError(
distributed.comm.core.CommClosedError: ConnectionPoolnotrunning. Status: Status.closedtornado.application-ERROR-Exceptionincallbackfunctools.partial(<boundmethodIOLoop._discard_future_resultof<tornado.platform.asyncio.AsyncIOLoopobjectat0x000001BE71AA2C70>>, <Taskfinishedname='Task-69'coro=<Worker.close() done, definedatD:\a\distributed\distributed\distributed\worker.py:1314>exception=CommClosedError('ConnectionPool not running. Status: Status.closed')>)
Traceback (mostrecentcalllast):
File"D:\a\distributed\distributed\distributed\core.py", line1032, inconnectcomm=awaitfutasyncio.exceptions.CancelledErrorTheaboveexceptionwasthedirectcauseofthefollowingexception:
Traceback (mostrecentcalllast):
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line741, in_run_callbackret=callback()
File"C:\Miniconda3\envs\dask-distributed\lib\site-packages\tornado\ioloop.py", line765, in_discard_future_resultfuture.result()
File"D:\a\distributed\distributed\distributed\worker.py", line1342, incloseawaitr.close_gracefully()
File"D:\a\distributed\distributed\distributed\core.py", line860, insend_recv_from_rpccomm=awaitself.pool.connect(self.addr)
File"D:\a\distributed\distributed\distributed\core.py", line1043, inconnectraiseCommClosedError(
distributed.comm.core.CommClosedError: ConnectionPoolnotrunning. Status: Status.closed
The text was updated successfully, but these errors were encountered:
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:
The text was updated successfully, but these errors were encountered: