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

Unblock event loop while waiting for ThreadpoolExecutor to shut down #5883

Merged
merged 4 commits into from
Mar 8, 2022

Conversation

fjetter
Copy link
Member

@fjetter fjetter commented Mar 1, 2022

I don't exactly understand what's happening but this may be an explanation for why we're occasionally seeing a lot of "event loop was blocked ..." warnings in our test suites. Potentially this may also explain some other spurious timeout errors

What's happening is that, by default, we close the workers after a test with gen_cluster in end_cluster using Worker.close(report=False), i.e. the default values of timeout=30 and executor_wait=True are respected.

I went through the test_semaphore cases because I noticed them all to be very slow. I figured I'd adjust a few parameters and speed up the entire thing. The test test_close_async was particularly interesting since it had a timeout of 120s defined which is required because a task is scheduled (fire_and_forget) that never finishes since it waits for a locked semaphore. Therefore, the ThreadPool could never gracefully shut down. Adding a timeout to this call actually didn't help but instead I saw this warning pop up.

It turns out that the executor.shutdown does block the event loop while waiting for a threading lock to be released
image

The code I wrote resolves all of this but it feels weird...

cc @graingert @gjoseph92 @crusaderky

@fjetter
Copy link
Member Author

fjetter commented Mar 1, 2022

Particularly concerning is that the default value for Worker.close(timeout=30) is identical to our test timeout. See also #5791 where we're discussing setting the connect timeout to the same value. We might want to be more selective in picking these values since there is clearly a certain hierarchy involved we should respect

@fjetter fjetter requested review from graingert and crusaderky March 1, 2022 14:29
@fjetter fjetter force-pushed the unblock_shutdown_event_loop branch 4 times, most recently from 2eb9c4d to 223114c Compare March 1, 2022 16:20
distributed/compatibility.py Outdated Show resolved Hide resolved
distributed/compatibility.py Outdated Show resolved Hide resolved
distributed/worker.py Outdated Show resolved Hide resolved
distributed/tests/test_worker.py Outdated Show resolved Hide resolved
distributed/tests/test_worker.py Outdated Show resolved Hide resolved
distributed/tests/test_worker.py Outdated Show resolved Hide resolved
@github-actions
Copy link
Contributor

github-actions bot commented Mar 1, 2022

Unit Test Results

       12 files  ±0         12 suites  ±0   6h 50m 48s ⏱️ - 9m 53s
  2 624 tests +1    2 539 ✔️  - 2    81 💤 +1  4 +3 
15 668 runs  +6  14 802 ✔️ +4  861 💤 ±0  5 +3 

For more details on these failures, see this check.

Results for commit c608315. ± Comparison against base commit 39c5e88.

♻️ This comment has been updated with latest results.

@graingert
Copy link
Member

graingert commented Mar 2, 2022

I'm wondering if it's better to use executor = w.executors["default"] directly - then you don't have to worry about pickling a threading.Event():

@gen_cluster(nthreads=[])
async def test_do_not_block_event_loop_during_shutdown(s, c):
    loop = asyncio.get_running_loop()
    called_handler = threading.Event()
    block_handler = threading.Event()

    w = await Worker(s.address)
    executor = w.executors["default"]

    async def block():
        def fn():
            called_handler.set()
            assert block_handler.wait(10)

        await loop.run_in_executor(executor, fn)

    async def set_future():
        while True:
            try:
                await loop.run_in_executor(executor, sleep, 0.1)
            except RuntimeError:  # executor has started shutting down
                block_handler.set()
                return

    async def close():
        called_handler.wait()
        # executor_wait is True by default but we want to be explicit here
        await w.close(executor_wait=True)

    await asyncio.gather(block(), close(), set_future())

@fjetter fjetter mentioned this pull request Mar 4, 2022
3 tasks
@fjetter fjetter self-assigned this Mar 4, 2022
@fjetter fjetter force-pushed the unblock_shutdown_event_loop branch from 0077270 to c608315 Compare March 7, 2022 16:20
@fjetter
Copy link
Member Author

fjetter commented Mar 8, 2022

Failing tests on ubu are stuck while closing a worker, so I suspect this to be related to #5910

  • ubu py3.8 test_worker_stream_died_during_comm
  • ubu p3.9 test_missing_data_errant_worker
  • OSX py3.9 ci1 test_missing_data_errant_worker

The other test failures are unrelated as far as I can tell

  • OSX py3.8 test_reconnect, known offender
  • OSX py3.9 not ci test_dashboard_non_standard_ports connection error while trying to connect a client. I assume this is unrelated.

@fjetter fjetter merged commit de94b40 into dask:main Mar 8, 2022
crusaderky added a commit to crusaderky/distributed that referenced this pull request Mar 18, 2022
crusaderky added a commit that referenced this pull request Mar 18, 2022
@fjetter fjetter deleted the unblock_shutdown_event_loop branch April 29, 2022 12:26
mrocklin pushed a commit that referenced this pull request Apr 29, 2022
…6091)

This reinstates #5883
which was reverted in #5961 / #5932

I could confirm the flakyness of `test_missing_data_errant_worker` after this change and am reasonably certain this is caused by #5910 which causes a closing worker to be restarted such that, even after `Worker.close` is done, the worker still appears to be partially up. 

The only reason I can see why this change promotes this behaviour is that if we no longer block the event loop while the threadpool is closing, this opens a much larger window for incoming requests to come in and being processed while close is running.

Closes #6239
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.

3 participants