From 05e449608dfc39b53a113fdf639cb4b4e98bfafc Mon Sep 17 00:00:00 2001 From: Lawrence Mitchell Date: Mon, 24 Oct 2022 14:14:36 +0100 Subject: [PATCH] WIP: Removing make_current; quite cargo-culted --- distributed/nanny.py | 4 +--- distributed/tests/test_client.py | 27 +++++++++++---------------- distributed/utils_test.py | 18 ------------------ 3 files changed, 12 insertions(+), 37 deletions(-) diff --git a/distributed/nanny.py b/distributed/nanny.py index 43523694cc6..aa4b3aed42d 100644 --- a/distributed/nanny.py +++ b/distributed/nanny.py @@ -866,9 +866,7 @@ def _run( if silence_logs: logger.setLevel(silence_logs) - IOLoop.clear_instance() - loop = IOLoop() - loop.make_current() + loop = IOLoop.current() worker = Worker(**worker_kwargs) async def do_stop(timeout=5, executor_wait=True): diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index d911487bbcd..1c62c283e3b 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -104,7 +104,6 @@ map_varying, nodebug, popen, - pristine_loop, randominc, save_sys_modules, slowadd, @@ -2206,12 +2205,13 @@ async def test_multi_client(s, a, b): def long_running_client_connection(address): - with pristine_loop(): + async def run(): c = Client(address) x = c.submit(lambda x: x + 1, 10) - x.result() sleep(100) + asyncio.new_event_loop().run_until_complete(run()) + @gen_cluster() async def test_cleanup_after_broken_client_connection(s, a, b): @@ -5538,23 +5538,18 @@ async def test_future_auto_inform(c, s, a, b): await asyncio.sleep(0.01) -@pytest.mark.filterwarnings("ignore:There is no current event loop:DeprecationWarning") -@pytest.mark.filterwarnings("ignore:make_current is deprecated:DeprecationWarning") -@pytest.mark.filterwarnings("ignore:clear_current is deprecated:DeprecationWarning") def test_client_async_before_loop_starts(cleanup): - async def close(): + async def run(): + loop = IOLoop.current() + client = Client(asynchronous=True, loop=loop) + assert client.asynchronous + assert isinstance(client.close(), NoOpAwaitable) async with client: pass - with pristine_loop() as loop: - with pytest.warns( - DeprecationWarning, - match=r"Constructing LoopRunner\(loop=loop\) without a running loop is deprecated", - ): - client = Client(asynchronous=True, loop=loop) - assert client.asynchronous - assert isinstance(client.close(), NoOpAwaitable) - loop.run_sync(close) # TODO: client.close() does not unset global client + loop = asyncio.new_event_loop() + loop.run_until_complete(run()) + loop.close() # FIXME shouldn't consistently fail on windows, may be an actual bug diff --git a/distributed/utils_test.py b/distributed/utils_test.py index 8a92f1db1dd..b643da0e691 100644 --- a/distributed/utils_test.py +++ b/distributed/utils_test.py @@ -160,24 +160,6 @@ async def run(): return -@contextmanager -def pristine_loop(): - IOLoop.clear_instance() - IOLoop.clear_current() - loop = IOLoop() - loop.make_current() - assert IOLoop.current() is loop - try: - yield loop - finally: - try: - loop.close(all_fds=True) - except (KeyError, ValueError): - pass - IOLoop.clear_instance() - IOLoop.clear_current() - - original_config = copy.deepcopy(dask.config.config)