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

always bind the Scheduler to the IOLoop.current() - and deprecate the Scheduler(loop= kwarg #6443

Merged
merged 2 commits into from
May 27, 2022

Conversation

graingert
Copy link
Member

@graingert graingert commented May 25, 2022

refs #6163

Passing anything other than IOLoop.current() will fail when you try to use the Scheduler anyway:
self._lock will be bound to IOLoop.current().asyncio_loop aka asyncio.get_running_loop():

self._lock = asyncio.Lock()

and start_http_server runs on the IOLoop.current() also:
self.start_http_server(routes, dashboard_address, default_port=8787)

Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/home/graingert/projects/distributed/distributed/scheduler.py", line 2957, in __init__
    self.start_http_server(routes, dashboard_address, default_port=8787)
  File "/home/graingert/projects/distributed/distributed/node.py", line 158, in start_http_server
    self.http_server.listen(**tlz.merge(http_address, {"port": 0}))
  File "/home/graingert/miniconda3/envs/dask-distributed/lib/python3.10/site-packages/tornado/tcpserver.py", line 152, in listen
    self.add_sockets(sockets)
  File "/home/graingert/miniconda3/envs/dask-distributed/lib/python3.10/site-packages/tornado/tcpserver.py", line 165, in add_sockets
    self._handlers[sock.fileno()] = add_accept_handler(
  File "/home/graingert/miniconda3/envs/dask-distributed/lib/python3.10/site-packages/tornado/netutil.py", line 246, in add_accept_handler
    io_loop = IOLoop.current()
  File "/home/graingert/miniconda3/envs/dask-distributed/lib/python3.10/site-packages/tornado/ioloop.py", line 263, in current
    loop = asyncio.get_event_loop()
DeprecationWarning: There is no current event loop
  • Tests added / passed
  • Passes pre-commit run --all-files

@graingert graingert force-pushed the deprecate-scheduler-loop-kwarg branch from fc42dbd to e5a8ca3 Compare May 25, 2022 10:47
@fjetter
Copy link
Member

fjetter commented May 25, 2022

Besides this currently being broken, this implies that users who actually want the scheduler to be using a different event loop will need to make sure that the scheduler is initialized while running in a coro/task on that loop, correct? IIUC, this is how stdlib asyncio also suggest for their objects (e.g. Lock) to be used.

@mrocklin
Copy link
Member

I'm fine with this in principle. cc'ing @jacobtomlinson to keep him informed.

@jacobtomlinson
Copy link
Member

This seems reasonable to me. Thanks for keeping me up to date.

@graingert graingert marked this pull request as draft May 25, 2022 14:24
@graingert
Copy link
Member Author

this needs a fix from #6231 - converting back into draft for now

@github-actions
Copy link
Contributor

github-actions bot commented May 25, 2022

Unit Test Results

       15 files  +       15         15 suites  +15   6h 20m 40s ⏱️ + 6h 20m 40s
  2 819 tests +  2 819    2 738 ✔️ +  2 738    80 💤 +  80  1 +1 
20 896 runs  +20 896  19 958 ✔️ +19 958  937 💤 +937  1 +1 

For more details on these failures, see this check.

Results for commit 7f5f8ff. ± Comparison against base commit f20f776.

♻️ This comment has been updated with latest results.

@graingert
Copy link
Member Author

this also applies to subclasses of Server, eg ServerNode, Worker and Scheduler

the self._startup_lock gets bound to the current event loop, not the self.loop.asyncio_loop

self._startup_lock = asyncio.Lock()

import asyncio
import concurrent.futures
import contextlib
import sys

from tornado.ioloop import IOLoop

from distributed import Worker
from distributed.utils import sync


def _run_and_close_tornado(async_fn, /, *args, **kwargs):
    loop = None

    async def coro():
        nonlocal loop
        loop = IOLoop.current()
        return await async_fn(*args, **kwargs)

    try:
        asyncio.run(coro())
    finally:
        loop.close(close_fds=True)


@contextlib.contextmanager
def loop_in_thread():
    loop_started = concurrent.futures.Future()
    with concurrent.futures.ThreadPoolExecutor(
        1, thread_name_prefix="test IOLoop"
    ) as tpe:

        async def run():
            io_loop = IOLoop.current()
            stop_event = asyncio.Event()
            loop_started.set_result((io_loop, stop_event))
            await stop_event.wait()

        # run asyncio.run in a thread and collect exceptions from *either*
        # the loop failing to start, or failing to close
        ran = tpe.submit(_run_and_close_tornado, run)
        for f in concurrent.futures.as_completed((loop_started, ran)):
            if f is loop_started:
                io_loop, stop_event = loop_started.result()
                try:
                    yield io_loop
                finally:
                    io_loop.add_callback(stop_event.set)

            elif f is ran:
                # if this is the first iteration the loop failed to start
                # if it's the second iteration the loop has finished or
                # the loop failed to close and we need to raise the exception
                ran.result()
                return


def test_io_loop_in_thread():
    async def with_worker_contention(worker):
        async def with_worker():
            async with worker:
                pass

        return await asyncio.gather(with_worker(), with_worker())

    async def main(loop):
        worker = Worker(f"tcp://127.0.0.1:1234", loop=loop)
        sync(loop, with_worker_contention, worker=worker)

    with loop_in_thread() as loop:
        asyncio.run(main(loop))


if __name__ == "__main__":
    sys.exit(test_io_loop_in_thread())
2022-05-27 12:21:35,979 - distributed.worker - INFO -       Start worker at:      tcp://127.0.0.1:39339
2022-05-27 12:21:35,980 - distributed.worker - INFO -          Listening to:      tcp://127.0.0.1:39339
2022-05-27 12:21:35,980 - distributed.worker - INFO -          dashboard at:            127.0.0.1:45825
2022-05-27 12:21:35,980 - distributed.worker - INFO - Waiting to connect to:       tcp://127.0.0.1:1234
2022-05-27 12:21:35,980 - distributed.worker - INFO - -------------------------------------------------
2022-05-27 12:21:35,980 - distributed.worker - INFO -               Threads:                          8
2022-05-27 12:21:35,980 - distributed.worker - INFO -                Memory:                  15.36 GiB
2022-05-27 12:21:35,980 - distributed.worker - INFO -       Local Directory: /home/graingert/projects/distributed/dask-worker-space/worker-j82z74aw
2022-05-27 12:21:35,980 - distributed.worker - INFO - -------------------------------------------------
Traceback (most recent call last):
  File "/home/graingert/projects/distributed/demo.py", line 75, in <module>
    sys.exit(test_io_loop_in_thread())
  File "/home/graingert/projects/distributed/demo.py", line 71, in test_io_loop_in_thread
    asyncio.run(main(loop))
  File "/home/graingert/miniconda3/envs/dask-distributed-39/lib/python3.9/asyncio/runners.py", line 44, in run
    return loop.run_until_complete(main)
  File "/home/graingert/miniconda3/envs/dask-distributed-39/lib/python3.9/asyncio/base_events.py", line 647, in run_until_complete
    return future.result()
  File "/home/graingert/projects/distributed/demo.py", line 68, in main
    sync(loop, with_worker_contention, worker=worker)
  File "/home/graingert/projects/distributed/distributed/utils.py", line 387, in sync
    raise exc.with_traceback(tb)
  File "/home/graingert/projects/distributed/distributed/utils.py", line 360, in f
    result = yield future
  File "/home/graingert/miniconda3/envs/dask-distributed-39/lib/python3.9/site-packages/tornado/gen.py", line 762, in run
    value = future.result()
  File "/home/graingert/projects/distributed/demo.py", line 64, in with_worker_contention
    return await asyncio.gather(with_worker(), with_worker())
  File "/home/graingert/projects/distributed/demo.py", line 61, in with_worker
    async with worker:
  File "/home/graingert/projects/distributed/distributed/core.py", line 332, in __aenter__
    await self
  File "/home/graingert/projects/distributed/distributed/core.py", line 304, in start
    async with self._startup_lock:
  File "/home/graingert/miniconda3/envs/dask-distributed-39/lib/python3.9/asyncio/locks.py", line 14, in __aenter__
    await self.acquire()
  File "/home/graingert/miniconda3/envs/dask-distributed-39/lib/python3.9/asyncio/locks.py", line 120, in acquire
    await fut
RuntimeError: Task <Task pending name='Task-5' coro=<test_io_loop_in_thread.<locals>.with_worker_contention.<locals>.with_worker() running at /home/graingert/projects/distributed/demo.py:61> cb=[_gather.<locals>._done_callback() at /home/graingert/miniconda3/envs/dask-distributed-39/lib/python3.9/asyncio/tasks.py:767]> got Future <Future pending> attached to a different loop

@mrocklin mrocklin marked this pull request as ready for review May 27, 2022 14:25
@mrocklin
Copy link
Member

Merged in the other PR. I've marked as ready for review and restarted CI.

@mrocklin mrocklin merged commit 5feb171 into dask:main May 27, 2022
@mrocklin
Copy link
Member

This is in. Thanks @graingert

@graingert graingert deleted the deprecate-scheduler-loop-kwarg branch May 30, 2022 14:17
graingert added a commit to graingert/distributed that referenced this pull request May 30, 2022
passing anything other than IOLoop.current() is already unsupported:

see dask#6443 (comment)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants