trio icon indicating copy to clipboard operation
trio copied to clipboard

Fix `from_thread.run(_sync)?` failing in any thread running Trio

Open gschaffner opened this issue 3 years ago • 4 comments

resolves #2191. resolves its duplicate #2534.

with this patch, trio would only prevent users from running from_thread.run(_sync)? when trio knows that such a call would deadlock. so, with this patch, trio still prevents "first-order" from_thread.run(_sync)? deadlocks like https://github.com/python-trio/trio/pull/1574/files#r435039316. i call this first-order because it's the smallest possible cyclic graph: trio_threadA is blocking waiting for trio_threadA.

there is a side-effect of this, however. the deadlock heuristic's current aggressiveness means that trio also currently prevents higher-order from_thread.run(_sync)? deadlocks, such as a second-order deadlock where the graph cycle is trio_threadA -> trio_threadB -> trio_threadA. higher-order deadlocks can be created through contrived examples:

(click to expand) a higher order `trio.from_thread.run(_sync)?` cyclic deadlock
from collections.abc import Awaitable
from collections.abc import Callable
from concurrent.futures import Future
from typing import Any
from typing import TypeVar

import trio
from trio import Nursery
from trio.lowlevel import TrioToken

T = TypeVar("T")


async def cyclic_deadlock(order: int) -> None:
    # create a cyclic deadlock where `order` threads form a cycle and each thread blocks
    # itself by calling `from_thread.run(_sync)?` to wait on the next thread in the
    # cycle.
    #
    # for reference, the `order=1` case reduces to
    # https://github.com/python-trio/trio/pull/1574/files#r435039316.
    init_token = trio.lowlevel.current_trio_token()

    if order < 1:
        raise ValueError("order must be positive")

    async def cyclic_deadlock(order: int) -> None:
        print(f"blocking thread #{order} now")
        remaining = order - 1
        if remaining:
            # in concept, this ACM block is like
            # `anyio.from_thread.start_blocking_portal`.
            async with trio.open_nursery() as nursery:
                next_token_future = Future[TrioToken]()

                async def next_thread_worker() -> None:
                    next_token_future.set_result(trio.lowlevel.current_trio_token())
                    await trio.sleep_forever()

                nursery.start_soon(to_thread_run, next_thread_worker)
                try:
                    next_token = await resolve_future(next_token_future)
                    # in concept, this call is like `BlockingPortal.call`.
                    trio.from_thread.run(
                        cyclic_deadlock, remaining, trio_token=next_token
                    )
                finally:
                    nursery.cancel_scope.cancel()
        else:
            print("entering the cyclic deadlock now!")
            trio.from_thread.run_sync(lambda: None, trio_token=init_token)

    await cyclic_deadlock(order)


async def to_thread_run(afn: Callable[..., Awaitable[T]], *args: Any) -> T:
    # like `to_thread.run_sync`, but runs an async function instead of a sync function.
    #
    # this differs from `to_thread.run_sync(trio.run, partial(afn, *args),
    # cancellable=True)` in an important way: it obeys normal cancellation semantics,
    # i.e. it doesn't unstructure the concurrency, i.e. it can't leave behind a dangling
    # thread!
    #
    # this function can be used to create a "daemonic-like" thread: a thread that exits
    # if it's still alive when the nursery that started it cancels. unlike Python's
    # normal daemonic threads, a daemonic-like thread exits when its "parent" nursery
    # exits rather than when all non-daemonic threads exit. also unlike Python's normal
    # daemonic threads, daemonic-like threads do not skip cleanup code (such as `except
    # Cancelled`, `finally`, `__exit__`, `__aexit__`, etc.) when they exit.
    #
    # ...
    #
    # this seems pretty obscure and unlikely to be needed in real code, though. if
    # you're in a trio thread and need to run a async function, you just...run the async
    # function. conceivably, i suppose, you might want to spin out a second thread
    # running trio to run an async function that is not well-behaved (i.e. it goes a
    # long time without checkpointing, interfering with other tasks on the event loop?).
    # doing so can obviously be buggy when used with interthread communication if not
    # done carefully since (a) there may not be a GIL and (b) if there is a GIL, it's
    # free to switch at places that are not checkpoints.
    nursery_future = Future[Nursery]()

    async def worker() -> T:
        async with trio.open_nursery() as nursery:
            nursery_future.set_result(nursery)
            return await afn(*args)

    try:
        return await trio.to_thread.run_sync(trio.run, worker, cancellable=True)
    finally:
        try:
            nursery = nursery_future.result(timeout=0)
        except TimeoutError:
            # `to_thread.run_sync` raised either because it failed to start `worker` or
            # because `worker` failed while opening the nursery or setting
            # `nursery_future`. thus the nursery either never opened or has already died
            # on its own, so it's okay that we can't cancel it.
            pass
        else:
            nursery.cancel_scope.cancel()


async def resolve_future(future: Future[T], /) -> T:
    # async variant of `Future.result`.
    #
    # note: this implementation requires Python >= 3.8 (as it does not support `Future`s
    # that have a result/exception set multiple times).

    # note: it probably would be better to do done-signaling with sockets instead so
    # that trio can handle the polling and we wouldn't have to hardcode a sleep interval
    # to prevent spinlocks.
    while not future.done():
        await trio.sleep(0.05)
    return future.result()


if __name__ == "__main__":
    trio.run(cyclic_deadlock, 2)

but while a higher-order deadlock can be created through a contrived example, i don't think they would ever arise naturally; they seem to require having:

  1. multiple threads running trio

  2. at least two trio threads sharing their tokens with another trio thread (in a manner that creates a graph cycle of length >= 2)

  3. a trio thread (say it has token tokenA) calling trio.from_thread.run(_sync)?(foo, trio_token=tokenB)

  4. foo being a function that will result in a trio thread calling trio.from_thread.run(_sync)?(..., trio_token=tokenA).

it seems like writing code that does all three of these things is not something that someone could actually do without realizing that they've probably written a deadlock. (1) is probably uncommon already, (2) requires pretty heavy sharing of lowlevel.TrioTokens in a way that raises red flags and screams "potential deadlock!", and (3) requires choosing to call a blocking function (that is not short-lived) in a trio thread.

it seems to me that the motivation for the current deadlock heuristic was to prevent first-order from_thread.run(_sync)? deadlocks (their implementation is only one line of code, after all), but i don't think that from_thread.run(_sync)? should be artificially restricted to prevent users from shooting themselves with a footgun that (unless i have missed something important...) appears to be rather difficult to manage to shoot oneself with.

gschaffner avatar Jan 18 '23 02:01 gschaffner

Codecov Report

Merging #2535 (eb7c901) into master (4286063) will increase coverage by 0.00%. The diff coverage is 100.00%.

Additional details and impacted files
@@           Coverage Diff           @@
##           master    #2535   +/-   ##
=======================================
  Coverage   92.44%   92.44%           
=======================================
  Files         118      118           
  Lines       16336    16341    +5     
  Branches     3157     3157           
=======================================
+ Hits        15101    15106    +5     
- Misses       1105     1121   +16     
+ Partials      130      114   -16     
Impacted Files Coverage Δ
trio/_threads.py 100.00% <100.00%> (ø)
trio/tests/test_threads.py 98.45% <100.00%> (+0.01%) :arrow_up:
trio/_core/_io_kqueue.py 0.00% <0.00%> (-0.81%) :arrow_down:
trio/_socket.py 94.50% <0.00%> (-0.40%) :arrow_down:
trio/_core/_run.py 98.49% <0.00%> (-0.01%) :arrow_down:
trio/_core/_io_windows.py 0.00% <0.00%> (ø)
trio/_dtls.py 96.54% <0.00%> (+0.38%) :arrow_up:

codecov[bot] avatar Jan 18 '23 02:01 codecov[bot]

Coverage is failing because the else case of the new condition you added is not exercised -- could you add a test that exercises it?

oremanj avatar Jan 18 '23 03:01 oremanj

.....I'm not super keen. If you want to call between Trio threads, then it would be better to have an async way to do that? Intentionally doing blocking calls from a Trio thread can work under the right circumstances, but it's Sketchy. Also, while running multiple Trio threads simultaneously is definitely supported, it's a bit odd and I wonder what you're doing, and if there's a better way to accomplish it?

njsmith avatar Jan 18 '23 05:01 njsmith

.....I'm not super keen. If you want to call between Trio threads, then it would be better to have an async way to do that?

i agree. however the use-case i'm after requires sync cross-calling; async cross-calling isn't suitable for it. see also https://github.com/python-trio/trio/issues/2534#issuecomment-1403291369

Intentionally doing blocking calls from a Trio thread can work under the right circumstances, but it's Sketchy.

i agree. blocking cross-calls are only safe when the function being cross-called exits in a very short bounded time. so, if foo() or await foo() would block for some time, one also shouldn't block by calling run_in_other_event_loop_thread(foo).

i think this is just the normal rule of "don't call a sync function that will block the event loop for a significant period of time", though.

Also, while running multiple Trio threads simultaneously is definitely supported, it's a bit odd and I wonder what you're doing, and if there's a better way to accomplish it?

see https://github.com/python-trio/trio/issues/2534#issuecomment-1403290474

gschaffner avatar Jan 25 '23 09:01 gschaffner